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 * @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 065 * used and it 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 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "EI_EXPOSE_STATIC_REP2", 080 justification = "ignore for now, improve TestCompoundBloomFilter later") 081 public static void setRandomGeneratorForTest(Random random) { 082 randomGeneratorForTest = random; 083 } 084 085 /** 086 * The maximum number of keys we can put into a Bloom filter of a certain size to maintain the 087 * given error rate, assuming the number of hash functions is chosen optimally and does not even 088 * have to be an integer (hence the "ideal" in the function name). 089 * @return maximum number of keys that can be inserted into the Bloom filter 090 * @see #computeMaxKeys(long, double, int) for a more precise estimate 091 */ 092 public static long idealMaxKeys(long bitSize, double errorRate) { 093 // The reason we need to use floor here is that otherwise we might put 094 // more keys in a Bloom filter than is allowed by the target error rate. 095 return (long) (bitSize * (LOG2_SQUARED / -Math.log(errorRate))); 096 } 097 098 /** 099 * The maximum number of keys we can put into a Bloom filter of a certain size to get the given 100 * error rate, with the given number of hash functions. 101 * @return the maximum number of keys that can be inserted in a Bloom filter to maintain the 102 * target error rate, if the number of hash functions is provided. 103 */ 104 public static long computeMaxKeys(long bitSize, double errorRate, int hashCount) { 105 return (long) (-bitSize * 1.0 / hashCount 106 * Math.log(1 - Math.exp(Math.log(errorRate) / hashCount))); 107 } 108 109 /** 110 * Computes the actual error rate for the given number of elements, number of bits, and number of 111 * hash functions. Taken directly from the 112 * <a href= "http://en.wikipedia.org/wiki/Bloom_filter#Probability_of_false_positives" > Wikipedia 113 * Bloom filter article</a>. 114 * @return the actual error rate 115 */ 116 public static double actualErrorRate(long maxKeys, long bitSize, int functionCount) { 117 return Math 118 .exp(Math.log(1 - Math.exp(-functionCount * maxKeys * 1.0 / bitSize)) * functionCount); 119 } 120 121 /** 122 * Increases the given byte size of a Bloom filter until it can be folded by the given factor. 123 * @return Foldable byte size 124 */ 125 public static int computeFoldableByteSize(long bitSize, int foldFactor) { 126 long byteSizeLong = (bitSize + 7) / 8; 127 int mask = (1 << foldFactor) - 1; 128 if ((mask & byteSizeLong) != 0) { 129 byteSizeLong >>= foldFactor; 130 ++byteSizeLong; 131 byteSizeLong <<= foldFactor; 132 } 133 if (byteSizeLong > Integer.MAX_VALUE) { 134 throw new IllegalArgumentException("byteSize=" + byteSizeLong + " too " + "large for bitSize=" 135 + bitSize + ", foldFactor=" + foldFactor); 136 } 137 return (int) byteSizeLong; 138 } 139 140 public static int optimalFunctionCount(int maxKeys, long bitSize) { 141 long i = bitSize / maxKeys; 142 double result = Math.ceil(Math.log(2) * i); 143 if (result > Integer.MAX_VALUE) { 144 throw new IllegalArgumentException("result too large for integer value."); 145 } 146 return (int) result; 147 } 148 149 /** 150 * Creates a Bloom filter chunk of the given size. 151 * @param byteSizeHint the desired number of bytes for the Bloom filter bit array. Will be 152 * increased so that folding is possible. 153 * @param errorRate target false positive rate of the Bloom filter 154 * @param hashType Bloom filter hash function type 155 * @return the new Bloom filter of the desired size 156 */ 157 public static BloomFilterChunk createBySize(int byteSizeHint, double errorRate, int hashType, 158 int foldFactor, BloomType bloomType) { 159 BloomFilterChunk bbf = new BloomFilterChunk(hashType, bloomType); 160 161 bbf.byteSize = computeFoldableByteSize(byteSizeHint * 8L, foldFactor); 162 long bitSize = bbf.byteSize * 8; 163 bbf.maxKeys = (int) idealMaxKeys(bitSize, errorRate); 164 bbf.hashCount = optimalFunctionCount(bbf.maxKeys, bitSize); 165 166 // Adjust max keys to bring error rate closer to what was requested, 167 // because byteSize was adjusted to allow for folding, and hashCount was 168 // rounded. 169 bbf.maxKeys = (int) computeMaxKeys(bitSize, errorRate, bbf.hashCount); 170 171 return bbf; 172 } 173 174 public static boolean contains(byte[] buf, int offset, int length, ByteBuff bloomBuf, 175 int bloomOffset, int bloomSize, Hash hash, int hashCount) { 176 HashKey<byte[]> hashKey = new ByteArrayHashKey(buf, offset, length); 177 return contains(bloomBuf, bloomOffset, bloomSize, hash, hashCount, hashKey); 178 } 179 180 private static <T> boolean contains(ByteBuff bloomBuf, int bloomOffset, int bloomSize, Hash hash, 181 int hashCount, HashKey<T> hashKey) { 182 int hash1 = hash.hash(hashKey, 0); 183 int bloomBitSize = bloomSize << 3; 184 185 int hash2 = 0; 186 int compositeHash = 0; 187 188 if (randomGeneratorForTest == null) { 189 // Production mode 190 compositeHash = hash1; 191 hash2 = hash.hash(hashKey, hash1); 192 } 193 194 for (int i = 0; i < hashCount; i++) { 195 int hashLoc = (randomGeneratorForTest == null 196 // Production mode 197 ? Math.abs(compositeHash % bloomBitSize) 198 // Test mode with "fake look-ups" to estimate "ideal false positive rate" 199 : randomGeneratorForTest.nextInt(bloomBitSize)); 200 compositeHash += hash2; 201 if (!checkBit(hashLoc, bloomBuf, bloomOffset)) { 202 return false; 203 } 204 } 205 return true; 206 } 207 208 public static boolean contains(Cell cell, ByteBuff bloomBuf, int bloomOffset, int bloomSize, 209 Hash hash, int hashCount, BloomType type) { 210 HashKey<Cell> hashKey = 211 type == BloomType.ROWCOL ? new RowColBloomHashKey(cell) : new RowBloomHashKey(cell); 212 return contains(bloomBuf, bloomOffset, bloomSize, hash, hashCount, hashKey); 213 } 214 215 /** 216 * Check if bit at specified index is 1. 217 * @param pos index of bit 218 * @return true if bit at specified index is 1, false if 0. 219 */ 220 static boolean checkBit(int pos, ByteBuff bloomBuf, int bloomOffset) { 221 int bytePos = pos >> 3; // pos / 8 222 int bitPos = pos & 0x7; // pos % 8 223 byte curByte = bloomBuf.get(bloomOffset + bytePos); 224 curByte &= bitvals[bitPos]; 225 return (curByte != 0); 226 } 227 228 /** 229 * A human-readable string with statistics for the given Bloom filter. 230 * @param bloomFilter the Bloom filter to output statistics for; 231 * @return a string consisting of "<key>: <value>" parts separated by 232 * {@link #STATS_RECORD_SEP}. 233 */ 234 public static String formatStats(BloomFilterBase bloomFilter) { 235 StringBuilder sb = new StringBuilder(); 236 long k = bloomFilter.getKeyCount(); 237 long m = bloomFilter.getMaxKeys(); 238 239 sb.append("BloomSize: " + bloomFilter.getByteSize() + STATS_RECORD_SEP); 240 sb.append("No of Keys in bloom: " + k + STATS_RECORD_SEP); 241 sb.append("Max Keys for bloom: " + m); 242 if (m > 0) { 243 sb.append(STATS_RECORD_SEP + "Percentage filled: " 244 + NumberFormat.getPercentInstance().format(k * 1.0 / m)); 245 } 246 return sb.toString(); 247 } 248 249 public static String toString(BloomFilterChunk bloomFilter) { 250 return formatStats(bloomFilter) + STATS_RECORD_SEP + "Actual error rate: " 251 + String.format("%.8f", bloomFilter.actualErrorRate()); 252 } 253 254 public static byte[] getBloomFilterParam(BloomType bloomFilterType, Configuration conf) 255 throws IllegalArgumentException { 256 byte[] bloomParam = null; 257 String message = "Bloom filter type is " + bloomFilterType + ", "; 258 if (bloomFilterType.equals(ROWPREFIX_FIXED_LENGTH)) { 259 String prefixLengthString = conf.get(PREFIX_LENGTH_KEY); 260 if (prefixLengthString == null) { 261 message += PREFIX_LENGTH_KEY + " not specified."; 262 throw new IllegalArgumentException(message); 263 } 264 int prefixLength; 265 try { 266 prefixLength = Integer.parseInt(prefixLengthString); 267 if (prefixLength <= 0 || prefixLength > HConstants.MAX_ROW_LENGTH) { 268 message += 269 "the value of " + PREFIX_LENGTH_KEY + " must >=0 and < " + HConstants.MAX_ROW_LENGTH; 270 throw new IllegalArgumentException(message); 271 } 272 } catch (NumberFormatException nfe) { 273 message = "Number format exception when parsing " + PREFIX_LENGTH_KEY + " for BloomType " 274 + bloomFilterType.toString() + ":" + prefixLengthString; 275 throw new IllegalArgumentException(message, nfe); 276 } 277 bloomParam = Bytes.toBytes(prefixLength); 278 } 279 return bloomParam; 280 } 281}