1
0

Implement Savepoints and required metadata timeline (#86)

- Introduce avro to save clean metadata with details about the last commit that was retained
- Save rollback metadata in the meta timeline
- Create savepoint metadata and add API to createSavepoint, deleteSavepoint and rollbackToSavepoint
- Savepointed commit should not be rolledback or cleaned or archived
- introduce cli commands to show, create and rollback to savepoints
- Write unit tests to test savepoints and rollbackToSavepoints
This commit is contained in:
prazanna
2017-03-13 15:12:03 -07:00
committed by GitHub
parent 69d3950a32
commit 6f36e1eaaf
27 changed files with 1423 additions and 130 deletions

View File

@@ -16,6 +16,10 @@
package com.uber.hoodie.io;
import com.clearspring.analytics.util.Lists;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.HoodieCleanStat;
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
@@ -33,6 +37,8 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
/**
@@ -47,13 +53,6 @@ import java.util.stream.Collectors;
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 HoodieTable hoodieTable;
@@ -86,13 +85,18 @@ public class HoodieCleaner {
fileSystemView.getEveryVersionInPartition(partitionPath)
.collect(Collectors.toList());
List<String> deletePaths = new ArrayList<>();
List<String> savepoints = hoodieTable.getSavepoints();
for (List<HoodieDataFile> versionsForFileId : fileVersions) {
int keepVersions = config.getCleanerFileVersionsRetained();
Iterator<HoodieDataFile> commitItr = versionsForFileId.iterator();
while (commitItr.hasNext() && keepVersions > 0) {
// Skip this most recent version
commitItr.next();
HoodieDataFile next = commitItr.next();
if(savepoints.contains(next.getCommitTime())) {
// do not clean datafiles that are savepointed
continue;
}
keepVersions--;
}
// Delete the remaining files
@@ -130,6 +134,8 @@ public class HoodieCleaner {
"Cleaning " + partitionPath + ", retaining latest " + commitsRetained + " commits. ");
List<String> deletePaths = new ArrayList<>();
List<String> savepoints = hoodieTable.getSavepoints();
// determine if we have enough commits, to start cleaning.
if (commitTimeline.countInstants() > commitsRetained) {
HoodieInstant earliestCommitToRetain =
@@ -146,6 +152,10 @@ public class HoodieCleaner {
// i.e always spare the last commit.
for (HoodieDataFile afile : fileList) {
String fileCommitTime = afile.getCommitTime();
if(savepoints.contains(fileCommitTime)) {
// do not clean up a savepoint data file
continue;
}
// 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.
@@ -196,30 +206,42 @@ public class HoodieCleaner {
*
* @throws IllegalArgumentException if unknown cleaning policy is provided
*/
public int clean(String partitionPath) throws IOException {
CleaningPolicy policy = config.getCleanerPolicy();
public HoodieCleanStat clean(String partitionPath) throws IOException {
HoodieCleaningPolicy policy = config.getCleanerPolicy();
List<String> deletePaths;
if (policy == CleaningPolicy.KEEP_LATEST_COMMITS) {
Optional<HoodieInstant> earliestCommitToRetain = Optional.empty();
if (policy == HoodieCleaningPolicy.KEEP_LATEST_COMMITS) {
deletePaths = getFilesToCleanKeepingLatestCommits(partitionPath);
} else if (policy == CleaningPolicy.KEEP_LATEST_FILE_VERSIONS) {
int commitsRetained = config.getCleanerCommitsRetained();
if (commitTimeline.countInstants() > commitsRetained) {
earliestCommitToRetain =
commitTimeline.nthInstant(commitTimeline.countInstants() - commitsRetained);
}
} else if (policy == HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) {
deletePaths = getFilesToCleanKeepingLatestVersions(partitionPath);
} else {
throw new IllegalArgumentException("Unknown cleaning policy : " + policy.name());
}
// perform the actual deletes
Map<FileStatus, Boolean> deletedFiles = Maps.newHashMap();
for (String deletePath : deletePaths) {
logger.info("Working on delete path :" + deletePath);
FileStatus[] deleteVersions = fs.globStatus(new Path(deletePath));
if (deleteVersions != null) {
for (FileStatus deleteVersion : deleteVersions) {
if (fs.delete(deleteVersion.getPath(), false)) {
logger.info("Cleaning file at path :" + deleteVersion.getPath());
boolean deleteResult = fs.delete(deleteVersion.getPath(), false);
deletedFiles.put(deleteVersion, deleteResult);
if (deleteResult) {
logger.info("Cleaned file at path :" + deleteVersion.getPath());
}
}
}
}
logger.info(deletePaths.size() + " files deleted for partition path:" + partitionPath);
return deletePaths.size();
logger.info(deletePaths.size() + " patterns used to delete in partition path:" + partitionPath);
return HoodieCleanStat.newBuilder().withPolicy(policy).withDeletePathPattern(deletePaths)
.withPartitionPath(partitionPath).withEarliestCommitRetained(earliestCommitToRetain)
.withDeletedFileResults(deletedFiles).build();
}
}