1
0

[HUDI-3844] Update props in indexer based on table config (#5293)

This commit is contained in:
Sagar Sumit
2022-04-12 03:46:06 +05:30
committed by GitHub
parent 458fdd5611
commit 3d8fc78c66
4 changed files with 118 additions and 20 deletions

View File

@@ -379,21 +379,24 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
}
// if metadata table exists, then check if any of the enabled partition types needs to be initialized
Set<String> inflightAndCompletedPartitions = getInflightAndCompletedMetadataPartitions(dataMetaClient.getTableConfig());
List<MetadataPartitionType> partitionsToInit = this.enabledPartitionTypes.stream()
.filter(p -> !inflightAndCompletedPartitions.contains(p.getPartitionPath()) && !MetadataPartitionType.FILES.equals(p))
.collect(Collectors.toList());
// NOTE: It needs to be guarded by async index config because if that is enabled then initialization happens through the index scheduler.
if (!dataWriteConfig.isMetadataAsyncIndex()) {
Set<String> inflightAndCompletedPartitions = getInflightAndCompletedMetadataPartitions(dataMetaClient.getTableConfig());
LOG.info("Async metadata indexing enabled and following partitions already initialized: " + inflightAndCompletedPartitions);
List<MetadataPartitionType> partitionsToInit = this.enabledPartitionTypes.stream()
.filter(p -> !inflightAndCompletedPartitions.contains(p.getPartitionPath()) && !MetadataPartitionType.FILES.equals(p))
.collect(Collectors.toList());
// if there are no partitions to initialize or there is a pending operation, then don't initialize in this round
if (partitionsToInit.isEmpty() || anyPendingDataInstant(dataMetaClient, inflightInstantTimestamp)) {
return;
}
// if there are no partitions to initialize or there is a pending operation, then don't initialize in this round
if (partitionsToInit.isEmpty() || anyPendingDataInstant(dataMetaClient, inflightInstantTimestamp)) {
return;
String createInstantTime = getInitialCommitInstantTime(dataMetaClient);
initTableMetadata(); // re-init certain flags in BaseTableMetadata
initializeEnabledFileGroups(dataMetaClient, createInstantTime, partitionsToInit);
initialCommit(createInstantTime, partitionsToInit);
updateInitializedPartitionsInTableConfig(partitionsToInit);
}
String createInstantTime = getInitialCommitInstantTime(dataMetaClient);
initTableMetadata(); // re-init certain flags in BaseTableMetadata
initializeEnabledFileGroups(dataMetaClient, createInstantTime, partitionsToInit);
initialCommit(createInstantTime, partitionsToInit);
updateInitializedPartitionsInTableConfig(partitionsToInit);
}
private <T extends SpecificRecordBase> boolean metadataTableExists(HoodieTableMetaClient dataMetaClient,
@@ -557,6 +560,8 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
List<HoodieInstant> pendingDataInstant = dataMetaClient.getActiveTimeline()
.getInstants().filter(i -> !i.isCompleted())
.filter(i -> !inflightInstantTimestamp.isPresent() || !i.getTimestamp().equals(inflightInstantTimestamp.get()))
// regular writers should not be blocked due to pending indexing action
.filter(i -> !HoodieTimeline.INDEXING_ACTION.equals(i.getAction()))
.collect(Collectors.toList());
if (!pendingDataInstant.isEmpty()) {

View File

@@ -87,7 +87,7 @@ public class ScheduleIndexActionExecutor<T extends HoodieRecordPayload, I, K, O>
Set<String> requestedPartitions = partitionIndexTypes.stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet());
requestedPartitions.removeAll(indexesInflightOrCompleted);
if (!requestedPartitions.isEmpty()) {
LOG.warn(String.format("Following partitions already exist or inflight: %s. Going to index only these partitions: %s",
LOG.warn(String.format("Following partitions already exist or inflight: %s. Going to schedule indexing of only these partitions: %s",
indexesInflightOrCompleted, requestedPartitions));
} else {
LOG.error("All requested index types are inflight or completed: " + partitionIndexTypes);