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