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,7 +18,8 @@
package org.apache.hudi.avro;
import org.apache.hudi.common.BloomFilter;
import org.apache.hudi.common.bloom.filter.BloomFilter;
import org.apache.hudi.common.bloom.filter.HoodieDynamicBoundedBloomFilter;
import org.apache.avro.Schema;
import org.apache.parquet.avro.AvroWriteSupport;
@@ -40,6 +41,7 @@ public class HoodieAvroWriteSupport extends AvroWriteSupport {
public static final String HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY = "org.apache.hudi.bloomfilter";
public static final String HOODIE_MIN_RECORD_KEY_FOOTER = "hoodie_min_record_key";
public static final String HOODIE_MAX_RECORD_KEY_FOOTER = "hoodie_max_record_key";
public static final String HOODIE_BLOOM_FILTER_TYPE_CODE = "hoodie_bloom_filter_type_code";
public HoodieAvroWriteSupport(MessageType schema, Schema avroSchema, BloomFilter bloomFilter) {
super(schema, avroSchema);
@@ -55,6 +57,9 @@ public class HoodieAvroWriteSupport extends AvroWriteSupport {
extraMetaData.put(HOODIE_MIN_RECORD_KEY_FOOTER, minRecordKey);
extraMetaData.put(HOODIE_MAX_RECORD_KEY_FOOTER, maxRecordKey);
}
if (bloomFilter.getBloomFilterTypeCode().name().contains(HoodieDynamicBoundedBloomFilter.TYPE_CODE_PREFIX)) {
extraMetaData.put(HOODIE_BLOOM_FILTER_TYPE_CODE, bloomFilter.getBloomFilterTypeCode().name());
}
}
return new WriteSupport.FinalizedWriteContext(extraMetaData);
}

View File

@@ -0,0 +1,50 @@
/*
* 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;
/**
* A Bloom filter interface.
*/
public interface BloomFilter {
/**
* Add a key to the {@link BloomFilter}.
*
* @param key the key to the added to the {@link BloomFilter}
*/
void add(String key);
/**
* Tests for key membership.
*
* @param key the key to be checked for membership
* @return {@code true} if key may be found, {@code false} if key is not found for sure.
*/
boolean mightContain(String key);
/**
* Serialize the bloom filter as a string.
*/
String serializeToString();
/**
* @return the bloom index type code.
**/
BloomFilterTypeCode getBloomFilterTypeCode();
}

View File

@@ -0,0 +1,63 @@
/*
* 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;
/**
* 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 bloomFilterTypeCode bloom filter type code
* @return the {@link BloomFilter} thus created
*/
public static BloomFilter createBloomFilter(int numEntries, double errorRate, int maxNumberOfEntries,
String bloomFilterTypeCode) {
if (bloomFilterTypeCode.equalsIgnoreCase(BloomFilterTypeCode.SIMPLE.name())) {
return new SimpleBloomFilter(numEntries, errorRate, Hash.MURMUR_HASH);
} else if (bloomFilterTypeCode.equalsIgnoreCase(BloomFilterTypeCode.DYNAMIC_V0.name())) {
return new HoodieDynamicBoundedBloomFilter(numEntries, errorRate, Hash.MURMUR_HASH, maxNumberOfEntries);
} else {
throw new IllegalArgumentException("Bloom Filter type code not recognizable " + bloomFilterTypeCode);
}
}
/**
* Generate {@link BloomFilter} from serialized String.
*
* @param serString the serialized string of the {@link BloomFilter}
* @param bloomFilterTypeCode bloom filter type code as string
* @return the {@link BloomFilter} thus generated from the passed in serialized string
*/
public static BloomFilter fromString(String serString, String bloomFilterTypeCode) {
if (bloomFilterTypeCode.equalsIgnoreCase(BloomFilterTypeCode.SIMPLE.name())) {
return new SimpleBloomFilter(serString);
} else if (bloomFilterTypeCode.equalsIgnoreCase(BloomFilterTypeCode.DYNAMIC_V0.name())) {
return new HoodieDynamicBoundedBloomFilter(serString, BloomFilterTypeCode.DYNAMIC_V0);
} else {
throw new IllegalArgumentException("Bloom Filter type code not recognizable " + bloomFilterTypeCode);
}
}
}

View File

@@ -0,0 +1,28 @@
/*
* 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.
* Please do not change the order of the entries.
*/
public enum BloomFilterTypeCode {
SIMPLE,
DYNAMIC_V0
}

View File

