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