1
0

Take 2: Refactor hoodie-common and create right abstractions for Hoodie Storage V2.0

- Refactored timelines to be a single timeline for all active events and one for archived events. CommitTimeline and other timelines can be inferred by applying a filter on the activeTimelime
- Introduced HoodieInstant to abstract different types of action, commit time and if isInFlight
- Implemented other review comments
This commit is contained in:
Prasanna Rajaperumal
2017-01-18 01:00:36 -08:00
parent 8ee777a9bb
commit ccd8cb2407
46 changed files with 1194 additions and 1106 deletions

View File

@@ -19,8 +19,8 @@ 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.common.table.timeline.HoodieArchivedTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.common.file.HoodieAppendLog;
import com.uber.hoodie.exception.HoodieCommitException;
@@ -34,6 +34,7 @@ import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -47,11 +48,10 @@ public class HoodieCommitArchiveLog {
private final FileSystem fs;
private final HoodieWriteConfig config;
public HoodieCommitArchiveLog(HoodieWriteConfig config,
FileSystem fs) {
public HoodieCommitArchiveLog(HoodieWriteConfig config, FileSystem fs) {
this.fs = fs;
this.config = config;
this.archiveFilePath = HoodieArchivedCommitTimeline
this.archiveFilePath = HoodieArchivedTimeline
.getArchiveLogPath(config.getBasePath() + "/" + HoodieTableMetaClient.METAFOLDER_NAME);
}
@@ -59,7 +59,7 @@ public class HoodieCommitArchiveLog {
* Check if commits need to be archived. If yes, archive commits.
*/
public boolean archiveIfRequired() {
List<String> commitsToArchive = getCommitsToArchive().collect(Collectors.toList());
List<HoodieInstant> commitsToArchive = getCommitsToArchive().collect(Collectors.toList());
if (commitsToArchive.iterator().hasNext()) {
log.info("Archiving commits " + commitsToArchive);
archive(commitsToArchive);
@@ -70,41 +70,42 @@ public class HoodieCommitArchiveLog {
}
}
private Stream<String> getCommitsToArchive() {
private Stream<HoodieInstant> getCommitsToArchive() {
int maxCommitsToKeep = config.getMaxCommitsToKeep();
int minCommitsToKeep = config.getMinCommitsToKeep();
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
HoodieTimeline commitTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
if (commitTimeline.hasInstants() && commitTimeline.getTotalInstants() > maxCommitsToKeep) {
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
// Actually do the commits
return commitTimeline.getInstants()
.limit(commitTimeline.getTotalInstants() - minCommitsToKeep);
.limit(commitTimeline.countInstants() - minCommitsToKeep);
}
return Stream.empty();
}
private boolean deleteCommits(List<String> commitsToArchive) {
private boolean deleteCommits(List<HoodieInstant> commitsToArchive) {
log.info("Deleting commits " + commitsToArchive);
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
HoodieTimeline commitTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
boolean success = true;
for(String commitToArchive:commitsToArchive) {
Path commitFile = new Path(metaClient.getMetaPath(),
((HoodieActiveCommitTimeline) commitTimeline)
.getCompletedFileName(commitToArchive));
for (HoodieInstant commitToArchive : commitsToArchive) {
Path commitFile =
new Path(metaClient.getMetaPath(), commitToArchive.getFileName());
try {
if (fs.exists(commitFile)) {
success &= fs.delete(commitFile, false);
log.info("Archived and deleted commit file " + commitFile);
}
} catch (IOException e) {
throw new HoodieIOException(
"Failed to delete archived commit " + commitToArchive, e);
throw new HoodieIOException("Failed to delete archived commit " + commitToArchive,
e);
}
}
return success;
@@ -120,18 +121,19 @@ public class HoodieCommitArchiveLog {
.compression(HoodieAppendLog.CompressionType.RECORD, new BZip2Codec()));
}
private void archive(List<String> commits) throws HoodieCommitException {
private void archive(List<HoodieInstant> commits) throws HoodieCommitException {
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
HoodieTimeline commitTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
HoodieAppendLog.Writer writer = null;
try {
writer = openWriter();
for (String commitTime : commits) {
Text k = new Text(commitTime);
for (HoodieInstant commitTime : commits) {
Text k = new Text(commitTime.getTimestamp());
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(commitTimeline.readInstantDetails(commitTime).get());
.fromBytes(commitTimeline.getInstantDetails(commitTime).get());
Text v = new Text(commitMetadata.toJsonString());
writer.append(k, v);
log.info("Wrote " + k);