.../hadoop/hbase/io/hfile/HFileBlockIndex.java | 1 -
.../hadoop/hbase/io/hfile/HFilePrettyPrinter.java | 6 +-
.../hadoop/hbase/regionserver/StoreFile.java | 42 +--
.../org/apache/hadoop/hbase/util/BloomFilter.java | 13 +-
.../apache/hadoop/hbase/util/BloomFilterBase.java | 7 -
.../hadoop/hbase/util/BloomFilterFactory.java | 7 -
.../apache/hadoop/hbase/util/BloomFilterUtil.java | 289 +++++++++++++++++++
.../hadoop/hbase/util/BloomFilterWriter.java | 4 -
.../apache/hadoop/hbase/util/ByteBloomFilter.java | 313 +--------------------
.../hadoop/hbase/util/CompoundBloomFilter.java | 26 +-
.../hadoop/hbase/util/CompoundBloomFilterBase.java | 22 --
.../hbase/util/CompoundBloomFilterWriter.java | 15 +-
.../regionserver/TestCompoundBloomFilter.java | 19 +-
.../hadoop/hbase/util/TestByteBloomFilter.java | 56 ++--
14 files changed, 409 insertions(+), 411 deletions(-)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
index e6e1fff..7707115 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
@@ -448,7 +448,6 @@ public class HFileBlockIndex {
* Key to find
*/
public int rootBlockContainingKey(final Cell key) {
- // Here the comparator should not be null as this happens for the root-level block
int pos = Bytes.binarySearch(blockKeys, key, comparator);
// pos is between -(blockKeys.length + 1) to blockKeys.length - 1, see
// binarySearch's javadoc.
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 7463e83..7cc31d0 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 1992479..dba35b3 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
@@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
import org.apache.hadoop.hbase.util.BloomFilter;
import org.apache.hadoop.hbase.util.BloomFilterFactory;
import org.apache.hadoop.hbase.util.BloomFilterWriter;
+import org.apache.hadoop.hbase.util.BloomFilterUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.io.WritableUtils;
@@ -707,7 +708,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 +754,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 +862,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,9 +877,10 @@ 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(),
+ bloomKeyKV = BloomFilterUtil.createBloomKeyValue(cell.getRowArray(),
cell.getRowOffset(), cell.getRowLength(), cell.getQualifierArray(),
cell.getQualifierOffset(), cell.getQualifierLength());
+ bloomKey = bloomKeyKV.getKey();
bloomKeyOffset = 0;
bloomKeyLen = bloomKey.length;
break;
@@ -889,17 +890,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 +1253,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,7 +1271,7 @@ public class StoreFile {
break;
case ROWCOL:
- key = bloomFilter.createBloomKey(row, rowOffset, rowLen, col,
+ kvKey = BloomFilterUtil.createBloomKeyValue(row, rowOffset, rowLen, col,
colOffset, colLen);
break;
@@ -1304,9 +1308,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 +1317,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,
+ KeyValue rowBloomKey = BloomFilterUtil.createBloomKeyValue(row, rowOffset, rowLen,
null, 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 b314bf6..a4de98e 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,6 +20,7 @@ package org.apache.hadoop.hbase.util;
import java.nio.ByteBuffer;
+import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
@@ -45,7 +46,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 kvKey the keyvalue 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(KeyValue kvKey, 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 0396965..43cf75d 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/BloomFilterFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java
index 1a63f6b..aecbdf8 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 0000000..f41cc21
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java
@@ -0,0 +1,289 @@
+/*
+ * 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.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
+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 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;
+ }
+
+ /** 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();
+ }
+
+ 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.
+ */
+ public static KeyValue createBloomKeyValue(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.
+ return KeyValueUtil.createFirstOnRow(row, roffset, rlength, DUMMY, 0, 0, qualifier, qoffset,
+ qlength);
+ }
+
+ public static String toString(ByteBloomFilter 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 aa7f503..2aba737 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
index 1ea81dd..7dcde7f 100644
--- 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
@@ -23,11 +23,8 @@ 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.
@@ -76,8 +73,7 @@ import org.apache.hadoop.io.Writable;
* 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 {
+public class ByteBloomFilter implements BloomFilterBase {
/** Current file format version */
public static final int VERSION = 1;
@@ -97,33 +93,6 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
/** 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
@@ -145,52 +114,6 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
}
/**
- * @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
@@ -200,61 +123,10 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
* @return error rate for this particular Bloom filter
*/
public double actualErrorRate() {
- return actualErrorRate(keyCount, byteSize * 8, hashCount);
+ return BloomFilterUtil.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) {
+ public ByteBloomFilter(int hashType) {
this.hashType = hashType;
this.hash = Hash.getInstance(hashType);
}
@@ -278,44 +150,17 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
int foldFactor) throws IllegalArgumentException {
this(hashType);
- long bitSize = computeBitSize(maxKeys, errorRate);
- hashCount = optimalFunctionCount(maxKeys, bitSize);
+ long bitSize = BloomFilterUtil.computeBitSize(maxKeys, errorRate);
+ hashCount = BloomFilterUtil.optimalFunctionCount(maxKeys, bitSize);
this.maxKeys = maxKeys;
// increase byteSize so folding is possible
- byteSize = computeFoldableByteSize(bitSize, foldFactor);
+ byteSize = BloomFilterUtil.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.
*
@@ -329,7 +174,6 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
return bbf;
}
- @Override
public void allocBloom() {
if (this.bloom != null) {
throw new IllegalArgumentException("can only create bloom once.");
@@ -367,7 +211,6 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
add(buf, 0, buf.length);
}
- @Override
public void add(byte [] buf, int offset, int len) {
/*
* For faster hashing, use combinatorial generation
@@ -399,7 +242,6 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
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
@@ -413,37 +255,8 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
+ " 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;
+ return BloomFilterUtil.contains(buf, offset, length, theBloom, 0, (int) byteSize, hash,
+ hashCount);
}
//---------------------------------------------------------------------------
@@ -458,7 +271,7 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
int bytePos = (int)(pos / 8);
int bitPos = (int)(pos % 8);
byte curByte = bloom.get(bytePos);
- curByte |= bitvals[bitPos];
+ curByte |= BloomFilterUtil.bitvals[bitPos];
bloom.put(bytePos, curByte);
}
@@ -473,7 +286,7 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
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];
+ curByte &= BloomFilterUtil.bitvals[bitPos];
return (curByte != 0);
}
@@ -496,7 +309,6 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
return hashType;
}
- @Override
public void compactBloom() {
// see if the actual size is exponentially smaller than expected.
if (this.keyCount > 0 && this.bloom.hasArray()) {
@@ -532,59 +344,17 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
}
}
-
- //---------------------------------------------------------------------------
-
/**
* 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 {
+ 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);
- }
+ out.write(this.bloom.array(), this.bloom.arrayOffset(), this.bloom.limit());
}
public int getHashCount() {
@@ -592,63 +362,8 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
}
@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());
+ return BloomFilterUtil.toString(this);
}
}
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 cae9d31..08beb0b 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,7 @@ import java.io.DataInput;
import java.io.IOException;
import java.nio.ByteBuffer;
+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;
@@ -90,10 +91,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 +116,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 +131,15 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase
return result;
}
+ @Override
+ public boolean contains(KeyValue kvKey, 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(kvKey);
+ byte[] key = kvKey.getKey();
+ return checkContains(key, 0, key.length, block);
+ }
+
public boolean supportsAutoLoading() {
return true;
}
@@ -166,10 +180,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 d68c78d..7c29ab2 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 594fc94..b9c61b3 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
@@ -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
@@ -202,7 +202,7 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase
ReadyChunk readyChunk = readyChunks.peek();
ByteBloomFilter readyChunkBloom = readyChunk.chunk;
- readyChunkBloom.getDataWriter().write(out);
+ readyChunkBloom.writeBloom(out);
}
@Override
@@ -256,17 +256,12 @@ 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 9bd9099..e91be92 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);
+ byte[] rowKey = BloomFilterUtil.createBloomKeyValue(row, 0, row.length,
+ row, 0, 0).getKey();
+ byte[] rowColKey = BloomFilterUtil.createBloomKeyValue(row, 0, row.length,
+ 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/TestByteBloomFilter.java
index 21d7490..62f3a82 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/TestByteBloomFilter.java
@@ -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,10 +59,12 @@ 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)
@@ -67,12 +73,18 @@ public class TestByteBloomFilter extends TestCase {
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) );
+ 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
@@ -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);
@@ -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));
}