1
0

Enable multi/nested rollbacks for MOR table type

This commit is contained in:
Nishith Agarwal
2019-01-02 19:13:55 -08:00
committed by n3nash
parent a6c45feb2c
commit 0dd4a90b03
6 changed files with 341 additions and 74 deletions

View File

@@ -33,6 +33,7 @@ import com.uber.hoodie.common.minicluster.HdfsTestService;
import com.uber.hoodie.common.model.FileSlice;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieFileGroup;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRollingStat;
@@ -564,6 +565,178 @@ public class TestMergeOnReadTable {
}).findAny().isPresent());
}
@Test
public void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception {
HoodieWriteConfig cfg = getConfig(false);
final HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
List<String> allCommits = new ArrayList<>();
/**
* Write 1 (only inserts)
*/
String newCommitTime = "001";
allCommits.add(newCommitTime);
client.startCommitWithTime(newCommitTime);
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
JavaRDD<WriteStatus> writeStatusJavaRDD = client.upsert(writeRecords, newCommitTime);
client.commit(newCommitTime, writeStatusJavaRDD);
List<WriteStatus> statuses = writeStatusJavaRDD.collect();
assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
assertTrue(deltaCommit.isPresent());
assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp());
Optional<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
assertFalse(commit.isPresent());
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient,
metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles();
assertTrue(!dataFilesToRead.findAny().isPresent());
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
dataFilesToRead = roView.getLatestDataFiles();
assertTrue("ReadOptimizedTableView should list the parquet files we wrote in the delta commit",
dataFilesToRead.findAny().isPresent());
/**
* Write 2 (inserts + updates)
*/
newCommitTime = "002";
allCommits.add(newCommitTime);
// WriteClient with custom config (disable small file handling)
HoodieWriteClient nClient = new HoodieWriteClient(jsc, HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2)
.withAutoCommit(false).withAssumeDatePartitioning(true).withCompactionConfig(HoodieCompactionConfig.newBuilder()
.compactionSmallFileSize(1 * 1024).withInlineCompaction(false)
.withMaxNumDeltaCommitsBeforeCompaction(1).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1 * 1024).build())
.forTable("test-trip-table").build());
nClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> copyOfRecords = new ArrayList<>(records);
copyOfRecords = dataGen.generateUpdates(newCommitTime, copyOfRecords);
copyOfRecords.addAll(dataGen.generateInserts(newCommitTime, 200));
List<String> dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList());
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath);
assertEquals(recordsRead.size(), 200);
statuses = nClient.upsert(jsc.parallelize(copyOfRecords, 1), newCommitTime).collect();
// Verify there are no errors
assertNoWriteErrors(statuses);
nClient.commit(newCommitTime, writeStatusJavaRDD);
copyOfRecords.clear();
// Schedule a compaction
/**
* Write 3 (inserts + updates)
*/
newCommitTime = "003";
allCommits.add(newCommitTime);
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> newInserts = dataGen.generateInserts(newCommitTime, 100);
records = dataGen.generateUpdates(newCommitTime, records);
records.addAll(newInserts);
writeRecords = jsc.parallelize(records, 1);
writeStatusJavaRDD = client.upsert(writeRecords, newCommitTime);
client.commit(newCommitTime, writeStatusJavaRDD);
statuses = writeStatusJavaRDD.collect();
// Verify there are no errors
assertNoWriteErrors(statuses);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
String compactionInstantTime = "004";
allCommits.add(compactionInstantTime);
client.scheduleCompactionAtInstant(compactionInstantTime, Optional.empty());
// Compaction commit
/**
* Write 4 (updates)
*/
newCommitTime = "005";
allCommits.add(newCommitTime);
client.startCommitWithTime(newCommitTime);
records = dataGen.generateUpdates(newCommitTime, records);
writeRecords = jsc.parallelize(records, 1);
writeStatusJavaRDD = client.upsert(writeRecords, newCommitTime);
client.commit(newCommitTime, writeStatusJavaRDD);
statuses = writeStatusJavaRDD.collect();
// Verify there are no errors
assertNoWriteErrors(statuses);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
compactionInstantTime = "006";
allCommits.add(compactionInstantTime);
client.scheduleCompactionAtInstant(compactionInstantTime, Optional.empty());
JavaRDD<WriteStatus> ws = client.compact(compactionInstantTime);
client.commitCompaction(compactionInstantTime, ws, Optional.empty());
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles);
final String compactedCommitTime = metaClient.getActiveTimeline().reload().getCommitsTimeline().lastInstant().get()
.getTimestamp();
assertTrue(roView.getLatestDataFiles().filter(file -> {
if (compactedCommitTime.equals(file.getCommitTime())) {
return true;
} else {
return false;
}
}).findAny().isPresent());
/**
* Write 5 (updates)
*/
newCommitTime = "007";
allCommits.add(newCommitTime);
client.startCommitWithTime(newCommitTime);
copyOfRecords = new ArrayList<>(records);
copyOfRecords = dataGen.generateUpdates(newCommitTime, copyOfRecords);
copyOfRecords.addAll(dataGen.generateInserts(newCommitTime, 200));
statuses = client.upsert(jsc.parallelize(copyOfRecords, 1), newCommitTime).collect();
// Verify there are no errors
assertNoWriteErrors(statuses);
client.commit(newCommitTime, writeStatusJavaRDD);
copyOfRecords.clear();
// Rollback latest commit first
client.restoreToCommit("000");
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
roView = new HoodieTableFileSystemView(metaClient,
metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
dataFilesToRead = roView.getLatestDataFiles();
assertTrue(!dataFilesToRead.findAny().isPresent());
HoodieTableFileSystemView.RealtimeView rtView = new HoodieTableFileSystemView(metaClient,
metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
List<HoodieFileGroup> fileGroups = ((HoodieTableFileSystemView) rtView).getAllFileGroups().collect(Collectors
.toList());
assertTrue(fileGroups.isEmpty());
}
@Test
public void testUpsertPartitioner() throws Exception {
HoodieWriteConfig cfg = getConfig(true);