1
0

[HUDI-1350] Support Partition level delete API in HUDI (#2254)

* [HUDI-1350] Support Partition level delete API in HUDI

* [HUDI-1350] Support Partition level delete API in HUDI base InsertOverwriteCommitAction

* [HUDI-1350] Support Partition level delete API in HUDI base InsertOverwriteCommitAction
This commit is contained in:
lw0090
2020-12-29 07:01:06 +08:00
committed by GitHub
parent 6cdf59d92b
commit e177466fd2
11 changed files with 228 additions and 16 deletions

View File

@@ -245,6 +245,13 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
return postWrite(result, instantTime, table);
}
public HoodieWriteResult deletePartitions(List<String> partitions, String instantTime) {
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table = getTableAndInitCtx(WriteOperationType.DELETE_PARTITION, instantTime);
setOperationType(WriteOperationType.DELETE_PARTITION);
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.deletePartitions(context,instantTime, partitions);
return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds());
}
@Override
protected JavaRDD<WriteStatus> postWrite(HoodieWriteMetadata<JavaRDD<WriteStatus>> result,
String instantTime,

View File

@@ -50,6 +50,7 @@ import org.apache.hudi.table.action.cluster.SparkClusteringPlanActionExecutor;
import org.apache.hudi.table.action.commit.SparkBulkInsertCommitActionExecutor;
import org.apache.hudi.table.action.commit.SparkBulkInsertPreppedCommitActionExecutor;
import org.apache.hudi.table.action.commit.SparkDeleteCommitActionExecutor;
import org.apache.hudi.table.action.commit.SparkDeletePartitionCommitActionExecutor;
import org.apache.hudi.table.action.commit.SparkInsertCommitActionExecutor;
import org.apache.hudi.table.action.commit.SparkInsertOverwriteCommitActionExecutor;
import org.apache.hudi.table.action.commit.SparkInsertOverwriteTableCommitActionExecutor;
@@ -108,6 +109,11 @@ public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload> extends
return new SparkDeleteCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, keys).execute();
}
@Override
public HoodieWriteMetadata deletePartitions(HoodieEngineContext context, String instantTime, List<String> partitions) {
return new SparkDeletePartitionCommitActionExecutor(context, config, this, instantTime, partitions).execute();
}
@Override
public HoodieWriteMetadata<JavaRDD<WriteStatus>> upsertPrepped(HoodieEngineContext context, String instantTime,
JavaRDD<HoodieRecord<T>> preppedRecords) {

View File

@@ -0,0 +1,68 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.table.action.commit;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.WriteOperationType;
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.table.HoodieTable;
import org.apache.hudi.table.WorkloadProfile;
import org.apache.hudi.table.WorkloadStat;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import scala.Tuple2;
import java.time.Duration;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class SparkDeletePartitionCommitActionExecutor<T extends HoodieRecordPayload<T>>
extends SparkInsertOverwriteCommitActionExecutor<T> {
private List<String> partitions;
public SparkDeletePartitionCommitActionExecutor(HoodieEngineContext context,
HoodieWriteConfig config, HoodieTable table,
String instantTime, List<String> partitions) {
super(context, config, table, instantTime,null, WriteOperationType.DELETE_PARTITION);
this.partitions = partitions;
}
@Override
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
HoodieTimer timer = new HoodieTimer().startTimer();
Map<String, List<String>> partitionToReplaceFileIds = jsc.parallelize(partitions, partitions.size()).distinct()
.mapToPair(partitionPath -> new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap();
HoodieWriteMetadata result = new HoodieWriteMetadata();
result.setPartitionToReplaceFileIds(partitionToReplaceFileIds);
result.setIndexUpdateDuration(Duration.ofMillis(timer.endTimer()));
result.setWriteStatuses(jsc.emptyRDD());
this.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), instantTime);
this.commitOnAutoCommit(result);
return result;
}
}

View File

@@ -77,7 +77,7 @@ public class SparkInsertOverwriteCommitActionExecutor<T extends HoodieRecordPayl
new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap();
}
private List<String> getAllExistingFileIds(String partitionPath) {
protected List<String> getAllExistingFileIds(String partitionPath) {
// because new commit is not complete. it is safe to mark all existing file Ids as old files
return table.getSliceView().getLatestFileSlices(partitionPath).map(fg -> fg.getFileId()).distinct().collect(Collectors.toList());
}

View File

@@ -36,7 +36,6 @@ import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
public class SparkInsertOverwriteTableCommitActionExecutor<T extends HoodieRecordPayload<T>>
extends SparkInsertOverwriteCommitActionExecutor<T> {
@@ -47,11 +46,6 @@ public class SparkInsertOverwriteTableCommitActionExecutor<T extends HoodieRecor
super(context, config, table, instantTime, inputRecordsRDD, WriteOperationType.INSERT_OVERWRITE_TABLE);
}
protected List<String> getAllExistingFileIds(String partitionPath) {
return table.getSliceView().getLatestFileSlices(partitionPath)
.map(fg -> fg.getFileId()).distinct().collect(Collectors.toList());
}
@Override
protected Map<String, List<String>> getPartitionToReplacedFileIds(JavaRDD<WriteStatus> writeStatuses) {
Map<String, List<String>> partitionToExistingFileIds = new HashMap<>();