HBASE-1200 Add bloomfilters
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@946468 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
a8806266b8
commit
ac98087cc6
|
@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.io;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.SortedSet;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -31,6 +32,7 @@ import org.apache.hadoop.hbase.io.hfile.BlockCache;
|
|||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
|
@ -46,8 +48,8 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
*
|
||||
* <p>This file is not splitable. Calls to {@link #midkey()} return null.
|
||||
*/
|
||||
public class HalfHFileReader extends HFile.Reader {
|
||||
final Log LOG = LogFactory.getLog(HalfHFileReader.class);
|
||||
public class HalfStoreFileReader extends StoreFile.Reader {
|
||||
final Log LOG = LogFactory.getLog(HalfStoreFileReader.class);
|
||||
final boolean top;
|
||||
// This is the key we split around. Its the first possible entry on a row:
|
||||
// i.e. empty column and a timestamp of LATEST_TIMESTAMP.
|
||||
|
@ -60,7 +62,7 @@ public class HalfHFileReader extends HFile.Reader {
|
|||
* @param r
|
||||
* @throws IOException
|
||||
*/
|
||||
public HalfHFileReader(final FileSystem fs, final Path p, final BlockCache c,
|
||||
public HalfStoreFileReader(final FileSystem fs, final Path p, final BlockCache c,
|
||||
final Reference r)
|
||||
throws IOException {
|
||||
super(fs, p, c, false);
|
||||
|
@ -153,6 +155,11 @@ public class HalfHFileReader extends HFile.Reader {
|
|||
return this.delegate.seekBefore(key, offset, length);
|
||||
}
|
||||
|
||||
public boolean shouldSeek(byte[] row,
|
||||
final SortedSet<byte[]> columns) {
|
||||
return this.delegate.shouldSeek(row, columns);
|
||||
}
|
||||
|
||||
public boolean seekTo() throws IOException {
|
||||
if (top) {
|
||||
int r = this.delegate.seekTo(splitkey);
|
|
@ -0,0 +1,122 @@
|
|||
/**
|
||||
* Copyright 2009 The Apache Software Foundation
|
||||
*
|
||||
* 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.hadoop.hbase.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.io.Writable;
|
||||
|
||||
/**
|
||||
* Defines the general behavior of a bloom filter.
|
||||
* <p>
|
||||
* The Bloom filter is a data structure that was introduced in 1970 and that has been adopted by
|
||||
* the networking research community in the past decade thanks to the bandwidth efficiencies that it
|
||||
* offers for the transmission of set membership information between networked hosts. A sender encodes
|
||||
* the information into a bit vector, the Bloom filter, that is more compact than a conventional
|
||||
* representation. Computation and space costs for construction are linear in the number of elements.
|
||||
* The receiver uses the filter to test whether various elements are members of the set. Though the
|
||||
* filter will occasionally return a false positive, it will never return a false negative. When creating
|
||||
* the filter, the sender can choose its desired point in a trade-off between the false positive rate and the size.
|
||||
*
|
||||
* <p>
|
||||
* Originally created by
|
||||
* <a href="http://www.one-lab.org">European Commission One-Lab Project 034819</a>.
|
||||
*
|
||||
* <p>
|
||||
* It must be extended in order to define the real behavior.
|
||||
*/
|
||||
public interface BloomFilter {
|
||||
/**
|
||||
* Allocate memory for the bloom filter data. Note that bloom data isn't
|
||||
* allocated by default because it can grow large & reads would be better
|
||||
* managed by the LRU cache.
|
||||
*/
|
||||
void allocBloom();
|
||||
|
||||
/**
|
||||
* Add the specified binary to the bloom filter.
|
||||
*
|
||||
* @param buf data to be added to the bloom
|
||||
*/
|
||||
void add(byte []buf);
|
||||
|
||||
/**
|
||||
* Add the specified binary to the bloom filter.
|
||||
*
|
||||
* @param buf data to be added to the bloom
|
||||
* @param offset offset into the data to be added
|
||||
* @param len length of the data to be added
|
||||
*/
|
||||
void add(byte []buf, int offset, int len);
|
||||
|
||||
/**
|
||||
* Check if the specified key is contained in the bloom filter.
|
||||
*
|
||||
* @param buf data to check for existence of
|
||||
* @param bloom bloom filter data to search
|
||||
* @return true if matched by bloom, false if not
|
||||
*/
|
||||
boolean contains(byte [] buf, ByteBuffer bloom);
|
||||
|
||||
/**
|
||||
* Check if the specified key is contained in the bloom filter.
|
||||
*
|
||||
* @param buf data to check for existence of
|
||||
* @param offset offset into the data
|
||||
* @param length length of the data
|
||||
* @param bloom bloom filter data to search
|
||||
* @return true if matched by bloom, false if not
|
||||
*/
|
||||
boolean contains(byte [] buf, int offset, int length, ByteBuffer bloom);
|
||||
|
||||
/**
|
||||
* @return The number of keys added to the bloom
|
||||
*/
|
||||
int getKeyCount();
|
||||
|
||||
/**
|
||||
* @return The max number of keys that can be inserted
|
||||
* to maintain the desired error rate
|
||||
*/
|
||||
public int getMaxKeys();
|
||||
|
||||
/**
|
||||
* Size of the bloom, in bytes
|
||||
*/
|
||||
public int getByteSize();
|
||||
|
||||
/**
|
||||
* Finalize the bloom before writing metadata & data to disk
|
||||
*/
|
||||
void finalize();
|
||||
|
||||
/**
|
||||
* Get a writable interface into bloom filter meta data.
|
||||
* @return writable class
|
||||
*/
|
||||
Writable getMetaWriter();
|
||||
|
||||
/**
|
||||
* Get a writable interface into bloom filter data (actual bloom).
|
||||
* @return writable class
|
||||
*/
|
||||
Writable getDataWriter();
|
||||
}
|
|
@ -0,0 +1,422 @@
|
|||
/**
|
||||
*
|
||||
* Copyright (c) 2005, European Commission project OneLab under contract 034819 (http://www.one-lab.org)
|
||||
* All rights reserved.
|
||||
* Redistribution and use in source and binary forms, with or
|
||||
* without modification, are permitted provided that the following
|
||||
* conditions are met:
|
||||
* - Redistributions of source code must retain the above copyright
|
||||
* notice, this list of conditions and the following disclaimer.
|
||||
* - Redistributions in binary form must reproduce the above copyright
|
||||
* notice, this list of conditions and the following disclaimer in
|
||||
* the documentation and/or other materials provided with the distribution.
|
||||
* - Neither the name of the University Catholique de Louvain - UCL
|
||||
* nor the names of its contributors may be used to endorse or
|
||||
* promote products derived from this software without specific prior
|
||||
* written permission.
|
||||
*
|
||||
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
|
||||
* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
|
||||
* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS
|
||||
* FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE
|
||||
* COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT,
|
||||
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING,
|
||||
* BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
|
||||
* LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
|
||||
* CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
|
||||
* LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN
|
||||
* ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
|
||||
* POSSIBILITY OF SUCH DAMAGE.
|
||||
*/
|
||||
|
||||
/**
|
||||
* 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.hadoop.hbase.util;
|
||||
|
||||
import java.io.DataOutput;
|
||||
import java.io.DataInput;
|
||||
import java.io.IOException;
|
||||
import java.lang.Math;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.util.bloom.Filter;
|
||||
|
||||
/**
|
||||
* Implements a <i>Bloom filter</i>, as defined by Bloom in 1970.
|
||||
* <p>
|
||||
* The Bloom filter is a data structure that was introduced in 1970 and that has been adopted by
|
||||
* the networking research community in the past decade thanks to the bandwidth efficiencies that it
|
||||
* offers for the transmission of set membership information between networked hosts. A sender encodes
|
||||
* the information into a bit vector, the Bloom filter, that is more compact than a conventional
|
||||
* representation. Computation and space costs for construction are linear in the number of elements.
|
||||
* The receiver uses the filter to test whether various elements are members of the set. Though the
|
||||
* filter will occasionally return a false positive, it will never return a false negative. When creating
|
||||
* the filter, the sender can choose its desired point in a trade-off between the false positive rate and the size.
|
||||
*
|
||||
* <p>
|
||||
* Originally created by
|
||||
* <a href="http://www.one-lab.org">European Commission One-Lab Project 034819</a>.
|
||||
*
|
||||
* @see BloomFilter The general behavior of a filter
|
||||
*
|
||||
* @see <a href="http://portal.acm.org/citation.cfm?id=362692&dl=ACM&coll=portal">Space/Time Trade-Offs in Hash Coding with Allowable Errors</a>
|
||||
*/
|
||||
public class ByteBloomFilter implements BloomFilter {
|
||||
/** Current file format version */
|
||||
public static final int VERSION = 1;
|
||||
|
||||
/** Bytes (B) in the array */
|
||||
protected int byteSize;
|
||||
/** Number of hash functions */
|
||||
protected final int hashCount;
|
||||
/** Hash type */
|
||||
protected final int hashType;
|
||||
/** Hash Function */
|
||||
protected final Hash hash;
|
||||
/** Keys currently in the bloom */
|
||||
protected int keyCount;
|
||||
/** Max Keys expected for the bloom */
|
||||
protected int maxKeys;
|
||||
/** Bloom bits */
|
||||
protected ByteBuffer bloom;
|
||||
|
||||
/** Bit-value lookup array to prevent doing the same work over and over */
|
||||
private static final byte [] bitvals = {
|
||||
(byte) 0x01,
|
||||
(byte) 0x02,
|
||||
(byte) 0x04,
|
||||
(byte) 0x08,
|
||||
(byte) 0x10,
|
||||
(byte) 0x20,
|
||||
(byte) 0x40,
|
||||
(byte) 0x80
|
||||
};
|
||||
|
||||
/**
|
||||
* Loads bloom filter meta data from file input.
|
||||
* @param meta stored bloom meta data
|
||||
* @throws IllegalArgumentException meta data is invalid
|
||||
*/
|
||||
public ByteBloomFilter(ByteBuffer meta)
|
||||
throws IllegalArgumentException {
|
||||
int version = meta.getInt();
|
||||
if (version != VERSION) throw new IllegalArgumentException("Bad version");
|
||||
|
||||
this.byteSize = meta.getInt();
|
||||
this.hashCount = meta.getInt();
|
||||
this.hashType = meta.getInt();
|
||||
this.keyCount = meta.getInt();
|
||||
this.maxKeys = this.keyCount;
|
||||
|
||||
this.hash = Hash.getInstance(this.hashType);
|
||||
sanityCheck();
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines & initializes bloom filter meta data from user config. Call
|
||||
* {@link #allocBloom()} to allocate bloom filter data.
|
||||
* @param maxKeys Maximum expected number of keys that will be stored in this bloom
|
||||
* @param errorRate Desired false positive error rate. Lower rate = more storage required
|
||||
* @param hashType Type of hash function to use
|
||||
* @param foldFactor When finished adding entries, you may be able to 'fold'
|
||||
* this bloom to save space. Tradeoff potentially excess bytes in bloom for
|
||||
* ability to fold if keyCount is exponentially greater than maxKeys.
|
||||
* @throws IllegalArgumentException
|
||||
*/
|
||||
public ByteBloomFilter(int maxKeys, float errorRate, int hashType, int foldFactor)
|
||||
throws IllegalArgumentException {
|
||||
/*
|
||||
* Bloom filters are very sensitive to the number of elements inserted
|
||||
* into them. For HBase, the number of entries depends on the size of the
|
||||
* data stored in the column. Currently the default region size is 256MB,
|
||||
* so entry count ~= 256MB / (average value size for column). Despite
|
||||
* this rule of thumb, there is no efficient way to calculate the entry
|
||||
* count after compactions. Therefore, it is often easier to use a
|
||||
* dynamic bloom filter that will add extra space instead of allowing the
|
||||
* error rate to grow.
|
||||
*
|
||||
* ( http://www.eecs.harvard.edu/~michaelm/NEWWORK/postscripts/BloomFilterSurvey.pdf )
|
||||
*
|
||||
* m denotes the number of bits in the Bloom filter (bitSize)
|
||||
* n denotes the number of elements inserted into the Bloom filter (maxKeys)
|
||||
* k represents the number of hash functions used (nbHash)
|
||||
* e represents the desired false positive rate for the bloom (err)
|
||||
*
|
||||
* If we fix the error rate (e) and know the number of entries, then
|
||||
* the optimal bloom size m = -(n * ln(err) / (ln(2)^2)
|
||||
* ~= n * ln(err) / ln(0.6185)
|
||||
*
|
||||
* The probability of false positives is minimized when k = m/n ln(2).
|
||||
*/
|
||||
int bitSize = (int)Math.ceil(maxKeys * (Math.log(errorRate) / Math.log(0.6185)));
|
||||
int functionCount = (int)Math.ceil(Math.log(2) * (bitSize / maxKeys));
|
||||
|
||||
// increase byteSize so folding is possible
|
||||
int byteSize = (bitSize + 7) / 8;
|
||||
int mask = (1 << foldFactor) - 1;
|
||||
if ( (mask & byteSize) != 0) {
|
||||
byteSize >>= foldFactor;
|
||||
++byteSize;
|
||||
byteSize <<= foldFactor;
|
||||
}
|
||||
|
||||
this.byteSize = byteSize;
|
||||
this.hashCount = functionCount;
|
||||
this.hashType = hashType;
|
||||
this.keyCount = 0;
|
||||
this.maxKeys = maxKeys;
|
||||
|
||||
this.hash = Hash.getInstance(hashType);
|
||||
sanityCheck();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void allocBloom() {
|
||||
if (this.bloom != null) {
|
||||
throw new IllegalArgumentException("can only create bloom once.");
|
||||
}
|
||||
this.bloom = ByteBuffer.allocate(this.byteSize);
|
||||
assert this.bloom.hasArray();
|
||||
}
|
||||
|
||||
void sanityCheck() throws IllegalArgumentException {
|
||||
if(this.byteSize <= 0) {
|
||||
throw new IllegalArgumentException("maxValue must be > 0");
|
||||
}
|
||||
|
||||
if(this.hashCount <= 0) {
|
||||
throw new IllegalArgumentException("Hash function count must be > 0");
|
||||
}
|
||||
|
||||
if (this.hash == null) {
|
||||
throw new IllegalArgumentException("hashType must be known");
|
||||
}
|
||||
|
||||
if (this.keyCount < 0) {
|
||||
throw new IllegalArgumentException("must have positive keyCount");
|
||||
}
|
||||
}
|
||||
|
||||
void bloomCheck(ByteBuffer bloom) throws IllegalArgumentException {
|
||||
if (this.byteSize != bloom.limit()) {
|
||||
throw new IllegalArgumentException(
|
||||
"Configured bloom length should match actual length");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(byte [] buf) {
|
||||
add(buf, 0, buf.length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(byte [] buf, int offset, int len) {
|
||||
/*
|
||||
* For faster hashing, use combinatorial generation
|
||||
* http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf
|
||||
*/
|
||||
int hash1 = this.hash.hash(buf, offset, len, 0);
|
||||
int hash2 = this.hash.hash(buf, offset, len, hash1);
|
||||
|
||||
for (int i = 0; i < this.hashCount; i++) {
|
||||
int hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8));
|
||||
set(hashLoc);
|
||||
}
|
||||
|
||||
++this.keyCount;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should only be used in tests when writing a bloom filter.
|
||||
*/
|
||||
boolean contains(byte [] buf) {
|
||||
return contains(buf, 0, buf.length, this.bloom);
|
||||
}
|
||||
|
||||
/**
|
||||
* Should only be used in tests when writing a bloom filter.
|
||||
*/
|
||||
boolean contains(byte [] buf, int offset, int length) {
|
||||
return contains(buf, offset, length, this.bloom);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean contains(byte [] buf, ByteBuffer theBloom) {
|
||||
return contains(buf, 0, buf.length, theBloom);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean contains(byte [] buf, int offset, int length,
|
||||
ByteBuffer theBloom) {
|
||||
|
||||
if(theBloom.limit() != this.byteSize) {
|
||||
throw new IllegalArgumentException("Bloom does not match expected size");
|
||||
}
|
||||
|
||||
int hash1 = this.hash.hash(buf, offset, length, 0);
|
||||
int hash2 = this.hash.hash(buf, offset, length, hash1);
|
||||
|
||||
for (int i = 0; i < this.hashCount; i++) {
|
||||
int hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8));
|
||||
if (!get(hashLoc, theBloom) ) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------
|
||||
/** Private helpers */
|
||||
|
||||
/**
|
||||
* Set the bit at the specified index to 1.
|
||||
*
|
||||
* @param pos index of bit
|
||||
*/
|
||||
void set(int pos) {
|
||||
int bytePos = pos / 8;
|
||||
int bitPos = pos % 8;
|
||||
byte curByte = bloom.get(bytePos);
|
||||
curByte |= bitvals[bitPos];
|
||||
bloom.put(bytePos, curByte);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if bit at specified index is 1.
|
||||
*
|
||||
* @param pos index of bit
|
||||
* @return true if bit at specified index is 1, false if 0.
|
||||
*/
|
||||
static boolean get(int pos, ByteBuffer theBloom) {
|
||||
int bytePos = pos / 8;
|
||||
int bitPos = pos % 8;
|
||||
byte curByte = theBloom.get(bytePos);
|
||||
curByte &= bitvals[bitPos];
|
||||
return (curByte != 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getKeyCount() {
|
||||
return this.keyCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxKeys() {
|
||||
return this.maxKeys;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getByteSize() {
|
||||
return this.byteSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finalize() {
|
||||
// see if the actual size is exponentially smaller than expected.
|
||||
if (this.keyCount > 0 && this.bloom.hasArray()) {
|
||||
int pieces = 1;
|
||||
int newByteSize = this.byteSize;
|
||||
int newMaxKeys = this.maxKeys;
|
||||
|
||||
// while exponentially smaller & folding is lossless
|
||||
while ( (newByteSize & 1) == 0 && newMaxKeys > (this.keyCount<<1) ) {
|
||||
pieces <<= 1;
|
||||
newByteSize >>= 1;
|
||||
newMaxKeys >>= 1;
|
||||
}
|
||||
|
||||
// if we should fold these into pieces
|
||||
if (pieces > 1) {
|
||||
byte[] array = this.bloom.array();
|
||||
int start = this.bloom.arrayOffset();
|
||||
int end = start + newByteSize;
|
||||
int off = end;
|
||||
for(int p = 1; p < pieces; ++p) {
|
||||
for(int pos = start; pos < end; ++pos) {
|
||||
array[pos] |= array[off++];
|
||||
}
|
||||
}
|
||||
// folding done, only use a subset of this array
|
||||
this.bloom.rewind();
|
||||
this.bloom.limit(newByteSize);
|
||||
this.bloom = this.bloom.slice();
|
||||
this.byteSize = newByteSize;
|
||||
this.maxKeys = newMaxKeys;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
//---------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* Writes just the bloom filter to the output array
|
||||
* @param out OutputStream to place bloom
|
||||
* @throws IOException Error writing bloom array
|
||||
*/
|
||||
public void writeBloom(final DataOutput out) throws IOException {
|
||||
if (!this.bloom.hasArray()) {
|
||||
throw new IOException("Only writes ByteBuffer with underlying array.");
|
||||
}
|
||||
out.write(bloom.array(), bloom.arrayOffset(), bloom.limit());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Writable getMetaWriter() {
|
||||
return new MetaWriter();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Writable getDataWriter() {
|
||||
return new DataWriter();
|
||||
}
|
||||
|
||||
private class MetaWriter implements Writable {
|
||||
protected MetaWriter() {}
|
||||
@Override
|
||||
public void readFields(DataInput arg0) throws IOException {
|
||||
throw new IOException("Cant read with this class.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
out.writeInt(VERSION);
|
||||
out.writeInt(byteSize);
|
||||
out.writeInt(hashCount);
|
||||
out.writeInt(hashType);
|
||||
out.writeInt(keyCount);
|
||||
}
|
||||
}
|
||||
|
||||
private class DataWriter implements Writable {
|
||||
protected DataWriter() {}
|
||||
@Override
|
||||
public void readFields(DataInput arg0) throws IOException {
|
||||
throw new IOException("Cant read with this class.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
writeBloom(out);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,334 @@
|
|||
/**
|
||||
*
|
||||
* Copyright (c) 2005, European Commission project OneLab under contract 034819 (http://www.one-lab.org)
|
||||
* All rights reserved.
|
||||
* Redistribution and use in source and binary forms, with or
|
||||
* without modification, are permitted provided that the following
|
||||
* conditions are met:
|
||||
* - Redistributions of source code must retain the above copyright
|
||||
* notice, this list of conditions and the following disclaimer.
|
||||
* - Redistributions in binary form must reproduce the above copyright
|
||||
* notice, this list of conditions and the following disclaimer in
|
||||
* the documentation and/or other materials provided with the distribution.
|
||||
* - Neither the name of the University Catholique de Louvain - UCL
|
||||
* nor the names of its contributors may be used to endorse or
|
||||
* promote products derived from this software without specific prior
|
||||
* written permission.
|
||||
*
|
||||
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
|
||||
* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
|
||||
* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS
|
||||
* FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE
|
||||
* COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT,
|
||||
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING,
|
||||
* BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
|
||||
* LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
|
||||
* CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
|
||||
* LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN
|
||||
* ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
|
||||
* POSSIBILITY OF SUCH DAMAGE.
|
||||
*/
|
||||
|
||||
/**
|
||||
* 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.hadoop.hbase.util;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.io.Writable;
|
||||
|
||||
/**
|
||||
* Implements a <i>dynamic Bloom filter</i>, as defined in the INFOCOM 2006 paper.
|
||||
* <p>
|
||||
* A dynamic Bloom filter (DBF) makes use of a <code>s * m</code> bit matrix but
|
||||
* each of the <code>s</code> rows is a standard Bloom filter. The creation
|
||||
* process of a DBF is iterative. At the start, the DBF is a <code>1 * m</code>
|
||||
* bit matrix, i.e., it is composed of a single standard Bloom filter.
|
||||
* It assumes that <code>n<sub>r</sub></code> elements are recorded in the
|
||||
* initial bit vector, where <code>n<sub>r</sub> <= n</code> (<code>n</code> is
|
||||
* the cardinality of the set <code>A</code> to record in the filter).
|
||||
* <p>
|
||||
* As the size of <code>A</code> grows during the execution of the application,
|
||||
* several keys must be inserted in the DBF. When inserting a key into the DBF,
|
||||
* one must first get an active Bloom filter in the matrix. A Bloom filter is
|
||||
* active when the number of recorded keys, <code>n<sub>r</sub></code>, is
|
||||
* strictly less than the current cardinality of <code>A</code>, <code>n</code>.
|
||||
* If an active Bloom filter is found, the key is inserted and
|
||||
* <code>n<sub>r</sub></code> is incremented by one. On the other hand, if there
|
||||
* is no active Bloom filter, a new one is created (i.e., a new row is added to
|
||||
* the matrix) according to the current size of <code>A</code> and the element
|
||||
* is added in this new Bloom filter and the <code>n<sub>r</sub></code> value of
|
||||
* this new Bloom filter is set to one. A given key is said to belong to the
|
||||
* DBF if the <code>k</code> positions are set to one in one of the matrix rows.
|
||||
* <p>
|
||||
* Originally created by
|
||||
* <a href="http://www.one-lab.org">European Commission One-Lab Project 034819</a>.
|
||||
*
|
||||
* @see BloomFilter A Bloom filter
|
||||
*
|
||||
* @see <a href="http://www.cse.fau.edu/~jie/research/publications/Publication_files/infocom2006.pdf">Theory and Network Applications of Dynamic Bloom Filters</a>
|
||||
*/
|
||||
public class DynamicByteBloomFilter implements BloomFilter {
|
||||
/** Current file format version */
|
||||
public static final int VERSION = 2;
|
||||
/** Maximum number of keys in a dynamic Bloom filter row. */
|
||||
protected final int keyInterval;
|
||||
/** The maximum false positive rate per bloom */
|
||||
protected final float errorRate;
|
||||
/** Hash type */
|
||||
protected final int hashType;
|
||||
/** The number of keys recorded in the current Bloom filter. */
|
||||
protected int curKeys;
|
||||
/** expected size of bloom filter matrix (used during reads) */
|
||||
protected int readMatrixSize;
|
||||
/** The matrix of Bloom filters (contains bloom data only during writes). */
|
||||
protected ByteBloomFilter[] matrix;
|
||||
|
||||
/**
|
||||
* Normal read constructor. Loads bloom filter meta data.
|
||||
* @param meta stored bloom meta data
|
||||
* @throws IllegalArgumentException meta data is invalid
|
||||
*/
|
||||
public DynamicByteBloomFilter(ByteBuffer meta)
|
||||
throws IllegalArgumentException {
|
||||
int version = meta.getInt();
|
||||
if (version != VERSION) throw new IllegalArgumentException("Bad version");
|
||||
|
||||
this.keyInterval = meta.getInt();
|
||||
this.errorRate = meta.getFloat();
|
||||
this.hashType = meta.getInt();
|
||||
this.readMatrixSize = meta.getInt();
|
||||
this.curKeys = meta.getInt();
|
||||
|
||||
readSanityCheck();
|
||||
|
||||
this.matrix = new ByteBloomFilter[1];
|
||||
this.matrix[0] = new ByteBloomFilter(keyInterval, errorRate, hashType, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Normal write constructor. Note that this doesn't allocate bloom data by
|
||||
* default. Instead, call allocBloom() before adding entries.
|
||||
* @param bitSize The vector size of <i>this</i> filter.
|
||||
* @param functionCount The number of hash function to consider.
|
||||
* @param hashType type of the hashing function (see
|
||||
* {@link org.apache.hadoop.util.hash.Hash}).
|
||||
* @param keyInterval Maximum number of keys to record per Bloom filter row.
|
||||
* @throws IllegalArgumentException The input parameters were invalid
|
||||
*/
|
||||
public DynamicByteBloomFilter(int keyInterval, float errorRate, int hashType)
|
||||
throws IllegalArgumentException {
|
||||
this.keyInterval = keyInterval;
|
||||
this.errorRate = errorRate;
|
||||
this.hashType = hashType;
|
||||
this.curKeys = 0;
|
||||
|
||||
if(keyInterval <= 0) {
|
||||
throw new IllegalArgumentException("keyCount must be > 0");
|
||||
}
|
||||
|
||||
this.matrix = new ByteBloomFilter[1];
|
||||
this.matrix[0] = new ByteBloomFilter(keyInterval, errorRate, hashType, 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void allocBloom() {
|
||||
this.matrix[0].allocBloom();
|
||||
}
|
||||
|
||||
void readSanityCheck() throws IllegalArgumentException {
|
||||
if (this.curKeys <= 0) {
|
||||
throw new IllegalArgumentException("last bloom's key count invalid");
|
||||
}
|
||||
|
||||
if (this.readMatrixSize <= 0) {
|
||||
throw new IllegalArgumentException("matrix size must be known");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(byte []buf, int offset, int len) {
|
||||
BloomFilter bf = getCurBloom();
|
||||
|
||||
if (bf == null) {
|
||||
addRow();
|
||||
bf = matrix[matrix.length - 1];
|
||||
curKeys = 0;
|
||||
}
|
||||
|
||||
bf.add(buf, offset, len);
|
||||
curKeys++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(byte []buf) {
|
||||
add(buf, 0, buf.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Should only be used in tests when writing a bloom filter.
|
||||
*/
|
||||
boolean contains(byte [] buf) {
|
||||
return contains(buf, 0, buf.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Should only be used in tests when writing a bloom filter.
|
||||
*/
|
||||
boolean contains(byte [] buf, int offset, int length) {
|
||||
for (int i = 0; i < matrix.length; i++) {
|
||||
if (matrix[i].contains(buf, offset, length)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean contains(byte [] buf, ByteBuffer theBloom) {
|
||||
return contains(buf, 0, buf.length, theBloom);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean contains(byte[] buf, int offset, int length,
|
||||
ByteBuffer theBloom) {
|
||||
if(offset + length > buf.length) {
|
||||
return false;
|
||||
}
|
||||
|
||||
// current version assumes uniform size
|
||||
int bytesPerBloom = this.matrix[0].getByteSize();
|
||||
|
||||
if(theBloom.limit() != bytesPerBloom * readMatrixSize) {
|
||||
throw new IllegalArgumentException("Bloom does not match expected size");
|
||||
}
|
||||
|
||||
ByteBuffer tmp = theBloom.duplicate();
|
||||
|
||||
// note: actually searching an array of blooms that have been serialized
|
||||
for (int m = 0; m < readMatrixSize; ++m) {
|
||||
tmp.position(m* bytesPerBloom);
|
||||
tmp.limit(tmp.position() + bytesPerBloom);
|
||||
boolean match = this.matrix[0].contains(buf, offset, length, tmp.slice());
|
||||
if (match) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
// matched no bloom filters
|
||||
return false;
|
||||
}
|
||||
|
||||
int bloomCount() {
|
||||
return Math.max(this.matrix.length, this.readMatrixSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getKeyCount() {
|
||||
return (bloomCount()-1) * this.keyInterval + this.curKeys;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxKeys() {
|
||||
return bloomCount() * this.keyInterval;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getByteSize() {
|
||||
return bloomCount() * this.matrix[0].getByteSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finalize() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a new row to <i>this</i> dynamic Bloom filter.
|
||||
*/
|
||||
private void addRow() {
|
||||
ByteBloomFilter[] tmp = new ByteBloomFilter[matrix.length + 1];
|
||||
|
||||
for (int i = 0; i < matrix.length; i++) {
|
||||
tmp[i] = matrix[i];
|
||||
}
|
||||
|
||||
tmp[tmp.length-1] = new ByteBloomFilter(keyInterval, errorRate, hashType, 0);
|
||||
tmp[tmp.length-1].allocBloom();
|
||||
matrix = tmp;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the currently-unfilled row in the dynamic Bloom Filter array.
|
||||
* @return BloomFilter The active standard Bloom filter.
|
||||
* <code>Null</code> otherwise.
|
||||
*/
|
||||
private BloomFilter getCurBloom() {
|
||||
if (curKeys >= keyInterval) {
|
||||
return null;
|
||||
}
|
||||
|
||||
return matrix[matrix.length - 1];
|
||||
}
|
||||
|
||||
@Override
|
||||
public Writable getMetaWriter() {
|
||||
return new MetaWriter();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Writable getDataWriter() {
|
||||
return new DataWriter();
|
||||
}
|
||||
|
||||
private class MetaWriter implements Writable {
|
||||
protected MetaWriter() {}
|
||||
@Override
|
||||
public void readFields(DataInput arg0) throws IOException {
|
||||
throw new IOException("Cant read with this class.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
out.writeInt(VERSION);
|
||||
out.writeInt(keyInterval);
|
||||
out.writeFloat(errorRate);
|
||||
out.writeInt(hashType);
|
||||
out.writeInt(matrix.length);
|
||||
out.writeInt(curKeys);
|
||||
}
|
||||
}
|
||||
|
||||
private class DataWriter implements Writable {
|
||||
protected DataWriter() {}
|
||||
@Override
|
||||
public void readFields(DataInput arg0) throws IOException {
|
||||
throw new IOException("Cant read with this class.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
for (int i = 0; i < matrix.length; ++i) {
|
||||
matrix[i].writeBloom(out);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,187 @@
|
|||
/*
|
||||
* Copyright 2010 The Apache Software Foundation
|
||||
*
|
||||
* 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.hadoop.hbase.util;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.BitSet;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
||||
public class TestByteBloomFilter extends TestCase {
|
||||
|
||||
public void testBasicBloom() throws Exception {
|
||||
ByteBloomFilter bf1 = new ByteBloomFilter(1000, (float)0.01, Hash.MURMUR_HASH, 0);
|
||||
ByteBloomFilter bf2 = new ByteBloomFilter(1000, (float)0.01, Hash.MURMUR_HASH, 0);
|
||||
bf1.allocBloom();
|
||||
bf2.allocBloom();
|
||||
|
||||
// test 1: verify no fundamental false negatives or positives
|
||||
byte[] key1 = {1,2,3,4,5,6,7,8,9};
|
||||
byte[] key2 = {1,2,3,4,5,6,7,8,7};
|
||||
|
||||
bf1.add(key1);
|
||||
bf2.add(key2);
|
||||
|
||||
assertTrue(bf1.contains(key1));
|
||||
assertFalse(bf1.contains(key2));
|
||||
assertFalse(bf2.contains(key1));
|
||||
assertTrue(bf2.contains(key2));
|
||||
|
||||
byte [] bkey = {1,2,3,4};
|
||||
byte [] bval = "this is a much larger byte array".getBytes();
|
||||
|
||||
bf1.add(bkey);
|
||||
bf1.add(bval, 1, bval.length-1);
|
||||
|
||||
assertTrue( bf1.contains(bkey) );
|
||||
assertTrue( bf1.contains(bval, 1, bval.length-1) );
|
||||
assertFalse( bf1.contains(bval) );
|
||||
assertFalse( bf1.contains(bval) );
|
||||
|
||||
// test 2: serialization & deserialization.
|
||||
// (convert bloom to byte array & read byte array back in as input)
|
||||
ByteArrayOutputStream bOut = new ByteArrayOutputStream();
|
||||
bf1.writeBloom(new DataOutputStream(bOut));
|
||||
ByteBuffer bb = ByteBuffer.wrap(bOut.toByteArray());
|
||||
ByteBloomFilter newBf1 = new ByteBloomFilter(1000, (float)0.01,
|
||||
Hash.MURMUR_HASH, 0);
|
||||
assertTrue(newBf1.contains(key1, bb));
|
||||
assertFalse(newBf1.contains(key2, bb));
|
||||
assertTrue( newBf1.contains(bkey, bb) );
|
||||
assertTrue( newBf1.contains(bval, 1, bval.length-1, bb) );
|
||||
assertFalse( newBf1.contains(bval, bb) );
|
||||
assertFalse( newBf1.contains(bval, bb) );
|
||||
|
||||
System.out.println("Serialized as " + bOut.size() + " bytes");
|
||||
assertTrue(bOut.size() - bf1.byteSize < 10); //... allow small padding
|
||||
}
|
||||
|
||||
public void testBloomFold() throws Exception {
|
||||
// test: foldFactor < log(max/actual)
|
||||
ByteBloomFilter b = new ByteBloomFilter(1003, (float)0.01, Hash.MURMUR_HASH, 2);
|
||||
b.allocBloom();
|
||||
int origSize = b.getByteSize();
|
||||
assertEquals(1204, origSize);
|
||||
for (int i = 0; i < 12; ++i) {
|
||||
b.add(Bytes.toBytes(i));
|
||||
}
|
||||
b.finalize();
|
||||
assertEquals(origSize>>2, b.getByteSize());
|
||||
int falsePositives = 0;
|
||||
for (int i = 0; i < 25; ++i) {
|
||||
if (b.contains(Bytes.toBytes(i))) {
|
||||
if(i >= 12) falsePositives++;
|
||||
} else {
|
||||
assertFalse(i < 12);
|
||||
}
|
||||
}
|
||||
assertTrue(falsePositives <= 1);
|
||||
|
||||
// test: foldFactor > log(max/actual)
|
||||
}
|
||||
|
||||
public void testBloomPerf() throws Exception {
|
||||
// add
|
||||
float err = (float)0.01;
|
||||
ByteBloomFilter b = new ByteBloomFilter(10*1000*1000, (float)err, Hash.MURMUR_HASH, 3);
|
||||
b.allocBloom();
|
||||
long startTime = System.currentTimeMillis();
|
||||
int origSize = b.getByteSize();
|
||||
for (int i = 0; i < 1*1000*1000; ++i) {
|
||||
b.add(Bytes.toBytes(i));
|
||||
}
|
||||
long endTime = System.currentTimeMillis();
|
||||
System.out.println("Total Add time = " + (endTime - startTime) + "ms");
|
||||
|
||||
// fold
|
||||
startTime = System.currentTimeMillis();
|
||||
b.finalize();
|
||||
endTime = System.currentTimeMillis();
|
||||
System.out.println("Total Fold time = " + (endTime - startTime) + "ms");
|
||||
assertTrue(origSize >= b.getByteSize()<<3);
|
||||
|
||||
// test
|
||||
startTime = System.currentTimeMillis();
|
||||
int falsePositives = 0;
|
||||
for (int i = 0; i < 2*1000*1000; ++i) {
|
||||
|
||||
if (b.contains(Bytes.toBytes(i))) {
|
||||
if(i >= 1*1000*1000) falsePositives++;
|
||||
} else {
|
||||
assertFalse(i < 1*1000*1000);
|
||||
}
|
||||
}
|
||||
endTime = System.currentTimeMillis();
|
||||
System.out.println("Total Contains time = " + (endTime - startTime) + "ms");
|
||||
System.out.println("False Positive = " + falsePositives);
|
||||
assertTrue(falsePositives <= (1*1000*1000)*err);
|
||||
|
||||
// test: foldFactor > log(max/actual)
|
||||
}
|
||||
|
||||
public void testDynamicBloom() throws Exception {
|
||||
int keyInterval = 1000;
|
||||
float err = (float)0.01;
|
||||
BitSet valid = new BitSet(keyInterval*4);
|
||||
|
||||
DynamicByteBloomFilter bf1 = new DynamicByteBloomFilter(keyInterval, err,
|
||||
Hash.MURMUR_HASH);
|
||||
bf1.allocBloom();
|
||||
|
||||
for (int i = 0; i < keyInterval*4; ++i) { // add
|
||||
if (Math.random() > 0.5) {
|
||||
bf1.add(Bytes.toBytes(i));
|
||||
valid.set(i);
|
||||
}
|
||||
}
|
||||
assertTrue(2 <= bf1.bloomCount() && bf1.bloomCount() <= 3);
|
||||
|
||||
// test serialization/deserialization
|
||||
ByteArrayOutputStream metaOut = new ByteArrayOutputStream();
|
||||
ByteArrayOutputStream dataOut = new ByteArrayOutputStream();
|
||||
bf1.getMetaWriter().write(new DataOutputStream(metaOut));
|
||||
bf1.getDataWriter().write(new DataOutputStream(dataOut));
|
||||
ByteBuffer bb = ByteBuffer.wrap(dataOut.toByteArray());
|
||||
DynamicByteBloomFilter newBf1 = new DynamicByteBloomFilter(
|
||||
ByteBuffer.wrap(metaOut.toByteArray()));
|
||||
|
||||
int falsePositives = 0;
|
||||
for (int i = 0; i < keyInterval*4; ++i) { // check
|
||||
if (newBf1.contains(Bytes.toBytes(i), bb)) {
|
||||
if (!valid.get(i)) ++falsePositives;
|
||||
} else {
|
||||
if (valid.get(i)) {
|
||||
assert false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// note that actualErr = err * bloomCount
|
||||
// error rate should be roughly: (keyInterval*2)*(err*2), allow some tolerance
|
||||
System.out.println("False positives: " + falsePositives);
|
||||
assertTrue(falsePositives <= (keyInterval*5)*err);
|
||||
}
|
||||
|
||||
}
|
Loading…
Reference in New Issue