View Javadoc

1   /*
2    *
3    * Copyright (c) 2005, European Commission project OneLab under contract 034819 (http://www.one-lab.org)
4    * All rights reserved.
5    * Redistribution and use in source and binary forms, with or 
6    * without modification, are permitted provided that the following 
7    * conditions are met:
8    *  - Redistributions of source code must retain the above copyright 
9    *    notice, this list of conditions and the following disclaimer.
10   *  - Redistributions in binary form must reproduce the above copyright 
11   *    notice, this list of conditions and the following disclaimer in 
12   *    the documentation and/or other materials provided with the distribution.
13   *  - Neither the name of the University Catholique de Louvain - UCL
14   *    nor the names of its contributors may be used to endorse or 
15   *    promote products derived from this software without specific prior 
16   *    written permission.
17   *    
18   * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 
19   * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 
20   * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS 
21   * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE 
22   * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, 
23   * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, 
24   * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; 
25   * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 
26   * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT 
27   * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN 
28   * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 
29   * POSSIBILITY OF SUCH DAMAGE.
30   */
31  
32  /*
33   * Licensed to the Apache Software Foundation (ASF) under one
34   * or more contributor license agreements.  See the NOTICE file
35   * distributed with this work for additional information
36   * regarding copyright ownership.  The ASF licenses this file
37   * to you under the Apache License, Version 2.0 (the
38   * "License"); you may not use this file except in compliance
39   * with the License.  You may obtain a copy of the License at
40   *
41   *     http://www.apache.org/licenses/LICENSE-2.0
42   *
43   * Unless required by applicable law or agreed to in writing, software
44   * distributed under the License is distributed on an "AS IS" BASIS,
45   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
46   * See the License for the specific language governing permissions and
47   * limitations under the License.
48   */
49  
50  package org.apache.accumulo.core.bloomfilter;
51  
52  import java.io.ByteArrayOutputStream;
53  import java.io.DataInput;
54  import java.io.DataInputStream;
55  import java.io.DataOutput;
56  import java.io.IOException;
57  import java.io.ObjectInputStream;
58  import java.io.ObjectOutputStream;
59  import java.util.BitSet;
60  
61  import org.apache.hadoop.util.bloom.Key;
62  
63  /**
64   * Implements a <i>Bloom filter</i>, as defined by Bloom in 1970.
65   * <p>
66   * The Bloom filter is a data structure that was introduced in 1970 and that has been adopted by the networking research community in the past decade thanks to
67   * the bandwidth efficiencies that it offers for the transmission of set membership information between networked hosts. A sender encodes the information into a
68   * bit vector, the Bloom filter, that is more compact than a conventional representation. Computation and space costs for construction are linear in the number
69   * of elements. The receiver uses the filter to test whether various elements are members of the set. Though the filter will occasionally return a false
70   * positive, it will never return a false negative. When creating the filter, the sender can choose its desired point in a trade-off between the false positive
71   * rate and the size.
72   * 
73   * <p>
74   * Originally created by <a href="http://www.one-lab.org">European Commission One-Lab Project 034819</a>.
75   * 
76   * @see Filter The general behavior of a filter
77   * 
78   * @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>
79   */
80  public class BloomFilter extends Filter {
81    
82    private static final byte[] bitvalues = new byte[] {(byte) 0x01, (byte) 0x02, (byte) 0x04, (byte) 0x08, (byte) 0x10, (byte) 0x20, (byte) 0x40, (byte) 0x80};
83    
84    /** The bit vector. */
85    BitSet bits;
86    
87    /** Default constructor - use with readFields */
88    public BloomFilter() {
89      super();
90    }
91    
92    /**
93     * Constructor
94     * 
95     * @param vectorSize
96     *          The vector size of <i>this</i> filter.
97     * @param nbHash
98     *          The number of hash function to consider.
99     * @param hashType
100    *          type of the hashing function (see {@link org.apache.hadoop.util.hash.Hash}).
101    */
102   public BloomFilter(final int vectorSize, final int nbHash, final int hashType) {
103     super(vectorSize, nbHash, hashType);
104     
105     bits = new BitSet(this.vectorSize);
106   }
107   
108   @Override
109   public boolean add(final Key key) {
110     if (key == null) {
111       throw new NullPointerException("key cannot be null");
112     }
113     
114     int[] h = hash.hash(key);
115     hash.clear();
116     
117     boolean bitsSet = false;
118 
119     for (int i = 0; i < nbHash; i++) {
120       bitsSet |= !bits.get(h[i]);
121       bits.set(h[i]);
122     }
123     
124     return bitsSet;
125   }
126   
127   @Override
128   public void and(final Filter filter) {
129     if (filter == null || !(filter instanceof BloomFilter) || filter.vectorSize != this.vectorSize || filter.nbHash != this.nbHash) {
130       throw new IllegalArgumentException("filters cannot be and-ed");
131     }
132     
133     this.bits.and(((BloomFilter) filter).bits);
134   }
135   
136   @Override
137   public boolean membershipTest(final Key key) {
138     if (key == null) {
139       throw new NullPointerException("key cannot be null");
140     }
141     
142     int[] h = hash.hash(key);
143     hash.clear();
144     for (int i = 0; i < nbHash; i++) {
145       if (!bits.get(h[i])) {
146         return false;
147       }
148     }
149     return true;
150   }
151   
152   @Override
153   public void not() {
154     bits.flip(0, vectorSize - 1);
155   }
156   
157   @Override
158   public void or(final Filter filter) {
159     if (filter == null || !(filter instanceof BloomFilter) || filter.vectorSize != this.vectorSize || filter.nbHash != this.nbHash) {
160       throw new IllegalArgumentException("filters cannot be or-ed");
161     }
162     bits.or(((BloomFilter) filter).bits);
163   }
164   
165   @Override
166   public void xor(final Filter filter) {
167     if (filter == null || !(filter instanceof BloomFilter) || filter.vectorSize != this.vectorSize || filter.nbHash != this.nbHash) {
168       throw new IllegalArgumentException("filters cannot be xor-ed");
169     }
170     bits.xor(((BloomFilter) filter).bits);
171   }
172   
173   @Override
174   public String toString() {
175     return bits.toString();
176   }
177   
178   /**
179    * @return size of the the bloomfilter
180    */
181   public int getVectorSize() {
182     return this.vectorSize;
183   }
184   
185   // Writable
186   
187   @Override
188   public void write(final DataOutput out) throws IOException {
189     super.write(out);
190     
191     ByteArrayOutputStream boas = new ByteArrayOutputStream();
192     ObjectOutputStream oos = new ObjectOutputStream(boas);
193     
194     oos.writeObject(bits);
195     oos.flush();
196     oos.close();
197     out.write(boas.toByteArray());
198   }
199   
200   @Override
201   public void readFields(final DataInput in) throws IOException {
202     
203     super.readFields(in);
204     
205     bits = new BitSet(this.vectorSize);
206     byte[] bytes = null;
207     
208     if (super.getSerialVersion() != super.getVersion()) {
209       bytes = new byte[getNBytes()];
210       in.readFully(bytes);
211     }
212     
213     if (super.getSerialVersion() == super.getVersion()) {
214       // ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
215       // ObjectInputStream ois = new ObjectInputStream(bais);
216       ObjectInputStream ois = new ObjectInputStream((DataInputStream) (in));
217       try {
218         bits = (BitSet) ois.readObject();
219       } catch (ClassNotFoundException e) {
220         e.printStackTrace();
221         throw new IOException("BloomFilter tried to deserialize as bitset: " + e);
222       }
223       // can not close ois, it would close in
224       
225     } else {
226       for (int i = 0, byteIndex = 0, bitIndex = 0; i < vectorSize; i++, bitIndex++) {
227         if (bitIndex == 8) {
228           bitIndex = 0;
229           byteIndex++;
230         }
231         if ((bytes[byteIndex] & bitvalues[bitIndex]) != 0) {
232           bits.set(i);
233         }
234       }
235     }
236     
237   }
238   
239   /* @return number of bytes needed to hold bit vector */
240   private int getNBytes() {
241     return (vectorSize + 7) / 8;
242   }
243   
244 }// end class