1
0

[HUDI-1479] Use HoodieEngineContext to parallelize fetching of partiton paths (#2417)

* [HUDI-1479] Use HoodieEngineContext to parallelize fetching of partition paths

* Adding testClass for FileSystemBackedTableMetadata

Co-authored-by: Nishith Agarwal <nagarwal@uber.com>
This commit is contained in:
Udit Mehrotra
2021-01-10 21:19:52 -08:00
committed by GitHub
parent 23e93d05c0
commit 7ce3ac778e
38 changed files with 509 additions and 100 deletions

View File

@@ -86,6 +86,7 @@ public class HoodieSnapshotCopier implements Serializable {
final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs.getConf(), baseDir);
final BaseFileOnlyView fsView = new HoodieTableFileSystemView(tableMetadata,
tableMetadata.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedInstants());
HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
// Get the latest commit
Option<HoodieInstant> latestCommit =
tableMetadata.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedInstants().lastInstant();
@@ -97,7 +98,7 @@ public class HoodieSnapshotCopier implements Serializable {
LOG.info(String.format("Starting to snapshot latest version files which are also no-late-than %s.",
latestCommitTimestamp));
List<String> partitions = FSUtils.getAllPartitionPaths(fs, baseDir, useFileListingFromMetadata, verifyMetadataFileListing, shouldAssumeDatePartitioning);
List<String> partitions = FSUtils.getAllPartitionPaths(context, fs, baseDir, useFileListingFromMetadata, verifyMetadataFileListing, shouldAssumeDatePartitioning);
if (partitions.size() > 0) {
LOG.info(String.format("The job needs to copy %d partitions.", partitions.size()));
@@ -108,7 +109,6 @@ public class HoodieSnapshotCopier implements Serializable {
fs.delete(new Path(outputDir), true);
}
HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
context.setJobStatus(this.getClass().getSimpleName(), "Creating a snapshot");
List<Tuple2<String, String>> filesToCopy = context.flatMap(partitions, partition -> {

View File

@@ -117,6 +117,7 @@ public class HoodieSnapshotExporter {
public void export(JavaSparkContext jsc, Config cfg) throws IOException {
FileSystem fs = FSUtils.getFs(cfg.sourceBasePath, jsc.hadoopConfiguration());
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
if (outputPathExists(fs, cfg)) {
throw new HoodieSnapshotExporterException("The target output path already exists.");
@@ -128,7 +129,7 @@ public class HoodieSnapshotExporter {
LOG.info(String.format("Starting to snapshot latest version files which are also no-late-than %s.",
latestCommitTimestamp));
final List<String> partitions = getPartitions(fs, cfg);
final List<String> partitions = getPartitions(engineContext, fs, cfg);
if (partitions.isEmpty()) {
throw new HoodieSnapshotExporterException("The source dataset has 0 partition to snapshot.");
}
@@ -153,8 +154,8 @@ public class HoodieSnapshotExporter {
return latestCommit.isPresent() ? Option.of(latestCommit.get().getTimestamp()) : Option.empty();
}
private List<String> getPartitions(FileSystem fs, Config cfg) throws IOException {
return FSUtils.getAllPartitionPaths(fs, cfg.sourceBasePath, true, false, false);
private List<String> getPartitions(HoodieEngineContext engineContext, FileSystem fs, Config cfg) throws IOException {
return FSUtils.getAllPartitionPaths(engineContext, fs, cfg.sourceBasePath, true, false, false);
}
private void createSuccessTag(FileSystem fs, Config cfg) throws IOException {

View File

@@ -85,13 +85,14 @@ public class TimelineServerPerf implements Serializable {
}
public void run() throws IOException {
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(timelineServer.getFs(), cfg.basePath, cfg.useFileListingFromMetadata,
cfg.verifyMetadataFileListing, true);
JavaSparkContext jsc = UtilHelpers.buildSparkContext("hudi-view-perf-" + cfg.basePath, cfg.sparkMaster);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(engineContext, timelineServer.getFs(), cfg.basePath,
cfg.useFileListingFromMetadata, cfg.verifyMetadataFileListing, true);
Collections.shuffle(allPartitionPaths);
List<String> selected = allPartitionPaths.stream().filter(p -> !p.contains("error")).limit(cfg.maxPartitions)
.collect(Collectors.toList());
JavaSparkContext jsc = UtilHelpers.buildSparkContext("hudi-view-perf-" + cfg.basePath, cfg.sparkMaster);
if (!useExternalTimelineServer) {
this.timelineServer.startService();
setHostAddrFromSparkConf(jsc.getConf());