[HUDI-2422] Adding rollback plan and rollback requested instant (#3651)
- This patch introduces rollback plan and rollback.requested instant. Rollback will be done in two phases, namely rollback plan and rollback action. In planning, we prepare the rollback plan and serialize it to rollback.requested. In the rollback action phase, we fetch details from the plan and just delete the files as per the plan. This will ensure final rollback commit metadata will contain all files that got rolled back even if rollback failed midway and retried again.
This commit is contained in:
committed by
GitHub
parent
4deaa30c8d
commit
b8dad628e5
@@ -82,6 +82,13 @@ public class HoodieSparkEngineContext extends HoodieEngineContext {
|
||||
}).reduceByKey(reduceFunc::apply).map(Tuple2::_2).collect();
|
||||
}
|
||||
|
||||
@Override
|
||||
public <I, K, V> List<V> reduceByKey(
|
||||
List<Pair<K, V>> data, SerializableBiFunction<V, V, V> reduceFunc, int parallelism) {
|
||||
return javaSparkContext.parallelize(data, parallelism).mapToPair(pair -> new Tuple2<K, V>(pair.getLeft(), pair.getRight()))
|
||||
.reduceByKey(reduceFunc::apply).map(Tuple2::_2).collect();
|
||||
}
|
||||
|
||||
@Override
|
||||
public <I, O> List<O> flatMap(List<I> data, SerializableFunction<I, Stream<O>> func, int parallelism) {
|
||||
return javaSparkContext.parallelize(data, parallelism).flatMap(x -> func.apply(x).iterator()).collect();
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.apache.hudi.avro.model.HoodieClusteringPlan;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackPlan;
|
||||
import org.apache.hudi.avro.model.HoodieSavepointMetadata;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
@@ -65,6 +66,7 @@ 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.rollback.BaseRollbackPlanActionExecutor;
|
||||
import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor;
|
||||
import org.apache.hudi.table.action.savepoint.SavepointActionExecutor;
|
||||
import org.apache.log4j.LogManager;
|
||||
@@ -187,6 +189,13 @@ public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload> extends
|
||||
return new SparkCleanPlanActionExecutor<>(context, config,this, instantTime, extraMetadata).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<HoodieRollbackPlan> scheduleRollback(HoodieEngineContext context,
|
||||
String instantTime,
|
||||
HoodieInstant instantToRollback, boolean skipTimelinePublish) {
|
||||
return new BaseRollbackPlanActionExecutor<>(context, config, this, instantTime, instantToRollback, skipTimelinePublish).execute();
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String instantTime, String partitionPath, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException {
|
||||
// these are updates
|
||||
|
||||
@@ -21,6 +21,7 @@ package org.apache.hudi.table;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackPlan;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
@@ -49,6 +50,7 @@ import org.apache.hudi.table.action.deltacommit.SparkUpsertDeltaCommitActionExec
|
||||
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.rollback.BaseRollbackPlanActionExecutor;
|
||||
import org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
@@ -142,6 +144,13 @@ public class HoodieSparkMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
new SparkMergeOnReadRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<HoodieRollbackPlan> scheduleRollback(HoodieEngineContext context,
|
||||
String instantTime,
|
||||
HoodieInstant instantToRollback, boolean skipTimelinePublish) {
|
||||
return new BaseRollbackPlanActionExecutor<>(context, config, this, instantTime, instantToRollback, skipTimelinePublish).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieRollbackMetadata rollback(HoodieEngineContext context,
|
||||
String rollbackInstantTime,
|
||||
|
||||
@@ -48,20 +48,23 @@ public class SparkCopyOnWriteRestoreActionExecutor<T extends HoodieRecordPayload
|
||||
|
||||
@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,
|
||||
HoodieActiveTimeline.createNewInstantTime(),
|
||||
instantTime,
|
||||
instantToRollback,
|
||||
true,
|
||||
true,
|
||||
false);
|
||||
if (!instantToRollback.getAction().equals(HoodieTimeline.COMMIT_ACTION)
|
||||
&& !instantToRollback.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) {
|
||||
throw new HoodieRollbackException("Unsupported action in rollback instant:" + instantToRollback);
|
||||
}
|
||||
return rollbackActionExecutor.execute();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -47,17 +47,6 @@ public class SparkMergeOnReadRestoreActionExecutor<T extends HoodieRecordPayload
|
||||
|
||||
@Override
|
||||
protected HoodieRollbackMetadata rollbackInstant(HoodieInstant instantToRollback) {
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
MergeOnReadRollbackActionExecutor rollbackActionExecutor = new MergeOnReadRollbackActionExecutor(
|
||||
context,
|
||||
config,
|
||||
table,
|
||||
HoodieActiveTimeline.createNewInstantTime(),
|
||||
instantToRollback,
|
||||
true,
|
||||
true,
|
||||
false);
|
||||
|
||||
switch (instantToRollback.getAction()) {
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION:
|
||||
@@ -66,9 +55,27 @@ public class SparkMergeOnReadRestoreActionExecutor<T extends HoodieRecordPayload
|
||||
// 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();
|
||||
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();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.table.upgrade;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieRollbackRequest;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
@@ -26,6 +27,7 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.rollback.BaseRollbackHelper;
|
||||
import org.apache.hudi.table.action.rollback.ListingBasedRollbackHelper;
|
||||
import org.apache.hudi.table.action.rollback.ListingBasedRollbackRequest;
|
||||
|
||||
@@ -44,7 +46,8 @@ public class ZeroToOneUpgradeHandler extends BaseZeroToOneUpgradeHandler {
|
||||
@Override
|
||||
List<HoodieRollbackStat> getListBasedRollBackStats(HoodieTableMetaClient metaClient, HoodieWriteConfig config, HoodieEngineContext context, Option<HoodieInstant> commitInstantOpt,
|
||||
List<ListingBasedRollbackRequest> rollbackRequests) {
|
||||
return new ListingBasedRollbackHelper(metaClient, config)
|
||||
.collectRollbackStats(context, commitInstantOpt.get(), rollbackRequests);
|
||||
List<HoodieRollbackRequest> hoodieRollbackRequests = new ListingBasedRollbackHelper(metaClient, config)
|
||||
.getRollbackRequestsForRollbackPlan(context, commitInstantOpt.get(), rollbackRequests);
|
||||
return new BaseRollbackHelper(metaClient, config).collectRollbackStats(context, commitInstantOpt.get(), hoodieRollbackRequests);
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user