source: proiecte/HadoopJUnit/hadoop-0.20.1/src/core/org/apache/hadoop/util/bloom/BloomFilter.java @ 120

Last change on this file since 120 was 120, checked in by (none), 14 years ago

Added the mail files for the Hadoop JUNit Project

  • Property svn:executable set to *
File size: 7.4 KB
Line 
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
50package org.apache.hadoop.util.bloom;
51
52import java.io.DataInput;
53import java.io.DataOutput;
54import java.io.IOException;
55
56import java.util.BitSet;
57
58/**
59 * Implements a <i>Bloom filter</i>, as defined by Bloom in 1970.
60 * <p>
61 * The Bloom filter is a data structure that was introduced in 1970 and that has been adopted by
62 * the networking research community in the past decade thanks to the bandwidth efficiencies that it
63 * offers for the transmission of set membership information between networked hosts.  A sender encodes
64 * the information into a bit vector, the Bloom filter, that is more compact than a conventional
65 * representation. Computation and space costs for construction are linear in the number of elements. 
66 * The receiver uses the filter to test whether various elements are members of the set. Though the
67 * filter will occasionally return a false positive, it will never return a false negative. When creating
68 * the filter, the sender can choose its desired point in a trade-off between the false positive rate and the size.
69 *
70 * <p>
71 * Originally created by
72 * <a href="http://www.one-lab.org">European Commission One-Lab Project 034819</a>.
73 *
74 * @see Filter The general behavior of a filter
75 *
76 * @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>
77 */
78public class BloomFilter extends Filter {
79  private static final byte[] bitvalues = new byte[] {
80    (byte)0x01,
81    (byte)0x02,
82    (byte)0x04,
83    (byte)0x08,
84    (byte)0x10,
85    (byte)0x20,
86    (byte)0x40,
87    (byte)0x80
88  };
89 
90  /** The bit vector. */
91  BitSet bits;
92
93  /** Default constructor - use with readFields */
94  public BloomFilter() {
95    super();
96  }
97 
98  /**
99   * Constructor
100   * @param vectorSize The vector size of <i>this</i> filter.
101   * @param nbHash The number of hash function to consider.
102   * @param hashType type of the hashing function (see
103   * {@link org.apache.hadoop.util.hash.Hash}).
104   */
105  public BloomFilter(int vectorSize, int nbHash, int hashType) {
106    super(vectorSize, nbHash, hashType);
107
108    bits = new BitSet(this.vectorSize);
109  }
110
111  @Override
112  public void add(Key key) {
113    if(key == null) {
114      throw new NullPointerException("key cannot be null");
115    }
116
117    int[] h = hash.hash(key);
118    hash.clear();
119
120    for(int i = 0; i < nbHash; i++) {
121      bits.set(h[i]);
122    }
123  }
124
125  @Override
126  public void and(Filter filter) {
127    if(filter == null
128        || !(filter instanceof BloomFilter)
129        || filter.vectorSize != this.vectorSize
130        || filter.nbHash != this.nbHash) {
131      throw new IllegalArgumentException("filters cannot be and-ed");
132    }
133
134    this.bits.and(((BloomFilter) filter).bits);
135  }
136
137  @Override
138  public boolean membershipTest(Key key) {
139    if(key == null) {
140      throw new NullPointerException("key cannot be null");
141    }
142
143    int[] h = hash.hash(key);
144    hash.clear();
145    for(int i = 0; i < nbHash; i++) {
146      if(!bits.get(h[i])) {
147        return false;
148      }
149    }
150    return true;
151  }
152
153  @Override
154  public void not() {
155    bits.flip(0, vectorSize - 1);
156  }
157
158  @Override
159  public void or(Filter filter) {
160    if(filter == null
161        || !(filter instanceof BloomFilter)
162        || filter.vectorSize != this.vectorSize
163        || filter.nbHash != this.nbHash) {
164      throw new IllegalArgumentException("filters cannot be or-ed");
165    }
166    bits.or(((BloomFilter) filter).bits);
167  }
168
169  @Override
170  public void xor(Filter filter) {
171    if(filter == null
172        || !(filter instanceof BloomFilter)
173        || filter.vectorSize != this.vectorSize
174        || filter.nbHash != this.nbHash) {
175      throw new IllegalArgumentException("filters cannot be xor-ed");
176    }
177    bits.xor(((BloomFilter) filter).bits);
178  }
179
180  @Override
181  public String toString() {
182    return bits.toString();
183  }
184
185  /**
186   * @return size of the the bloomfilter
187   */
188  public int getVectorSize() {
189    return this.vectorSize;
190  }
191
192  // Writable
193
194  @Override
195  public void write(DataOutput out) throws IOException {
196    super.write(out);
197    byte[] bytes = new byte[getNBytes()];
198    for(int i = 0, byteIndex = 0, bitIndex = 0; i < vectorSize; i++, bitIndex++) {
199      if (bitIndex == 8) {
200        bitIndex = 0;
201        byteIndex++;
202      }
203      if (bitIndex == 0) {
204        bytes[byteIndex] = 0;
205      }
206      if (bits.get(i)) {
207        bytes[byteIndex] |= bitvalues[bitIndex];
208      }
209    }
210    out.write(bytes);
211  }
212
213  @Override
214  public void readFields(DataInput in) throws IOException {
215    super.readFields(in);
216    bits = new BitSet(this.vectorSize);
217    byte[] bytes = new byte[getNBytes()];
218    in.readFully(bytes);
219    for(int i = 0, byteIndex = 0, bitIndex = 0; i < vectorSize; i++, bitIndex++) {
220      if (bitIndex == 8) {
221        bitIndex = 0;
222        byteIndex++;
223      }
224      if ((bytes[byteIndex] & bitvalues[bitIndex]) != 0) {
225        bits.set(i);
226      }
227    }
228  }
229 
230  /* @return number of bytes needed to hold bit vector */
231  private int getNBytes() {
232    return (vectorSize + 7) / 8;
233  }
234}//end class
Note: See TracBrowser for help on using the repository browser.