1
0

[HUDI-1450] Use metadata table for listing in HoodieROTablePathFilter (apache#2326)

[HUDI-1394] [RFC-15] Use metadata table (if present) to get all partition paths (apache#2351)
This commit is contained in:
Udit Mehrotra
2020-12-31 01:20:02 -08:00
committed by vinoth chandar
parent 298808baaf
commit 4e64226844
38 changed files with 308 additions and 102 deletions

View File

@@ -64,7 +64,7 @@ public class SparkHoodieGlobalBloomIndex<T extends HoodieRecordPayload> extends
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
try {
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
config.shouldAssumeDatePartitioning());
config.useFileListingMetadata(), config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning());
return super.loadInvolvedFiles(allPartitionPaths, context, hoodieTable);
} catch (IOException e) {
throw new HoodieIOException("Failed to load all partitions", e);

View File

@@ -104,7 +104,8 @@ public class SparkHoodieGlobalSimpleIndex<T extends HoodieRecordPayload> extends
final HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
try {
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), config.shouldAssumeDatePartitioning());
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
config.useFileListingMetadata(), config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning());
// Obtain the latest data files from all the partitions.
return getLatestBaseFilesForAllPartitions(allPartitionPaths, context, hoodieTable);
} catch (IOException e) {

View File

@@ -51,7 +51,8 @@ public class SparkInsertOverwriteTableCommitActionExecutor<T extends HoodieRecor
Map<String, List<String>> partitionToExistingFileIds = new HashMap<>();
try {
List<String> partitionPaths = FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(),
table.getMetaClient().getBasePath(), false);
table.getMetaClient().getBasePath(), config.useFileListingMetadata(), config.getFileListingMetadataVerify(),
false);
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
if (partitionPaths != null && partitionPaths.size() > 0) {
context.setJobStatus(this.getClass().getSimpleName(), "Getting ExistingFileIds of all partitions");

View File

@@ -196,7 +196,7 @@ public class HoodieSparkMergeOnReadTableCompactor<T extends HoodieRecordPayload>
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
LOG.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime);
List<String> partitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
config.shouldAssumeDatePartitioning());
config.useFileListingMetadata(), config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning());
// filter the partition paths if needed to reduce list status
partitionPaths = config.getCompactionStrategy().filterPartitionPaths(config, partitionPaths);

View File

@@ -66,8 +66,8 @@ public class SparkCopyOnWriteRollbackActionExecutor<T extends HoodieRecordPayloa
@Override
protected List<HoodieRollbackStat> executeRollbackUsingFileListing(HoodieInstant instantToRollback) {
List<ListingBasedRollbackRequest> rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(),
config.shouldAssumeDatePartitioning());
List<ListingBasedRollbackRequest> rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(),
table.getMetaClient().getBasePath(), config);
return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(context, instantToRollback, rollbackRequests);
}
}

View File

@@ -93,7 +93,7 @@ public class ZeroToOneUpgradeHandler implements UpgradeHandler {
List<ListingBasedRollbackRequest> rollbackRequests;
if (table.getMetaClient().getTableType() == HoodieTableType.COPY_ON_WRITE) {
rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(),
table.getConfig().shouldAssumeDatePartitioning());
table.getConfig());
} else {
rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(commitInstantOpt.get(), table, context);
}