[HUDI-3556] Re-use rollback instant for rolling back of clustering and compaction if rollback failed mid-way (#4971)
This commit is contained in:
committed by
GitHub
parent
e8918b6c2c
commit
e7bb0413af
@@ -957,8 +957,16 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
return inflightTimelineExcludeClusteringCommit;
|
||||
}
|
||||
|
||||
private Option<HoodiePendingRollbackInfo> getPendingRollbackInfo(HoodieTableMetaClient metaClient, String commitToRollback) {
|
||||
return getPendingRollbackInfos(metaClient).getOrDefault(commitToRollback, Option.empty());
|
||||
protected Option<HoodiePendingRollbackInfo> getPendingRollbackInfo(HoodieTableMetaClient metaClient, String commitToRollback) {
|
||||
return getPendingRollbackInfo(metaClient, commitToRollback, true);
|
||||
}
|
||||
|
||||
protected Option<HoodiePendingRollbackInfo> getPendingRollbackInfo(HoodieTableMetaClient metaClient, String commitToRollback, boolean ignoreCompactionAndClusteringInstants) {
|
||||
return getPendingRollbackInfos(metaClient, ignoreCompactionAndClusteringInstants).getOrDefault(commitToRollback, Option.empty());
|
||||
}
|
||||
|
||||
protected Map<String, Option<HoodiePendingRollbackInfo>> getPendingRollbackInfos(HoodieTableMetaClient metaClient) {
|
||||
return getPendingRollbackInfos(metaClient, true);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -966,20 +974,25 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
* @param metaClient instance of {@link HoodieTableMetaClient} to use.
|
||||
* @return map of pending commits to be rolled-back instants to Rollback Instant and Rollback plan Pair.
|
||||
*/
|
||||
protected Map<String, Option<HoodiePendingRollbackInfo>> getPendingRollbackInfos(HoodieTableMetaClient metaClient) {
|
||||
protected Map<String, Option<HoodiePendingRollbackInfo>> getPendingRollbackInfos(HoodieTableMetaClient metaClient, boolean ignoreCompactionAndClusteringInstants) {
|
||||
List<HoodieInstant> instants = metaClient.getActiveTimeline().filterPendingRollbackTimeline().getInstants().collect(Collectors.toList());
|
||||
Map<String, Option<HoodiePendingRollbackInfo>> infoMap = new HashMap<>();
|
||||
for (HoodieInstant instant : instants) {
|
||||
try {
|
||||
HoodieRollbackPlan rollbackPlan = RollbackUtils.getRollbackPlan(metaClient, instant);
|
||||
String action = rollbackPlan.getInstantToRollback().getAction();
|
||||
if (!HoodieTimeline.COMPACTION_ACTION.equals(action)) {
|
||||
boolean isClustering = HoodieTimeline.REPLACE_COMMIT_ACTION.equals(action)
|
||||
&& ClusteringUtils.getClusteringPlan(metaClient, instant).isPresent();
|
||||
if (!isClustering) {
|
||||
String instantToRollback = rollbackPlan.getInstantToRollback().getCommitTime();
|
||||
infoMap.putIfAbsent(instantToRollback, Option.of(new HoodiePendingRollbackInfo(instant, rollbackPlan)));
|
||||
if (ignoreCompactionAndClusteringInstants) {
|
||||
if (!HoodieTimeline.COMPACTION_ACTION.equals(action)) {
|
||||
boolean isClustering = HoodieTimeline.REPLACE_COMMIT_ACTION.equals(action)
|
||||
&& ClusteringUtils.getClusteringPlan(metaClient, new HoodieInstant(true, rollbackPlan.getInstantToRollback().getAction(),
|
||||
rollbackPlan.getInstantToRollback().getCommitTime())).isPresent();
|
||||
if (!isClustering) {
|
||||
String instantToRollback = rollbackPlan.getInstantToRollback().getCommitTime();
|
||||
infoMap.putIfAbsent(instantToRollback, Option.of(new HoodiePendingRollbackInfo(instant, rollbackPlan)));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
infoMap.putIfAbsent(rollbackPlan.getInstantToRollback().getCommitTime(), Option.of(new HoodiePendingRollbackInfo(instant, rollbackPlan)));
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Fetching rollback plan failed for " + infoMap + ", skip the plan", e);
|
||||
@@ -1211,7 +1224,8 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
}
|
||||
|
||||
protected void rollbackInflightClustering(HoodieInstant inflightInstant, HoodieTable<T, I, K, O> table) {
|
||||
String commitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
Option<HoodiePendingRollbackInfo> pendingRollbackInstantInfo = getPendingRollbackInfo(table.getMetaClient(), inflightInstant.getTimestamp(), false);
|
||||
String commitTime = pendingRollbackInstantInfo.map(entry -> entry.getRollbackInstant().getTimestamp()).orElse(HoodieActiveTimeline.createNewInstantTime());
|
||||
table.scheduleRollback(context, commitTime, inflightInstant, false, config.shouldRollbackUsingMarkers());
|
||||
table.rollback(context, commitTime, inflightInstant, false, false);
|
||||
table.getActiveTimeline().revertReplaceCommitInflightToRequested(inflightInstant);
|
||||
|
||||
@@ -28,6 +28,7 @@ import org.apache.hudi.avro.model.HoodieRestorePlan;
|
||||
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.common.HoodiePendingRollbackInfo;
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
@@ -89,6 +90,7 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
@@ -478,7 +480,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
|
||||
String instantTime,
|
||||
HoodieInstant instantToRollback,
|
||||
boolean skipTimelinePublish, boolean shouldRollbackUsingMarkers);
|
||||
|
||||
|
||||
/**
|
||||
* Rollback the (inflight/committed) record changes with the given commit time.
|
||||
* <pre>
|
||||
@@ -519,14 +521,19 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
|
||||
String restoreInstantTime,
|
||||
String instantToRestore);
|
||||
|
||||
public void rollbackInflightCompaction(HoodieInstant inflightInstant) {
|
||||
rollbackInflightCompaction(inflightInstant, s -> Option.empty());
|
||||
}
|
||||
|
||||
/**
|
||||
* Rollback failed compactions. Inflight rollbacks for compactions revert the .inflight file
|
||||
* to the .requested file.
|
||||
*
|
||||
* @param inflightInstant Inflight Compaction Instant
|
||||
*/
|
||||
public void rollbackInflightCompaction(HoodieInstant inflightInstant) {
|
||||
String commitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
public void rollbackInflightCompaction(HoodieInstant inflightInstant, Function<String, Option<HoodiePendingRollbackInfo>> getPendingRollbackInstantFunc) {
|
||||
final String commitTime = getPendingRollbackInstantFunc.apply(inflightInstant.getTimestamp()).map(entry
|
||||
-> entry.getRollbackInstant().getTimestamp()).orElse(HoodieActiveTimeline.createNewInstantTime());
|
||||
scheduleRollback(context, commitTime, inflightInstant, false, config.shouldRollbackUsingMarkers());
|
||||
rollback(context, commitTime, inflightInstant, false, false);
|
||||
getActiveTimeline().revertCompactionInflightToRequested(inflightInstant);
|
||||
|
||||
Reference in New Issue
Block a user