@@ -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) {
return (int) Math.ceil(numEntries * (-Math.log(errorRate) / LOG2_SQUARED));
}
/**
* @return the number of hashes given the bitsize and total number of entries.
*/
static int getNumHashes(int bitSize, int numEntries) {
// Number of the hash functions
return (int) Math.ceil(Math.log(2) * bitSize / numEntries);
}
}

View File

@@ -0,0 +1,109 @@
/*
* 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.hudi.exception.HoodieIndexException;
import org.apache.hadoop.util.bloom.Key;
import javax.xml.bind.DatatypeConverter;
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;
/**
* Hoodie's dynamic bloom bounded bloom filter. This is based largely on Hadoop's DynamicBloomFilter, but with a bound
* on amount of entries to dynamically expand to. Once the entries added reach the bound, false positive ratio may not
* be guaranteed.
*/
public class HoodieDynamicBoundedBloomFilter implements BloomFilter {
public static final String TYPE_CODE_PREFIX = "DYNAMIC";
private InternalDynamicBloomFilter internalDynamicBloomFilter;
/**
* Instantiates {@link HoodieDynamicBoundedBloomFilter} with the given args.
*
* @param numEntries The total number of entries.
* @param errorRate maximum allowable error rate.
* @param hashType type of the hashing function (see {@link org.apache.hadoop.util.hash.Hash}).
* @return the {@link HoodieDynamicBoundedBloomFilter} thus created
*/
HoodieDynamicBoundedBloomFilter(int numEntries, double errorRate, int hashType, int maxNoOfEntries) {
// Bit size
int bitSize = BloomFilterUtils.getBitSize(numEntries, errorRate);
// Number of the hash functions
int numHashs = BloomFilterUtils.getNumHashes(bitSize, numEntries);
this.internalDynamicBloomFilter = new InternalDynamicBloomFilter(bitSize, numHashs, hashType, numEntries,
maxNoOfEntries);
}
/**
* Generate {@link HoodieDynamicBoundedBloomFilter} from the given {@code serString} serialized string.
*
* @param serString the serialized string which represents the {@link HoodieDynamicBoundedBloomFilter}
* @param typeCode type code of the bloom filter
*/
HoodieDynamicBoundedBloomFilter(String serString, BloomFilterTypeCode typeCode) {
// ignoring the type code for now, since we have just one version
byte[] bytes = DatatypeConverter.parseBase64Binary(serString);
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(bytes));
try {
internalDynamicBloomFilter = new InternalDynamicBloomFilter();
internalDynamicBloomFilter.readFields(dis);
dis.close();
} catch (IOException e) {
throw new HoodieIndexException("Could not deserialize BloomFilter instance", e);
}
}
@Override
public void add(String key) {
internalDynamicBloomFilter.add(new Key(key.getBytes(StandardCharsets.UTF_8)));
}
@Override
public boolean mightContain(String key) {
return internalDynamicBloomFilter.membershipTest(new Key(key.getBytes(StandardCharsets.UTF_8)));
}
@Override
public String serializeToString() {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream dos = new DataOutputStream(baos);
try {
internalDynamicBloomFilter.write(dos);
byte[] bytes = baos.toByteArray();
dos.close();
return DatatypeConverter.printBase64Binary(bytes);
} catch (IOException e) {
throw new HoodieIndexException("Could not serialize BloomFilter instance", e);
}
}
@Override
public BloomFilterTypeCode getBloomFilterTypeCode() {
return BloomFilterTypeCode.DYNAMIC_V0;
}
}

View File

