1
0

[HUDI-3743] Support DELETE_PARTITION for metadata table (#5169)

In order to drop any metadata partition (index), we can reuse the DELETE_PARTITION operation in metadata table. Subsequent to this, we can support drop index (with table config update) for async metadata indexer.

- Add a new API in HoodieTableMetadataWriter
- Current only supported for Spark metadata writer
This commit is contained in:
Sagar Sumit
2022-04-01 06:59:17 +05:30
committed by GitHub
parent 28dafa774e
commit a048e940fd
10 changed files with 170 additions and 30 deletions

View File

@@ -25,8 +25,11 @@ import org.apache.hudi.common.data.HoodieData;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.metrics.Registry;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.CommitUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.config.HoodieWriteConfig;
@@ -43,6 +46,7 @@ import org.apache.spark.api.java.JavaRDD;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetadataWriter {
@@ -177,4 +181,16 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
// Update total size of the metadata and count of base/log files
metrics.ifPresent(m -> m.updateSizeMetrics(metadataMetaClient, metadata));
}
@Override
public void deletePartitions(String instantTime, List<MetadataPartitionType> partitions) {
List<String> partitionsToDrop = partitions.stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toList());
LOG.info("Deleting Metadata Table partitions: " + partitionsToDrop);
try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig, true)) {
String actionType = CommitUtils.getCommitActionType(WriteOperationType.DELETE_PARTITION, HoodieTableType.MERGE_ON_READ);
writeClient.startCommitWithTime(instantTime, actionType);
writeClient.deletePartitions(partitionsToDrop, instantTime);
}
}
}

View File

@@ -446,6 +446,73 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
testTableOperationsImpl(engineContext, writeConfig);
}
@ParameterizedTest
@EnumSource(HoodieTableType.class)
public void testMetadataTableDeletePartition(HoodieTableType tableType) throws IOException {
initPath();
int maxCommits = 1;
HoodieWriteConfig cfg = getConfigBuilder(TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.BLOOM, HoodieFailedWritesCleaningPolicy.EAGER)
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(maxCommits).build())
.withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1)
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build())
.build();
init(tableType);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, cfg)) {
// Write 1 (Bulk insert)
String newCommitTime = "0000001";
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 20);
client.startCommitWithTime(newCommitTime);
List<WriteStatus> writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect();
assertNoWriteErrors(writeStatuses);
validateMetadata(client);
// Write 2 (upserts)
newCommitTime = "0000002";
client.startCommitWithTime(newCommitTime);
validateMetadata(client);
records = dataGen.generateInserts(newCommitTime, 10);
writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
assertNoWriteErrors(writeStatuses);
// 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));
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()));
Option<HoodieInstant> completedReplaceInstant = metadataMetaClient.reloadActiveTimeline().getCompletedReplaceTimeline().lastInstant();
assertTrue(completedReplaceInstant.isPresent());
assertEquals("0000003", completedReplaceInstant.get().getTimestamp());
final Map<String, MetadataPartitionType> metadataEnabledPartitionTypes = new HashMap<>();
metadataWriter.getEnabledPartitionTypes().forEach(e -> metadataEnabledPartitionTypes.put(e.getPartitionPath(), e));
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)) {
// there should not be any file slice in column_stats partition
assertTrue(latestSlices.isEmpty());
} else {
assertFalse(latestSlices.isEmpty());
assertTrue(latestSlices.stream().map(FileSlice::getBaseFile).count()
<= metadataEnabledPartitionTypes.get(partition).getFileGroupCount(), "Should have a single latest base file per file group");
assertTrue(latestSlices.size()
<= metadataEnabledPartitionTypes.get(partition).getFileGroupCount(), "Should have a single latest file slice per file group");
}
});
}
}
/**
* Tests that virtual key configs are honored in base files after compaction in metadata table.
*
@@ -1760,7 +1827,6 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
newCommitTime = HoodieActiveTimeline.createNewInstantTime(5000);
client.startCommitWithTime(newCommitTime);
client.deletePartitions(singletonList(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH), newCommitTime);
validateMetadata(client);
// add 1 more commit
newCommitTime = HoodieActiveTimeline.createNewInstantTime(5000);
@@ -1775,7 +1841,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
writeStatuses = client.upsert(jsc.parallelize(upsertRecords, 1), newCommitTime).collect();
assertNoWriteErrors(writeStatuses);
// trigger clean which will actually triggger deletion of the partition
// trigger clean which will actually trigger deletion of the partition
newCommitTime = HoodieActiveTimeline.createNewInstantTime(5000);
HoodieCleanMetadata cleanMetadata = client.clean(newCommitTime);
validateMetadata(client);