1
0

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:
Prasanna Rajaperumal
2017-01-04 14:40:26 -08:00
parent 283269e57f
commit 8ee777a9bb
76 changed files with 3480 additions and 1932 deletions

View File

@@ -16,9 +16,12 @@
package com.uber.hoodie.io;
import com.uber.hoodie.common.model.HoodieDataFile;
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.config.HoodieWriteConfig;
import com.uber.hoodie.common.model.HoodieCommits;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.util.FSUtils;
import org.apache.hadoop.fs.FileStatus;
@@ -31,7 +34,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
* Cleaner is responsible for garbage collecting older files in a given partition path, such that
@@ -45,26 +48,25 @@ import java.util.Map;
*
*/
public class HoodieCleaner {
private static Logger logger = LogManager.getLogger(HoodieCleaner.class);
public enum CleaningPolicy {
KEEP_LATEST_FILE_VERSIONS,
KEEP_LATEST_COMMITS
}
private static Logger logger = LogManager.getLogger(HoodieCleaner.class);
private HoodieTableMetadata metadata;
private final TableFileSystemView fileSystemView;
private final HoodieTimeline commitTimeline;
private HoodieTableMetaClient metaClient;
private HoodieWriteConfig config;
private FileSystem fs;
public HoodieCleaner(HoodieTableMetadata metadata,
public HoodieCleaner(HoodieTableMetaClient metaClient,
HoodieWriteConfig config,
FileSystem fs) {
this.metadata = metadata;
this.metaClient = metaClient;
this.fileSystemView = new ReadOptimizedTableView(fs, metaClient);
this.commitTimeline = metaClient.getActiveCommitTimeline();
this.config = config;
this.fs = fs;
}
@@ -83,13 +85,13 @@ public class HoodieCleaner {
*/
private List<String> getFilesToCleanKeepingLatestVersions(String partitionPath) throws IOException {
logger.info("Cleaning "+ partitionPath+", retaining latest "+ config.getCleanerFileVersionsRetained()+" file versions. ");
Map<String, List<FileStatus>> fileVersions = metadata.getAllVersionsInPartition(fs, partitionPath);
List<List<HoodieDataFile>> fileVersions = fileSystemView.streamEveryVersionInPartition(partitionPath).collect(
Collectors.toList());
List<String> deletePaths = new ArrayList<>();
for (String file : fileVersions.keySet()) {
List<FileStatus> commitList = fileVersions.get(file);
for (List<HoodieDataFile> versionsForFileId : fileVersions) {
int keepVersions = config.getCleanerFileVersionsRetained();
Iterator<FileStatus> commitItr = commitList.iterator();
Iterator<HoodieDataFile> commitItr = versionsForFileId.iterator();
while (commitItr.hasNext() && keepVersions > 0) {
// Skip this most recent version
commitItr.next();
@@ -100,7 +102,7 @@ public class HoodieCleaner {
deletePaths.add(String.format("%s/%s/%s",
config.getBasePath(),
partitionPath,
commitItr.next().getPath().getName()));
commitItr.next().getFileName()));
}
}
return deletePaths;
@@ -133,22 +135,20 @@ public class HoodieCleaner {
List<String> deletePaths = new ArrayList<>();
// determine if we have enough commits, to start cleaning.
HoodieCommits commits = metadata.getAllCommits();
if (commits.getNumCommits() > commitsRetained) {
if (commitTimeline.getTotalInstants() > commitsRetained) {
String earliestCommitToRetain =
commits.nthCommit(commits.getNumCommits() - commitsRetained);
Map<String, List<FileStatus>> fileVersions =
metadata.getAllVersionsInPartition(fs, partitionPath);
for (String file : fileVersions.keySet()) {
List<FileStatus> fileList = fileVersions.get(file);
String lastVersion = FSUtils.getCommitTime(fileList.get(0).getPath().getName());
commitTimeline.nthInstant(commitTimeline.getTotalInstants() - commitsRetained).get();
List<List<HoodieDataFile>> fileVersions =
fileSystemView.streamEveryVersionInPartition(partitionPath).collect(Collectors.toList());
for (List<HoodieDataFile> fileList : fileVersions) {
String lastVersion = FSUtils.getCommitTime(fileList.get(0).getFileName());
String lastVersionBeforeEarliestCommitToRetain =
getLatestVersionBeforeCommit(fileList, earliestCommitToRetain);
// Ensure there are more than 1 version of the file (we only clean old files from updates)
// i.e always spare the last commit.
for (FileStatus afile : fileList) {
String fileCommitTime = FSUtils.getCommitTime(afile.getPath().getName());
for (HoodieDataFile afile : fileList) {
String fileCommitTime = afile.getCommitTime();
// Dont delete the latest commit and also the last commit before the earliest commit we are retaining
// The window of commit retain == max query run time. So a query could be running which still
// uses this file.
@@ -160,11 +160,12 @@ public class HoodieCleaner {
}
// Always keep the last commit
if (HoodieCommits.isCommit1After(earliestCommitToRetain, fileCommitTime)) {
if (commitTimeline.compareInstants(earliestCommitToRetain, fileCommitTime,
HoodieTimeline.GREATER)) {
// this is a commit, that should be cleaned.
deletePaths.add(String
.format("%s/%s/%s", config.getBasePath(), partitionPath,
FSUtils.maskWithoutTaskPartitionId(fileCommitTime, file)));
.format("%s/%s/%s", config.getBasePath(), partitionPath, FSUtils
.maskWithoutTaskPartitionId(fileCommitTime, afile.getFileId())));
}
}
}
@@ -176,10 +177,10 @@ public class HoodieCleaner {
/**
* Gets the latest version < commitTime. This version file could still be used by queries.
*/
private String getLatestVersionBeforeCommit(List<FileStatus> fileList, String commitTime) {
for (FileStatus file : fileList) {
String fileCommitTime = FSUtils.getCommitTime(file.getPath().getName());
if (HoodieCommits.isCommit1After(commitTime, fileCommitTime)) {
private String getLatestVersionBeforeCommit(List<HoodieDataFile> fileList, String commitTime) {
for (HoodieDataFile file : fileList) {
String fileCommitTime = FSUtils.getCommitTime(file.getFileName());
if (commitTimeline.compareInstants(commitTime, fileCommitTime, HoodieTimeline.GREATER)) {
// fileList is sorted on the reverse, so the first commit we find <= commitTime is the one we want
return fileCommitTime;
}

View File

@@ -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);
}

View File

@@ -16,9 +16,12 @@
package com.uber.hoodie.io;
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.config.HoodieWriteConfig;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.exception.HoodieIOException;
@@ -36,15 +39,19 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
protected final String commitTime;
protected final HoodieWriteConfig config;
protected final FileSystem fs;
protected final HoodieTableMetadata metadata;
protected final HoodieTableMetaClient metaClient;
protected final HoodieTimeline hoodieTimeline;
protected final TableFileSystemView fileSystemView;
protected final Schema schema;
public HoodieIOHandle(HoodieWriteConfig config, String commitTime,
HoodieTableMetadata metadata) {
HoodieTableMetaClient metaClient) {
this.commitTime = commitTime;
this.config = config;
this.fs = FSUtils.getFs();
this.metadata = metadata;
this.metaClient = metaClient;
this.hoodieTimeline = metaClient.getActiveCommitTimeline();
this.fileSystemView = new ReadOptimizedTableView(fs, metaClient);
this.schema =
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
}

View File

@@ -16,12 +16,12 @@
package com.uber.hoodie.io;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieInsertException;
@@ -45,7 +45,7 @@ public class HoodieInsertHandle<T extends HoodieRecordPayload> extends HoodieIOH
private int recordsWritten = 0;
public HoodieInsertHandle(HoodieWriteConfig config, String commitTime,
HoodieTableMetadata metadata, String partitionPath) {
HoodieTableMetaClient metadata, String partitionPath) {
super(config, commitTime, metadata);
this.status = new WriteStatus();
status.setFileId(UUID.randomUUID().toString());

View File

@@ -16,12 +16,12 @@
package com.uber.hoodie.io;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieUpsertException;
@@ -52,10 +52,10 @@ import java.util.Iterator;
public HoodieUpdateHandle(HoodieWriteConfig config,
String commitTime,
HoodieTableMetadata metadata,
HoodieTableMetaClient metaClient,
Iterator<HoodieRecord<T>> recordItr,
String fileId) {
super(config, commitTime, metadata);
super(config, commitTime, metaClient);
WriteStatus writeStatus = new WriteStatus();
writeStatus.setStat(new HoodieWriteStat());
this.writeStatus = writeStatus;
@@ -74,7 +74,9 @@ import java.util.Iterator;
HoodieRecord<T> record = newRecordsItr.next();
// If the first record, we need to extract some info out
if (oldFilePath == null) {
String latestValidFilePath = metadata.getFilenameForRecord(fs, record, fileId);
String latestValidFilePath = fileSystemView
.getLatestDataFilesForFileId(record.getPartitionPath(), fileId).findFirst()
.get().getFileName();
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
oldFilePath = new Path(
config.getBasePath() + "/" + record.getPartitionPath() + "/"
@@ -102,14 +104,14 @@ import java.util.Iterator;
}
// Create the writer for writing the new version file
storageWriter = HoodieStorageWriterFactory
.getStorageWriter(commitTime, newFilePath, metadata, config, schema);
.getStorageWriter(commitTime, newFilePath, metaClient, config, schema);
} catch (Exception e) {
logger.error("Error in update task at commit " + commitTime, e);
writeStatus.setGlobalError(e);
throw new HoodieUpsertException(
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit "
+ commitTime + " on HDFS path " + metadata.getBasePath());
+ commitTime + " on HDFS path " + metaClient.getBasePath());
}
}

View File

@@ -16,11 +16,11 @@
package com.uber.hoodie.io.storage;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
import com.uber.hoodie.common.BloomFilter;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.util.FSUtils;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
@@ -32,7 +32,7 @@ import java.io.IOException;
public class HoodieStorageWriterFactory {
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> getStorageWriter(
String commitTime, Path path, HoodieTableMetadata metadata, HoodieWriteConfig config, Schema schema)
String commitTime, Path path, HoodieTableMetaClient metaClient, HoodieWriteConfig config, Schema schema)
throws IOException {
//TODO - based on the metadata choose the implementation of HoodieStorageWriter
// Currently only parquet is supported