@@ -0,0 +1,249 @@
/*
* 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.bloom.BloomFilter;
import org.apache.hadoop.util.bloom.Key;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
/**
* Hoodie's internal dynamic Bloom Filter. This is largely based of {@link org.apache.hadoop.util.bloom.DynamicBloomFilter}
* with bounds on maximum number of entries. Once the max entries is reached, false positive gaurantees are not
* honored.
*/
class InternalDynamicBloomFilter extends InternalFilter {
/**
* 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 InternalDynamicBloomFilter() {
}
/**
* Constructor.
* <p>
* 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 InternalDynamicBloomFilter(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(InternalFilter filter) {
if (filter == null
|| !(filter instanceof InternalDynamicBloomFilter)
|| filter.vectorSize != this.vectorSize
|| filter.nbHash != this.nbHash) {
throw new IllegalArgumentException("filters cannot be and-ed");
}
InternalDynamicBloomFilter dbf = (InternalDynamicBloomFilter) 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(InternalFilter filter) {
if (filter == null
|| !(filter instanceof InternalDynamicBloomFilter)
|| filter.vectorSize != this.vectorSize
|| filter.nbHash != this.nbHash) {
throw new IllegalArgumentException("filters cannot be or-ed");
}
InternalDynamicBloomFilter dbf = (InternalDynamicBloomFilter) 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(InternalFilter filter) {
if (filter == null
|| !(filter instanceof InternalDynamicBloomFilter)
|| filter.vectorSize != this.vectorSize
|| filter.nbHash != this.nbHash) {
throw new IllegalArgumentException("filters cannot be xor-ed");
}
InternalDynamicBloomFilter dbf = (InternalDynamicBloomFilter) filter;
if (dbf.matrix.length != this.matrix.length || dbf.nr != this.nr) {
throw new IllegalArgumentException("filters cannot be xor-ed");
}
for (int i = 0; i < matrix.length; i++) {
matrix[i].xor(dbf.matrix[i]);
}
}
@Override
public String toString() {
StringBuilder res = new StringBuilder();
for (int i = 0; i < matrix.length; i++) {
res.append(matrix[i]);
res.append(Character.LINE_SEPARATOR);
}
return res.toString();
}
// Writable
@Override
public void write(DataOutput out) throws IOException {
super.write(out);
out.writeInt(nr);
out.writeInt(currentNbRecord);
out.writeInt(matrix.length);
for (int i = 0; i < matrix.length; i++) {
matrix[i].write(out);
}
}
@Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
nr = in.readInt();
currentNbRecord = in.readInt();
int len = in.readInt();
matrix = new org.apache.hadoop.util.bloom.BloomFilter[len];
for (int i = 0; i < matrix.length; i++) {
matrix[i] = new org.apache.hadoop.util.bloom.BloomFilter();
matrix[i].readFields(in);
}
}
/**
* Adds a new row to <i>this</i> dynamic Bloom filter.
*/
private void addRow() {
org.apache.hadoop.util.bloom.BloomFilter[] tmp = new org.apache.hadoop.util.bloom.BloomFilter[matrix.length + 1];
for (int i = 0; i < matrix.length; i++) {
tmp[i] = matrix[i];
}
tmp[tmp.length - 1] = new org.apache.hadoop.util.bloom.BloomFilter(vectorSize, nbHash, hashType);
matrix = tmp;
}
/**
* Returns the active standard Bloom filter in <i>this</i> dynamic Bloom filter.
*
* @return BloomFilter The active standard Bloom filter.
* <code>Null</code> otherwise.
*/
private BloomFilter getActiveStandardBF() {
if (reachedMax) {
return matrix[curMatrixIndex++ % matrix.length];
}
if (currentNbRecord >= nr && (matrix.length * nr) < maxNr) {
return null;
} else if (currentNbRecord >= nr && (matrix.length * nr) >= maxNr) {
reachedMax = true;
return matrix[0];
}
return matrix[matrix.length - 1];
}
}

View File

