1
0

Parallelize file version deletes during clean and related tests

This commit is contained in:
Kaushik Devarajaiah
2017-05-29 15:36:12 -07:00
committed by prazanna
parent dda28c0b4b
commit 521555c576
7 changed files with 459 additions and 299 deletions

View File

@@ -23,28 +23,18 @@ import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.HoodieCleanStat;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieKey;
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.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.func.LazyInsertIterable;
import com.uber.hoodie.io.HoodieCleanHelper;
import com.uber.hoodie.io.HoodieUpdateHandle;
import java.util.Optional;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.parquet.avro.AvroParquetReader;
import org.apache.parquet.avro.AvroReadSupport;
import org.apache.parquet.hadoop.ParquetReader;
import org.apache.spark.Partitioner;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
@@ -53,13 +43,26 @@ import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Random;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
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.parquet.avro.AvroParquetReader;
import org.apache.parquet.avro.AvroReadSupport;
import org.apache.parquet.hadoop.ParquetReader;
import org.apache.spark.Partitioner;
import org.apache.spark.api.java.JavaSparkContext;
import scala.Option;
import org.apache.spark.api.java.function.Function2;
import org.apache.spark.api.java.function.PairFlatMapFunction;
import scala.Tuple2;
import scala.Option;
/**
* Implementation of a very heavily read-optimized Hoodie Table where
@@ -479,4 +482,145 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
logger.info("Nothing to compact in COW storage format");
return Optional.empty();
}
/**
* Performs cleaning of partition paths according to cleaning policy and returns the number
* of files cleaned. Handles skews in partitions to clean by making files to clean as the
* unit of task distribution.
*
* @throws IllegalArgumentException if unknown cleaning policy is provided
*/
@Override
public List<HoodieCleanStat> clean(JavaSparkContext jsc) {
List<HoodieCleanStat> partitionCleanStats;
try {
List<String> partitionsToClean =
FSUtils.getAllPartitionPaths(getFs(), getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning());
logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config
.getCleanerPolicy());
partitionCleanStats = cleanPartitionPaths(partitionsToClean, jsc);
if (partitionsToClean.isEmpty()) {
logger.info("Nothing to clean here mom. It is already clean");
return partitionCleanStats;
}
} catch (IOException e) {
throw new HoodieIOException("Failed to clean up after commit", e);
}
return partitionCleanStats;
}
private static class PartitionCleanStat implements Serializable {
private final String partitionPath;
private final List<String> deletePathPatterns = new ArrayList<>();
private final List<String> successDeleteFiles = new ArrayList<>();
private final List<String> failedDeleteFiles = new ArrayList<>();
private PartitionCleanStat(String partitionPath) {
this.partitionPath = partitionPath;
}
private void addDeletedFileResult(String deletePathStr, Boolean deletedFileResult) {
if (deletedFileResult) {
successDeleteFiles.add(deletePathStr);
} else {
failedDeleteFiles.add(deletePathStr);
}
}
private void addDeleteFilePatterns(String deletePathStr) {
deletePathPatterns.add(deletePathStr);
}
private PartitionCleanStat merge(PartitionCleanStat other) {
if (!this.partitionPath.equals(other.partitionPath)) {
throw new RuntimeException(String.format(
"partitionPath is not a match: (%s, %s)",
partitionPath, other.partitionPath));
}
successDeleteFiles.addAll(other.successDeleteFiles);
deletePathPatterns.addAll(other.deletePathPatterns);
failedDeleteFiles.addAll(other.failedDeleteFiles);
return this;
}
}
private List<HoodieCleanStat> cleanPartitionPaths(List<String> partitionsToClean, JavaSparkContext jsc) {
int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism());
logger.info("Using cleanerParallelism: " + cleanerParallelism);
List<Tuple2<String, PartitionCleanStat>> partitionCleanStats = jsc
.parallelize(partitionsToClean, cleanerParallelism)
.flatMapToPair(getFilesToDeleteFunc(this, config))
.repartition(cleanerParallelism) // repartition to remove skews
.mapPartitionsToPair(deleteFilesFunc(this, config))
.reduceByKey( // merge partition level clean stats below
(Function2<PartitionCleanStat, PartitionCleanStat, PartitionCleanStat>) (e1, e2) -> e1
.merge(e2))
.collect();
Map<String, PartitionCleanStat> partitionCleanStatsMap = partitionCleanStats
.stream().collect(Collectors.toMap(e -> e._1(), e -> e._2()));
HoodieCleanHelper cleaner = new HoodieCleanHelper(this, config);
// Return PartitionCleanStat for each partition passed.
return partitionsToClean.stream().map(partitionPath -> {
PartitionCleanStat partitionCleanStat =
(partitionCleanStatsMap.containsKey(partitionPath)) ?
partitionCleanStatsMap.get(partitionPath)
: new PartitionCleanStat(partitionPath);
return HoodieCleanStat.newBuilder()
.withPolicy(config.getCleanerPolicy())
.withPartitionPath(partitionPath)
.withEarliestCommitRetained(cleaner.getEarliestCommitToRetain())
.withDeletePathPattern(partitionCleanStat.deletePathPatterns)
.withSuccessfulDeletes(partitionCleanStat.successDeleteFiles)
.withFailedDeletes(partitionCleanStat.failedDeleteFiles)
.build();
}).collect(Collectors.toList());
}
private PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat> deleteFilesFunc(
HoodieTable table, HoodieWriteConfig config) {
return (PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat>) iter -> {
HoodieCleanHelper cleaner = new HoodieCleanHelper(table, config);
Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>();
while (iter.hasNext()) {
Tuple2<String, String> partitionDelFileTuple = iter.next();
String partitionPath = partitionDelFileTuple._1();
String deletePathStr = partitionDelFileTuple._2();
Boolean deletedFileResult = deleteFileAndGetResult(deletePathStr);
if (!partitionCleanStatMap.containsKey(partitionPath)) {
partitionCleanStatMap.put(partitionPath,
new PartitionCleanStat(partitionPath));
}
PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath);
partitionCleanStat.addDeleteFilePatterns(deletePathStr);
partitionCleanStat.addDeletedFileResult(deletePathStr, deletedFileResult);
}
return partitionCleanStatMap.entrySet().stream()
.map(e -> new Tuple2<>(e.getKey(), e.getValue()))
.collect(Collectors.toList()).iterator();
};
}
private static PairFlatMapFunction<String, String, String> getFilesToDeleteFunc(
HoodieTable table, HoodieWriteConfig config) {
return (PairFlatMapFunction<String, String, String>) partitionPathToClean -> {
HoodieCleanHelper cleaner = new HoodieCleanHelper(table, config);
return cleaner.getDeletePaths(partitionPathToClean).stream()
.map(deleteFile -> new Tuple2<>(partitionPathToClean, deleteFile.toString()))
.iterator();
};
}
private Boolean deleteFileAndGetResult(String deletePathStr) throws IOException {
Path deletePath = new Path(deletePathStr);
logger.debug("Working on delete path :" + deletePath);
boolean deleteResult = getFs().delete(deletePath, false);
if (deleteResult) {
logger.debug("Cleaned file at path :" + deletePath);
}
return deleteResult;
}
}