Refactor hoodie-common and create right abstractions for Hoodie Storage V2.0
The following is the gist of changes done - All low-level operation of creating a commit code was in HoodieClient which made it hard to share code if there was a compaction commit. - HoodieTableMetadata contained a mix of metadata and filtering files. (Also few operations required FileSystem to be passed in because those were called from TaskExecutors and others had FileSystem as a global variable). Since merge-on-read requires a lot of that code, but will have to change slightly on how it operates on the metadata and how it filters the files. The two set of operation are split into HoodieTableMetaClient and TableFileSystemView. - Everything (active commits, archived commits, cleaner log, save point log and in future delta and compaction commits) in HoodieTableMetaClient is a HoodieTimeline. Timeline is a series of instants, which has an in-built concept of inflight and completed commit markers. - A timeline can be queries for ranges, contains and also use to create new datapoint (create a new commit etc). Commit (and all the above metadata) creation/deletion is streamlined in a timeline - Multiple timelines can be merged into a single timeline, giving us an audit timeline to whatever happened in a hoodie dataset. This also helps with #55. - Move to java 8 and introduce java 8 succinct syntax in refactored code
This commit is contained in:
@@ -19,16 +19,18 @@ package com.uber.hoodie.index;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Optional;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
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.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
@@ -43,6 +45,7 @@ import org.apache.spark.api.java.function.PairFunction;
|
||||
|
||||
import scala.Tuple2;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in
|
||||
@@ -63,10 +66,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
}
|
||||
|
||||
@Override
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, final HoodieTableMetadata metadata) {
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, final HoodieTableMetaClient metaClient) {
|
||||
|
||||
// Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey)
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD = recordRDD
|
||||
@@ -79,7 +79,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
|
||||
// Lookup indexes for all the partition/recordkey pair
|
||||
JavaPairRDD<String, String> rowKeyFilenamePairRDD =
|
||||
lookupIndex(partitionRecordKeyPairRDD, metadata);
|
||||
lookupIndex(partitionRecordKeyPairRDD, metaClient);
|
||||
|
||||
// Cache the result, for subsequent stages.
|
||||
rowKeyFilenamePairRDD.cache();
|
||||
@@ -93,7 +93,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
}
|
||||
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetadata metadata) {
|
||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetaClient metaClient) {
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD =
|
||||
hoodieKeys.mapToPair(new PairFunction<HoodieKey, String, String>() {
|
||||
@Override
|
||||
@@ -104,7 +104,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
|
||||
// Lookup indexes for all the partition/recordkey pair
|
||||
JavaPairRDD<String, String> rowKeyFilenamePairRDD =
|
||||
lookupIndex(partitionRecordKeyPairRDD, metadata);
|
||||
lookupIndex(partitionRecordKeyPairRDD, metaClient);
|
||||
|
||||
JavaPairRDD<String, HoodieKey> rowKeyHoodieKeyPairRDD =
|
||||
hoodieKeys.mapToPair(new PairFunction<HoodieKey, String, HoodieKey>() {
|
||||
@@ -115,17 +115,17 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
});
|
||||
|
||||
return rowKeyHoodieKeyPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).mapToPair(
|
||||
new PairFunction<Tuple2<String, Tuple2<HoodieKey, org.apache.spark.api.java.Optional<String>>>, HoodieKey, Optional<String>>() {
|
||||
new PairFunction<Tuple2<String, Tuple2<HoodieKey, Optional<String>>>, HoodieKey, Optional<String>>() {
|
||||
@Override
|
||||
public Tuple2<HoodieKey, Optional<String>> call(
|
||||
Tuple2<String, Tuple2<HoodieKey, org.apache.spark.api.java.Optional<String>>> keyPathTuple)
|
||||
Tuple2<String, Tuple2<HoodieKey, Optional<String>>> keyPathTuple)
|
||||
throws Exception {
|
||||
Optional<String> recordLocationPath;
|
||||
if (keyPathTuple._2._2.isPresent()) {
|
||||
String fileName = keyPathTuple._2._2.get();
|
||||
String partitionPath = keyPathTuple._2._1.getPartitionPath();
|
||||
recordLocationPath = Optional
|
||||
.of(new Path(new Path(metadata.getBasePath(), partitionPath), fileName)
|
||||
.of(new Path(new Path(metaClient.getBasePath(), partitionPath), fileName)
|
||||
.toUri().getPath());
|
||||
} else {
|
||||
recordLocationPath = Optional.absent();
|
||||
@@ -140,19 +140,19 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
* record keys already present and drop the record keys if not present
|
||||
*
|
||||
* @param partitionRecordKeyPairRDD
|
||||
* @param metadata
|
||||
* @param metaClient
|
||||
* @return
|
||||
*/
|
||||
private JavaPairRDD<String, String> lookupIndex(
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieTableMetadata metadata) {
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieTableMetaClient metaClient) {
|
||||
// Obtain records per partition, in the incoming records
|
||||
Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
|
||||
Map<String, Object> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
|
||||
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
|
||||
|
||||
// Step 2: Load all involved files as <Partition, filename> pairs
|
||||
JavaPairRDD<String, String> partitionFilePairRDD =
|
||||
loadInvolvedFiles(affectedPartitionPathList, metadata);
|
||||
Map<String, Long> filesPerPartition = partitionFilePairRDD.countByKey();
|
||||
loadInvolvedFiles(affectedPartitionPathList, metaClient);
|
||||
Map<String, Object> filesPerPartition = partitionFilePairRDD.countByKey();
|
||||
|
||||
// Compute total subpartitions, to split partitions into.
|
||||
Map<String, Long> subpartitionCountMap =
|
||||
@@ -174,7 +174,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
* @param filesPerPartition
|
||||
* @return
|
||||
*/
|
||||
private Map<String, Long> computeSubPartitions(Map<String, Long> recordsPerPartition, Map<String, Long> filesPerPartition) {
|
||||
private Map<String, Long> computeSubPartitions(Map<String, Object> recordsPerPartition, Map<String, Object> filesPerPartition) {
|
||||
Map<String, Long> subpartitionCountMap = new HashMap<>();
|
||||
long totalRecords = 0;
|
||||
long totalFiles = 0;
|
||||
@@ -210,21 +210,28 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
* Load all involved files as <Partition, filename> pair RDD.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
JavaPairRDD<String, String> loadInvolvedFiles(List<String> partitions, final HoodieTableMetadata metadata) {
|
||||
JavaPairRDD<String, String> loadInvolvedFiles(List<String> partitions,
|
||||
final HoodieTableMetaClient metaClient) {
|
||||
return jsc.parallelize(partitions, Math.max(partitions.size(), 1))
|
||||
.flatMapToPair(new PairFlatMapFunction<String, String, String>() {
|
||||
@Override
|
||||
public Iterator<Tuple2<String, String>> call(String partitionPath) {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
String latestCommitTime = metadata.getAllCommits().lastCommit();
|
||||
FileStatus[] filteredStatus = metadata.getLatestVersionInPartition(fs, partitionPath, latestCommitTime);
|
||||
List<Tuple2<String, String>> list = new ArrayList<>();
|
||||
for (FileStatus fileStatus : filteredStatus) {
|
||||
list.add(new Tuple2<>(partitionPath, fileStatus.getPath().getName()));
|
||||
.flatMapToPair(new PairFlatMapFunction<String, String, String>() {
|
||||
@Override
|
||||
public Iterable<Tuple2<String, String>> call(String partitionPath) {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
TableFileSystemView view = new ReadOptimizedTableView(fs, metaClient);
|
||||
java.util.Optional<String> latestCommitTime =
|
||||
metaClient.getActiveCommitTimeline().lastInstant();
|
||||
List<Tuple2<String, String>> list = new ArrayList<>();
|
||||
if (latestCommitTime.isPresent()) {
|
||||
List<HoodieDataFile> filteredFiles =
|
||||
view.streamLatestVersionInPartition(partitionPath,
|
||||
latestCommitTime.get()).collect(Collectors.toList());
|
||||
for (HoodieDataFile file : filteredFiles) {
|
||||
list.add(new Tuple2<>(partitionPath, file.getFileName()));
|
||||
}
|
||||
return list.iterator();
|
||||
}
|
||||
});
|
||||
return list;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -261,8 +268,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
})
|
||||
.flatMapToPair(new PairFlatMapFunction<List<Tuple2<String, String>>, String, String>() {
|
||||
@Override
|
||||
public Iterator<Tuple2<String, String>> call(List<Tuple2<String, String>> exploded) throws Exception {
|
||||
return exploded.iterator();
|
||||
public Iterable<Tuple2<String, String>> call(List<Tuple2<String, String>> exploded) throws Exception {
|
||||
return exploded;
|
||||
}
|
||||
});
|
||||
|
||||
@@ -323,7 +330,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
/**
|
||||
* Find out <RowKey, filename> pair. All workload grouped by file-level.
|
||||
*
|
||||
* // Join PairRDD(PartitionPath, RecordKey) and PairRDD(PartitionPath, File) and then repartition such that
|
||||
* // Join PairRDD(PartitionPath, RecordKey) and PairRDD(PartitionPath, File) & then repartition such that
|
||||
// each RDD partition is a file, then for each file, we do (1) load bloom filter, (2) load rowKeys, (3) Tag rowKey
|
||||
// Make sure the parallelism is atleast the groupby parallelism for tagging location
|
||||
*/
|
||||
@@ -362,9 +369,9 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(config.getBasePath()), true)
|
||||
.flatMap(new FlatMapFunction<List<IndexLookupResult>, IndexLookupResult>() {
|
||||
@Override
|
||||
public Iterator<IndexLookupResult> call(List<IndexLookupResult> indexLookupResults)
|
||||
public Iterable<IndexLookupResult> call(List<IndexLookupResult> indexLookupResults)
|
||||
throws Exception {
|
||||
return indexLookupResults.iterator();
|
||||
return indexLookupResults;
|
||||
}
|
||||
}).filter(new Function<IndexLookupResult, Boolean>() {
|
||||
@Override
|
||||
@@ -373,13 +380,13 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
}
|
||||
}).flatMapToPair(new PairFlatMapFunction<IndexLookupResult, String, String>() {
|
||||
@Override
|
||||
public Iterator<Tuple2<String, String>> call(IndexLookupResult lookupResult)
|
||||
public Iterable<Tuple2<String, String>> call(IndexLookupResult lookupResult)
|
||||
throws Exception {
|
||||
List<Tuple2<String, String>> vals = new ArrayList<>();
|
||||
for (String recordKey : lookupResult.getMatchingRecordKeys()) {
|
||||
vals.add(new Tuple2<>(recordKey, lookupResult.getFileName()));
|
||||
}
|
||||
return vals.iterator();
|
||||
return vals;
|
||||
}
|
||||
});
|
||||
}
|
||||
@@ -399,9 +406,9 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
|
||||
// Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null), so we do left outer join.
|
||||
return rowKeyRecordPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).values().map(
|
||||
new Function<Tuple2<HoodieRecord<T>, org.apache.spark.api.java.Optional<String>>, HoodieRecord<T>>() {
|
||||
new Function<Tuple2<HoodieRecord<T>, Optional<String>>, HoodieRecord<T>>() {
|
||||
@Override
|
||||
public HoodieRecord<T> call(Tuple2<HoodieRecord<T>, org.apache.spark.api.java.Optional<String>> v1) throws Exception {
|
||||
public HoodieRecord<T> call(Tuple2<HoodieRecord<T>, Optional<String>> v1) throws Exception {
|
||||
HoodieRecord<T> record = v1._1();
|
||||
if (v1._2().isPresent()) {
|
||||
String filename = v1._2().get();
|
||||
@@ -416,7 +423,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieTableMetadata metadata) {
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieTableMetaClient metaClient) {
|
||||
return writeStatusRDD;
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user