@@ -0,0 +1,178 @@
/*
* 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.io.Writable;
import org.apache.hadoop.util.bloom.HashFunction;
import org.apache.hadoop.util.bloom.Key;
import org.apache.hadoop.util.hash.Hash;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
/**
* Copied from {@link org.apache.hadoop.util.bloom.Filter}. {@link InternalDynamicBloomFilter} needs access to some of
* protected members of {@link org.apache.hadoop.util.bloom.Filter} and hence had to copy it locally.
*/
abstract class InternalFilter implements Writable {
private static final int VERSION = -1; // negative to accommodate for old format
protected int vectorSize;
protected HashFunction hash;
protected int nbHash;
protected int hashType;
protected InternalFilter() {
}
/**
* Constructor.
*
* @param vectorSize The vector size of <i>this</i> filter.
* @param nbHash The number of hash functions to consider.
* @param hashType type of the hashing function (see {@link Hash}).
*/
protected InternalFilter(int vectorSize, int nbHash, int hashType) {
this.vectorSize = vectorSize;
this.nbHash = nbHash;
this.hashType = hashType;
this.hash = new HashFunction(this.vectorSize, this.nbHash, this.hashType);
}
/**
* Adds a key to <i>this</i> filter.
*
* @param key The key to add.
*/
public abstract void add(Key key);
/**
* Determines wether a specified key belongs to <i>this</i> filter.
*
* @param key The key to test.
* @return boolean True if the specified key belongs to <i>this</i> filter. False otherwise.
*/
public abstract boolean membershipTest(Key key);
/**
* Peforms a logical AND between <i>this</i> filter and a specified filter.
* <p>
* <b>Invariant</b>: The result is assigned to <i>this</i> filter.
*
* @param filter The filter to AND with.
*/
public abstract void and(InternalFilter filter);
/**
* Peforms a logical OR between <i>this</i> filter and a specified filter.
* <p>
* <b>Invariant</b>: The result is assigned to <i>this</i> filter.
*
* @param filter The filter to OR with.
*/
public abstract void or(InternalFilter filter);
/**
* Peforms a logical XOR between <i>this</i> filter and a specified filter.
* <p>
* <b>Invariant</b>: The result is assigned to <i>this</i> filter.
*
* @param filter The filter to XOR with.
*/
public abstract void xor(InternalFilter filter);
/**
* Performs a logical NOT on <i>this</i> filter.
* <p>
* The result is assigned to <i>this</i> filter.
*/
public abstract void not();
/**
* Adds a list of keys to <i>this</i> filter.
*
* @param keys The list of keys.
*/
public void add(List<Key> keys) {
if (keys == null) {
throw new IllegalArgumentException("ArrayList<Key> may not be null");
}
for (Key key : keys) {
add(key);
}
} //end add()
/**
* Adds a collection of keys to <i>this</i> filter.
*
* @param keys The collection of keys.
*/
public void add(Collection<Key> keys) {
if (keys == null) {
throw new IllegalArgumentException("Collection<Key> may not be null");
}
for (Key key : keys) {
add(key);
}
} //end add()
/**
* Adds an array of keys to <i>this</i> filter.
*
* @param keys The array of keys.
*/
public void add(Key[] keys) {
if (keys == null) {
throw new IllegalArgumentException("Key[] may not be null");
}
for (int i = 0; i < keys.length; i++) {
add(keys[i]);
}
} //end add()
// Writable interface
@Override
public void write(DataOutput out) throws IOException {
out.writeInt(VERSION);
out.writeInt(this.nbHash);
out.writeByte(this.hashType);
out.writeInt(this.vectorSize);
}
@Override
public void readFields(DataInput in) throws IOException {
int ver = in.readInt();
if (ver > 0) { // old unversioned format
this.nbHash = ver;
this.hashType = Hash.JENKINS_HASH;
} else if (ver == VERSION) {
this.nbHash = in.readInt();
this.hashType = in.readByte();
} else {
throw new IOException("Unsupported version: " + ver);
}
this.vectorSize = in.readInt();
this.hash = new HashFunction(this.vectorSize, this.nbHash, this.hashType);
}
} //end class

View File

