1
0

Adding ability for inserts to be written to log files

This commit is contained in:
Nishith Agarwal
2018-05-13 16:25:11 -07:00
committed by vinoth chandar
parent 34827d50e1
commit 3da063f83b
52 changed files with 1061 additions and 519 deletions

View File

@@ -38,24 +38,23 @@ import org.apache.spark.api.java.JavaSparkContext;
public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Serializable {
protected final HoodieWriteConfig config;
protected transient JavaSparkContext jsc = null;
protected HoodieIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
protected HoodieIndex(HoodieWriteConfig config) {
this.config = config;
this.jsc = jsc;
}
public static <T extends HoodieRecordPayload> HoodieIndex<T> createIndex(HoodieWriteConfig config,
JavaSparkContext jsc) throws HoodieIndexException {
switch (config.getIndexType()) {
case HBASE:
return new HBaseIndex<>(config, jsc);
return new HBaseIndex<>(config);
case INMEMORY:
return new InMemoryHashIndex<>(config, jsc);
return new InMemoryHashIndex<>(config);
case BLOOM:
return new HoodieBloomIndex<>(config, jsc);
return new HoodieBloomIndex<>(config);
case BUCKETED:
return new BucketedIndex<>(config, jsc);
return new BucketedIndex<>(config);
default:
throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
}
@@ -68,22 +67,23 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
* file
*/
public abstract JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table);
JavaRDD<HoodieKey> hoodieKeys, final JavaSparkContext jsc, HoodieTable<T> hoodieTable);
/**
* 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,
HoodieTable<T> hoodieTable) throws HoodieIndexException;
JavaSparkContext jsc, HoodieTable<T> hoodieTable) throws HoodieIndexException;
/**
* Extracts the location of written records, and updates the index.
* <p>
* TODO(vc): We may need to propagate the record as well in a WriteStatus class
*/
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTable<T> hoodieTable) throws HoodieIndexException;
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable)
throws HoodieIndexException;
/**
* Rollback the efffects of the commit made at commitTime.

View File

@@ -43,25 +43,25 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
private static ConcurrentMap<HoodieKey, HoodieRecordLocation> recordLocationMap;
public InMemoryHashIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
public InMemoryHashIndex(HoodieWriteConfig config) {
super(config);
recordLocationMap = new ConcurrentHashMap<>();
}
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
final HoodieTable<T> table) {
JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
throw new UnsupportedOperationException("InMemory index does not implement check exist yet");
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) {
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) {
return writeStatusRDD.map(new Function<WriteStatus, WriteStatus>() {
@Override

View File

@@ -30,6 +30,7 @@ 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.HoodieTableMetaClient;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.ParquetUtils;
@@ -64,13 +65,13 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
private static int MAX_ITEMS_PER_SHUFFLE_PARTITION =
SPARK_MAXIMUM_BYTES_PER_PARTITION / BYTES_PER_PARTITION_FILE_KEY_TRIPLET;
public HoodieBloomIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
public HoodieBloomIndex(HoodieWriteConfig config) {
super(config);
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
final HoodieTable<T> hoodieTable) {
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) {
// Step 0: cache the input record RDD
if (config.getBloomIndexUseCaching()) {
@@ -82,8 +83,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()));
// Lookup indexes for all the partition/recordkey pair
JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD,
hoodieTable);
JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, jsc, hoodieTable);
// Cache the result, for subsequent stages.
if (config.getBloomIndexUseCaching()) {
@@ -108,13 +108,12 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
}
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
final HoodieTable<T> table) {
JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
JavaPairRDD<String, String> partitionRecordKeyPairRDD = hoodieKeys
.mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey()));
// Lookup indexes for all the partition/recordkey pair
JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD,
table);
JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, jsc, hoodieTable);
JavaPairRDD<String, HoodieKey> rowKeyHoodieKeyPairRDD = hoodieKeys
.mapToPair(key -> new Tuple2<>(key.getRecordKey(), key));
@@ -125,7 +124,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
String fileName = keyPathTuple._2._2.get();
String partitionPath = keyPathTuple._2._1.getPartitionPath();
recordLocationPath = Optional
.of(new Path(new Path(table.getMetaClient().getBasePath(), partitionPath), fileName)
.of(new Path(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath), fileName)
.toUri().getPath());
} else {
recordLocationPath = Optional.absent();
@@ -139,14 +138,15 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
* keys already present and drop the record keys if not present
*/
private JavaPairRDD<String, String> lookupIndex(
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieTable<T> hoodieTable) {
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final JavaSparkContext
jsc, final HoodieTable hoodieTable) {
// Obtain records per partition, in the incoming records
Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
// Step 2: Load all involved files as <Partition, filename> pairs
List<Tuple2<String, BloomIndexFileInfo>> fileInfoList = loadInvolvedFiles(
affectedPartitionPathList, hoodieTable);
List<Tuple2<String, BloomIndexFileInfo>> fileInfoList = loadInvolvedFiles(affectedPartitionPathList, jsc,
hoodieTable);
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo = fileInfoList.stream()
.collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList())));
@@ -154,8 +154,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
// that contains it.
int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo,
partitionRecordKeyPairRDD);
return findMatchingFilesForRecordKeys(hoodieTable, partitionToFileInfo,
partitionRecordKeyPairRDD, parallelism);
return findMatchingFilesForRecordKeys(partitionToFileInfo,
partitionRecordKeyPairRDD, parallelism, hoodieTable.getMetaClient());
}
/**
@@ -232,12 +232,12 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
* Load all involved files as <Partition, filename> pair RDD.
*/
@VisibleForTesting
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions,
final HoodieTable<T> hoodieTable) {
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final JavaSparkContext jsc,
final HoodieTable hoodieTable) {
// Obtain the latest data files from all the partitions.
List<Tuple2<String, HoodieDataFile>> dataFilesList = jsc
.parallelize(partitions, Math.max(partitions.size(), 1)).flatMapToPair(partitionPath -> {
java.util.Optional<HoodieInstant> latestCommitTime = hoodieTable.getCommitsTimeline()
java.util.Optional<HoodieInstant> latestCommitTime = hoodieTable.getMetaClient().getCommitsTimeline()
.filterCompletedInstants().lastInstant();
List<Tuple2<String, HoodieDataFile>> filteredFiles = new ArrayList<>();
if (latestCommitTime.isPresent()) {
@@ -352,9 +352,9 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
* Make sure the parallelism is atleast the groupby parallelism for tagging location
*/
@VisibleForTesting
JavaPairRDD<String, String> findMatchingFilesForRecordKeys(HoodieTable hoodieTable,
JavaPairRDD<String, String> findMatchingFilesForRecordKeys(
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
JavaPairRDD<String, String> partitionRecordKeyPairRDD, int totalSubpartitions) {
JavaPairRDD<String, String> partitionRecordKeyPairRDD, int totalSubpartitions, HoodieTableMetaClient metaClient) {
int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(),
totalSubpartitions);
@@ -367,7 +367,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
.sortByKey(true, joinParallelism);
return fileSortedTripletRDD.mapPartitionsWithIndex(
new HoodieBloomIndexCheckFunction(hoodieTable, config.getBasePath()), true)
new HoodieBloomIndexCheckFunction(metaClient, config.getBasePath()), true)
.flatMap(indexLookupResults -> indexLookupResults.iterator())
.filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0)
.flatMapToPair(lookupResult -> {
@@ -410,7 +410,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) {
return writeStatusRDD;
}

View File

@@ -20,11 +20,11 @@ package com.uber.hoodie.index.bloom;
import com.uber.hoodie.common.BloomFilter;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.util.ParquetUtils;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIndexException;
import com.uber.hoodie.func.LazyIterableIterator;
import com.uber.hoodie.table.HoodieTable;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.Iterator;
@@ -49,10 +49,10 @@ public class HoodieBloomIndexCheckFunction implements
private final String basePath;
private final HoodieTable table;
private final HoodieTableMetaClient metaClient;
public HoodieBloomIndexCheckFunction(HoodieTable table, String basePath) {
this.table = table;
public HoodieBloomIndexCheckFunction(HoodieTableMetaClient metaClient, String basePath) {
this.metaClient = metaClient;
this.basePath = basePath;
}
@@ -115,7 +115,7 @@ public class HoodieBloomIndexCheckFunction implements
try {
Path filePath = new Path(basePath + "/" + partitionPath + "/" + fileName);
bloomFilter = ParquetUtils
.readBloomFilterFromParquetMetadata(table.getHadoopConf(), filePath);
.readBloomFilterFromParquetMetadata(metaClient.getHadoopConf(), filePath);
candidateRecordKeys = new ArrayList<>();
currentFile = fileName;
currentParitionPath = partitionPath;
@@ -163,7 +163,7 @@ public class HoodieBloomIndexCheckFunction implements
.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
}
ret.add(new IndexLookupResult(currentFile,
checkCandidatesAgainstFile(table.getHadoopConf(), candidateRecordKeys, filePath)));
checkCandidatesAgainstFile(metaClient.getHadoopConf(), candidateRecordKeys, filePath)));
initState(fileName, partitionPath);
if (bloomFilter.mightContain(recordKey)) {
@@ -176,7 +176,7 @@ public class HoodieBloomIndexCheckFunction implements
}
}
// handle case, where we ran out of input, finish pending work, update return val
// handle case, where we ran out of input, close pending work, update return val
if (!inputItr.hasNext()) {
Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile);
logger.info(
@@ -186,7 +186,7 @@ public class HoodieBloomIndexCheckFunction implements
logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
}
ret.add(new IndexLookupResult(currentFile,
checkCandidatesAgainstFile(table.getHadoopConf(), candidateRecordKeys, filePath)));
checkCandidatesAgainstFile(metaClient.getHadoopConf(), candidateRecordKeys, filePath)));
}
} catch (Throwable e) {

View File

@@ -49,8 +49,8 @@ public class BucketedIndex<T extends HoodieRecordPayload> extends HoodieIndex<T>
private static Logger logger = LogManager.getLogger(BucketedIndex.class);
public BucketedIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
public BucketedIndex(HoodieWriteConfig config) {
super(config);
}
private String getBucket(String recordKey) {
@@ -59,13 +59,14 @@ public class BucketedIndex<T extends HoodieRecordPayload> extends HoodieIndex<T>
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
HoodieTable<T> table) {
JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
return hoodieKeys.mapToPair(hk -> new Tuple2<>(hk, Optional.of(getBucket(hk.getRecordKey()))));
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTable<T> hoodieTable) throws HoodieIndexException {
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable)
throws HoodieIndexException {
return recordRDD.map(record -> {
String bucket = getBucket(record.getRecordKey());
//HACK(vc) a non-existent commit is provided here.
@@ -75,8 +76,9 @@ public class BucketedIndex<T extends HoodieRecordPayload> extends HoodieIndex<T>
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTable<T> hoodieTable) throws HoodieIndexException {
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable)
throws HoodieIndexException {
return writeStatusRDD;
}

View File

@@ -25,6 +25,7 @@ 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.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieWriteConfig;
@@ -69,15 +70,15 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
private static Connection hbaseConnection = null;
private final String tableName;
public HBaseIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
public HBaseIndex(HoodieWriteConfig config) {
super(config);
this.tableName = config.getHbaseTableName();
addShutDownHook();
}
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
HoodieTable<T> table) {
JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
//TODO : Change/Remove filterExists in HoodieReadClient() and revisit
throw new UnsupportedOperationException("HBase index does not implement check exist");
}
@@ -119,8 +120,8 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN);
}
private boolean checkIfValidCommit(HoodieTable<T> hoodieTable, String commitTs) {
HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline();
private boolean checkIfValidCommit(HoodieTableMetaClient metaClient, String commitTs) {
HoodieTimeline commitTimeline = metaClient.getActiveTimeline().filterCompletedInstants();
// Check if the last commit ts for this row is 1) present in the timeline or
// 2) is less than the first commit ts in the timeline
return !commitTimeline.empty() && (commitTimeline
@@ -133,8 +134,8 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
/**
* Function that tags each HoodieRecord with an existing location, if known.
*/
private Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>>
locationTagFunction(HoodieTable<T> hoodieTable) {
private Function2<Integer, Iterator<HoodieRecord<T>>,
Iterator<HoodieRecord<T>>> locationTagFunction(HoodieTableMetaClient metaClient) {
return (Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>>)
(partitionNum, hoodieRecordIterator) -> {
@@ -176,7 +177,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
String partitionPath = Bytes
.toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN));
if (checkIfValidCommit(hoodieTable, commitTs)) {
if (checkIfValidCommit(metaClient, commitTs)) {
currentRecord = new HoodieRecord(
new HoodieKey(currentRecord.getRecordKey(), partitionPath),
currentRecord.getData());
@@ -211,13 +212,12 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) {
return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable), true);
return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable.getMetaClient()), true);
}
private Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>>
updateLocationFunction() {
private Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> updateLocationFunction() {
return (Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>>) (partition,
statusIterator) -> {
Integer multiPutBatchSize = config.getHbaseIndexPutBatchSize();
@@ -306,7 +306,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) {
return writeStatusRDD.mapPartitionsWithIndex(updateLocationFunction(), true);
}