[HUDI-2497] Refactor clean and restore actions in hudi-client module (#3734)
This commit is contained in:
@@ -25,18 +25,23 @@ import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
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.hudi.common.util.collection.ImmutablePair;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.PairFlatMapFunction;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
@@ -82,6 +87,20 @@ public class HoodieSparkEngineContext extends HoodieEngineContext {
|
||||
}).reduceByKey(reduceFunc::apply).map(Tuple2::_2).collect();
|
||||
}
|
||||
|
||||
@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 javaSparkContext.parallelize(data.collect(Collectors.toList()), parallelism)
|
||||
.mapPartitionsToPair((PairFlatMapFunction<Iterator<I>, K, V>) iterator ->
|
||||
flatMapToPairFunc.call(iterator).collect(Collectors.toList()).stream()
|
||||
.map(e -> new Tuple2<>(e.getKey(), e.getValue())).iterator()
|
||||
)
|
||||
.reduceByKey(reduceFunc::apply)
|
||||
.map(e -> new ImmutablePair<>(e._1, e._2))
|
||||
.collect().stream();
|
||||
}
|
||||
|
||||
@Override
|
||||
public <I, K, V> List<V> reduceByKey(
|
||||
List<Pair<K, V>> data, SerializableBiFunction<V, V, V> reduceFunc, int parallelism) {
|
||||
|
||||
@@ -50,8 +50,8 @@ import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
|
||||
import org.apache.hudi.table.action.bootstrap.SparkBootstrapCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.clean.SparkCleanActionExecutor;
|
||||
import org.apache.hudi.table.action.clean.SparkCleanPlanActionExecutor;
|
||||
import org.apache.hudi.table.action.clean.CleanActionExecutor;
|
||||
import org.apache.hudi.table.action.clean.CleanPlanActionExecutor;
|
||||
import org.apache.hudi.table.action.cluster.SparkClusteringPlanActionExecutor;
|
||||
import org.apache.hudi.table.action.cluster.SparkExecuteClusteringCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.SparkBulkInsertCommitActionExecutor;
|
||||
@@ -65,7 +65,7 @@ import org.apache.hudi.table.action.commit.SparkInsertPreppedCommitActionExecuto
|
||||
import org.apache.hudi.table.action.commit.SparkMergeHelper;
|
||||
import org.apache.hudi.table.action.commit.SparkUpsertCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.SparkUpsertPreppedCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.restore.SparkCopyOnWriteRestoreActionExecutor;
|
||||
import org.apache.hudi.table.action.restore.CopyOnWriteRestoreActionExecutor;
|
||||
import org.apache.hudi.table.action.rollback.BaseRollbackPlanActionExecutor;
|
||||
import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor;
|
||||
import org.apache.hudi.table.action.savepoint.SavepointActionExecutor;
|
||||
@@ -181,12 +181,12 @@ public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload> extends
|
||||
|
||||
@Override
|
||||
public void rollbackBootstrap(HoodieEngineContext context, String instantTime) {
|
||||
new SparkCopyOnWriteRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute();
|
||||
new CopyOnWriteRestoreActionExecutor(context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<HoodieCleanerPlan> scheduleCleaning(HoodieEngineContext context, String instantTime, Option<Map<String, String>> extraMetadata) {
|
||||
return new SparkCleanPlanActionExecutor<>(context, config,this, instantTime, extraMetadata).execute();
|
||||
return new CleanPlanActionExecutor<>(context, config, this, instantTime, extraMetadata).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -197,7 +197,7 @@ public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload> extends
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String instantTime, String partitionPath, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException {
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException {
|
||||
// these are updates
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(instantTime, partitionPath, fileId, keyToNewRecords, oldDataFile);
|
||||
return handleUpdateInternal(upsertHandle, instantTime, fileId);
|
||||
@@ -242,7 +242,7 @@ public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload> extends
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleInsert(String instantTime, String partitionPath, String fileId,
|
||||
Map<String, HoodieRecord<? extends HoodieRecordPayload>> recordMap) {
|
||||
Map<String, HoodieRecord<? extends HoodieRecordPayload>> recordMap) {
|
||||
HoodieCreateHandle<?,?,?,?> createHandle =
|
||||
new HoodieCreateHandle(config, instantTime, this, partitionPath, fileId, recordMap, taskContextSupplier);
|
||||
createHandle.write();
|
||||
@@ -251,7 +251,7 @@ public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload> extends
|
||||
|
||||
@Override
|
||||
public HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime) {
|
||||
return new SparkCleanActionExecutor((HoodieSparkEngineContext)context, config, this, cleanInstantTime).execute();
|
||||
return new CleanActionExecutor(context, config, this, cleanInstantTime).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -266,7 +266,7 @@ public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload> extends
|
||||
|
||||
@Override
|
||||
public HoodieRestoreMetadata restore(HoodieEngineContext context, String restoreInstantTime, String instantToRestore) {
|
||||
return new SparkCopyOnWriteRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, restoreInstantTime, instantToRestore).execute();
|
||||
return new CopyOnWriteRestoreActionExecutor(context, config, this, restoreInstantTime, instantToRestore).execute();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -39,6 +39,7 @@ import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.bootstrap.SparkBootstrapDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
|
||||
import org.apache.hudi.table.action.compact.BaseScheduleCompactionActionExecutor;
|
||||
import org.apache.hudi.table.action.compact.SparkRunCompactionActionExecutor;
|
||||
import org.apache.hudi.table.action.compact.SparkScheduleCompactionActionExecutor;
|
||||
import org.apache.hudi.table.action.deltacommit.SparkBulkInsertDeltaCommitActionExecutor;
|
||||
@@ -48,8 +49,7 @@ import org.apache.hudi.table.action.deltacommit.SparkInsertDeltaCommitActionExec
|
||||
import org.apache.hudi.table.action.deltacommit.SparkInsertPreppedDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.deltacommit.SparkUpsertDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.deltacommit.SparkUpsertPreppedDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.compact.BaseScheduleCompactionActionExecutor;
|
||||
import org.apache.hudi.table.action.restore.SparkMergeOnReadRestoreActionExecutor;
|
||||
import org.apache.hudi.table.action.restore.MergeOnReadRestoreActionExecutor;
|
||||
import org.apache.hudi.table.action.rollback.BaseRollbackPlanActionExecutor;
|
||||
import org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor;
|
||||
|
||||
@@ -141,7 +141,7 @@ public class HoodieSparkMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
|
||||
@Override
|
||||
public void rollbackBootstrap(HoodieEngineContext context, String instantTime) {
|
||||
new SparkMergeOnReadRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute();
|
||||
new MergeOnReadRestoreActionExecutor(context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -161,7 +161,7 @@ public class HoodieSparkMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
|
||||
@Override
|
||||
public HoodieRestoreMetadata restore(HoodieEngineContext context, String restoreInstantTime, String instantToRestore) {
|
||||
return new SparkMergeOnReadRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, restoreInstantTime, instantToRestore).execute();
|
||||
return new MergeOnReadRestoreActionExecutor(context, config, this, restoreInstantTime, instantToRestore).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -1,134 +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.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.model.HoodieActionInstant;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
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.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.PairFlatMapFunction;
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkCleanActionExecutor<T extends HoodieRecordPayload> extends
|
||||
BaseCleanActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(SparkCleanActionExecutor.class);
|
||||
|
||||
public SparkCleanActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
String instantTime) {
|
||||
super(context, config, table, instantTime);
|
||||
}
|
||||
|
||||
private static PairFlatMapFunction<Iterator<Tuple2<String, CleanFileInfo>>, String, PartitionCleanStat>
|
||||
deleteFilesFunc(HoodieTable table) {
|
||||
return (PairFlatMapFunction<Iterator<Tuple2<String, CleanFileInfo>>, String, PartitionCleanStat>) iter -> {
|
||||
Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>();
|
||||
FileSystem fs = table.getMetaClient().getFs();
|
||||
while (iter.hasNext()) {
|
||||
Tuple2<String, CleanFileInfo> partitionDelFileTuple = iter.next();
|
||||
String partitionPath = partitionDelFileTuple._1();
|
||||
Path deletePath = new Path(partitionDelFileTuple._2().getFilePath());
|
||||
String deletePathStr = deletePath.toString();
|
||||
Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr);
|
||||
if (!partitionCleanStatMap.containsKey(partitionPath)) {
|
||||
partitionCleanStatMap.put(partitionPath, new PartitionCleanStat(partitionPath));
|
||||
}
|
||||
boolean isBootstrapBasePathFile = partitionDelFileTuple._2().isBootstrapBaseFile();
|
||||
PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath);
|
||||
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 -> new Tuple2<>(e.getKey(), e.getValue()))
|
||||
.collect(Collectors.toList()).iterator();
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
List<HoodieCleanStat> clean(HoodieEngineContext context, HoodieCleanerPlan cleanerPlan) {
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
int cleanerParallelism = Math.min(
|
||||
(int) (cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().mapToInt(List::size).count()),
|
||||
config.getCleanerParallelism());
|
||||
LOG.info("Using cleanerParallelism: " + cleanerParallelism);
|
||||
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Perform cleaning of partitions");
|
||||
List<Tuple2<String, PartitionCleanStat>> partitionCleanStats = jsc
|
||||
.parallelize(cleanerPlan.getFilePathsToBeDeletedPerPartition().entrySet().stream()
|
||||
.flatMap(x -> x.getValue().stream().map(y -> new Tuple2<>(x.getKey(),
|
||||
new CleanFileInfo(y.getFilePath(), y.getIsBootstrapBaseFile()))))
|
||||
.collect(Collectors.toList()), cleanerParallelism)
|
||||
.mapPartitionsToPair(deleteFilesFunc(table))
|
||||
.reduceByKey(PartitionCleanStat::merge).collect();
|
||||
|
||||
Map<String, PartitionCleanStat> partitionCleanStatsMap = partitionCleanStats.stream()
|
||||
.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());
|
||||
}
|
||||
}
|
||||
@@ -1,55 +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.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;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkCleanPlanActionExecutor<T extends HoodieRecordPayload> extends
|
||||
BaseCleanPlanActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(SparkCleanPlanActionExecutor.class);
|
||||
|
||||
public SparkCleanPlanActionExecutor(HoodieEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
String instantTime,
|
||||
Option<Map<String, String>> extraMetadata) {
|
||||
super(context, config, table, instantTime, extraMetadata);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Option<HoodieCleanerPlan> createCleanerPlan() {
|
||||
return super.execute();
|
||||
}
|
||||
|
||||
}
|
||||
@@ -1,70 +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.restore;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
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.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieRollbackException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkCopyOnWriteRestoreActionExecutor<T extends HoodieRecordPayload> extends
|
||||
BaseRestoreActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
public SparkCopyOnWriteRestoreActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable table,
|
||||
String instantTime,
|
||||
String restoreInstantTime) {
|
||||
super(context, config, table, instantTime, restoreInstantTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieRollbackMetadata rollbackInstant(HoodieInstant instantToRollback) {
|
||||
if (!instantToRollback.getAction().equals(HoodieTimeline.COMMIT_ACTION)
|
||||
&& !instantToRollback.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) {
|
||||
throw new HoodieRollbackException("Unsupported action in rollback instant:" + instantToRollback);
|
||||
}
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
table.scheduleRollback(context, instantTime, instantToRollback, false);
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
CopyOnWriteRollbackActionExecutor rollbackActionExecutor = new CopyOnWriteRollbackActionExecutor(
|
||||
(HoodieSparkEngineContext) context,
|
||||
config,
|
||||
table,
|
||||
instantTime,
|
||||
instantToRollback,
|
||||
true,
|
||||
true,
|
||||
false);
|
||||
return rollbackActionExecutor.execute();
|
||||
}
|
||||
}
|
||||
@@ -1,81 +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.restore;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
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.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkMergeOnReadRestoreActionExecutor<T extends HoodieRecordPayload> extends
|
||||
BaseRestoreActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
public SparkMergeOnReadRestoreActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable table,
|
||||
String instantTime,
|
||||
String restoreInstantTime) {
|
||||
super(context, config, table, instantTime, restoreInstantTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieRollbackMetadata rollbackInstant(HoodieInstant instantToRollback) {
|
||||
switch (instantToRollback.getAction()) {
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION:
|
||||
case HoodieTimeline.COMPACTION_ACTION:
|
||||
case HoodieTimeline.REPLACE_COMMIT_ACTION:
|
||||
// TODO : Get file status and create a rollback stat and file
|
||||
// TODO : Delete the .aux files along with the instant file, okay for now since the archival process will
|
||||
// delete these files when it does not see a corresponding instant file under .hoodie
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("invalid action name " + instantToRollback.getAction());
|
||||
}
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
table.scheduleRollback(context, instantTime, instantToRollback, false);
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
MergeOnReadRollbackActionExecutor rollbackActionExecutor = new MergeOnReadRollbackActionExecutor(
|
||||
context,
|
||||
config,
|
||||
table,
|
||||
instantTime,
|
||||
instantToRollback,
|
||||
true,
|
||||
true,
|
||||
false);
|
||||
|
||||
// TODO : Get file status and create a rollback stat and file
|
||||
// TODO : Delete the .aux files along with the instant file, okay for now since the archival process will
|
||||
// delete these files when it does not see a corresponding instant file under .hoodie
|
||||
return rollbackActionExecutor.execute();
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user