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 org.apache.hadoop.classification.InterfaceAudience;
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  @InterfaceAudience.Private
80  public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
81  
82    /** Current file format version */
83    public static final int VERSION = 1;
84  
85    /** Bytes (B) in the array. This actually has to fit into an int. */
86    protected long byteSize;
87    /** Number of hash functions */
88    protected int hashCount;
89    /** Hash type */
90    protected final int hashType;
91    /** Hash Function */
92    protected final Hash hash;
93    /** Keys currently in the bloom */
94    protected int keyCount;
95    /** Max Keys expected for the bloom */
96    protected int maxKeys;
97    /** Bloom bits */
98    protected ByteBuffer bloom;
99  
100   /** Record separator for the Bloom filter statistics human-readable string */
101   public static final String STATS_RECORD_SEP = "; ";
102 
103   /**
104    * Used in computing the optimal Bloom filter size. This approximately equals
105    * 0.480453.
106    */
107   public static final double LOG2_SQUARED = Math.log(2) * Math.log(2);
108 
109   /**
110    * A random number generator to use for "fake lookups" when testing to
111    * estimate the ideal false positive rate.
112    */
113   private static Random randomGeneratorForTest;
114 
115   /** Bit-value lookup array to prevent doing the same work over and over */
116   private static final byte [] bitvals = {
117     (byte) 0x01,
118     (byte) 0x02,
119     (byte) 0x04,
120     (byte) 0x08,
121     (byte) 0x10,
122     (byte) 0x20,
123     (byte) 0x40,
124     (byte) 0x80
125   };
126 
127   /**
128    * Loads bloom filter meta data from file input.
129    * @param meta stored bloom meta data
130    * @throws IllegalArgumentException meta data is invalid
131    */
132   public ByteBloomFilter(DataInput meta)
133       throws IOException, IllegalArgumentException {
134     this.byteSize = meta.readInt();
135     this.hashCount = meta.readInt();
136     this.hashType = meta.readInt();
137     this.keyCount = meta.readInt();
138     this.maxKeys = this.keyCount;
139 
140     this.hash = Hash.getInstance(this.hashType);
141     if (hash == null) {
142       throw new IllegalArgumentException("Invalid hash type: " + hashType);
143     }
144     sanityCheck();
145   }
146 
147   /**
148    * @param maxKeys
149    * @param errorRate
150    * @return the number of bits for a Bloom filter than can hold the given
151    *         number of keys and provide the given error rate, assuming that the
152    *         optimal number of hash functions is used and it does not have to
153    *         be an integer.
154    */
155   public static long computeBitSize(long maxKeys, double errorRate) {
156     return (long) Math.ceil(maxKeys * (-Math.log(errorRate) / LOG2_SQUARED));
157   }
158 
159   /**
160    * The maximum number of keys we can put into a Bloom filter of a certain
161    * size to maintain the given error rate, assuming the number of hash
162    * functions is chosen optimally and does not even have to be an integer
163    * (hence the "ideal" in the function name).
164    *
165    * @param bitSize
166    * @param errorRate
167    * @return maximum number of keys that can be inserted into the Bloom filter
168    * @see #computeMaxKeys(long, double, int) for a more precise estimate
169    */
170   public static long idealMaxKeys(long bitSize, double errorRate) {
171     // The reason we need to use floor here is that otherwise we might put
172     // more keys in a Bloom filter than is allowed by the target error rate.
173     return (long) (bitSize * (LOG2_SQUARED / -Math.log(errorRate)));
174   }
175 
176   /**
177    * The maximum number of keys we can put into a Bloom filter of a certain
178    * size to get the given error rate, with the given number of hash functions.
179    *
180    * @param bitSize
181    * @param errorRate
182    * @param hashCount
183    * @return the maximum number of keys that can be inserted in a Bloom filter
184    *         to maintain the target error rate, if the number of hash functions
185    *         is provided.
186    */
187   public static long computeMaxKeys(long bitSize, double errorRate,
188       int hashCount) {
189     return (long) (-bitSize * 1.0 / hashCount *
190         Math.log(1 - Math.exp(Math.log(errorRate) / hashCount)));
191   }
192 
193   /**
194    * Computes the error rate for this Bloom filter, taking into account the
195    * actual number of hash functions and keys inserted. The return value of
196    * this function changes as a Bloom filter is being populated. Used for
197    * reporting the actual error rate of compound Bloom filters when writing
198    * them out.
199    *
200    * @return error rate for this particular Bloom filter
201    */
202   public double actualErrorRate() {
203     return actualErrorRate(keyCount, byteSize * 8, hashCount);
204   }
205 
206   /**
207    * Computes the actual error rate for the given number of elements, number
208    * of bits, and number of hash functions. Taken directly from the
209    * <a href=
210    * "http://en.wikipedia.org/wiki/Bloom_filter#Probability_of_false_positives"
211    * > Wikipedia Bloom filter article</a>.
212    *
213    * @param maxKeys
214    * @param bitSize
215    * @param functionCount
216    * @return the actual error rate
217    */
218   public static double actualErrorRate(long maxKeys, long bitSize,
219       int functionCount) {
220     return Math.exp(Math.log(1 - Math.exp(-functionCount * maxKeys * 1.0
221         / bitSize)) * functionCount);
222   }
223 
224   /**
225    * Increases the given byte size of a Bloom filter until it can be folded by
226    * the given factor.
227    *
228    * @param bitSize
229    * @param foldFactor
230    * @return Foldable byte size
231    */
232   public static int computeFoldableByteSize(long bitSize, int foldFactor) {
233     long byteSizeLong = (bitSize + 7) / 8;
234     int mask = (1 << foldFactor) - 1;
235     if ((mask & byteSizeLong) != 0) {
236       byteSizeLong >>= foldFactor;
237       ++byteSizeLong;
238       byteSizeLong <<= foldFactor;
239     }
240     if (byteSizeLong > Integer.MAX_VALUE) {
241       throw new IllegalArgumentException("byteSize=" + byteSizeLong + " too "
242           + "large for bitSize=" + bitSize + ", foldFactor=" + foldFactor);
243     }
244     return (int) byteSizeLong;
245   }
246 
247   private static int optimalFunctionCount(int maxKeys, long bitSize) {
248     long i = bitSize / maxKeys;
249     double result = Math.ceil(Math.log(2) * i);
250     if (result > Integer.MAX_VALUE){
251       throw new IllegalArgumentException("result too large for integer value.");
252     }
253     return (int)result;
254   }
255 
256   /** Private constructor used by other constructors. */
257   private ByteBloomFilter(int hashType) {
258     this.hashType = hashType;
259     this.hash = Hash.getInstance(hashType);
260   }
261 
262   /**
263    * Determines & initializes bloom filter meta data from user config. Call
264    * {@link #allocBloom()} to allocate bloom filter data.
265    *
266    * @param maxKeys Maximum expected number of keys that will be stored in this
267    *          bloom
268    * @param errorRate Desired false positive error rate. Lower rate = more
269    *          storage required
270    * @param hashType Type of hash function to use
271    * @param foldFactor When finished adding entries, you may be able to 'fold'
272    *          this bloom to save space. Tradeoff potentially excess bytes in
273    *          bloom for ability to fold if keyCount is exponentially greater
274    *          than maxKeys.
275    * @throws IllegalArgumentException
276    */
277   public ByteBloomFilter(int maxKeys, double errorRate, int hashType,
278       int foldFactor) throws IllegalArgumentException {
279     this(hashType);
280 
281     long bitSize = computeBitSize(maxKeys, errorRate);
282     hashCount = optimalFunctionCount(maxKeys, bitSize);
283     this.maxKeys = maxKeys;
284 
285     // increase byteSize so folding is possible
286     byteSize = computeFoldableByteSize(bitSize, foldFactor);
287 
288     sanityCheck();
289   }
290 
291   /**
292    * Creates a Bloom filter of the given size.
293    *
294    * @param byteSizeHint the desired number of bytes for the Bloom filter bit
295    *          array. Will be increased so that folding is possible.
296    * @param errorRate target false positive rate of the Bloom filter
297    * @param hashType Bloom filter hash function type
298    * @param foldFactor
299    * @return the new Bloom filter of the desired size
300    */
301   public static ByteBloomFilter createBySize(int byteSizeHint,
302       double errorRate, int hashType, int foldFactor) {
303     ByteBloomFilter bbf = new ByteBloomFilter(hashType);
304 
305     bbf.byteSize = computeFoldableByteSize(byteSizeHint * 8L, foldFactor);
306     long bitSize = bbf.byteSize * 8;
307     bbf.maxKeys = (int) idealMaxKeys(bitSize, errorRate);
308     bbf.hashCount = optimalFunctionCount(bbf.maxKeys, bitSize);
309 
310     // Adjust max keys to bring error rate closer to what was requested,
311     // because byteSize was adjusted to allow for folding, and hashCount was
312     // rounded.
313     bbf.maxKeys = (int) computeMaxKeys(bitSize, errorRate, bbf.hashCount);
314 
315     return bbf;
316   }
317 
318   /**
319    * Creates another similar Bloom filter. Does not copy the actual bits, and
320    * sets the new filter's key count to zero.
321    *
322    * @return a Bloom filter with the same configuration as this
323    */
324   public ByteBloomFilter createAnother() {
325     ByteBloomFilter bbf = new ByteBloomFilter(hashType);
326     bbf.byteSize = byteSize;
327     bbf.hashCount = hashCount;
328     bbf.maxKeys = maxKeys;
329     return bbf;
330   }
331 
332   @Override
333   public void allocBloom() {
334     if (this.bloom != null) {
335       throw new IllegalArgumentException("can only create bloom once.");
336     }
337     this.bloom = ByteBuffer.allocate((int)this.byteSize);
338     assert this.bloom.hasArray();
339   }
340 
341   void sanityCheck() throws IllegalArgumentException {
342     if(0 >= this.byteSize || this.byteSize > Integer.MAX_VALUE) {
343       throw new IllegalArgumentException("Invalid byteSize: " + this.byteSize);
344     }
345 
346     if(this.hashCount <= 0) {
347       throw new IllegalArgumentException("Hash function count must be > 0");
348     }
349 
350     if (this.hash == null) {
351       throw new IllegalArgumentException("hashType must be known");
352     }
353 
354     if (this.keyCount < 0) {
355       throw new IllegalArgumentException("must have positive keyCount");
356     }
357   }
358 
359   void bloomCheck(ByteBuffer bloom)  throws IllegalArgumentException {
360     if (this.byteSize != bloom.limit()) {
361       throw new IllegalArgumentException(
362           "Configured bloom length should match actual length");
363     }
364   }
365 
366   public void add(byte [] buf) {
367     add(buf, 0, buf.length);
368   }
369 
370   @Override
371   public void add(byte [] buf, int offset, int len) {
372     /*
373      * For faster hashing, use combinatorial generation
374      * http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf
375      */
376     int hash1 = this.hash.hash(buf, offset, len, 0);
377     int hash2 = this.hash.hash(buf, offset, len, hash1);
378 
379     for (int i = 0; i < this.hashCount; i++) {
380       long hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8));
381       set(hashLoc);
382     }
383 
384     ++this.keyCount;
385   }
386 
387   /** Should only be used in tests */
388   boolean contains(byte [] buf) {
389     return contains(buf, 0, buf.length, this.bloom);
390   }
391 
392   /** Should only be used in tests */
393   boolean contains(byte [] buf, int offset, int length) {
394     return contains(buf, offset, length, bloom);
395   }
396 
397   /** Should only be used in tests */
398   boolean contains(byte[] buf, ByteBuffer bloom) {
399     return contains(buf, 0, buf.length, bloom);
400   }
401 
402   @Override
403   public boolean contains(byte[] buf, int offset, int length,
404       ByteBuffer theBloom) {
405     if (theBloom == null) {
406       // In a version 1 HFile Bloom filter data is stored in a separate meta
407       // block which is loaded on demand, but in version 2 it is pre-loaded.
408       // We want to use the same API in both cases.
409       theBloom = bloom;
410     }
411 
412     if (theBloom.limit() != byteSize) {
413       throw new IllegalArgumentException("Bloom does not match expected size:"
414           + " theBloom.limit()=" + theBloom.limit() + ", byteSize=" + byteSize);
415     }
416 
417     return contains(buf, offset, length, theBloom.array(),
418         theBloom.arrayOffset(), (int) byteSize, hash, hashCount);
419   }
420 
421   public static boolean contains(byte[] buf, int offset, int length,
422       byte[] bloomArray, int bloomOffset, int bloomSize, Hash hash,
423       int hashCount) {
424 
425     int hash1 = hash.hash(buf, offset, length, 0);
426     int hash2 = hash.hash(buf, offset, length, hash1);
427     int bloomBitSize = bloomSize << 3;
428     
429     if (randomGeneratorForTest == null) {
430       // Production mode.
431       int compositeHash = hash1;
432       for (int i = 0; i < hashCount; i++) {
433         int hashLoc = Math.abs(compositeHash % bloomBitSize);
434         compositeHash += hash2;
435         if (!get(hashLoc, bloomArray, bloomOffset)) {
436           return false;
437         }
438       }
439     } else {
440       // Test mode with "fake lookups" to estimate "ideal false positive rate".
441       for (int i = 0; i < hashCount; i++) {
442         int hashLoc = randomGeneratorForTest.nextInt(bloomBitSize);
443         if (!get(hashLoc, bloomArray, bloomOffset)){
444           return false;
445         }
446       }
447     }
448     return true;
449   }
450 
451   //---------------------------------------------------------------------------
452   /** Private helpers */
453 
454   /**
455    * Set the bit at the specified index to 1.
456    *
457    * @param pos index of bit
458    */
459   void set(long pos) {
460     int bytePos = (int)(pos / 8);
461     int bitPos = (int)(pos % 8);
462     byte curByte = bloom.get(bytePos);
463     curByte |= bitvals[bitPos];
464     bloom.put(bytePos, curByte);
465   }
466 
467   /**
468    * Check if bit at specified index is 1.
469    *
470    * @param pos index of bit
471    * @return true if bit at specified index is 1, false if 0.
472    */
473   static boolean get(int pos, byte[] bloomArray, int bloomOffset) {
474     int bytePos = pos >> 3; //pos / 8
475     int bitPos = pos & 0x7; //pos % 8
476     byte curByte = bloomArray[bloomOffset + bytePos];
477     curByte &= bitvals[bitPos];
478     return (curByte != 0);
479   }
480 
481   @Override
482   public long getKeyCount() {
483     return keyCount;
484   }
485 
486   @Override
487   public long getMaxKeys() {
488     return maxKeys;
489   }
490 
491   @Override
492   public long getByteSize() {
493     return byteSize;
494   }
495 
496   public int getHashType() {
497     return hashType;
498   }
499 
500   @Override
501   public void compactBloom() {
502     // see if the actual size is exponentially smaller than expected.
503     if (this.keyCount > 0 && this.bloom.hasArray()) {
504       int pieces = 1;
505       int newByteSize = (int)this.byteSize;
506       int newMaxKeys = this.maxKeys;
507 
508       // while exponentially smaller & folding is lossless
509       while ( (newByteSize & 1) == 0 && newMaxKeys > (this.keyCount<<1) ) {
510         pieces <<= 1;
511         newByteSize >>= 1;
512         newMaxKeys >>= 1;
513       }
514 
515       // if we should fold these into pieces
516       if (pieces > 1) {
517         byte[] array = this.bloom.array();
518         int start = this.bloom.arrayOffset();
519         int end = start + newByteSize;
520         int off = end;
521         for(int p = 1; p < pieces; ++p) {
522           for(int pos = start; pos < end; ++pos) {
523             array[pos] |= array[off++];
524           }
525         }
526         // folding done, only use a subset of this array
527         this.bloom.rewind();
528         this.bloom.limit(newByteSize);
529         this.bloom = this.bloom.slice();
530         this.byteSize = newByteSize;
531         this.maxKeys = newMaxKeys;
532       }
533     }
534   }
535 
536 
537   //---------------------------------------------------------------------------
538 
539   /**
540    * Writes just the bloom filter to the output array
541    * @param out OutputStream to place bloom
542    * @throws IOException Error writing bloom array
543    */
544   public void writeBloom(final DataOutput out) throws IOException {
545     if (!this.bloom.hasArray()) {
546       throw new IOException("Only writes ByteBuffer with underlying array.");
547     }
548     out.write(bloom.array(), bloom.arrayOffset(), bloom.limit());
549   }
550 
551   @Override
552   public Writable getMetaWriter() {
553     return new MetaWriter();
554   }
555 
556   @Override
557   public Writable getDataWriter() {
558     return new DataWriter();
559   }
560 
561   private class MetaWriter implements Writable {
562     protected MetaWriter() {}
563     @Override
564     public void readFields(DataInput arg0) throws IOException {
565       throw new IOException("Cant read with this class.");
566     }
567 
568     @Override
569     public void write(DataOutput out) throws IOException {
570       out.writeInt(VERSION);
571       out.writeInt((int) byteSize);
572       out.writeInt(hashCount);
573       out.writeInt(hashType);
574       out.writeInt(keyCount);
575     }
576   }
577 
578   private class DataWriter implements Writable {
579     protected DataWriter() {}
580     @Override
581     public void readFields(DataInput arg0) throws IOException {
582       throw new IOException("Cant read with this class.");
583     }
584 
585     @Override
586     public void write(DataOutput out) throws IOException {
587       writeBloom(out);
588     }
589   }
590 
591   public int getHashCount() {
592     return hashCount;
593   }
594 
595   @Override
596   public boolean supportsAutoLoading() {
597     return bloom != null;
598   }
599 
600   public static void setFakeLookupMode(boolean enabled) {
601     if (enabled) {
602       randomGeneratorForTest = new Random(283742987L);
603     } else {
604       randomGeneratorForTest = null;
605     }
606   }
607 
608   /**
609    * {@inheritDoc}
610    * Just concatenate row and column by default. May return the original row
611    * buffer if the column qualifier is empty.
612    */
613   @Override
614   public byte[] createBloomKey(byte[] rowBuf, int rowOffset, int rowLen,
615       byte[] qualBuf, int qualOffset, int qualLen) {
616     // Optimize the frequent case when only the row is provided.
617     if (qualLen <= 0 && rowOffset == 0 && rowLen == rowBuf.length)
618       return rowBuf;
619 
620     byte [] result = new byte[rowLen + qualLen];
621     System.arraycopy(rowBuf, rowOffset, result, 0,  rowLen);
622     if (qualLen > 0)
623       System.arraycopy(qualBuf, qualOffset, result, rowLen, qualLen);
624     return result;
625   }
626 
627   @Override
628   public RawComparator<byte[]> getComparator() {
629     return Bytes.BYTES_RAWCOMPARATOR;
630   }
631 
632   /**
633    * A human-readable string with statistics for the given Bloom filter.
634    *
635    * @param bloomFilter the Bloom filter to output statistics for;
636    * @return a string consisting of "&lt;key&gt;: &lt;value&gt;" parts
637    *         separated by {@link #STATS_RECORD_SEP}.
638    */
639   public static String formatStats(BloomFilterBase bloomFilter) {
640     StringBuilder sb = new StringBuilder();
641     long k = bloomFilter.getKeyCount();
642     long m = bloomFilter.getMaxKeys();
643 
644     sb.append("BloomSize: " + bloomFilter.getByteSize() + STATS_RECORD_SEP);
645     sb.append("No of Keys in bloom: " + k + STATS_RECORD_SEP);
646     sb.append("Max Keys for bloom: " + m);
647     if (m > 0) {
648       sb.append(STATS_RECORD_SEP + "Percentage filled: "
649           + NumberFormat.getPercentInstance().format(k * 1.0 / m));
650     }
651     return sb.toString();
652   }
653 
654   @Override
655   public String toString() {
656     return formatStats(this) + STATS_RECORD_SEP + "Actual error rate: "
657         + String.format("%.8f", actualErrorRate());
658   }
659 
660 }