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.Writable;
24  
25  import java.io.DataInput;
26  import java.io.DataOutput;
27  import java.io.IOException;
28  import java.nio.ByteBuffer;
29  
30  /**
31   * Implements a <i>Bloom filter</i>, as defined by Bloom in 1970.
32   * <p>
33   * The Bloom filter is a data structure that was introduced in 1970 and that has been adopted by
34   * the networking research community in the past decade thanks to the bandwidth efficiencies that it
35   * offers for the transmission of set membership information between networked hosts.  A sender encodes
36   * the information into a bit vector, the Bloom filter, that is more compact than a conventional
37   * representation. Computation and space costs for construction are linear in the number of elements.
38   * The receiver uses the filter to test whether various elements are members of the set. Though the
39   * filter will occasionally return a false positive, it will never return a false negative. When creating
40   * the filter, the sender can choose its desired point in a trade-off between the false positive rate and the size.
41   *
42   * <p>
43   * Originally inspired by
44   * <a href="http://www.one-lab.org">European Commission One-Lab Project 034819</a>.
45   *
46   * @see BloomFilter The general behavior of a filter
47   *
48   * @see <a href="http://portal.acm.org/citation.cfm?id=362692&dl=ACM&coll=portal">Space/Time Trade-Offs in Hash Coding with Allowable Errors</a>
49   */
50  public class ByteBloomFilter implements BloomFilter {
51    /** Current file format version */
52    public static final int VERSION = 1;
53  
54    /** Bytes (B) in the array */
55    protected long byteSize;
56    /** Number of hash functions */
57    protected final int hashCount;
58    /** Hash type */
59    protected final int hashType;
60    /** Hash Function */
61    protected final Hash hash;
62    /** Keys currently in the bloom */
63    protected int keyCount;
64    /** Max Keys expected for the bloom */
65    protected int maxKeys;
66    /** Bloom bits */
67    protected ByteBuffer bloom;
68  
69    /** Bit-value lookup array to prevent doing the same work over and over */
70    private static final byte [] bitvals = {
71      (byte) 0x01,
72      (byte) 0x02,
73      (byte) 0x04,
74      (byte) 0x08,
75      (byte) 0x10,
76      (byte) 0x20,
77      (byte) 0x40,
78      (byte) 0x80
79      };
80  
81    /**
82     * Loads bloom filter meta data from file input.
83     * @param meta stored bloom meta data
84     * @throws IllegalArgumentException meta data is invalid
85     */
86    public ByteBloomFilter(ByteBuffer meta)
87    throws IllegalArgumentException {
88      int version = meta.getInt();
89      if (version != VERSION) throw new IllegalArgumentException("Bad version");
90  
91      this.byteSize = meta.getInt();
92      this.hashCount = meta.getInt();
93      this.hashType = meta.getInt();
94      this.keyCount = meta.getInt();
95      this.maxKeys = this.keyCount;
96  
97      this.hash = Hash.getInstance(this.hashType);
98      sanityCheck();
99    }
100 
101   /**
102    * Determines & initializes bloom filter meta data from user config.  Call
103    * {@link #allocBloom()} to allocate bloom filter data.
104    * @param maxKeys Maximum expected number of keys that will be stored in this bloom
105    * @param errorRate Desired false positive error rate.  Lower rate = more storage required
106    * @param hashType Type of hash function to use
107    * @param foldFactor When finished adding entries, you may be able to 'fold'
108    * this bloom to save space.  Tradeoff potentially excess bytes in bloom for
109    * ability to fold if keyCount is exponentially greater than maxKeys.
110    * @throws IllegalArgumentException
111    */
112   public ByteBloomFilter(int maxKeys, float errorRate, int hashType, int foldFactor)
113       throws IllegalArgumentException {
114     /*
115      * Bloom filters are very sensitive to the number of elements inserted
116      * into them. For HBase, the number of entries depends on the size of the
117      * data stored in the column. Currently the default region size is 256MB,
118      * so entry count ~= 256MB / (average value size for column).  Despite
119      * this rule of thumb, there is no efficient way to calculate the entry
120      * count after compactions.  Therefore, it is often easier to use a
121      * dynamic bloom filter that will add extra space instead of allowing the
122      * error rate to grow.
123      *
124      * ( http://www.eecs.harvard.edu/~michaelm/NEWWORK/postscripts/BloomFilterSurvey.pdf )
125      *
126      * m denotes the number of bits in the Bloom filter (bitSize)
127      * n denotes the number of elements inserted into the Bloom filter (maxKeys)
128      * k represents the number of hash functions used (nbHash)
129      * e represents the desired false positive rate for the bloom (err)
130      *
131      * If we fix the error rate (e) and know the number of entries, then
132      * the optimal bloom size m = -(n * ln(err) / (ln(2)^2)
133      *                         ~= n * ln(err) / ln(0.6185)
134      *
135      * The probability of false positives is minimized when k = m/n ln(2).
136      */
137     long bitSize = (long)Math.ceil(maxKeys * (Math.log(errorRate) / Math.log(0.6185)));
138     int functionCount = (int)Math.ceil(Math.log(2) * (bitSize / maxKeys));
139 
140     // increase byteSize so folding is possible
141     long byteSize = (bitSize + 7) / 8;
142     int mask = (1 << foldFactor) - 1;
143     if ( (mask & byteSize) != 0) {
144       byteSize >>= foldFactor;
145       ++byteSize;
146       byteSize <<= foldFactor;
147     }
148 
149     this.byteSize = byteSize;
150     this.hashCount = functionCount;
151     this.hashType = hashType;
152     this.keyCount = 0;
153     this.maxKeys = maxKeys;
154 
155     this.hash = Hash.getInstance(hashType);
156     sanityCheck();
157   }
158 
159   @Override
160   public void allocBloom() {
161     if (this.bloom != null) {
162       throw new IllegalArgumentException("can only create bloom once.");
163     }
164     this.bloom = ByteBuffer.allocate((int)this.byteSize);
165     assert this.bloom.hasArray();
166   }
167 
168   void sanityCheck() throws IllegalArgumentException {
169     if(0 >= this.byteSize || this.byteSize > Integer.MAX_VALUE) {
170       throw new IllegalArgumentException("Invalid byteSize: " + this.byteSize);
171     }
172 
173     if(this.hashCount <= 0) {
174       throw new IllegalArgumentException("Hash function count must be > 0");
175     }
176 
177     if (this.hash == null) {
178       throw new IllegalArgumentException("hashType must be known");
179     }
180 
181     if (this.keyCount < 0) {
182       throw new IllegalArgumentException("must have positive keyCount");
183     }
184   }
185 
186   void bloomCheck(ByteBuffer bloom)  throws IllegalArgumentException {
187     if (this.byteSize != bloom.limit()) {
188       throw new IllegalArgumentException(
189           "Configured bloom length should match actual length");
190     }
191   }
192 
193   @Override
194   public void add(byte [] buf) {
195     add(buf, 0, buf.length);
196   }
197 
198   @Override
199   public void add(byte [] buf, int offset, int len) {
200     /*
201      * For faster hashing, use combinatorial generation
202      * http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf
203      */
204     int hash1 = this.hash.hash(buf, offset, len, 0);
205     int hash2 = this.hash.hash(buf, offset, len, hash1);
206 
207     for (int i = 0; i < this.hashCount; i++) {
208       long hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8));
209       set(hashLoc);
210     }
211 
212     ++this.keyCount;
213   }
214 
215   /**
216    * Should only be used in tests when writing a bloom filter.
217    */
218   boolean contains(byte [] buf) {
219     return contains(buf, 0, buf.length, this.bloom);
220   }
221 
222   /**
223    * Should only be used in tests when writing a bloom filter.
224    */
225   boolean contains(byte [] buf, int offset, int length) {
226     return contains(buf, offset, length, this.bloom);
227   }
228 
229   @Override
230   public boolean contains(byte [] buf, ByteBuffer theBloom) {
231     return contains(buf, 0, buf.length, theBloom);
232   }
233 
234   @Override
235   public boolean contains(byte [] buf, int offset, int length,
236       ByteBuffer theBloom) {
237 
238     if(theBloom.limit() != this.byteSize) {
239       throw new IllegalArgumentException("Bloom does not match expected size");
240     }
241 
242     int hash1 = this.hash.hash(buf, offset, length, 0);
243     int hash2 = this.hash.hash(buf, offset, length, hash1);
244 
245     for (int i = 0; i < this.hashCount; i++) {
246       long hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8));
247       if (!get(hashLoc, theBloom) ) {
248         return false;
249       }
250     }
251     return true;
252   }
253 
254   //---------------------------------------------------------------------------
255   /** Private helpers */
256 
257   /**
258    * Set the bit at the specified index to 1.
259    *
260    * @param pos index of bit
261    */
262   void set(long pos) {
263     int bytePos = (int)(pos / 8);
264     int bitPos = (int)(pos % 8);
265     byte curByte = bloom.get(bytePos);
266     curByte |= bitvals[bitPos];
267     bloom.put(bytePos, curByte);
268   }
269 
270   /**
271    * Check if bit at specified index is 1.
272    *
273    * @param pos index of bit
274    * @return true if bit at specified index is 1, false if 0.
275    */
276   static boolean get(long pos, ByteBuffer theBloom) {
277     int bytePos = (int)(pos / 8);
278     int bitPos = (int)(pos % 8);
279     byte curByte = theBloom.get(bytePos);
280     curByte &= bitvals[bitPos];
281     return (curByte != 0);
282   }
283 
284   @Override
285   public int getKeyCount() {
286     return this.keyCount;
287   }
288 
289   @Override
290   public int getMaxKeys() {
291     return this.maxKeys;
292   }
293 
294   @Override
295   public int getByteSize() {
296     return (int)this.byteSize;
297   }
298 
299   @Override
300   public void compactBloom() {
301     // see if the actual size is exponentially smaller than expected.
302     if (this.keyCount > 0 && this.bloom.hasArray()) {
303       int pieces = 1;
304       int newByteSize = (int)this.byteSize;
305       int newMaxKeys = this.maxKeys;
306 
307       // while exponentially smaller & folding is lossless
308       while ( (newByteSize & 1) == 0 && newMaxKeys > (this.keyCount<<1) ) {
309         pieces <<= 1;
310         newByteSize >>= 1;
311         newMaxKeys >>= 1;
312       }
313 
314       // if we should fold these into pieces
315       if (pieces > 1) {
316         byte[] array = this.bloom.array();
317         int start = this.bloom.arrayOffset();
318         int end = start + newByteSize;
319         int off = end;
320         for(int p = 1; p < pieces; ++p) {
321           for(int pos = start; pos < end; ++pos) {
322             array[pos] |= array[off++];
323           }
324         }
325         // folding done, only use a subset of this array
326         this.bloom.rewind();
327         this.bloom.limit(newByteSize);
328         this.bloom = this.bloom.slice();
329         this.byteSize = newByteSize;
330         this.maxKeys = newMaxKeys;
331       }
332     }
333   }
334 
335 
336   //---------------------------------------------------------------------------
337 
338   /**
339    * Writes just the bloom filter to the output array
340    * @param out OutputStream to place bloom
341    * @throws IOException Error writing bloom array
342    */
343   public void writeBloom(final DataOutput out) throws IOException {
344     if (!this.bloom.hasArray()) {
345       throw new IOException("Only writes ByteBuffer with underlying array.");
346     }
347     out.write(bloom.array(), bloom.arrayOffset(), bloom.limit());
348   }
349 
350   @Override
351   public Writable getMetaWriter() {
352     return new MetaWriter();
353   }
354 
355   @Override
356   public Writable getDataWriter() {
357     return new DataWriter();
358   }
359 
360   private class MetaWriter implements Writable {
361     protected MetaWriter() {}
362     @Override
363     public void readFields(DataInput arg0) throws IOException {
364       throw new IOException("Cant read with this class.");
365     }
366 
367     @Override
368     public void write(DataOutput out) throws IOException {
369       out.writeInt(VERSION);
370       out.writeInt((int)byteSize);
371       out.writeInt(hashCount);
372       out.writeInt(hashType);
373       out.writeInt(keyCount);
374     }
375   }
376 
377   private class DataWriter implements Writable {
378     protected DataWriter() {}
379     @Override
380     public void readFields(DataInput arg0) throws IOException {
381       throw new IOException("Cant read with this class.");
382     }
383 
384     @Override
385     public void write(DataOutput out) throws IOException {
386       writeBloom(out);
387     }
388   }
389 
390 }