1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
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
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50 public class ByteBloomFilter implements BloomFilter {
51
52 public static final int VERSION = 1;
53
54
55 protected long byteSize;
56
57 protected final int hashCount;
58
59 protected final int hashType;
60
61 protected final Hash hash;
62
63 protected int keyCount;
64
65 protected int maxKeys;
66
67 protected ByteBuffer bloom;
68
69
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
83
84
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
103
104
105
106
107
108
109
110
111
112 public ByteBloomFilter(int maxKeys, float errorRate, int hashType, int foldFactor)
113 throws IllegalArgumentException {
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
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
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
202
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
217
218 boolean contains(byte [] buf) {
219 return contains(buf, 0, buf.length, this.bloom);
220 }
221
222
223
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
256
257
258
259
260
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
272
273
274
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
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
308 while ( (newByteSize & 1) == 0 && newMaxKeys > (this.keyCount<<1) ) {
309 pieces <<= 1;
310 newByteSize >>= 1;
311 newMaxKeys >>= 1;
312 }
313
314
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
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
340
341
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 }