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