[HUDI-3135] Make delete partitions lazy to be executed by the cleaner (#4489)
As of now, delete partitions will ensure all file groups are deleted, but the partition as such is not deleted. So, get all partitions might be returning the deleted partitions as well. but no data will be served since all file groups are deleted. With this patch, we are fixing it. We are letting cleaner take care of deleting the partitions when all file groups pertaining to a partitions are deleted. - Fixed the CleanPlanActionExecutor to return meta info about list of partitions to be deleted. If there are no valid file groups for a partition, clean planner will include the partition to be deleted. - Fixed HoodieCleanPlan avro schema to include the list of partitions to be deleted - CleanActionExecutor is fixed to delete partitions if any (as per clean plan) - Same info is added to HoodieCleanMetadata - Metadata table when applying clean metadata, will check for partitions to be deleted and will update the "all_partitions" record for the deleted partitions. Co-authored-by: sivabalan <n.siva.b@gmail.com>
This commit is contained in:
@@ -18,24 +18,32 @@
|
||||
|
||||
package org.apache.hudi.table.action.commit;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
||||
import org.apache.hudi.common.util.HoodieTimer;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.data.HoodieJavaPairRDD;
|
||||
import org.apache.hudi.exception.HoodieDeletePartitionException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.WorkloadProfile;
|
||||
import org.apache.hudi.table.WorkloadStat;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED;
|
||||
import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION;
|
||||
|
||||
public class SparkDeletePartitionCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends SparkInsertOverwriteCommitActionExecutor<T> {
|
||||
@@ -50,16 +58,35 @@ public class SparkDeletePartitionCommitActionExecutor<T extends HoodieRecordPayl
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<HoodieData<WriteStatus>> execute() {
|
||||
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Gather all file ids from all deleting partitions.");
|
||||
Map<String, List<String>> partitionToReplaceFileIds = HoodieJavaPairRDD.getJavaPairRDD(context.parallelize(partitions).distinct()
|
||||
.mapToPair(partitionPath -> Pair.of(partitionPath, getAllExistingFileIds(partitionPath)))).collectAsMap();
|
||||
HoodieWriteMetadata<HoodieData<WriteStatus>> result = new HoodieWriteMetadata<>();
|
||||
result.setPartitionToReplaceFileIds(partitionToReplaceFileIds);
|
||||
result.setIndexUpdateDuration(Duration.ofMillis(timer.endTimer()));
|
||||
result.setWriteStatuses(context.emptyHoodieData());
|
||||
this.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), instantTime);
|
||||
this.commitOnAutoCommit(result);
|
||||
return result;
|
||||
try {
|
||||
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Gather all file ids from all deleting partitions.");
|
||||
Map<String, List<String>> partitionToReplaceFileIds =
|
||||
HoodieJavaPairRDD.getJavaPairRDD(context.parallelize(partitions).distinct()
|
||||
.mapToPair(partitionPath -> Pair.of(partitionPath, getAllExistingFileIds(partitionPath)))).collectAsMap();
|
||||
HoodieWriteMetadata<HoodieData<WriteStatus>> result = new HoodieWriteMetadata<>();
|
||||
result.setPartitionToReplaceFileIds(partitionToReplaceFileIds);
|
||||
result.setIndexUpdateDuration(Duration.ofMillis(timer.endTimer()));
|
||||
result.setWriteStatuses(context.emptyHoodieData());
|
||||
|
||||
// created requested
|
||||
HoodieInstant dropPartitionsInstant = new HoodieInstant(REQUESTED, REPLACE_COMMIT_ACTION, instantTime);
|
||||
if (!table.getMetaClient().getFs().exists(new Path(table.getMetaClient().getMetaPath(),
|
||||
dropPartitionsInstant.getFileName()))) {
|
||||
HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder()
|
||||
.setOperationType(WriteOperationType.DELETE_PARTITION.name())
|
||||
.setExtraMetadata(extraMetadata.orElse(Collections.emptyMap()))
|
||||
.build();
|
||||
table.getMetaClient().getActiveTimeline().saveToPendingReplaceCommit(dropPartitionsInstant,
|
||||
TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedReplaceMetadata));
|
||||
}
|
||||
|
||||
this.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())),
|
||||
instantTime);
|
||||
this.commitOnAutoCommit(result);
|
||||
return result;
|
||||
} catch (Exception e) {
|
||||
throw new HoodieDeletePartitionException("Failed to drop partitions for commit time " + instantTime, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,7 +18,17 @@
|
||||
|
||||
package org.apache.hudi.client.functional;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieMetadataRecord;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
@@ -32,6 +42,7 @@ import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.metrics.Registry;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
@@ -89,16 +100,6 @@ import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.upgrade.SparkUpgradeDowngradeHelper;
|
||||
import org.apache.hudi.table.upgrade.UpgradeDowngrade;
|
||||
import org.apache.hudi.testutils.MetadataMergeWriteStatus;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
@@ -1727,6 +1728,61 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDeletePartitions() throws Exception {
|
||||
init(HoodieTableType.COPY_ON_WRITE);
|
||||
|
||||
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();
|
||||
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) {
|
||||
String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 10);
|
||||
List<HoodieRecord> upsertRecords = new ArrayList<>();
|
||||
for (HoodieRecord entry : records) {
|
||||
if (entry.getPartitionPath().equals(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)
|
||||
|| entry.getPartitionPath().equals(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)) {
|
||||
upsertRecords.add(entry);
|
||||
}
|
||||
}
|
||||
List<WriteStatus> writeStatuses = client.upsert(jsc.parallelize(upsertRecords, 1), newCommitTime).collect();
|
||||
assertNoWriteErrors(writeStatuses);
|
||||
validateMetadata(client);
|
||||
|
||||
// delete partitions
|
||||
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);
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
records = dataGen.generateInserts(newCommitTime, 10);
|
||||
upsertRecords = new ArrayList<>();
|
||||
for (HoodieRecord entry : records) {
|
||||
if (entry.getPartitionPath().equals(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)) {
|
||||
upsertRecords.add(entry);
|
||||
}
|
||||
}
|
||||
writeStatuses = client.upsert(jsc.parallelize(upsertRecords, 1), newCommitTime).collect();
|
||||
assertNoWriteErrors(writeStatuses);
|
||||
|
||||
// trigger clean which will actually triggger deletion of the partition
|
||||
newCommitTime = HoodieActiveTimeline.createNewInstantTime(5000);
|
||||
HoodieCleanMetadata cleanMetadata = client.clean(newCommitTime);
|
||||
validateMetadata(client);
|
||||
assertEquals(1, metadata(client).getAllPartitionPaths().size());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test various error scenarios.
|
||||
*/
|
||||
|
||||
@@ -79,7 +79,7 @@ public class TestHoodieSparkCopyOnWriteTableArchiveWithReplace extends SparkClie
|
||||
client.startCommitWithTime(instantTime4, HoodieActiveTimeline.REPLACE_COMMIT_ACTION);
|
||||
client.deletePartitions(Arrays.asList(DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH), instantTime4);
|
||||
|
||||
// 2nd write batch; 4 commits for the 3rd partition; the 3rd commit to trigger archiving the replace commit
|
||||
// 2nd write batch; 4 commits for the 4th partition; the 4th commit to trigger archiving the replace commit
|
||||
for (int i = 5; i < 9; i++) {
|
||||
String instantTime = HoodieActiveTimeline.createNewInstantTime(i * 1000);
|
||||
client.startCommitWithTime(instantTime);
|
||||
@@ -97,7 +97,7 @@ public class TestHoodieSparkCopyOnWriteTableArchiveWithReplace extends SparkClie
|
||||
// verify records
|
||||
final HoodieTimeline timeline2 = metaClient.getCommitTimeline().filterCompletedInstants();
|
||||
assertEquals(5, countRecordsOptionallySince(jsc(), basePath(), sqlContext(), timeline2, Option.empty()),
|
||||
"should only have the 4 records from the 3rd partition.");
|
||||
"should only have the 5 records from the 3rd partition.");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -687,7 +687,7 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
|
||||
|
||||
public HoodieInstant createCleanMetadata(String instantTime, boolean inflightOnly, boolean isEmpty) throws IOException {
|
||||
HoodieCleanerPlan cleanerPlan = new HoodieCleanerPlan(new HoodieActionInstant("", "", ""), "", new HashMap<>(),
|
||||
CleanPlanV2MigrationHandler.VERSION, new HashMap<>());
|
||||
CleanPlanV2MigrationHandler.VERSION, new HashMap<>(), new ArrayList<>());
|
||||
if (inflightOnly) {
|
||||
HoodieTestTable.of(metaClient).addInflightClean(instantTime, cleanerPlan);
|
||||
} else {
|
||||
|
||||
Reference in New Issue
Block a user