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:
@@ -18,7 +18,7 @@ package com.uber.hoodie.utilities;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.utilities.exception.HoodieIncrementalPullException;
|
||||
import com.uber.hoodie.utilities.exception.HoodieIncrementalPullSQLException;
|
||||
@@ -44,7 +44,9 @@ import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Scanner;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Utility to pull data after a given commit, based on the supplied HiveQL and save the delta as another hive temporary table.
|
||||
@@ -263,9 +265,10 @@ public class HiveIncrementalPuller {
|
||||
if(!fs.exists(new Path(targetDataPath)) || !fs.exists(new Path(targetDataPath + "/.hoodie"))) {
|
||||
return "0";
|
||||
}
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, targetDataPath);
|
||||
String lastCommit = metadata.getAllCommits().lastCommit();
|
||||
return lastCommit == null ? "0" : lastCommit;
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, targetDataPath);
|
||||
|
||||
Optional<String> lastCommit = metadata.getActiveCommitTimeline().lastInstant();
|
||||
return lastCommit.orElse("0");
|
||||
}
|
||||
|
||||
private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime)
|
||||
@@ -295,12 +298,14 @@ public class HiveIncrementalPuller {
|
||||
}
|
||||
|
||||
private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation) throws IOException {
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, sourceTableLocation);
|
||||
List<String> commitsToSync =
|
||||
metadata.getAllCommits().findCommitsAfter(config.fromCommitTime, config.maxCommits);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, sourceTableLocation);
|
||||
List<String> commitsToSync = metadata.getActiveCommitTimeline()
|
||||
.findInstantsAfter(config.fromCommitTime, config.maxCommits)
|
||||
.collect(Collectors.toList());
|
||||
if (commitsToSync.isEmpty()) {
|
||||
log.warn("Nothing to sync. All commits in " + config.sourceTable + " are " + metadata
|
||||
.getAllCommits().getCommitList() + " and from commit time is " + config.fromCommitTime);
|
||||
.getActiveCommitTimeline().getInstants().collect(Collectors.toList())
|
||||
+ " and from commit time is " + config.fromCommitTime);
|
||||
return null;
|
||||
}
|
||||
log.info("Syncing commits " + commitsToSync);
|
||||
|
||||
@@ -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();
|
||||
}
|
||||
|
||||
@@ -19,8 +19,12 @@ package com.uber.hoodie.utilities;
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieCommits;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.table.HoodieTableConfig;
|
||||
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 org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -40,6 +44,8 @@ import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Hoodie snapshot copy job which copies latest files from all partitions to another place, for snapshot backup.
|
||||
@@ -57,11 +63,15 @@ public class HoodieSnapshotCopier implements Serializable {
|
||||
|
||||
public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDir) throws IOException {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
final HoodieTableMetadata tableMetadata = new HoodieTableMetadata(fs, baseDir);
|
||||
|
||||
final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs, baseDir);
|
||||
final TableFileSystemView fsView = new ReadOptimizedTableView(fs, tableMetadata);
|
||||
// Get the latest commit
|
||||
final String latestCommit = tableMetadata.getAllCommits().lastCommit();
|
||||
logger.info(String.format("Starting to snapshot latest version files which are also no-late-than %s.", latestCommit));
|
||||
final Optional<String> latestCommit = tableMetadata.getActiveCommitTimeline().lastInstant();
|
||||
if(!latestCommit.isPresent()) {
|
||||
logger.warn("No commits present. Nothing to snapshot");
|
||||
} else {
|
||||
logger.info(String.format("Starting to snapshot latest version files which are also no-late-than %s.", latestCommit.get()));
|
||||
}
|
||||
|
||||
List<String> partitions = FSUtils.getAllPartitionPaths(fs, baseDir);
|
||||
if (partitions.size() > 0) {
|
||||
@@ -80,8 +90,10 @@ public class HoodieSnapshotCopier implements Serializable {
|
||||
// Only take latest version files <= latestCommit.
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
List<Tuple2<String, String>> filePaths = new ArrayList<>();
|
||||
for (FileStatus fileStatus : tableMetadata.getLatestVersionInPartition(fs, partition, latestCommit)) {
|
||||
filePaths.add(new Tuple2<>(partition, fileStatus.getPath().toString()));
|
||||
for (HoodieDataFile hoodieDataFile : fsView
|
||||
.streamLatestVersionInPartition(partition, latestCommit.get())
|
||||
.collect(Collectors.toList())) {
|
||||
filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath()));
|
||||
}
|
||||
return filePaths.iterator();
|
||||
}
|
||||
@@ -102,22 +114,25 @@ public class HoodieSnapshotCopier implements Serializable {
|
||||
});
|
||||
|
||||
// Also copy the .commit files
|
||||
logger.info(String.format("Copying .commit files which are no-late-than %s.", latestCommit));
|
||||
logger.info(String.format("Copying .commit files which are no-late-than %s.", latestCommit.get()));
|
||||
FileStatus[] commitFilesToCopy = fs.listStatus(
|
||||
new Path(baseDir + "/" + HoodieTableMetadata.METAFOLDER_NAME), new PathFilter() {
|
||||
new Path(baseDir + "/" + HoodieTableMetaClient.METAFOLDER_NAME), new PathFilter() {
|
||||
@Override
|
||||
public boolean accept(Path commitFilePath) {
|
||||
if (commitFilePath.getName().equals(HoodieTableMetadata.HOODIE_PROPERTIES_FILE)) {
|
||||
if (commitFilePath.getName().equals(HoodieTableConfig.HOODIE_PROPERTIES_FILE)) {
|
||||
return true;
|
||||
} else {
|
||||
String commitTime = FSUtils.getCommitFromCommitFile(commitFilePath.getName());
|
||||
return HoodieCommits.isCommit1BeforeOrOn(commitTime, latestCommit);
|
||||
String commitTime =
|
||||
FSUtils.getCommitFromCommitFile(commitFilePath.getName());
|
||||
return tableMetadata.getActiveCommitTimeline()
|
||||
.compareInstants(commitTime, latestCommit.get(), HoodieTimeline.GREATER);
|
||||
}
|
||||
}
|
||||
});
|
||||
for (FileStatus commitStatus : commitFilesToCopy) {
|
||||
Path targetFilePath =
|
||||
new Path(outputDir + "/" + HoodieTableMetadata.METAFOLDER_NAME + "/" + commitStatus.getPath().getName());
|
||||
Path targetFilePath = new Path(
|
||||
outputDir + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitStatus
|
||||
.getPath().getName());
|
||||
if (! fs.exists(targetFilePath.getParent())) {
|
||||
fs.mkdirs(targetFilePath.getParent());
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user