-
Notifications
You must be signed in to change notification settings - Fork 2.5k
Add support for dynamic bloom filter to increase efficiency of bloom filter for static sizing #666
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -41,7 +41,7 @@ public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieSto | |
| 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()); | ||
| config.getBloomFilterFPP(), false); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. should n't we pass the config here as well?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. good catch
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @n3nash without the config here, actually we would not have written dynamic filters at all during the tests?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I ran tests isolated only on the bloom filter and generated UUID's + random strings (to simulate non uuid based keys), the test don't go through this code path. |
||
| HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport( | ||
| new AvroSchemaConverter().convert(schema), schema, filter); | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -28,7 +28,8 @@ | |
| import org.apache.hadoop.util.hash.Hash; | ||
|
|
||
| /** | ||
| * A Bloom filter implementation built on top of {@link org.apache.hadoop.util.bloom.BloomFilter}. | ||
| * A Bloom filter implementation built on top of {@link org.apache.hadoop.util.bloom.BloomFilter} or | ||
| * {@link org.apache.hadoop.util.bloom.DynamicBloomFilter}. | ||
| */ | ||
| public class BloomFilter { | ||
|
|
||
|
|
@@ -37,33 +38,49 @@ public class BloomFilter { | |
| */ | ||
| public static final double LOG2_SQUARED = Math.log(2) * Math.log(2); | ||
|
|
||
| private org.apache.hadoop.util.bloom.BloomFilter filter = null; | ||
| private org.apache.hadoop.util.bloom.BloomFilter bloomFilter = null; | ||
|
|
||
| public BloomFilter(int numEntries, double errorRate) { | ||
| this(numEntries, errorRate, Hash.MURMUR_HASH); | ||
| private org.apache.hadoop.util.bloom.DynamicBloomFilter dynamicBloomFilter = null; | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. would be curious to see which other projects use this.. if its easy to do that..
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hbase -> https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/util/BloomFilter.html
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Accumulo seems to be implementing something..
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. https://github.com/apache/accumulo/blob/master/core/src/main/java/org/apache/accumulo/core/bloomfilter/DynamicBloomFilter.java also has one.. It'd be good to know tradeoffs each made. esp HBase and Accumulo |
||
|
|
||
| private boolean isDynamic; | ||
|
|
||
| public BloomFilter(int numEntries, double errorRate, boolean isDynamic) { | ||
| this(numEntries, errorRate, Hash.MURMUR_HASH, isDynamic); | ||
| } | ||
|
|
||
| /** | ||
| * Create a new Bloom filter with the given configurations. | ||
| */ | ||
| public BloomFilter(int numEntries, double errorRate, int hashType) { | ||
| public BloomFilter(int numEntries, double errorRate, int hashType, boolean isDynamic) { | ||
| // Bit size | ||
| int bitSize = (int) Math.ceil(numEntries * (-Math.log(errorRate) / LOG2_SQUARED)); | ||
| // Number of the hash functions | ||
| int numHashs = (int) Math.ceil(Math.log(2) * bitSize / numEntries); | ||
| // The filter | ||
| this.filter = new org.apache.hadoop.util.bloom.BloomFilter(bitSize, numHashs, hashType); | ||
| if (isDynamic) { | ||
| this.dynamicBloomFilter = new org.apache.hadoop.util.bloom.DynamicBloomFilter(bitSize, numHashs, hashType, | ||
| numEntries); | ||
| } else { | ||
| this.bloomFilter = new org.apache.hadoop.util.bloom.BloomFilter(bitSize, numHashs, hashType); | ||
| } | ||
| this.isDynamic = isDynamic; | ||
| } | ||
|
|
||
| /** | ||
| * Create the bloom filter from serialized string. | ||
| */ | ||
| public BloomFilter(String filterStr) { | ||
| this.filter = new org.apache.hadoop.util.bloom.BloomFilter(); | ||
| public BloomFilter(String filterStr, boolean isDynamic) { | ||
| this.isDynamic = isDynamic; | ||
| byte[] bytes = DatatypeConverter.parseBase64Binary(filterStr); | ||
| DataInputStream dis = new DataInputStream(new ByteArrayInputStream(bytes)); | ||
| try { | ||
| this.filter.readFields(dis); | ||
| if (isDynamic) { | ||
| this.dynamicBloomFilter = new org.apache.hadoop.util.bloom.DynamicBloomFilter(); | ||
| this.dynamicBloomFilter.readFields(dis); | ||
| } else { | ||
| this.bloomFilter = new org.apache.hadoop.util.bloom.BloomFilter(); | ||
| this.bloomFilter.readFields(dis); | ||
| } | ||
| dis.close(); | ||
| } catch (IOException e) { | ||
| throw new HoodieIndexException("Could not deserialize BloomFilter instance", e); | ||
|
|
@@ -74,14 +91,22 @@ public void add(String key) { | |
| if (key == null) { | ||
| throw new NullPointerException("Key cannot by null"); | ||
| } | ||
| filter.add(new Key(key.getBytes(StandardCharsets.UTF_8))); | ||
| if (isDynamic) { | ||
| this.dynamicBloomFilter.add(new Key(key.getBytes(StandardCharsets.UTF_8))); | ||
| } else { | ||
| this.bloomFilter.add(new Key(key.getBytes(StandardCharsets.UTF_8))); | ||
| } | ||
| } | ||
|
|
||
| public boolean mightContain(String key) { | ||
| if (key == null) { | ||
| throw new NullPointerException("Key cannot by null"); | ||
| } | ||
| return filter.membershipTest(new Key(key.getBytes(StandardCharsets.UTF_8))); | ||
| if (isDynamic) { | ||
| return this.dynamicBloomFilter.membershipTest(new Key(key.getBytes(StandardCharsets.UTF_8))); | ||
| } else { | ||
| return this.bloomFilter.membershipTest(new Key(key.getBytes(StandardCharsets.UTF_8))); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -91,7 +116,11 @@ public String serializeToString() { | |
| ByteArrayOutputStream baos = new ByteArrayOutputStream(); | ||
| DataOutputStream dos = new DataOutputStream(baos); | ||
| try { | ||
| filter.write(dos); | ||
| if (isDynamic) { | ||
| dynamicBloomFilter.write(dos); | ||
| } else { | ||
| bloomFilter.write(dos); | ||
| } | ||
| byte[] bytes = baos.toByteArray(); | ||
| dos.close(); | ||
| return DatatypeConverter.printBase64Binary(bytes); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -146,10 +146,10 @@ public static Schema readAvroSchema(Configuration configuration, Path parquetFil | |
| * Read out the bloom filter from the parquet file meta data. | ||
| */ | ||
| public static BloomFilter readBloomFilterFromParquetMetadata(Configuration configuration, | ||
| Path parquetFilePath) { | ||
| Path parquetFilePath, boolean enableDynamicBloomFilter) { | ||
| String footerVal = readParquetFooter(configuration, parquetFilePath, | ||
| HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY).get(0); | ||
| return new BloomFilter(footerVal); | ||
| return new BloomFilter(footerVal, enableDynamicBloomFilter); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am assuming this is needed coz only then we 'd know the type of bloom filter and deserialize properly?Can BloomFilter read a DynamicBloomFilter serialized value? what happens when we have a mix of files written using normal and dynamic filterS? should we resolve this using an additional footer, instead of making this a writer side config
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Have some thoughts around this, will discuss f2f |
||
| } | ||
|
|
||
| public static String[] readMinMaxRecordKeys(Configuration configuration, Path parquetFilePath) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,53 @@ | ||
| package com.uber.hoodie.common;/* | ||
| * Copyright (c) 2019 Uber Technologies, Inc. ([email protected]) | ||
| * | ||
| * 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. | ||
| */ | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.UUID; | ||
|
|
||
| public abstract class AbstractBloomFilter { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. rename : AbstractBloomFilterTest
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. done |
||
|
|
||
| protected void testAddKey(boolean isDynamic) { | ||
| BloomFilter filter = new BloomFilter(100, 0.0000001, isDynamic); | ||
| filter.add("key1"); | ||
| assert (filter.mightContain("key1")); | ||
| } | ||
|
|
||
| protected int testOverloadBloom(boolean isDynamic) { | ||
| BloomFilter filter = new BloomFilter(10, 0.0000001, isDynamic); | ||
| for (int i = 0; i < 100; i++) { | ||
| filter.add(UUID.randomUUID().toString()); | ||
| } | ||
| int falsePositives = 0; | ||
| for (int i = 0; i < 100; i++) { | ||
| if (filter.mightContain(UUID.randomUUID().toString())) { | ||
| falsePositives++; | ||
| } | ||
| } | ||
| return falsePositives; | ||
| } | ||
|
|
||
| protected void testSerialize(boolean isDynamic) throws IOException, ClassNotFoundException { | ||
| BloomFilter filter = new BloomFilter(1000, 0.0000001, isDynamic); | ||
| filter.add("key1"); | ||
| filter.add("key2"); | ||
| String filterStr = filter.serializeToString(); | ||
|
|
||
| // Rebuild | ||
| BloomFilter newFilter = new BloomFilter(filterStr, isDynamic); | ||
| assert (newFilter.mightContain("key1")); | ||
| assert (newFilter.mightContain("key2")); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,40 @@ | ||
| /* | ||
| * Copyright (c) 2019 Uber Technologies, Inc. ([email protected]) | ||
| * | ||
| * 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.common; | ||
|
|
||
| import java.io.IOException; | ||
| import org.junit.Test; | ||
|
|
||
| public class TestDynamicBloomFilter extends AbstractBloomFilter { | ||
|
|
||
| @Test | ||
| public void testAddKey() { | ||
| testAddKey(true); | ||
| } | ||
|
|
||
| @Test | ||
| public void testOverloadBloom() { | ||
| int falsePositives = testOverloadBloom(true); | ||
| assert (falsePositives == 0); | ||
| } | ||
|
|
||
| @Test | ||
| public void testSerialize() throws IOException, ClassNotFoundException { | ||
| testSerialize(true); | ||
| } | ||
|
|
||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
rename to "hoodie.bloom.index.dynamic.bloomfilter" or "hoodie.bloom.index.auto.tune.bloomfilter" ? to make it clearer it's about the bloom filter and not the index checking..
Also this should probably belong in storage config? since its about how we write the parquet file right
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I suggest we leave it in the IndexConfig since logically that's where people would look for all Index/BloomFilter related settings ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
depends on how you look at it.. At the code level, its weird to suddenly access an index config in storage.. we can leave it here for now. but let's rename?