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

@@ -20,6 +20,7 @@ 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.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.common.util.FSUtils;
@@ -38,55 +39,56 @@ import java.util.stream.Collectors;
/**
* Cleaner is responsible for garbage collecting older files in a given partition path, such that
*
* <p>
* 1) It provides sufficient time for existing queries running on older versions, to finish
*
* <p>
* 2) It bounds the growth of the files in the file system
*
* <p>
* TODO: Should all cleaning be done based on {@link com.uber.hoodie.common.model.HoodieCommitMetadata}
*
*
*/
public class HoodieCleaner {
private static Logger logger = LogManager.getLogger(HoodieCleaner.class);
public enum CleaningPolicy {
KEEP_LATEST_FILE_VERSIONS,
KEEP_LATEST_COMMITS
}
private final TableFileSystemView fileSystemView;
private final HoodieTimeline commitTimeline;
private HoodieTableMetaClient metaClient;
private HoodieWriteConfig config;
private FileSystem fs;
public HoodieCleaner(HoodieTableMetaClient metaClient,
HoodieWriteConfig config,
FileSystem fs) {
public HoodieCleaner(HoodieTableMetaClient metaClient, HoodieWriteConfig config,
FileSystem fs) {
this.metaClient = metaClient;
this.fileSystemView = new ReadOptimizedTableView(fs, metaClient);
this.commitTimeline = metaClient.getActiveCommitTimeline();
this.commitTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
this.config = config;
this.fs = fs;
}
/**
*
* Selects the older versions of files for cleaning, such that it bounds the number of versions of each file.
* This policy is useful, if you are simply interested in querying the table, and you don't want too many
* versions for a single file (i.e run it with versionsRetained = 1)
*
*
* @param partitionPath
* @return
* @throws IOException
*/
private List<String> getFilesToCleanKeepingLatestVersions(String partitionPath) throws IOException {
logger.info("Cleaning "+ partitionPath+", retaining latest "+ config.getCleanerFileVersionsRetained()+" file versions. ");
List<List<HoodieDataFile>> fileVersions = fileSystemView.streamEveryVersionInPartition(partitionPath).collect(
Collectors.toList());
private List<String> getFilesToCleanKeepingLatestVersions(String partitionPath)
throws IOException {
logger.info("Cleaning " + partitionPath + ", retaining latest " + config
.getCleanerFileVersionsRetained() + " file versions. ");
List<List<HoodieDataFile>> fileVersions =
fileSystemView.getEveryVersionInPartition(partitionPath)
.collect(Collectors.toList());
List<String> deletePaths = new ArrayList<>();
for (List<HoodieDataFile> versionsForFileId : fileVersions) {
@@ -99,10 +101,8 @@ public class HoodieCleaner {
}
// Delete the remaining files
while (commitItr.hasNext()) {
deletePaths.add(String.format("%s/%s/%s",
config.getBasePath(),
partitionPath,
commitItr.next().getFileName()));
deletePaths.add(String.format("%s/%s/%s", config.getBasePath(), partitionPath,
commitItr.next().getFileName()));
}
}
return deletePaths;
@@ -111,17 +111,17 @@ public class HoodieCleaner {
/**
* Selects the versions for file for cleaning, such that it
*
* - Leaves the latest version of the file untouched
* - For older versions,
* - It leaves all the commits untouched which has occured in last <code>config.getCleanerCommitsRetained()</code> commits
* - It leaves ONE commit before this window. We assume that the max(query execution time) == commit_batch_time * config.getCleanerCommitsRetained(). This is 12 hours by default.
* This is essential to leave the file used by the query thats running for the max time.
*
* This provides the effect of having lookback into all changes that happened in the last X
* commits. (eg: if you retain 24 commits, and commit batch time is 30 mins, then you have 12 hrs of lookback)
*
* This policy is the default.
* <p>
* - Leaves the latest version of the file untouched
* - For older versions,
* - It leaves all the commits untouched which has occured in last <code>config.getCleanerCommitsRetained()</code> commits
* - It leaves ONE commit before this window. We assume that the max(query execution time) == commit_batch_time * config.getCleanerCommitsRetained(). This is 12 hours by default.
* This is essential to leave the file used by the query thats running for the max time.
* <p>
* This provides the effect of having lookback into all changes that happened in the last X
* commits. (eg: if you retain 24 commits, and commit batch time is 30 mins, then you have 12 hrs of lookback)
* <p>
* This policy is the default.
*
* @param partitionPath
* @return
@@ -135,11 +135,12 @@ public class HoodieCleaner {
List<String> deletePaths = new ArrayList<>();
// determine if we have enough commits, to start cleaning.
if (commitTimeline.getTotalInstants() > commitsRetained) {
String earliestCommitToRetain =
commitTimeline.nthInstant(commitTimeline.getTotalInstants() - commitsRetained).get();
if (commitTimeline.countInstants() > commitsRetained) {
HoodieInstant earliestCommitToRetain =
commitTimeline.nthInstant(commitTimeline.countInstants() - commitsRetained).get();
List<List<HoodieDataFile>> fileVersions =
fileSystemView.streamEveryVersionInPartition(partitionPath).collect(Collectors.toList());
fileSystemView.getEveryVersionInPartition(partitionPath)
.collect(Collectors.toList());
for (List<HoodieDataFile> fileList : fileVersions) {
String lastVersion = FSUtils.getCommitTime(fileList.get(0).getFileName());
String lastVersionBeforeEarliestCommitToRetain =
@@ -160,8 +161,9 @@ public class HoodieCleaner {
}
// Always keep the last commit
if (commitTimeline.compareInstants(earliestCommitToRetain, fileCommitTime,
HoodieTimeline.GREATER)) {
if (commitTimeline
.compareTimestamps(earliestCommitToRetain.getTimestamp(), fileCommitTime,
HoodieTimeline.GREATER)) {
// this is a commit, that should be cleaned.
deletePaths.add(String
.format("%s/%s/%s", config.getBasePath(), partitionPath, FSUtils
@@ -177,10 +179,12 @@ public class HoodieCleaner {
/**
* Gets the latest version < commitTime. This version file could still be used by queries.
*/
private String getLatestVersionBeforeCommit(List<HoodieDataFile> fileList, String commitTime) {
private String getLatestVersionBeforeCommit(List<HoodieDataFile> fileList,
HoodieInstant commitTime) {
for (HoodieDataFile file : fileList) {
String fileCommitTime = FSUtils.getCommitTime(file.getFileName());
if (commitTimeline.compareInstants(commitTime, fileCommitTime, HoodieTimeline.GREATER)) {
if (commitTimeline.compareTimestamps(commitTime.getTimestamp(), 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

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

View File

@@ -50,7 +50,8 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
this.config = config;
this.fs = FSUtils.getFs();
this.metaClient = metaClient;
this.hoodieTimeline = metaClient.getActiveCommitTimeline();
this.hoodieTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
this.fileSystemView = new ReadOptimizedTableView(fs, metaClient);
this.schema =
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));