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,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;
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user