1
0

[HUDI-3805] Delete existing corrupted requested rollback plan during rollback (#5245)

This commit is contained in:
Y Ethan Guo
2022-04-07 03:02:34 -07:00
committed by GitHub
parent 531381faff
commit 9d744bb35c
3 changed files with 140 additions and 11 deletions

View File

@@ -20,6 +20,7 @@ package org.apache.hudi.client;
import org.apache.hudi.avro.model.HoodieInstantInfo;
import org.apache.hudi.avro.model.HoodieRollbackPlan;
import org.apache.hudi.avro.model.HoodieRollbackRequest;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieBaseFile;
@@ -61,9 +62,11 @@ import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.hudi.common.util.StringUtils.EMPTY_STRING;
import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -481,4 +484,107 @@ public class TestClientRollback extends HoodieClientTestBase {
assertFalse(testTable.baseFilesExist(partitionAndFileId3, commitTime3));
}
}
private static Stream<Arguments> testRollbackWithRequestedRollbackPlanParams() {
return Arrays.stream(new Boolean[][] {
{true, true}, {true, false}, {false, true}, {false, false},
}).map(Arguments::of);
}
@ParameterizedTest
@MethodSource("testRollbackWithRequestedRollbackPlanParams")
public void testRollbackWithRequestedRollbackPlan(boolean enableMetadataTable, boolean isRollbackPlanCorrupted) throws Exception {
// Let's create some commit files and base files
final String p1 = "2022/04/05";
final String p2 = "2022/04/06";
final String commitTime1 = "20220406010101002";
final String commitTime2 = "20220406020601002";
final String commitTime3 = "20220406030611002";
final String rollbackInstantTime = "20220406040611002";
Map<String, String> partitionAndFileId1 = new HashMap<String, String>() {
{
put(p1, "id11");
put(p2, "id12");
}
};
Map<String, String> partitionAndFileId2 = new HashMap<String, String>() {
{
put(p1, "id21");
put(p2, "id22");
}
};
Map<String, String> partitionAndFileId3 = new HashMap<String, String>() {
{
put(p1, "id31");
put(p2, "id32");
}
};
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withRollbackUsingMarkers(false)
.withMetadataConfig(
HoodieMetadataConfig.newBuilder()
// Column Stats Index is disabled, since these tests construct tables which are
// not valid (empty commit metadata, invalid parquet files)
.withMetadataIndexColumnStats(false)
.enable(enableMetadataTable)
.build()
)
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build())
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
HoodieTestTable testTable = enableMetadataTable
? HoodieMetadataTestTable.of(metaClient, SparkHoodieBackedTableMetadataWriter.create(
metaClient.getHadoopConf(), config, context))
: HoodieTestTable.of(metaClient);
testTable.withPartitionMetaFiles(p1, p2)
.addCommit(commitTime1)
.withBaseFilesInPartitions(partitionAndFileId1)
.addCommit(commitTime2)
.withBaseFilesInPartitions(partitionAndFileId2)
.addInflightCommit(commitTime3)
.withBaseFilesInPartitions(partitionAndFileId3);
try (SparkRDDWriteClient client = getHoodieWriteClient(config)) {
if (isRollbackPlanCorrupted) {
// Add a corrupted requested rollback plan
FileCreateUtils.createRequestedRollbackFile(metaClient.getBasePath(), rollbackInstantTime, new byte[] {0, 1, 2});
} else {
// Add a valid requested rollback plan to roll back commitTime3
HoodieRollbackPlan rollbackPlan = new HoodieRollbackPlan();
List<HoodieRollbackRequest> rollbackRequestList = partitionAndFileId3.keySet().stream()
.map(partition -> new HoodieRollbackRequest(partition, EMPTY_STRING, EMPTY_STRING,
Collections.singletonList(metaClient.getBasePath() + "/" + partition + "/"
+ FileCreateUtils.baseFileName(commitTime3, partitionAndFileId3.get(p1))),
Collections.emptyMap()))
.collect(Collectors.toList());
rollbackPlan.setRollbackRequests(rollbackRequestList);
rollbackPlan.setInstantToRollback(new HoodieInstantInfo(commitTime3, HoodieTimeline.COMMIT_ACTION));
FileCreateUtils.createRequestedRollbackFile(metaClient.getBasePath(), rollbackInstantTime, rollbackPlan);
}
// 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());
// Corrupted requested rollback plan should be deleted before scheduling a new one
assertEquals(rollbackInstants.size(), 1);
HoodieInstant rollbackInstant = rollbackInstants.get(0);
assertTrue(rollbackInstant.isCompleted());
if (isRollbackPlanCorrupted) {
// Should create a new rollback instant
assertNotEquals(rollbackInstantTime, rollbackInstant.getTimestamp());
} else {
// Should reuse the rollback instant
assertEquals(rollbackInstantTime, rollbackInstant.getTimestamp());
}
}
}
}