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

@@ -26,12 +26,12 @@ 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.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.index.HoodieBloomIndex;
import com.uber.hoodie.table.HoodieTable;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
@@ -79,7 +79,7 @@ public class HoodieReadClient implements Serializable {
*/
private transient final HoodieBloomIndex index;
private final HoodieTimeline commitTimeline;
private HoodieTableMetaClient metaClient;
private HoodieTable hoodieTable;
private transient Optional<SQLContext> sqlContextOpt;
@@ -89,9 +89,11 @@ public class HoodieReadClient implements Serializable {
public HoodieReadClient(JavaSparkContext jsc, String basePath) {
this.jsc = jsc;
this.fs = FSUtils.getFs();
this.metaClient = new HoodieTableMetaClient(fs, basePath, true);
this.commitTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
// Create a Hoodie table which encapsulated the commits and files visible
this.hoodieTable = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
this.commitTimeline = hoodieTable.getCompletedCompactionCommitTimeline();
this.index =
new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
this.sqlContextOpt = Optional.absent();
@@ -134,7 +136,7 @@ public class HoodieReadClient implements Serializable {
assertSqlContext();
JavaPairRDD<HoodieKey, Optional<String>> keyToFileRDD =
index.fetchRecordLocation(hoodieKeys, metaClient);
index.fetchRecordLocation(hoodieKeys, hoodieTable);
List<String> paths = keyToFileRDD
.filter(new Function<Tuple2<HoodieKey, Optional<String>>, Boolean>() {
@Override
@@ -184,14 +186,14 @@ public class HoodieReadClient implements Serializable {
public Dataset<Row> read(String... paths) {
assertSqlContext();
List<String> filteredPaths = new ArrayList<>();
TableFileSystemView fileSystemView = new ReadOptimizedTableView(fs, metaClient);
TableFileSystemView fileSystemView = hoodieTable.getFileSystemView();
try {
for (String path : paths) {
if (!path.contains(metaClient.getBasePath())) {
if (!path.contains(hoodieTable.getMetaClient().getBasePath())) {
throw new HoodieException("Path " + path
+ " does not seem to be a part of a Hoodie dataset at base path "
+ metaClient.getBasePath());
+ hoodieTable.getMetaClient().getBasePath());
}
List<HoodieDataFile> latestFiles = fileSystemView.getLatestVersions(fs.globStatus(new Path(path))).collect(
@@ -243,8 +245,9 @@ public class HoodieReadClient implements Serializable {
*/
public Dataset<Row> readCommit(String commitTime) {
assertSqlContext();
String actionType = hoodieTable.getCompactedCommitActionType();
HoodieInstant commitInstant =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
new HoodieInstant(false, actionType, commitTime);
if (!commitTimeline.containsInstant(commitInstant)) {
new HoodieException("No commit exists at " + commitTime);
}
@@ -261,6 +264,7 @@ public class HoodieReadClient implements Serializable {
}
}
/**
* Checks if the given [Keys] exists in the hoodie table and returns [Key,
* Optional[FullFilePath]] If the optional FullFilePath value is not present, then the key is
@@ -269,7 +273,7 @@ public class HoodieReadClient implements Serializable {
*/
public JavaPairRDD<HoodieKey, Optional<String>> checkExists(
JavaRDD<HoodieKey> hoodieKeys) {
return index.fetchRecordLocation(hoodieKeys, metaClient);
return index.fetchRecordLocation(hoodieKeys, hoodieTable);
}
/**
@@ -280,7 +284,7 @@ public class HoodieReadClient implements Serializable {
* @return A subset of hoodieRecords RDD, with existing records filtered out.
*/
public JavaRDD<HoodieRecord> filterExists(JavaRDD<HoodieRecord> hoodieRecords) {
JavaRDD<HoodieRecord> recordsWithLocation = index.tagLocation(hoodieRecords, metaClient);
JavaRDD<HoodieRecord> recordsWithLocation = index.tagLocation(hoodieRecords, hoodieTable);
return recordsWithLocation.filter(new Function<HoodieRecord, Boolean>() {
@Override
public Boolean call(HoodieRecord v1) throws Exception {