feature(HoodieGlobalBloomIndex): adds a new type of bloom index to allow global record key lookup
This commit is contained in:
committed by
vinoth chandar
parent
7ba842c0fe
commit
98fd97b65f
@@ -24,6 +24,7 @@ import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieIndexException;
|
||||
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
|
||||
import com.uber.hoodie.index.bloom.HoodieGlobalBloomIndex;
|
||||
import com.uber.hoodie.index.bucketed.BucketedIndex;
|
||||
import com.uber.hoodie.index.hbase.HBaseIndex;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
@@ -53,6 +54,8 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
|
||||
return new InMemoryHashIndex<>(config);
|
||||
case BLOOM:
|
||||
return new HoodieBloomIndex<>(config);
|
||||
case GLOBAL_BLOOM:
|
||||
return new HoodieGlobalBloomIndex<>(config);
|
||||
case BUCKETED:
|
||||
return new BucketedIndex<>(config);
|
||||
default:
|
||||
@@ -116,6 +119,6 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
|
||||
|
||||
|
||||
public enum IndexType {
|
||||
HBASE, INMEMORY, BLOOM, BUCKETED
|
||||
HBASE, INMEMORY, BLOOM, GLOBAL_BLOOM, BUCKETED
|
||||
}
|
||||
}
|
||||
|
||||
@@ -303,7 +303,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
* if we dont have key ranges, then also we need to compare against the file. no other choice if
|
||||
* we do, then only compare the file if the record key falls in range.
|
||||
*/
|
||||
private boolean shouldCompareWithFile(BloomIndexFileInfo indexInfo, String recordKey) {
|
||||
boolean shouldCompareWithFile(BloomIndexFileInfo indexInfo, String recordKey) {
|
||||
return !indexInfo.hasKeyRanges() || indexInfo.isKeyInRange(recordKey);
|
||||
}
|
||||
|
||||
@@ -313,10 +313,11 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
* record's key needs to be checked. For datasets, where the keys have a definite insert order
|
||||
* (e.g: timestamp as prefix), the number of files to be compared gets cut down a lot from range
|
||||
* pruning.
|
||||
*
|
||||
* Sub-partition to ensure the records can be looked up against files & also prune
|
||||
* file<=>record comparisons based on recordKey
|
||||
* ranges in the index info.
|
||||
*/
|
||||
// sub-partition to ensure the records can be looked up against files & also prune
|
||||
// file<=>record comparisons based on recordKey
|
||||
// ranges in the index info.
|
||||
@VisibleForTesting
|
||||
JavaPairRDD<String, Tuple2<String, HoodieKey>> explodeRecordRDDWithFileComparisons(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
|
||||
@@ -0,0 +1,110 @@
|
||||
/*
|
||||
* Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*
|
||||
*
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.index.bloom;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* This filter will only work with hoodie dataset since it will only load partitions
|
||||
* with .hoodie_partition_metadata file in it.
|
||||
*/
|
||||
public class HoodieGlobalBloomIndex<T extends HoodieRecordPayload> extends HoodieBloomIndex<T> {
|
||||
|
||||
public HoodieGlobalBloomIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all involved files as <Partition, filename> pair RDD from all partitions in the table.
|
||||
*/
|
||||
@Override
|
||||
@VisibleForTesting
|
||||
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final JavaSparkContext jsc,
|
||||
final HoodieTable hoodieTable) {
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
try {
|
||||
List<String> allPartitionPaths = FSUtils
|
||||
.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
return super.loadInvolvedFiles(allPartitionPaths, jsc, hoodieTable);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to load all partitions", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* For each incoming record, produce N output records, 1 each for each file against which the
|
||||
* record's key needs to be checked. For datasets, where the keys have a definite insert order
|
||||
* (e.g: timestamp as prefix), the number of files to be compared gets cut down a lot from range
|
||||
* pruning.
|
||||
*
|
||||
* Sub-partition to ensure the records can be looked up against files & also prune
|
||||
* file<=>record comparisons based on recordKey
|
||||
* ranges in the index info.
|
||||
* the partition path of the incoming record (partitionRecordKeyPairRDD._2()) will be ignored
|
||||
* since the search scope should be bigger than that
|
||||
*/
|
||||
@Override
|
||||
@VisibleForTesting
|
||||
JavaPairRDD<String, Tuple2<String, HoodieKey>> explodeRecordRDDWithFileComparisons(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
|
||||
List<Tuple2<String, BloomIndexFileInfo>> indexInfos =
|
||||
partitionToFileIndexInfo.entrySet().stream()
|
||||
.flatMap(e1 -> e1.getValue().stream()
|
||||
.map(e2 -> new Tuple2<>(e1.getKey(), e2)))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
return partitionRecordKeyPairRDD.map(partitionRecordKeyPair -> {
|
||||
String recordKey = partitionRecordKeyPair._2();
|
||||
|
||||
List<Tuple2<String, Tuple2<String, HoodieKey>>> recordComparisons = new ArrayList<>();
|
||||
if (indexInfos != null) { // could be null, if there are no files in a given partition yet.
|
||||
// for each candidate file in partition, that needs to be compared.
|
||||
for (Tuple2<String, BloomIndexFileInfo> indexInfo : indexInfos) {
|
||||
if (shouldCompareWithFile(indexInfo._2(), recordKey)) {
|
||||
recordComparisons.add(
|
||||
new Tuple2<>(String.format("%s#%s", indexInfo._2().getFileName(), recordKey),
|
||||
new Tuple2<>(indexInfo._2().getFileName(),
|
||||
new HoodieKey(recordKey, indexInfo._1()))));
|
||||
}
|
||||
}
|
||||
}
|
||||
return recordComparisons;
|
||||
}).flatMapToPair(t -> t.iterator());
|
||||
}
|
||||
|
||||
}
|
||||
@@ -195,7 +195,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
"Error in finding the old file path at commit " + commitTime + " for fileId: " + fileId);
|
||||
} else {
|
||||
AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema());
|
||||
ParquetReader<IndexedRecord> reader = AvroParquetReader.builder(upsertHandle.getOldFilePath())
|
||||
ParquetReader<IndexedRecord> reader = AvroParquetReader.<IndexedRecord>builder(upsertHandle.getOldFilePath())
|
||||
.withConf(getHadoopConf()).build();
|
||||
BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
|
||||
try {
|
||||
|
||||
Reference in New Issue
Block a user