Skip to content

HTTPS clone URL

Subversion checkout URL

You can clone with
or
.
Download ZIP
Browse files

refactor Filter heirarchy, making hash generation easily customizable…

…. Use Murmur

hash + combinatorics to generate hashes.
  • Loading branch information...
commit 179dfe5c66488473735ff2683d9ec39e8de74462 1 parent d26164f
Jonathan Ellis authored
View
101 src/org/apache/cassandra/utils/BloomCalculations.java
@@ -30,106 +30,103 @@
*/
public class BloomCalculations {
- private static final int maxBits = 15;
- private static final int minBits = 2;
+ private static final int maxBuckets = 15;
+ private static final int minBuckets = 2;
private static final int minK = 1;
private static final int maxK = 8;
- private static final int[] optKPerBits =
- new int[]{1, // dummy K for 0 bits per element
- 1, // dummy K for 1 bits per element
+ private static final int[] optKPerBuckets =
+ new int[]{1, // dummy K for 0 buckets per element
+ 1, // dummy K for 1 buckets per element
1, 2, 3, 3, 4, 5, 5, 6, 7, 8, 8, 8, 8, 8};
/**
- * In the following table, the row 'i' shows false positive rates if i bits
+ * In the following table, the row 'i' shows false positive rates if i buckets
* per element are used. Column 'j' shows false positive rates if j hash
* functions are used. The first row is 'i=0', the first column is 'j=0'.
- * Each cell (i,j) the false positive rate determined by using i bits per
+ * Each cell (i,j) the false positive rate determined by using i buckets per
* element and j hash functions.
*/
- private static final double[][] probs = new double[][]{
- {1.0}, // dummy row representing 0 bits per element
- {1.0, 1.0}, // dummy row representing 1 bits per element
+ static final double[][] probs = new double[][]{
+ {1.0}, // dummy row representing 0 buckets per element
+ {1.0, 1.0}, // dummy row representing 1 buckets per element
{1.0, 0.393, 0.400},
{1.0, 0.283, 0.237, 0.253},
{1.0, 0.221, 0.155, 0.147, 0.160},
- {1.0, 0.181, 0.109, 0.092, 0.092, 0.101},
+ {1.0, 0.181, 0.109, 0.092, 0.092, 0.101}, // 5
{1.0, 0.154, 0.0804, 0.0609, 0.0561, 0.0578, 0.0638},
{1.0, 0.133, 0.0618, 0.0423, 0.0359, 0.0347, 0.0364},
{1.0, 0.118, 0.0489, 0.0306, 0.024, 0.0217, 0.0216, 0.0229},
- {1.0, 0.105, 0.0397, 0.0228, 0.0166, 0.0141, 0.0133, 0.0135, 0.0145},
+ {1.0, 0.105, 0.0397, 0.0228, 0.0166, 0.0141, 0.0133, 0.0135, 0.0145}, // 9
{1.0, 0.0952, 0.0329, 0.0174, 0.0118, 0.00943, 0.00844, 0.00819, 0.00846},
{1.0, 0.0869, 0.0276, 0.0136, 0.00864, 0.0065, 0.00552, 0.00513, 0.00509},
{1.0, 0.08, 0.0236, 0.0108, 0.00646, 0.00459, 0.00371, 0.00329, 0.00314},
{1.0, 0.074, 0.0203, 0.00875, 0.00492, 0.00332, 0.00255, 0.00217, 0.00199},
{1.0, 0.0689, 0.0177, 0.00718, 0.00381, 0.00244, 0.00179, 0.00146, 0.00129},
- {1.0, 0.0645, 0.0156, 0.00596, 0.003, 0.00183, 0.00128, 0.001, 0.000852}
+ {1.0, 0.0645, 0.0156, 0.00596, 0.003, 0.00183, 0.00128, 0.001, 0.000852} // 15
}; // the first column is a dummy column representing K=0.
- public static double getFailureRate(int bitsPerElement){
- int k = computeBestK(bitsPerElement);
- if(bitsPerElement >= probs.length) bitsPerElement = probs.length-1;
- return probs[bitsPerElement][k];
- }
-
/**
- * Given the number of bits that can be used per element, return the optimal
+ * Given the number of buckets that can be used per element, return the optimal
* number of hash functions in order to minimize the false positive rate.
*
- * @param bitsPerElement
+ * @param bucketsPerElement
* @return The number of hash functions that minimize the false positive rate.
*/
- public static int computeBestK(int bitsPerElement){
- if(bitsPerElement < 0)
- return optKPerBits[0];
- if(bitsPerElement >= optKPerBits.length)
- return optKPerBits[optKPerBits.length-1];
- return optKPerBits[bitsPerElement];
+ public static int computeBestK(int bucketsPerElement){
+ assert bucketsPerElement >= 0;
+ if(bucketsPerElement >= optKPerBuckets.length)
+ return optKPerBuckets[optKPerBuckets.length-1];
+ return optKPerBuckets[bucketsPerElement];
}
/**
* A wrapper class that holds two key parameters for a Bloom Filter: the
- * number of hash functions used, and the number of bits per element used.
+ * number of hash functions used, and the number of buckets per element used.
*/
- public static class BloomSpecification {
- int K; // number of hash functions.
- int bitsPerElement;
+ public static final class BloomSpecification {
+ final int K; // number of hash functions.
+ final int bucketsPerElement;
+
+ public BloomSpecification(int k, int bucketsPerElement) {
+ K = k;
+ this.bucketsPerElement = bucketsPerElement;
+ }
}
/**
* Given a maximum tolerable false positive probability, compute a Bloom
* specification which will give less than the specified false positive rate,
- * but minimize the number of bits per element and the number of hash
+ * but minimize the number of buckets per element and the number of hash
* functions used. Because bandwidth (and therefore total bitvector size)
* is considered more expensive than computing power, preference is given
- * to minimizing bits per element rather than number of hash funtions.
+ * to minimizing buckets per element rather than number of hash funtions.
*
* @param maxFalsePosProb The maximum tolerable false positive rate.
* @return A Bloom Specification which would result in a false positive rate
* less than specified by the function call.
*/
- public static BloomSpecification computeBitsAndK(double maxFalsePosProb){
- BloomSpecification spec = new BloomSpecification();
- spec.bitsPerElement = 2;
- spec.K = optKPerBits[spec.bitsPerElement];
-
- // Handle the trivial cases:
- if(maxFalsePosProb >= probs[minBits][minK]) return spec;
- if(maxFalsePosProb < probs[maxBits][maxK]) {
- spec.bitsPerElement = maxBits;
- spec.K = maxK;
- return spec;
+ public static BloomSpecification computeBucketsAndK(double maxFalsePosProb){
+ // Handle the trivial cases
+ if(maxFalsePosProb >= probs[minBuckets][minK]) {
+ return new BloomSpecification(2, optKPerBuckets[2]);
+ }
+ if(maxFalsePosProb < probs[maxBuckets][maxK]) {
+ return new BloomSpecification(maxK, maxBuckets);
}
- // First find the minimal required number of bits:
- while(probs[spec.bitsPerElement][spec.K] > maxFalsePosProb){
- spec.bitsPerElement++;
- spec.K = optKPerBits[spec.bitsPerElement];
+ // First find the minimal required number of buckets:
+ int bucketsPerElement = 2;
+ int K = optKPerBuckets[2];
+ while(probs[bucketsPerElement][K] > maxFalsePosProb){
+ bucketsPerElement++;
+ K = optKPerBuckets[bucketsPerElement];
}
- // Now that the number of bits is sufficient, see if we can relax K
+ // Now that the number of buckets is sufficient, see if we can relax K
// without losing too much precision.
- while(probs[spec.bitsPerElement][spec.K-1] <= maxFalsePosProb){
- spec.K--;
+ while(probs[bucketsPerElement][K - 1] <= maxFalsePosProb){
+ K--;
}
- return spec;
+
+ return new BloomSpecification(K, bucketsPerElement);
}
}
View
327 src/org/apache/cassandra/utils/BloomFilter.java
@@ -18,48 +18,15 @@
package org.apache.cassandra.utils;
-import java.math.*;
-import java.nio.ByteBuffer;
-import java.nio.LongBuffer;
-import java.io.*;
-import java.security.*;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-import java.util.zip.*;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
-import javax.xml.bind.annotation.XmlElement;
-
-import org.apache.cassandra.io.DataInputBuffer;
-import org.apache.cassandra.io.DataOutputBuffer;
import org.apache.cassandra.io.ICompactSerializer;
-import org.apache.cassandra.io.SSTable;
-
-/**
- * Author : Avinash Lakshman ( alakshman@facebook.com) & Prashant Malik ( pmalik@facebook.com )
- */
-
-public class BloomFilter implements Serializable
-{
- private static List<ISimpleHash> hashLibrary_ = new ArrayList<ISimpleHash>();
- private static ICompactSerializer<BloomFilter> serializer_;
-
- static
- {
- serializer_ = new BloomFilterSerializer();
- hashLibrary_.add(new RSHash());
- hashLibrary_.add(new JSHash());
- hashLibrary_.add(new PJWHash());
- hashLibrary_.add(new ELFHash());
- hashLibrary_.add(new BKDRHash());
- hashLibrary_.add(new SDBMHash());
- hashLibrary_.add(new DJBHash());
- hashLibrary_.add(new DEKHash());
- hashLibrary_.add(new BPHash());
- hashLibrary_.add(new FNVHash());
- hashLibrary_.add(new APHash());
- }
+public class BloomFilter extends Filter
+{
+ static ICompactSerializer<BloomFilter> serializer_ = new BloomFilterSerializer();
public static ICompactSerializer<BloomFilter> serializer()
{
@@ -67,52 +34,37 @@
}
private BitSet filter_;
- private int count_;
- private int size_;
- private int hashes_;
- private Random random_ = new Random(System.currentTimeMillis());
- public BloomFilter(int numElements, int bitsPerElement)
+ public BloomFilter(int numElements, int bucketsPerElement)
{
- // TODO -- think about the trivial cases more.
- // Note that it should indeed be possible to send a bloom filter that
- // encodes the empty set.
- if (numElements < 0 || bitsPerElement < 1)
- throw new IllegalArgumentException("Number of elements and bits "
- + "must be non-negative.");
- // Adding a small random number of bits so that even if the set
- // of elements hasn't changed, we'll get different false positives.
- count_ = numElements;
- size_ = numElements * bitsPerElement + 20 + random_.nextInt(64);
- filter_ = new BitSet(size_);
- //hashes_ = BloomCalculations.computeBestK(bitsPerElement);
- hashes_ = 8;
+ this(BloomCalculations.computeBestK(bucketsPerElement), new BitSet(numElements * bucketsPerElement + 20));
+ }
+
+ public BloomFilter(int numElements, double maxFalsePosProbability)
+ {
+ BloomCalculations.BloomSpecification spec = BloomCalculations
+ .computeBucketsAndK(maxFalsePosProbability);
+ filter_ = new BitSet(numElements * spec.bucketsPerElement + 20);
+ hashCount = spec.K;
}
/*
- * This version is only used by the deserializer.
+ * This version is only used by the deserializer.
*/
- BloomFilter(int count, int hashes, int size, BitSet filter)
+ BloomFilter(int hashes, BitSet filter)
{
- count_ = count;
- hashes_ = hashes;
- size_ = size;
+ hashCount = hashes;
filter_ = filter;
}
- int count()
+ public void clear()
{
- return count_;
- }
-
- int size()
- {
- return size_;
+ filter_.clear();
}
- int hashes()
+ int buckets()
{
- return hashes_;
+ return filter_.size();
}
BitSet filter()
@@ -122,19 +74,14 @@ BitSet filter()
public boolean isPresent(String key)
{
- boolean bVal = true;
- for (int i = 0; i < hashes_; ++i)
+ for (int bucketIndex : getHashBuckets(key))
{
- ISimpleHash hash = hashLibrary_.get(i);
- int hashValue = hash.hash(key);
- int index = Math.abs(hashValue % size_);
- if (!filter_.get(index))
+ if (!filter_.get(bucketIndex))
{
- bVal = false;
- break;
+ return false;
}
}
- return bVal;
+ return true;
}
/*
@@ -144,12 +91,9 @@ public boolean isPresent(String key)
*/
public void add(String key)
{
- for (int i = 0; i < hashes_; ++i)
+ for (int bucketIndex : getHashBuckets(key))
{
- ISimpleHash hash = hashLibrary_.get(i);
- int hashValue = hash.hash(key);
- int index = Math.abs(hashValue % size_);
- filter_.set(index);
+ filter_.set(bucketIndex);
}
}
@@ -157,218 +101,39 @@ public String toString()
{
return filter_.toString();
}
-}
-
-class BloomFilterSerializer implements ICompactSerializer<BloomFilter>
-{
- /*
- * The following methods are used for compact representation
- * of BloomFilter. This is essential, since we want to determine
- * the size of the serialized Bloom Filter blob before it is
- * populated armed with the knowledge of how many elements are
- * going to reside in it.
- */
-
- public void serialize(BloomFilter bf, DataOutputStream dos) throws IOException
- {
- /* write out the count of the BloomFilter */
- dos.writeInt(bf.count());
- /* write the number of hash functions used */
- dos.writeInt(bf.hashes());
- /* write the size of the BloomFilter */
- dos.writeInt(bf.size());
- BitSet.serializer().serialize(bf.filter(), dos);
- }
- public BloomFilter deserialize(DataInputStream dis) throws IOException
+ ICompactSerializer tserializer()
{
- /* read the count of the BloomFilter */
- int count = dis.readInt();
- /* read the number of hash functions */
- int hashes = dis.readInt();
- /* read the size of the bloom filter */
- int size = dis.readInt();
- BitSet bs = BitSet.serializer().deserialize(dis);
- return new BloomFilter(count, hashes, size, bs);
- }
-}
-
-interface ISimpleHash
-{
- public int hash(String str);
-}
-
-class RSHash implements ISimpleHash
-{
- public int hash(String str)
- {
- int b = 378551;
- int a = 63689;
- int hash = 0;
-
- for (int i = 0; i < str.length(); i++)
- {
- hash = hash * a + str.charAt(i);
- a = a * b;
- }
- return hash;
- }
-}
-
-class JSHash implements ISimpleHash
-{
- public int hash(String str)
- {
- int hash = 1315423911;
- for (int i = 0; i < str.length(); i++)
- {
- hash ^= ((hash << 5) + str.charAt(i) + (hash >> 2));
- }
- return hash;
- }
-}
-
-class PJWHash implements ISimpleHash
-{
- public int hash(String str)
- {
- int bitsInUnsignedInt = (4 * 8);
- int threeQuarters = (bitsInUnsignedInt * 3) / 4;
- int oneEighth = bitsInUnsignedInt / 8;
- int highBits = (0xFFFFFFFF) << (bitsInUnsignedInt - oneEighth);
- int hash = 0;
- int test = 0;
-
- for (int i = 0; i < str.length(); i++)
- {
- hash = (hash << oneEighth) + str.charAt(i);
-
- if ((test = hash & highBits) != 0)
- {
- hash = ((hash ^ (test >> threeQuarters)) & (~highBits));
- }
- }
- return hash;
+ return serializer_;
}
-}
-class ELFHash implements ISimpleHash
-{
- public int hash(String str)
+ int emptyBuckets()
{
- int hash = 0;
- int x = 0;
- for (int i = 0; i < str.length(); i++)
+ int n = 0;
+ for (int i = 0; i < buckets(); i++)
{
- hash = (hash << 4) + str.charAt(i);
-
- if ((x = hash & 0xF0000000) != 0)
+ if (!filter_.get(i))
{
- hash ^= (x >> 24);
+ n++;
}
- hash &= ~x;
- }
- return hash;
- }
-}
-
-class BKDRHash implements ISimpleHash
-{
- public int hash(String str)
- {
- int seed = 131; // 31 131 1313 13131 131313 etc..
- int hash = 0;
- for (int i = 0; i < str.length(); i++)
- {
- hash = (hash * seed) + str.charAt(i);
- }
- return hash;
- }
-}
-
-class SDBMHash implements ISimpleHash
-{
- public int hash(String str)
- {
- int hash = 0;
- for (int i = 0; i < str.length(); i++)
- {
- hash = str.charAt(i) + (hash << 6) + (hash << 16) - hash;
- }
- return hash;
- }
-}
-
-class DJBHash implements ISimpleHash
-{
- public int hash(String str)
- {
- int hash = 5381;
- for (int i = 0; i < str.length(); i++)
- {
- hash = ((hash << 5) + hash) + str.charAt(i);
- }
- return hash;
- }
-}
-
-class DEKHash implements ISimpleHash
-{
- public int hash(String str)
- {
- int hash = str.length();
- for (int i = 0; i < str.length(); i++)
- {
- hash = ((hash << 5) ^ (hash >> 27)) ^ str.charAt(i);
- }
- return hash;
- }
-}
-
-class BPHash implements ISimpleHash
-{
- public int hash(String str)
- {
- int hash = 0;
- for (int i = 0; i < str.length(); i++)
- {
- hash = hash << 7 ^ str.charAt(i);
}
- return hash;
+ return n;
}
}
-class FNVHash implements ISimpleHash
+class BloomFilterSerializer implements ICompactSerializer<BloomFilter>
{
- public int hash(String str)
+ public void serialize(BloomFilter bf, DataOutputStream dos)
+ throws IOException
{
- int fnv_prime = 0x811C9DC5;
- int hash = 0;
- for (int i = 0; i < str.length(); i++)
- {
- hash *= fnv_prime;
- hash ^= str.charAt(i);
- }
- return hash;
+ dos.writeInt(bf.getHashCount());
+ BitSet.serializer().serialize(bf.filter(), dos);
}
-}
-class APHash implements ISimpleHash
-{
- public int hash(String str)
+ public BloomFilter deserialize(DataInputStream dis) throws IOException
{
- int hash = 0xAAAAAAAA;
- for (int i = 0; i < str.length(); i++)
- {
- if ((i & 1) == 0)
- {
- hash ^= ((hash << 7) ^ str.charAt(i) ^ (hash >> 3));
- }
- else
- {
- hash ^= (~((hash << 11) ^ str.charAt(i) ^ (hash >> 5)));
- }
- }
- return hash;
+ int hashes = dis.readInt();
+ BitSet bs = BitSet.serializer().deserialize(dis);
+ return new BloomFilter(hashes, bs);
}
}
View
72 src/org/apache/cassandra/utils/Filter.java
@@ -0,0 +1,72 @@
+package org.apache.cassandra.utils;
+
+import java.io.UnsupportedEncodingException;
+import java.lang.reflect.Method;
+
+import org.apache.cassandra.io.ICompactSerializer;
+
+public abstract class Filter
+{
+ int hashCount;
+
+ private static MurmurHash hasher = new MurmurHash();
+
+ int getHashCount()
+ {
+ return hashCount;
+ }
+
+ public int[] getHashBuckets(String key)
+ {
+ return Filter.getHashBuckets(key, hashCount, buckets());
+ }
+
+ abstract int buckets();
+
+ public abstract void add(String key);
+
+ public abstract boolean isPresent(String key);
+
+ // for testing
+ abstract int emptyBuckets();
+
+ ICompactSerializer<Filter> getSerializer()
+ {
+ Method method = null;
+ try
+ {
+ method = getClass().getMethod("serializer");
+ return (ICompactSerializer<Filter>) method.invoke(null);
+ }
+ catch (Exception e)
+ {
+ throw new RuntimeException(e);
+ }
+ }
+
+ // murmur is faster than a sha-based approach and provides as-good collision
+ // resistance. the combinatorial generation approach described in
+ // http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf
+ // does prove to work in actual tests, and is obviously faster
+ // than performing further iterations of murmur.
+ static int[] getHashBuckets(String key, int hashCount, int max)
+ {
+ byte[] b;
+ try
+ {
+ b = key.getBytes("UTF-16");
+ }
+ catch (UnsupportedEncodingException e)
+ {
+ throw new RuntimeException(e);
+ }
+ int[] result = new int[hashCount];
+ int hash1 = hasher.hash(b, b.length, 0);
+ int hash2 = hasher.hash(b, b.length, hash1);
+ for (int i = 0; i < hashCount; i++)
+ {
+ result[i] = Math.abs((hash1 + i * hash2) % max);
+ }
+ return result;
+ }
+}
View
98 test/org/apache/cassandra/utils/BloomFilterTest.java
@@ -0,0 +1,98 @@
+package org.apache.cassandra.utils;
+
+import java.io.IOException;
+
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+public class BloomFilterTest
+{
+ public BloomFilter bf;
+ public BloomCalculations.BloomSpecification spec = BloomCalculations.computeBucketsAndK(0.0001);
+ static final int ELEMENTS = 10000;
+
+ public BloomFilterTest()
+ {
+ bf = new BloomFilter(ELEMENTS, spec.bucketsPerElement);
+ assert bf != null;
+ }
+
+ @BeforeMethod
+ public void clear()
+ {
+ bf.clear();
+ }
+
+ @Test
+ public void testOne()
+ {
+ bf.add("a");
+ assert bf.isPresent("a");
+ assert !bf.isPresent("b");
+ }
+
+ @Test
+ public void testFalsePositivesInt()
+ {
+ FilterTest.testFalsePositives(bf, FilterTest.intKeys(), FilterTest.randomKeys2());
+ }
+
+ @Test
+ public void testFalsePositivesRandom()
+ {
+ FilterTest.testFalsePositives(bf, FilterTest.randomKeys(), FilterTest.randomKeys2());
+ }
+
+ @Test
+ public void testWords()
+ {
+ if (KeyGenerator.WordGenerator.WORDS == 0)
+ {
+ return;
+ }
+ BloomFilter bf2 = new BloomFilter(KeyGenerator.WordGenerator.WORDS / 2, FilterTest.spec.bucketsPerElement);
+ int skipEven = KeyGenerator.WordGenerator.WORDS % 2 == 0 ? 0 : 2;
+ FilterTest.testFalsePositives(bf2,
+ new KeyGenerator.WordGenerator(skipEven, 2),
+ new KeyGenerator.WordGenerator(1, 2));
+ }
+
+ @Test
+ public void testSerialize() throws IOException
+ {
+ FilterTest.testSerialize(bf);
+ }
+
+ /* TODO move these into a nightly suite (they take 5-10 minutes each)
+ @Test
+ // run with -mx1G
+ public void testBigInt() {
+ int size = 100 * 1000 * 1000;
+ bf = new BloomFilter(size, FilterTest.spec.bucketsPerElement);
+ FilterTest.testFalsePositives(bf,
+ new KeyGenerator.IntGenerator(size),
+ new KeyGenerator.IntGenerator(size, size * 2));
+ }
+
+ @Test
+ public void testBigRandom() {
+ int size = 100 * 1000 * 1000;
+ bf = new BloomFilter(size, FilterTest.spec.bucketsPerElement);
+ FilterTest.testFalsePositives(bf,
+ new KeyGenerator.RandomStringGenerator(new Random().nextInt(), size),
+ new KeyGenerator.RandomStringGenerator(new Random().nextInt(), size));
+ }
+
+ @Test
+ public void timeit() {
+ int size = 300 * FilterTest.ELEMENTS;
+ bf = new BloomFilter(size, FilterTest.spec.bucketsPerElement);
+ for (int i = 0; i < 10; i++) {
+ FilterTest.testFalsePositives(bf,
+ new KeyGenerator.RandomStringGenerator(new Random().nextInt(), size),
+ new KeyGenerator.RandomStringGenerator(new Random().nextInt(), size));
+ bf.clear();
+ }
+ }
+ */
+}
View
95 test/org/apache/cassandra/utils/FilterTest.java
@@ -0,0 +1,95 @@
+package org.apache.cassandra.utils;
+
+import java.util.Iterator;
+import java.util.Set;
+import java.util.HashSet;
+import java.io.IOException;
+
+import org.testng.annotations.Test;
+import org.apache.cassandra.io.DataInputBuffer;
+import org.apache.cassandra.io.DataOutputBuffer;
+
+public class FilterTest
+{
+ public void testManyHashes(Iterator<String> keys)
+ {
+ int MAX_HASH_COUNT = 128;
+ Set<Integer> hashes = new HashSet<Integer>();
+ int collisions = 0;
+ while (keys.hasNext())
+ {
+ hashes.clear();
+ for (int hashIndex : Filter.getHashBuckets(keys.next(), MAX_HASH_COUNT, 1024 * 1024))
+ {
+ hashes.add(hashIndex);
+ }
+ collisions += (MAX_HASH_COUNT - hashes.size());
+ }
+ assert collisions <= 100;
+ }
+
+ @Test
+ public void testManyRandom()
+ {
+ testManyHashes(randomKeys());
+ }
+
+ // used by filter subclass tests
+
+ static final double MAX_FAILURE_RATE = 0.1;
+ public static final BloomCalculations.BloomSpecification spec = BloomCalculations.computeBucketsAndK(MAX_FAILURE_RATE);
+ static final int ELEMENTS = 10000;
+
+ static final ResetableIterator<String> intKeys()
+ {
+ return new KeyGenerator.IntGenerator(ELEMENTS);
+ }
+
+ static final ResetableIterator<String> randomKeys()
+ {
+ return new KeyGenerator.RandomStringGenerator(314159, ELEMENTS);
+ }
+
+ static final ResetableIterator<String> randomKeys2()
+ {
+ return new KeyGenerator.RandomStringGenerator(271828, ELEMENTS);
+ }
+
+ public static void testFalsePositives(Filter f, ResetableIterator<String> keys, ResetableIterator<String> otherkeys)
+ {
+ assert keys.size() == otherkeys.size();
+
+ while (keys.hasNext())
+ {
+ f.add(keys.next());
+ }
+
+ int fp = 0;
+ while (otherkeys.hasNext())
+ {
+ if (f.isPresent(otherkeys.next()))
+ {
+ fp++;
+ }
+ }
+
+ double fp_ratio = fp / (keys.size() * BloomCalculations.probs[spec.bucketsPerElement][spec.K]);
+ assert fp_ratio < 1.03 : fp_ratio;
+ }
+
+ public static Filter testSerialize(Filter f) throws IOException
+ {
+ f.add("a");
+ DataOutputBuffer out = new DataOutputBuffer();
+ f.getSerializer().serialize(f, out);
+
+ DataInputBuffer in = new DataInputBuffer();
+ in.reset(out.getData(), out.getLength());
+ Filter f2 = f.getSerializer().deserialize(in);
+
+ assert f2.isPresent("a");
+ assert !f2.isPresent("b");
+ return f2;
+ }
+
+}
View
2  test/org/apache/cassandra/utils/KeyGenerator.java
@@ -90,7 +90,7 @@ public void remove() {
WORDS++;
}
} catch (IOException e) {
- throw new RuntimeException(e);
+ WORDS = 0;
}
}

0 comments on commit 179dfe5

Please sign in to comment.
Something went wrong with that request. Please try again.