1
0

[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:
Sivabalan Narayanan
2019-12-17 19:06:24 -08:00
committed by vinoth chandar
parent 7498ca71cb
commit 14881e99e0
23 changed files with 1069 additions and 83 deletions

View File

@@ -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;

View File

@@ -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

View File

@@ -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;

View File

@@ -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);

View File

@@ -21,6 +21,9 @@ package org.apache.hudi.common;
import org.apache.hudi.HoodieReadClient;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.avro.HoodieAvroWriteSupport;
import org.apache.hudi.common.bloom.filter.BloomFilter;
import org.apache.hudi.common.bloom.filter.BloomFilterFactory;
import org.apache.hudi.common.bloom.filter.BloomFilterTypeCode;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieKey;
@@ -147,7 +150,7 @@ public class HoodieClientTestUtils {
}
public static HashMap<String, String> getLatestFileIDsToFullPath(String basePath, HoodieTimeline commitTimeline,
List<HoodieInstant> commitsToReturn) throws IOException {
List<HoodieInstant> commitsToReturn) throws IOException {
HashMap<String, String> fileIdToFullPath = new HashMap<>();
for (HoodieInstant commit : commitsToReturn) {
HoodieCommitMetadata metadata =
@@ -158,7 +161,7 @@ public class HoodieClientTestUtils {
}
public static Dataset<Row> readCommit(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline,
String commitTime) {
String commitTime) {
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
if (!commitTimeline.containsInstant(commitInstant)) {
new HoodieException("No commit exists at " + commitTime);
@@ -178,7 +181,7 @@ public class HoodieClientTestUtils {
* Obtain all new data written into the Hoodie dataset since the given timestamp.
*/
public static Dataset<Row> readSince(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline,
String lastCommitTime) {
String lastCommitTime) {
List<HoodieInstant> commitsToReturn =
commitTimeline.findInstantsAfter(lastCommitTime, Integer.MAX_VALUE).getInstants().collect(Collectors.toList());
try {
@@ -195,7 +198,7 @@ public class HoodieClientTestUtils {
* Reads the paths under the a hoodie dataset out as a DataFrame.
*/
public static Dataset<Row> read(JavaSparkContext jsc, String basePath, SQLContext sqlContext, FileSystem fs,
String... paths) {
String... paths) {
List<String> filteredPaths = new ArrayList<>();
try {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true);
@@ -214,10 +217,11 @@ public class HoodieClientTestUtils {
}
public static String writeParquetFile(String basePath, String partitionPath, String filename,
List<HoodieRecord> records, Schema schema, BloomFilter filter, boolean createCommitTime) throws IOException {
List<HoodieRecord> records, Schema schema, BloomFilter filter, boolean createCommitTime) throws IOException {
if (filter == null) {
filter = new BloomFilter(10000, 0.0000001);
filter = BloomFilterFactory
.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name());
}
HoodieAvroWriteSupport writeSupport =
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
@@ -245,7 +249,7 @@ public class HoodieClientTestUtils {
}
public static String writeParquetFile(String basePath, String partitionPath, List<HoodieRecord> records,
Schema schema, BloomFilter filter, boolean createCommitTime) throws IOException, InterruptedException {
Schema schema, BloomFilter filter, boolean createCommitTime) throws IOException, InterruptedException {
Thread.sleep(1000);
String commitTime = HoodieTestUtils.makeNewCommitTime();
String fileId = UUID.randomUUID().toString();

View File

@@ -19,9 +19,11 @@
package org.apache.hudi.index.bloom;
import org.apache.hudi.HoodieClientTestHarness;
import org.apache.hudi.common.BloomFilter;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.TestRawTripPayload;
import org.apache.hudi.common.bloom.filter.BloomFilter;
import org.apache.hudi.common.bloom.filter.BloomFilterFactory;
import org.apache.hudi.common.bloom.filter.BloomFilterTypeCode;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -248,7 +250,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
// We write record1, record2 to a parquet file, but the bloom filter contains (record1,
// record2, record3).
BloomFilter filter = new BloomFilter(10000, 0.0000001);
BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name());
filter.add(record3.getRecordKey());
String filename = HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Arrays.asList(record1, record2),
schema, filter, true);
@@ -451,7 +453,8 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
HoodieRecord record2 =
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
BloomFilter filter = new BloomFilter(10000, 0.0000001);
BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1,
BloomFilterTypeCode.SIMPLE.name());
filter.add(record2.getRecordKey());
String filename =
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Arrays.asList(record1), schema, filter, true);

View File

@@ -20,11 +20,11 @@ package org.apache.hudi.table;
import org.apache.hudi.HoodieClientTestHarness;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.common.BloomFilter;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.TestRawTripPayload;
import org.apache.hudi.common.TestRawTripPayload.MetadataMergeWriteStatus;
import org.apache.hudi.common.bloom.filter.BloomFilter;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;