[HUDI-3836] Improve the way of fetching metadata partitions from table (#5286)
Co-authored-by: xicm <xicm@asiainfo.com>
This commit is contained in:
@@ -53,7 +53,6 @@ import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty;
|
||||
import static org.apache.hudi.common.util.ValidationUtils.checkArgument;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_BLOOM_FILTERS;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getCompletedMetadataPartitions;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getInflightAndCompletedMetadataPartitions;
|
||||
import static org.apache.hudi.utilities.UtilHelpers.EXECUTE;
|
||||
import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE;
|
||||
@@ -243,7 +242,7 @@ public class HoodieIndexer {
|
||||
}
|
||||
|
||||
private boolean indexExists(List<MetadataPartitionType> partitionTypes) {
|
||||
Set<String> indexedMetadataPartitions = getCompletedMetadataPartitions(metaClient.getTableConfig());
|
||||
Set<String> indexedMetadataPartitions = metaClient.getTableConfig().getMetadataPartitions();
|
||||
Set<String> requestedIndexPartitionPaths = partitionTypes.stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet());
|
||||
requestedIndexPartitionPaths.retainAll(indexedMetadataPartitions);
|
||||
if (!requestedIndexPartitionPaths.isEmpty()) {
|
||||
@@ -254,7 +253,7 @@ public class HoodieIndexer {
|
||||
}
|
||||
|
||||
private boolean isMetadataInitialized() {
|
||||
Set<String> indexedMetadataPartitions = getCompletedMetadataPartitions(metaClient.getTableConfig());
|
||||
Set<String> indexedMetadataPartitions = metaClient.getTableConfig().getMetadataPartitions();
|
||||
return !indexedMetadataPartitions.isEmpty();
|
||||
}
|
||||
|
||||
|
||||
@@ -56,7 +56,6 @@ import java.util.Set;
|
||||
|
||||
import static org.apache.hudi.common.table.HoodieTableMetaClient.reload;
|
||||
import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getCompletedMetadataPartitions;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.metadataPartitionExists;
|
||||
import static org.apache.hudi.metadata.MetadataPartitionType.BLOOM_FILTERS;
|
||||
import static org.apache.hudi.metadata.MetadataPartitionType.COLUMN_STATS;
|
||||
@@ -142,8 +141,8 @@ public class TestHoodieIndexer extends HoodieCommonTestHarness implements SparkP
|
||||
initializeWriteClient(metadataConfigBuilder.build());
|
||||
|
||||
// validate table config
|
||||
assertTrue(getCompletedMetadataPartitions(reload(metaClient).getTableConfig()).contains(FILES.getPartitionPath()));
|
||||
assertTrue(getCompletedMetadataPartitions(reload(metaClient).getTableConfig()).contains(BLOOM_FILTERS.getPartitionPath()));
|
||||
assertTrue(reload(metaClient).getTableConfig().getMetadataPartitions().contains(FILES.getPartitionPath()));
|
||||
assertTrue(reload(metaClient).getTableConfig().getMetadataPartitions().contains(BLOOM_FILTERS.getPartitionPath()));
|
||||
|
||||
// build indexer config which has only column_stats enabled (files and bloom filter is already enabled)
|
||||
indexMetadataPartitionsAndAssert(COLUMN_STATS, Arrays.asList(new MetadataPartitionType[]{FILES, BLOOM_FILTERS}), Collections.emptyList());
|
||||
@@ -158,7 +157,7 @@ public class TestHoodieIndexer extends HoodieCommonTestHarness implements SparkP
|
||||
initializeWriteClient(metadataConfigBuilder.build());
|
||||
|
||||
// validate table config
|
||||
assertFalse(getCompletedMetadataPartitions(reload(metaClient).getTableConfig()).contains(FILES.getPartitionPath()));
|
||||
assertFalse(reload(metaClient).getTableConfig().getMetadataPartitions().contains(FILES.getPartitionPath()));
|
||||
|
||||
// build indexer config which has only files enabled
|
||||
indexMetadataPartitionsAndAssert(FILES, Collections.emptyList(), Arrays.asList(new MetadataPartitionType[]{COLUMN_STATS, BLOOM_FILTERS}));
|
||||
@@ -176,7 +175,7 @@ public class TestHoodieIndexer extends HoodieCommonTestHarness implements SparkP
|
||||
HoodieMetadataConfig.Builder metadataConfigBuilder = getMetadataConfigBuilder(false, false);
|
||||
initializeWriteClient(metadataConfigBuilder.build());
|
||||
// validate table config
|
||||
assertFalse(getCompletedMetadataPartitions(reload(metaClient).getTableConfig()).contains(FILES.getPartitionPath()));
|
||||
assertFalse(reload(metaClient).getTableConfig().getMetadataPartitions().contains(FILES.getPartitionPath()));
|
||||
|
||||
// build indexer config which has only column stats enabled. expected to throw exception.
|
||||
HoodieIndexer.Config config = new HoodieIndexer.Config();
|
||||
@@ -192,9 +191,9 @@ public class TestHoodieIndexer extends HoodieCommonTestHarness implements SparkP
|
||||
|
||||
// validate table config
|
||||
metaClient = reload(metaClient);
|
||||
assertFalse(getCompletedMetadataPartitions(metaClient.getTableConfig()).contains(FILES.getPartitionPath()));
|
||||
assertFalse(getCompletedMetadataPartitions(metaClient.getTableConfig()).contains(COLUMN_STATS.getPartitionPath()));
|
||||
assertFalse(getCompletedMetadataPartitions(metaClient.getTableConfig()).contains(BLOOM_FILTERS.getPartitionPath()));
|
||||
assertFalse(metaClient.getTableConfig().getMetadataPartitions().contains(FILES.getPartitionPath()));
|
||||
assertFalse(metaClient.getTableConfig().getMetadataPartitions().contains(COLUMN_STATS.getPartitionPath()));
|
||||
assertFalse(metaClient.getTableConfig().getMetadataPartitions().contains(BLOOM_FILTERS.getPartitionPath()));
|
||||
// validate metadata partitions actually exist
|
||||
assertFalse(metadataPartitionExists(basePath, context, FILES));
|
||||
|
||||
@@ -229,7 +228,7 @@ public class TestHoodieIndexer extends HoodieCommonTestHarness implements SparkP
|
||||
|
||||
// validate table config
|
||||
metaClient = reload(metaClient);
|
||||
Set<String> completedPartitions = getCompletedMetadataPartitions(metaClient.getTableConfig());
|
||||
Set<String> completedPartitions = metaClient.getTableConfig().getMetadataPartitions();
|
||||
assertTrue(completedPartitions.contains(partitionTypeToIndex.getPartitionPath()));
|
||||
alreadyCompletedPartitions.forEach(entry -> assertTrue(completedPartitions.contains(entry.getPartitionPath())));
|
||||
nonExistantPartitions.forEach(entry -> assertFalse(completedPartitions.contains(entry.getPartitionPath())));
|
||||
@@ -257,9 +256,9 @@ public class TestHoodieIndexer extends HoodieCommonTestHarness implements SparkP
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// validate partitions built successfully
|
||||
assertTrue(getCompletedMetadataPartitions(reload(metaClient).getTableConfig()).contains(FILES.getPartitionPath()));
|
||||
assertTrue(reload(metaClient).getTableConfig().getMetadataPartitions().contains(FILES.getPartitionPath()));
|
||||
assertTrue(metadataPartitionExists(basePath, context, FILES));
|
||||
assertTrue(getCompletedMetadataPartitions(reload(metaClient).getTableConfig()).contains(BLOOM_FILTERS.getPartitionPath()));
|
||||
assertTrue(reload(metaClient).getTableConfig().getMetadataPartitions().contains(BLOOM_FILTERS.getPartitionPath()));
|
||||
assertTrue(metadataPartitionExists(basePath, context, BLOOM_FILTERS));
|
||||
|
||||
// build indexer config which has only column_stats enabled (files is enabled by default)
|
||||
@@ -288,9 +287,9 @@ public class TestHoodieIndexer extends HoodieCommonTestHarness implements SparkP
|
||||
assertFalse(metadataPartitionExists(basePath, context, COLUMN_STATS));
|
||||
|
||||
// check other partitions are intact
|
||||
assertTrue(getCompletedMetadataPartitions(reload(metaClient).getTableConfig()).contains(FILES.getPartitionPath()));
|
||||
assertTrue(reload(metaClient).getTableConfig().getMetadataPartitions().contains(FILES.getPartitionPath()));
|
||||
assertTrue(metadataPartitionExists(basePath, context, FILES));
|
||||
assertTrue(getCompletedMetadataPartitions(reload(metaClient).getTableConfig()).contains(BLOOM_FILTERS.getPartitionPath()));
|
||||
assertTrue(reload(metaClient).getTableConfig().getMetadataPartitions().contains(BLOOM_FILTERS.getPartitionPath()));
|
||||
assertTrue(metadataPartitionExists(basePath, context, BLOOM_FILTERS));
|
||||
}
|
||||
|
||||
@@ -312,7 +311,7 @@ public class TestHoodieIndexer extends HoodieCommonTestHarness implements SparkP
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// validate files partition built successfully
|
||||
assertTrue(getCompletedMetadataPartitions(reload(metaClient).getTableConfig()).contains(FILES.getPartitionPath()));
|
||||
assertTrue(reload(metaClient).getTableConfig().getMetadataPartitions().contains(FILES.getPartitionPath()));
|
||||
assertTrue(metadataPartitionExists(basePath, context, FILES));
|
||||
|
||||
// build indexer config which has only bloom_filters enabled
|
||||
@@ -320,7 +319,7 @@ public class TestHoodieIndexer extends HoodieCommonTestHarness implements SparkP
|
||||
// start the indexer and validate bloom_filters index is also complete
|
||||
HoodieIndexer indexer = new HoodieIndexer(jsc, config);
|
||||
assertEquals(0, indexer.start(0));
|
||||
assertTrue(getCompletedMetadataPartitions(reload(metaClient).getTableConfig()).contains(BLOOM_FILTERS.getPartitionPath()));
|
||||
assertTrue(reload(metaClient).getTableConfig().getMetadataPartitions().contains(BLOOM_FILTERS.getPartitionPath()));
|
||||
assertTrue(metadataPartitionExists(basePath, context, BLOOM_FILTERS));
|
||||
|
||||
// completed index timeline for later validation
|
||||
@@ -344,9 +343,9 @@ public class TestHoodieIndexer extends HoodieCommonTestHarness implements SparkP
|
||||
dropIndexAndAssert(COLUMN_STATS, "delta-streamer-config/indexer.properties", Option.empty());
|
||||
|
||||
// check other partitions are intact
|
||||
assertTrue(getCompletedMetadataPartitions(reload(metaClient).getTableConfig()).contains(FILES.getPartitionPath()));
|
||||
assertTrue(reload(metaClient).getTableConfig().getMetadataPartitions().contains(FILES.getPartitionPath()));
|
||||
assertTrue(metadataPartitionExists(basePath, context, FILES));
|
||||
assertTrue(getCompletedMetadataPartitions(reload(metaClient).getTableConfig()).contains(BLOOM_FILTERS.getPartitionPath()));
|
||||
assertTrue(reload(metaClient).getTableConfig().getMetadataPartitions().contains(BLOOM_FILTERS.getPartitionPath()));
|
||||
assertTrue(metadataPartitionExists(basePath, context, BLOOM_FILTERS));
|
||||
|
||||
// drop bloom filter partition. timeline files should not be deleted since the index building is complete.
|
||||
|
||||
Reference in New Issue
Block a user