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