1
0

1) Separated rollback as a table operation 2) Implement rollback for MOR

This commit is contained in:
Nishith Agarwal
2017-08-17 17:31:55 -07:00
committed by vinoth chandar
parent e1fe3ab937
commit c7d63a7622
28 changed files with 1200 additions and 260 deletions

View File

@@ -181,6 +181,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
String newCommitTime = "001";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
@@ -193,6 +195,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
HoodieTestUtils.doesCommitExist(basePath, newCommitTime));
newCommitTime = "002";
client.startCommitWithTime(newCommitTime);
records = dataGen.generateUpdates(newCommitTime, 100);
JavaRDD<HoodieRecord> updateRecords = jsc.parallelize(records, 1);
result = client.upsert(updateRecords, newCommitTime);
@@ -214,6 +218,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
* Write 1 (only inserts)
*/
String newCommitTime = "001";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
@@ -242,6 +248,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
* Write 2 (updates)
*/
newCommitTime = "004";
client.startCommitWithTime(newCommitTime);
records = dataGen.generateUpdates(newCommitTime, 100);
LinkedHashMap<HoodieKey, HoodieRecord> recordsMap = new LinkedHashMap<>();
for (HoodieRecord rec : records) {
@@ -299,6 +307,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
* Write actual 200 insert records and ignore 100 delete records
*/
String newCommitTime = "001";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> fewRecordsForInsert = dataGen.generateInserts(newCommitTime, 200);
List<HoodieRecord> fewRecordsForDelete = dataGen.generateDeletes(newCommitTime, 100);
@@ -327,6 +337,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
* Write 2 (deletes+writes)
*/
newCommitTime = "004";
client.startCommitWithTime(newCommitTime);
fewRecordsForDelete = records.subList(0,50);
List<HoodieRecord> fewRecordsForUpdate = records.subList(50,100);
records = dataGen.generateDeletesFromExistingRecords(fewRecordsForDelete);
@@ -378,6 +390,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
* Write 1 (only inserts)
*/
String newCommitTime = "001";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
List<WriteStatus> statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
assertNoWriteErrors(statuses);
@@ -386,6 +400,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
* Write 2 (updates)
*/
newCommitTime = "002";
client.startCommitWithTime(newCommitTime);
records = dataGen.generateUpdates(newCommitTime, records);
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors
@@ -403,6 +419,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
* Write 3 (updates)
*/
newCommitTime = "003";
client.startCommitWithTime(newCommitTime);
records = dataGen.generateUpdates(newCommitTime, records);
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors
@@ -412,6 +430,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
* Write 4 (updates)
*/
newCommitTime = "004";
client.startCommitWithTime(newCommitTime);
records = dataGen.generateUpdates(newCommitTime, records);
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors
@@ -433,6 +453,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
table.getCompletedSavepointTimeline().getInstants().findFirst().get().getTimestamp());
// rollback and reupsert 004
client.rollback(newCommitTime);
client.startCommitWithTime(newCommitTime);
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors
assertNoWriteErrors(statuses);
@@ -462,6 +484,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
* Write 1 (only inserts)
*/
String newCommitTime = "001";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
@@ -472,6 +496,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
* Write 2 (updates)
*/
newCommitTime = "002";
client.startCommitWithTime(newCommitTime);
records = dataGen.generateUpdates(newCommitTime, records);
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors
@@ -483,6 +509,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
* Write 3 (updates)
*/
newCommitTime = "003";
client.startCommitWithTime(newCommitTime);
records = dataGen.generateUpdates(newCommitTime, records);
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors
@@ -502,6 +530,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
* Write 4 (updates)
*/
newCommitTime = "004";
client.startCommitWithTime(newCommitTime);
records = dataGen.generateUpdates(newCommitTime, records);
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors
@@ -917,7 +947,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// Inserts => will write file1
String commitTime1 = "001";
List<HoodieRecord> inserts1 = dataGen.generateInserts(commitTime1, INSERT_SPLIT_LIMIT); // this writes ~5000kb
client.startCommitWithTime(commitTime1);
List<HoodieRecord> inserts1 = dataGen.generateInserts(commitTime1, INSERT_SPLIT_LIMIT); // this writes ~500kb
Set<String> keys1 = HoodieClientTestUtils.getRecordKeys(inserts1);
JavaRDD<HoodieRecord> insertRecordsRDD1 = jsc.parallelize(inserts1, 1);
@@ -933,6 +964,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// Update + Inserts such that they just expand file1
String commitTime2 = "002";
client.startCommitWithTime(commitTime2);
List<HoodieRecord> inserts2 = dataGen.generateInserts(commitTime2, 40);
Set<String> keys2 = HoodieClientTestUtils.getRecordKeys(inserts2);
List<HoodieRecord> insertsAndUpdates2 = new ArrayList<>();
@@ -958,6 +990,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// update + inserts such that file1 is updated and expanded, a new file2 is created.
String commitTime3 = "003";
client.startCommitWithTime(commitTime3);
List<HoodieRecord> insertsAndUpdates3 = dataGen.generateInserts(commitTime3, 200);
Set<String> keys3 = HoodieClientTestUtils.getRecordKeys(insertsAndUpdates3);
List<HoodieRecord> updates3 = dataGen.generateUpdates(commitTime3, inserts2);
@@ -1017,7 +1050,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// Inserts => will write file1
String commitTime1 = "001";
List<HoodieRecord> inserts1 = dataGen.generateInserts(commitTime1, INSERT_SPLIT_LIMIT); // this writes ~5000kb
client.startCommitWithTime(commitTime1);
List<HoodieRecord> inserts1 = dataGen.generateInserts(commitTime1, INSERT_SPLIT_LIMIT); // this writes ~500kb
Set<String> keys1 = HoodieClientTestUtils.getRecordKeys(inserts1);
JavaRDD<HoodieRecord> insertRecordsRDD1 = jsc.parallelize(inserts1, 1);
List<WriteStatus> statuses= client.insert(insertRecordsRDD1, commitTime1).collect();
@@ -1033,6 +1067,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// Second, set of Inserts should just expand file1
String commitTime2 = "002";
client.startCommitWithTime(commitTime2);
List<HoodieRecord> inserts2 = dataGen.generateInserts(commitTime2, 40);
Set<String> keys2 = HoodieClientTestUtils.getRecordKeys(inserts2);
JavaRDD<HoodieRecord> insertRecordsRDD2 = jsc.parallelize(inserts2, 1);
@@ -1055,6 +1090,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// Lots of inserts such that file1 is updated and expanded, a new file2 is created.
String commitTime3 = "003";
client.startCommitWithTime(commitTime3);
List<HoodieRecord> insert3 = dataGen.generateInserts(commitTime3, 200);
JavaRDD<HoodieRecord> insertRecordsRDD3 = jsc.parallelize(insert3, 1);
statuses = client.insert(insertRecordsRDD3, commitTime3).collect();
@@ -1369,6 +1405,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg);
String commitTime = "000";
client.startCommitWithTime(commitTime);
List<HoodieRecord> records = dataGen.generateInserts(commitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);

