Timeline Service with Incremental View Syncing support
This commit is contained in:
committed by
vinoth chandar
parent
446f99aa0f
commit
64fec64097
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.uber.hoodie.HoodieReadClient;
|
||||
import com.uber.hoodie.HoodieWriteClient;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
@@ -42,6 +43,7 @@ import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
@@ -53,7 +55,6 @@ 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;
|
||||
import java.util.ArrayList;
|
||||
@@ -83,7 +84,7 @@ import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class TestMergeOnReadTable {
|
||||
|
||||
private static String basePath = null;
|
||||
protected String basePath = null;
|
||||
//NOTE : Be careful in using DFS (FileSystem.class) vs LocalFs(RawLocalFileSystem.class)
|
||||
//The implementation and gurantees of many API's differ, for example check rename(src,dst)
|
||||
private static MiniDFSCluster dfsCluster;
|
||||
@@ -97,6 +98,9 @@ public class TestMergeOnReadTable {
|
||||
if (hdfsTestService != null) {
|
||||
hdfsTestService.stop();
|
||||
dfsCluster.shutdown();
|
||||
dfsCluster = null;
|
||||
dfs = null;
|
||||
hdfsTestService = null;
|
||||
}
|
||||
// Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the
|
||||
// same JVM
|
||||
@@ -441,14 +445,7 @@ public class TestMergeOnReadTable {
|
||||
*/
|
||||
final String commitTime1 = "002";
|
||||
// WriteClient with custom config (disable small file handling)
|
||||
client = 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());
|
||||
client = new HoodieWriteClient(jsc, getHoodieWriteConfigWithSmallFileHandlingOff());
|
||||
client.startCommitWithTime(commitTime1);
|
||||
|
||||
List<HoodieRecord> copyOfRecords = new ArrayList<>(records);
|
||||
@@ -614,14 +611,7 @@ public class TestMergeOnReadTable {
|
||||
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());
|
||||
HoodieWriteClient nClient = new HoodieWriteClient(jsc, getHoodieWriteConfigWithSmallFileHandlingOff());
|
||||
nClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> copyOfRecords = new ArrayList<>(records);
|
||||
@@ -736,6 +726,16 @@ public class TestMergeOnReadTable {
|
||||
assertTrue(fileGroups.isEmpty());
|
||||
}
|
||||
|
||||
protected HoodieWriteConfig getHoodieWriteConfigWithSmallFileHandlingOff() {
|
||||
return 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();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpsertPartitioner() throws Exception {
|
||||
@@ -834,13 +834,14 @@ public class TestMergeOnReadTable {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
HoodieTimeline timeline2 = metaClient.getActiveTimeline();
|
||||
newCommitTime = "101";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
|
||||
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
|
||||
HoodieIndex index = new HoodieBloomIndex<>(config);
|
||||
updatedRecords = index.tagLocation(updatedRecordsRDD, jsc, table).collect();
|
||||
HoodieReadClient readClient = new HoodieReadClient(jsc, config);
|
||||
updatedRecords = readClient.tagLocation(updatedRecordsRDD).collect();
|
||||
|
||||
// Write them to corresponding avro logfiles
|
||||
HoodieTestUtils
|
||||
@@ -850,6 +851,9 @@ public class TestMergeOnReadTable {
|
||||
// Verify that all data file has one log file
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
// In writeRecordsToLogFiles, no commit files are getting added, so resetting file-system view state
|
||||
((SyncableFileSystemView)(table.getRTFileSystemView())).reset();
|
||||
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
List<FileSlice> groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath)
|
||||
.collect(Collectors.toList());
|
||||
@@ -1065,6 +1069,9 @@ public class TestMergeOnReadTable {
|
||||
writeClient.commitCompaction(newCommitTime, statuses, Optional.empty());
|
||||
// Trigger a rollback of compaction
|
||||
writeClient.rollback(newCommitTime);
|
||||
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, jsc);
|
||||
tableRTFileSystemView = table.getRTFileSystemView();
|
||||
((SyncableFileSystemView)tableRTFileSystemView).reset();
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
|
||||
fileSlice.getDataFile().isPresent()).count() == 0);
|
||||
@@ -1292,13 +1299,14 @@ public class TestMergeOnReadTable {
|
||||
return getConfigBuilder(autoCommit).build();
|
||||
}
|
||||
|
||||
private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
|
||||
protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
|
||||
return getConfigBuilder(autoCommit, IndexType.BLOOM);
|
||||
}
|
||||
|
||||
private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, HoodieIndex.IndexType indexType) {
|
||||
protected 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(
|
||||
.withAutoCommit(autoCommit).withAssumeDatePartitioning(true)
|
||||
.withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024).withInlineCompaction(false)
|
||||
.withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build())
|
||||
|
||||
Reference in New Issue
Block a user