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 java.io.ByteArrayOutputStream;
24 import java.io.DataOutputStream;
25 import java.nio.ByteBuffer;
26 import java.util.BitSet;
27
28 import junit.framework.TestCase;
29
30 public class TestByteBloomFilter extends TestCase {
31
32 public void testBasicBloom() throws Exception {
33 ByteBloomFilter bf1 = new ByteBloomFilter(1000, (float)0.01, Hash.MURMUR_HASH, 0);
34 ByteBloomFilter bf2 = new ByteBloomFilter(1000, (float)0.01, Hash.MURMUR_HASH, 0);
35 bf1.allocBloom();
36 bf2.allocBloom();
37
38
39 byte[] key1 = {1,2,3,4,5,6,7,8,9};
40 byte[] key2 = {1,2,3,4,5,6,7,8,7};
41
42 bf1.add(key1);
43 bf2.add(key2);
44
45 assertTrue(bf1.contains(key1));
46 assertFalse(bf1.contains(key2));
47 assertFalse(bf2.contains(key1));
48 assertTrue(bf2.contains(key2));
49
50 byte [] bkey = {1,2,3,4};
51 byte [] bval = "this is a much larger byte array".getBytes();
52
53 bf1.add(bkey);
54 bf1.add(bval, 1, bval.length-1);
55
56 assertTrue( bf1.contains(bkey) );
57 assertTrue( bf1.contains(bval, 1, bval.length-1) );
58 assertFalse( bf1.contains(bval) );
59 assertFalse( bf1.contains(bval) );
60
61
62
63 ByteArrayOutputStream bOut = new ByteArrayOutputStream();
64 bf1.writeBloom(new DataOutputStream(bOut));
65 ByteBuffer bb = ByteBuffer.wrap(bOut.toByteArray());
66 ByteBloomFilter newBf1 = new ByteBloomFilter(1000, (float)0.01,
67 Hash.MURMUR_HASH, 0);
68 assertTrue(newBf1.contains(key1, bb));
69 assertFalse(newBf1.contains(key2, bb));
70 assertTrue( newBf1.contains(bkey, bb) );
71 assertTrue( newBf1.contains(bval, 1, bval.length-1, bb) );
72 assertFalse( newBf1.contains(bval, bb) );
73 assertFalse( newBf1.contains(bval, bb) );
74
75 System.out.println("Serialized as " + bOut.size() + " bytes");
76 assertTrue(bOut.size() - bf1.byteSize < 10);
77 }
78
79 public void testBloomFold() throws Exception {
80
81 ByteBloomFilter b = new ByteBloomFilter(1003, (float)0.01, Hash.MURMUR_HASH, 2);
82 b.allocBloom();
83 int origSize = b.getByteSize();
84 assertEquals(1204, origSize);
85 for (int i = 0; i < 12; ++i) {
86 b.add(Bytes.toBytes(i));
87 }
88 b.compactBloom();
89 assertEquals(origSize>>2, b.getByteSize());
90 int falsePositives = 0;
91 for (int i = 0; i < 25; ++i) {
92 if (b.contains(Bytes.toBytes(i))) {
93 if(i >= 12) falsePositives++;
94 } else {
95 assertFalse(i < 12);
96 }
97 }
98 assertTrue(falsePositives <= 1);
99
100
101 }
102
103 public void testBloomPerf() throws Exception {
104
105 float err = (float)0.01;
106 ByteBloomFilter b = new ByteBloomFilter(10*1000*1000, (float)err, Hash.MURMUR_HASH, 3);
107 b.allocBloom();
108 long startTime = System.currentTimeMillis();
109 int origSize = b.getByteSize();
110 for (int i = 0; i < 1*1000*1000; ++i) {
111 b.add(Bytes.toBytes(i));
112 }
113 long endTime = System.currentTimeMillis();
114 System.out.println("Total Add time = " + (endTime - startTime) + "ms");
115
116
117 startTime = System.currentTimeMillis();
118 b.compactBloom();
119 endTime = System.currentTimeMillis();
120 System.out.println("Total Fold time = " + (endTime - startTime) + "ms");
121 assertTrue(origSize >= b.getByteSize()<<3);
122
123
124 startTime = System.currentTimeMillis();
125 int falsePositives = 0;
126 for (int i = 0; i < 2*1000*1000; ++i) {
127
128 if (b.contains(Bytes.toBytes(i))) {
129 if(i >= 1*1000*1000) falsePositives++;
130 } else {
131 assertFalse(i < 1*1000*1000);
132 }
133 }
134 endTime = System.currentTimeMillis();
135 System.out.println("Total Contains time = " + (endTime - startTime) + "ms");
136 System.out.println("False Positive = " + falsePositives);
137 assertTrue(falsePositives <= (1*1000*1000)*err);
138
139
140 }
141 }