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

@@ -32,6 +32,7 @@ import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.index.HoodieBloomIndex;
import com.uber.hoodie.table.HoodieTable;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
@@ -40,8 +41,6 @@ import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.api.java.function.PairFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext;
@@ -73,8 +72,8 @@ public class HoodieReadClient implements Serializable {
private transient final FileSystem fs;
/**
* TODO: We need to persist the index type into hoodie.properties and be able to access the index
* just with a simple basepath pointing to the dataset. Until, then just always assume a
* TODO: We need to persist the index type into hoodie.properties and be able to access the
* index just with a simple basepath pointing to the dataset. Until, then just always assume a
* BloomIndex
*/
private transient final HoodieBloomIndex index;
@@ -91,11 +90,11 @@ public class HoodieReadClient implements Serializable {
this.fs = FSUtils.getFs();
// Create a Hoodie table which encapsulated the commits and files visible
this.hoodieTable = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
this.commitTimeline = hoodieTable.getCompletedCompactionCommitTimeline();
this.index =
new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
this.sqlContextOpt = Optional.absent();
}
@@ -138,19 +137,9 @@ public class HoodieReadClient implements Serializable {
JavaPairRDD<HoodieKey, Optional<String>> keyToFileRDD =
index.fetchRecordLocation(hoodieKeys, hoodieTable);
List<String> paths = keyToFileRDD
.filter(new Function<Tuple2<HoodieKey, Optional<String>>, Boolean>() {
@Override
public Boolean call(Tuple2<HoodieKey, Optional<String>> keyFileTuple) throws Exception {
return keyFileTuple._2().isPresent();
}
})
.map(new Function<Tuple2<HoodieKey, Optional<String>>, String>() {
@Override
public String call(Tuple2<HoodieKey, Optional<String>> keyFileTuple) throws Exception {
return keyFileTuple._2().get();
}
}).collect();
.filter(keyFileTuple -> keyFileTuple._2().isPresent())
.map(keyFileTuple -> keyFileTuple._2().get())
.collect();
// record locations might be same for multiple keys, so need a unique list
Set<String> uniquePaths = new HashSet<>(paths);
@@ -158,24 +147,16 @@ public class HoodieReadClient implements Serializable {
.parquet(uniquePaths.toArray(new String[uniquePaths.size()]));
StructType schema = originalDF.schema();
JavaPairRDD<HoodieKey, Row> keyRowRDD = originalDF.javaRDD()
.mapToPair(new PairFunction<Row, HoodieKey, Row>() {
@Override
public Tuple2<HoodieKey, Row> call(Row row) throws Exception {
HoodieKey key = new HoodieKey(
row.<String>getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD),
row.<String>getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD));
return new Tuple2<>(key, row);
}
.mapToPair(row -> {
HoodieKey key = new HoodieKey(
row.<String>getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD),
row.<String>getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD));
return new Tuple2<>(key, row);
});
// Now, we need to further filter out, for only rows that match the supplied hoodie keys
JavaRDD<Row> rowRDD = keyRowRDD.join(keyToFileRDD, parallelism)
.map(new Function<Tuple2<HoodieKey, Tuple2<Row, Optional<String>>>, Row>() {
@Override
public Row call(Tuple2<HoodieKey, Tuple2<Row, Optional<String>>> tuple) throws Exception {
return tuple._2()._1();
}
});
.map(tuple -> tuple._2()._1());
return sqlContextOpt.get().createDataFrame(rowRDD, schema);
}
@@ -197,7 +178,7 @@ public class HoodieReadClient implements Serializable {
}
List<HoodieDataFile> latestFiles = fileSystemView.getLatestVersions(fs.globStatus(new Path(path))).collect(
Collectors.toList());
Collectors.toList());
for (HoodieDataFile file : latestFiles) {
filteredPaths.add(file.getPath());
}
@@ -218,16 +199,16 @@ public class HoodieReadClient implements Serializable {
public Dataset<Row> readSince(String lastCommitTimestamp) {
List<HoodieInstant> commitsToReturn =
commitTimeline.findInstantsAfter(lastCommitTimestamp, Integer.MAX_VALUE)
.getInstants().collect(Collectors.toList());
commitTimeline.findInstantsAfter(lastCommitTimestamp, Integer.MAX_VALUE)
.getInstants().collect(Collectors.toList());
//TODO: we can potentially trim this down to only affected partitions, using CommitMetadata
try {
// Go over the commit metadata, and obtain the new files that need to be read.
HashMap<String, String> fileIdToFullPath = new HashMap<>();
for (HoodieInstant commit: commitsToReturn) {
for (HoodieInstant commit : commitsToReturn) {
HoodieCommitMetadata metadata =
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get());
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get());
// get files from each commit, and replace any previous versions
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths());
}
@@ -247,14 +228,14 @@ public class HoodieReadClient implements Serializable {
assertSqlContext();
String actionType = hoodieTable.getCompactedCommitActionType();
HoodieInstant commitInstant =
new HoodieInstant(false, actionType, commitTime);
new HoodieInstant(false, actionType, commitTime);
if (!commitTimeline.containsInstant(commitInstant)) {
new HoodieException("No commit exists at " + commitTime);
}
try {
HoodieCommitMetadata commitMetdata =
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commitInstant).get());
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commitInstant).get());
Collection<String> paths = commitMetdata.getFileIdAndFullPaths().values();
return sqlContextOpt.get().read()
.parquet(paths.toArray(new String[paths.size()]))
@@ -271,8 +252,7 @@ public class HoodieReadClient implements Serializable {
* not found. If the FullFilePath value is present, it is the path component (without scheme) of
* the URI underlying file
*/
public JavaPairRDD<HoodieKey, Optional<String>> checkExists(
JavaRDD<HoodieKey> hoodieKeys) {
public JavaPairRDD<HoodieKey, Optional<String>> checkExists(JavaRDD<HoodieKey> hoodieKeys) {
return index.fetchRecordLocation(hoodieKeys, hoodieTable);
}
@@ -285,12 +265,7 @@ public class HoodieReadClient implements Serializable {
*/
public JavaRDD<HoodieRecord> filterExists(JavaRDD<HoodieRecord> hoodieRecords) {
JavaRDD<HoodieRecord> recordsWithLocation = index.tagLocation(hoodieRecords, hoodieTable);
return recordsWithLocation.filter(new Function<HoodieRecord, Boolean>() {
@Override
public Boolean call(HoodieRecord v1) throws Exception {
return !v1.isCurrentLocationKnown();
}
});
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
}
/**
@@ -308,7 +283,7 @@ public class HoodieReadClient implements Serializable {
*/
public List<String> listCommitsSince(String commitTimestamp) {
return commitTimeline.findInstantsAfter(commitTimestamp, Integer.MAX_VALUE).getInstants()
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
}
/**