View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  package org.apache.hadoop.hbase.util;
21  
22  import java.io.DataInput;
23  import java.io.DataOutput;
24  import java.io.IOException;
25  import java.nio.ByteBuffer;
26  import java.text.NumberFormat;
27  import java.util.Random;
28  
29  import org.apache.hadoop.hbase.classification.InterfaceAudience;
30  import org.apache.hadoop.io.Writable;
31  
32  /**
33   * Implements a <i>Bloom filter</i>, as defined by Bloom in 1970.
34   * <p>
35   * The Bloom filter is a data structure that was introduced in 1970 and that has
36   * been adopted by the networking research community in the past decade thanks
37   * to the bandwidth efficiencies that it offers for the transmission of set
38   * membership information between networked hosts. A sender encodes the
39   * information into a bit vector, the Bloom filter, that is more compact than a
40   * conventional representation. Computation and space costs for construction are
41   * linear in the number of elements. The receiver uses the filter to test
42   * whether various elements are members of the set. Though the filter will
43   * occasionally return a false positive, it will never return a false negative.
44   * When creating the filter, the sender can choose its desired point in a
45   * trade-off between the false positive rate and the size.
46   *
47   * <p>
48   * Originally inspired by <a href="http://www.one-lab.org">European Commission
49   * One-Lab Project 034819</a>.
50   *
51   * Bloom filters are very sensitive to the number of elements inserted into
52   * them. For HBase, the number of entries depends on the size of the data stored
53   * in the column. Currently the default region size is 256MB, so entry count ~=
54   * 256MB / (average value size for column). Despite this rule of thumb, there is
55   * no efficient way to calculate the entry count after compactions. Therefore,
56   * it is often easier to use a dynamic bloom filter that will add extra space
57   * instead of allowing the error rate to grow.
58   *
59   * ( http://www.eecs.harvard.edu/~michaelm/NEWWORK/postscripts/BloomFilterSurvey
60   * .pdf )
61   *
62   * m denotes the number of bits in the Bloom filter (bitSize) n denotes the
63   * number of elements inserted into the Bloom filter (maxKeys) k represents the
64   * number of hash functions used (nbHash) e represents the desired false
65   * positive rate for the bloom (err)
66   *
67   * If we fix the error rate (e) and know the number of entries, then the optimal
68   * bloom size m = -(n * ln(err) / (ln(2)^2) ~= n * ln(err) / ln(0.6185)
69   *
70   * The probability of false positives is minimized when k = m/n ln(2).
71   *
72   * @see BloomFilter The general behavior of a filter
73   *
74   * @see <a
75   *      href="http://portal.acm.org/citation.cfm?id=362692&dl=ACM&coll=portal">
76   *      Space/Time Trade-Offs in Hash Coding with Allowable Errors</a>
77   */
78  @InterfaceAudience.Private
79  public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
80  
81    /** Current file format version */
82    public static final int VERSION = 1;
83  
84    /** Bytes (B) in the array. This actually has to fit into an int. */
85    protected long byteSize;
86    /** Number of hash functions */
87    protected int hashCount;
88    /** Hash type */
89    protected final int hashType;
90    /** Hash Function */
91    protected final Hash hash;
92    /** Keys currently in the bloom */
93    protected int keyCount;
94    /** Max Keys expected for the bloom */
95    protected int maxKeys;
96    /** Bloom bits */
97    protected ByteBuffer bloom;
98  
99    /** Record separator for the Bloom filter statistics human-readable string */
100   public static final String STATS_RECORD_SEP = "; ";
101 
102   /**
103    * Used in computing the optimal Bloom filter size. This approximately equals
104    * 0.480453.
105    */
106   public static final double LOG2_SQUARED = Math.log(2) * Math.log(2);
107 
108   /**
109    * A random number generator to use for "fake lookups" when testing to
110    * estimate the ideal false positive rate.
111    */
112   private static Random randomGeneratorForTest;
113 
114   /** Bit-value lookup array to prevent doing the same work over and over */
115   private static final byte [] bitvals = {
116     (byte) 0x01,
117     (byte) 0x02,
118     (byte) 0x04,
119     (byte) 0x08,
120     (byte) 0x10,
121     (byte) 0x20,
122     (byte) 0x40,
123     (byte) 0x80
124   };
125 
126   /**
127    * Loads bloom filter meta data from file input.
128    * @param meta stored bloom meta data
129    * @throws IllegalArgumentException meta data is invalid
130    */
131   public ByteBloomFilter(DataInput meta)
132       throws IOException, IllegalArgumentException {
133     this.byteSize = meta.readInt();
134     this.hashCount = meta.readInt();
135     this.hashType = meta.readInt();
136     this.keyCount = meta.readInt();
137     this.maxKeys = this.keyCount;
138 
139     this.hash = Hash.getInstance(this.hashType);
140     if (hash == null) {
141       throw new IllegalArgumentException("Invalid hash type: " + hashType);
142     }
143     sanityCheck();
144   }
145 
146   /**
147    * @param maxKeys
148    * @param errorRate
149    * @return the number of bits for a Bloom filter than can hold the given
150    *         number of keys and provide the given error rate, assuming that the
151    *         optimal number of hash functions is used and it does not have to
152    *         be an integer.
153    */
154   public static long computeBitSize(long maxKeys, double errorRate) {
155     return (long) Math.ceil(maxKeys * (-Math.log(errorRate) / LOG2_SQUARED));
156   }
157 
158   /**
159    * The maximum number of keys we can put into a Bloom filter of a certain
160    * size to maintain the given error rate, assuming the number of hash
161    * functions is chosen optimally and does not even have to be an integer
162    * (hence the "ideal" in the function name).
163    *
164    * @param bitSize
165    * @param errorRate
166    * @return maximum number of keys that can be inserted into the Bloom filter
167    * @see #computeMaxKeys(long, double, int) for a more precise estimate
168    */
169   public static long idealMaxKeys(long bitSize, double errorRate) {
170     // The reason we need to use floor here is that otherwise we might put
171     // more keys in a Bloom filter than is allowed by the target error rate.
172     return (long) (bitSize * (LOG2_SQUARED / -Math.log(errorRate)));
173   }
174 
175   /**
176    * The maximum number of keys we can put into a Bloom filter of a certain
177    * size to get the given error rate, with the given number of hash functions.
178    *
179    * @param bitSize
180    * @param errorRate
181    * @param hashCount
182    * @return the maximum number of keys that can be inserted in a Bloom filter
183    *         to maintain the target error rate, if the number of hash functions
184    *         is provided.
185    */
186   public static long computeMaxKeys(long bitSize, double errorRate,
187       int hashCount) {
188     return (long) (-bitSize * 1.0 / hashCount *
189         Math.log(1 - Math.exp(Math.log(errorRate) / hashCount)));
190   }
191 
192   /**
193    * Computes the error rate for this Bloom filter, taking into account the
194    * actual number of hash functions and keys inserted. The return value of
195    * this function changes as a Bloom filter is being populated. Used for
196    * reporting the actual error rate of compound Bloom filters when writing
197    * them out.
198    *
199    * @return error rate for this particular Bloom filter
200    */
201   public double actualErrorRate() {
202     return actualErrorRate(keyCount, byteSize * 8, hashCount);
203   }
204 
205   /**
206    * Computes the actual error rate for the given number of elements, number
207    * of bits, and number of hash functions. Taken directly from the
208    * <a href=
209    * "http://en.wikipedia.org/wiki/Bloom_filter#Probability_of_false_positives"
210    * > Wikipedia Bloom filter article</a>.
211    *
212    * @param maxKeys
213    * @param bitSize
214    * @param functionCount
215    * @return the actual error rate
216    */
217   public static double actualErrorRate(long maxKeys, long bitSize,
218       int functionCount) {
219     return Math.exp(Math.log(1 - Math.exp(-functionCount * maxKeys * 1.0
220         / bitSize)) * functionCount);
221   }
222 
223   /**
224    * Increases the given byte size of a Bloom filter until it can be folded by
225    * the given factor.
226    *
227    * @param bitSize
228    * @param foldFactor
229    * @return Foldable byte size
230    */
231   public static int computeFoldableByteSize(long bitSize, int foldFactor) {
232     long byteSizeLong = (bitSize + 7) / 8;
233     int mask = (1 << foldFactor) - 1;
234     if ((mask & byteSizeLong) != 0) {
235       byteSizeLong >>= foldFactor;
236       ++byteSizeLong;
237       byteSizeLong <<= foldFactor;
238     }
239     if (byteSizeLong > Integer.MAX_VALUE) {
240       throw new IllegalArgumentException("byteSize=" + byteSizeLong + " too "
241           + "large for bitSize=" + bitSize + ", foldFactor=" + foldFactor);
242     }
243     return (int) byteSizeLong;
244   }
245 
246   private static int optimalFunctionCount(int maxKeys, long bitSize) {
247     long i = bitSize / maxKeys;
248     double result = Math.ceil(Math.log(2) * i);
249     if (result > Integer.MAX_VALUE){
250       throw new IllegalArgumentException("result too large for integer value.");
251     }
252     return (int)result;
253   }
254 
255   /** Private constructor used by other constructors. */
256   private ByteBloomFilter(int hashType) {
257     this.hashType = hashType;
258     this.hash = Hash.getInstance(hashType);
259   }
260 
261   /**
262    * Determines & initializes bloom filter meta data from user config. Call
263    * {@link #allocBloom()} to allocate bloom filter data.
264    *
265    * @param maxKeys Maximum expected number of keys that will be stored in this
266    *          bloom
267    * @param errorRate Desired false positive error rate. Lower rate = more
268    *          storage required
269    * @param hashType Type of hash function to use
270    * @param foldFactor When finished adding entries, you may be able to 'fold'
271    *          this bloom to save space. Tradeoff potentially excess bytes in
272    *          bloom for ability to fold if keyCount is exponentially greater
273    *          than maxKeys.
274    * @throws IllegalArgumentException
275    */
276   public ByteBloomFilter(int maxKeys, double errorRate, int hashType,
277       int foldFactor) throws IllegalArgumentException {
278     this(hashType);
279 
280     long bitSize = computeBitSize(maxKeys, errorRate);
281     hashCount = optimalFunctionCount(maxKeys, bitSize);
282     this.maxKeys = maxKeys;
283 
284     // increase byteSize so folding is possible
285     byteSize = computeFoldableByteSize(bitSize, foldFactor);
286 
287     sanityCheck();
288   }
289 
290   /**
291    * Creates a Bloom filter of the given size.
292    *
293    * @param byteSizeHint the desired number of bytes for the Bloom filter bit
294    *          array. Will be increased so that folding is possible.
295    * @param errorRate target false positive rate of the Bloom filter
296    * @param hashType Bloom filter hash function type
297    * @param foldFactor
298    * @return the new Bloom filter of the desired size
299    */
300   public static ByteBloomFilter createBySize(int byteSizeHint,
301       double errorRate, int hashType, int foldFactor) {
302     ByteBloomFilter bbf = new ByteBloomFilter(hashType);
303 
304     bbf.byteSize = computeFoldableByteSize(byteSizeHint * 8L, foldFactor);
305     long bitSize = bbf.byteSize * 8;
306     bbf.maxKeys = (int) idealMaxKeys(bitSize, errorRate);
307     bbf.hashCount = optimalFunctionCount(bbf.maxKeys, bitSize);
308 
309     // Adjust max keys to bring error rate closer to what was requested,
310     // because byteSize was adjusted to allow for folding, and hashCount was
311     // rounded.
312     bbf.maxKeys = (int) computeMaxKeys(bitSize, errorRate, bbf.hashCount);
313 
314     return bbf;
315   }
316 
317   /**
318    * Creates another similar Bloom filter. Does not copy the actual bits, and
319    * sets the new filter's key count to zero.
320    *
321    * @return a Bloom filter with the same configuration as this
322    */
323   public ByteBloomFilter createAnother() {
324     ByteBloomFilter bbf = new ByteBloomFilter(hashType);
325     bbf.byteSize = byteSize;
326     bbf.hashCount = hashCount;
327     bbf.maxKeys = maxKeys;
328     return bbf;
329   }
330 
331   @Override
332   public void allocBloom() {
333     if (this.bloom != null) {
334       throw new IllegalArgumentException("can only create bloom once.");
335     }
336     this.bloom = ByteBuffer.allocate((int)this.byteSize);
337     assert this.bloom.hasArray();
338   }
339 
340   void sanityCheck() throws IllegalArgumentException {
341     if(0 >= this.byteSize || this.byteSize > Integer.MAX_VALUE) {
342       throw new IllegalArgumentException("Invalid byteSize: " + this.byteSize);
343     }
344 
345     if(this.hashCount <= 0) {
346       throw new IllegalArgumentException("Hash function count must be > 0");
347     }
348 
349     if (this.hash == null) {
350       throw new IllegalArgumentException("hashType must be known");
351     }
352 
353     if (this.keyCount < 0) {
354       throw new IllegalArgumentException("must have positive keyCount");
355     }
356   }
357 
358   void bloomCheck(ByteBuffer bloom)  throws IllegalArgumentException {
359     if (this.byteSize != bloom.limit()) {
360       throw new IllegalArgumentException(
361           "Configured bloom length should match actual length");
362     }
363   }
364 
365   public void add(byte [] buf) {
366     add(buf, 0, buf.length);
367   }
368 
369   @Override
370   public void add(byte [] buf, int offset, int len) {
371     /*
372      * For faster hashing, use combinatorial generation
373      * http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf
374      */
375     int hash1 = this.hash.hash(buf, offset, len, 0);
376     int hash2 = this.hash.hash(buf, offset, len, hash1);
377 
378     for (int i = 0; i < this.hashCount; i++) {
379       long hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8));
380       set(hashLoc);
381     }
382 
383     ++this.keyCount;
384   }
385 
386   /** Should only be used in tests */
387   boolean contains(byte [] buf) {
388     return contains(buf, 0, buf.length, this.bloom);
389   }
390 
391   /** Should only be used in tests */
392   boolean contains(byte [] buf, int offset, int length) {
393     return contains(buf, offset, length, bloom);
394   }
395 
396   /** Should only be used in tests */
397   boolean contains(byte[] buf, ByteBuffer bloom) {
398     return contains(buf, 0, buf.length, bloom);
399   }
400 
401   @Override
402   public boolean contains(byte[] buf, int offset, int length,
403       ByteBuffer theBloom) {
404     if (theBloom == null) {
405       // In a version 1 HFile Bloom filter data is stored in a separate meta
406       // block which is loaded on demand, but in version 2 it is pre-loaded.
407       // We want to use the same API in both cases.
408       theBloom = bloom;
409     }
410 
411     if (theBloom.limit() != byteSize) {
412       throw new IllegalArgumentException("Bloom does not match expected size:"
413           + " theBloom.limit()=" + theBloom.limit() + ", byteSize=" + byteSize);
414     }
415 
416     return contains(buf, offset, length, theBloom, 0, (int) byteSize, hash, hashCount);
417   }
418 
419   public static boolean contains(byte[] buf, int offset, int length,
420       ByteBuffer bloomBuf, int bloomOffset, int bloomSize, Hash hash,
421       int hashCount) {
422 
423     int hash1 = hash.hash(buf, offset, length, 0);
424     int hash2 = hash.hash(buf, offset, length, hash1);
425     int bloomBitSize = bloomSize << 3;
426     
427     if (randomGeneratorForTest == null) {
428       // Production mode.
429       int compositeHash = hash1;
430       for (int i = 0; i < hashCount; i++) {
431         int hashLoc = Math.abs(compositeHash % bloomBitSize);
432         compositeHash += hash2;
433         if (!get(hashLoc, bloomBuf, bloomOffset)) {
434           return false;
435         }
436       }
437     } else {
438       // Test mode with "fake lookups" to estimate "ideal false positive rate".
439       for (int i = 0; i < hashCount; i++) {
440         int hashLoc = randomGeneratorForTest.nextInt(bloomBitSize);
441         if (!get(hashLoc, bloomBuf, bloomOffset)){
442           return false;
443         }
444       }
445     }
446     return true;
447   }
448 
449   //---------------------------------------------------------------------------
450   /** Private helpers */
451 
452   /**
453    * Set the bit at the specified index to 1.
454    *
455    * @param pos index of bit
456    */
457   void set(long pos) {
458     int bytePos = (int)(pos / 8);
459     int bitPos = (int)(pos % 8);
460     byte curByte = bloom.get(bytePos);
461     curByte |= bitvals[bitPos];
462     bloom.put(bytePos, curByte);
463   }
464 
465   /**
466    * Check if bit at specified index is 1.
467    *
468    * @param pos index of bit
469    * @return true if bit at specified index is 1, false if 0.
470    */
471   static boolean get(int pos, ByteBuffer bloomBuf, int bloomOffset) {
472     int bytePos = pos >> 3; //pos / 8
473     int bitPos = pos & 0x7; //pos % 8
474     // TODO access this via Util API which can do Unsafe access if possible(?)
475     byte curByte = bloomBuf.get(bloomOffset + bytePos);
476     curByte &= bitvals[bitPos];
477     return (curByte != 0);
478   }
479 
480   @Override
481   public long getKeyCount() {
482     return keyCount;
483   }
484 
485   @Override
486   public long getMaxKeys() {
487     return maxKeys;
488   }
489 
490   @Override
491   public long getByteSize() {
492     return byteSize;
493   }
494 
495   public int getHashType() {
496     return hashType;
497   }
498 
499   @Override
500   public void compactBloom() {
501     // see if the actual size is exponentially smaller than expected.
502     if (this.keyCount > 0 && this.bloom.hasArray()) {
503       int pieces = 1;
504       int newByteSize = (int)this.byteSize;
505       int newMaxKeys = this.maxKeys;
506 
507       // while exponentially smaller & folding is lossless
508       while ( (newByteSize & 1) == 0 && newMaxKeys > (this.keyCount<<1) ) {
509         pieces <<= 1;
510         newByteSize >>= 1;
511         newMaxKeys >>= 1;
512       }
513 
514       // if we should fold these into pieces
515       if (pieces > 1) {
516         byte[] array = this.bloom.array();
517         int start = this.bloom.arrayOffset();
518         int end = start + newByteSize;
519         int off = end;
520         for(int p = 1; p < pieces; ++p) {
521           for(int pos = start; pos < end; ++pos) {
522             array[pos] |= array[off++];
523           }
524         }
525         // folding done, only use a subset of this array
526         this.bloom.rewind();
527         this.bloom.limit(newByteSize);
528         this.bloom = this.bloom.slice();
529         this.byteSize = newByteSize;
530         this.maxKeys = newMaxKeys;
531       }
532     }
533   }
534 
535 
536   //---------------------------------------------------------------------------
537 
538   /**
539    * Writes just the bloom filter to the output array
540    * @param out OutputStream to place bloom
541    * @throws IOException Error writing bloom array
542    */
543   public void writeBloom(final DataOutput out) throws IOException {
544     if (!this.bloom.hasArray()) {
545       throw new IOException("Only writes ByteBuffer with underlying array.");
546     }
547     out.write(bloom.array(), bloom.arrayOffset(), bloom.limit());
548   }
549 
550   @Override
551   public Writable getMetaWriter() {
552     return new MetaWriter();
553   }
554 
555   @Override
556   public Writable getDataWriter() {
557     return new DataWriter();
558   }
559 
560   private class MetaWriter implements Writable {
561     protected MetaWriter() {}
562     @Override
563     public void readFields(DataInput arg0) throws IOException {
564       throw new IOException("Cant read with this class.");
565     }
566 
567     @Override
568     public void write(DataOutput out) throws IOException {
569       out.writeInt(VERSION);
570       out.writeInt((int) byteSize);
571       out.writeInt(hashCount);
572       out.writeInt(hashType);
573       out.writeInt(keyCount);
574     }
575   }
576 
577   private class DataWriter implements Writable {
578     protected DataWriter() {}
579     @Override
580     public void readFields(DataInput arg0) throws IOException {
581       throw new IOException("Cant read with this class.");
582     }
583 
584     @Override
585     public void write(DataOutput out) throws IOException {
586       writeBloom(out);
587     }
588   }
589 
590   public int getHashCount() {
591     return hashCount;
592   }
593 
594   @Override
595   public boolean supportsAutoLoading() {
596     return bloom != null;
597   }
598 
599   public static void setFakeLookupMode(boolean enabled) {
600     if (enabled) {
601       randomGeneratorForTest = new Random(283742987L);
602     } else {
603       randomGeneratorForTest = null;
604     }
605   }
606 
607   /**
608    * {@inheritDoc}
609    * Just concatenate row and column by default. May return the original row
610    * buffer if the column qualifier is empty.
611    */
612   @Override
613   public byte[] createBloomKey(byte[] rowBuf, int rowOffset, int rowLen,
614       byte[] qualBuf, int qualOffset, int qualLen) {
615     // Optimize the frequent case when only the row is provided.
616     if (qualLen <= 0 && rowOffset == 0 && rowLen == rowBuf.length)
617       return rowBuf;
618 
619     byte [] result = new byte[rowLen + qualLen];
620     System.arraycopy(rowBuf, rowOffset, result, 0,  rowLen);
621     if (qualLen > 0)
622       System.arraycopy(qualBuf, qualOffset, result, rowLen, qualLen);
623     return result;
624   }
625 
626   /**
627    * A human-readable string with statistics for the given Bloom filter.
628    *
629    * @param bloomFilter the Bloom filter to output statistics for;
630    * @return a string consisting of "&lt;key&gt;: &lt;value&gt;" parts
631    *         separated by {@link #STATS_RECORD_SEP}.
632    */
633   public static String formatStats(BloomFilterBase bloomFilter) {
634     StringBuilder sb = new StringBuilder();
635     long k = bloomFilter.getKeyCount();
636     long m = bloomFilter.getMaxKeys();
637 
638     sb.append("BloomSize: " + bloomFilter.getByteSize() + STATS_RECORD_SEP);
639     sb.append("No of Keys in bloom: " + k + STATS_RECORD_SEP);
640     sb.append("Max Keys for bloom: " + m);
641     if (m > 0) {
642       sb.append(STATS_RECORD_SEP + "Percentage filled: "
643           + NumberFormat.getPercentInstance().format(k * 1.0 / m));
644     }
645     return sb.toString();
646   }
647 
648   @Override
649   public String toString() {
650     return formatStats(this) + STATS_RECORD_SEP + "Actual error rate: "
651         + String.format("%.8f", actualErrorRate());
652   }
653 
654 }