[HUDI-2712] Fixing a bug with rollback of partially failed commit which has new partitions (#3947)
This commit is contained in:
committed by
GitHub
parent
38b6934352
commit
bff8769ed4
@@ -82,7 +82,6 @@ import org.apache.hadoop.util.Time;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.Disabled;
|
||||
import org.junit.jupiter.api.Tag;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
@@ -93,6 +92,7 @@ import org.junit.jupiter.params.provider.MethodSource;
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
@@ -307,9 +307,8 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
||||
/**
|
||||
* Test rollback of various table operations sync to Metadata Table correctly.
|
||||
*/
|
||||
//@ParameterizedTest
|
||||
//@EnumSource(HoodieTableType.class)
|
||||
@Disabled
|
||||
@ParameterizedTest
|
||||
@EnumSource(HoodieTableType.class)
|
||||
public void testRollbackOperations(HoodieTableType tableType) throws Exception {
|
||||
init(tableType);
|
||||
doWriteInsertAndUpsert(testTable);
|
||||
@@ -1087,18 +1086,72 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
||||
assertTrue(oldStatus.getModificationTime() < newStatus.getModificationTime());
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests rollback of a commit which has new partitions which is not present in hudi table prior to the commit being rolledback.
|
||||
*
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void testRollbackOfPartiallyFailedCommitWithNewPartitions() throws Exception {
|
||||
init(HoodieTableType.COPY_ON_WRITE);
|
||||
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
|
||||
|
||||
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext,
|
||||
getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, true, false).build(),
|
||||
true)) {
|
||||
String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 10);
|
||||
List<HoodieRecord> upsertRecords = new ArrayList<>();
|
||||
for (HoodieRecord entry : records) {
|
||||
if (entry.getPartitionPath().equals(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)
|
||||
|| entry.getPartitionPath().equals(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)) {
|
||||
upsertRecords.add(entry);
|
||||
}
|
||||
}
|
||||
List<WriteStatus> writeStatuses = client.upsert(jsc.parallelize(upsertRecords, 1), newCommitTime).collect();
|
||||
assertNoWriteErrors(writeStatuses);
|
||||
validateMetadata(client);
|
||||
|
||||
newCommitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
records = dataGen.generateInserts(newCommitTime, 20);
|
||||
writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
assertNoWriteErrors(writeStatuses);
|
||||
validateMetadata(client);
|
||||
|
||||
// There is no way to simulate failed commit on the main dataset, hence we simply delete the completed
|
||||
// instant so that only the inflight is left over.
|
||||
String commitInstantFileName = HoodieTimeline.makeCommitFileName(newCommitTime);
|
||||
assertTrue(fs.delete(new Path(basePath + Path.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME,
|
||||
commitInstantFileName), false));
|
||||
}
|
||||
|
||||
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext,
|
||||
getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, true, false).build(),
|
||||
true)) {
|
||||
String newCommitTime = client.startCommit();
|
||||
// Next insert
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 20);
|
||||
List<WriteStatus> writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
assertNoWriteErrors(writeStatuses);
|
||||
validateMetadata(client);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test various error scenarios.
|
||||
*/
|
||||
//@Test
|
||||
@Disabled
|
||||
@Test
|
||||
public void testErrorCases() throws Exception {
|
||||
init(HoodieTableType.COPY_ON_WRITE);
|
||||
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
|
||||
|
||||
// TESTCASE: If commit on the metadata table succeeds but fails on the dataset, then on next init the metadata table
|
||||
// should be rolled back to last valid commit.
|
||||
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) {
|
||||
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext,
|
||||
getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, true, false).build(),
|
||||
true)) {
|
||||
String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 10);
|
||||
@@ -1111,6 +1164,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
||||
records = dataGen.generateInserts(newCommitTime, 5);
|
||||
writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
assertNoWriteErrors(writeStatuses);
|
||||
validateMetadata(client);
|
||||
|
||||
// There is no way to simulate failed commit on the main dataset, hence we simply delete the completed
|
||||
// instant so that only the inflight is left over.
|
||||
@@ -1119,7 +1173,9 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
||||
commitInstantFileName), false));
|
||||
}
|
||||
|
||||
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) {
|
||||
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext,
|
||||
getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, true, false).build(),
|
||||
true)) {
|
||||
String newCommitTime = client.startCommit();
|
||||
// Next insert
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 5);
|
||||
|
||||
@@ -87,7 +87,8 @@ public class TestHoodieMetadataBase extends HoodieClientTestHarness {
|
||||
initMetaClient(tableType);
|
||||
initTestDataGenerator();
|
||||
metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath);
|
||||
writeConfig = getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, enableMetadataTable, enableMetrics, enableFullScan).build();
|
||||
writeConfig = getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, enableMetadataTable, enableMetrics,
|
||||
enableFullScan, true).build();
|
||||
initWriteConfigAndMetatableWriter(writeConfig, enableMetadataTable);
|
||||
}
|
||||
|
||||
@@ -265,11 +266,11 @@ public class TestHoodieMetadataBase extends HoodieClientTestHarness {
|
||||
|
||||
protected HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy policy, boolean autoCommit, boolean useFileListingMetadata,
|
||||
boolean enableMetrics) {
|
||||
return getWriteConfigBuilder(policy, autoCommit, useFileListingMetadata, enableMetrics, true);
|
||||
return getWriteConfigBuilder(policy, autoCommit, useFileListingMetadata, enableMetrics, true, true);
|
||||
}
|
||||
|
||||
protected HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy policy, boolean autoCommit, boolean useFileListingMetadata,
|
||||
boolean enableMetrics, boolean enableFullScan) {
|
||||
boolean enableMetrics, boolean enableFullScan, boolean useRollbackUsingMarkers) {
|
||||
Properties properties = new Properties();
|
||||
properties.put(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key(), SimpleKeyGenerator.class.getName());
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA)
|
||||
@@ -292,6 +293,7 @@ public class TestHoodieMetadataBase extends HoodieClientTestHarness {
|
||||
.withExecutorMetrics(true).build())
|
||||
.withMetricsGraphiteConfig(HoodieMetricsGraphiteConfig.newBuilder()
|
||||
.usePrefix("unit-test").build())
|
||||
.withRollbackUsingMarkers(useRollbackUsingMarkers)
|
||||
.withProperties(properties);
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user