[HUDI-2861] Re-use same rollback instant time for failed rollbacks (#4123)
This commit is contained in:
committed by
GitHub
parent
a88691fed3
commit
f8e0176eb0
@@ -32,6 +32,7 @@ import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
|||||||
import org.apache.hudi.client.heartbeat.HeartbeatUtils;
|
import org.apache.hudi.client.heartbeat.HeartbeatUtils;
|
||||||
import org.apache.hudi.client.transaction.TransactionManager;
|
import org.apache.hudi.client.transaction.TransactionManager;
|
||||||
import org.apache.hudi.client.utils.TransactionUtils;
|
import org.apache.hudi.client.utils.TransactionUtils;
|
||||||
|
import org.apache.hudi.common.HoodiePendingRollbackInfo;
|
||||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||||
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
|
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
|
||||||
@@ -67,6 +68,7 @@ import org.apache.hudi.table.BulkInsertPartitioner;
|
|||||||
import org.apache.hudi.table.HoodieTable;
|
import org.apache.hudi.table.HoodieTable;
|
||||||
import org.apache.hudi.table.HoodieTimelineArchiveLog;
|
import org.apache.hudi.table.HoodieTimelineArchiveLog;
|
||||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||||
|
import org.apache.hudi.table.action.rollback.RollbackUtils;
|
||||||
import org.apache.hudi.table.action.savepoint.SavepointHelpers;
|
import org.apache.hudi.table.action.savepoint.SavepointHelpers;
|
||||||
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||||
|
|
||||||
@@ -80,6 +82,8 @@ import java.nio.charset.StandardCharsets;
|
|||||||
import java.text.ParseException;
|
import java.text.ParseException;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
@@ -584,7 +588,23 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
|||||||
|
|
||||||
@Deprecated
|
@Deprecated
|
||||||
public boolean rollback(final String commitInstantTime) throws HoodieRollbackException {
|
public boolean rollback(final String commitInstantTime) throws HoodieRollbackException {
|
||||||
return rollback(commitInstantTime, false);
|
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
|
||||||
|
Option<HoodiePendingRollbackInfo> pendingRollbackInfo = getPendingRollbackInfo(table.getMetaClient(), commitInstantTime);
|
||||||
|
return rollback(commitInstantTime, pendingRollbackInfo, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @Deprecated
|
||||||
|
* Rollback the inflight record changes with the given commit time. This
|
||||||
|
* will be removed in future in favor of {@link AbstractHoodieWriteClient#restoreToInstant(String)}
|
||||||
|
* Adding this api for backwards compatability.
|
||||||
|
* @param commitInstantTime Instant time of the commit
|
||||||
|
* @param skipLocking if this is triggered by another parent transaction, locking can be skipped.
|
||||||
|
* @throws HoodieRollbackException if rollback cannot be performed successfully
|
||||||
|
*/
|
||||||
|
@Deprecated
|
||||||
|
public boolean rollback(final String commitInstantTime, boolean skipLocking) throws HoodieRollbackException {
|
||||||
|
return rollback(commitInstantTime, Option.empty(), skipLocking);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -593,13 +613,15 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
|||||||
* will be removed in future in favor of {@link AbstractHoodieWriteClient#restoreToInstant(String)}
|
* will be removed in future in favor of {@link AbstractHoodieWriteClient#restoreToInstant(String)}
|
||||||
*
|
*
|
||||||
* @param commitInstantTime Instant time of the commit
|
* @param commitInstantTime Instant time of the commit
|
||||||
|
* @param pendingRollbackInfo pending rollback instant and plan if rollback failed from previous attempt.
|
||||||
* @param skipLocking if this is triggered by another parent transaction, locking can be skipped.
|
* @param skipLocking if this is triggered by another parent transaction, locking can be skipped.
|
||||||
* @throws HoodieRollbackException if rollback cannot be performed successfully
|
* @throws HoodieRollbackException if rollback cannot be performed successfully
|
||||||
*/
|
*/
|
||||||
@Deprecated
|
@Deprecated
|
||||||
public boolean rollback(final String commitInstantTime, boolean skipLocking) throws HoodieRollbackException {
|
public boolean rollback(final String commitInstantTime, Option<HoodiePendingRollbackInfo> pendingRollbackInfo, boolean skipLocking) throws HoodieRollbackException {
|
||||||
LOG.info("Begin rollback of instant " + commitInstantTime);
|
LOG.info("Begin rollback of instant " + commitInstantTime);
|
||||||
final String rollbackInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
boolean pendingRollback = pendingRollbackInfo.isPresent();
|
||||||
|
final String rollbackInstantTime = pendingRollback ? pendingRollbackInfo.get().getRollbackInstant().getTimestamp() : HoodieActiveTimeline.createNewInstantTime();
|
||||||
final Timer.Context timerContext = this.metrics.getRollbackCtx();
|
final Timer.Context timerContext = this.metrics.getRollbackCtx();
|
||||||
try {
|
try {
|
||||||
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
|
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
|
||||||
@@ -608,7 +630,7 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
|||||||
.findFirst());
|
.findFirst());
|
||||||
if (commitInstantOpt.isPresent()) {
|
if (commitInstantOpt.isPresent()) {
|
||||||
LOG.info("Scheduling Rollback at instant time :" + rollbackInstantTime);
|
LOG.info("Scheduling Rollback at instant time :" + rollbackInstantTime);
|
||||||
Option<HoodieRollbackPlan> rollbackPlanOption = table.scheduleRollback(context, rollbackInstantTime,
|
Option<HoodieRollbackPlan> rollbackPlanOption = pendingRollback ? Option.of(pendingRollbackInfo.get().getRollbackPlan()) : table.scheduleRollback(context, rollbackInstantTime,
|
||||||
commitInstantOpt.get(), false, config.shouldRollbackUsingMarkers());
|
commitInstantOpt.get(), false, config.shouldRollbackUsingMarkers());
|
||||||
if (rollbackPlanOption.isPresent()) {
|
if (rollbackPlanOption.isPresent()) {
|
||||||
// execute rollback
|
// execute rollback
|
||||||
@@ -838,6 +860,29 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
|||||||
return inflightTimelineExcludeClusteringCommit;
|
return inflightTimelineExcludeClusteringCommit;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private Option<HoodiePendingRollbackInfo> getPendingRollbackInfo(HoodieTableMetaClient metaClient, String commitToRollback) {
|
||||||
|
Option<HoodiePendingRollbackInfo> pendingRollbackInfo = getPendingRollbackInfos(metaClient).get(commitToRollback);
|
||||||
|
return pendingRollbackInfo != null ? pendingRollbackInfo : Option.empty();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Fetch map of pending commits to be rolledback to {@link HoodiePendingRollbackInfo}.
|
||||||
|
* @param metaClient instance of {@link HoodieTableMetaClient} to use.
|
||||||
|
* @return map of pending commits to be rolledback instants to Rollback Instnat and Rollback plan Pair.
|
||||||
|
*/
|
||||||
|
protected Map<String, Option<HoodiePendingRollbackInfo>> getPendingRollbackInfos(HoodieTableMetaClient metaClient) {
|
||||||
|
return metaClient.getActiveTimeline().filterPendingRollbackTimeline().getInstants().map(
|
||||||
|
entry -> {
|
||||||
|
try {
|
||||||
|
HoodieRollbackPlan rollbackPlan = RollbackUtils.getRollbackPlan(metaClient, entry);
|
||||||
|
return Pair.of(rollbackPlan.getInstantToRollback().getCommitTime(), Option.of(new HoodiePendingRollbackInfo(entry, rollbackPlan)));
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieIOException("Fetching rollback plan failed for " + entry, e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
).collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Rollback all failed writes.
|
* Rollback all failed writes.
|
||||||
*/
|
*/
|
||||||
@@ -851,22 +896,28 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
|||||||
*/
|
*/
|
||||||
public Boolean rollbackFailedWrites(boolean skipLocking) {
|
public Boolean rollbackFailedWrites(boolean skipLocking) {
|
||||||
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
|
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
|
||||||
List<String> instantsToRollback = getInstantsToRollback(table.getMetaClient(), config.getFailedWritesCleanPolicy(),
|
List<String> instantsToRollback = getInstantsToRollback(table.getMetaClient(), config.getFailedWritesCleanPolicy(), Option.empty());
|
||||||
Option.empty());
|
Map<String, Option<HoodiePendingRollbackInfo>> pendingRollbacks = getPendingRollbackInfos(table.getMetaClient());
|
||||||
rollbackFailedWrites(instantsToRollback, skipLocking);
|
instantsToRollback.forEach(entry -> pendingRollbacks.putIfAbsent(entry, Option.empty()));
|
||||||
|
|
||||||
|
HashMap<String, Option<HoodiePendingRollbackInfo>> reverseSortedRollbackInstants = pendingRollbacks.entrySet()
|
||||||
|
.stream().sorted((i1, i2) -> i2.getKey().compareTo(i1.getKey()))
|
||||||
|
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue, (e1, e2) -> e1, LinkedHashMap::new));
|
||||||
|
rollbackFailedWrites(reverseSortedRollbackInstants, skipLocking);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void rollbackFailedWrites(List<String> instantsToRollback, boolean skipLocking) {
|
protected void rollbackFailedWrites(Map<String, Option<HoodiePendingRollbackInfo>> instantsToRollback, boolean skipLocking) {
|
||||||
for (String instant : instantsToRollback) {
|
for (Map.Entry<String, Option<HoodiePendingRollbackInfo>> entry : instantsToRollback.entrySet()) {
|
||||||
if (HoodieTimeline.compareTimestamps(instant, HoodieTimeline.LESSER_THAN_OR_EQUALS,
|
if (HoodieTimeline.compareTimestamps(entry.getKey(), HoodieTimeline.LESSER_THAN_OR_EQUALS,
|
||||||
HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS)) {
|
HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS)) {
|
||||||
|
// do we need to handle failed rollback of a bootstrap
|
||||||
rollbackFailedBootstrap();
|
rollbackFailedBootstrap();
|
||||||
HeartbeatUtils.deleteHeartbeatFile(fs, basePath, instant, config);
|
HeartbeatUtils.deleteHeartbeatFile(fs, basePath, entry.getKey(), config);
|
||||||
break;
|
break;
|
||||||
} else {
|
} else {
|
||||||
rollback(instant, skipLocking);
|
rollback(entry.getKey(), entry.getValue(), skipLocking);
|
||||||
HeartbeatUtils.deleteHeartbeatFile(fs, basePath, instant, config);
|
HeartbeatUtils.deleteHeartbeatFile(fs, basePath, entry.getKey(), config);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -62,7 +62,7 @@ public class RollbackUtils {
|
|||||||
* @return Rollback plan corresponding to rollback instant
|
* @return Rollback plan corresponding to rollback instant
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
static HoodieRollbackPlan getRollbackPlan(HoodieTableMetaClient metaClient, HoodieInstant rollbackInstant)
|
public static HoodieRollbackPlan getRollbackPlan(HoodieTableMetaClient metaClient, HoodieInstant rollbackInstant)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
// TODO: add upgrade step if required.
|
// TODO: add upgrade step if required.
|
||||||
return TimelineMetadataUtils.deserializeAvroMetadata(
|
return TimelineMetadataUtils.deserializeAvroMetadata(
|
||||||
|
|||||||
@@ -21,6 +21,7 @@ package org.apache.hudi.client;
|
|||||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||||
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
||||||
import org.apache.hudi.client.utils.TransactionUtils;
|
import org.apache.hudi.client.utils.TransactionUtils;
|
||||||
|
import org.apache.hudi.common.HoodiePendingRollbackInfo;
|
||||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||||
import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
|
import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
|
||||||
import org.apache.hudi.common.metrics.Registry;
|
import org.apache.hudi.common.metrics.Registry;
|
||||||
@@ -443,7 +444,10 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
|
|||||||
this.txnManager.beginTransaction();
|
this.txnManager.beginTransaction();
|
||||||
try {
|
try {
|
||||||
// Ensure no inflight commits by setting EAGER policy and explicitly cleaning all failed commits
|
// Ensure no inflight commits by setting EAGER policy and explicitly cleaning all failed commits
|
||||||
this.rollbackFailedWrites(getInstantsToRollback(metaClient, HoodieFailedWritesCleaningPolicy.EAGER, Option.of(instantTime)), true);
|
List<String> instantsToRollback = getInstantsToRollback(metaClient, HoodieFailedWritesCleaningPolicy.EAGER, Option.of(instantTime));
|
||||||
|
Map<String, Option<HoodiePendingRollbackInfo>> pendingRollbacks = getPendingRollbackInfos(metaClient);
|
||||||
|
instantsToRollback.forEach(entry -> pendingRollbacks.putIfAbsent(entry, Option.empty()));
|
||||||
|
this.rollbackFailedWrites(pendingRollbacks, true);
|
||||||
new UpgradeDowngrade(
|
new UpgradeDowngrade(
|
||||||
metaClient, config, context, SparkUpgradeDowngradeHelper.getInstance())
|
metaClient, config, context, SparkUpgradeDowngradeHelper.getInstance())
|
||||||
.run(HoodieTableVersion.current(), instantTime);
|
.run(HoodieTableVersion.current(), instantTime);
|
||||||
|
|||||||
@@ -26,6 +26,7 @@ import org.apache.hudi.common.model.HoodieRecord;
|
|||||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||||
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
|
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
|
||||||
|
import org.apache.hudi.common.testutils.FileCreateUtils;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||||
@@ -247,6 +248,84 @@ public class TestClientRollback extends HoodieClientTestBase {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test Cases for effects of rollbacking completed/inflight commits.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testFailedRollbackCommit() throws Exception {
|
||||||
|
// Let's create some commit files and base files
|
||||||
|
final String p1 = "2016/05/01";
|
||||||
|
final String p2 = "2016/05/02";
|
||||||
|
final String p3 = "2016/05/06";
|
||||||
|
final String commitTime1 = "20160501010101";
|
||||||
|
final String commitTime2 = "20160502020601";
|
||||||
|
final String commitTime3 = "20160506030611";
|
||||||
|
Map<String, String> partitionAndFileId1 = new HashMap<String, String>() {
|
||||||
|
{
|
||||||
|
put(p1, "id11");
|
||||||
|
put(p2, "id12");
|
||||||
|
put(p3, "id13");
|
||||||
|
}
|
||||||
|
};
|
||||||
|
Map<String, String> partitionAndFileId2 = new HashMap<String, String>() {
|
||||||
|
{
|
||||||
|
put(p1, "id21");
|
||||||
|
put(p2, "id22");
|
||||||
|
put(p3, "id23");
|
||||||
|
}
|
||||||
|
};
|
||||||
|
Map<String, String> partitionAndFileId3 = new HashMap<String, String>() {
|
||||||
|
{
|
||||||
|
put(p1, "id31");
|
||||||
|
put(p2, "id32");
|
||||||
|
put(p3, "id33");
|
||||||
|
}
|
||||||
|
};
|
||||||
|
HoodieTestTable testTable = HoodieTestTable.of(metaClient)
|
||||||
|
.withPartitionMetaFiles(p1, p2, p3)
|
||||||
|
.addCommit(commitTime1)
|
||||||
|
.withBaseFilesInPartitions(partitionAndFileId1)
|
||||||
|
.addCommit(commitTime2)
|
||||||
|
.withBaseFilesInPartitions(partitionAndFileId2)
|
||||||
|
.addInflightCommit(commitTime3)
|
||||||
|
.withBaseFilesInPartitions(partitionAndFileId3);
|
||||||
|
|
||||||
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
|
.withRollbackUsingMarkers(false)
|
||||||
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||||
|
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build())
|
||||||
|
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||||
|
|
||||||
|
try (SparkRDDWriteClient client = getHoodieWriteClient(config)) {
|
||||||
|
|
||||||
|
// Rollback commit3
|
||||||
|
client.rollback(commitTime3);
|
||||||
|
assertFalse(testTable.inflightCommitExists(commitTime3));
|
||||||
|
assertFalse(testTable.baseFilesExist(partitionAndFileId3, commitTime3));
|
||||||
|
assertTrue(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
|
||||||
|
|
||||||
|
metaClient.reloadActiveTimeline();
|
||||||
|
List<HoodieInstant> rollbackInstants = metaClient.getActiveTimeline().getRollbackTimeline().getInstants().collect(Collectors.toList());
|
||||||
|
assertEquals(rollbackInstants.size(), 1);
|
||||||
|
HoodieInstant rollbackInstant = rollbackInstants.get(0);
|
||||||
|
|
||||||
|
// delete rollback completed meta file and retry rollback.
|
||||||
|
FileCreateUtils.deleteRollbackCommit(basePath, rollbackInstant.getTimestamp());
|
||||||
|
|
||||||
|
// recreate actual commit files so that we can retry the rollback
|
||||||
|
testTable.addInflightCommit(commitTime3).withBaseFilesInPartitions(partitionAndFileId3);
|
||||||
|
|
||||||
|
// retry rolling back the commit again.
|
||||||
|
client.rollback(commitTime3);
|
||||||
|
|
||||||
|
// verify there are no extra rollback instants
|
||||||
|
metaClient.reloadActiveTimeline();
|
||||||
|
rollbackInstants = metaClient.getActiveTimeline().getRollbackTimeline().getInstants().collect(Collectors.toList());
|
||||||
|
assertEquals(rollbackInstants.size(), 1);
|
||||||
|
assertEquals(rollbackInstants.get(0), rollbackInstant);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test auto-rollback of commits which are in flight.
|
* Test auto-rollback of commits which are in flight.
|
||||||
*/
|
*/
|
||||||
|
|||||||
@@ -0,0 +1,44 @@
|
|||||||
|
/*
|
||||||
|
* 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.common;
|
||||||
|
|
||||||
|
import org.apache.hudi.avro.model.HoodieRollbackPlan;
|
||||||
|
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Holds rollback instant and rollback plan for a pending rollback.
|
||||||
|
*/
|
||||||
|
public class HoodiePendingRollbackInfo {
|
||||||
|
|
||||||
|
private final HoodieInstant rollbackInstant;
|
||||||
|
private final HoodieRollbackPlan rollbackPlan;
|
||||||
|
|
||||||
|
public HoodiePendingRollbackInfo(HoodieInstant rollbackInstant, HoodieRollbackPlan rollbackPlan) {
|
||||||
|
this.rollbackInstant = rollbackInstant;
|
||||||
|
this.rollbackPlan = rollbackPlan;
|
||||||
|
}
|
||||||
|
|
||||||
|
public HoodieInstant getRollbackInstant() {
|
||||||
|
return rollbackInstant;
|
||||||
|
}
|
||||||
|
|
||||||
|
public HoodieRollbackPlan getRollbackPlan() {
|
||||||
|
return rollbackPlan;
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -124,6 +124,12 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
|
|||||||
s -> s.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION) && !s.isCompleted()), details);
|
s -> s.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION) && !s.isCompleted()), details);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public HoodieTimeline filterPendingRollbackTimeline() {
|
||||||
|
return new HoodieDefaultTimeline(instants.stream().filter(
|
||||||
|
s -> s.getAction().equals(HoodieTimeline.ROLLBACK_ACTION) && !s.isCompleted()), details);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public HoodieTimeline filterPendingCompactionTimeline() {
|
public HoodieTimeline filterPendingCompactionTimeline() {
|
||||||
return new HoodieDefaultTimeline(
|
return new HoodieDefaultTimeline(
|
||||||
|
|||||||
@@ -157,6 +157,11 @@ public interface HoodieTimeline extends Serializable {
|
|||||||
*/
|
*/
|
||||||
HoodieTimeline filterPendingReplaceTimeline();
|
HoodieTimeline filterPendingReplaceTimeline();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Filter this timeline to include pending rollbacks.
|
||||||
|
*/
|
||||||
|
HoodieTimeline filterPendingRollbackTimeline();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a new Timeline with all the instants after startTs.
|
* Create a new Timeline with all the instants after startTs.
|
||||||
*/
|
*/
|
||||||
|
|||||||
@@ -340,6 +340,10 @@ public class FileCreateUtils {
|
|||||||
removeMetaFile(basePath, instantTime, HoodieTimeline.REPLACE_COMMIT_EXTENSION);
|
removeMetaFile(basePath, instantTime, HoodieTimeline.REPLACE_COMMIT_EXTENSION);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static void deleteRollbackCommit(String basePath, String instantTime) throws IOException {
|
||||||
|
removeMetaFile(basePath, instantTime, HoodieTimeline.ROLLBACK_EXTENSION);
|
||||||
|
}
|
||||||
|
|
||||||
public static long getTotalMarkerFileCount(String basePath, String partitionPath, String instantTime, IOType ioType) throws IOException {
|
public static long getTotalMarkerFileCount(String basePath, String partitionPath, String instantTime, IOType ioType) throws IOException {
|
||||||
Path parentPath = Paths.get(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME, instantTime, partitionPath);
|
Path parentPath = Paths.get(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME, instantTime, partitionPath);
|
||||||
if (Files.notExists(parentPath)) {
|
if (Files.notExists(parentPath)) {
|
||||||
|
|||||||
Reference in New Issue
Block a user