View Javadoc

1   /*
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  
21  package org.apache.hadoop.hbase.util;
22  
23  import org.apache.hadoop.io.RawComparator;
24  import org.apache.hadoop.io.Writable;
25  
26  import java.io.DataInput;
27  import java.io.DataOutput;
28  import java.io.IOException;
29  import java.nio.ByteBuffer;
30  import java.text.NumberFormat;
31  import java.util.Random;
32  
33  /**
34   * Implements a <i>Bloom filter</i>, as defined by Bloom in 1970.
35   * <p>
36   * The Bloom filter is a data structure that was introduced in 1970 and that has
37   * been adopted by the networking research community in the past decade thanks
38   * to the bandwidth efficiencies that it offers for the transmission of set
39   * membership information between networked hosts. A sender encodes the
40   * information into a bit vector, the Bloom filter, that is more compact than a
41   * conventional representation. Computation and space costs for construction are
42   * linear in the number of elements. The receiver uses the filter to test
43   * whether various elements are members of the set. Though the filter will
44   * occasionally return a false positive, it will never return a false negative.
45   * When creating the filter, the sender can choose its desired point in a
46   * trade-off between the false positive rate and the size.
47   *
48   * <p>
49   * Originally inspired by <a href="http://www.one-lab.org">European Commission
50   * One-Lab Project 034819</a>.
51   *
52   * Bloom filters are very sensitive to the number of elements inserted into
53   * them. For HBase, the number of entries depends on the size of the data stored
54   * in the column. Currently the default region size is 256MB, so entry count ~=
55   * 256MB / (average value size for column). Despite this rule of thumb, there is
56   * no efficient way to calculate the entry count after compactions. Therefore,
57   * it is often easier to use a dynamic bloom filter that will add extra space
58   * instead of allowing the error rate to grow.
59   *
60   * ( http://www.eecs.harvard.edu/~michaelm/NEWWORK/postscripts/BloomFilterSurvey
61   * .pdf )
62   *
63   * m denotes the number of bits in the Bloom filter (bitSize) n denotes the
64   * number of elements inserted into the Bloom filter (maxKeys) k represents the
65   * number of hash functions used (nbHash) e represents the desired false
66   * positive rate for the bloom (err)
67   *
68   * If we fix the error rate (e) and know the number of entries, then the optimal
69   * bloom size m = -(n * ln(err) / (ln(2)^2) ~= n * ln(err) / ln(0.6185)
70   *
71   * The probability of false positives is minimized when k = m/n ln(2).
72   *
73   * @see BloomFilter The general behavior of a filter
74   *
75   * @see <a
76   *      href="http://portal.acm.org/citation.cfm?id=362692&dl=ACM&coll=portal">
77   *      Space/Time Trade-Offs in Hash Coding with Allowable Errors</a>
78   */
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     return (int) Math.ceil(Math.log(2) * (bitSize / maxKeys));
248   }
249 
250   /** Private constructor used by other constructors. */
251   private ByteBloomFilter(int hashType) {
252     this.hashType = hashType;
253     this.hash = Hash.getInstance(hashType);
254   }
255 
256   /**
257    * Determines & initializes bloom filter meta data from user config. Call
258    * {@link #allocBloom()} to allocate bloom filter data.
259    *
260    * @param maxKeys Maximum expected number of keys that will be stored in this
261    *          bloom
262    * @param errorRate Desired false positive error rate. Lower rate = more
263    *          storage required
264    * @param hashType Type of hash function to use
265    * @param foldFactor When finished adding entries, you may be able to 'fold'
266    *          this bloom to save space. Tradeoff potentially excess bytes in
267    *          bloom for ability to fold if keyCount is exponentially greater
268    *          than maxKeys.
269    * @throws IllegalArgumentException
270    */
271   public ByteBloomFilter(int maxKeys, double errorRate, int hashType,
272       int foldFactor) throws IllegalArgumentException {
273     this(hashType);
274 
275     long bitSize = computeBitSize(maxKeys, errorRate);
276     hashCount = optimalFunctionCount(maxKeys, bitSize);
277     this.maxKeys = maxKeys;
278 
279     // increase byteSize so folding is possible
280     byteSize = computeFoldableByteSize(bitSize, foldFactor);
281 
282     sanityCheck();
283   }
284 
285   /**
286    * Creates a Bloom filter of the given size.
287    *
288    * @param byteSizeHint the desired number of bytes for the Bloom filter bit
289    *          array. Will be increased so that folding is possible.
290    * @param errorRate target false positive rate of the Bloom filter
291    * @param hashType Bloom filter hash function type
292    * @param foldFactor
293    * @return the new Bloom filter of the desired size
294    */
295   public static ByteBloomFilter createBySize(int byteSizeHint,
296       double errorRate, int hashType, int foldFactor) {
297     ByteBloomFilter bbf = new ByteBloomFilter(hashType);
298 
299     bbf.byteSize = computeFoldableByteSize(byteSizeHint * 8, foldFactor);
300     long bitSize = bbf.byteSize * 8;
301     bbf.maxKeys = (int) idealMaxKeys(bitSize, errorRate);
302     bbf.hashCount = optimalFunctionCount(bbf.maxKeys, bitSize);
303 
304     // Adjust max keys to bring error rate closer to what was requested,
305     // because byteSize was adjusted to allow for folding, and hashCount was
306     // rounded.
307     bbf.maxKeys = (int) computeMaxKeys(bitSize, errorRate, bbf.hashCount);
308 
309     return bbf;
310   }
311 
312   /**
313    * Creates another similar Bloom filter. Does not copy the actual bits, and
314    * sets the new filter's key count to zero.
315    *
316    * @return a Bloom filter with the same configuration as this
317    */
318   public ByteBloomFilter createAnother() {
319     ByteBloomFilter bbf = new ByteBloomFilter(hashType);
320     bbf.byteSize = byteSize;
321     bbf.hashCount = hashCount;
322     bbf.maxKeys = maxKeys;
323     return bbf;
324   }
325 
326   @Override
327   public void allocBloom() {
328     if (this.bloom != null) {
329       throw new IllegalArgumentException("can only create bloom once.");
330     }
331     this.bloom = ByteBuffer.allocate((int)this.byteSize);
332     assert this.bloom.hasArray();
333   }
334 
335   void sanityCheck() throws IllegalArgumentException {
336     if(0 >= this.byteSize || this.byteSize > Integer.MAX_VALUE) {
337       throw new IllegalArgumentException("Invalid byteSize: " + this.byteSize);
338     }
339 
340     if(this.hashCount <= 0) {
341       throw new IllegalArgumentException("Hash function count must be > 0");
342     }
343 
344     if (this.hash == null) {
345       throw new IllegalArgumentException("hashType must be known");
346     }
347 
348     if (this.keyCount < 0) {
349       throw new IllegalArgumentException("must have positive keyCount");
350     }
351   }
352 
353   void bloomCheck(ByteBuffer bloom)  throws IllegalArgumentException {
354     if (this.byteSize != bloom.limit()) {
355       throw new IllegalArgumentException(
356           "Configured bloom length should match actual length");
357     }
358   }
359 
360   public void add(byte [] buf) {
361     add(buf, 0, buf.length);
362   }
363 
364   @Override
365   public void add(byte [] buf, int offset, int len) {
366     /*
367      * For faster hashing, use combinatorial generation
368      * http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf
369      */
370     int hash1 = this.hash.hash(buf, offset, len, 0);
371     int hash2 = this.hash.hash(buf, offset, len, hash1);
372 
373     for (int i = 0; i < this.hashCount; i++) {
374       long hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8));
375       set(hashLoc);
376     }
377 
378     ++this.keyCount;
379   }
380 
381   /** Should only be used in tests */
382   boolean contains(byte [] buf) {
383     return contains(buf, 0, buf.length, this.bloom);
384   }
385 
386   /** Should only be used in tests */
387   boolean contains(byte [] buf, int offset, int length) {
388     return contains(buf, offset, length, bloom);
389   }
390 
391   /** Should only be used in tests */
392   boolean contains(byte[] buf, ByteBuffer bloom) {
393     return contains(buf, 0, buf.length, bloom);
394   }
395 
396   @Override
397   public boolean contains(byte[] buf, int offset, int length,
398       ByteBuffer theBloom) {
399     if (theBloom == null) {
400       // In a version 1 HFile Bloom filter data is stored in a separate meta
401       // block which is loaded on demand, but in version 2 it is pre-loaded.
402       // We want to use the same API in both cases.
403       theBloom = bloom;
404     }
405 
406     if (theBloom.limit() != byteSize) {
407       throw new IllegalArgumentException("Bloom does not match expected size:"
408           + " theBloom.limit()=" + theBloom.limit() + ", byteSize=" + byteSize);
409     }
410 
411     return contains(buf, offset, length, theBloom.array(),
412         theBloom.arrayOffset(), (int) byteSize, hash, hashCount);
413   }
414 
415   public static boolean contains(byte[] buf, int offset, int length,
416       byte[] bloomArray, int bloomOffset, int bloomSize, Hash hash,
417       int hashCount) {
418 
419     int hash1 = hash.hash(buf, offset, length, 0);
420     int hash2 = hash.hash(buf, offset, length, hash1);
421     int bloomBitSize = bloomSize << 3;
422     
423     if (randomGeneratorForTest == null) {
424       // Production mode.
425       int compositeHash = hash1;
426       for (int i = 0; i < hashCount; i++) {
427         int hashLoc = Math.abs(compositeHash % bloomBitSize);
428         compositeHash += hash2;
429         if (!get(hashLoc, bloomArray, bloomOffset)) {
430           return false;
431         }
432       }
433     } else {
434       // Test mode with "fake lookups" to estimate "ideal false positive rate".
435       for (int i = 0; i < hashCount; i++) {
436         int hashLoc = randomGeneratorForTest.nextInt(bloomBitSize);
437         if (!get(hashLoc, bloomArray, bloomOffset)){
438           return false;
439         }
440       }
441     }
442     return true;
443   }
444 
445   //---------------------------------------------------------------------------
446   /** Private helpers */
447 
448   /**
449    * Set the bit at the specified index to 1.
450    *
451    * @param pos index of bit
452    */
453   void set(long pos) {
454     int bytePos = (int)(pos / 8);
455     int bitPos = (int)(pos % 8);
456     byte curByte = bloom.get(bytePos);
457     curByte |= bitvals[bitPos];
458     bloom.put(bytePos, curByte);
459   }
460 
461   /**
462    * Check if bit at specified index is 1.
463    *
464    * @param pos index of bit
465    * @return true if bit at specified index is 1, false if 0.
466    */
467   static boolean get(int pos, byte[] bloomArray, int bloomOffset) {
468     int bytePos = pos >> 3; //pos / 8
469     int bitPos = pos & 0x7; //pos % 8
470     byte curByte = bloomArray[bloomOffset + bytePos];
471     curByte &= bitvals[bitPos];
472     return (curByte != 0);
473   }
474 
475   @Override
476   public long getKeyCount() {
477     return keyCount;
478   }
479 
480   @Override
481   public long getMaxKeys() {
482     return maxKeys;
483   }
484 
485   @Override
486   public long getByteSize() {
487     return byteSize;
488   }
489 
490   public int getHashType() {
491     return hashType;
492   }
493 
494   @Override
495   public void compactBloom() {
496     // see if the actual size is exponentially smaller than expected.
497     if (this.keyCount > 0 && this.bloom.hasArray()) {
498       int pieces = 1;
499       int newByteSize = (int)this.byteSize;
500       int newMaxKeys = this.maxKeys;
501 
502       // while exponentially smaller & folding is lossless
503       while ( (newByteSize & 1) == 0 && newMaxKeys > (this.keyCount<<1) ) {
504         pieces <<= 1;
505         newByteSize >>= 1;
506         newMaxKeys >>= 1;
507       }
508 
509       // if we should fold these into pieces
510       if (pieces > 1) {
511         byte[] array = this.bloom.array();
512         int start = this.bloom.arrayOffset();
513         int end = start + newByteSize;
514         int off = end;
515         for(int p = 1; p < pieces; ++p) {
516           for(int pos = start; pos < end; ++pos) {
517             array[pos] |= array[off++];
518           }
519         }
520         // folding done, only use a subset of this array
521         this.bloom.rewind();
522         this.bloom.limit(newByteSize);
523         this.bloom = this.bloom.slice();
524         this.byteSize = newByteSize;
525         this.maxKeys = newMaxKeys;
526       }
527     }
528   }
529 
530 
531   //---------------------------------------------------------------------------
532 
533   /**
534    * Writes just the bloom filter to the output array
535    * @param out OutputStream to place bloom
536    * @throws IOException Error writing bloom array
537    */
538   public void writeBloom(final DataOutput out) throws IOException {
539     if (!this.bloom.hasArray()) {
540       throw new IOException("Only writes ByteBuffer with underlying array.");
541     }
542     out.write(bloom.array(), bloom.arrayOffset(), bloom.limit());
543   }
544 
545   @Override
546   public Writable getMetaWriter() {
547     return new MetaWriter();
548   }
549 
550   @Override
551   public Writable getDataWriter() {
552     return new DataWriter();
553   }
554 
555   private class MetaWriter implements Writable {
556     protected MetaWriter() {}
557     @Override
558     public void readFields(DataInput arg0) throws IOException {
559       throw new IOException("Cant read with this class.");
560     }
561 
562     @Override
563     public void write(DataOutput out) throws IOException {
564       out.writeInt(VERSION);
565       out.writeInt((int) byteSize);
566       out.writeInt(hashCount);
567       out.writeInt(hashType);
568       out.writeInt(keyCount);
569     }
570   }
571 
572   private class DataWriter implements Writable {
573     protected DataWriter() {}
574     @Override
575     public void readFields(DataInput arg0) throws IOException {
576       throw new IOException("Cant read with this class.");
577     }
578 
579     @Override
580     public void write(DataOutput out) throws IOException {
581       writeBloom(out);
582     }
583   }
584 
585   public int getHashCount() {
586     return hashCount;
587   }
588 
589   @Override
590   public boolean supportsAutoLoading() {
591     return bloom != null;
592   }
593 
594   public static void setFakeLookupMode(boolean enabled) {
595     if (enabled) {
596       randomGeneratorForTest = new Random(283742987L);
597     } else {
598       randomGeneratorForTest = null;
599     }
600   }
601 
602   /**
603    * {@inheritDoc}
604    * Just concatenate row and column by default. May return the original row
605    * buffer if the column qualifier is empty.
606    */
607   @Override
608   public byte[] createBloomKey(byte[] rowBuf, int rowOffset, int rowLen,
609       byte[] qualBuf, int qualOffset, int qualLen) {
610     // Optimize the frequent case when only the row is provided.
611     if (qualLen <= 0 && rowOffset == 0 && rowLen == rowBuf.length)
612       return rowBuf;
613 
614     byte [] result = new byte[rowLen + qualLen];
615     System.arraycopy(rowBuf, rowOffset, result, 0,  rowLen);
616     if (qualLen > 0)
617       System.arraycopy(qualBuf, qualOffset, result, rowLen, qualLen);
618     return result;
619   }
620 
621   @Override
622   public RawComparator<byte[]> getComparator() {
623     return Bytes.BYTES_RAWCOMPARATOR;
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 }