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