@@ -16,56 +16,57 @@
* limitations under the License.
*/
package org.apache.hudi.common;
package org.apache.hudi.common.bloom.filter;
import org.apache.hudi.exception.HoodieIndexException;
import org.apache.hadoop.util.bloom.Key;
import org.apache.hadoop.util.hash.Hash;
import javax.xml.bind.DatatypeConverter;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInput;
import java.io.DataInputStream;
import java.io.DataOutput;
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.nio.charset.StandardCharsets;
/**
* A Bloom filter implementation built on top of {@link org.apache.hadoop.util.bloom.BloomFilter}.
* A Simple Bloom filter implementation built on top of {@link org.apache.hadoop.util.bloom.BloomFilter}.
*/
public class BloomFilter {
/**
* 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);
public class SimpleBloomFilter implements BloomFilter {
private org.apache.hadoop.util.bloom.BloomFilter filter = null;
public BloomFilter(int numEntries, double errorRate) {
this(numEntries, errorRate, Hash.MURMUR_HASH);
}
/**
* Create a new Bloom filter with the given configurations.
*
* @param numEntries The total number of entries.
* @param errorRate maximum allowable error rate.
* @param hashType type of the hashing function (see {@link org.apache.hadoop.util.hash.Hash}).
*/
public BloomFilter(int numEntries, double errorRate, int hashType) {
public SimpleBloomFilter(int numEntries, double errorRate, int hashType) {
// Bit size
int bitSize = (int) Math.ceil(numEntries * (-Math.log(errorRate) / LOG2_SQUARED));
int bitSize = BloomFilterUtils.getBitSize(numEntries, errorRate);
// Number of the hash functions
int numHashs = (int) Math.ceil(Math.log(2) * bitSize / numEntries);
int numHashs = BloomFilterUtils.getNumHashes(bitSize, numEntries);
// The filter
this.filter = new org.apache.hadoop.util.bloom.BloomFilter(bitSize, numHashs, hashType);
}
/**
* Create the bloom filter from serialized string.
*
* @param serString serialized string which represents the {@link SimpleBloomFilter}
*/
public BloomFilter(String filterStr) {
public SimpleBloomFilter(String serString) {
this.filter = new org.apache.hadoop.util.bloom.BloomFilter();
byte[] bytes = DatatypeConverter.parseBase64Binary(filterStr);
byte[] bytes = DatatypeConverter.parseBase64Binary(serString);
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(bytes));
try {
this.filter.readFields(dis);
@@ -75,6 +76,7 @@ public class BloomFilter {
}
}
@Override
public void add(String key) {
if (key == null) {
throw new NullPointerException("Key cannot by null");
@@ -82,6 +84,7 @@ public class BloomFilter {
filter.add(new Key(key.getBytes(StandardCharsets.UTF_8)));
}
@Override
public boolean mightContain(String key) {
if (key == null) {
throw new NullPointerException("Key cannot by null");
@@ -92,6 +95,7 @@ public class BloomFilter {
/**
* Serialize the bloom filter as a string.
*/
@Override
public String serializeToString() {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream dos = new DataOutputStream(baos);
@@ -104,4 +108,32 @@ public class BloomFilter {
throw new HoodieIndexException("Could not serialize BloomFilter instance", e);
}
}
private void writeObject(ObjectOutputStream os)
throws IOException {
filter.write(os);
}
private void readObject(ObjectInputStream is)
throws IOException, ClassNotFoundException {
filter = new org.apache.hadoop.util.bloom.BloomFilter();
filter.readFields(is);
}
// @Override
public void write(DataOutput out) throws IOException {
out.write(filter.toString().getBytes());
}
//@Override
public void readFields(DataInput in) throws IOException {
filter = new org.apache.hadoop.util.bloom.BloomFilter();
filter.readFields(in);
}
@Override
public BloomFilterTypeCode getBloomFilterTypeCode() {
return BloomFilterTypeCode.SIMPLE;
}
}

View File

@@ -19,7 +19,9 @@
package org.apache.hudi.common.util;
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.bloom.filter.BloomFilterTypeCode;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
@@ -54,7 +56,7 @@ public class ParquetUtils {
/**
* Read the rowKey list from the given parquet file.
*
* @param filePath The parquet file path.
* @param filePath The parquet file path.
* @param configuration configuration to build fs object
* @return Set Set of row keys
*/
@@ -66,9 +68,9 @@ public class ParquetUtils {
* Read the rowKey list matching the given filter, from the given parquet file. If the filter is empty, then this will
* return all the rowkeys.
*
* @param filePath The parquet file path.
* @param filePath The parquet file path.
* @param configuration configuration to build fs object
* @param filter record keys filter
* @param filter record keys filter
* @return Set Set of row keys matching candidateRecordKeys
*/
public static Set<String> filterParquetRowKeys(Configuration configuration, Path filePath, Set<String> filter) {
@@ -120,7 +122,7 @@ public class ParquetUtils {
}
private static Map<String, String> readParquetFooter(Configuration configuration, boolean required,
Path parquetFilePath, String... footerNames) {
Path parquetFilePath, String... footerNames) {
Map<String, String> footerVals = new HashMap<>();
ParquetMetadata footer = readMetadata(configuration, parquetFilePath);
Map<String, String> metadata = footer.getFileMetaData().getKeyValueMetaData();
@@ -143,15 +145,26 @@ public class ParquetUtils {
* Read out the bloom filter from the parquet file meta data.
*/
public static BloomFilter readBloomFilterFromParquetMetadata(Configuration configuration, Path parquetFilePath) {
Map<String, String> footerVals = readParquetFooter(configuration, false, parquetFilePath,
HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY,
HoodieAvroWriteSupport.OLD_HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY);
Map<String, String> 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)) {
toReturn = BloomFilterFactory.fromString(footerVal,
footerVals.get(HoodieAvroWriteSupport.HOODIE_BLOOM_FILTER_TYPE_CODE));
} else {
toReturn = BloomFilterFactory.fromString(footerVal, BloomFilterTypeCode.SIMPLE.name());
}
}
return toReturn;
}
public static String[] readMinMaxRecordKeys(Configuration configuration, Path parquetFilePath) {
@@ -197,6 +210,7 @@ public class ParquetUtils {
}
static class RecordKeysFilterFunction implements Function<String, Boolean> {
private final Set<String> candidateKeys;
RecordKeysFilterFunction(Set<String> candidateKeys) {