[HUDI-3782] Fixing table config when any of the index is disabled (#5222)
This commit is contained in:
@@ -146,6 +146,10 @@ import static org.apache.hudi.common.model.WriteOperationType.DELETE;
|
||||
import static org.apache.hudi.common.model.WriteOperationType.INSERT;
|
||||
import static org.apache.hudi.common.model.WriteOperationType.UPSERT;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getCompletedMetadataPartitions;
|
||||
import static org.apache.hudi.metadata.MetadataPartitionType.BLOOM_FILTERS;
|
||||
import static org.apache.hudi.metadata.MetadataPartitionType.COLUMN_STATS;
|
||||
import static org.apache.hudi.metadata.MetadataPartitionType.FILES;
|
||||
import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
@@ -202,6 +206,119 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
||||
validateMetadata(testTable, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTurnOffMetadataIndexAfterEnable() throws Exception {
|
||||
initPath();
|
||||
HoodieWriteConfig cfg = getConfigBuilder(TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.BLOOM, HoodieFailedWritesCleaningPolicy.EAGER)
|
||||
.withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1)
|
||||
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
|
||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build())
|
||||
.build();
|
||||
init(COPY_ON_WRITE);
|
||||
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
|
||||
// metadata enabled with only FILES partition
|
||||
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, cfg)) {
|
||||
// Insert
|
||||
String commitTime = "0000001";
|
||||
List<HoodieRecord> records = dataGen.generateInserts(commitTime, 20);
|
||||
client.startCommitWithTime(commitTime);
|
||||
List<WriteStatus> writeStatuses = client.insert(jsc.parallelize(records, 1), commitTime).collect();
|
||||
assertNoWriteErrors(writeStatuses);
|
||||
|
||||
// Upsert
|
||||
commitTime = "0000002";
|
||||
client.startCommitWithTime(commitTime);
|
||||
records = dataGen.generateUniqueUpdates(commitTime, 10);
|
||||
writeStatuses = client.upsert(jsc.parallelize(records, 1), commitTime).collect();
|
||||
assertNoWriteErrors(writeStatuses);
|
||||
validateMetadata(client);
|
||||
}
|
||||
// check table config
|
||||
HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTableConfig tableConfig = metaClient.getTableConfig();
|
||||
assertFalse(tableConfig.getMetadataPartitions().isEmpty());
|
||||
assertTrue(getCompletedMetadataPartitions(tableConfig).contains(FILES.getPartitionPath()));
|
||||
assertFalse(getCompletedMetadataPartitions(tableConfig).contains(COLUMN_STATS.getPartitionPath()));
|
||||
assertFalse(getCompletedMetadataPartitions(tableConfig).contains(BLOOM_FILTERS.getPartitionPath()));
|
||||
|
||||
// enable column stats and run 1 upserts
|
||||
HoodieWriteConfig cfgWithColStatsEnabled = HoodieWriteConfig.newBuilder()
|
||||
.withProperties(cfg.getProps())
|
||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder()
|
||||
.withProperties(cfg.getMetadataConfig().getProps())
|
||||
.withMetadataIndexColumnStats(true)
|
||||
.build())
|
||||
.build();
|
||||
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, cfgWithColStatsEnabled)) {
|
||||
// Upsert
|
||||
String commitTime = "0000003";
|
||||
client.startCommitWithTime(commitTime);
|
||||
List<HoodieRecord> records = dataGen.generateUniqueUpdates(commitTime, 10);
|
||||
List<WriteStatus> writeStatuses = client.upsert(jsc.parallelize(records, 1), commitTime).collect();
|
||||
assertNoWriteErrors(writeStatuses);
|
||||
validateMetadata(client);
|
||||
}
|
||||
// check table config
|
||||
HoodieTableMetaClient.reload(metaClient);
|
||||
tableConfig = metaClient.getTableConfig();
|
||||
assertFalse(tableConfig.getMetadataPartitions().isEmpty());
|
||||
assertTrue(getCompletedMetadataPartitions(tableConfig).contains(FILES.getPartitionPath()));
|
||||
assertTrue(getCompletedMetadataPartitions(tableConfig).contains(COLUMN_STATS.getPartitionPath()));
|
||||
assertFalse(getCompletedMetadataPartitions(tableConfig).contains(BLOOM_FILTERS.getPartitionPath()));
|
||||
|
||||
// disable column stats and run 1 upsert
|
||||
HoodieWriteConfig cfgWithColStatsDisabled = HoodieWriteConfig.newBuilder()
|
||||
.withProperties(cfg.getProps())
|
||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder()
|
||||
.withProperties(cfg.getMetadataConfig().getProps())
|
||||
.withMetadataIndexColumnStats(false)
|
||||
.build())
|
||||
.build();
|
||||
|
||||
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, cfgWithColStatsDisabled)) {
|
||||
// Upsert
|
||||
String commitTime = "0000004";
|
||||
client.startCommitWithTime(commitTime);
|
||||
List<HoodieRecord> records = dataGen.generateUniqueUpdates(commitTime, 10);
|
||||
List<WriteStatus> writeStatuses = client.upsert(jsc.parallelize(records, 1), commitTime).collect();
|
||||
assertNoWriteErrors(writeStatuses);
|
||||
validateMetadata(client);
|
||||
}
|
||||
// check table config
|
||||
HoodieTableMetaClient.reload(metaClient);
|
||||
tableConfig = metaClient.getTableConfig();
|
||||
assertFalse(tableConfig.getMetadataPartitions().isEmpty());
|
||||
assertTrue(getCompletedMetadataPartitions(tableConfig).contains(FILES.getPartitionPath()));
|
||||
assertFalse(getCompletedMetadataPartitions(tableConfig).contains(COLUMN_STATS.getPartitionPath()));
|
||||
assertFalse(getCompletedMetadataPartitions(tableConfig).contains(BLOOM_FILTERS.getPartitionPath()));
|
||||
|
||||
// enable bloom filter as well as column stats and run 1 upsert
|
||||
HoodieWriteConfig cfgWithBloomFilterEnabled = HoodieWriteConfig.newBuilder()
|
||||
.withProperties(cfgWithColStatsEnabled.getProps())
|
||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder()
|
||||
.withProperties(cfgWithColStatsEnabled.getMetadataConfig().getProps())
|
||||
.withMetadataIndexBloomFilter(true)
|
||||
.build())
|
||||
.build();
|
||||
|
||||
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, cfgWithBloomFilterEnabled)) {
|
||||
// Upsert
|
||||
String commitTime = "0000005";
|
||||
client.startCommitWithTime(commitTime);
|
||||
List<HoodieRecord> records = dataGen.generateUniqueUpdates(commitTime, 10);
|
||||
List<WriteStatus> writeStatuses = client.upsert(jsc.parallelize(records, 1), commitTime).collect();
|
||||
assertNoWriteErrors(writeStatuses);
|
||||
validateMetadata(client);
|
||||
}
|
||||
// check table config
|
||||
HoodieTableMetaClient.reload(metaClient);
|
||||
tableConfig = metaClient.getTableConfig();
|
||||
assertFalse(tableConfig.getMetadataPartitions().isEmpty());
|
||||
assertTrue(getCompletedMetadataPartitions(tableConfig).contains(FILES.getPartitionPath()));
|
||||
assertTrue(getCompletedMetadataPartitions(tableConfig).contains(COLUMN_STATS.getPartitionPath()));
|
||||
assertTrue(getCompletedMetadataPartitions(tableConfig).contains(BLOOM_FILTERS.getPartitionPath()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTurnOffMetadataTableAfterEnable() throws Exception {
|
||||
init(COPY_ON_WRITE, true);
|
||||
@@ -549,13 +666,13 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
||||
// metadata writer to delete column_stats partition
|
||||
HoodieBackedTableMetadataWriter metadataWriter = metadataWriter(client);
|
||||
assertNotNull(metadataWriter, "MetadataWriter should have been initialized");
|
||||
metadataWriter.deletePartitions("0000003", Arrays.asList(MetadataPartitionType.COLUMN_STATS));
|
||||
metadataWriter.deletePartitions("0000003", Arrays.asList(COLUMN_STATS));
|
||||
|
||||
HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataTableBasePath).build();
|
||||
List<String> metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, metadataMetaClient.getBasePath(), false, false);
|
||||
// partition should be physically deleted
|
||||
assertEquals(metadataWriter.getEnabledPartitionTypes().size(), metadataTablePartitions.size());
|
||||
assertFalse(metadataTablePartitions.contains(MetadataPartitionType.COLUMN_STATS.getPartitionPath()));
|
||||
assertFalse(metadataTablePartitions.contains(COLUMN_STATS.getPartitionPath()));
|
||||
|
||||
Option<HoodieInstant> completedReplaceInstant = metadataMetaClient.reloadActiveTimeline().getCompletedReplaceTimeline().lastInstant();
|
||||
assertTrue(completedReplaceInstant.isPresent());
|
||||
@@ -566,7 +683,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
||||
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metadataMetaClient, metadataMetaClient.getActiveTimeline());
|
||||
metadataTablePartitions.forEach(partition -> {
|
||||
List<FileSlice> latestSlices = fsView.getLatestFileSlices(partition).collect(Collectors.toList());
|
||||
if (MetadataPartitionType.COLUMN_STATS.getPartitionPath().equals(partition)) {
|
||||
if (COLUMN_STATS.getPartitionPath().equals(partition)) {
|
||||
// there should not be any file slice in column_stats partition
|
||||
assertTrue(latestSlices.isEmpty());
|
||||
} else {
|
||||
@@ -819,7 +936,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
||||
// Compaction should not be triggered yet. Let's verify no base file
|
||||
// and few log files available.
|
||||
List<FileSlice> fileSlices = table.getSliceView()
|
||||
.getLatestFileSlices(MetadataPartitionType.FILES.getPartitionPath()).collect(Collectors.toList());
|
||||
.getLatestFileSlices(FILES.getPartitionPath()).collect(Collectors.toList());
|
||||
if (fileSlices.isEmpty()) {
|
||||
throw new IllegalStateException("LogFile slices are not available!");
|
||||
}
|
||||
@@ -912,7 +1029,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
||||
.withBasePath(metadataMetaClient.getBasePath())
|
||||
.withLogFilePaths(logFilePaths)
|
||||
.withLatestInstantTime(latestCommitTimestamp)
|
||||
.withPartition(MetadataPartitionType.FILES.getPartitionPath())
|
||||
.withPartition(FILES.getPartitionPath())
|
||||
.withReaderSchema(schema)
|
||||
.withMaxMemorySizeInBytes(100000L)
|
||||
.withBufferSize(4096)
|
||||
@@ -942,7 +1059,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
||||
private void verifyMetadataRecordKeyExcludeFromPayloadBaseFiles(HoodieTable table, boolean enableMetaFields) throws IOException {
|
||||
table.getHoodieView().sync();
|
||||
List<FileSlice> fileSlices = table.getSliceView()
|
||||
.getLatestFileSlices(MetadataPartitionType.FILES.getPartitionPath()).collect(Collectors.toList());
|
||||
.getLatestFileSlices(FILES.getPartitionPath()).collect(Collectors.toList());
|
||||
if (!fileSlices.get(0).getBaseFile().isPresent()) {
|
||||
throw new IllegalStateException("Base file not available!");
|
||||
}
|
||||
@@ -2005,7 +2122,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
||||
assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_STR + ".count"));
|
||||
assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_STR + ".totalDuration"));
|
||||
assertTrue(metricsRegistry.getAllCounts().get(HoodieMetadataMetrics.INITIALIZE_STR + ".count") >= 1L);
|
||||
final String prefix = MetadataPartitionType.FILES.getPartitionPath() + ".";
|
||||
final String prefix = FILES.getPartitionPath() + ".";
|
||||
assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_COUNT_BASE_FILES));
|
||||
assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_COUNT_LOG_FILES));
|
||||
assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_TOTAL_BASE_FILE_SIZE));
|
||||
@@ -2218,10 +2335,10 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
||||
+ numFileVersions + " per file group, but was " + latestSlices.size());
|
||||
List<HoodieLogFile> logFiles = latestSlices.get(0).getLogFiles().collect(Collectors.toList());
|
||||
try {
|
||||
if (MetadataPartitionType.FILES.getPartitionPath().equals(partition)) {
|
||||
if (FILES.getPartitionPath().equals(partition)) {
|
||||
verifyMetadataRawRecords(table, logFiles, false);
|
||||
}
|
||||
if (MetadataPartitionType.COLUMN_STATS.getPartitionPath().equals(partition)) {
|
||||
if (COLUMN_STATS.getPartitionPath().equals(partition)) {
|
||||
verifyMetadataColumnStatsRecords(logFiles);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
|
||||
@@ -95,6 +95,10 @@ public class TestHoodieMetadataBase extends HoodieClientTestHarness {
|
||||
init(tableType, enableMetadataTable, true, false, false);
|
||||
}
|
||||
|
||||
public void init(HoodieTableType tableType, boolean enableMetadataTable, boolean enableColumnStats) throws IOException {
|
||||
init(tableType, enableMetadataTable, true, false, false);
|
||||
}
|
||||
|
||||
public void init(HoodieTableType tableType, boolean enableMetadataTable, boolean enableFullScan, boolean enableMetrics, boolean
|
||||
validateMetadataPayloadStateConsistency) throws IOException {
|
||||
init(tableType, Option.empty(), enableMetadataTable, enableFullScan, enableMetrics,
|
||||
|
||||
Reference in New Issue
Block a user