001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.util;
019
020import java.text.NumberFormat;
021import java.util.Random;
022
023import org.apache.hadoop.hbase.Cell;
024import org.apache.hadoop.hbase.nio.ByteBuff;
025import org.apache.hadoop.hbase.regionserver.BloomType;
026import org.apache.yetus.audience.InterfaceAudience;
027
028import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
029
030/**
031 * Utility methods related to BloomFilters
032 */
033@InterfaceAudience.Private
034public final class BloomFilterUtil {
035
036  /** Record separator for the Bloom filter statistics human-readable string */
037  public static final String STATS_RECORD_SEP = "; ";
038  /**
039   * Used in computing the optimal Bloom filter size. This approximately equals
040   * 0.480453.
041   */
042  public static final double LOG2_SQUARED = Math.log(2) * Math.log(2);
043  
044  /**
045   * A random number generator to use for "fake lookups" when testing to
046   * estimate the ideal false positive rate.
047   */
048  private static Random randomGeneratorForTest;
049  
050  /** Bit-value lookup array to prevent doing the same work over and over */
051  public static final byte [] bitvals = {
052    (byte) 0x01,
053    (byte) 0x02,
054    (byte) 0x04,
055    (byte) 0x08,
056    (byte) 0x10,
057    (byte) 0x20,
058    (byte) 0x40,
059    (byte) 0x80
060  };
061
062  /**
063   * Private constructor to keep this class from being instantiated.
064   */
065  private BloomFilterUtil() {
066  }
067
068  /**
069   * @param maxKeys
070   * @param errorRate
071   * @return the number of bits for a Bloom filter than can hold the given
072   *         number of keys and provide the given error rate, assuming that the
073   *         optimal number of hash functions is used and it does not have to
074   *         be an integer.
075   */
076  public static long computeBitSize(long maxKeys, double errorRate) {
077    return (long) Math.ceil(maxKeys * (-Math.log(errorRate) / LOG2_SQUARED));
078  }
079
080  /**
081   * Sets a random generator to be used for look-ups instead of computing hashes. Can be used to
082   * simulate uniformity of accesses better in a test environment. Should not be set in a real
083   * environment where correctness matters!
084   * <p>
085 *   This gets used in {@link #contains(ByteBuff, int, int, Hash, int, HashKey)}
086   * @param random The random number source to use, or null to compute actual hashes
087   */
088  @VisibleForTesting
089  public static void setRandomGeneratorForTest(Random random) {
090    randomGeneratorForTest = random;
091  }
092
093  /**
094   * The maximum number of keys we can put into a Bloom filter of a certain
095   * size to maintain the given error rate, assuming the number of hash
096   * functions is chosen optimally and does not even have to be an integer
097   * (hence the "ideal" in the function name).
098   *
099   * @param bitSize
100   * @param errorRate
101   * @return maximum number of keys that can be inserted into the Bloom filter
102   * @see #computeMaxKeys(long, double, int) for a more precise estimate
103   */
104  public static long idealMaxKeys(long bitSize, double errorRate) {
105    // The reason we need to use floor here is that otherwise we might put
106    // more keys in a Bloom filter than is allowed by the target error rate.
107    return (long) (bitSize * (LOG2_SQUARED / -Math.log(errorRate)));
108  }
109
110  /**
111   * The maximum number of keys we can put into a Bloom filter of a certain
112   * size to get the given error rate, with the given number of hash functions.
113   *
114   * @param bitSize
115   * @param errorRate
116   * @param hashCount
117   * @return the maximum number of keys that can be inserted in a Bloom filter
118   *         to maintain the target error rate, if the number of hash functions
119   *         is provided.
120   */
121  public static long computeMaxKeys(long bitSize, double errorRate,
122      int hashCount) {
123    return (long) (-bitSize * 1.0 / hashCount *
124        Math.log(1 - Math.exp(Math.log(errorRate) / hashCount)));
125  }
126
127  /**
128   * Computes the actual error rate for the given number of elements, number
129   * of bits, and number of hash functions. Taken directly from the
130   * <a href=
131   * "http://en.wikipedia.org/wiki/Bloom_filter#Probability_of_false_positives"
132   * > Wikipedia Bloom filter article</a>.
133   *
134   * @param maxKeys
135   * @param bitSize
136   * @param functionCount
137   * @return the actual error rate
138   */
139  public static double actualErrorRate(long maxKeys, long bitSize,
140      int functionCount) {
141    return Math.exp(Math.log(1 - Math.exp(-functionCount * maxKeys * 1.0
142        / bitSize)) * functionCount);
143  }
144
145  /**
146   * Increases the given byte size of a Bloom filter until it can be folded by
147   * the given factor.
148   *
149   * @param bitSize
150   * @param foldFactor
151   * @return Foldable byte size
152   */
153  public static int computeFoldableByteSize(long bitSize, int foldFactor) {
154    long byteSizeLong = (bitSize + 7) / 8;
155    int mask = (1 << foldFactor) - 1;
156    if ((mask & byteSizeLong) != 0) {
157      byteSizeLong >>= foldFactor;
158      ++byteSizeLong;
159      byteSizeLong <<= foldFactor;
160    }
161    if (byteSizeLong > Integer.MAX_VALUE) {
162      throw new IllegalArgumentException("byteSize=" + byteSizeLong + " too "
163          + "large for bitSize=" + bitSize + ", foldFactor=" + foldFactor);
164    }
165    return (int) byteSizeLong;
166  }
167
168  public static int optimalFunctionCount(int maxKeys, long bitSize) {
169    long i = bitSize / maxKeys;
170    double result = Math.ceil(Math.log(2) * i);
171    if (result > Integer.MAX_VALUE){
172      throw new IllegalArgumentException("result too large for integer value.");
173    }
174    return (int)result;
175  }
176  
177  /**
178   * Creates a Bloom filter chunk of the given size.
179   *
180   * @param byteSizeHint the desired number of bytes for the Bloom filter bit
181   *          array. Will be increased so that folding is possible.
182   * @param errorRate target false positive rate of the Bloom filter
183   * @param hashType Bloom filter hash function type
184   * @param foldFactor
185   * @param bloomType
186   * @return the new Bloom filter of the desired size
187   */
188  public static BloomFilterChunk createBySize(int byteSizeHint,
189      double errorRate, int hashType, int foldFactor, BloomType bloomType) {
190    BloomFilterChunk bbf = new BloomFilterChunk(hashType, bloomType);
191
192    bbf.byteSize = computeFoldableByteSize(byteSizeHint * 8L, foldFactor);
193    long bitSize = bbf.byteSize * 8;
194    bbf.maxKeys = (int) idealMaxKeys(bitSize, errorRate);
195    bbf.hashCount = optimalFunctionCount(bbf.maxKeys, bitSize);
196
197    // Adjust max keys to bring error rate closer to what was requested,
198    // because byteSize was adjusted to allow for folding, and hashCount was
199    // rounded.
200    bbf.maxKeys = (int) computeMaxKeys(bitSize, errorRate, bbf.hashCount);
201
202    return bbf;
203  }
204
205  public static boolean contains(byte[] buf, int offset, int length,
206      ByteBuff bloomBuf, int bloomOffset, int bloomSize, Hash hash,
207      int hashCount) {
208    HashKey<byte[]> hashKey = new ByteArrayHashKey(buf, offset, length);
209    return contains(bloomBuf, bloomOffset, bloomSize, hash, hashCount, hashKey);
210  }
211
212  private static <T> boolean contains(ByteBuff bloomBuf, int bloomOffset, int bloomSize, Hash hash,
213      int hashCount, HashKey<T> hashKey) {
214    int hash1 = hash.hash(hashKey, 0);
215    int bloomBitSize = bloomSize << 3;
216
217    int hash2 = 0;
218    int compositeHash = 0;
219
220    if (randomGeneratorForTest == null) {
221      // Production mode
222      compositeHash = hash1;
223      hash2 = hash.hash(hashKey, hash1);
224    }
225
226    for (int i = 0; i < hashCount; i++) {
227      int hashLoc = (randomGeneratorForTest == null
228          // Production mode
229          ? Math.abs(compositeHash % bloomBitSize)
230          // Test mode with "fake look-ups" to estimate "ideal false positive rate"
231          : randomGeneratorForTest.nextInt(bloomBitSize));
232      compositeHash += hash2;
233      if (!checkBit(hashLoc, bloomBuf, bloomOffset)) {
234        return false;
235      }
236    }
237    return true;
238  }
239
240  public static boolean contains(Cell cell, ByteBuff bloomBuf, int bloomOffset, int bloomSize,
241      Hash hash, int hashCount, BloomType type) {
242    HashKey<Cell> hashKey = type == BloomType.ROW ? new RowBloomHashKey(cell)
243        : new RowColBloomHashKey(cell);
244    return contains(bloomBuf, bloomOffset, bloomSize, hash, hashCount, hashKey);
245  }
246
247  /**
248   * Check if bit at specified index is 1.
249   *
250   * @param pos index of bit
251   * @return true if bit at specified index is 1, false if 0.
252   */
253   static boolean checkBit(int pos, ByteBuff bloomBuf, int bloomOffset) {
254    int bytePos = pos >> 3; //pos / 8
255    int bitPos = pos & 0x7; //pos % 8
256    byte curByte = bloomBuf.get(bloomOffset + bytePos);
257    curByte &= bitvals[bitPos];
258    return (curByte != 0);
259  }
260
261  /**
262   * A human-readable string with statistics for the given Bloom filter.
263   *
264   * @param bloomFilter the Bloom filter to output statistics for;
265   * @return a string consisting of "&lt;key&gt;: &lt;value&gt;" parts
266   *         separated by {@link #STATS_RECORD_SEP}.
267   */
268  public static String formatStats(BloomFilterBase bloomFilter) {
269    StringBuilder sb = new StringBuilder();
270    long k = bloomFilter.getKeyCount();
271    long m = bloomFilter.getMaxKeys();
272
273    sb.append("BloomSize: " + bloomFilter.getByteSize() + STATS_RECORD_SEP);
274    sb.append("No of Keys in bloom: " + k + STATS_RECORD_SEP);
275    sb.append("Max Keys for bloom: " + m);
276    if (m > 0) {
277      sb.append(STATS_RECORD_SEP + "Percentage filled: "
278          + NumberFormat.getPercentInstance().format(k * 1.0 / m));
279    }
280    return sb.toString();
281  }
282
283  public static String toString(BloomFilterChunk bloomFilter) {
284    return formatStats(bloomFilter) + STATS_RECORD_SEP + "Actual error rate: "
285        + String.format("%.8f", bloomFilter.actualErrorRate());
286  }
287}