1
0

Cleanup code based on Java8 Lambdas (#84)

This commit is contained in:
vinoth chandar
2017-02-27 15:52:13 -08:00
committed by prazanna
parent c4fa585b27
commit 116a78094f
3 changed files with 151 additions and 318 deletions

View File

@@ -45,7 +45,6 @@ import com.uber.hoodie.table.WorkloadProfile;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.Accumulator;
@@ -53,7 +52,6 @@ import org.apache.spark.Partitioner;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.api.java.function.Function2;
import org.apache.spark.api.java.function.PairFunction;
@@ -66,7 +64,6 @@ import java.nio.charset.StandardCharsets;
import java.text.ParseException;
import java.util.Collections;
import java.util.Date;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
@@ -133,12 +130,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, table);
return recordsWithLocation.filter(new Function<HoodieRecord<T>, Boolean>() {
@Override
public Boolean call(HoodieRecord<T> v1) throws Exception {
return !v1.isCurrentLocationKnown();
}
});
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
}
/**
@@ -220,30 +212,20 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
try {
// De-dupe/merge if needed
JavaRDD<HoodieRecord<T>> dedupedRecords =
combineOnCondition(config.shouldCombineBeforeInsert(), records,
config.getInsertShuffleParallelism());
combineOnCondition(config.shouldCombineBeforeInsert(), records, config.getInsertShuffleParallelism());
// Now, sort the records and line them up nicely for loading.
JavaRDD<HoodieRecord<T>> sortedRecords =
dedupedRecords.sortBy(new Function<HoodieRecord<T>, String>() {
@Override
public String call(HoodieRecord<T> record) {
JavaRDD<HoodieRecord<T>> sortedRecords = dedupedRecords
.sortBy(record -> {
// Let's use "partitionPath + key" as the sort key. Spark, will ensure
// the records split evenly across RDD partitions, such that small partitions fit
// into 1 RDD partition, while big ones spread evenly across multiple RDD partitions
return String
.format("%s+%s", record.getPartitionPath(), record.getRecordKey());
}
}, true, config.getInsertShuffleParallelism());
}, true, config.getInsertShuffleParallelism());
JavaRDD<WriteStatus> writeStatusRDD = sortedRecords
.mapPartitionsWithIndex(new BulkInsertMapFunction<T>(commitTime, config, table),
true).flatMap(new FlatMapFunction<List<WriteStatus>, WriteStatus>() {
@Override
public Iterator<WriteStatus> call(List<WriteStatus> writeStatuses)
throws Exception {
return writeStatuses.iterator();
}
});
.mapPartitionsWithIndex(new BulkInsertMapFunction<T>(commitTime, config, table), true)
.flatMap(writeStatuses -> writeStatuses.iterator());
return updateIndexAndCommitIfNeeded(writeStatusRDD, table, commitTime);
} catch (Throwable e) {
@@ -291,11 +273,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
// partition using the insert partitioner
final Partitioner partitioner = getPartitioner(hoodieTable, isUpsert, profile);
JavaRDD<HoodieRecord<T>> partitionedRecords = partition(preppedRecords, partitioner);
JavaRDD<WriteStatus> writeStatusRDD = partitionedRecords.mapPartitionsWithIndex(
new Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>>() {
@Override
public Iterator<List<WriteStatus>> call(Integer partition,
Iterator<HoodieRecord<T>> recordItr) throws Exception {
JavaRDD<WriteStatus> writeStatusRDD = partitionedRecords
.mapPartitionsWithIndex((partition, recordItr) -> {
if (isUpsert) {
return hoodieTable
.handleUpsertPartition(commitTime, partition, recordItr, partitioner);
@@ -303,14 +282,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
return hoodieTable
.handleInsertPartition(commitTime, partition, recordItr, partitioner);
}
}
}, true).flatMap(new FlatMapFunction<List<WriteStatus>, WriteStatus>() {
@Override
public Iterator<WriteStatus> call(List<WriteStatus> writeStatuses)
throws Exception {
return writeStatuses.iterator();
}
});
}, true)
.flatMap(writeStatuses -> writeStatuses.iterator());
return updateIndexAndCommitIfNeeded(writeStatusRDD, hoodieTable, commitTime);
}
@@ -323,9 +296,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
}
}
private JavaRDD<WriteStatus> updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTable<T> table,
String commitTime) {
private JavaRDD<WriteStatus> updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD, HoodieTable<T> table, String commitTime) {
// Update the index back
JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, table);
// Trigger the insert and collect statuses
@@ -335,23 +306,11 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
}
private JavaRDD<HoodieRecord<T>> partition(JavaRDD<HoodieRecord<T>> dedupedRecords, Partitioner partitioner) {
return dedupedRecords.mapToPair(
new PairFunction<HoodieRecord<T>, Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>>() {
@Override
public Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>> call(
HoodieRecord<T> record) throws Exception {
return new Tuple2<>(new Tuple2<>(record.getKey(),
Option.apply(record.getCurrentLocation())), record);
}
}).partitionBy(partitioner).map(
new Function<Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>>, HoodieRecord<T>>() {
@Override
public HoodieRecord<T> call(
Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>> tuple)
throws Exception {
return tuple._2();
}
});
return dedupedRecords
.mapToPair((PairFunction<HoodieRecord<T>, Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>>) record ->
new Tuple2<>(new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())), record))
.partitionBy(partitioner)
.map((Function<Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>>, HoodieRecord<T>>) tuple -> tuple._2());
}
/**
@@ -365,14 +324,10 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
List<Tuple2<String, HoodieWriteStat>> stats =
writeStatuses.mapToPair(new PairFunction<WriteStatus, String, HoodieWriteStat>() {
@Override
public Tuple2<String, HoodieWriteStat> call(WriteStatus writeStatus)
throws Exception {
return new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat());
}
}).collect();
List<Tuple2<String, HoodieWriteStat>> stats = writeStatuses
.mapToPair((PairFunction<WriteStatus, String, HoodieWriteStat>) writeStatus ->
new Tuple2<String, HoodieWriteStat>(writeStatus.getPartitionPath(), writeStatus.getStat()))
.collect();
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
for (Tuple2<String, HoodieWriteStat> stat : stats) {
@@ -460,26 +415,20 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
final Accumulator<Integer> numFilesDeletedAccu = jsc.accumulator(0);
jsc.parallelize(
FSUtils.getAllPartitionPaths(fs, table.getMetaClient().getBasePath()))
.foreach(new VoidFunction<String>() {
@Override
public void call(String partitionPath) throws Exception {
// Scan all partitions files with this commit time
FileSystem fs = FSUtils.getFs();
FileStatus[] toBeDeleted =
fs.listStatus(new Path(config.getBasePath(), partitionPath),
new PathFilter() {
@Override
public boolean accept(Path path) {
return commitTime
.equals(FSUtils.getCommitTime(path.getName()));
}
.foreach((VoidFunction<String>) partitionPath -> {
// Scan all partitions files with this commit time
FileSystem fs1 = FSUtils.getFs();
FileStatus[] toBeDeleted =
fs1.listStatus(new Path(config.getBasePath(), partitionPath),
path -> {
return commitTime
.equals(FSUtils.getCommitTime(path.getName()));
});
for (FileStatus file : toBeDeleted) {
boolean success = fs.delete(file.getPath(), false);
logger.info("Delete file " + file.getPath() + "\t" + success);
if (success) {
numFilesDeletedAccu.add(1);
}
for (FileStatus file : toBeDeleted) {
boolean success = fs1.delete(file.getPath(), false);
logger.info("Delete file " + file.getPath() + "\t" + success);
if (success) {
numFilesDeletedAccu.add(1);
}
}
});
@@ -530,19 +479,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism());
int numFilesDeleted = jsc.parallelize(partitionsToClean, cleanerParallelism)
.map(new Function<String, Integer>() {
@Override
public Integer call(String partitionPathToClean) throws Exception {
FileSystem fs = FSUtils.getFs();
.map((Function<String, Integer>) partitionPathToClean -> {
HoodieCleaner cleaner = new HoodieCleaner(table, config);
return cleaner.clean(partitionPathToClean);
}
}).reduce(new Function2<Integer, Integer, Integer>() {
@Override
public Integer call(Integer v1, Integer v2) throws Exception {
return v1 + v2;
}
});
})
.reduce((Function2<Integer, Integer, Integer>) (v1, v2) -> v1 + v2);
logger.info("Cleaned " + numFilesDeleted + " files");
// Emit metrics (duration, numFilesDeleted) if needed
if (context != null) {