-
Notifications
You must be signed in to change notification settings - Fork 507
Expand file tree
/
Copy pathBloomFilter.java
More file actions
339 lines (293 loc) · 10.2 KB
/
BloomFilter.java
File metadata and controls
339 lines (293 loc) · 10.2 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
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
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.orc.util;
import java.nio.charset.Charset;
import java.util.Arrays;
/**
* BloomFilter is a probabilistic data structure for set membership check. BloomFilters are
* highly space efficient when compared to using a HashSet. Because of the probabilistic nature of
* bloom filter false positive (element not present in bloom filter but test() says true) are
* possible but false negatives are not possible (if element is present then test() will never
* say false). The false positive probability is configurable (default: 5%) depending on which
* storage requirement may increase or decrease. Lower the false positive probability greater
* is the space requirement.
* Bloom filters are sensitive to number of elements that will be inserted in the bloom filter.
* During the creation of bloom filter expected number of entries must be specified. If the number
* of insertions exceed the specified initial number of entries then false positive probability will
* increase accordingly.
* <p>
* Internally, this implementation of bloom filter uses Murmur3 fast non-cryptographic hash
* algorithm. Although Murmur2 is slightly faster than Murmur3 in Java, it suffers from hash
* collisions for specific sequence of repeating bytes. Check the following link for more info
* https://code.google.com/p/smhasher/wiki/MurmurHash2Flaw
* <p>
* Note that this class is here for backwards compatibility, because it uses
* the JVM default character set for strings. All new users should
* BloomFilterUtf8, which always uses UTF8 for the encoding.
* @since 1.3.0
*/
public class BloomFilter {
public static final double DEFAULT_FPP = 0.05;
private final BitSet bitSet;
private final int numBits;
private final int numHashFunctions;
static void checkArgument(boolean expression, String message) {
if (!expression) {
throw new IllegalArgumentException(message);
}
}
public BloomFilter(long expectedEntries) {
this(expectedEntries, DEFAULT_FPP);
}
public BloomFilter(long expectedEntries, double fpp) {
expectedEntries = Math.max(expectedEntries, 1);
checkArgument(fpp > 0.0 && fpp < 1.0, "False positive probability should be > 0.0 & < 1.0");
int nb = optimalNumOfBits(expectedEntries, fpp);
// make 'm' multiple of 64
this.numBits = nb + (Long.SIZE - (nb % Long.SIZE));
this.numHashFunctions = optimalNumOfHashFunctions(expectedEntries, numBits);
this.bitSet = new BitSet(numBits);
}
/**
* A constructor to support rebuilding the BloomFilter from a serialized representation.
* @param bits the serialized bits
* @param numFuncs the number of functions used
*/
public BloomFilter(long[] bits, int numFuncs) {
super();
bitSet = new BitSet(bits);
this.numBits = (int) bitSet.bitSize();
numHashFunctions = numFuncs;
}
static int optimalNumOfHashFunctions(long n, long m) {
return Math.max(1, (int) Math.round((double) m / n * Math.log(2)));
}
static int optimalNumOfBits(long n, double p) {
return (int) (-n * Math.log(p) / (Math.log(2) * Math.log(2)));
}
@Override
public boolean equals(Object other) {
return other != null &&
other.getClass() == getClass() &&
numBits == ((BloomFilter) other).numBits &&
numHashFunctions == ((BloomFilter) other).numHashFunctions &&
bitSet.equals(((BloomFilter) other).bitSet);
}
@Override
public int hashCode() {
return bitSet.hashCode() + numHashFunctions * 5;
}
public void add(byte[] val) {
addBytes(val, 0, val == null ? 0 : val.length);
}
public void addBytes(byte[] val, int offset, int length) {
// We use the trick mentioned in "Less Hashing, Same Performance: Building a Better Bloom Filter"
// by Kirsch et.al. From abstract 'only two hash functions are necessary to effectively
// implement a Bloom filter without any loss in the asymptotic false positive probability'
// Lets split up 64-bit hashcode into two 32-bit hash codes and employ the technique mentioned
// in the above paper
long hash64 = val == null ? Murmur3.NULL_HASHCODE :
Murmur3.hash64(val, offset, length);
addHash(hash64);
}
private void addHash(long hash64) {
int hash1 = (int) hash64;
int hash2 = (int) (hash64 >>> 32);
for (int i = 1; i <= numHashFunctions; i++) {
int combinedHash = hash1 + (i * hash2);
// hashcode should be positive, flip all the bits if it's negative
if (combinedHash < 0) {
combinedHash = ~combinedHash;
}
int pos = combinedHash % numBits;
bitSet.set(pos);
}
}
public void addString(String val) {
if (val == null) {
add(null);
} else {
add(val.getBytes(Charset.defaultCharset()));
}
}
public void addLong(long val) {
addHash(getLongHash(val));
}
public void addDouble(double val) {
addLong(Double.doubleToLongBits(val));
}
public boolean test(byte[] val) {
return testBytes(val, 0, val == null ? 0 : val.length);
}
public boolean testBytes(byte[] val, int offset, int length) {
long hash64 = val == null ? Murmur3.NULL_HASHCODE :
Murmur3.hash64(val, offset, length);
return testHash(hash64);
}
private boolean testHash(long hash64) {
int hash1 = (int) hash64;
int hash2 = (int) (hash64 >>> 32);
for (int i = 1; i <= numHashFunctions; i++) {
int combinedHash = hash1 + (i * hash2);
// hashcode should be positive, flip all the bits if it's negative
if (combinedHash < 0) {
combinedHash = ~combinedHash;
}
int pos = combinedHash % numBits;
if (!bitSet.get(pos)) {
return false;
}
}
return true;
}
public boolean testString(String val) {
if (val == null) {
return test(null);
} else {
return test(val.getBytes(Charset.defaultCharset()));
}
}
public boolean testLong(long val) {
return testHash(getLongHash(val));
}
// Thomas Wang's integer hash function
// https://web.archive.org/web/20071223173210/http://www.concentric.net/~Ttwang/tech/inthash.htm
static long getLongHash(long key) {
key = (~key) + (key << 21); // key = (key << 21) - key - 1;
key = key ^ (key >> 24);
key = (key + (key << 3)) + (key << 8); // key * 265
key = key ^ (key >> 14);
key = (key + (key << 2)) + (key << 4); // key * 21
key = key ^ (key >> 28);
key = key + (key << 31);
return key;
}
public boolean testDouble(double val) {
return testLong(Double.doubleToLongBits(val));
}
public long sizeInBytes() {
return getBitSize() / 8;
}
public int getBitSize() {
return bitSet.getData().length * Long.SIZE;
}
public int getNumHashFunctions() {
return numHashFunctions;
}
public long[] getBitSet() {
return bitSet.getData();
}
@Override
public String toString() {
return "m: " + numBits + " k: " + numHashFunctions;
}
/**
* Merge the specified bloom filter with current bloom filter.
*
* @param that - bloom filter to merge
*/
public void merge(BloomFilter that) {
if (this != that && numBits == that.numBits && numHashFunctions == that.numHashFunctions) {
this.bitSet.putAll(that.bitSet);
} else {
throw new IllegalArgumentException("BloomFilters are not compatible for merging." +
" this - " + this + " that - " + that);
}
}
public void reset() {
this.bitSet.clear();
}
/**
* Helper method that only used for tests. Check if the given position in the bitSet is
* true. Use default visibility.
*/
boolean testBitSetPos(int pos) {
return this.bitSet.get(pos);
}
/**
* Bare metal bit set implementation. For performance reasons, this implementation does not check
* for index bounds nor expand the bit set size if the specified index is greater than the size.
*/
public static class BitSet {
private final long[] data;
public BitSet(long bits) {
this(new long[(int) Math.ceil((double) bits / (double) Long.SIZE)]);
}
/**
* Deserialize long array as bit set.
*
* @param data - bit array
*/
public BitSet(long[] data) {
assert data.length > 0 : "data length is zero!";
this.data = data;
}
/**
* Sets the bit at specified index.
*
* @param index - position
*/
public void set(int index) {
data[index >>> 6] |= (1L << index);
}
/**
* Returns true if the bit is set in the specified index.
*
* @param index - position
* @return - value at the bit position
*/
public boolean get(int index) {
return (data[index >>> 6] & (1L << index)) != 0;
}
/**
* Number of bits
*/
public long bitSize() {
return (long) data.length * Long.SIZE;
}
public long[] getData() {
return data;
}
/**
* Combines the two BitArrays using bitwise OR.
*/
public void putAll(BitSet array) {
assert data.length == array.data.length :
"BitArrays must be of equal length (" + data.length + "!= " + array.data.length + ")";
for (int i = 0; i < data.length; i++) {
data[i] |= array.data[i];
}
}
/**
* Clear the bit set.
*/
public void clear() {
Arrays.fill(data, 0);
}
@Override
public boolean equals(Object other) {
return other != null &&
other.getClass() == getClass() &&
Arrays.equals(data, ((BitSet) other).data);
}
@Override
public int hashCode() {
return Arrays.hashCode(data);
}
}
}