1
0

[HUDI-1347] Fix Hbase index to make rollback synchronous (via config) (#2188)

Co-authored-by: huangjing <huangjing@clinbrain.com>
Co-authored-by: Sivabalan Narayanan <sivabala@uber.com>
This commit is contained in:
hj2016
2021-02-24 09:56:58 +08:00
committed by GitHub
parent ab9933f206
commit 77ba561a6b
4 changed files with 163 additions and 11 deletions

View File

@@ -117,7 +117,7 @@ public class TestHBaseIndex extends FunctionalTestHarness {
utility = new HBaseTestingUtility(hbaseConfig);
utility.startMiniCluster();
hbaseConfig = utility.getConnection().getConfiguration();
utility.createTable(TableName.valueOf(TABLE_NAME), Bytes.toBytes("_s"));
utility.createTable(TableName.valueOf(TABLE_NAME), Bytes.toBytes("_s"),2);
}
@BeforeEach
@@ -198,8 +198,8 @@ public class TestHBaseIndex extends FunctionalTestHarness {
JavaRDD<HoodieRecord> newWriteRecords = jsc().parallelize(newRecords, 1);
JavaRDD<HoodieRecord> oldWriteRecords = jsc().parallelize(oldRecords, 1);
HoodieWriteConfig config = getConfig(true);
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(getConfig(true));
HoodieWriteConfig config = getConfig(true, false);
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(getConfig(true, false));
try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
// allowed path change test
@@ -225,7 +225,7 @@ public class TestHBaseIndex extends FunctionalTestHarness {
assertEquals(numRecords, taggedRecords.stream().filter(record -> !record.getKey().getPartitionPath().equals(oldPartitionPath)).count());
// not allowed path change test
index = new SparkHoodieHBaseIndex<>(getConfig(false));
index = new SparkHoodieHBaseIndex<>(getConfig(false, false));
List<HoodieRecord> notAllowPathChangeRecords = index.tagLocation(newWriteRecords, context, hoodieTable).collect();
assertEquals(numRecords, notAllowPathChangeRecords.stream().count());
assertEquals(numRecords, taggedRecords.stream().filter(hoodieRecord -> hoodieRecord.isCurrentLocationKnown()
@@ -272,6 +272,66 @@ public class TestHBaseIndex extends FunctionalTestHarness {
&& record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count());
}
@Test
public void testTagLocationAndPartitionPathUpdateWithExplicitRollback() throws Exception {
final int numRecords = 10;
final String oldPartitionPath = "1970/01/01";
final String emptyHoodieRecordPayloadClasssName = EmptyHoodieRecordPayload.class.getName();
HoodieWriteConfig config = getConfig(true, true);
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
final String firstCommitTime = writeClient.startCommit();
List<HoodieRecord> newRecords = dataGen.generateInserts(firstCommitTime, numRecords);
List<HoodieRecord> oldRecords = new LinkedList();
for (HoodieRecord newRecord: newRecords) {
HoodieKey key = new HoodieKey(newRecord.getRecordKey(), oldPartitionPath);
HoodieRecord hoodieRecord = new HoodieRecord(key, newRecord.getData());
oldRecords.add(hoodieRecord);
}
JavaRDD<HoodieRecord> newWriteRecords = jsc().parallelize(newRecords, 1);
JavaRDD<HoodieRecord> oldWriteRecords = jsc().parallelize(oldRecords, 1);
// first commit old record
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
List<HoodieRecord> beforeFirstTaggedRecords = index.tagLocation(oldWriteRecords, context, hoodieTable).collect();
JavaRDD<WriteStatus> oldWriteStatues = writeClient.upsert(oldWriteRecords, firstCommitTime);
index.updateLocation(oldWriteStatues, context, hoodieTable);
writeClient.commit(firstCommitTime, oldWriteStatues);
List<HoodieRecord> afterFirstTaggedRecords = index.tagLocation(oldWriteRecords, context, hoodieTable).collect();
metaClient = HoodieTableMetaClient.reload(metaClient);
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
final String secondCommitTime = writeClient.startCommit();
List<HoodieRecord> beforeSecondTaggedRecords = index.tagLocation(newWriteRecords, context, hoodieTable).collect();
JavaRDD<WriteStatus> newWriteStatues = writeClient.upsert(newWriteRecords, secondCommitTime);
index.updateLocation(newWriteStatues, context, hoodieTable);
writeClient.commit(secondCommitTime, newWriteStatues);
List<HoodieRecord> afterSecondTaggedRecords = index.tagLocation(newWriteRecords, context, hoodieTable).collect();
writeClient.rollback(secondCommitTime);
List<HoodieRecord> afterRollback = index.tagLocation(newWriteRecords, context, hoodieTable).collect();
// Verify the first commit
assertEquals(numRecords, beforeFirstTaggedRecords.stream().filter(record -> record.getCurrentLocation() == null).count());
assertEquals(numRecords, afterFirstTaggedRecords.stream().filter(HoodieRecord::isCurrentLocationKnown).count());
// Verify the second commit
assertEquals(numRecords, beforeSecondTaggedRecords.stream()
.filter(record -> record.getKey().getPartitionPath().equals(oldPartitionPath)
&& record.getData().getClass().getName().equals(emptyHoodieRecordPayloadClasssName)).count());
assertEquals(numRecords * 2, beforeSecondTaggedRecords.stream().count());
assertEquals(numRecords, afterSecondTaggedRecords.stream().count());
assertEquals(numRecords, afterSecondTaggedRecords.stream().filter(record -> !record.getKey().getPartitionPath().equals(oldPartitionPath)).count());
// Verify the rollback
// If an exception occurs after hbase writes the index and the index does not roll back,
// the currentLocation information will not be returned.
assertEquals(numRecords, afterRollback.stream().filter(record -> record.getKey().getPartitionPath().equals(oldPartitionPath)
&& record.getData().getClass().getName().equals(emptyHoodieRecordPayloadClasssName)).count());
assertEquals(numRecords * 2, beforeSecondTaggedRecords.stream().count());
assertEquals(numRecords, afterRollback.stream().filter(HoodieRecord::isCurrentLocationKnown)
.filter(record -> record.getCurrentLocation().getInstantTime().equals(firstCommitTime)).count());
}
}
@Test
public void testSimpleTagLocationAndUpdateWithRollback() throws Exception {
// Load to memory
@@ -413,7 +473,7 @@ public class TestHBaseIndex extends FunctionalTestHarness {
params.put(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP, "1");
params.put(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP, "3");
params.put(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP, "2");
HoodieWriteConfig config = getConfigBuilder(100, false).withProps(params).build();
HoodieWriteConfig config = getConfigBuilder(100, false, false).withProps(params).build();
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
@@ -723,18 +783,18 @@ public class TestHBaseIndex extends FunctionalTestHarness {
}
private HoodieWriteConfig getConfig() {
return getConfigBuilder(100, false).build();
return getConfigBuilder(100, false, false).build();
}
private HoodieWriteConfig getConfig(int hbaseIndexBatchSize) {
return getConfigBuilder(hbaseIndexBatchSize, false).build();
return getConfigBuilder(hbaseIndexBatchSize, false, false).build();
}
private HoodieWriteConfig getConfig(boolean updatePartitionPath) {
return getConfigBuilder(100, updatePartitionPath).build();
private HoodieWriteConfig getConfig(boolean updatePartitionPath, boolean rollbackSync) {
return getConfigBuilder(100, updatePartitionPath, rollbackSync).build();
}
private HoodieWriteConfig.Builder getConfigBuilder(int hbaseIndexBatchSize, boolean updatePartitionPath) {
private HoodieWriteConfig.Builder getConfigBuilder(int hbaseIndexBatchSize, boolean updatePartitionPath, boolean rollbackSync) {
return HoodieWriteConfig.newBuilder().withPath(basePath()).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(1, 1).withDeleteParallelism(1)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024)
@@ -749,6 +809,7 @@ public class TestHBaseIndex extends FunctionalTestHarness {
.hbaseZkZnodeParent(hbaseConfig.get("zookeeper.znode.parent", ""))
.hbaseZkQuorum(hbaseConfig.get("hbase.zookeeper.quorum")).hbaseTableName(TABLE_NAME)
.hbaseIndexUpdatePartitionPath(updatePartitionPath)
.hbaseIndexRollbackSync(rollbackSync)
.hbaseIndexGetBatchSize(hbaseIndexBatchSize).build())
.build());
}