1
0

Implement Merge on Read Storage (#76)

1. Create HoodieTable abstraction for commits and fileSystemView
2. HoodieMergeOnReadTable created
3. View is now always obtained from the table and the correct view based on the table type is returned
This commit is contained in:
prazanna
2017-02-21 15:24:00 -08:00
committed by Prasanna Rajaperumal
parent 11d2fd3428
commit eb46e7c72b
47 changed files with 1113 additions and 421 deletions

View File

@@ -30,6 +30,7 @@ import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.exception.HoodieDependentSystemUnavailableException;
import com.uber.hoodie.exception.HoodieIndexException;
import com.uber.hoodie.table.HoodieTable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.TableName;
@@ -67,7 +68,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, HoodieTableMetaClient metaClient) {
JavaRDD<HoodieKey> hoodieKeys, HoodieTable<T> hoodieTable) {
throw new UnsupportedOperationException("HBase index does not implement check exist yet");
}
@@ -93,10 +94,10 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
class LocationTagFunction
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
private final HoodieTableMetaClient metaClient;
private final HoodieTable<T> hoodieTable;
LocationTagFunction(HoodieTableMetaClient metaClient) {
this.metaClient = metaClient;
LocationTagFunction(HoodieTable<T> hoodieTable) {
this.hoodieTable = hoodieTable;
}
@Override
@@ -129,9 +130,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
String fileId =
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
HoodieTimeline commitTimeline =
metaClient.getActiveTimeline().getCommitTimeline()
.filterCompletedInstants();
HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline();
// if the last commit ts for this row is less than the system commit ts
if (!commitTimeline.empty() && commitTimeline.containsInstant(
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs))) {
@@ -160,9 +159,8 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTableMetaClient metaClient) {
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(metaClient), true);
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, HoodieTable<T> hoodieTable) {
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(hoodieTable), true);
}
class UpdateLocationTask implements Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> {
@@ -223,7 +221,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTableMetaClient metaClient) {
HoodieTable<T> hoodieTable) {
return writeStatusRDD.mapPartitionsWithIndex(new UpdateLocationTask(), true);
}

View File

@@ -18,21 +18,16 @@ 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.timeline.HoodieInstant;
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.HoodieDataFile;
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.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils;
import org.apache.hadoop.fs.FileSystem;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieTable;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -43,9 +38,12 @@ import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.api.java.function.PairFlatMapFunction;
import org.apache.spark.api.java.function.PairFunction;
import scala.Tuple2;
import java.util.*;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
@@ -67,7 +65,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, final HoodieTableMetaClient metaClient) {
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, final HoodieTable<T> hoodieTable) {
// Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey)
JavaPairRDD<String, String> partitionRecordKeyPairRDD = recordRDD
@@ -80,7 +78,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
// Lookup indexes for all the partition/recordkey pair
JavaPairRDD<String, String> rowKeyFilenamePairRDD =
lookupIndex(partitionRecordKeyPairRDD, metaClient);
lookupIndex(partitionRecordKeyPairRDD, hoodieTable);
// Cache the result, for subsequent stages.
rowKeyFilenamePairRDD.cache();
@@ -94,7 +92,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
}
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetaClient metaClient) {
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> hoodieTable) {
JavaPairRDD<String, String> partitionRecordKeyPairRDD =
hoodieKeys.mapToPair(new PairFunction<HoodieKey, String, String>() {
@Override
@@ -105,7 +103,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
// Lookup indexes for all the partition/recordkey pair
JavaPairRDD<String, String> rowKeyFilenamePairRDD =
lookupIndex(partitionRecordKeyPairRDD, metaClient);
lookupIndex(partitionRecordKeyPairRDD, hoodieTable);
JavaPairRDD<String, HoodieKey> rowKeyHoodieKeyPairRDD =
hoodieKeys.mapToPair(new PairFunction<HoodieKey, String, HoodieKey>() {
@@ -125,9 +123,9 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
if (keyPathTuple._2._2.isPresent()) {
String fileName = keyPathTuple._2._2.get();
String partitionPath = keyPathTuple._2._1.getPartitionPath();
recordLocationPath = Optional
.of(new Path(new Path(metaClient.getBasePath(), partitionPath), fileName)
.toUri().getPath());
recordLocationPath = Optional.of(new Path(
new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath),
fileName).toUri().getPath());
} else {
recordLocationPath = Optional.absent();
}
@@ -141,18 +139,18 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
* record keys already present and drop the record keys if not present
*
* @param partitionRecordKeyPairRDD
* @param metaClient
* @param hoodieTable
* @return
*/
private JavaPairRDD<String, String> lookupIndex(
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieTableMetaClient metaClient) {
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieTable<T> hoodieTable) {
// Obtain records per partition, in the incoming records
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, metaClient);
loadInvolvedFiles(affectedPartitionPathList, hoodieTable);
Map<String, Object> filesPerPartition = partitionFilePairRDD.countByKey();
// Compute total subpartitions, to split partitions into.
@@ -212,19 +210,17 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
*/
@VisibleForTesting
JavaPairRDD<String, String> loadInvolvedFiles(List<String> partitions,
final HoodieTableMetaClient metaClient) {
final HoodieTable<T> hoodieTable) {
return jsc.parallelize(partitions, Math.max(partitions.size(), 1))
.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<HoodieInstant> latestCommitTime =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants().lastInstant();
hoodieTable.getCommitTimeline().filterCompletedInstants().lastInstant();
List<Tuple2<String, String>> list = new ArrayList<>();
if (latestCommitTime.isPresent()) {
List<HoodieDataFile> filteredFiles =
view.getLatestVersionInPartition(partitionPath,
hoodieTable.getFileSystemView().getLatestVersionInPartition(partitionPath,
latestCommitTime.get().getTimestamp()).collect(Collectors.toList());
for (HoodieDataFile file : filteredFiles) {
list.add(new Tuple2<>(partitionPath, file.getFileName()));
@@ -235,6 +231,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
});
}
@Override
public boolean rollbackCommit(String commitTime) {
// Nope, don't need to do anything.
@@ -424,7 +421,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieTableMetaClient metaClient) {
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieTable<T> hoodieTable) {
return writeStatusRDD;
}
}

View File

@@ -26,6 +26,7 @@ import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.exception.HoodieIndexException;
import com.uber.hoodie.table.HoodieTable;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
@@ -63,15 +64,14 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
* @return
*/
public abstract JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetaClient metaClient);
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> metaClient);
/**
* Looks up the index and tags each incoming record with a location of a file that contains the
* row (if it is actually present)
*/
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTableMetaClient metaClient) throws
HoodieIndexException;
HoodieTable<T> hoodieTable) throws HoodieIndexException;
/**
* Extracts the location of written records, and updates the index.
@@ -79,7 +79,7 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
* TODO(vc): We may need to propagate the record as well in a WriteStatus class
*/
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTableMetaClient metaClient) throws HoodieIndexException;
HoodieTable<T> hoodieTable) throws HoodieIndexException;
/**
* Rollback the efffects of the commit made at commitTime.

View File

@@ -25,6 +25,7 @@ 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.table.HoodieTable;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
@@ -55,7 +56,7 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetaClient metaClient) {
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> hoodieTable) {
throw new UnsupportedOperationException("InMemory index does not implement check exist yet");
}
@@ -81,13 +82,13 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTableMetaClient metaClient) {
HoodieTable<T> hoodieTable) {
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTableMetaClient metaClient) {
HoodieTable<T> hoodieTable) {
return writeStatusRDD.map(new Function<WriteStatus, WriteStatus>() {
@Override
public WriteStatus call(WriteStatus writeStatus) {