-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-1312] [RFC-15] Support for metadata listing for snapshot queries through Hive/SparkSQL #2366
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,6 +27,7 @@ | |
| import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; | ||
| import org.apache.hudi.common.table.timeline.HoodieInstant; | ||
| import org.apache.hudi.common.table.timeline.HoodieTimeline; | ||
| import org.apache.hudi.common.table.view.FileSystemViewManager; | ||
| import org.apache.hudi.common.table.view.HoodieTableFileSystemView; | ||
| import org.apache.hudi.common.table.view.TableFileSystemView; | ||
| import org.apache.hudi.common.util.Option; | ||
|
|
@@ -62,6 +63,11 @@ | |
| import java.util.function.Function; | ||
| import java.util.stream.Collectors; | ||
|
|
||
| import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP; | ||
| import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; | ||
| import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP; | ||
| import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE; | ||
|
|
||
| public class HoodieInputFormatUtils { | ||
|
|
||
| // These positions have to be deterministic across all tables | ||
|
|
@@ -391,27 +397,48 @@ public static Map<HoodieTableMetaClient, List<FileStatus>> groupFileStatusForSna | |
| return grouped; | ||
| } | ||
|
|
||
| public static Map<HoodieTableMetaClient, List<Path>> groupSnapshotPathsByMetaClient( | ||
| Collection<HoodieTableMetaClient> metaClientList, | ||
| List<Path> snapshotPaths | ||
| ) { | ||
| Map<HoodieTableMetaClient, List<Path>> grouped = new HashMap<>(); | ||
| metaClientList.forEach(metaClient -> grouped.put(metaClient, new ArrayList<>())); | ||
| for (Path path : snapshotPaths) { | ||
| // Find meta client associated with the input path | ||
| metaClientList.stream().filter(metaClient -> path.toString().contains(metaClient.getBasePath())) | ||
| .forEach(metaClient -> grouped.get(metaClient).add(path)); | ||
| } | ||
| return grouped; | ||
| } | ||
|
|
||
| /** | ||
| * Filters data files for a snapshot queried table. | ||
| * Filters data files under @param paths for a snapshot queried table. | ||
| * @param job | ||
| * @param metadata | ||
| * @param fileStatuses | ||
| * @param metaClient | ||
| * @param paths | ||
| * @return | ||
| */ | ||
| public static List<FileStatus> filterFileStatusForSnapshotMode( | ||
| JobConf job, HoodieTableMetaClient metadata, List<FileStatus> fileStatuses) throws IOException { | ||
| FileStatus[] statuses = fileStatuses.toArray(new FileStatus[0]); | ||
| JobConf job, HoodieTableMetaClient metaClient, List<Path> paths) throws IOException { | ||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug("Hoodie Metadata initialized with completed commit Ts as :" + metadata); | ||
| LOG.debug("Hoodie Metadata initialized with completed commit Ts as :" + metaClient); | ||
| } | ||
| // Get all commits, delta commits, compactions, as all of them produce a base parquet file today | ||
| HoodieTimeline timeline = metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); | ||
| TableFileSystemView.BaseFileOnlyView roView = new HoodieTableFileSystemView(metadata, timeline, statuses); | ||
| // filter files on the latest commit found | ||
| List<HoodieBaseFile> filteredFiles = roView.getLatestBaseFiles().collect(Collectors.toList()); | ||
| LOG.info("Total paths to process after hoodie filter " + filteredFiles.size()); | ||
|
|
||
| boolean useFileListingFromMetadata = job.getBoolean(METADATA_ENABLE_PROP, DEFAULT_METADATA_ENABLE_FOR_READERS); | ||
| boolean verifyFileListing = job.getBoolean(METADATA_VALIDATE_PROP, DEFAULT_METADATA_VALIDATE); | ||
| HoodieTableFileSystemView fsView = FileSystemViewManager.createInMemoryFileSystemView(metaClient, | ||
| useFileListingFromMetadata, verifyFileListing); | ||
|
|
||
| List<HoodieBaseFile> filteredBaseFiles = new ArrayList<>(); | ||
| for (Path p : paths) { | ||
| String relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), p); | ||
| List<HoodieBaseFile> matched = fsView.getLatestBaseFiles(relativePartitionPath).collect(Collectors.toList()); | ||
| filteredBaseFiles.addAll(matched); | ||
| } | ||
|
|
||
| LOG.info("Total paths to process after hoodie filter " + filteredBaseFiles.size()); | ||
| List<FileStatus> returns = new ArrayList<>(); | ||
| for (HoodieBaseFile filteredFile : filteredFiles) { | ||
| for (HoodieBaseFile filteredFile : filteredBaseFiles) { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. hmmm. slightly orthogonal, but the |
||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug("Processing latest hoodie file - " + filteredFile.getPath()); | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,6 +26,7 @@ | |
| import org.apache.hudi.common.table.HoodieTableMetaClient; | ||
| import org.apache.hudi.common.table.timeline.HoodieInstant; | ||
| import org.apache.hudi.common.table.timeline.HoodieTimeline; | ||
| import org.apache.hudi.common.table.view.FileSystemViewManager; | ||
| import org.apache.hudi.common.table.view.HoodieTableFileSystemView; | ||
| import org.apache.hudi.common.util.CollectionUtils; | ||
| import org.apache.hudi.common.util.Option; | ||
|
|
@@ -53,6 +54,11 @@ | |
| import java.util.stream.Collectors; | ||
| import java.util.stream.Stream; | ||
|
|
||
| import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP; | ||
| import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; | ||
| import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP; | ||
| import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE; | ||
|
|
||
| public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils { | ||
|
|
||
| private static final Logger LOG = LogManager.getLogger(HoodieRealtimeInputFormatUtils.class); | ||
|
|
@@ -63,13 +69,25 @@ public static InputSplit[] getRealtimeSplits(Configuration conf, Stream<FileSpli | |
| // TODO(vc): Should we handle also non-hoodie splits here? | ||
| Map<Path, HoodieTableMetaClient> partitionsToMetaClient = getTableMetaClientByBasePath(conf, partitionsToParquetSplits.keySet()); | ||
|
|
||
| boolean useFileListingFromMetadata = conf.getBoolean(METADATA_ENABLE_PROP, DEFAULT_METADATA_ENABLE_FOR_READERS); | ||
| boolean verifyFileListing = conf.getBoolean(METADATA_VALIDATE_PROP, DEFAULT_METADATA_VALIDATE); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. These 2 config options is every where, can we just pass the |
||
| // Create file system cache so metadata table is only instantiated once. Also can benefit normal file listing if | ||
| // partition path is listed twice so file groups will already be loaded in file system | ||
| Map<HoodieTableMetaClient, HoodieTableFileSystemView> fsCache = new HashMap<>(); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. makes sense |
||
| // for all unique split parents, obtain all delta files based on delta commit timeline, | ||
| // grouped on file id | ||
| List<InputSplit> rtSplits = new ArrayList<>(); | ||
| partitionsToParquetSplits.keySet().forEach(partitionPath -> { | ||
| // for each partition path obtain the data & log file groupings, then map back to inputsplits | ||
| HoodieTableMetaClient metaClient = partitionsToMetaClient.get(partitionPath); | ||
| HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline()); | ||
| if (!fsCache.containsKey(metaClient)) { | ||
|
|
||
| HoodieTableFileSystemView fsView = FileSystemViewManager.createInMemoryFileSystemView(metaClient, | ||
| useFileListingFromMetadata, verifyFileListing); | ||
| fsCache.put(metaClient, fsView); | ||
| } | ||
| HoodieTableFileSystemView fsView = fsCache.get(metaClient); | ||
|
|
||
| String relPartitionPath = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), partitionPath); | ||
|
|
||
| try { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
note to self: doing this by path is ok, since the FileSystemView internally caches per partition.