View File

@@ -112,7 +112,7 @@ public class TestHoodieCommitArchiveLog {
//read the file
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(),
new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")), HoodieArchivedMetaEntry.getClassSchema());
new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")), HoodieArchivedMetaEntry.getClassSchema(), false);
int archivedRecordsCount = 0;
List<IndexedRecord> readRecords = new ArrayList<>();

View File

@@ -138,6 +138,8 @@ public class TestHoodieCompactor {
HoodieWriteConfig config = getConfig();
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
String newCommitTime = "100";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
List<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime).collect();
@@ -147,6 +149,8 @@ public class TestHoodieCompactor {
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
newCommitTime = "101";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
HoodieIndex index = new HoodieBloomIndex<>(config, jsc);

View File

@@ -44,7 +44,6 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.compact.HoodieCompactor;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -148,13 +147,15 @@ public class TestMergeOnReadTable {
@Test
public void testSimpleInsertAndUpdate() throws Exception {
HoodieWriteConfig cfg = getConfig();
HoodieWriteConfig cfg = getConfig(true);
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
/**
* Write 1 (only inserts)
*/
String newCommitTime = "001";
client.startCommitWithTime(newCommitTime);
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
@@ -166,12 +167,12 @@ public class TestMergeOnReadTable {
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
Optional<HoodieInstant> deltaCommit =
metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
assertTrue(deltaCommit.isPresent());
assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp());
Optional<HoodieInstant> commit =
metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
assertFalse(commit.isPresent());
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
@@ -189,6 +190,8 @@ public class TestMergeOnReadTable {
* Write 2 (updates)
*/
newCommitTime = "004";
client.startCommitWithTime(newCommitTime);
records = dataGen.generateUpdates(newCommitTime, 100);
Map<HoodieKey, HoodieRecord> recordsMap = new HashMap<>();
for (HoodieRecord rec : records) {
@@ -211,9 +214,9 @@ public class TestMergeOnReadTable {
HoodieCompactor compactor = new HoodieRealtimeTableCompactor();
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(true));
compactor.compact(jsc, getConfig(), table);
compactor.compact(jsc, getConfig(true), table);
allFiles = HoodieTestUtils.listAllDataFilesInPath(fs, cfg.getBasePath());
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles);
@@ -221,7 +224,7 @@ public class TestMergeOnReadTable {
assertTrue(dataFilesToRead.findAny().isPresent());
// verify that there is a commit
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(fs, cfg.getBasePath(), true), getConfig());
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(fs, cfg.getBasePath(), true), getConfig(false));
HoodieTimeline timeline = table.getCompletedCompactionCommitTimeline();
assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp();
@@ -235,7 +238,7 @@ public class TestMergeOnReadTable {
// Check if record level metadata is aggregated properly at the end of write.
@Test
public void testMetadataAggregateFromWriteStatus() throws Exception {
HoodieWriteConfig cfg = getConfigBuilder().withWriteStatusClass(MetadataMergeWriteStatus.class).build();
HoodieWriteConfig cfg = getConfigBuilder(false).withWriteStatusClass(MetadataMergeWriteStatus.class).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
String newCommitTime = "001";
@@ -243,6 +246,8 @@ public class TestMergeOnReadTable {
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
client.startCommit();
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
Map<String, String> allWriteStatusMergedMetadataMap = MetadataMergeWriteStatus .mergeMetadataForWriteStatuses(statuses);
@@ -253,13 +258,15 @@ public class TestMergeOnReadTable {
@Test
public void testSimpleInsertAndDelete() throws Exception {
HoodieWriteConfig cfg = getConfig();
HoodieWriteConfig cfg = getConfig(true);
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
/**
* Write 1 (only inserts, written as parquet file)
*/
String newCommitTime = "001";
client.startCommitWithTime(newCommitTime);
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 20);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
@@ -294,6 +301,8 @@ public class TestMergeOnReadTable {
* Write 2 (only inserts, written to .log file)
*/
newCommitTime = "002";
client.startCommitWithTime(newCommitTime);
records = dataGen.generateInserts(newCommitTime, 20);
writeRecords = jsc.parallelize(records, 1);
statuses = client.upsert(writeRecords, newCommitTime).collect();
@@ -303,6 +312,8 @@ public class TestMergeOnReadTable {
* Write 2 (only deletes, written to .log file)
*/
newCommitTime = "004";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> fewRecordsForDelete = dataGen.generateDeletesFromExistingRecords(records);
statuses = client.upsert(jsc.parallelize(fewRecordsForDelete, 1), newCommitTime).collect();
@@ -328,18 +339,207 @@ public class TestMergeOnReadTable {
assertEquals("Must contain 20 records", 20, recordsRead.size());
}
private HoodieWriteConfig getConfig() {
return getConfigBuilder().build();
@Test
public void testCOWToMORConvertedDatasetRollback() throws Exception {
//Set TableType to COW
HoodieTestUtils.initTableType(basePath, HoodieTableType.COPY_ON_WRITE);
HoodieWriteConfig cfg = getConfig(true);
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
/**
* Write 1 (only inserts)
*/
String newCommitTime = "001";
client.startCommitWithTime(newCommitTime);
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
//verify there are no errors
assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
Optional<HoodieInstant> commit =
metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
assertTrue(commit.isPresent());
assertEquals("commit should be 001", "001", commit.get().getTimestamp());
/**
* Write 2 (updates)
*/
newCommitTime = "002";
client.startCommitWithTime(newCommitTime);
records = dataGen.generateUpdates(newCommitTime, records);
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors
assertNoWriteErrors(statuses);
//Set TableType to MOR
HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ);
//rollback a COW commit when TableType is MOR
client.rollback(newCommitTime);
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
FileStatus [] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles);
final String absentCommit = newCommitTime;
assertFalse(roView.getLatestDataFiles().filter(file -> {
if(absentCommit.equals(file.getCommitTime()))
return true;
else
return false;
}).findAny().isPresent());
}
private HoodieWriteConfig.Builder getConfigBuilder() {
@Test
public void testRollbackWithDeltaAndCompactionCommit() throws Exception {
HoodieWriteConfig cfg = getConfig(true);
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
// Test delta commit rollback (with all log files)
/**
* Write 1 (only inserts)
*/
String newCommitTime = "001";
client.startCommitWithTime(newCommitTime);
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
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,
hoodieTable.getCompletedCompactionCommitTimeline(), allFiles);
Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles();
assertTrue(!dataFilesToRead.findAny().isPresent());
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles);
dataFilesToRead = roView.getLatestDataFiles();
assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit",
dataFilesToRead.findAny().isPresent());
/**
* Write 2 (updates)
*/
newCommitTime = "002";
client.startCommitWithTime(newCommitTime);
records = dataGen.generateUpdates(newCommitTime, 200);
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors
assertNoWriteErrors(statuses);
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant();
assertTrue(deltaCommit.isPresent());
assertEquals("Latest Delta commit should be 002", "002", deltaCommit.get().getTimestamp());
commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
assertFalse(commit.isPresent());
List<String> dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList());
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles);
assertEquals(recordsRead.size(), 200);
// Test delta commit rollback
client.rollback(newCommitTime);
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles);
dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList());
recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles);
assertEquals(recordsRead.size(), 200);
//Test compaction commit rollback
/**
* Write 2 (updates)
*/
newCommitTime = "003";
client.startCommitWithTime(newCommitTime);
records = dataGen.generateUpdates(newCommitTime, 400);
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
assertNoWriteErrors(statuses);
HoodieCompactor compactor = new HoodieRealtimeTableCompactor();
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(true));
compactor.compact(jsc, getConfig(true), table);
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompactionCommitTimeline(), allFiles);
final String compactedCommitTime = metaClient.getActiveTimeline().reload().getCommitsAndCompactionsTimeline().lastInstant().get().getTimestamp();
assertTrue(roView.getLatestDataFiles().filter(file -> {
if(compactedCommitTime.equals(file.getCommitTime()))
return true;
else
return false;
}).findAny().isPresent());
client.rollback(compactedCommitTime);
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompactionCommitTimeline(), allFiles);
assertFalse(roView.getLatestDataFiles().filter(file -> {
if(compactedCommitTime.equals(file.getCommitTime()))
return true;
else
return false;
}).findAny().isPresent());
}
private HoodieWriteConfig getConfig(Boolean autoCommit) {
return getConfigBuilder(autoCommit).build();
}
private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
return HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withAutoCommit(autoCommit)
.withCompactionConfig(
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024)
.withInlineCompaction(false).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
.forTable("test-trip-table").withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build());
}
@@ -350,4 +550,4 @@ public class TestMergeOnReadTable {
assertFalse("Errors found in write of " + status.getFileId(), status.hasErrors());
}
}
}
}