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