1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
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
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
215
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
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
240 private int getNBytes() {
241 return (vectorSize + 7) / 8;
242 }
243
244 }