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:
@@ -22,10 +22,10 @@ import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import com.uber.hoodie.HoodieWriteClient;
|
||||
import com.uber.hoodie.common.HoodieJsonPayload;
|
||||
import com.uber.hoodie.common.model.HoodieCommits;
|
||||
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.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
@@ -65,22 +65,23 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
private void sync() throws Exception {
|
||||
JavaSparkContext sc = getSparkContext(cfg);
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
HoodieTableMetadata targetHoodieMetadata =
|
||||
new HoodieTableMetadata(fs, cfg.targetPath, cfg.targetTableName);
|
||||
HoodieTableMetaClient targetHoodieMetadata = new HoodieTableMetaClient(fs, cfg.targetPath);
|
||||
HoodieTimeline timeline = targetHoodieMetadata.getActiveCommitTimeline();
|
||||
String lastCommitPulled = findLastCommitPulled(fs, cfg.dataPath);
|
||||
log.info("Last commit pulled on the source dataset is " + lastCommitPulled);
|
||||
if (!targetHoodieMetadata.getAllCommits().isEmpty() && HoodieCommits
|
||||
.isCommit1After(targetHoodieMetadata.getAllCommits().lastCommit(), lastCommitPulled)) {
|
||||
if (!timeline.getInstants().iterator().hasNext() && timeline
|
||||
.compareInstants(timeline.lastInstant().get(), lastCommitPulled,
|
||||
HoodieTimeline.GREATER)) {
|
||||
// this should never be the case
|
||||
throw new IllegalStateException(
|
||||
"Last commit pulled from source table " + lastCommitPulled
|
||||
+ " is before the last commit in the target table " + targetHoodieMetadata
|
||||
.getAllCommits().lastCommit());
|
||||
"Last commit pulled from source table " + lastCommitPulled
|
||||
+ " is before the last commit in the target table " + timeline.lastInstant()
|
||||
.get());
|
||||
}
|
||||
if (!cfg.override && targetHoodieMetadata.getAllCommits().contains(lastCommitPulled)) {
|
||||
if (!cfg.override && timeline.containsOrBeforeTimelineStarts(lastCommitPulled)) {
|
||||
throw new IllegalStateException(
|
||||
"Target Table already has the commit " + lastCommitPulled
|
||||
+ ". Not overriding as cfg.override is false");
|
||||
"Target Table already has the commit " + lastCommitPulled
|
||||
+ ". Not overriding as cfg.override is false");
|
||||
}
|
||||
syncTill(lastCommitPulled, targetHoodieMetadata, sc);
|
||||
}
|
||||
@@ -98,7 +99,7 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
return commitTimes.get(0);
|
||||
}
|
||||
|
||||
private void syncTill(String lastCommitPulled, HoodieTableMetadata target,
|
||||
private void syncTill(String lastCommitPulled, HoodieTableMetaClient target,
|
||||
JavaSparkContext sc) throws Exception {
|
||||
// Step 1 : Scan incrementally and get the input records as a RDD of source format
|
||||
String dataPath = cfg.dataPath + "/" + lastCommitPulled;
|
||||
@@ -159,13 +160,13 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
// })
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getHoodieClientConfig(HoodieTableMetadata metadata)
|
||||
private HoodieWriteConfig getHoodieClientConfig(HoodieTableMetaClient metadata)
|
||||
throws Exception {
|
||||
final String schemaStr = Files.toString(new File(cfg.schemaFile), Charset.forName("UTF-8"));
|
||||
return HoodieWriteConfig.newBuilder().withPath(metadata.getBasePath())
|
||||
.withSchema(schemaStr)
|
||||
.withParallelism(cfg.groupByParallelism, cfg.groupByParallelism)
|
||||
.forTable(metadata.getTableName()).withIndexConfig(
|
||||
.forTable(metadata.getTableConfig().getTableName()).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.build();
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user