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:
@@ -16,10 +16,13 @@
|
||||
|
||||
package com.uber.hoodie.io;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveCommitTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieArchivedCommitTimeline;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.common.file.HoodieAppendLog;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieCommitException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -30,65 +33,70 @@ import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Log to hold older historical commits, to bound the growth of .commit files
|
||||
*/
|
||||
public class HoodieCommitArchiveLog {
|
||||
private static Logger log = LogManager.getLogger(HoodieCommitArchiveLog.class);
|
||||
private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE = "commits.archived";
|
||||
|
||||
private final Path archiveFilePath;
|
||||
private final FileSystem fs;
|
||||
private final HoodieWriteConfig config;
|
||||
|
||||
public HoodieCommitArchiveLog(HoodieWriteConfig config) {
|
||||
this.archiveFilePath =
|
||||
new Path(config.getBasePath(),
|
||||
HoodieTableMetadata.METAFOLDER_NAME + "/" +HOODIE_COMMIT_ARCHIVE_LOG_FILE);
|
||||
this.fs = FSUtils.getFs();
|
||||
public HoodieCommitArchiveLog(HoodieWriteConfig config,
|
||||
FileSystem fs) {
|
||||
this.fs = fs;
|
||||
this.config = config;
|
||||
this.archiveFilePath = HoodieArchivedCommitTimeline
|
||||
.getArchiveLogPath(config.getBasePath() + "/" + HoodieTableMetaClient.METAFOLDER_NAME);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if commits need to be archived. If yes, archive commits.
|
||||
*/
|
||||
public boolean archiveIfRequired() {
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, config.getBasePath());
|
||||
List<String> commitsToArchive = getCommitsToArchive(metadata);
|
||||
if (!commitsToArchive.isEmpty()) {
|
||||
List<String> commitsToArchive = getCommitsToArchive().collect(Collectors.toList());
|
||||
if (commitsToArchive.iterator().hasNext()) {
|
||||
log.info("Archiving commits " + commitsToArchive);
|
||||
archive(metadata, commitsToArchive);
|
||||
return deleteCommits(metadata, commitsToArchive);
|
||||
archive(commitsToArchive);
|
||||
return deleteCommits(commitsToArchive);
|
||||
} else {
|
||||
log.info("No Commits to archive");
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
private List<String> getCommitsToArchive(HoodieTableMetadata metadata) {
|
||||
private Stream<String> getCommitsToArchive() {
|
||||
int maxCommitsToKeep = config.getMaxCommitsToKeep();
|
||||
int minCommitsToKeep = config.getMinCommitsToKeep();
|
||||
|
||||
List<String> commits = metadata.getAllCommits().getCommitList();
|
||||
List<String> commitsToArchive = new ArrayList<String>();
|
||||
if (commits.size() > maxCommitsToKeep) {
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
|
||||
if (commitTimeline.hasInstants() && commitTimeline.getTotalInstants() > maxCommitsToKeep) {
|
||||
// Actually do the commits
|
||||
commitsToArchive = commits.subList(0, commits.size() - minCommitsToKeep);
|
||||
return commitTimeline.getInstants()
|
||||
.limit(commitTimeline.getTotalInstants() - minCommitsToKeep);
|
||||
}
|
||||
return commitsToArchive;
|
||||
return Stream.empty();
|
||||
}
|
||||
|
||||
private boolean deleteCommits(HoodieTableMetadata metadata, List<String> commitsToArchive) {
|
||||
private boolean deleteCommits(List<String> commitsToArchive) {
|
||||
log.info("Deleting commits " + commitsToArchive);
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
|
||||
boolean success = true;
|
||||
for(String commitToArchive:commitsToArchive) {
|
||||
Path commitFile =
|
||||
new Path(metadata.getBasePath() + "/" +
|
||||
HoodieTableMetadata.METAFOLDER_NAME + "/" +
|
||||
FSUtils.makeCommitFileName(commitToArchive));
|
||||
Path commitFile = new Path(metaClient.getMetaPath(),
|
||||
((HoodieActiveCommitTimeline) commitTimeline)
|
||||
.getCompletedFileName(commitToArchive));
|
||||
try {
|
||||
if (fs.exists(commitFile)) {
|
||||
success &= fs.delete(commitFile, false);
|
||||
@@ -112,14 +120,19 @@ public class HoodieCommitArchiveLog {
|
||||
.compression(HoodieAppendLog.CompressionType.RECORD, new BZip2Codec()));
|
||||
}
|
||||
|
||||
private void archive(HoodieTableMetadata metadata, List<String> commits)
|
||||
throws HoodieCommitException {
|
||||
private void archive(List<String> commits) throws HoodieCommitException {
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
|
||||
HoodieAppendLog.Writer writer = null;
|
||||
try {
|
||||
writer = openWriter();
|
||||
for (String commitTime : commits) {
|
||||
Text k = new Text(commitTime);
|
||||
Text v = new Text(metadata.getCommitMetadata(commitTime).toJsonString());
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.readInstantDetails(commitTime).get());
|
||||
Text v = new Text(commitMetadata.toJsonString());
|
||||
writer.append(k, v);
|
||||
log.info("Wrote " + k);
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user