Refactor hoodie-common and create right abstractions for Hoodie Storage V2.0
The following is the gist of changes done - All low-level operation of creating a commit code was in HoodieClient which made it hard to share code if there was a compaction commit. - HoodieTableMetadata contained a mix of metadata and filtering files. (Also few operations required FileSystem to be passed in because those were called from TaskExecutors and others had FileSystem as a global variable). Since merge-on-read requires a lot of that code, but will have to change slightly on how it operates on the metadata and how it filters the files. The two set of operation are split into HoodieTableMetaClient and TableFileSystemView. - Everything (active commits, archived commits, cleaner log, save point log and in future delta and compaction commits) in HoodieTableMetaClient is a HoodieTimeline. Timeline is a series of instants, which has an in-built concept of inflight and completed commit markers. - A timeline can be queries for ranges, contains and also use to create new datapoint (create a new commit etc). Commit (and all the above metadata) creation/deletion is streamlined in a timeline - Multiple timelines can be merged into a single timeline, giving us an audit timeline to whatever happened in a hoodie dataset. This also helps with #55. - Move to java 8 and introduce java 8 succinct syntax in refactored code
This commit is contained in:
@@ -19,17 +19,18 @@ package com.uber.hoodie;
|
||||
import com.google.common.base.Optional;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieCommits;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||
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.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 org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
@@ -53,6 +54,7 @@ import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
@@ -75,7 +77,8 @@ public class HoodieReadClient implements Serializable {
|
||||
* BloomIndex
|
||||
*/
|
||||
private transient final HoodieBloomIndex index;
|
||||
private HoodieTableMetadata metadata;
|
||||
private final HoodieTimeline commitTimeline;
|
||||
private HoodieTableMetaClient metaClient;
|
||||
private transient Optional<SQLContext> sqlContextOpt;
|
||||
|
||||
|
||||
@@ -85,7 +88,8 @@ public class HoodieReadClient implements Serializable {
|
||||
public HoodieReadClient(JavaSparkContext jsc, String basePath) {
|
||||
this.jsc = jsc;
|
||||
this.fs = FSUtils.getFs();
|
||||
this.metadata = new HoodieTableMetadata(fs, basePath);
|
||||
this.metaClient = new HoodieTableMetaClient(fs, basePath, true);
|
||||
this.commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
this.index = new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
|
||||
this.sqlContextOpt = Optional.absent();
|
||||
}
|
||||
@@ -127,7 +131,7 @@ public class HoodieReadClient implements Serializable {
|
||||
|
||||
assertSqlContext();
|
||||
JavaPairRDD<HoodieKey, Optional<String>> keyToFileRDD =
|
||||
index.fetchRecordLocation(hoodieKeys, metadata);
|
||||
index.fetchRecordLocation(hoodieKeys, metaClient);
|
||||
List<String> paths = keyToFileRDD
|
||||
.filter(new Function<Tuple2<HoodieKey, Optional<String>>, Boolean>() {
|
||||
@Override
|
||||
@@ -177,17 +181,20 @@ public class HoodieReadClient implements Serializable {
|
||||
public Dataset<Row> read(String... paths) {
|
||||
assertSqlContext();
|
||||
List<String> filteredPaths = new ArrayList<>();
|
||||
TableFileSystemView fileSystemView = new ReadOptimizedTableView(fs, metaClient);
|
||||
|
||||
try {
|
||||
for (String path : paths) {
|
||||
if (!path.contains(metadata.getBasePath())) {
|
||||
if (!path.contains(metaClient.getBasePath())) {
|
||||
throw new HoodieException("Path " + path
|
||||
+ " does not seem to be a part of a Hoodie dataset at base path "
|
||||
+ metadata.getBasePath());
|
||||
+ metaClient.getBasePath());
|
||||
}
|
||||
|
||||
FileStatus[] latestFiles = metadata.getLatestVersions(fs.globStatus(new Path(path)));
|
||||
for (FileStatus file : latestFiles) {
|
||||
filteredPaths.add(file.getPath().toString());
|
||||
List<HoodieDataFile> latestFiles = fileSystemView.streamLatestVersions(fs.globStatus(new Path(path))).collect(
|
||||
Collectors.toList());
|
||||
for (HoodieDataFile file : latestFiles) {
|
||||
filteredPaths.add(file.getPath());
|
||||
}
|
||||
}
|
||||
return sqlContextOpt.get().read()
|
||||
@@ -205,15 +212,19 @@ public class HoodieReadClient implements Serializable {
|
||||
*/
|
||||
public Dataset<Row> readSince(String lastCommitTimestamp) {
|
||||
|
||||
List<String> commitsToReturn = metadata.findCommitsAfter(lastCommitTimestamp, Integer.MAX_VALUE);
|
||||
List<String> commitsToReturn =
|
||||
commitTimeline.findInstantsAfter(lastCommitTimestamp, Integer.MAX_VALUE)
|
||||
.collect(Collectors.toList());
|
||||
//TODO: we can potentially trim this down to only affected partitions, using CommitMetadata
|
||||
try {
|
||||
|
||||
// Go over the commit metadata, and obtain the new files that need to be read.
|
||||
HashMap<String, String> fileIdToFullPath = new HashMap<>();
|
||||
for (String commit: commitsToReturn) {
|
||||
HoodieCommitMetadata metadata =
|
||||
HoodieCommitMetadata.fromBytes(commitTimeline.readInstantDetails(commit).get());
|
||||
// get files from each commit, and replace any previous versions
|
||||
fileIdToFullPath.putAll(metadata.getCommitMetadata(commit).getFileIdAndFullPaths());
|
||||
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths());
|
||||
}
|
||||
|
||||
return sqlContextOpt.get().read()
|
||||
@@ -229,13 +240,13 @@ public class HoodieReadClient implements Serializable {
|
||||
*/
|
||||
public Dataset<Row> readCommit(String commitTime) {
|
||||
assertSqlContext();
|
||||
HoodieCommits commits = metadata.getAllCommits();
|
||||
if (!commits.contains(commitTime)) {
|
||||
if (!commitTimeline.containsInstant(commitTime)) {
|
||||
new HoodieException("No commit exists at " + commitTime);
|
||||
}
|
||||
|
||||
try {
|
||||
HoodieCommitMetadata commitMetdata = metadata.getCommitMetadata(commitTime);
|
||||
HoodieCommitMetadata commitMetdata =
|
||||
HoodieCommitMetadata.fromBytes(commitTimeline.readInstantDetails(commitTime).get());
|
||||
Collection<String> paths = commitMetdata.getFileIdAndFullPaths().values();
|
||||
return sqlContextOpt.get().read()
|
||||
.parquet(paths.toArray(new String[paths.size()]))
|
||||
@@ -253,7 +264,7 @@ public class HoodieReadClient implements Serializable {
|
||||
*/
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> checkExists(
|
||||
JavaRDD<HoodieKey> hoodieKeys) {
|
||||
return index.fetchRecordLocation(hoodieKeys, metadata);
|
||||
return index.fetchRecordLocation(hoodieKeys, metaClient);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -264,7 +275,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, metadata);
|
||||
JavaRDD<HoodieRecord> recordsWithLocation = index.tagLocation(hoodieRecords, metaClient);
|
||||
return recordsWithLocation.filter(new Function<HoodieRecord, Boolean>() {
|
||||
@Override
|
||||
public Boolean call(HoodieRecord v1) throws Exception {
|
||||
@@ -287,13 +298,13 @@ public class HoodieReadClient implements Serializable {
|
||||
* @return
|
||||
*/
|
||||
public List<String> listCommitsSince(String commitTimestamp) {
|
||||
return metadata.getAllCommits().findCommitsAfter(commitTimestamp, Integer.MAX_VALUE);
|
||||
return commitTimeline.findInstantsAfter(commitTimestamp, Integer.MAX_VALUE).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the last successful commit (a successful write operation) into a Hoodie table.
|
||||
*/
|
||||
public String latestCommit() {
|
||||
return metadata.getAllCommits().lastCommit();
|
||||
return commitTimeline.lastInstant().get();
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user