[HUDI-2497] Refactor clean and restore actions in hudi-client module (#3734)
This commit is contained in:
@@ -26,11 +26,13 @@ import org.apache.hudi.common.engine.TaskContextSupplier;
|
||||
import org.apache.hudi.common.function.SerializableBiFunction;
|
||||
import org.apache.hudi.common.function.SerializableConsumer;
|
||||
import org.apache.hudi.common.function.SerializableFunction;
|
||||
import org.apache.hudi.common.function.SerializablePairFlatMapFunction;
|
||||
import org.apache.hudi.common.function.SerializablePairFunction;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import org.apache.flink.api.common.functions.RuntimeContext;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
@@ -38,9 +40,11 @@ import java.util.function.Supplier;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import org.apache.hudi.common.util.collection.ImmutablePair;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.util.FlinkClientUtil;
|
||||
|
||||
import static org.apache.hudi.common.function.FunctionWrapper.throwingFlatMapToPairWrapper;
|
||||
import static org.apache.hudi.common.function.FunctionWrapper.throwingFlatMapWrapper;
|
||||
import static org.apache.hudi.common.function.FunctionWrapper.throwingForeachWrapper;
|
||||
import static org.apache.hudi.common.function.FunctionWrapper.throwingMapToPairWrapper;
|
||||
@@ -86,6 +90,17 @@ public class HoodieFlinkEngineContext extends HoodieEngineContext {
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public <I, K, V> Stream<ImmutablePair<K, V>> mapPartitionsToPairAndReduceByKey(
|
||||
Stream<I> data, SerializablePairFlatMapFunction<Iterator<I>, K, V> flatMapToPairFunc,
|
||||
SerializableBiFunction<V, V, V> reduceFunc, int parallelism) {
|
||||
return throwingFlatMapToPairWrapper(flatMapToPairFunc).apply(data.parallel().iterator())
|
||||
.collect(Collectors.groupingBy(Pair::getKey)).entrySet().stream()
|
||||
.map(entry -> new ImmutablePair<>(entry.getKey(), entry.getValue().stream().map(
|
||||
Pair::getValue).reduce(throwingReduceWrapper(reduceFunc)).orElse(null)))
|
||||
.filter(Objects::nonNull);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <I, K, V> List<V> reduceByKey(
|
||||
List<Pair<K, V>> data, SerializableBiFunction<V, V, V> reduceFunc, int parallelism) {
|
||||
|
||||
@@ -44,8 +44,8 @@ import org.apache.hudi.io.HoodieSortedMergeHandle;
|
||||
import org.apache.hudi.io.HoodieWriteHandle;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
|
||||
import org.apache.hudi.table.action.clean.FlinkCleanActionExecutor;
|
||||
import org.apache.hudi.table.action.clean.FlinkScheduleCleanActionExecutor;
|
||||
import org.apache.hudi.table.action.clean.CleanActionExecutor;
|
||||
import org.apache.hudi.table.action.clean.CleanPlanActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.FlinkDeleteCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.FlinkInsertCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.FlinkInsertOverwriteCommitActionExecutor;
|
||||
@@ -297,7 +297,7 @@ public class HoodieFlinkCopyOnWriteTable<T extends HoodieRecordPayload> extends
|
||||
*/
|
||||
@Override
|
||||
public Option<HoodieCleanerPlan> scheduleCleaning(HoodieEngineContext context, String instantTime, Option<Map<String, String>> extraMetadata) {
|
||||
return new FlinkScheduleCleanActionExecutor(context, config, this, instantTime, extraMetadata).execute();
|
||||
return new CleanPlanActionExecutor(context, config, this, instantTime, extraMetadata).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -308,7 +308,7 @@ public class HoodieFlinkCopyOnWriteTable<T extends HoodieRecordPayload> extends
|
||||
|
||||
@Override
|
||||
public HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime) {
|
||||
return new FlinkCleanActionExecutor(context, config, this, cleanInstantTime).execute();
|
||||
return new CleanActionExecutor(context, config, this, cleanInstantTime).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -329,7 +329,7 @@ public class HoodieFlinkCopyOnWriteTable<T extends HoodieRecordPayload> extends
|
||||
// -------------------------------------------------------------------------
|
||||
// Used for compaction
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String instantTime, String partitionPath, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException {
|
||||
// these are updates
|
||||
|
||||
@@ -1,128 +0,0 @@
|
||||
/*
|
||||
* 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.clean;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieActionInstant;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.CleanFileInfo;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
public class FlinkCleanActionExecutor<T extends HoodieRecordPayload> extends
|
||||
BaseCleanActionExecutor<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(FlinkCleanActionExecutor.class);
|
||||
|
||||
public FlinkCleanActionExecutor(HoodieEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table,
|
||||
String instantTime) {
|
||||
super(context, config, table, instantTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
List<HoodieCleanStat> clean(HoodieEngineContext context, HoodieCleanerPlan cleanerPlan) {
|
||||
Stream<Tuple2<String, CleanFileInfo>> filesToBeDeletedPerPartition = cleanerPlan.getFilePathsToBeDeletedPerPartition().entrySet().stream()
|
||||
.flatMap(x -> x.getValue().stream().map(y -> new Tuple2<>(x.getKey(), new CleanFileInfo(y.getFilePath(), y.getIsBootstrapBaseFile()))));
|
||||
|
||||
Stream<Tuple2<String, PartitionCleanStat>> partitionCleanStats =
|
||||
deleteFilesFunc(filesToBeDeletedPerPartition, table)
|
||||
.collect(Collectors.groupingBy(Pair::getLeft))
|
||||
.entrySet().stream()
|
||||
.map(x -> new Tuple2(x.getKey(), x.getValue().stream().map(y -> y.getRight()).reduce(PartitionCleanStat::merge).get()));
|
||||
|
||||
Map<String, PartitionCleanStat> partitionCleanStatsMap = partitionCleanStats
|
||||
.collect(Collectors.toMap(Tuple2::_1, Tuple2::_2));
|
||||
|
||||
// Return PartitionCleanStat for each partition passed.
|
||||
return cleanerPlan.getFilePathsToBeDeletedPerPartition().keySet().stream().map(partitionPath -> {
|
||||
PartitionCleanStat partitionCleanStat = partitionCleanStatsMap.containsKey(partitionPath)
|
||||
? partitionCleanStatsMap.get(partitionPath)
|
||||
: new PartitionCleanStat(partitionPath);
|
||||
HoodieActionInstant actionInstant = cleanerPlan.getEarliestInstantToRetain();
|
||||
return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy()).withPartitionPath(partitionPath)
|
||||
.withEarliestCommitRetained(Option.ofNullable(
|
||||
actionInstant != null
|
||||
? new HoodieInstant(HoodieInstant.State.valueOf(actionInstant.getState()),
|
||||
actionInstant.getAction(), actionInstant.getTimestamp())
|
||||
: null))
|
||||
.withDeletePathPattern(partitionCleanStat.deletePathPatterns())
|
||||
.withSuccessfulDeletes(partitionCleanStat.successDeleteFiles())
|
||||
.withFailedDeletes(partitionCleanStat.failedDeleteFiles())
|
||||
.withDeleteBootstrapBasePathPatterns(partitionCleanStat.getDeleteBootstrapBasePathPatterns())
|
||||
.withSuccessfulDeleteBootstrapBaseFiles(partitionCleanStat.getSuccessfulDeleteBootstrapBaseFiles())
|
||||
.withFailedDeleteBootstrapBaseFiles(partitionCleanStat.getFailedDeleteBootstrapBaseFiles())
|
||||
.build();
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private static Stream<Pair<String, PartitionCleanStat>> deleteFilesFunc(Stream<Tuple2<String, CleanFileInfo>> cleanFileInfo, HoodieTable table) {
|
||||
Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>();
|
||||
FileSystem fs = table.getMetaClient().getFs();
|
||||
|
||||
cleanFileInfo.parallel().forEach(partitionDelFileTuple -> {
|
||||
String partitionPath = partitionDelFileTuple._1();
|
||||
Path deletePath = new Path(partitionDelFileTuple._2().getFilePath());
|
||||
String deletePathStr = deletePath.toString();
|
||||
Boolean deletedFileResult = null;
|
||||
try {
|
||||
deletedFileResult = deleteFileAndGetResult(fs, deletePathStr);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Delete file failed");
|
||||
}
|
||||
final PartitionCleanStat partitionCleanStat;
|
||||
synchronized (partitionCleanStatMap) {
|
||||
partitionCleanStat = partitionCleanStatMap.computeIfAbsent(partitionPath, k -> new PartitionCleanStat(partitionPath));
|
||||
}
|
||||
boolean isBootstrapBasePathFile = partitionDelFileTuple._2().isBootstrapBaseFile();
|
||||
if (isBootstrapBasePathFile) {
|
||||
// For Bootstrap Base file deletions, store the full file path.
|
||||
partitionCleanStat.addDeleteFilePatterns(deletePath.toString(), true);
|
||||
partitionCleanStat.addDeletedFileResult(deletePath.toString(), deletedFileResult, true);
|
||||
} else {
|
||||
partitionCleanStat.addDeleteFilePatterns(deletePath.getName(), false);
|
||||
partitionCleanStat.addDeletedFileResult(deletePath.getName(), deletedFileResult, false);
|
||||
}
|
||||
});
|
||||
return partitionCleanStatMap.entrySet().stream().map(e -> Pair.of(e.getKey(), e.getValue()));
|
||||
}
|
||||
}
|
||||
@@ -1,52 +0,0 @@
|
||||
/*
|
||||
* 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.clean;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
public class FlinkScheduleCleanActionExecutor<T extends HoodieRecordPayload> extends
|
||||
BaseCleanPlanActionExecutor<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(FlinkScheduleCleanActionExecutor.class);
|
||||
|
||||
public FlinkScheduleCleanActionExecutor(HoodieEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table,
|
||||
String instantTime,
|
||||
Option<Map<String, String>> extraMetadata) {
|
||||
super(context, config, table, instantTime, extraMetadata);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Option<HoodieCleanerPlan> createCleanerPlan() {
|
||||
return super.execute();
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user