[HUDI-106] Adding support for DynamicBloomFilter (#976)
- Introduced configs for bloom filter type - Implemented dynamic bloom filter with configurable max number of keys - BloomFilterFactory abstractions; Defaults to current simple bloom filter
This commit is contained in:
committed by
vinoth chandar
parent
7498ca71cb
commit
14881e99e0
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
import org.apache.hudi.common.bloom.filter.BloomFilterTypeCode;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
@@ -27,7 +28,6 @@ import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
|
||||
/**
|
||||
* Indexing related config.
|
||||
*/
|
||||
@@ -54,6 +54,11 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
|
||||
// TODO: On by default. Once stable, we will remove the other mode.
|
||||
public static final String BLOOM_INDEX_BUCKETIZED_CHECKING_PROP = "hoodie.bloom.index.bucketized.checking";
|
||||
public static final String DEFAULT_BLOOM_INDEX_BUCKETIZED_CHECKING = "true";
|
||||
public static final String BLOOM_INDEX_FILTER_TYPE = "hoodie.bloom.index.filter.type";
|
||||
public static final String DEFAULT_BLOOM_INDEX_FILTER_TYPE = BloomFilterTypeCode.SIMPLE.name();
|
||||
public static final String HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES = "hoodie.bloom.index.filter.dynamic.max.entries";
|
||||
public static final String DEFAULT_HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES = "100000";
|
||||
|
||||
// 1B bloom filter checks happen in 250 seconds. 500ms to read a bloom filter.
|
||||
// 10M checks in 2500ms, thus amortizing the cost of reading bloom filter across partitions.
|
||||
public static final String BLOOM_INDEX_KEYS_PER_BUCKET_PROP = "hoodie.bloom.index.keys.per.bucket";
|
||||
@@ -194,6 +199,10 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
|
||||
BLOOM_INDEX_BUCKETIZED_CHECKING_PROP, DEFAULT_BLOOM_INDEX_BUCKETIZED_CHECKING);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_KEYS_PER_BUCKET_PROP),
|
||||
BLOOM_INDEX_KEYS_PER_BUCKET_PROP, DEFAULT_BLOOM_INDEX_KEYS_PER_BUCKET);
|
||||
setDefaultOnCondition(props, !props.contains(BLOOM_INDEX_FILTER_TYPE),
|
||||
BLOOM_INDEX_FILTER_TYPE, DEFAULT_BLOOM_INDEX_FILTER_TYPE);
|
||||
setDefaultOnCondition(props, !props.contains(HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES),
|
||||
HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES, DEFAULT_HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES);
|
||||
// Throws IllegalArgumentException if the value set is not a known Hoodie Index Type
|
||||
HoodieIndex.IndexType.valueOf(props.getProperty(INDEX_TYPE_PROP));
|
||||
return config;
|
||||
|
||||
@@ -96,6 +96,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
private static final String MAX_CONSISTENCY_CHECKS_PROP = "hoodie.consistency.check.max_checks";
|
||||
private static int DEFAULT_MAX_CONSISTENCY_CHECKS = 7;
|
||||
|
||||
|
||||
private ConsistencyGuardConfig consistencyGuardConfig;
|
||||
|
||||
// Hoodie Write Client transparently rewrites File System View config when embedded mode is enabled
|
||||
@@ -369,6 +370,14 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
return Integer.valueOf(props.getProperty(HoodieHBaseIndexConfig.HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS));
|
||||
}
|
||||
|
||||
public String getBloomFilterType() {
|
||||
return props.getProperty(HoodieIndexConfig.BLOOM_INDEX_FILTER_TYPE);
|
||||
}
|
||||
|
||||
public int getDynamicBloomFilterMaxNumEntries() {
|
||||
return Integer.parseInt(props.getProperty(HoodieIndexConfig.HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES));
|
||||
}
|
||||
|
||||
/**
|
||||
* Fraction of the global share of QPS that should be allocated to this job. Let's say there are 3 jobs which have
|
||||
* input size in terms of number of rows required for HbaseIndexing as x, 2x, 3x respectively. Then this fraction for
|
||||
|
||||
@@ -18,7 +18,7 @@
|
||||
|
||||
package org.apache.hudi.io;
|
||||
|
||||
import org.apache.hudi.common.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.filter.BloomFilter;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
@@ -55,7 +55,7 @@ public class HoodieKeyLookupHandle<T extends HoodieRecordPayload> extends Hoodie
|
||||
private long totalKeysChecked;
|
||||
|
||||
public HoodieKeyLookupHandle(HoodieWriteConfig config, HoodieTable<T> hoodieTable,
|
||||
Pair<String, String> partitionPathFilePair) {
|
||||
Pair<String, String> partitionPathFilePair) {
|
||||
super(config, null, hoodieTable, partitionPathFilePair);
|
||||
this.tableType = hoodieTable.getMetaClient().getTableType();
|
||||
this.candidateRecordKeys = new ArrayList<>();
|
||||
@@ -70,7 +70,7 @@ public class HoodieKeyLookupHandle<T extends HoodieRecordPayload> extends Hoodie
|
||||
* Given a list of row keys and one file, return only row keys existing in that file.
|
||||
*/
|
||||
public static List<String> checkCandidatesAgainstFile(Configuration configuration, List<String> candidateRecordKeys,
|
||||
Path filePath) throws HoodieIndexException {
|
||||
Path filePath) throws HoodieIndexException {
|
||||
List<String> foundRecordKeys = new ArrayList<>();
|
||||
try {
|
||||
// Load all rowKeys from the file, to double-confirm
|
||||
@@ -134,7 +134,7 @@ public class HoodieKeyLookupHandle<T extends HoodieRecordPayload> extends Hoodie
|
||||
private final String partitionPath;
|
||||
|
||||
public KeyLookupResult(String fileId, String partitionPath, String baseInstantTime,
|
||||
List<String> matchingRecordKeys) {
|
||||
List<String> matchingRecordKeys) {
|
||||
this.fileId = fileId;
|
||||
this.partitionPath = partitionPath;
|
||||
this.baseInstantTime = baseInstantTime;
|
||||
|
||||
@@ -19,7 +19,8 @@
|
||||
package org.apache.hudi.io.storage;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.hudi.common.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.filter.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.filter.BloomFilterFactory;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
@@ -51,7 +52,10 @@ public class HoodieStorageWriterFactory {
|
||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> newParquetStorageWriter(
|
||||
String commitTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable)
|
||||
throws IOException {
|
||||
BloomFilter filter = new BloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP());
|
||||
BloomFilter filter = BloomFilterFactory
|
||||
.createBloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP(),
|
||||
config.getDynamicBloomFilterMaxNumEntries(),
|
||||
config.getBloomFilterType());
|
||||
HoodieAvroWriteSupport writeSupport =
|
||||
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
|
||||
|
||||
Reference in New Issue
Block a user