[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r358021740 ## File path: hudi-common/src/test/java/org/apache/hudi/common/bloom/filter/TestInternalDynamicBloomFilter.java ## @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.common.bloom.filter; + +import org.apache.hadoop.util.hash.Hash; +import org.junit.Test; + +/** + * Unit tests {@link InternalDynamicBloomFilter} for size bounding. + */ +public class TestInternalDynamicBloomFilter { + + @Test + public void testBoundedSize() { + +int[] batchSizes = {1000, 1, 1, 10, 10, 1}; +int indexForMaxGrowth = 3; +int maxSize = batchSizes[0] * 100; +BloomFilter filter = new HoodieDynamicBoundedBloomFilter(batchSizes[0], 0.01, Hash.MURMUR_HASH, maxSize); +int index = 0; +int lastKnownBloomSize = 0; +while (index < batchSizes.length) { + for (int i = 0; i < batchSizes[index]; i++) { +String key = org.apache.commons.lang.RandomStringUtils.randomAlphanumeric(50); +filter.add(key); + } + + String serString = filter.serializeToString(); + if (index != 0) { +int curLength = serString.length(); +if (index > indexForMaxGrowth) { + assert curLength == lastKnownBloomSize; Review comment: could you please use the junit assert methods? Also this is better written as a parameterized test? (like the bloom filter test; leave it you to make the final call) This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r357194820 ## File path: hudi-common/src/main/java/org/apache/hudi/common/bloom/filter/BloomFilter.java ## @@ -16,34 +16,35 @@ * limitations under the License. */ -package org.apache.hudi.common; - -import org.junit.Test; - -import java.io.IOException; +package org.apache.hudi.common.bloom.filter; /** - * Tests bloom filter {@link BloomFilter}. + * A Bloom filter interface */ -public class TestBloomFilter { +public interface BloomFilter { Review comment: its funny how git thinks this is a a file move.. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r357197955 ## File path: hudi-common/src/main/java/org/apache/hudi/common/util/CleanerUtils.java ## @@ -29,6 +29,7 @@ import org.apache.hudi.common.versioning.clean.CleanV2MigrationHandler; public class CleanerUtils { + Review comment: nit:extra line This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r357197035 ## File path: hudi-common/src/main/java/org/apache/hudi/common/bloom/filter/HoodieDynamicBoundedBloomFilter.java ## @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.common.bloom.filter; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import javax.xml.bind.DatatypeConverter; +import org.apache.hadoop.util.bloom.Key; +import org.apache.hudi.exception.HoodieIndexException; + +/** + * Hoodie's dynamic bloom bounded bloom filter + */ +public class HoodieDynamicBoundedBloomFilter extends LocalDynamicBloomFilter implements BloomFilter { + + public static final String TYPE_CODE_PREFIX = "DYNAMIC"; Review comment: Do we still need this field? This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r357195468 ## File path: hudi-common/src/main/java/org/apache/hudi/common/bloom/filter/BloomFilterTypeCode.java ## @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.common.bloom.filter; + +/** + * Bloom filter type codes + */ +public enum BloomFilterTypeCode { + SIMPLE, Review comment: add a comment here, to warn people not to change this order.. (it will mess up the type decoding)? This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r355141017 ## File path: hudi-client/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java ## @@ -54,6 +52,11 @@ // 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 = Integer.toString(BloomFilterTypeCode.SIMPLE.ordinal()); Review comment: could we keep it as a string.. much easier to configure than the ordinal? This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r355140925 ## File path: hudi-common/src/main/java/org/apache/hudi/common/bloom/filter/BloomFilterUtils.java ## @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.common.bloom.filter; + +/** + * Bloom filter utils + */ +class BloomFilterUtils { + + /** + * Used in computing the optimal Bloom filter size. This approximately equals 0.480453. + */ + private static final double LOG2_SQUARED = Math.log(2) * Math.log(2); + + /** + * @return the bitsize given the total number of entries and error rate + */ + static int getBitSize(int numEntries, double errorRate) { Review comment: and I guess we already checked from your tests that it meets the fp ratio? if so, we are good. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r353202994 ## File path: hudi-common/src/main/java/org/apache/hudi/common/bloom/filter/LocalDynamicBloomFilter.java ## @@ -0,0 +1,251 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.common.bloom.filter; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import org.apache.hadoop.util.bloom.BloomFilter; +import org.apache.hadoop.util.bloom.Key; + +/** + * Hoodie's Local dynamic Bloom Filter + */ +class LocalDynamicBloomFilter extends LocalFilter { + + /** + * Threshold for the maximum number of key to record in a dynamic Bloom filter row. + */ + private int nr; + + /** + * The number of keys recorded in the current standard active Bloom filter. + */ + private int currentNbRecord; + private int maxNr; + private boolean reachedMax = false; + private int curMatrixIndex = 0; + + /** + * The matrix of Bloom filter. + */ + private org.apache.hadoop.util.bloom.BloomFilter[] matrix; + + /** + * Zero-args constructor for the serialization. + */ + public LocalDynamicBloomFilter() { + } + + /** + * Constructor. + * + * Builds an empty Dynamic Bloom filter. + * + * @param vectorSize The number of bits in the vector. + * @param nbHash The number of hash function to consider. + * @param hashType type of the hashing function (see {@link org.apache.hadoop.util.hash.Hash}). + * @param nr The threshold for the maximum number of keys to record in a dynamic Bloom filter row. + */ + public LocalDynamicBloomFilter(int vectorSize, int nbHash, int hashType, int nr, int maxNr) { +super(vectorSize, nbHash, hashType); + +this.nr = nr; +this.currentNbRecord = 0; +this.maxNr = maxNr; + +matrix = new org.apache.hadoop.util.bloom.BloomFilter[1]; +matrix[0] = new org.apache.hadoop.util.bloom.BloomFilter(this.vectorSize, this.nbHash, this.hashType); + } + + @Override + public void add(Key key) { +if (key == null) { + throw new NullPointerException("Key can not be null"); +} + +org.apache.hadoop.util.bloom.BloomFilter bf = getActiveStandardBF(); + +if (bf == null) { + addRow(); + bf = matrix[matrix.length - 1]; + currentNbRecord = 0; +} + +bf.add(key); + +currentNbRecord++; + } + + @Override + public void and(LocalFilter filter) { +if (filter == null +|| !(filter instanceof LocalDynamicBloomFilter) +|| filter.vectorSize != this.vectorSize +|| filter.nbHash != this.nbHash) { + throw new IllegalArgumentException("filters cannot be and-ed"); +} + +LocalDynamicBloomFilter dbf = (LocalDynamicBloomFilter) filter; + +if (dbf.matrix.length != this.matrix.length || dbf.nr != this.nr) { + throw new IllegalArgumentException("filters cannot be and-ed"); +} + +for (int i = 0; i < matrix.length; i++) { + matrix[i].and(dbf.matrix[i]); +} + } + + @Override + public boolean membershipTest(Key key) { +if (key == null) { + return true; +} + +for (int i = 0; i < matrix.length; i++) { + if (matrix[i].membershipTest(key)) { +return true; + } +} + +return false; + } + + @Override + public void not() { +for (int i = 0; i < matrix.length; i++) { + matrix[i].not(); +} + } + + @Override + public void or(LocalFilter filter) { +if (filter == null +|| !(filter instanceof LocalDynamicBloomFilter) +|| filter.vectorSize != this.vectorSize +|| filter.nbHash != this.nbHash) { + throw new IllegalArgumentException("filters cannot be or-ed"); +} + +LocalDynamicBloomFilter dbf = (LocalDynamicBloomFilter) filter; + +if (dbf.matrix.length != this.matrix.length || dbf.nr != this.nr) { + throw new IllegalArgumentException("filters cannot be or-ed"); +} +for (int i = 0; i < matrix.length; i++) { + matrix[i].or(dbf.matrix[i]); +} + } + + @Override + public void xor(LocalFilter filter) { +if (filter == null +|| !(filter
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r353201825 ## File path: hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java ## @@ -141,15 +143,26 @@ 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) { -Map footerVals = readParquetFooter(configuration, false, parquetFilePath, -HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, -HoodieAvroWriteSupport.OLD_HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY); +Map footerVals = +readParquetFooter(configuration, false, parquetFilePath, +HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, +HoodieAvroWriteSupport.OLD_HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, +HoodieAvroWriteSupport.HOODIE_BLOOM_FILTER_TYPE_CODE); String footerVal = footerVals.get(HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY); if (null == footerVal) { // We use old style key "com.uber.hoodie.bloomfilter" footerVal = footerVals.get(HoodieAvroWriteSupport.OLD_HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY); } -return footerVal != null ? new BloomFilter(footerVal) : null; +BloomFilter toReturn = null; +if (footerVal != null) { + if (footerVals.containsKey(HoodieAvroWriteSupport.HOODIE_BLOOM_FILTER_TYPE_CODE)) { Review comment: add a test around this? This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r353197790 ## File path: hudi-common/src/main/java/org/apache/hudi/common/bloom/filter/HoodieDynamicBoundedBloomFilter.java ## @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.common.bloom.filter; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import javax.xml.bind.DatatypeConverter; +import org.apache.hadoop.util.bloom.Key; +import org.apache.hudi.exception.HoodieIndexException; + +/** + * Hoodie's dynamic bloom bounded bloom filter Review comment: good to credit the source always... Add a line such as `based largely on Hadoop's DynamicBloomFilter, but with a bound on amount of entries to dynamically expand to` This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r353194946 ## File path: hudi-client/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java ## @@ -51,6 +52,11 @@ // 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_PROP = "hoodie.bloom.index.filter.type.prop"; Review comment: remove `.prop` from property name This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r353196810 ## File path: hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java ## @@ -346,6 +347,18 @@ public int getHBaseIndexDesiredPutsTime() { return Integer.valueOf(props.getProperty(HoodieHBaseIndexConfig.HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS)); } + public boolean enableAutoTuneBloomFilter() { Review comment: the filter type is not a boolean right? This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r353198779 ## File path: hudi-common/src/main/java/org/apache/hudi/common/bloom/filter/HoodieDynamicBoundedBloomFilter.java ## @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.common.bloom.filter; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import javax.xml.bind.DatatypeConverter; +import org.apache.hadoop.util.bloom.Key; +import org.apache.hudi.exception.HoodieIndexException; + +/** + * Hoodie's dynamic bloom bounded bloom filter + */ +public class HoodieDynamicBoundedBloomFilter extends LocalDynamicBloomFilter implements BloomFilter { + + public static final String TYPE_CODE_PREFIX = "DYNAMIC"; + public static final String TYPE_CODE = TYPE_CODE_PREFIX + "_V0"; + private LocalDynamicBloomFilter localDynamicBloomFilter; Review comment: why do we have to both extend and wrap `LocalDynamicBloomFilter` ? This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r353200854 ## File path: hudi-common/src/main/java/org/apache/hudi/common/bloom/filter/LocalFilter.java ## @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.common.bloom.filter; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.util.bloom.HashFunction; +import org.apache.hadoop.util.bloom.Key; +import org.apache.hadoop.util.hash.Hash; + +abstract class LocalFilter implements Writable { Review comment: do we really need to get this abstract class as well? did we make any changes to the hadoop class? This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r353198094 ## File path: hudi-common/src/main/java/org/apache/hudi/common/bloom/filter/HoodieDynamicBoundedBloomFilter.java ## @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.common.bloom.filter; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import javax.xml.bind.DatatypeConverter; +import org.apache.hadoop.util.bloom.Key; +import org.apache.hudi.exception.HoodieIndexException; + +/** + * Hoodie's dynamic bloom bounded bloom filter + */ +public class HoodieDynamicBoundedBloomFilter extends LocalDynamicBloomFilter implements BloomFilter { + + public static final String TYPE_CODE_PREFIX = "DYNAMIC"; Review comment: why not a ENUM for type code? This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r353196338 ## File path: hudi-client/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java ## @@ -51,6 +52,11 @@ // 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_PROP = "hoodie.bloom.index.filter.type.prop"; + public static final String DEFAULT_BLOOM_INDEX_FILTER_TYPE_PROP = SimpleBloomFilter.TYPE_CODE; Review comment: we have a certain convention to naming property and the default. please follow the same to keep consistent. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r353195684 ## File path: hudi-client/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java ## @@ -51,6 +52,11 @@ // 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_PROP = "hoodie.bloom.index.filter.type.prop"; + public static final String DEFAULT_BLOOM_INDEX_FILTER_TYPE_PROP = SimpleBloomFilter.TYPE_CODE; + public static final String DYNAMIC_BLOOM_FILTER_MAX_ENTRIES = "hoodie.index.dynamic.bloom.max.entries"; Review comment: consistent naming as `hoodie.bloom.index.filter.dynamic.max.entries` ? (its already too long :/) This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r353205531 ## File path: hudi-common/src/main/java/org/apache/hudi/common/bloom/filter/BloomFilterUtils.java ## @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.common.bloom.filter; + +/** + * Bloom filter utils + */ +class BloomFilterUtils { + + /** + * Used in computing the optimal Bloom filter size. This approximately equals 0.480453. + */ + private static final double LOG2_SQUARED = Math.log(2) * Math.log(2); + + /** + * @return the bitsize given the total number of entries and error rate + */ + static int getBitSize(int numEntries, double errorRate) { Review comment: orthogonal comment.. while we are this deep, we should also understand how correct this bitSizing is also.. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r340826913 ## File path: hudi-common/src/main/java/org/apache/hudi/common/BloomFilterFactory.java ## @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.common; + +import org.apache.hadoop.util.hash.Hash; + +/** + * A Factory class to generate different versions of {@link BloomFilter} + */ +public class BloomFilterFactory { + + /** + * Creates a new {@link BloomFilter} with the given args + * @param numEntries total number of entries + * @param errorRate max allowed error rate + * @param bloomFilterVersion bloom filter version + * @return the {@link BloomFilter} thus created + */ + public static BloomFilter createBloomFilter(int numEntries, double errorRate, int bloomFilterVersion) { +return createBloomFilter(numEntries, errorRate, Hash.MURMUR_HASH, bloomFilterVersion); + } + + /** + * Creates a new {@link BloomFilter} with the given args + * @param numEntries total number of entries + * @param errorRate max allowed error rate + * @param hashType type of the hashing function (see + * {@link org.apache.hadoop.util.hash.Hash}). + * @param bloomFilterVersion bloom filter version + * @return the {@link BloomFilter} thus created + */ + public static BloomFilter createBloomFilter(int numEntries, double errorRate, int hashType, int bloomFilterVersion) { +if(bloomFilterVersion == SimpleBloomFilter.VERSION){ Review comment: yes I was wondering if we need a version on top of type code? but dont think we would version this often per se.. So may be just have type code now is ok. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r340178616 ## File path: hudi-common/src/main/java/org/apache/hudi/common/BloomFilterFactory.java ## @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.common; + +import org.apache.hadoop.util.hash.Hash; + +/** + * A Factory class to generate different versions of {@link BloomFilter} + */ +public class BloomFilterFactory { + + /** + * Creates a new {@link BloomFilter} with the given args + * @param numEntries total number of entries + * @param errorRate max allowed error rate + * @param bloomFilterVersion bloom filter version + * @return the {@link BloomFilter} thus created + */ + public static BloomFilter createBloomFilter(int numEntries, double errorRate, int bloomFilterVersion) { +return createBloomFilter(numEntries, errorRate, Hash.MURMUR_HASH, bloomFilterVersion); + } + + /** + * Creates a new {@link BloomFilter} with the given args + * @param numEntries total number of entries + * @param errorRate max allowed error rate + * @param hashType type of the hashing function (see + * {@link org.apache.hadoop.util.hash.Hash}). + * @param bloomFilterVersion bloom filter version + * @return the {@link BloomFilter} thus created + */ + public static BloomFilter createBloomFilter(int numEntries, double errorRate, int hashType, int bloomFilterVersion) { +if(bloomFilterVersion == SimpleBloomFilter.VERSION){ Review comment: No. i think we are on the same page. just saying instead of calling it versions, lets just call it type code. `TYPE_CODE` .. What we have is different types of bloom filters. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r340058340 ## File path: hudi-common/src/main/java/org/apache/hudi/common/BloomFilterFactory.java ## @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.common; + +import org.apache.hadoop.util.hash.Hash; + +/** + * A Factory class to generate different versions of {@link BloomFilter} + */ +public class BloomFilterFactory { + + /** + * Creates a new {@link BloomFilter} with the given args + * @param numEntries total number of entries + * @param errorRate max allowed error rate + * @param bloomFilterVersion bloom filter version + * @return the {@link BloomFilter} thus created + */ + public static BloomFilter createBloomFilter(int numEntries, double errorRate, int bloomFilterVersion) { +return createBloomFilter(numEntries, errorRate, Hash.MURMUR_HASH, bloomFilterVersion); + } + + /** + * Creates a new {@link BloomFilter} with the given args + * @param numEntries total number of entries + * @param errorRate max allowed error rate + * @param hashType type of the hashing function (see + * {@link org.apache.hadoop.util.hash.Hash}). + * @param bloomFilterVersion bloom filter version + * @return the {@link BloomFilter} thus created + */ + public static BloomFilter createBloomFilter(int numEntries, double errorRate, int hashType, int bloomFilterVersion) { +if(bloomFilterVersion == SimpleBloomFilter.VERSION){ Review comment: I wonder if we are mixing concepts of versioning the bytes written by the bloom filter with the type of BloomFilter? how would we handle a different serialization for the dynamic bloom filter itself for e.g? Would that need to be a separate `DynamicBloomFilterV2` implementation? if so, it makes sense to call this `typeCode` than `VERSION` This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r340060736 ## File path: hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java ## @@ -144,13 +146,22 @@ public static BloomFilter readBloomFilterFromParquetMetadata(Configuration confi Map footerVals = readParquetFooter(configuration, false, parquetFilePath, HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, -HoodieAvroWriteSupport.OLD_HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY); +HoodieAvroWriteSupport.OLD_HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, +HoodieAvroWriteSupport.HOODIE_BLOOM_INDEX_VERSION); Review comment: rename .. `HOODIE_BLOOM_FILTER_VERSION` This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r340058698 ## File path: hudi-common/src/main/java/org/apache/hudi/common/BloomFilterFactory.java ## @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.common; + +import org.apache.hadoop.util.hash.Hash; + +/** + * A Factory class to generate different versions of {@link BloomFilter} + */ +public class BloomFilterFactory { + + /** + * Creates a new {@link BloomFilter} with the given args + * @param numEntries total number of entries + * @param errorRate max allowed error rate + * @param bloomFilterVersion bloom filter version + * @return the {@link BloomFilter} thus created + */ + public static BloomFilter createBloomFilter(int numEntries, double errorRate, int bloomFilterVersion) { +return createBloomFilter(numEntries, errorRate, Hash.MURMUR_HASH, bloomFilterVersion); + } + + /** + * Creates a new {@link BloomFilter} with the given args + * @param numEntries total number of entries + * @param errorRate max allowed error rate + * @param hashType type of the hashing function (see + * {@link org.apache.hadoop.util.hash.Hash}). + * @param bloomFilterVersion bloom filter version + * @return the {@link BloomFilter} thus created + */ + public static BloomFilter createBloomFilter(int numEntries, double errorRate, int hashType, int bloomFilterVersion) { +if(bloomFilterVersion == SimpleBloomFilter.VERSION){ + return new SimpleBloomFilter(numEntries, errorRate, hashType); +} else if(bloomFilterVersion == HudiDynamicBloomFilter.VERSION){ + return new HudiDynamicBloomFilter(numEntries, errorRate, hashType); +} else{ + throw new IllegalArgumentException("Bloom Filter version not recognizable "+ bloomFilterVersion); +} + } + + /** + * Generate {@link BloomFilter} from serialized String + * @param serString the serialized string of the {@link BloomFilter} + * @param bloomFilterVersion bloom filter version + * @return the {@link BloomFilter} thus generated from the passed in serialized string + */ + public static BloomFilter getBloomFilterFromSerializedString(String serString, int bloomFilterVersion){ Review comment: +1 on `fromString`. I think we already do it for metadata This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r340059491 ## File path: hudi-common/src/main/java/org/apache/hudi/common/HudiDynamicBloomFilter.java ## @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.common; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import javax.xml.bind.DatatypeConverter; +import org.apache.hadoop.util.bloom.DynamicBloomFilter; +import org.apache.hadoop.util.bloom.Key; +import org.apache.hudi.exception.HoodieIndexException; + +/** + * Hudi's Dynamic Bloom Filter based out of {@link org.apache.hadoop.util.bloom.DynamicBloomFilter} + */ +public class HudiDynamicBloomFilter extends DynamicBloomFilter implements BloomFilter { Review comment: when you say, its subclassing thats all right? cc @bvaradar if any code was reused we need to attribute this correctly.. Interface is a good idea. thanks for doing it This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r340059786 ## File path: hudi-common/src/main/java/org/apache/hudi/common/HudiDynamicBloomFilter.java ## @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.common; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import javax.xml.bind.DatatypeConverter; +import org.apache.hadoop.util.bloom.DynamicBloomFilter; +import org.apache.hadoop.util.bloom.Key; +import org.apache.hudi.exception.HoodieIndexException; + +/** + * Hudi's Dynamic Bloom Filter based out of {@link org.apache.hadoop.util.bloom.DynamicBloomFilter} + */ +public class HudiDynamicBloomFilter extends DynamicBloomFilter implements BloomFilter { Review comment: Rename to `HoodieDynamicBloomFilter` or just `DynamicBloomFilter` we still have all classes named Hoodie. not Hudi This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter
vinothchandar commented on a change in pull request #976: [HUDI-106] Adding support for DynamicBloomFilter URL: https://github.com/apache/incubator-hudi/pull/976#discussion_r340060321 ## File path: hudi-common/src/main/java/org/apache/hudi/common/SimpleBloomFilter.java ## @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hudi.common; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import javax.xml.bind.DatatypeConverter; +import org.apache.hadoop.util.bloom.Key; +import org.apache.hadoop.util.hash.Hash; +import org.apache.hudi.exception.HoodieIndexException; + +/** + * A Simple Bloom filter implementation built on top of {@link org.apache.hadoop.util.bloom.BloomFilter}. + */ + +public class SimpleBloomFilter implements BloomFilter{ + + public static final int VERSION = 0; + /** + * Used in computing the optimal Bloom filter size. This approximately equals 0.480453. + */ + public static final double LOG2_SQUARED = Math.log(2) * Math.log(2); Review comment: move this to the interface (if possible?) and reuse? This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services