1
0

Implement Merge on Read Storage (#76)

1. Create HoodieTable abstraction for commits and fileSystemView
2. HoodieMergeOnReadTable created
3. View is now always obtained from the table and the correct view based on the table type is returned
This commit is contained in:
prazanna
2017-02-21 15:24:00 -08:00
committed by Prasanna Rajaperumal
parent 11d2fd3428
commit eb46e7c72b
47 changed files with 1113 additions and 421 deletions

View File

@@ -30,7 +30,6 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.ParquetUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
@@ -41,6 +40,7 @@ import com.uber.hoodie.exception.HoodieRollbackException;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.HoodieCleaner;
import com.uber.hoodie.table.HoodieTable;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -203,7 +203,10 @@ public class TestHoodieClient implements Serializable {
assertEquals("Latest commit should be 001",readClient.latestCommit(), newCommitTime);
assertEquals("Must contain 200 records", readClient.readCommit(newCommitTime).count(), records.size());
// Should have 100 records in table (check using Index), all in locations marked at commit
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), new HoodieTableMetaClient(fs, basePath)).collect();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect();
checkTaggedRecords(taggedRecords, "001");
/**
@@ -228,8 +231,11 @@ public class TestHoodieClient implements Serializable {
assertEquals("Expecting two commits.", readClient.listCommitsSince("000").size(), 2);
assertEquals("Latest commit should be 004",readClient.latestCommit(), newCommitTime);
metaClient = new HoodieTableMetaClient(fs, basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig());
// Index should be able to locate all updates in correct locations.
taggedRecords = index.tagLocation(jsc.parallelize(dedupedRecords, 1), new HoodieTableMetaClient(fs, basePath)).collect();
taggedRecords = index.tagLocation(jsc.parallelize(dedupedRecords, 1), table).collect();
checkTaggedRecords(taggedRecords, "004");
// Check the entire dataset has 100 records still
@@ -276,7 +282,9 @@ public class TestHoodieClient implements Serializable {
assertEquals("Expecting a single commit.", new HoodieReadClient(jsc, basePath).listCommitsSince("000").size(), 1);
// Should have 100 records in table (check using Index), all in locations marked at commit
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), new HoodieTableMetaClient(fs, basePath)).collect();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect();
checkTaggedRecords(taggedRecords, newCommitTime);
// Keep doing some writes and clean inline. Make sure we have expected number of files remaining.
@@ -291,10 +299,10 @@ public class TestHoodieClient implements Serializable {
assertNoWriteErrors(statuses);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitTimeline();
table = HoodieTable.getHoodieTable(metadata, getConfig());
HoodieTimeline timeline = table.getCommitTimeline();
TableFileSystemView fsView = new ReadOptimizedTableView(fs, metadata);
TableFileSystemView fsView = table.getFileSystemView();
// Need to ensure the following
for (String partitionPath : dataGen.getPartitionPaths()) {
// compute all the versions of all files, from time 0
@@ -358,7 +366,10 @@ public class TestHoodieClient implements Serializable {
// verify that there is a commit
assertEquals("Expecting a single commit.", new HoodieReadClient(jsc, basePath).listCommitsSince("000").size(), 1);
// Should have 100 records in table (check using Index), all in locations marked at commit
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), new HoodieTableMetaClient(fs, basePath)).collect();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect();
checkTaggedRecords(taggedRecords, newCommitTime);
// Keep doing some writes and clean inline. Make sure we have expected number of files remaining.
@@ -372,7 +383,8 @@ public class TestHoodieClient implements Serializable {
assertNoWriteErrors(statuses);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieTimeline activeTimeline = metadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg);
HoodieTimeline activeTimeline = table1.getCompletedCommitTimeline();
Optional<HoodieInstant>
earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1);
Set<HoodieInstant> acceptableCommits =
@@ -384,7 +396,7 @@ public class TestHoodieClient implements Serializable {
acceptableCommits.add(earliestRetainedCommit.get());
}
TableFileSystemView fsView = new ReadOptimizedTableView(fs, metadata);
TableFileSystemView fsView = table1.getFileSystemView();
// Need to ensure the following
for (String partitionPath : dataGen.getPartitionPaths()) {
List<List<HoodieDataFile>> fileVersions = fsView.getEveryVersionInPartition(partitionPath).collect(Collectors.toList());
@@ -637,7 +649,8 @@ public class TestHoodieClient implements Serializable {
assertEquals("2 files needs to be committed.", 2, statuses.size());
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
TableFileSystemView fileSystemView = new ReadOptimizedTableView(fs, metadata);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
TableFileSystemView fileSystemView = table.getFileSystemView();
List<HoodieDataFile> files = fileSystemView.getLatestVersionInPartition(TEST_PARTITION_PATH, commitTime3).collect(
Collectors.toList());
int numTotalInsertsInCommit3 = 0;