From 5e7e626ef54ada9e75b18b31bb49e23b22ae9fe9 Mon Sep 17 00:00:00 2001 From: ramkrishna Date: Tue, 19 May 2015 10:22:56 +0530 Subject: [PATCH] HBASE-13510 - Purge ByteBloomFilter (Ram) --- .../hbase/io/hfile/HFilePrettyPrinter.java | 6 +- .../hadoop/hbase/regionserver/StoreFile.java | 56 +- .../apache/hadoop/hbase/util/BloomFilter.java | 59 +- .../hadoop/hbase/util/BloomFilterBase.java | 7 - .../hadoop/hbase/util/BloomFilterChunk.java | 322 +++++++++ .../hadoop/hbase/util/BloomFilterFactory.java | 7 - .../hadoop/hbase/util/BloomFilterUtil.java | 269 +++++++ .../hadoop/hbase/util/BloomFilterWriter.java | 4 - .../hadoop/hbase/util/ByteBloomFilter.java | 654 ------------------ .../hbase/util/CompoundBloomFilter.java | 32 +- .../hbase/util/CompoundBloomFilterBase.java | 22 - .../hbase/util/CompoundBloomFilterWriter.java | 25 +- .../regionserver/TestCompoundBloomFilter.java | 19 +- ...mFilter.java => TestBloomFilterChunk.java} | 68 +- 14 files changed, 760 insertions(+), 790 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterChunk.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java rename hbase-server/src/test/java/org/apache/hadoop/hbase/util/{TestByteBloomFilter.java => TestBloomFilterChunk.java} (62%) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java index 7463e8326ab..7cc31d015a0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java @@ -59,8 +59,8 @@ import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; import org.apache.hadoop.hbase.util.BloomFilter; +import org.apache.hadoop.hbase.util.BloomFilterUtil; import org.apache.hadoop.hbase.util.BloomFilterFactory; -import org.apache.hadoop.hbase.util.ByteBloomFilter; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Writables; @@ -424,7 +424,7 @@ public class HFilePrettyPrinter extends Configured implements Tool { System.out.println("Bloom filter:"); if (bloomFilter != null) { System.out.println(FOUR_SPACES + bloomFilter.toString().replaceAll( - ByteBloomFilter.STATS_RECORD_SEP, "\n" + FOUR_SPACES)); + BloomFilterUtil.STATS_RECORD_SEP, "\n" + FOUR_SPACES)); } else { System.out.println(FOUR_SPACES + "Not present"); } @@ -438,7 +438,7 @@ public class HFilePrettyPrinter extends Configured implements Tool { System.out.println("Delete Family Bloom filter:"); if (bloomFilter != null) { System.out.println(FOUR_SPACES - + bloomFilter.toString().replaceAll(ByteBloomFilter.STATS_RECORD_SEP, + + bloomFilter.toString().replaceAll(BloomFilterUtil.STATS_RECORD_SEP, "\n" + FOUR_SPACES)); } else { System.out.println(FOUR_SPACES + "Not present"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java index 1992479e129..eba3689a801 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java @@ -707,7 +707,6 @@ public class StoreFile { private final BloomType bloomType; private byte[] lastBloomKey; private int lastBloomKeyOffset, lastBloomKeyLen; - private CellComparator kvComparator; private Cell lastCell = null; private long earliestPutTs = HConstants.LATEST_TIMESTAMP; private Cell lastDeleteFamilyCell = null; @@ -754,8 +753,6 @@ public class StoreFile { .withFileContext(fileContext) .create(); - this.kvComparator = comparator; - generalBloomFilterWriter = BloomFilterFactory.createGeneralBloomAtWrite( conf, cacheConf, bloomType, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer); @@ -864,7 +861,9 @@ public class StoreFile { * 1. Row = Row * 2. RowCol = Row + Qualifier */ - byte[] bloomKey; + byte[] bloomKey = null; + // Used with ROW_COL bloom + KeyValue bloomKeyKV = null; int bloomKeyOffset, bloomKeyLen; switch (bloomType) { @@ -877,11 +876,14 @@ public class StoreFile { // merge(row, qualifier) // TODO: could save one buffer copy in case of compound Bloom // filters when this involves creating a KeyValue - bloomKey = generalBloomFilterWriter.createBloomKey(cell.getRowArray(), - cell.getRowOffset(), cell.getRowLength(), cell.getQualifierArray(), - cell.getQualifierOffset(), cell.getQualifierLength()); - bloomKeyOffset = 0; - bloomKeyLen = bloomKey.length; + bloomKeyKV = KeyValueUtil.createFirstOnRow(cell.getRowArray(), cell.getRowOffset(), + cell.getRowLength(), + HConstants.EMPTY_BYTE_ARRAY, 0, 0, cell.getQualifierArray(), + cell.getQualifierOffset(), + cell.getQualifierLength()); + bloomKey = bloomKeyKV.getBuffer(); + bloomKeyOffset = bloomKeyKV.getKeyOffset(); + bloomKeyLen = bloomKeyKV.getKeyLength(); break; default: throw new IOException("Invalid Bloom filter type: " + bloomType + @@ -889,17 +891,17 @@ public class StoreFile { } generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen); if (lastBloomKey != null) { - boolean res = false; + int res = 0; // hbase:meta does not have blooms. So we need not have special interpretation // of the hbase:meta cells. We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom if (bloomType == BloomType.ROW) { res = Bytes.BYTES_RAWCOMPARATOR.compare(bloomKey, bloomKeyOffset, bloomKeyLen, - lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen) <= 0; + lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen); } else { - res = (CellComparator.COMPARATOR.compare(lastBloomKeyOnlyKV, bloomKey, - bloomKeyOffset, bloomKeyLen) >= 0); + // TODO : Caching of kv components becomes important in these cases + res = CellComparator.COMPARATOR.compare(bloomKeyKV, lastBloomKeyOnlyKV); } - if (res) { + if (res <= 0) { throw new IOException("Non-increasing Bloom keys: " + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen) + " after " + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen)); @@ -1252,7 +1254,10 @@ public class StoreFile { return true; } - byte[] key; + // Used in ROW bloom + byte[] key = null; + // Used in ROW_COL bloom + KeyValue kvKey = null; switch (bloomFilterType) { case ROW: if (col != null) { @@ -1267,8 +1272,9 @@ public class StoreFile { break; case ROWCOL: - key = bloomFilter.createBloomKey(row, rowOffset, rowLen, col, - colOffset, colLen); + kvKey = KeyValueUtil.createFirstOnRow(row, rowOffset, rowLen, + HConstants.EMPTY_BYTE_ARRAY, 0, 0, col, colOffset, + colLen); break; default: @@ -1304,9 +1310,7 @@ public class StoreFile { if (bloomFilterType == BloomType.ROW) { keyIsAfterLast = (Bytes.BYTES_RAWCOMPARATOR.compare(key, lastBloomKey) > 0); } else { - // TODO : Convert key to Cell so that we could use compare(Cell, Cell) - keyIsAfterLast = (CellComparator.COMPARATOR.compare(lastBloomKeyOnlyKV, key, 0, - key.length)) < 0; + keyIsAfterLast = (CellComparator.COMPARATOR.compare(kvKey, lastBloomKeyOnlyKV)) > 0; } } @@ -1315,19 +1319,17 @@ public class StoreFile { // columns, a file might be skipped if using row+col Bloom filter. // In order to ensure this file is included an additional check is // required looking only for a row bloom. - byte[] rowBloomKey = bloomFilter.createBloomKey(row, rowOffset, rowLen, - null, 0, 0); + KeyValue rowBloomKey = KeyValueUtil.createFirstOnRow(row, rowOffset, rowLen, + HConstants.EMPTY_BYTE_ARRAY, 0, 0, HConstants.EMPTY_BYTE_ARRAY, 0, 0); // hbase:meta does not have blooms. So we need not have special interpretation // of the hbase:meta cells. We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom if (keyIsAfterLast - && (CellComparator.COMPARATOR.compare(lastBloomKeyOnlyKV, rowBloomKey, 0, - rowBloomKey.length)) < 0) { + && (CellComparator.COMPARATOR.compare(rowBloomKey, lastBloomKeyOnlyKV)) > 0) { exists = false; } else { exists = - bloomFilter.contains(key, 0, key.length, bloom) || - bloomFilter.contains(rowBloomKey, 0, rowBloomKey.length, - bloom); + bloomFilter.contains(kvKey, bloom) || + bloomFilter.contains(rowBloomKey, bloom); } } else { exists = !keyIsAfterLast diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java index b314bf62fed..f1190860bdc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java @@ -20,24 +20,57 @@ package org.apache.hadoop.hbase.util; import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.classification.InterfaceAudience; /** - * Defines the general behavior of a bloom filter. * + * Implements a Bloom filter, as defined by Bloom in 1970. *

- * 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 + * 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 + * 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. * + *

+ * Originally inspired by European Commission + * One-Lab Project 034819. + * + * 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). + * + * @see BloomFilter The general behavior of a filter + * + * @see + * Space/Time Trade-Offs in Hash Coding with Allowable Errors + * * @see BloomFilterWriter for the ability to add elements to a Bloom filter */ @InterfaceAudience.Private @@ -45,7 +78,17 @@ public interface BloomFilter extends BloomFilterBase { /** * Check if the specified key is contained in the bloom filter. - * + * Used in ROW_COL blooms where the blooms are serialized as KeyValues + * @param keyCell the key to check for the existence of + * @param bloom bloom filter data to search. This can be null if auto-loading + * is supported. + * @return true if matched by bloom, false if not + */ + boolean contains(Cell keyCell, ByteBuffer bloom); + + /** + * Check if the specified key is contained in the bloom filter. + * Used in ROW bloom where the blooms are just plain byte[] * @param buf data to check for existence of * @param offset offset into the data * @param length length of the data diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java index 0396965a17d..43cf75d25d7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java @@ -41,11 +41,4 @@ public interface BloomFilterBase { * @return Size of the bloom, in bytes */ long getByteSize(); - - /** - * Create a key for a row-column Bloom filter. - */ - byte[] createBloomKey(byte[] rowBuf, int rowOffset, int rowLen, - byte[] qualBuf, int qualOffset, int qualLen); - } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterChunk.java new file mode 100644 index 00000000000..a80a2018b2a --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterChunk.java @@ -0,0 +1,322 @@ +/* + * + * 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.hbase.classification.InterfaceAudience; + +import com.google.common.annotations.VisibleForTesting; + +/** + * The basic building block for the {@link CompoundBloomFilter} + */ +@InterfaceAudience.Private +public class BloomFilterChunk implements BloomFilterBase { + + /** Bytes (B) in the array. This actually has to fit into an int. */ + protected long byteSize; + /** Number of hash functions */ + protected 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; + + /** + * Loads bloom filter meta data from file input. + * @param meta stored bloom meta data + * @throws IllegalArgumentException meta data is invalid + */ + public BloomFilterChunk(DataInput meta) + throws IOException, IllegalArgumentException { + this.byteSize = meta.readInt(); + this.hashCount = meta.readInt(); + this.hashType = meta.readInt(); + this.keyCount = meta.readInt(); + this.maxKeys = this.keyCount; + + this.hash = Hash.getInstance(this.hashType); + if (hash == null) { + throw new IllegalArgumentException("Invalid hash type: " + hashType); + } + sanityCheck(); + } + + /** + * Computes the error rate for this Bloom filter, taking into account the + * actual number of hash functions and keys inserted. The return value of + * this function changes as a Bloom filter is being populated. Used for + * reporting the actual error rate of compound Bloom filters when writing + * them out. + * + * @return error rate for this particular Bloom filter + */ + public double actualErrorRate() { + return BloomFilterUtil.actualErrorRate(keyCount, byteSize * 8, hashCount); + } + + public BloomFilterChunk(int hashType) { + this.hashType = hashType; + this.hash = Hash.getInstance(hashType); + } + + /** + * 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 BloomFilterChunk(int maxKeys, double errorRate, int hashType, + int foldFactor) throws IllegalArgumentException { + this(hashType); + + long bitSize = BloomFilterUtil.computeBitSize(maxKeys, errorRate); + hashCount = BloomFilterUtil.optimalFunctionCount(maxKeys, bitSize); + this.maxKeys = maxKeys; + + // increase byteSize so folding is possible + byteSize = BloomFilterUtil.computeFoldableByteSize(bitSize, foldFactor); + + sanityCheck(); + } + + /** + * Creates another similar Bloom filter. Does not copy the actual bits, and + * sets the new filter's key count to zero. + * + * @return a Bloom filter with the same configuration as this + */ + public BloomFilterChunk createAnother() { + BloomFilterChunk bbf = new BloomFilterChunk(hashType); + bbf.byteSize = byteSize; + bbf.hashCount = hashCount; + bbf.maxKeys = maxKeys; + return bbf; + } + + public void allocBloom() { + if (this.bloom != null) { + throw new IllegalArgumentException("can only create bloom once."); + } + this.bloom = ByteBuffer.allocate((int)this.byteSize); + assert this.bloom.hasArray(); + } + + void sanityCheck() throws IllegalArgumentException { + if(0 >= this.byteSize || this.byteSize > Integer.MAX_VALUE) { + throw new IllegalArgumentException("Invalid byteSize: " + this.byteSize); + } + + 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"); + } + } + + public void add(byte [] buf) { + add(buf, 0, buf.length); + } + + 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++) { + long hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8)); + set(hashLoc); + } + + ++this.keyCount; + } + + @VisibleForTesting + boolean contains(byte [] buf) { + return contains(buf, 0, buf.length, this.bloom); + } + + @VisibleForTesting + boolean contains(byte [] buf, int offset, int length) { + return contains(buf, offset, length, bloom); + } + + @VisibleForTesting + boolean contains(byte[] buf, ByteBuffer bloom) { + return contains(buf, 0, buf.length, bloom); + } + + public boolean contains(byte[] buf, int offset, int length, ByteBuffer theBloom) { + if (theBloom == null) { + theBloom = bloom; + } + + if (theBloom.limit() != byteSize) { + throw new IllegalArgumentException("Bloom does not match expected size:" + + " theBloom.limit()=" + theBloom.limit() + ", byteSize=" + byteSize); + } + + return BloomFilterUtil.contains(buf, offset, length, theBloom, 0, (int) byteSize, hash, + hashCount); + } + + //--------------------------------------------------------------------------- + /** Private helpers */ + + /** + * Set the bit at the specified index to 1. + * + * @param pos index of bit + */ + void set(long pos) { + int bytePos = (int)(pos / 8); + int bitPos = (int)(pos % 8); + byte curByte = bloom.get(bytePos); + curByte |= BloomFilterUtil.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 bloomBuf, int bloomOffset) { + int bytePos = pos >> 3; //pos / 8 + int bitPos = pos & 0x7; //pos % 8 + // TODO access this via Util API which can do Unsafe access if possible(?) + byte curByte = bloomBuf.get(bloomOffset + bytePos); + curByte &= BloomFilterUtil.bitvals[bitPos]; + return (curByte != 0); + } + + @Override + public long getKeyCount() { + return keyCount; + } + + @Override + public long getMaxKeys() { + return maxKeys; + } + + @Override + public long getByteSize() { + return byteSize; + } + + public int getHashType() { + return hashType; + } + + public void compactBloom() { + // see if the actual size is exponentially smaller than expected. + if (this.keyCount > 0 && this.bloom.hasArray()) { + int pieces = 1; + int newByteSize = (int)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(this.bloom.array(), this.bloom.arrayOffset(), this.bloom.limit()); + } + + public int getHashCount() { + return hashCount; + } + + @Override + public String toString() { + return BloomFilterUtil.toString(this); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java index 1a63f6be3dd..aecbdf81e10 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java @@ -99,13 +99,6 @@ public final class BloomFilterFactory { throws IllegalArgumentException, IOException { int version = meta.readInt(); switch (version) { - case ByteBloomFilter.VERSION: - // This is only possible in a version 1 HFile. We are ignoring the - // passed comparator because raw byte comparators are always used - // in version 1 Bloom filters. - // TODO:Remove this code - use only CompoundBloomFilter - return new ByteBloomFilter(meta); - case CompoundBloomFilterBase.VERSION: return new CompoundBloomFilter(meta, reader); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java new file mode 100644 index 00000000000..29b08d28e6d --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java @@ -0,0 +1,269 @@ +/* + * 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.nio.ByteBuffer; +import java.text.NumberFormat; +import java.util.Random; + +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * Utility methods related to BloomFilters + */ +@InterfaceAudience.Private +public class BloomFilterUtil { + + /** Record separator for the Bloom filter statistics human-readable string */ + public static final String STATS_RECORD_SEP = "; "; + /** + * 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); + + /** + * A random number generator to use for "fake lookups" when testing to + * estimate the ideal false positive rate. + */ + private static Random randomGeneratorForTest; + + /** Bit-value lookup array to prevent doing the same work over and over */ + public static final byte [] bitvals = { + (byte) 0x01, + (byte) 0x02, + (byte) 0x04, + (byte) 0x08, + (byte) 0x10, + (byte) 0x20, + (byte) 0x40, + (byte) 0x80 + }; + /** + * @param maxKeys + * @param errorRate + * @return the number of bits for a Bloom filter than can hold the given + * number of keys and provide the given error rate, assuming that the + * optimal number of hash functions is used and it does not have to + * be an integer. + */ + public static long computeBitSize(long maxKeys, double errorRate) { + return (long) Math.ceil(maxKeys * (-Math.log(errorRate) / LOG2_SQUARED)); + } + + public static void setFakeLookupMode(boolean enabled) { + if (enabled) { + randomGeneratorForTest = new Random(283742987L); + } else { + randomGeneratorForTest = null; + } + } + + /** + * The maximum number of keys we can put into a Bloom filter of a certain + * size to maintain the given error rate, assuming the number of hash + * functions is chosen optimally and does not even have to be an integer + * (hence the "ideal" in the function name). + * + * @param bitSize + * @param errorRate + * @return maximum number of keys that can be inserted into the Bloom filter + * @see #computeMaxKeys(long, double, int) for a more precise estimate + */ + public static long idealMaxKeys(long bitSize, double errorRate) { + // The reason we need to use floor here is that otherwise we might put + // more keys in a Bloom filter than is allowed by the target error rate. + return (long) (bitSize * (LOG2_SQUARED / -Math.log(errorRate))); + } + + /** + * The maximum number of keys we can put into a Bloom filter of a certain + * size to get the given error rate, with the given number of hash functions. + * + * @param bitSize + * @param errorRate + * @param hashCount + * @return the maximum number of keys that can be inserted in a Bloom filter + * to maintain the target error rate, if the number of hash functions + * is provided. + */ + public static long computeMaxKeys(long bitSize, double errorRate, + int hashCount) { + return (long) (-bitSize * 1.0 / hashCount * + Math.log(1 - Math.exp(Math.log(errorRate) / hashCount))); + } + + /** + * Computes the actual error rate for the given number of elements, number + * of bits, and number of hash functions. Taken directly from the + * Wikipedia Bloom filter article. + * + * @param maxKeys + * @param bitSize + * @param functionCount + * @return the actual error rate + */ + public static double actualErrorRate(long maxKeys, long bitSize, + int functionCount) { + return Math.exp(Math.log(1 - Math.exp(-functionCount * maxKeys * 1.0 + / bitSize)) * functionCount); + } + + /** + * Increases the given byte size of a Bloom filter until it can be folded by + * the given factor. + * + * @param bitSize + * @param foldFactor + * @return Foldable byte size + */ + public static int computeFoldableByteSize(long bitSize, int foldFactor) { + long byteSizeLong = (bitSize + 7) / 8; + int mask = (1 << foldFactor) - 1; + if ((mask & byteSizeLong) != 0) { + byteSizeLong >>= foldFactor; + ++byteSizeLong; + byteSizeLong <<= foldFactor; + } + if (byteSizeLong > Integer.MAX_VALUE) { + throw new IllegalArgumentException("byteSize=" + byteSizeLong + " too " + + "large for bitSize=" + bitSize + ", foldFactor=" + foldFactor); + } + return (int) byteSizeLong; + } + + public static int optimalFunctionCount(int maxKeys, long bitSize) { + long i = bitSize / maxKeys; + double result = Math.ceil(Math.log(2) * i); + if (result > Integer.MAX_VALUE){ + throw new IllegalArgumentException("result too large for integer value."); + } + return (int)result; + } + + /** + * Creates a Bloom filter chunk of the given size. + * + * @param byteSizeHint the desired number of bytes for the Bloom filter bit + * array. Will be increased so that folding is possible. + * @param errorRate target false positive rate of the Bloom filter + * @param hashType Bloom filter hash function type + * @param foldFactor + * @return the new Bloom filter of the desired size + */ + public static BloomFilterChunk createBySize(int byteSizeHint, + double errorRate, int hashType, int foldFactor) { + BloomFilterChunk bbf = new BloomFilterChunk(hashType); + + bbf.byteSize = computeFoldableByteSize(byteSizeHint * 8L, foldFactor); + long bitSize = bbf.byteSize * 8; + bbf.maxKeys = (int) idealMaxKeys(bitSize, errorRate); + bbf.hashCount = optimalFunctionCount(bbf.maxKeys, bitSize); + + // Adjust max keys to bring error rate closer to what was requested, + // because byteSize was adjusted to allow for folding, and hashCount was + // rounded. + bbf.maxKeys = (int) computeMaxKeys(bitSize, errorRate, bbf.hashCount); + + return bbf; + } + + /** Should only be used in tests */ + public static boolean contains(byte[] buf, int offset, int length, ByteBuffer bloom) { + return contains(buf, offset, length, bloom); + } + + /** Should only be used in tests */ + boolean contains(byte[] buf, ByteBuffer bloom) { + return contains(buf, 0, buf.length, bloom); + } + + public static boolean contains(byte[] buf, int offset, int length, + ByteBuffer bloomBuf, int bloomOffset, int bloomSize, Hash hash, + int hashCount) { + + int hash1 = hash.hash(buf, offset, length, 0); + int hash2 = hash.hash(buf, offset, length, hash1); + int bloomBitSize = bloomSize << 3; + + if (randomGeneratorForTest == null) { + // Production mode. + int compositeHash = hash1; + for (int i = 0; i < hashCount; i++) { + int hashLoc = Math.abs(compositeHash % bloomBitSize); + compositeHash += hash2; + if (!get(hashLoc, bloomBuf, bloomOffset)) { + return false; + } + } + } else { + // Test mode with "fake lookups" to estimate "ideal false positive rate". + for (int i = 0; i < hashCount; i++) { + int hashLoc = randomGeneratorForTest.nextInt(bloomBitSize); + if (!get(hashLoc, bloomBuf, bloomOffset)){ + return false; + } + } + } + return true; + } + + /** + * 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. + */ + public static boolean get(int pos, ByteBuffer bloomBuf, int bloomOffset) { + int bytePos = pos >> 3; //pos / 8 + int bitPos = pos & 0x7; //pos % 8 + // TODO access this via Util API which can do Unsafe access if possible(?) + byte curByte = bloomBuf.get(bloomOffset + bytePos); + curByte &= bitvals[bitPos]; + return (curByte != 0); + } + + /** + * A human-readable string with statistics for the given Bloom filter. + * + * @param bloomFilter the Bloom filter to output statistics for; + * @return a string consisting of "<key>: <value>" parts + * separated by {@link #STATS_RECORD_SEP}. + */ + public static String formatStats(BloomFilterBase bloomFilter) { + StringBuilder sb = new StringBuilder(); + long k = bloomFilter.getKeyCount(); + long m = bloomFilter.getMaxKeys(); + + sb.append("BloomSize: " + bloomFilter.getByteSize() + STATS_RECORD_SEP); + sb.append("No of Keys in bloom: " + k + STATS_RECORD_SEP); + sb.append("Max Keys for bloom: " + m); + if (m > 0) { + sb.append(STATS_RECORD_SEP + "Percentage filled: " + + NumberFormat.getPercentInstance().format(k * 1.0 / m)); + } + return sb.toString(); + } + + public static String toString(BloomFilterChunk bloomFilter) { + return formatStats(bloomFilter) + STATS_RECORD_SEP + "Actual error rate: " + + String.format("%.8f", bloomFilter.actualErrorRate()); + } +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java index aa7f503e299..2aba737b575 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java @@ -29,12 +29,8 @@ import org.apache.hadoop.io.Writable; @InterfaceAudience.Private public interface BloomFilterWriter extends BloomFilterBase { - /** Allocate memory for the bloom filter data. */ - void allocBloom(); - /** Compact the Bloom filter before writing metadata & data to disk. */ void compactBloom(); - /** * Get a writable interface into bloom filter meta data. * diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java deleted file mode 100644 index 1ea81ddd627..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java +++ /dev/null @@ -1,654 +0,0 @@ -/* - * - * 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 java.text.NumberFormat; -import java.util.Random; - -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.io.Writable; - -/** - * Implements a Bloom filter, as defined by Bloom in 1970. - *

- * 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. - * - *

- * Originally inspired by European Commission - * One-Lab Project 034819. - * - * 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). - * - * @see BloomFilter The general behavior of a filter - * - * @see - * Space/Time Trade-Offs in Hash Coding with Allowable Errors - */ -@InterfaceAudience.Private -// TODO : Remove this ByteBloomFilter as an instance of BloomFilter -public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { - - /** Current file format version */ - public static final int VERSION = 1; - - /** Bytes (B) in the array. This actually has to fit into an int. */ - protected long byteSize; - /** Number of hash functions */ - protected 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; - - /** Record separator for the Bloom filter statistics human-readable string */ - public static final String STATS_RECORD_SEP = "; "; - - /** - * 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); - - /** - * A random number generator to use for "fake lookups" when testing to - * estimate the ideal false positive rate. - */ - private static Random randomGeneratorForTest; - - /** 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(DataInput meta) - throws IOException, IllegalArgumentException { - this.byteSize = meta.readInt(); - this.hashCount = meta.readInt(); - this.hashType = meta.readInt(); - this.keyCount = meta.readInt(); - this.maxKeys = this.keyCount; - - this.hash = Hash.getInstance(this.hashType); - if (hash == null) { - throw new IllegalArgumentException("Invalid hash type: " + hashType); - } - sanityCheck(); - } - - /** - * @param maxKeys - * @param errorRate - * @return the number of bits for a Bloom filter than can hold the given - * number of keys and provide the given error rate, assuming that the - * optimal number of hash functions is used and it does not have to - * be an integer. - */ - public static long computeBitSize(long maxKeys, double errorRate) { - return (long) Math.ceil(maxKeys * (-Math.log(errorRate) / LOG2_SQUARED)); - } - - /** - * The maximum number of keys we can put into a Bloom filter of a certain - * size to maintain the given error rate, assuming the number of hash - * functions is chosen optimally and does not even have to be an integer - * (hence the "ideal" in the function name). - * - * @param bitSize - * @param errorRate - * @return maximum number of keys that can be inserted into the Bloom filter - * @see #computeMaxKeys(long, double, int) for a more precise estimate - */ - public static long idealMaxKeys(long bitSize, double errorRate) { - // The reason we need to use floor here is that otherwise we might put - // more keys in a Bloom filter than is allowed by the target error rate. - return (long) (bitSize * (LOG2_SQUARED / -Math.log(errorRate))); - } - - /** - * The maximum number of keys we can put into a Bloom filter of a certain - * size to get the given error rate, with the given number of hash functions. - * - * @param bitSize - * @param errorRate - * @param hashCount - * @return the maximum number of keys that can be inserted in a Bloom filter - * to maintain the target error rate, if the number of hash functions - * is provided. - */ - public static long computeMaxKeys(long bitSize, double errorRate, - int hashCount) { - return (long) (-bitSize * 1.0 / hashCount * - Math.log(1 - Math.exp(Math.log(errorRate) / hashCount))); - } - - /** - * Computes the error rate for this Bloom filter, taking into account the - * actual number of hash functions and keys inserted. The return value of - * this function changes as a Bloom filter is being populated. Used for - * reporting the actual error rate of compound Bloom filters when writing - * them out. - * - * @return error rate for this particular Bloom filter - */ - public double actualErrorRate() { - return actualErrorRate(keyCount, byteSize * 8, hashCount); - } - - /** - * Computes the actual error rate for the given number of elements, number - * of bits, and number of hash functions. Taken directly from the - * Wikipedia Bloom filter article. - * - * @param maxKeys - * @param bitSize - * @param functionCount - * @return the actual error rate - */ - public static double actualErrorRate(long maxKeys, long bitSize, - int functionCount) { - return Math.exp(Math.log(1 - Math.exp(-functionCount * maxKeys * 1.0 - / bitSize)) * functionCount); - } - - /** - * Increases the given byte size of a Bloom filter until it can be folded by - * the given factor. - * - * @param bitSize - * @param foldFactor - * @return Foldable byte size - */ - public static int computeFoldableByteSize(long bitSize, int foldFactor) { - long byteSizeLong = (bitSize + 7) / 8; - int mask = (1 << foldFactor) - 1; - if ((mask & byteSizeLong) != 0) { - byteSizeLong >>= foldFactor; - ++byteSizeLong; - byteSizeLong <<= foldFactor; - } - if (byteSizeLong > Integer.MAX_VALUE) { - throw new IllegalArgumentException("byteSize=" + byteSizeLong + " too " - + "large for bitSize=" + bitSize + ", foldFactor=" + foldFactor); - } - return (int) byteSizeLong; - } - - private static int optimalFunctionCount(int maxKeys, long bitSize) { - long i = bitSize / maxKeys; - double result = Math.ceil(Math.log(2) * i); - if (result > Integer.MAX_VALUE){ - throw new IllegalArgumentException("result too large for integer value."); - } - return (int)result; - } - - /** Private constructor used by other constructors. */ - private ByteBloomFilter(int hashType) { - this.hashType = hashType; - this.hash = Hash.getInstance(hashType); - } - - /** - * 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, double errorRate, int hashType, - int foldFactor) throws IllegalArgumentException { - this(hashType); - - long bitSize = computeBitSize(maxKeys, errorRate); - hashCount = optimalFunctionCount(maxKeys, bitSize); - this.maxKeys = maxKeys; - - // increase byteSize so folding is possible - byteSize = computeFoldableByteSize(bitSize, foldFactor); - - sanityCheck(); - } - - /** - * Creates a Bloom filter of the given size. - * - * @param byteSizeHint the desired number of bytes for the Bloom filter bit - * array. Will be increased so that folding is possible. - * @param errorRate target false positive rate of the Bloom filter - * @param hashType Bloom filter hash function type - * @param foldFactor - * @return the new Bloom filter of the desired size - */ - public static ByteBloomFilter createBySize(int byteSizeHint, - double errorRate, int hashType, int foldFactor) { - ByteBloomFilter bbf = new ByteBloomFilter(hashType); - - bbf.byteSize = computeFoldableByteSize(byteSizeHint * 8L, foldFactor); - long bitSize = bbf.byteSize * 8; - bbf.maxKeys = (int) idealMaxKeys(bitSize, errorRate); - bbf.hashCount = optimalFunctionCount(bbf.maxKeys, bitSize); - - // Adjust max keys to bring error rate closer to what was requested, - // because byteSize was adjusted to allow for folding, and hashCount was - // rounded. - bbf.maxKeys = (int) computeMaxKeys(bitSize, errorRate, bbf.hashCount); - - return bbf; - } - - /** - * Creates another similar Bloom filter. Does not copy the actual bits, and - * sets the new filter's key count to zero. - * - * @return a Bloom filter with the same configuration as this - */ - public ByteBloomFilter createAnother() { - ByteBloomFilter bbf = new ByteBloomFilter(hashType); - bbf.byteSize = byteSize; - bbf.hashCount = hashCount; - bbf.maxKeys = maxKeys; - return bbf; - } - - @Override - public void allocBloom() { - if (this.bloom != null) { - throw new IllegalArgumentException("can only create bloom once."); - } - this.bloom = ByteBuffer.allocate((int)this.byteSize); - assert this.bloom.hasArray(); - } - - void sanityCheck() throws IllegalArgumentException { - if(0 >= this.byteSize || this.byteSize > Integer.MAX_VALUE) { - throw new IllegalArgumentException("Invalid byteSize: " + this.byteSize); - } - - 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"); - } - } - - 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++) { - long hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8)); - set(hashLoc); - } - - ++this.keyCount; - } - - /** Should only be used in tests */ - boolean contains(byte [] buf) { - return contains(buf, 0, buf.length, this.bloom); - } - - /** Should only be used in tests */ - boolean contains(byte [] buf, int offset, int length) { - return contains(buf, offset, length, bloom); - } - - /** Should only be used in tests */ - boolean contains(byte[] buf, ByteBuffer bloom) { - return contains(buf, 0, buf.length, bloom); - } - - @Override - public boolean contains(byte[] buf, int offset, int length, ByteBuffer theBloom) { - if (theBloom == null) { - // In a version 1 HFile Bloom filter data is stored in a separate meta - // block which is loaded on demand, but in version 2 it is pre-loaded. - // We want to use the same API in both cases. - theBloom = bloom; - } - - if (theBloom.limit() != byteSize) { - throw new IllegalArgumentException("Bloom does not match expected size:" - + " theBloom.limit()=" + theBloom.limit() + ", byteSize=" + byteSize); - } - - return contains(buf, offset, length, theBloom, 0, (int) byteSize, hash, hashCount); - } - - public static boolean contains(byte[] buf, int offset, int length, - ByteBuffer bloomBuf, int bloomOffset, int bloomSize, Hash hash, - int hashCount) { - - int hash1 = hash.hash(buf, offset, length, 0); - int hash2 = hash.hash(buf, offset, length, hash1); - int bloomBitSize = bloomSize << 3; - - if (randomGeneratorForTest == null) { - // Production mode. - int compositeHash = hash1; - for (int i = 0; i < hashCount; i++) { - int hashLoc = Math.abs(compositeHash % bloomBitSize); - compositeHash += hash2; - if (!get(hashLoc, bloomBuf, bloomOffset)) { - return false; - } - } - } else { - // Test mode with "fake lookups" to estimate "ideal false positive rate". - for (int i = 0; i < hashCount; i++) { - int hashLoc = randomGeneratorForTest.nextInt(bloomBitSize); - if (!get(hashLoc, bloomBuf, bloomOffset)){ - return false; - } - } - } - return true; - } - - //--------------------------------------------------------------------------- - /** Private helpers */ - - /** - * Set the bit at the specified index to 1. - * - * @param pos index of bit - */ - void set(long pos) { - int bytePos = (int)(pos / 8); - int bitPos = (int)(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 bloomBuf, int bloomOffset) { - int bytePos = pos >> 3; //pos / 8 - int bitPos = pos & 0x7; //pos % 8 - // TODO access this via Util API which can do Unsafe access if possible(?) - byte curByte = bloomBuf.get(bloomOffset + bytePos); - curByte &= bitvals[bitPos]; - return (curByte != 0); - } - - @Override - public long getKeyCount() { - return keyCount; - } - - @Override - public long getMaxKeys() { - return maxKeys; - } - - @Override - public long getByteSize() { - return byteSize; - } - - public int getHashType() { - return hashType; - } - - @Override - public void compactBloom() { - // see if the actual size is exponentially smaller than expected. - if (this.keyCount > 0 && this.bloom.hasArray()) { - int pieces = 1; - int newByteSize = (int)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((int) 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); - } - } - - public int getHashCount() { - return hashCount; - } - - @Override - public boolean supportsAutoLoading() { - return bloom != null; - } - - public static void setFakeLookupMode(boolean enabled) { - if (enabled) { - randomGeneratorForTest = new Random(283742987L); - } else { - randomGeneratorForTest = null; - } - } - - /** - * {@inheritDoc} - * Just concatenate row and column by default. May return the original row - * buffer if the column qualifier is empty. - */ - @Override - public byte[] createBloomKey(byte[] rowBuf, int rowOffset, int rowLen, - byte[] qualBuf, int qualOffset, int qualLen) { - // Optimize the frequent case when only the row is provided. - if (qualLen <= 0 && rowOffset == 0 && rowLen == rowBuf.length) - return rowBuf; - - byte [] result = new byte[rowLen + qualLen]; - System.arraycopy(rowBuf, rowOffset, result, 0, rowLen); - if (qualLen > 0) - System.arraycopy(qualBuf, qualOffset, result, rowLen, qualLen); - return result; - } - - /** - * A human-readable string with statistics for the given Bloom filter. - * - * @param bloomFilter the Bloom filter to output statistics for; - * @return a string consisting of "<key>: <value>" parts - * separated by {@link #STATS_RECORD_SEP}. - */ - public static String formatStats(BloomFilterBase bloomFilter) { - StringBuilder sb = new StringBuilder(); - long k = bloomFilter.getKeyCount(); - long m = bloomFilter.getMaxKeys(); - - sb.append("BloomSize: " + bloomFilter.getByteSize() + STATS_RECORD_SEP); - sb.append("No of Keys in bloom: " + k + STATS_RECORD_SEP); - sb.append("Max Keys for bloom: " + m); - if (m > 0) { - sb.append(STATS_RECORD_SEP + "Percentage filled: " - + NumberFormat.getPercentInstance().format(k * 1.0 / m)); - } - return sb.toString(); - } - - @Override - public String toString() { - return formatStats(this) + STATS_RECORD_SEP + "Actual error rate: " - + String.format("%.8f", actualErrorRate()); - } - -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java index cae9d3168c8..984742f7902 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java @@ -23,6 +23,8 @@ import java.io.DataInput; import java.io.IOException; import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.hfile.BlockType; import org.apache.hadoop.hbase.io.hfile.FixedFileTrailer; @@ -31,7 +33,7 @@ import org.apache.hadoop.hbase.io.hfile.HFileBlock; import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex; /** - * A Bloom filter implementation built on top of {@link ByteBloomFilter}, + * A Bloom filter implementation built on top of {@link BloomFilterChunk}, * encapsulating a set of fixed-size Bloom filters written out at the time of * {@link org.apache.hadoop.hbase.io.hfile.HFile} generation into the data * block stream, and loaded on demand at query time. This class only provides @@ -90,10 +92,14 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase public boolean contains(byte[] key, int keyOffset, int keyLength, ByteBuffer bloom) { // We try to store the result in this variable so we can update stats for // testing, but when an error happens, we log a message and return. - boolean result; int block = index.rootBlockContainingKey(key, keyOffset, - keyLength, comparator); + keyLength); + return checkContains(key, keyOffset, keyLength, block); + } + + private boolean checkContains(byte[] key, int keyOffset, int keyLength, int block) { + boolean result; if (block < 0) { result = false; // This key is not in the file. } else { @@ -111,7 +117,7 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase } ByteBuffer bloomBuf = bloomBlock.getBufferReadOnly(); - result = ByteBloomFilter.contains(key, keyOffset, keyLength, + result = BloomFilterUtil.contains(key, keyOffset, keyLength, bloomBuf, bloomBlock.headerSize(), bloomBlock.getUncompressedSizeWithoutHeader(), hash, hashCount); } @@ -126,6 +132,18 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase return result; } + @Override + public boolean contains(Cell keyCell, ByteBuffer bloom) { + // We try to store the result in this variable so we can update stats for + // testing, but when an error happens, we log a message and return. + int block = index.rootBlockContainingKey(keyCell); + // TODO : Will be true KeyValue for now. + // When Offheap comes in we can add an else condition to work + // on the bytes in offheap + KeyValue kvKey = (KeyValue) keyCell; + return checkContains(kvKey.getBuffer(), kvKey.getKeyOffset(), kvKey.getKeyLength(), block); + } + public boolean supportsAutoLoading() { return true; } @@ -166,10 +184,10 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase @Override public String toString() { StringBuilder sb = new StringBuilder(); - sb.append(ByteBloomFilter.formatStats(this)); - sb.append(ByteBloomFilter.STATS_RECORD_SEP + + sb.append(BloomFilterUtil.formatStats(this)); + sb.append(BloomFilterUtil.STATS_RECORD_SEP + "Number of chunks: " + numChunks); - sb.append(ByteBloomFilter.STATS_RECORD_SEP + + sb.append(BloomFilterUtil.STATS_RECORD_SEP + ((comparator != null) ? "Comparator: " + comparator.getClass().getSimpleName() : "Comparator: " + Bytes.BYTES_RAWCOMPARATOR.getClass().getSimpleName())); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java index d68c78d003e..7c29ab21721 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java @@ -22,8 +22,6 @@ package org.apache.hadoop.hbase.util; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.CellComparator; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueUtil; @InterfaceAudience.Private public class CompoundBloomFilterBase implements BloomFilterBase { @@ -69,24 +67,4 @@ public class CompoundBloomFilterBase implements BloomFilterBase { return totalByteSize; } - private static final byte[] DUMMY = new byte[0]; - - /** - * Prepare an ordered pair of row and qualifier to be compared using - * KeyValue.KeyComparator. This is only used for row-column Bloom - * filters. - */ - @Override - public byte[] createBloomKey(byte[] row, int roffset, int rlength, - byte[] qualifier, int qoffset, int qlength) { - if (qualifier == null) - qualifier = DUMMY; - - // Make sure this does not specify a timestamp so that the default maximum - // (most recent) timestamp is used. - KeyValue kv = KeyValueUtil.createFirstOnRow(row, roffset, rlength, DUMMY, 0, 0, - qualifier, qoffset, qlength); - return kv.getKey(); - } - } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java index 594fc94f15c..93c6a7b27c8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java @@ -47,10 +47,10 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase LogFactory.getLog(CompoundBloomFilterWriter.class); /** The current chunk being written to */ - private ByteBloomFilter chunk; + private BloomFilterChunk chunk; /** Previous chunk, so that we can create another similar chunk */ - private ByteBloomFilter prevChunk; + private BloomFilterChunk prevChunk; /** Maximum fold factor */ private int maxFold; @@ -62,7 +62,7 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase private static class ReadyChunk { int chunkId; byte[] firstKey; - ByteBloomFilter chunk; + BloomFilterChunk chunk; } private Queue readyChunks = new LinkedList(); @@ -90,7 +90,7 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase public CompoundBloomFilterWriter(int chunkByteSizeHint, float errorRate, int hashType, int maxFold, boolean cacheOnWrite, CellComparator comparator) { - chunkByteSize = ByteBloomFilter.computeFoldableByteSize( + chunkByteSize = BloomFilterUtil.computeFoldableByteSize( chunkByteSizeHint * 8L, maxFold); this.errorRate = errorRate; @@ -174,7 +174,7 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase if (prevChunk == null) { // First chunk - chunk = ByteBloomFilter.createBySize(chunkByteSize, errorRate, + chunk = BloomFilterUtil.createBySize(chunkByteSize, errorRate, hashType, maxFold); } else { // Use the same parameters as the last chunk, but a new array and @@ -201,8 +201,8 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase // again for cache-on-write. ReadyChunk readyChunk = readyChunks.peek(); - ByteBloomFilter readyChunkBloom = readyChunk.chunk; - readyChunkBloom.getDataWriter().write(out); + BloomFilterChunk readyChunkBloom = readyChunk.chunk; + readyChunkBloom.writeBloom(out); } @Override @@ -225,7 +225,7 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase } /** - * This is modeled after {@link ByteBloomFilter.MetaWriter} for simplicity, + * This is modeled after {@link BloomFilterChunk.MetaWriter} for simplicity, * although the two metadata formats do not have to be consistent. This * does have to be consistent with how {@link * CompoundBloomFilter#CompoundBloomFilter(DataInput, @@ -255,18 +255,13 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase } } - @Override - public Writable getMetaWriter() { - return new MetaWriter(); - } - @Override public void compactBloom() { } @Override - public void allocBloom() { - // Nothing happens here. All allocation happens on demand. + public Writable getMetaWriter() { + return new MetaWriter(); } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java index 9bd9099a2f9..ce405158d86 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java @@ -51,11 +51,11 @@ import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; import org.apache.hadoop.hbase.io.hfile.TestHFileWriterV2; import org.apache.hadoop.hbase.util.BloomFilterFactory; -import org.apache.hadoop.hbase.util.ByteBloomFilter; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CompoundBloomFilter; import org.apache.hadoop.hbase.util.CompoundBloomFilterBase; import org.apache.hadoop.hbase.util.CompoundBloomFilterWriter; +import org.apache.hadoop.hbase.util.BloomFilterUtil; import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -220,7 +220,7 @@ public class TestCompoundBloomFilter { // Test for false positives (some percentage allowed). We test in two modes: // "fake lookup" which ignores the key distribution, and production mode. for (boolean fakeLookupEnabled : new boolean[] { true, false }) { - ByteBloomFilter.setFakeLookupMode(fakeLookupEnabled); + BloomFilterUtil.setFakeLookupMode(fakeLookupEnabled); try { String fakeLookupModeStr = ", fake lookup is " + (fakeLookupEnabled ? "enabled" : "disabled"); @@ -270,7 +270,7 @@ public class TestCompoundBloomFilter { validateFalsePosRate(falsePosRate, nTrials, -2.58, cbf, fakeLookupModeStr); } finally { - ByteBloomFilter.setFakeLookupMode(false); + BloomFilterUtil.setFakeLookupMode(false); } } @@ -337,11 +337,11 @@ public class TestCompoundBloomFilter { int bloomBlockByteSize = 4096; int bloomBlockBitSize = bloomBlockByteSize * 8; double targetErrorRate = 0.01; - long maxKeysPerChunk = ByteBloomFilter.idealMaxKeys(bloomBlockBitSize, + long maxKeysPerChunk = BloomFilterUtil.idealMaxKeys(bloomBlockBitSize, targetErrorRate); long bloomSize1 = bloomBlockByteSize * 8; - long bloomSize2 = ByteBloomFilter.computeBitSize(maxKeysPerChunk, + long bloomSize2 = BloomFilterUtil.computeBitSize(maxKeysPerChunk, targetErrorRate); double bloomSizeRatio = (bloomSize2 * 1.0 / bloomSize1); @@ -350,13 +350,12 @@ public class TestCompoundBloomFilter { @Test public void testCreateKey() { - CompoundBloomFilterBase cbfb = new CompoundBloomFilterBase(); byte[] row = "myRow".getBytes(); byte[] qualifier = "myQualifier".getBytes(); - byte[] rowKey = cbfb.createBloomKey(row, 0, row.length, - row, 0, 0); - byte[] rowColKey = cbfb.createBloomKey(row, 0, row.length, - qualifier, 0, qualifier.length); + // Mimic what Storefile.createBloomKeyValue() does + byte[] rowKey = KeyValueUtil.createFirstOnRow(row, 0, row.length, new byte[0], 0, 0, row, 0, 0).getKey(); + byte[] rowColKey = KeyValueUtil.createFirstOnRow(row, 0, row.length, + new byte[0], 0, 0, qualifier, 0, qualifier.length).getKey(); KeyValue rowKV = KeyValueUtil.createKeyValueFromKey(rowKey); KeyValue rowColKV = KeyValueUtil.createKeyValueFromKey(rowColKey); assertEquals(rowKV.getTimestamp(), rowColKV.getTimestamp()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBloomFilterChunk.java similarity index 62% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBloomFilterChunk.java index 21d7490eded..4d8ad4b4082 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBloomFilterChunk.java @@ -29,11 +29,11 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.experimental.categories.Category; @Category({MiscTests.class, SmallTests.class}) -public class TestByteBloomFilter extends TestCase { +public class TestBloomFilterChunk 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); + BloomFilterChunk bf1 = new BloomFilterChunk(1000, (float)0.01, Hash.MURMUR_HASH, 0); + BloomFilterChunk bf2 = new BloomFilterChunk(1000, (float)0.01, Hash.MURMUR_HASH, 0); bf1.allocBloom(); bf2.allocBloom(); @@ -44,10 +44,14 @@ public class TestByteBloomFilter extends TestCase { bf1.add(key1); bf2.add(key2); - assertTrue(bf1.contains(key1)); - assertFalse(bf1.contains(key2)); - assertFalse(bf2.contains(key1)); - assertTrue(bf2.contains(key2)); + assertTrue(BloomFilterUtil.contains(key1, 0, key1.length, bf1.bloom, 0, (int) bf1.byteSize, + bf1.hash, bf1.hashCount)); + assertFalse(BloomFilterUtil.contains(key2, 0, key2.length, bf1.bloom, 0, (int) bf1.byteSize, + bf1.hash, bf1.hashCount)); + assertFalse(BloomFilterUtil.contains(key1, 0, key1.length, bf2.bloom, 0, (int) bf2.byteSize, + bf2.hash, bf2.hashCount)); + assertTrue(BloomFilterUtil.contains(key2, 0, key2.length, bf2.bloom, 0, (int) bf2.byteSize, + bf2.hash, bf2.hashCount)); byte [] bkey = {1,2,3,4}; byte [] bval = "this is a much larger byte array".getBytes(); @@ -55,24 +59,32 @@ public class TestByteBloomFilter extends TestCase { 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) ); + assertTrue(BloomFilterUtil.contains(bkey, 0, bkey.length, bf1.bloom, 0, (int) bf1.byteSize, + bf1.hash, bf1.hashCount)); + assertTrue(BloomFilterUtil.contains(bval, 1, bval.length - 1, bf1.bloom, 0, (int) bf1.byteSize, + bf1.hash, bf1.hashCount)); + assertFalse(BloomFilterUtil.contains(bval, 0, bval.length, bf1.bloom, 0, (int) bf1.byteSize, + bf1.hash, bf1.hashCount)); // 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, + BloomFilterChunk newBf1 = new BloomFilterChunk(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) ); + assertTrue(BloomFilterUtil.contains(key1, 0, key1.length, bb, 0, (int) newBf1.byteSize, + newBf1.hash, newBf1.hashCount)); + assertFalse(BloomFilterUtil.contains(key2, 0, key2.length, bb, 0, (int) newBf1.byteSize, + newBf1.hash, newBf1.hashCount)); + assertTrue(BloomFilterUtil.contains(bkey, 0, bkey.length, bb, 0, (int) newBf1.byteSize, + newBf1.hash, newBf1.hashCount)); + assertTrue(BloomFilterUtil.contains(bval, 1, bval.length - 1, bb, 0, (int) newBf1.byteSize, + newBf1.hash, newBf1.hashCount)); + assertFalse(BloomFilterUtil.contains(bval, 0, bval.length, bb, 0, (int) newBf1.byteSize, + newBf1.hash, newBf1.hashCount)); + assertFalse(BloomFilterUtil.contains(bval, 0, bval.length, bb, 0, (int) newBf1.byteSize, + newBf1.hash, newBf1.hashCount)); System.out.println("Serialized as " + bOut.size() + " bytes"); assertTrue(bOut.size() - bf1.byteSize < 10); //... allow small padding @@ -80,7 +92,7 @@ public class TestByteBloomFilter extends TestCase { public void testBloomFold() throws Exception { // test: foldFactor < log(max/actual) - ByteBloomFilter b = new ByteBloomFilter(1003, (float) 0.01, + BloomFilterChunk b = new BloomFilterChunk(1003, (float) 0.01, Hash.MURMUR_HASH, 2); b.allocBloom(); long origSize = b.getByteSize(); @@ -92,7 +104,9 @@ public class TestByteBloomFilter extends TestCase { assertEquals(origSize>>2, b.getByteSize()); int falsePositives = 0; for (int i = 0; i < 25; ++i) { - if (b.contains(Bytes.toBytes(i))) { + byte[] bytes = Bytes.toBytes(i); + if (BloomFilterUtil.contains(bytes, 0, bytes.length, b.bloom, 0, (int) b.byteSize, b.hash, + b.hashCount)) { if(i >= 12) falsePositives++; } else { assertFalse(i < 12); @@ -106,7 +120,7 @@ public class TestByteBloomFilter extends TestCase { public void testBloomPerf() throws Exception { // add float err = (float)0.01; - ByteBloomFilter b = new ByteBloomFilter(10*1000*1000, (float)err, Hash.MURMUR_HASH, 3); + BloomFilterChunk b = new BloomFilterChunk(10*1000*1000, (float)err, Hash.MURMUR_HASH, 3); b.allocBloom(); long startTime = System.currentTimeMillis(); long origSize = b.getByteSize(); @@ -128,7 +142,9 @@ public class TestByteBloomFilter extends TestCase { int falsePositives = 0; for (int i = 0; i < 2*1000*1000; ++i) { - if (b.contains(Bytes.toBytes(i))) { + byte[] bytes = Bytes.toBytes(i); + if (BloomFilterUtil.contains(bytes, 0, bytes.length, b.bloom, 0, (int) b.byteSize, b.hash, + b.hashCount)) { if(i >= 1*1000*1000) falsePositives++; } else { assertFalse(i < 1*1000*1000); @@ -148,20 +164,20 @@ public class TestByteBloomFilter extends TestCase { // How many keys can we store in a Bloom filter of this size maintaining // the given false positive rate, not taking into account that the n - long maxKeys = ByteBloomFilter.idealMaxKeys(bitSize, errorRate); + long maxKeys = BloomFilterUtil.idealMaxKeys(bitSize, errorRate); assertEquals(136570, maxKeys); // A reverse operation: how many bits would we need to store this many keys // and keep the same low false positive rate? - long bitSize2 = ByteBloomFilter.computeBitSize(maxKeys, errorRate); + long bitSize2 = BloomFilterUtil.computeBitSize(maxKeys, errorRate); // The bit size comes out a little different due to rounding. assertTrue(Math.abs(bitSize2 - bitSize) * 1.0 / bitSize < 1e-5); } public void testFoldableByteSize() { - assertEquals(128, ByteBloomFilter.computeFoldableByteSize(1000, 5)); - assertEquals(640, ByteBloomFilter.computeFoldableByteSize(5001, 4)); + assertEquals(128, BloomFilterUtil.computeFoldableByteSize(1000, 5)); + assertEquals(640, BloomFilterUtil.computeFoldableByteSize(5001, 4)); }