1
0

Adding ability for inserts to be written to log files

This commit is contained in:
Nishith Agarwal
2018-05-13 16:25:11 -07:00
committed by vinoth chandar
parent 34827d50e1
commit 3da063f83b
52 changed files with 1061 additions and 519 deletions

View File

@@ -47,6 +47,7 @@ import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieStorageConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.index.HoodieIndex.IndexType;
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
import java.io.File;
import java.io.IOException;
@@ -70,7 +71,6 @@ import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
@@ -155,7 +155,7 @@ public class TestMergeOnReadTable {
assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
assertTrue(deltaCommit.isPresent());
@@ -166,7 +166,7 @@ public class TestMergeOnReadTable {
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient,
hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles);
metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles();
assertTrue(!dataFilesToRead.findAny().isPresent());
@@ -209,9 +209,8 @@ public class TestMergeOnReadTable {
assertTrue(dataFilesToRead.findAny().isPresent());
// verify that there is a commit
HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true), getConfig(false));
HoodieTimeline timeline = table.getCommitTimeline().filterCompletedInstants();
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true);
HoodieTimeline timeline = metaClient.getCommitTimeline().filterCompletedInstants();
assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp();
assertTrue(HoodieTimeline.compareTimestamps("000", latestCompactionCommitTime, HoodieTimeline.LESSER));
@@ -263,7 +262,7 @@ public class TestMergeOnReadTable {
assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
assertTrue(deltaCommit.isPresent());
@@ -274,7 +273,7 @@ public class TestMergeOnReadTable {
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient,
hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles);
metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles();
assertTrue(!dataFilesToRead.findAny().isPresent());
@@ -372,7 +371,7 @@ public class TestMergeOnReadTable {
client.rollback(newCommitTime);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient,
hoodieTable.getCompletedCommitTimeline(), allFiles);
@@ -408,7 +407,7 @@ public class TestMergeOnReadTable {
assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
assertTrue(deltaCommit.isPresent());
@@ -419,7 +418,7 @@ public class TestMergeOnReadTable {
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient,
hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles);
metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles();
assertTrue(!dataFilesToRead.findAny().isPresent());
@@ -456,7 +455,7 @@ public class TestMergeOnReadTable {
client.rollback(newCommitTime);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles);
dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList());
recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath);
@@ -482,8 +481,8 @@ public class TestMergeOnReadTable {
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), allFiles);
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles);
final String compactedCommitTime = metaClient.getActiveTimeline().reload().getCommitsTimeline().lastInstant().get()
.getTimestamp();
@@ -500,8 +499,8 @@ public class TestMergeOnReadTable {
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), allFiles);
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles);
assertFalse(roView.getLatestDataFiles().filter(file -> {
if (compactedCommitTime.equals(file.getCommitTime())) {
@@ -531,7 +530,7 @@ public class TestMergeOnReadTable {
assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
assertTrue(deltaCommit.isPresent());
@@ -542,7 +541,7 @@ public class TestMergeOnReadTable {
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient,
hoodieTable.getCommitsTimeline().filterCompletedInstants(), allFiles);
metaClient.getCommitsTimeline().filterCompletedInstants(), allFiles);
Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles();
Map<String, Long> parquetFileIdToSize = dataFilesToRead.collect(
Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize));
@@ -591,7 +590,6 @@ public class TestMergeOnReadTable {
}
@Test
@Ignore
public void testLogFileCountsAfterCompaction() throws Exception {
// insert 100 records
HoodieWriteConfig config = getConfig(true);
@@ -606,15 +604,15 @@ public class TestMergeOnReadTable {
// Update all the 100 records
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
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);
updatedRecords = index.tagLocation(updatedRecordsRDD, table).collect();
HoodieIndex index = new HoodieBloomIndex<>(config);
updatedRecords = index.tagLocation(updatedRecordsRDD, jsc, table).collect();
// Write them to corresponding avro logfiles
HoodieTestUtils
@@ -623,7 +621,7 @@ public class TestMergeOnReadTable {
// Verify that all data file has one log file
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, config);
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
for (String partitionPath : dataGen.getPartitionPaths()) {
List<FileSlice> groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath)
.collect(Collectors.toList());
@@ -634,14 +632,14 @@ public class TestMergeOnReadTable {
// Do a compaction
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, config);
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
String commitTime = writeClient.startCompaction();
JavaRDD<WriteStatus> result = writeClient.compact(commitTime);
// Verify that recently written compacted data file has no log file
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, config);
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
assertTrue("Compaction commit should be > than last insert", HoodieTimeline.compareTimestamps(
@@ -677,7 +675,7 @@ public class TestMergeOnReadTable {
// total time taken for creating files should be greater than 0
long totalCreateTime = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalCreateTime())
.reduce((a,b) -> a + b).intValue();
.reduce((a, b) -> a + b).intValue();
Assert.assertTrue(totalCreateTime > 0);
// Update all the 100 records
@@ -690,7 +688,7 @@ public class TestMergeOnReadTable {
writeClient.commit(newCommitTime, statuses);
// total time taken for upsert all records should be greater than 0
long totalUpsertTime = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalUpsertTime())
.reduce((a,b) -> a + b).intValue();
.reduce((a, b) -> a + b).intValue();
Assert.assertTrue(totalUpsertTime > 0);
// Do a compaction
@@ -699,22 +697,168 @@ public class TestMergeOnReadTable {
writeClient.commitCompaction(commitTime, statuses);
// total time taken for scanning log files should be greater than 0
long timeTakenForScanner = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalScanTime())
.reduce((a,b) -> a + b).longValue();
.reduce((a, b) -> a + b).longValue();
Assert.assertTrue(timeTakenForScanner > 0);
}
@Test
public void testSimpleInsertsGeneratedIntoLogFiles() throws Exception {
// insert 100 records
// Setting IndexType to be InMemory to simulate Global Index nature
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
String newCommitTime = "100";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
JavaRDD<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime);
writeClient.commit(newCommitTime, statuses);
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config,
jsc);
TableFileSystemView.RealtimeView tableRTFileSystemView = table.getRTFileSystemView();
long numLogFiles = 0;
for (String partitionPath : dataGen.getPartitionPaths()) {
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getDataFile().isPresent()).count() == 0);
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count() > 0);
numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count();
}
Assert.assertTrue(numLogFiles > 0);
// Do a compaction
String commitTime = writeClient.startCompaction();
statuses = writeClient.compact(commitTime);
Assert.assertTrue(statuses.map(status -> status.getStat().getPath().contains("parquet")).count() == numLogFiles);
Assert.assertEquals(statuses.count(), numLogFiles);
writeClient.commitCompaction(commitTime, statuses);
}
@Test
public void testInsertsGeneratedIntoLogFilesRollback() throws Exception {
// insert 100 records
// Setting IndexType to be InMemory to simulate Global Index nature
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
String newCommitTime = "100";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
JavaRDD<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime);
// trigger an action
List<WriteStatus> writeStatuses = statuses.collect();
// Ensure that inserts are written to only log files
Assert.assertEquals(writeStatuses.stream().filter(writeStatus -> !writeStatus.getStat().getPath().contains("log")
).count(), 0);
Assert.assertTrue(writeStatuses.stream().filter(writeStatus -> writeStatus.getStat().getPath().contains("log")
).count() > 0);
// rollback a failed commit
boolean rollback = writeClient.rollback(newCommitTime);
Assert.assertTrue(rollback);
newCommitTime = "101";
writeClient.startCommitWithTime(newCommitTime);
// insert 100 records
records = dataGen.generateInserts(newCommitTime, 100);
recordsRDD = jsc.parallelize(records, 1);
statuses = writeClient.insert(recordsRDD, newCommitTime);
writeClient.commit(newCommitTime, statuses);
// rollback a successful commit
writeClient.rollback(newCommitTime);
final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
TableFileSystemView.RealtimeView tableRTFileSystemView = table.getRTFileSystemView();
long numLogFiles = 0;
for (String partitionPath : dataGen.getPartitionPaths()) {
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getDataFile().isPresent()).count() == 0);
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count() == 0);
numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count();
}
Assert.assertTrue(numLogFiles == 0);
}
@Test
public void testInsertsGeneratedIntoLogFilesRollbackAfterCompaction() throws Exception {
// insert 100 records
// Setting IndexType to be InMemory to simulate Global Index nature
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
String newCommitTime = "100";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
JavaRDD<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime);
writeClient.commit(newCommitTime, statuses);
// trigger an action
statuses.collect();
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config,
jsc);
TableFileSystemView.RealtimeView tableRTFileSystemView = table.getRTFileSystemView();
long numLogFiles = 0;
for (String partitionPath : dataGen.getPartitionPaths()) {
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getDataFile().isPresent()).count() == 0);
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count() > 0);
numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count();
}
Assert.assertTrue(numLogFiles > 0);
// Do a compaction
newCommitTime = writeClient.startCompaction();
statuses = writeClient.compact(newCommitTime);
// Ensure all log files have been compacted into parquet files
Assert.assertTrue(statuses.map(status -> status.getStat().getPath().contains("parquet")).count() == numLogFiles);
Assert.assertEquals(statuses.count(), numLogFiles);
writeClient.commitCompaction(newCommitTime, statuses);
// Trigger a rollback of compaction
writeClient.rollback(newCommitTime);
for (String partitionPath : dataGen.getPartitionPaths()) {
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getDataFile().isPresent()).count() == 0);
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count() > 0);
}
}
private HoodieWriteConfig getConfig(Boolean autoCommit) {
return getConfigBuilder(autoCommit).build();
}
private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
return getConfigBuilder(autoCommit, IndexType.BLOOM);
}
private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, HoodieIndex.IndexType indexType) {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withAutoCommit(autoCommit).withAssumeDatePartitioning(true).withCompactionConfig(
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024).withInlineCompaction(false)
.withMaxNumDeltaCommitsBeforeCompaction(1).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build())
.forTable("test-trip-table")
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build());
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build());
}
private void assertNoWriteErrors(List<WriteStatus> statuses) {