[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:
@@ -119,6 +119,10 @@ public abstract class AbstractHoodieClient implements Serializable, AutoCloseabl
|
||||
return config;
|
||||
}
|
||||
|
||||
public HoodieEngineContext getEngineContext() {
|
||||
return context;
|
||||
}
|
||||
|
||||
protected void initWrapperFSMetrics() {
|
||||
// no-op.
|
||||
}
|
||||
|
||||
@@ -221,8 +221,9 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
||||
protected abstract void initialize(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient);
|
||||
|
||||
private void initTableMetadata() {
|
||||
this.metadata = new HoodieBackedTableMetadata(hadoopConf.get(), datasetWriteConfig.getBasePath(), datasetWriteConfig.getSpillableMapBasePath(),
|
||||
datasetWriteConfig.useFileListingMetadata(), datasetWriteConfig.getFileListingMetadataVerify(), false,
|
||||
this.metadata = new HoodieBackedTableMetadata(engineContext, datasetWriteConfig.getBasePath(),
|
||||
datasetWriteConfig.getSpillableMapBasePath(), datasetWriteConfig.useFileListingMetadata(),
|
||||
datasetWriteConfig.getFileListingMetadataVerify(), false,
|
||||
datasetWriteConfig.shouldAssumeDatePartitioning());
|
||||
this.metaClient = metadata.getMetaClient();
|
||||
}
|
||||
|
||||
@@ -31,6 +31,7 @@ import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieSavepointMetadata;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.engine.HoodieLocalEngineContext;
|
||||
import org.apache.hudi.common.engine.TaskContextSupplier;
|
||||
import org.apache.hudi.common.fs.ConsistencyGuard;
|
||||
import org.apache.hudi.common.fs.ConsistencyGuard.FileVisibility;
|
||||
@@ -93,6 +94,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
|
||||
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final HoodieTableMetaClient metaClient;
|
||||
protected final transient HoodieEngineContext context;
|
||||
protected final HoodieIndex<T, I, K, O> index;
|
||||
|
||||
private SerializableConfiguration hadoopConfiguration;
|
||||
@@ -108,6 +110,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
|
||||
config.getViewStorageConfig());
|
||||
this.metaClient = metaClient;
|
||||
this.index = getIndex(config, context);
|
||||
this.context = context;
|
||||
this.taskContextSupplier = context.getTaskContextSupplier();
|
||||
}
|
||||
|
||||
@@ -660,8 +663,16 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
|
||||
|
||||
public HoodieTableMetadata metadata() {
|
||||
if (metadata == null) {
|
||||
metadata = HoodieTableMetadata.create(hadoopConfiguration.get(), config.getBasePath(), config.getSpillableMapBasePath(),
|
||||
config.useFileListingMetadata(), config.getFileListingMetadataVerify(), config.isMetricsOn(), config.shouldAssumeDatePartitioning());
|
||||
HoodieEngineContext engineContext = context;
|
||||
if (engineContext == null) {
|
||||
// This is to handle scenarios where this is called at the executor tasks which do not have access
|
||||
// to engine context, and it ends up being null (as its not serializable and marked transient here).
|
||||
engineContext = new HoodieLocalEngineContext(hadoopConfiguration.get());
|
||||
}
|
||||
|
||||
metadata = HoodieTableMetadata.create(engineContext, config.getBasePath(), config.getSpillableMapBasePath(),
|
||||
config.useFileListingMetadata(), config.getFileListingMetadataVerify(), config.isMetricsOn(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
}
|
||||
return metadata;
|
||||
}
|
||||
|
||||
@@ -67,7 +67,7 @@ public abstract class PartitionAwareClusteringPlanStrategy<T extends HoodieRecor
|
||||
HoodieTableMetaClient metaClient = getHoodieTable().getMetaClient();
|
||||
LOG.info("Scheduling clustering for " + metaClient.getBasePath());
|
||||
HoodieWriteConfig config = getWriteConfig();
|
||||
List<String> partitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
|
||||
List<String> partitionPaths = FSUtils.getAllPartitionPaths(getEngineContext(), metaClient.getFs(), metaClient.getBasePath(),
|
||||
config.useFileListingMetadata(), config.getFileListingMetadataVerify(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
|
||||
|
||||
@@ -91,9 +91,10 @@ public class RollbackUtils {
|
||||
* @param config instance of {@link HoodieWriteConfig} to use.
|
||||
* @return {@link List} of {@link ListingBasedRollbackRequest}s thus collected.
|
||||
*/
|
||||
public static List<ListingBasedRollbackRequest> generateRollbackRequestsByListingCOW(FileSystem fs, String basePath, HoodieWriteConfig config) {
|
||||
public static List<ListingBasedRollbackRequest> generateRollbackRequestsByListingCOW(HoodieEngineContext engineContext,
|
||||
FileSystem fs, String basePath, HoodieWriteConfig config) {
|
||||
try {
|
||||
return FSUtils.getAllPartitionPaths(fs, basePath, config.useFileListingMetadata(),
|
||||
return FSUtils.getAllPartitionPaths(engineContext, fs, basePath, config.useFileListingMetadata(),
|
||||
config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning()).stream()
|
||||
.map(ListingBasedRollbackRequest::createRollbackRequestWithDeleteDataAndLogFilesAction)
|
||||
.collect(Collectors.toList());
|
||||
@@ -113,7 +114,7 @@ public class RollbackUtils {
|
||||
public static List<ListingBasedRollbackRequest> generateRollbackRequestsUsingFileListingMOR(HoodieInstant instantToRollback, HoodieTable table, HoodieEngineContext context) throws IOException {
|
||||
String commit = instantToRollback.getTimestamp();
|
||||
HoodieWriteConfig config = table.getConfig();
|
||||
List<String> partitions = FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(),
|
||||
List<String> partitions = FSUtils.getAllPartitionPaths(context, table.getMetaClient().getFs(), table.getMetaClient().getBasePath(),
|
||||
config.useFileListingMetadata(), config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning());
|
||||
int sparkPartitions = Math.max(Math.min(partitions.size(), config.getRollbackParallelism()), 1);
|
||||
context.setJobStatus(RollbackUtils.class.getSimpleName(), "Generate all rollback requests");
|
||||
|
||||
@@ -90,7 +90,7 @@ public class SavepointActionExecutor<T extends HoodieRecordPayload, I, K, O> ext
|
||||
"Could not savepoint commit " + instantTime + " as this is beyond the lookup window " + lastCommitRetained);
|
||||
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Collecting latest files for savepoint " + instantTime);
|
||||
List<String> partitions = FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(),
|
||||
List<String> partitions = FSUtils.getAllPartitionPaths(context, table.getMetaClient().getFs(),
|
||||
table.getMetaClient().getBasePath(), config.useFileListingMetadata(), config.getFileListingMetadataVerify(),
|
||||
config.shouldAssumeDatePartitioning()
|
||||
);
|
||||
|
||||
Reference in New Issue
Block a user