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.RawComparator;
24 import org.apache.hadoop.io.Writable;
25
26 import java.io.DataInput;
27 import java.io.DataOutput;
28 import java.io.IOException;
29 import java.nio.ByteBuffer;
30 import java.text.NumberFormat;
31 import java.util.Random;
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79 public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
80
81
82 public static final int VERSION = 1;
83
84
85 protected long byteSize;
86
87 protected int hashCount;
88
89 protected final int hashType;
90
91 protected final Hash hash;
92
93 protected int keyCount;
94
95 protected int maxKeys;
96
97 protected ByteBuffer bloom;
98
99
100 public static final String STATS_RECORD_SEP = "; ";
101
102
103
104
105
106 public static final double LOG2_SQUARED = Math.log(2) * Math.log(2);
107
108
109
110
111
112 private static Random randomGeneratorForTest;
113
114
115 private static final byte [] bitvals = {
116 (byte) 0x01,
117 (byte) 0x02,
118 (byte) 0x04,
119 (byte) 0x08,
120 (byte) 0x10,
121 (byte) 0x20,
122 (byte) 0x40,
123 (byte) 0x80
124 };
125
126
127
128
129
130
131 public ByteBloomFilter(DataInput meta)
132 throws IOException, IllegalArgumentException {
133 this.byteSize = meta.readInt();
134 this.hashCount = meta.readInt();
135 this.hashType = meta.readInt();
136 this.keyCount = meta.readInt();
137 this.maxKeys = this.keyCount;
138
139 this.hash = Hash.getInstance(this.hashType);
140 if (hash == null) {
141 throw new IllegalArgumentException("Invalid hash type: " + hashType);
142 }
143 sanityCheck();
144 }
145
146
147
148
149
150
151
152
153
154 public static long computeBitSize(long maxKeys, double errorRate) {
155 return (long) Math.ceil(maxKeys * (-Math.log(errorRate) / LOG2_SQUARED));
156 }
157
158
159
160
161
162
163
164
165
166
167
168
169 public static long idealMaxKeys(long bitSize, double errorRate) {
170
171
172 return (long) (bitSize * (LOG2_SQUARED / -Math.log(errorRate)));
173 }
174
175
176
177
178
179
180
181
182
183
184
185
186 public static long computeMaxKeys(long bitSize, double errorRate,
187 int hashCount) {
188 return (long) (-bitSize * 1.0 / hashCount *
189 Math.log(1 - Math.exp(Math.log(errorRate) / hashCount)));
190 }
191
192
193
194
195
196
197
198
199
200
201 public double actualErrorRate() {
202 return actualErrorRate(keyCount, byteSize * 8, hashCount);
203 }
204
205
206
207
208
209
210
211
212
213
214
215
216
217 public static double actualErrorRate(long maxKeys, long bitSize,
218 int functionCount) {
219 return Math.exp(Math.log(1 - Math.exp(-functionCount * maxKeys * 1.0
220 / bitSize)) * functionCount);
221 }
222
223
224
225
226
227
228
229
230
231 public static int computeFoldableByteSize(long bitSize, int foldFactor) {
232 long byteSizeLong = (bitSize + 7) / 8;
233 int mask = (1 << foldFactor) - 1;
234 if ((mask & byteSizeLong) != 0) {
235 byteSizeLong >>= foldFactor;
236 ++byteSizeLong;
237 byteSizeLong <<= foldFactor;
238 }
239 if (byteSizeLong > Integer.MAX_VALUE) {
240 throw new IllegalArgumentException("byteSize=" + byteSizeLong + " too "
241 + "large for bitSize=" + bitSize + ", foldFactor=" + foldFactor);
242 }
243 return (int) byteSizeLong;
244 }
245
246 private static int optimalFunctionCount(int maxKeys, long bitSize) {
247 return (int) Math.ceil(Math.log(2) * (bitSize / maxKeys));
248 }
249
250
251 private ByteBloomFilter(int hashType) {
252 this.hashType = hashType;
253 this.hash = Hash.getInstance(hashType);
254 }
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271 public ByteBloomFilter(int maxKeys, double errorRate, int hashType,
272 int foldFactor) throws IllegalArgumentException {
273 this(hashType);
274
275 long bitSize = computeBitSize(maxKeys, errorRate);
276 hashCount = optimalFunctionCount(maxKeys, bitSize);
277 this.maxKeys = maxKeys;
278
279
280 byteSize = computeFoldableByteSize(bitSize, foldFactor);
281
282 sanityCheck();
283 }
284
285
286
287
288
289
290
291
292
293
294
295 public static ByteBloomFilter createBySize(int byteSizeHint,
296 double errorRate, int hashType, int foldFactor) {
297 ByteBloomFilter bbf = new ByteBloomFilter(hashType);
298
299 bbf.byteSize = computeFoldableByteSize(byteSizeHint * 8, foldFactor);
300 long bitSize = bbf.byteSize * 8;
301 bbf.maxKeys = (int) idealMaxKeys(bitSize, errorRate);
302 bbf.hashCount = optimalFunctionCount(bbf.maxKeys, bitSize);
303
304
305
306
307 bbf.maxKeys = (int) computeMaxKeys(bitSize, errorRate, bbf.hashCount);
308
309 return bbf;
310 }
311
312
313
314
315
316
317
318 public ByteBloomFilter createAnother() {
319 ByteBloomFilter bbf = new ByteBloomFilter(hashType);
320 bbf.byteSize = byteSize;
321 bbf.hashCount = hashCount;
322 bbf.maxKeys = maxKeys;
323 return bbf;
324 }
325
326 @Override
327 public void allocBloom() {
328 if (this.bloom != null) {
329 throw new IllegalArgumentException("can only create bloom once.");
330 }
331 this.bloom = ByteBuffer.allocate((int)this.byteSize);
332 assert this.bloom.hasArray();
333 }
334
335 void sanityCheck() throws IllegalArgumentException {
336 if(0 >= this.byteSize || this.byteSize > Integer.MAX_VALUE) {
337 throw new IllegalArgumentException("Invalid byteSize: " + this.byteSize);
338 }
339
340 if(this.hashCount <= 0) {
341 throw new IllegalArgumentException("Hash function count must be > 0");
342 }
343
344 if (this.hash == null) {
345 throw new IllegalArgumentException("hashType must be known");
346 }
347
348 if (this.keyCount < 0) {
349 throw new IllegalArgumentException("must have positive keyCount");
350 }
351 }
352
353 void bloomCheck(ByteBuffer bloom) throws IllegalArgumentException {
354 if (this.byteSize != bloom.limit()) {
355 throw new IllegalArgumentException(
356 "Configured bloom length should match actual length");
357 }
358 }
359
360 public void add(byte [] buf) {
361 add(buf, 0, buf.length);
362 }
363
364 @Override
365 public void add(byte [] buf, int offset, int len) {
366
367
368
369
370 int hash1 = this.hash.hash(buf, offset, len, 0);
371 int hash2 = this.hash.hash(buf, offset, len, hash1);
372
373 for (int i = 0; i < this.hashCount; i++) {
374 long hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8));
375 set(hashLoc);
376 }
377
378 ++this.keyCount;
379 }
380
381
382 boolean contains(byte [] buf) {
383 return contains(buf, 0, buf.length, this.bloom);
384 }
385
386
387 boolean contains(byte [] buf, int offset, int length) {
388 return contains(buf, offset, length, bloom);
389 }
390
391
392 boolean contains(byte[] buf, ByteBuffer bloom) {
393 return contains(buf, 0, buf.length, bloom);
394 }
395
396 @Override
397 public boolean contains(byte[] buf, int offset, int length,
398 ByteBuffer theBloom) {
399 if (theBloom == null) {
400
401
402
403 theBloom = bloom;
404 }
405
406 if (theBloom.limit() != byteSize) {
407 throw new IllegalArgumentException("Bloom does not match expected size:"
408 + " theBloom.limit()=" + theBloom.limit() + ", byteSize=" + byteSize);
409 }
410
411 return contains(buf, offset, length, theBloom.array(),
412 theBloom.arrayOffset(), (int) byteSize, hash, hashCount);
413 }
414
415 public static boolean contains(byte[] buf, int offset, int length,
416 byte[] bloomArray, int bloomOffset, int bloomSize, Hash hash,
417 int hashCount) {
418
419 int hash1 = hash.hash(buf, offset, length, 0);
420 int hash2 = hash.hash(buf, offset, length, hash1);
421 int bloomBitSize = bloomSize << 3;
422
423 if (randomGeneratorForTest == null) {
424
425 int compositeHash = hash1;
426 for (int i = 0; i < hashCount; i++) {
427 int hashLoc = Math.abs(compositeHash % bloomBitSize);
428 compositeHash += hash2;
429 if (!get(hashLoc, bloomArray, bloomOffset)) {
430 return false;
431 }
432 }
433 } else {
434
435 for (int i = 0; i < hashCount; i++) {
436 int hashLoc = randomGeneratorForTest.nextInt(bloomBitSize);
437 if (!get(hashLoc, bloomArray, bloomOffset)){
438 return false;
439 }
440 }
441 }
442 return true;
443 }
444
445
446
447
448
449
450
451
452
453 void set(long pos) {
454 int bytePos = (int)(pos / 8);
455 int bitPos = (int)(pos % 8);
456 byte curByte = bloom.get(bytePos);
457 curByte |= bitvals[bitPos];
458 bloom.put(bytePos, curByte);
459 }
460
461
462
463
464
465
466
467 static boolean get(int pos, byte[] bloomArray, int bloomOffset) {
468 int bytePos = pos >> 3;
469 int bitPos = pos & 0x7;
470 byte curByte = bloomArray[bloomOffset + bytePos];
471 curByte &= bitvals[bitPos];
472 return (curByte != 0);
473 }
474
475 @Override
476 public long getKeyCount() {
477 return keyCount;
478 }
479
480 @Override
481 public long getMaxKeys() {
482 return maxKeys;
483 }
484
485 @Override
486 public long getByteSize() {
487 return byteSize;
488 }
489
490 public int getHashType() {
491 return hashType;
492 }
493
494 @Override
495 public void compactBloom() {
496
497 if (this.keyCount > 0 && this.bloom.hasArray()) {
498 int pieces = 1;
499 int newByteSize = (int)this.byteSize;
500 int newMaxKeys = this.maxKeys;
501
502
503 while ( (newByteSize & 1) == 0 && newMaxKeys > (this.keyCount<<1) ) {
504 pieces <<= 1;
505 newByteSize >>= 1;
506 newMaxKeys >>= 1;
507 }
508
509
510 if (pieces > 1) {
511 byte[] array = this.bloom.array();
512 int start = this.bloom.arrayOffset();
513 int end = start + newByteSize;
514 int off = end;
515 for(int p = 1; p < pieces; ++p) {
516 for(int pos = start; pos < end; ++pos) {
517 array[pos] |= array[off++];
518 }
519 }
520
521 this.bloom.rewind();
522 this.bloom.limit(newByteSize);
523 this.bloom = this.bloom.slice();
524 this.byteSize = newByteSize;
525 this.maxKeys = newMaxKeys;
526 }
527 }
528 }
529
530
531
532
533
534
535
536
537
538 public void writeBloom(final DataOutput out) throws IOException {
539 if (!this.bloom.hasArray()) {
540 throw new IOException("Only writes ByteBuffer with underlying array.");
541 }
542 out.write(bloom.array(), bloom.arrayOffset(), bloom.limit());
543 }
544
545 @Override
546 public Writable getMetaWriter() {
547 return new MetaWriter();
548 }
549
550 @Override
551 public Writable getDataWriter() {
552 return new DataWriter();
553 }
554
555 private class MetaWriter implements Writable {
556 protected MetaWriter() {}
557 @Override
558 public void readFields(DataInput arg0) throws IOException {
559 throw new IOException("Cant read with this class.");
560 }
561
562 @Override
563 public void write(DataOutput out) throws IOException {
564 out.writeInt(VERSION);
565 out.writeInt((int) byteSize);
566 out.writeInt(hashCount);
567 out.writeInt(hashType);
568 out.writeInt(keyCount);
569 }
570 }
571
572 private class DataWriter implements Writable {
573 protected DataWriter() {}
574 @Override
575 public void readFields(DataInput arg0) throws IOException {
576 throw new IOException("Cant read with this class.");
577 }
578
579 @Override
580 public void write(DataOutput out) throws IOException {
581 writeBloom(out);
582 }
583 }
584
585 public int getHashCount() {
586 return hashCount;
587 }
588
589 @Override
590 public boolean supportsAutoLoading() {
591 return bloom != null;
592 }
593
594 public static void setFakeLookupMode(boolean enabled) {
595 if (enabled) {
596 randomGeneratorForTest = new Random(283742987L);
597 } else {
598 randomGeneratorForTest = null;
599 }
600 }
601
602
603
604
605
606
607 @Override
608 public byte[] createBloomKey(byte[] rowBuf, int rowOffset, int rowLen,
609 byte[] qualBuf, int qualOffset, int qualLen) {
610
611 if (qualLen <= 0 && rowOffset == 0 && rowLen == rowBuf.length)
612 return rowBuf;
613
614 byte [] result = new byte[rowLen + qualLen];
615 System.arraycopy(rowBuf, rowOffset, result, 0, rowLen);
616 if (qualLen > 0)
617 System.arraycopy(qualBuf, qualOffset, result, rowLen, qualLen);
618 return result;
619 }
620
621 @Override
622 public RawComparator<byte[]> getComparator() {
623 return Bytes.BYTES_RAWCOMPARATOR;
624 }
625
626
627
628
629
630
631
632
633 public static String formatStats(BloomFilterBase bloomFilter) {
634 StringBuilder sb = new StringBuilder();
635 long k = bloomFilter.getKeyCount();
636 long m = bloomFilter.getMaxKeys();
637
638 sb.append("BloomSize: " + bloomFilter.getByteSize() + STATS_RECORD_SEP);
639 sb.append("No of Keys in bloom: " + k + STATS_RECORD_SEP);
640 sb.append("Max Keys for bloom: " + m);
641 if (m > 0) {
642 sb.append(STATS_RECORD_SEP + "Percentage filled: "
643 + NumberFormat.getPercentInstance().format(k * 1.0 / m));
644 }
645 return sb.toString();
646 }
647
648 @Override
649 public String toString() {
650 return formatStats(this) + STATS_RECORD_SEP + "Actual error rate: "
651 + String.format("%.8f", actualErrorRate());
652 }
653
654 }