Parallelize file version deletes during clean and related tests
This commit is contained in:
committed by
prazanna
parent
dda28c0b4b
commit
521555c576
@@ -16,9 +16,6 @@
|
||||
|
||||
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.model.HoodieRecordPayload;
|
||||
@@ -30,22 +27,16 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
|
||||
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.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Cleaner is responsible for garbage collecting older files in a given partition path, such that
|
||||
@@ -56,8 +47,8 @@ import java.util.stream.Stream;
|
||||
* <p>
|
||||
* TODO: Should all cleaning be done based on {@link com.uber.hoodie.common.model.HoodieCommitMetadata}
|
||||
*/
|
||||
public class HoodieCleaner<T extends HoodieRecordPayload<T>> {
|
||||
private static Logger logger = LogManager.getLogger(HoodieCleaner.class);
|
||||
public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
private static Logger logger = LogManager.getLogger(HoodieCleanHelper.class);
|
||||
|
||||
private final TableFileSystemView fileSystemView;
|
||||
private final HoodieTimeline commitTimeline;
|
||||
@@ -65,7 +56,7 @@ public class HoodieCleaner<T extends HoodieRecordPayload<T>> {
|
||||
private HoodieWriteConfig config;
|
||||
private FileSystem fs;
|
||||
|
||||
public HoodieCleaner(HoodieTable<T> hoodieTable, HoodieWriteConfig config) {
|
||||
public HoodieCleanHelper(HoodieTable<T> hoodieTable, HoodieWriteConfig config) {
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.fileSystemView = hoodieTable.getCompactedFileSystemView();
|
||||
this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||
@@ -110,11 +101,11 @@ public class HoodieCleaner<T extends HoodieRecordPayload<T>> {
|
||||
// Delete the remaining files
|
||||
while (commitItr.hasNext()) {
|
||||
HoodieDataFile nextRecord = commitItr.next();
|
||||
deletePaths.add(String.format("%s/%s/%s", config.getBasePath(), partitionPath,
|
||||
nextRecord.getFileName()));
|
||||
deletePaths.add(nextRecord.getFileStatus().getPath().toString());
|
||||
if (hoodieTable.getMetaClient().getTableType()
|
||||
== HoodieTableType.MERGE_ON_READ) {
|
||||
// If merge on read, then clean the log files for the commits as well
|
||||
// todo: fix below for MERGE_ON_READ
|
||||
deletePaths.add(String
|
||||
.format("%s/%s/%s", config.getBasePath(), partitionPath,
|
||||
FSUtils.maskWithoutLogVersion(nextRecord.getCommitTime(),
|
||||
@@ -158,8 +149,7 @@ public class HoodieCleaner<T extends HoodieRecordPayload<T>> {
|
||||
|
||||
// determine if we have enough commits, to start cleaning.
|
||||
if (commitTimeline.countInstants() > commitsRetained) {
|
||||
HoodieInstant earliestCommitToRetain =
|
||||
commitTimeline.nthInstant(commitTimeline.countInstants() - commitsRetained).get();
|
||||
HoodieInstant earliestCommitToRetain = getEarliestCommitToRetain().get();
|
||||
List<List<HoodieDataFile>> fileVersions =
|
||||
fileSystemView.getEveryVersionInPartition(partitionPath)
|
||||
.collect(Collectors.toList());
|
||||
@@ -192,12 +182,11 @@ public class HoodieCleaner<T extends HoodieRecordPayload<T>> {
|
||||
fileCommitTime,
|
||||
HoodieTimeline.GREATER)) {
|
||||
// this is a commit, that should be cleaned.
|
||||
deletePaths.add(String
|
||||
.format("%s/%s/%s", config.getBasePath(), partitionPath, FSUtils
|
||||
.maskWithoutTaskPartitionId(fileCommitTime, afile.getFileId())));
|
||||
deletePaths.add(afile.getFileStatus().getPath().toString());
|
||||
if (hoodieTable.getMetaClient().getTableType()
|
||||
== HoodieTableType.MERGE_ON_READ) {
|
||||
// If merge on read, then clean the log files for the commits as well
|
||||
// todo: fix below for MERGE_ON_READ
|
||||
deletePaths.add(String
|
||||
.format("%s/%s/%s", config.getBasePath(), partitionPath,
|
||||
FSUtils.maskWithoutLogVersion(fileCommitTime, afile.getFileId(),
|
||||
@@ -228,49 +217,36 @@ public class HoodieCleaner<T extends HoodieRecordPayload<T>> {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Performs cleaning of the partition path according to cleaning policy and returns the number
|
||||
* of files cleaned.
|
||||
*
|
||||
* @throws IllegalArgumentException if unknown cleaning policy is provided
|
||||
* Returns files to be cleaned for the given partitionPath based on cleaning policy.
|
||||
*/
|
||||
public HoodieCleanStat clean(String partitionPath) throws IOException {
|
||||
public List<String> getDeletePaths(String partitionPath) throws IOException {
|
||||
HoodieCleaningPolicy policy = config.getCleanerPolicy();
|
||||
List<String> deletePaths;
|
||||
Optional<HoodieInstant> earliestCommitToRetain = Optional.empty();
|
||||
if (policy == HoodieCleaningPolicy.KEEP_LATEST_COMMITS) {
|
||||
deletePaths = getFilesToCleanKeepingLatestCommits(partitionPath);
|
||||
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());
|
||||
}
|
||||
logger.info(
|
||||
deletePaths.size() + " patterns used to delete in partition path:" + partitionPath);
|
||||
|
||||
// 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) {
|
||||
boolean deleteResult = fs.delete(deleteVersion.getPath(), false);
|
||||
deletedFiles.put(deleteVersion, deleteResult);
|
||||
if (deleteResult) {
|
||||
logger.info("Cleaned file at path :" + deleteVersion.getPath());
|
||||
}
|
||||
}
|
||||
}
|
||||
return deletePaths;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns earliest commit to retain based on cleaning policy.
|
||||
*/
|
||||
public Optional<HoodieInstant> getEarliestCommitToRetain() {
|
||||
Optional<HoodieInstant> earliestCommitToRetain = Optional.empty();
|
||||
int commitsRetained = config.getCleanerCommitsRetained();
|
||||
if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_COMMITS
|
||||
&& commitTimeline.countInstants() > commitsRetained) {
|
||||
earliestCommitToRetain =
|
||||
commitTimeline.nthInstant(commitTimeline.countInstants() - commitsRetained);
|
||||
}
|
||||
|
||||
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();
|
||||
return earliestCommitToRetain;
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user