You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sn...@apache.org on 2015/07/05 17:15:32 UTC
cassandra git commit: Bloom filter false positive ratio is not
honoured
Repository: cassandra
Updated Branches:
refs/heads/trunk 036ddaf53 -> 23fd75f27
Bloom filter false positive ratio is not honoured
patch by Robert Stupp, reviewed by Benedict Elliott-Smith for CASSANDRA-8413
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/23fd75f2
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/23fd75f2
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/23fd75f2
Branch: refs/heads/trunk
Commit: 23fd75f27c40462636f09920719b5dcbef5b8f36
Parents: 036ddaf
Author: Robert Stupp <sn...@snazy.de>
Authored: Sun Jul 5 22:14:36 2015 +0700
Committer: Robert Stupp <sn...@snazy.de>
Committed: Sun Jul 5 22:14:36 2015 +0700
----------------------------------------------------------------------
CHANGES.txt | 1 +
NEWS.txt | 3 +
.../io/sstable/format/SSTableReader.java | 8 +-
.../cassandra/io/sstable/format/Version.java | 8 +-
.../io/sstable/format/big/BigFormat.java | 19 ++-
.../io/sstable/format/big/BigTableWriter.java | 2 +-
.../org/apache/cassandra/utils/BloomFilter.java | 31 +++-
.../cassandra/utils/BloomFilterSerializer.java | 24 ++-
.../apache/cassandra/utils/FilterFactory.java | 18 +--
.../org/apache/cassandra/utils/IFilter.java | 2 +-
test/data/bloom-filter/ka/foo.cql | 64 ++++++++
.../ka/foo/foo-atable-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
.../bloom-filter/ka/foo/foo-atable-ka-1-Data.db | Bin 0 -> 256 bytes
.../ka/foo/foo-atable-ka-1-Digest.sha1 | 1 +
.../ka/foo/foo-atable-ka-1-Filter.db | Bin 0 -> 24 bytes
.../ka/foo/foo-atable-ka-1-Index.db | Bin 0 -> 180 bytes
.../ka/foo/foo-atable-ka-1-Statistics.db | Bin 0 -> 4468 bytes
.../ka/foo/foo-atable-ka-1-Summary.db | Bin 0 -> 80 bytes
.../bloom-filter/ka/foo/foo-atable-ka-1-TOC.txt | 8 +
.../la/foo/la-1-big-CompressionInfo.db | Bin 0 -> 43 bytes
test/data/bloom-filter/la/foo/la-1-big-Data.db | Bin 0 -> 244 bytes
.../bloom-filter/la/foo/la-1-big-Digest.adler32 | 1 +
.../data/bloom-filter/la/foo/la-1-big-Filter.db | Bin 0 -> 24 bytes
test/data/bloom-filter/la/foo/la-1-big-Index.db | Bin 0 -> 180 bytes
.../bloom-filter/la/foo/la-1-big-Statistics.db | Bin 0 -> 4468 bytes
.../bloom-filter/la/foo/la-1-big-Summary.db | Bin 0 -> 80 bytes
test/data/bloom-filter/la/foo/la-1-big-TOC.txt | 8 +
.../serialization/3.0/utils.BloomFilter.bin | Bin 0 -> 2500016 bytes
.../serialization/3.0/utils.BloomFilter1000.bin | Bin 0 -> 2500016 bytes
.../3.0/utils.EstimatedHistogram.bin | Bin 0 -> 97500 bytes
.../cassandra/utils/LongBloomFilterTest.java | 159 +++++++++++++++++--
.../cassandra/AbstractSerializationsTester.java | 16 +-
.../org/apache/cassandra/utils/BitSetTest.java | 11 +-
.../apache/cassandra/utils/BloomFilterTest.java | 116 ++++++++++----
.../cassandra/utils/SerializationsTest.java | 148 ++++++++++++++++-
35 files changed, 550 insertions(+), 98 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a1018e3..bb3141e 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
3.0
+ * Bloom filter false positive ratio is not honoured (CASSANDRA-8413)
* New option for cassandra-stress to leave a ratio of columns null (CASSANDRA-9522)
* Change hinted_handoff_enabled yaml setting, JMX (CASSANDRA-9035)
* Add algorithmic token allocation (CASSANDRA-7032)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 3a9fe6a..54ed7c6 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -18,6 +18,9 @@ using the provided 'sstableupgrade' tool.
Upgrading
---------
+ _ New SSTable version 'la' with improved bloom-filter false-positive handling
+ compared to previous version 'ka' used in 2.2 and 2.1. Running sstableupgrade
+ is not necessary but recommended.
- Pig's CassandraStorage has been removed. Use CqlNativeStorage instead.
- Hadoop BulkOutputFormat and BulkRecordWriter have been removed; use
CqlBulkOutputFormat and CqlBulkRecordWriter instead.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index deff4c1..ab357a5 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -657,7 +657,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
{
// bf is enabled and fp chance matches the currently configured value.
load(false, true);
- loadBloomFilter();
+ loadBloomFilter(descriptor.version.hasOldBfHashOrder());
}
}
@@ -666,11 +666,11 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
*
* @throws IOException
*/
- private void loadBloomFilter() throws IOException
+ private void loadBloomFilter(boolean oldBfHashOrder) throws IOException
{
try (DataInputStream stream = new DataInputStream(new BufferedInputStream(new FileInputStream(descriptor.filenameFor(Component.FILTER)))))
{
- bf = FilterFactory.deserialize(stream, true);
+ bf = FilterFactory.deserialize(stream, true, oldBfHashOrder);
}
}
@@ -743,7 +743,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
: estimateRowsFromIndex(primaryIndex); // statistics is supposed to be optional
if (recreateBloomFilter)
- bf = FilterFactory.getFilter(estimatedKeys, metadata.getBloomFilterFpChance(), true);
+ bf = FilterFactory.getFilter(estimatedKeys, metadata.getBloomFilterFpChance(), true, descriptor.version.hasOldBfHashOrder());
try (IndexSummaryBuilder summaryBuilder = summaryLoaded ? null : new IndexSummaryBuilder(estimatedKeys, metadata.getMinIndexInterval(), samplingLevel))
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/src/java/org/apache/cassandra/io/sstable/format/Version.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/Version.java b/src/java/org/apache/cassandra/io/sstable/format/Version.java
index 8077a45..10ceb24 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/Version.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/Version.java
@@ -17,6 +17,8 @@
*/
package org.apache.cassandra.io.sstable.format;
+import java.util.regex.Pattern;
+
/**
* A set of feature flags associated with a SSTable format
*
@@ -30,6 +32,8 @@ package org.apache.cassandra.io.sstable.format;
*/
public abstract class Version
{
+ private static final Pattern VALIDATION = Pattern.compile("[a-z]+");
+
protected final String version;
protected final SSTableFormat format;
protected Version(SSTableFormat format, String version)
@@ -56,6 +60,8 @@ public abstract class Version
public abstract int correspondingMessagingVersion(); // Only use by storage that 'storeRows' so far
+ public abstract boolean hasOldBfHashOrder();
+
public String getVersion()
{
return version;
@@ -73,7 +79,7 @@ public abstract class Version
*/
public static boolean validate(String ver)
{
- return ver != null && ver.matches("[a-z]+");
+ return ver != null && VALIDATION.matcher(ver).matches();
}
abstract public boolean isCompatible();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
index fd0b5d5..d5f506b 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
@@ -17,25 +17,20 @@
*/
package org.apache.cassandra.io.sstable.format.big;
-import java.util.Iterator;
import java.util.Set;
-import com.google.common.collect.ImmutableList;
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.db.RowIndexEntry;
import org.apache.cassandra.db.SerializationHeader;
-import org.apache.cassandra.db.compaction.CompactionController;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.io.sstable.Component;
import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.IndexHelper;
import org.apache.cassandra.io.sstable.format.SSTableFormat;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableWriter;
import org.apache.cassandra.io.sstable.format.Version;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
-import org.apache.cassandra.io.util.FileDataInput;
import org.apache.cassandra.net.MessagingService;
/**
@@ -129,8 +124,13 @@ public class BigFormat implements SSTableFormat
private final boolean newFileName;
public final boolean storeRows;
public final int correspondingMessagingVersion; // Only use by storage that 'storeRows' so far
+ /**
+ * CASSANDRA-8413: 3.0 bloom filter representation changed (two longs just swapped)
+ * have no 'static' bits caused by using the same upper bits for both bloom filter and token distribution.
+ */
+ private final boolean hasOldBfHashOrder;
- public BigVersion(String version)
+ BigVersion(String version)
{
super(instance,version);
@@ -143,6 +143,7 @@ public class BigFormat implements SSTableFormat
newFileName = version.compareTo("la") >= 0;
storeRows = version.compareTo("la") >= 0;
correspondingMessagingVersion = storeRows ? MessagingService.VERSION_30 : MessagingService.VERSION_21;
+ hasOldBfHashOrder = version.compareTo("la") < 0;
}
@Override
@@ -182,6 +183,12 @@ public class BigFormat implements SSTableFormat
}
@Override
+ public boolean hasOldBfHashOrder()
+ {
+ return hasOldBfHashOrder;
+ }
+
+ @Override
public boolean hasNewFileName()
{
return newFileName;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
index e10beb0..a991d99 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
@@ -428,7 +428,7 @@ public class BigTableWriter extends SSTableWriter
indexFile = SequentialWriter.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)));
builder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(), false);
summary = new IndexSummaryBuilder(keyCount, metadata.getMinIndexInterval(), Downsampling.BASE_SAMPLING_LEVEL);
- bf = FilterFactory.getFilter(keyCount, metadata.getBloomFilterFpChance(), true);
+ bf = FilterFactory.getFilter(keyCount, metadata.getBloomFilterFpChance(), true, descriptor.version.hasOldBfHashOrder());
// register listeners to be alerted when the data files are flushed
indexFile.setPostFlushListener(new Runnable()
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/src/java/org/apache/cassandra/utils/BloomFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/BloomFilter.java b/src/java/org/apache/cassandra/utils/BloomFilter.java
index dbd489f..e1d9f20 100644
--- a/src/java/org/apache/cassandra/utils/BloomFilter.java
+++ b/src/java/org/apache/cassandra/utils/BloomFilter.java
@@ -34,26 +34,31 @@ public class BloomFilter extends WrappedSharedCloseable implements IFilter
public final IBitSet bitset;
public final int hashCount;
+ /**
+ * CASSANDRA-8413: 3.0 (inverted) bloom filters have no 'static' bits caused by using the same upper bits
+ * for both bloom filter and token distribution.
+ */
+ public final boolean oldBfHashOrder;
- BloomFilter(int hashCount, IBitSet bitset)
+ BloomFilter(int hashCount, IBitSet bitset, boolean oldBfHashOrder)
{
super(bitset);
this.hashCount = hashCount;
this.bitset = bitset;
+ this.oldBfHashOrder = oldBfHashOrder;
}
- BloomFilter(BloomFilter copy)
+ private BloomFilter(BloomFilter copy)
{
super(copy);
this.hashCount = copy.hashCount;
this.bitset = copy.bitset;
+ this.oldBfHashOrder = copy.oldBfHashOrder;
}
- public static final BloomFilterSerializer serializer = new BloomFilterSerializer();
-
public long serializedSize()
{
- return serializer.serializedSize(this);
+ return BloomFilterSerializer.serializedSize(this);
}
// Murmur is faster than an SHA-based approach and provides as-good collision
@@ -70,7 +75,7 @@ public class BloomFilter extends WrappedSharedCloseable implements IFilter
long[] hash = new long[2];
key.filterHash(hash);
long[] indexes = new long[hashCount];
- setIndexes(hash[0], hash[1], hashCount, max, indexes);
+ setIndexes(hash[1], hash[0], hashCount, max, indexes);
return indexes;
}
@@ -84,12 +89,19 @@ public class BloomFilter extends WrappedSharedCloseable implements IFilter
// so that we do not need to allocate two arrays.
long[] indexes = reusableIndexes.get();
key.filterHash(indexes);
- setIndexes(indexes[0], indexes[1], hashCount, bitset.capacity(), indexes);
+ setIndexes(indexes[1], indexes[0], hashCount, bitset.capacity(), indexes);
return indexes;
}
private void setIndexes(long base, long inc, int count, long max, long[] results)
{
+ if (oldBfHashOrder)
+ {
+ long x = inc;
+ inc = base;
+ base = x;
+ }
+
for (int i = 0; i < count; i++)
{
results[i] = FBUtilities.abs(base % max);
@@ -134,4 +146,9 @@ public class BloomFilter extends WrappedSharedCloseable implements IFilter
{
return bitset.offHeapSize();
}
+
+ public String toString()
+ {
+ return "BloomFilter[hashCount=" + hashCount + ";oldBfHashOrder=" + oldBfHashOrder + ";capacity=" + bitset.capacity() + ']';
+ }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java b/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
index 00bb153..6f57fc8 100644
--- a/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
+++ b/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
@@ -21,37 +21,35 @@ import java.io.DataInput;
import java.io.IOException;
import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.io.ISerializer;
-import org.apache.cassandra.io.util.DataInputPlus;
import org.apache.cassandra.io.util.DataOutputPlus;
import org.apache.cassandra.utils.obs.IBitSet;
import org.apache.cassandra.utils.obs.OffHeapBitSet;
import org.apache.cassandra.utils.obs.OpenBitSet;
-class BloomFilterSerializer implements ISerializer<BloomFilter>
+final class BloomFilterSerializer
{
- public void serialize(BloomFilter bf, DataOutputPlus out) throws IOException
+ private BloomFilterSerializer()
+ {
+ }
+
+ public static void serialize(BloomFilter bf, DataOutputPlus out) throws IOException
{
out.writeInt(bf.hashCount);
bf.bitset.serialize(out);
}
- public BloomFilter deserialize(DataInputPlus in) throws IOException
+ public static BloomFilter deserialize(DataInput in, boolean oldBfHashOrder) throws IOException
{
- return deserialize(in, false);
+ return deserialize(in, false, oldBfHashOrder);
}
@SuppressWarnings("resource")
- public BloomFilter deserialize(DataInput in, boolean offheap) throws IOException
+ public static BloomFilter deserialize(DataInput in, boolean offheap, boolean oldBfHashOrder) throws IOException
{
int hashes = in.readInt();
IBitSet bs = offheap ? OffHeapBitSet.deserialize(in) : OpenBitSet.deserialize(in);
- return createFilter(hashes, bs);
- }
- BloomFilter createFilter(int hashes, IBitSet bs)
- {
- return new BloomFilter(hashes, bs);
+ return new BloomFilter(hashes, bs, oldBfHashOrder);
}
/**
@@ -61,7 +59,7 @@ class BloomFilterSerializer implements ISerializer<BloomFilter>
*
* @return serialized size of the given bloom filter
*/
- public long serializedSize(BloomFilter bf)
+ public static long serializedSize(BloomFilter bf)
{
int size = TypeSizes.sizeof(bf.hashCount); // hash count
size += bf.bitset.serializedSize();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/src/java/org/apache/cassandra/utils/FilterFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/FilterFactory.java b/src/java/org/apache/cassandra/utils/FilterFactory.java
index d77500c..869f3fa 100644
--- a/src/java/org/apache/cassandra/utils/FilterFactory.java
+++ b/src/java/org/apache/cassandra/utils/FilterFactory.java
@@ -37,19 +37,19 @@ public class FilterFactory
public static void serialize(IFilter bf, DataOutputPlus output) throws IOException
{
- BloomFilter.serializer.serialize((BloomFilter) bf, output);
+ BloomFilterSerializer.serialize((BloomFilter) bf, output);
}
- public static IFilter deserialize(DataInput input, boolean offheap) throws IOException
+ public static IFilter deserialize(DataInput input, boolean offheap, boolean oldBfHashOrder) throws IOException
{
- return BloomFilter.serializer.deserialize(input, offheap);
+ return BloomFilterSerializer.deserialize(input, offheap, oldBfHashOrder);
}
/**
* @return A BloomFilter with the lowest practical false positive
* probability for the given number of elements.
*/
- public static IFilter getFilter(long numElements, int targetBucketsPerElem, boolean offheap)
+ public static IFilter getFilter(long numElements, int targetBucketsPerElem, boolean offheap, boolean oldBfHashOrder)
{
int maxBucketsPerElement = Math.max(1, BloomCalculations.maxBucketsPerElement(numElements));
int bucketsPerElement = Math.min(targetBucketsPerElem, maxBucketsPerElement);
@@ -58,7 +58,7 @@ public class FilterFactory
logger.warn(String.format("Cannot provide an optimal BloomFilter for %d elements (%d/%d buckets per element).", numElements, bucketsPerElement, targetBucketsPerElem));
}
BloomCalculations.BloomSpecification spec = BloomCalculations.computeBloomSpec(bucketsPerElement);
- return createFilter(spec.K, numElements, spec.bucketsPerElement, offheap);
+ return createFilter(spec.K, numElements, spec.bucketsPerElement, offheap, oldBfHashOrder);
}
/**
@@ -68,21 +68,21 @@ public class FilterFactory
* Asserts that the given probability can be satisfied using this
* filter.
*/
- public static IFilter getFilter(long numElements, double maxFalsePosProbability, boolean offheap)
+ public static IFilter getFilter(long numElements, double maxFalsePosProbability, boolean offheap, boolean oldBfHashOrder)
{
assert maxFalsePosProbability <= 1.0 : "Invalid probability";
if (maxFalsePosProbability == 1.0)
return new AlwaysPresentFilter();
int bucketsPerElement = BloomCalculations.maxBucketsPerElement(numElements);
BloomCalculations.BloomSpecification spec = BloomCalculations.computeBloomSpec(bucketsPerElement, maxFalsePosProbability);
- return createFilter(spec.K, numElements, spec.bucketsPerElement, offheap);
+ return createFilter(spec.K, numElements, spec.bucketsPerElement, offheap, oldBfHashOrder);
}
@SuppressWarnings("resource")
- private static IFilter createFilter(int hash, long numElements, int bucketsPer, boolean offheap)
+ private static IFilter createFilter(int hash, long numElements, int bucketsPer, boolean offheap, boolean oldBfHashOrder)
{
long numBits = (numElements * bucketsPer) + BITSET_EXCESS;
IBitSet bitset = offheap ? new OffHeapBitSet(numBits) : new OpenBitSet(numBits);
- return new BloomFilter(hash, bitset);
+ return new BloomFilter(hash, bitset, oldBfHashOrder);
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/src/java/org/apache/cassandra/utils/IFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/IFilter.java b/src/java/org/apache/cassandra/utils/IFilter.java
index 2f59864..b5eb2c4 100644
--- a/src/java/org/apache/cassandra/utils/IFilter.java
+++ b/src/java/org/apache/cassandra/utils/IFilter.java
@@ -21,7 +21,7 @@ import org.apache.cassandra.utils.concurrent.SharedCloseable;
public interface IFilter extends SharedCloseable
{
- public interface FilterKey
+ interface FilterKey
{
/** Places the murmur3 hash of the key in the given long array of size at least two. */
void filterHash(long[] dest);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/bloom-filter/ka/foo.cql
----------------------------------------------------------------------
diff --git a/test/data/bloom-filter/ka/foo.cql b/test/data/bloom-filter/ka/foo.cql
new file mode 100644
index 0000000..c4aed6a
--- /dev/null
+++ b/test/data/bloom-filter/ka/foo.cql
@@ -0,0 +1,64 @@
+create keyspace foo with replication = {'class':'SimpleStrategy', 'replication_factor':1};
+use foo ;
+create table atable ( pk int primary key, val int);
+insert into atable (pk, val) VALUES ( 1,1);
+insert into atable (pk, val) VALUES ( 2,2);
+insert into atable (pk, val) VALUES ( 3,3);
+insert into atable (pk, val) VALUES ( 4,4);
+insert into atable (pk, val) VALUES ( 5,5);
+insert into atable (pk, val) VALUES ( 6,6);
+insert into atable (pk, val) VALUES ( 7,7);
+insert into atable (pk, val) VALUES ( 8,8);
+insert into atable (pk, val) VALUES ( 9,9);
+insert into atable (pk, val) VALUES ( 10,10);
+
+
+
+[
+{"key": "5",
+ "cells": [["","",1428529571195019],
+ ["val","5",1428529571195019]]},
+{"key": "10",
+ "cells": [["","",1428529588242944],
+ ["val","10",1428529588242944]]},
+{"key": "1",
+ "cells": [["","",1428529563371015],
+ ["val","1",1428529563371015]]},
+{"key": "8",
+ "cells": [["","",1428529582362836],
+ ["val","8",1428529582362836]]},
+{"key": "2",
+ "cells": [["","",1428529565275080],
+ ["val","2",1428529565275080]]},
+{"key": "4",
+ "cells": [["","",1428529569331171],
+ ["val","4",1428529569331171]]},
+{"key": "7",
+ "cells": [["","",1428529575898967],
+ ["val","7",1428529575898967]]},
+{"key": "6",
+ "cells": [["","",1428529573027018],
+ ["val","6",1428529573027018]]},
+{"key": "9",
+ "cells": [["","",1428529585667042],
+ ["val","9",1428529585667042]]},
+{"key": "3",
+ "cells": [["","",1428529567379095],
+ ["val","3",1428529567379095]]}
+]
+
+
+
+SSTable: test/data/legacy-sstables/ka/foo/foo-atable-ka-1
+Partitioner: org.apache.cassandra.dht.Murmur3Partitioner
+Bloom Filter FP chance: 0,010000
+Minimum timestamp: 1428529563371015
+Maximum timestamp: 1428529588242944
+SSTable max local deletion time: 2147483647
+Compression ratio: 0.4
+Estimated droppable tombstones: 0.0
+SSTable Level: 0
+Repaired at: 0
+ReplayPosition(segmentId=1428529465658, position=6481)
+Estimated tombstone drop times:%n
+
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/bloom-filter/ka/foo/foo-atable-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/bloom-filter/ka/foo/foo-atable-ka-1-CompressionInfo.db b/test/data/bloom-filter/ka/foo/foo-atable-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..0c9c6e6
Binary files /dev/null and b/test/data/bloom-filter/ka/foo/foo-atable-ka-1-CompressionInfo.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Data.db b/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Data.db
new file mode 100644
index 0000000..c6a646b
Binary files /dev/null and b/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Data.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Digest.sha1 b/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Digest.sha1
new file mode 100644
index 0000000..e2c1b2a
--- /dev/null
+++ b/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Digest.sha1
@@ -0,0 +1 @@
+4153355033
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Filter.db b/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Filter.db
new file mode 100644
index 0000000..ea01eae
Binary files /dev/null and b/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Filter.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Index.db b/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Index.db
new file mode 100644
index 0000000..480fd51
Binary files /dev/null and b/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Index.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Statistics.db b/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Statistics.db
new file mode 100644
index 0000000..037e0b4
Binary files /dev/null and b/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Statistics.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Summary.db b/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Summary.db
new file mode 100644
index 0000000..602ec06
Binary files /dev/null and b/test/data/bloom-filter/ka/foo/foo-atable-ka-1-Summary.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/bloom-filter/ka/foo/foo-atable-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/bloom-filter/ka/foo/foo-atable-ka-1-TOC.txt b/test/data/bloom-filter/ka/foo/foo-atable-ka-1-TOC.txt
new file mode 100644
index 0000000..8fb2ca0
--- /dev/null
+++ b/test/data/bloom-filter/ka/foo/foo-atable-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+CompressionInfo.db
+Statistics.db
+Filter.db
+Index.db
+Summary.db
+Digest.sha1
+Data.db
+TOC.txt
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/bloom-filter/la/foo/la-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/bloom-filter/la/foo/la-1-big-CompressionInfo.db b/test/data/bloom-filter/la/foo/la-1-big-CompressionInfo.db
new file mode 100644
index 0000000..0c9c6e6
Binary files /dev/null and b/test/data/bloom-filter/la/foo/la-1-big-CompressionInfo.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/bloom-filter/la/foo/la-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/bloom-filter/la/foo/la-1-big-Data.db b/test/data/bloom-filter/la/foo/la-1-big-Data.db
new file mode 100644
index 0000000..dc3536d
Binary files /dev/null and b/test/data/bloom-filter/la/foo/la-1-big-Data.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/bloom-filter/la/foo/la-1-big-Digest.adler32
----------------------------------------------------------------------
diff --git a/test/data/bloom-filter/la/foo/la-1-big-Digest.adler32 b/test/data/bloom-filter/la/foo/la-1-big-Digest.adler32
new file mode 100644
index 0000000..d6952e2
--- /dev/null
+++ b/test/data/bloom-filter/la/foo/la-1-big-Digest.adler32
@@ -0,0 +1 @@
+3417187619
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/bloom-filter/la/foo/la-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/bloom-filter/la/foo/la-1-big-Filter.db b/test/data/bloom-filter/la/foo/la-1-big-Filter.db
new file mode 100644
index 0000000..533a611
Binary files /dev/null and b/test/data/bloom-filter/la/foo/la-1-big-Filter.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/bloom-filter/la/foo/la-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/bloom-filter/la/foo/la-1-big-Index.db b/test/data/bloom-filter/la/foo/la-1-big-Index.db
new file mode 100644
index 0000000..480fd51
Binary files /dev/null and b/test/data/bloom-filter/la/foo/la-1-big-Index.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/bloom-filter/la/foo/la-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/bloom-filter/la/foo/la-1-big-Statistics.db b/test/data/bloom-filter/la/foo/la-1-big-Statistics.db
new file mode 100644
index 0000000..b8127c2
Binary files /dev/null and b/test/data/bloom-filter/la/foo/la-1-big-Statistics.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/bloom-filter/la/foo/la-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/bloom-filter/la/foo/la-1-big-Summary.db b/test/data/bloom-filter/la/foo/la-1-big-Summary.db
new file mode 100644
index 0000000..602ec06
Binary files /dev/null and b/test/data/bloom-filter/la/foo/la-1-big-Summary.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/bloom-filter/la/foo/la-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/bloom-filter/la/foo/la-1-big-TOC.txt b/test/data/bloom-filter/la/foo/la-1-big-TOC.txt
new file mode 100644
index 0000000..ee47456
--- /dev/null
+++ b/test/data/bloom-filter/la/foo/la-1-big-TOC.txt
@@ -0,0 +1,8 @@
+Data.db
+CompressionInfo.db
+Summary.db
+Index.db
+TOC.txt
+Filter.db
+Statistics.db
+Digest.adler32
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/serialization/3.0/utils.BloomFilter.bin
----------------------------------------------------------------------
diff --git a/test/data/serialization/3.0/utils.BloomFilter.bin b/test/data/serialization/3.0/utils.BloomFilter.bin
new file mode 100644
index 0000000..e8bfb4f
Binary files /dev/null and b/test/data/serialization/3.0/utils.BloomFilter.bin differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/serialization/3.0/utils.BloomFilter1000.bin
----------------------------------------------------------------------
diff --git a/test/data/serialization/3.0/utils.BloomFilter1000.bin b/test/data/serialization/3.0/utils.BloomFilter1000.bin
new file mode 100644
index 0000000..cde99c9
Binary files /dev/null and b/test/data/serialization/3.0/utils.BloomFilter1000.bin differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/data/serialization/3.0/utils.EstimatedHistogram.bin
----------------------------------------------------------------------
diff --git a/test/data/serialization/3.0/utils.EstimatedHistogram.bin b/test/data/serialization/3.0/utils.EstimatedHistogram.bin
new file mode 100644
index 0000000..bedd39b
Binary files /dev/null and b/test/data/serialization/3.0/utils.EstimatedHistogram.bin differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/long/org/apache/cassandra/utils/LongBloomFilterTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/utils/LongBloomFilterTest.java b/test/long/org/apache/cassandra/utils/LongBloomFilterTest.java
index 8d916a0..c50296d 100644
--- a/test/long/org/apache/cassandra/utils/LongBloomFilterTest.java
+++ b/test/long/org/apache/cassandra/utils/LongBloomFilterTest.java
@@ -19,11 +19,15 @@
package org.apache.cassandra.utils;
import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import static org.apache.cassandra.utils.FilterFactory.getFilter;
+import static org.apache.cassandra.utils.FilterTestHelper.testFalsePositives;
+
public class LongBloomFilterTest
{
private static final Logger logger = LoggerFactory.getLogger(LongBloomFilterTest.class);
@@ -34,39 +38,164 @@ public class LongBloomFilterTest
@Test
public void testBigInt()
{
+ testBigInt(false);
+ testBigInt(true);
+ }
+ private static void testBigInt(boolean oldBfHashOrder)
+ {
int size = 10 * 1000 * 1000;
- IFilter bf = FilterFactory.getFilter(size, FilterTestHelper.spec.bucketsPerElement, false);
- double fp = FilterTestHelper.testFalsePositives(bf,
- new KeyGenerator.IntGenerator(size),
- new KeyGenerator.IntGenerator(size, size * 2));
- logger.info("Bloom filter false positive: {}", fp);
+ IFilter bf = getFilter(size, FilterTestHelper.spec.bucketsPerElement, false, oldBfHashOrder);
+ double fp = testFalsePositives(bf,
+ new KeyGenerator.IntGenerator(size),
+ new KeyGenerator.IntGenerator(size, size * 2));
+ logger.info("Bloom filter false positive for oldBfHashOrder={}: {}", oldBfHashOrder, fp);
}
@Test
public void testBigRandom()
{
+ testBigRandom(false);
+ testBigRandom(true);
+ }
+ private static void testBigRandom(boolean oldBfHashOrder)
+ {
+ int size = 10 * 1000 * 1000;
+ IFilter bf = getFilter(size, FilterTestHelper.spec.bucketsPerElement, false, oldBfHashOrder);
+ double fp = testFalsePositives(bf,
+ new KeyGenerator.RandomStringGenerator(new Random().nextInt(), size),
+ new KeyGenerator.RandomStringGenerator(new Random().nextInt(), size));
+ logger.info("Bloom filter false positive for oldBfHashOrder={}: {}", oldBfHashOrder, fp);
+ }
+
+ /**
+ * NB: needs to run with -mx1G
+ */
+ @Test
+ public void testConstrained()
+ {
+ testConstrained(false);
+ testConstrained(true);
+ }
+ private static void testConstrained(boolean oldBfHashOrder)
+ {
int size = 10 * 1000 * 1000;
- IFilter bf = FilterFactory.getFilter(size, FilterTestHelper.spec.bucketsPerElement, false);
- double fp = FilterTestHelper.testFalsePositives(bf,
- new KeyGenerator.RandomStringGenerator(new Random().nextInt(), size),
- new KeyGenerator.RandomStringGenerator(new Random().nextInt(), size));
- logger.info("Bloom filter false positive: {}", fp);
+ try (IFilter bf = getFilter(size, 0.01, false, oldBfHashOrder))
+ {
+ double fp = testFalsePositives(bf,
+ new KeyGenerator.IntGenerator(size),
+ new KeyGenerator.IntGenerator(size, size * 2));
+ logger.info("Bloom filter false positive for oldBfHashOrder={}: {}", oldBfHashOrder, fp);
+ }
+ }
+
+ private static void testConstrained(double targetFp, int elements, boolean oldBfHashOrder, int staticBitCount, long ... staticBits)
+ {
+ for (long bits : staticBits)
+ {
+ try (IFilter bf = getFilter(elements, targetFp, false, oldBfHashOrder);)
+ {
+ SequentialHashGenerator gen = new SequentialHashGenerator(staticBitCount, bits);
+ long[] hash = new long[2];
+ for (int i = 0 ; i < elements ; i++)
+ {
+ gen.nextHash(hash);
+ bf.add(filterKey(hash[0], hash[1]));
+ }
+ int falsePositiveCount = 0;
+ for (int i = 0 ; i < elements ; i++)
+ {
+ gen.nextHash(hash);
+ if (bf.isPresent(filterKey(hash[0], hash[1])))
+ falsePositiveCount++;
+ }
+ double fp = falsePositiveCount / (double) elements;
+ double ratio = fp/targetFp;
+ System.out.printf("%.2f, ", ratio);
+ }
+ }
+ System.out.printf("%d elements, %d static bits, %.2f target\n", elements, staticBitCount, targetFp);
+ }
+
+ private static IFilter.FilterKey filterKey(final long hash1, final long hash2)
+ {
+ return new IFilter.FilterKey()
+ {
+ public void filterHash(long[] dest)
+ {
+ dest[0] = hash1;
+ dest[1] = hash2;
+ }
+ };
+ }
+
+ @Test
+ public void testBffp()
+ {
+ bffp(false);
+ bffp(true);
+ }
+
+ private static void bffp(boolean flipInputs)
+ {
+ System.out.println("Bloom filter false posiitive with flipInputs=" + flipInputs);
+ long[] staticBits = staticBits(4, 0);
+ testConstrained(0.01d, 10 << 20, flipInputs, 0, staticBits);
+ testConstrained(0.01d, 1 << 20, flipInputs, 6, staticBits);
+ testConstrained(0.01d, 10 << 20, flipInputs, 6, staticBits);
+ testConstrained(0.01d, 1 << 19, flipInputs, 10, staticBits);
+ testConstrained(0.01d, 1 << 20, flipInputs, 10, staticBits);
+ testConstrained(0.01d, 10 << 20, flipInputs, 10, staticBits);
+ testConstrained(0.1d, 10 << 20, flipInputs, 0, staticBits);
+ testConstrained(0.1d, 10 << 20, flipInputs, 8, staticBits);
+ testConstrained(0.1d, 10 << 20, flipInputs, 10, staticBits);
+ }
+
+ static long[] staticBits(int random, long ... fixed)
+ {
+ long[] result = new long[random + fixed.length];
+ System.arraycopy(fixed, 0, result, 0, fixed.length);
+ for (int i = 0 ; i < random ; i++)
+ result[fixed.length + i] = ThreadLocalRandom.current().nextLong();
+ return result;
+ }
+
+ private static class SequentialHashGenerator
+ {
+ final long mask;
+ final long staticBits;
+ int next;
+ private SequentialHashGenerator(int staticBitCount, long staticBits) {
+ this.mask = -1 >>> staticBitCount;
+ this.staticBits = staticBits & ~mask;
+ }
+ void nextHash(long[] fill)
+ {
+ MurmurHash.hash3_x64_128(ByteBufferUtil.bytes(next), 0, 4, 0, fill);
+ fill[0] &= mask;
+ fill[0] |= staticBits;
+ next++;
+ }
}
@Test
public void timeit()
{
+ timeit(false);
+ timeit(true);
+ }
+ private static void timeit(boolean oldBfHashOrder)
+ {
int size = 300 * FilterTestHelper.ELEMENTS;
- IFilter bf = FilterFactory.getFilter(size, FilterTestHelper.spec.bucketsPerElement, false);
+ IFilter bf = getFilter(size, FilterTestHelper.spec.bucketsPerElement, false, oldBfHashOrder);
double sumfp = 0;
for (int i = 0; i < 10; i++)
{
- FilterTestHelper.testFalsePositives(bf,
- new KeyGenerator.RandomStringGenerator(new Random().nextInt(), size),
- new KeyGenerator.RandomStringGenerator(new Random().nextInt(), size));
+ testFalsePositives(bf,
+ new KeyGenerator.RandomStringGenerator(new Random().nextInt(), size),
+ new KeyGenerator.RandomStringGenerator(new Random().nextInt(), size));
bf.clear();
}
- logger.info("Bloom filter mean false positive: {}", sumfp / 10);
+ logger.info("Bloom filter mean false positive for oldBfHashOrder={}: {}", oldBfHashOrder, sumfp / 10);
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/AbstractSerializationsTester.java b/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
index 636d673..4ee5ce4 100644
--- a/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
+++ b/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
@@ -45,6 +45,7 @@ public class AbstractSerializationsTester
put("1.2", MessagingService.VERSION_12);
put("2.0", MessagingService.VERSION_20);
put("2.1", MessagingService.VERSION_21);
+ put("2.2", MessagingService.VERSION_22);
}};
protected static final boolean EXECUTE_WRITES = Boolean.getBoolean("cassandra.test-serialization-writes");
@@ -63,7 +64,12 @@ public class AbstractSerializationsTester
protected static DataInputStreamPlus getInput(String name) throws IOException
{
- File f = new File("test/data/serialization/" + CUR_VER + "/" + name);
+ return getInput(CUR_VER, name);
+ }
+
+ protected static DataInputStreamPlus getInput(String version, String name) throws IOException
+ {
+ File f = new File("test/data/serialization/" + version + '/' + name);
assert f.exists() : f.getPath();
return new DataInputPlus.DataInputStreamPlus(new FileInputStream(f));
}
@@ -71,7 +77,13 @@ public class AbstractSerializationsTester
@SuppressWarnings("resource")
protected static DataOutputStreamPlus getOutput(String name) throws IOException
{
- File f = new File("test/data/serialization/" + CUR_VER + "/" + name);
+ return getOutput(CUR_VER, name);
+ }
+
+ @SuppressWarnings("resource")
+ protected static DataOutputStreamPlus getOutput(String version, String name) throws IOException
+ {
+ File f = new File("test/data/serialization/" + version + '/' + name);
f.getParentFile().mkdirs();
return new BufferedDataOutputStreamPlus(new FileOutputStream(f).getChannel());
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/unit/org/apache/cassandra/utils/BitSetTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/BitSetTest.java b/test/unit/org/apache/cassandra/utils/BitSetTest.java
index a7a0660..0f51531 100644
--- a/test/unit/org/apache/cassandra/utils/BitSetTest.java
+++ b/test/unit/org/apache/cassandra/utils/BitSetTest.java
@@ -44,8 +44,13 @@ public class BitSetTest
@Test
public void compareBitSets()
{
- BloomFilter bf2 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE, false);
- BloomFilter bf3 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE, true);
+ compareBitSets(false);
+ compareBitSets(true);
+ }
+ private static void compareBitSets(boolean oldBfHashOrder)
+ {
+ BloomFilter bf2 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE, false, oldBfHashOrder);
+ BloomFilter bf3 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE, true, oldBfHashOrder);
RandomStringGenerator gen1 = new KeyGenerator.RandomStringGenerator(new Random().nextInt(), FilterTestHelper.ELEMENTS);
@@ -85,7 +90,7 @@ public class BitSetTest
}
}
- private void populateAndReserialize(IBitSet bs) throws IOException
+ private static void populateAndReserialize(IBitSet bs) throws IOException
{
for (long i = 0; i < bs.capacity(); i++)
if (random.nextBoolean())
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/unit/org/apache/cassandra/utils/BloomFilterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/BloomFilterTest.java b/test/unit/org/apache/cassandra/utils/BloomFilterTest.java
index 72f2825..2e76e0e 100644
--- a/test/unit/org/apache/cassandra/utils/BloomFilterTest.java
+++ b/test/unit/org/apache/cassandra/utils/BloomFilterTest.java
@@ -20,13 +20,16 @@ package org.apache.cassandra.utils;
import java.io.*;
import java.nio.ByteBuffer;
+import java.util.ArrayList;
import java.util.HashSet;
import java.util.Iterator;
+import java.util.List;
import java.util.Random;
import java.util.Set;
import org.junit.*;
+import org.apache.cassandra.db.marshal.Int32Type;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.dht.Murmur3Partitioner;
import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
@@ -38,21 +41,22 @@ import org.apache.cassandra.utils.KeyGenerator.RandomStringGenerator;
public class BloomFilterTest
{
- public IFilter bf;
+ public IFilter bfOldFormat;
+ public IFilter bfInvHashes;
public BloomFilterTest()
{
}
- public static IFilter testSerialize(IFilter f) throws IOException
+ public static IFilter testSerialize(IFilter f, boolean oldBfHashOrder) throws IOException
{
f.add(FilterTestHelper.bytes("a"));
DataOutputBuffer out = new DataOutputBuffer();
FilterFactory.serialize(f, out);
ByteArrayInputStream in = new ByteArrayInputStream(out.getData(), 0, out.getLength());
- IFilter f2 = FilterFactory.deserialize(new DataInputStream(in), true);
+ IFilter f2 = FilterFactory.deserialize(new DataInputStream(in), true, oldBfHashOrder);
assert f2.isPresent(FilterTestHelper.bytes("a"));
assert !f2.isPresent(FilterTestHelper.bytes("b"));
@@ -63,13 +67,15 @@ public class BloomFilterTest
@Before
public void setup()
{
- bf = FilterFactory.getFilter(10000L, FilterTestHelper.MAX_FAILURE_RATE, true);
+ bfOldFormat = FilterFactory.getFilter(10000L, FilterTestHelper.MAX_FAILURE_RATE, true, true);
+ bfInvHashes = FilterFactory.getFilter(10000L, FilterTestHelper.MAX_FAILURE_RATE, true, false);
}
@After
public void destroy()
{
- bf.close();
+ bfOldFormat.close();
+ bfInvHashes.close();
}
@Test(expected = UnsupportedOperationException.class)
@@ -88,21 +94,29 @@ public class BloomFilterTest
@Test
public void testOne()
{
- bf.add(FilterTestHelper.bytes("a"));
- assert bf.isPresent(FilterTestHelper.bytes("a"));
- assert !bf.isPresent(FilterTestHelper.bytes("b"));
+ bfOldFormat.add(FilterTestHelper.bytes("a"));
+ assert bfOldFormat.isPresent(FilterTestHelper.bytes("a"));
+ assert !bfOldFormat.isPresent(FilterTestHelper.bytes("b"));
+
+ bfInvHashes.add(FilterTestHelper.bytes("a"));
+ assert bfInvHashes.isPresent(FilterTestHelper.bytes("a"));
+ assert !bfInvHashes.isPresent(FilterTestHelper.bytes("b"));
}
@Test
public void testFalsePositivesInt()
{
- FilterTestHelper.testFalsePositives(bf, FilterTestHelper.intKeys(), FilterTestHelper.randomKeys2());
+ FilterTestHelper.testFalsePositives(bfOldFormat, FilterTestHelper.intKeys(), FilterTestHelper.randomKeys2());
+
+ FilterTestHelper.testFalsePositives(bfInvHashes, FilterTestHelper.intKeys(), FilterTestHelper.randomKeys2());
}
@Test
public void testFalsePositivesRandom()
{
- FilterTestHelper.testFalsePositives(bf, FilterTestHelper.randomKeys(), FilterTestHelper.randomKeys2());
+ FilterTestHelper.testFalsePositives(bfOldFormat, FilterTestHelper.randomKeys(), FilterTestHelper.randomKeys2());
+
+ FilterTestHelper.testFalsePositives(bfInvHashes, FilterTestHelper.randomKeys(), FilterTestHelper.randomKeys2());
}
@Test
@@ -112,30 +126,48 @@ public class BloomFilterTest
{
return;
}
- IFilter bf2 = FilterFactory.getFilter(KeyGenerator.WordGenerator.WORDS / 2, FilterTestHelper.MAX_FAILURE_RATE, true);
+ IFilter bf2 = FilterFactory.getFilter(KeyGenerator.WordGenerator.WORDS / 2, FilterTestHelper.MAX_FAILURE_RATE, true, false);
int skipEven = KeyGenerator.WordGenerator.WORDS % 2 == 0 ? 0 : 2;
FilterTestHelper.testFalsePositives(bf2,
new KeyGenerator.WordGenerator(skipEven, 2),
new KeyGenerator.WordGenerator(1, 2));
bf2.close();
+
+ // new, swapped hash values bloom filter
+ bf2 = FilterFactory.getFilter(KeyGenerator.WordGenerator.WORDS / 2, FilterTestHelper.MAX_FAILURE_RATE, true, true);
+ FilterTestHelper.testFalsePositives(bf2,
+ new KeyGenerator.WordGenerator(skipEven, 2),
+ new KeyGenerator.WordGenerator(1, 2));
+ bf2.close();
}
@Test
public void testSerialize() throws IOException
{
- BloomFilterTest.testSerialize(bf).close();
+ BloomFilterTest.testSerialize(bfOldFormat, true).close();
+
+ BloomFilterTest.testSerialize(bfInvHashes, false).close();
+ }
+
+ @Test
+ @Ignore
+ public void testManyRandom()
+ {
+ testManyRandom(FilterTestHelper.randomKeys(), false);
+
+ testManyRandom(FilterTestHelper.randomKeys(), true);
}
- public void testManyHashes(Iterator<ByteBuffer> keys)
+ private static void testManyRandom(Iterator<ByteBuffer> keys, boolean oldBfHashOrder)
{
int MAX_HASH_COUNT = 128;
- Set<Long> hashes = new HashSet<Long>();
+ Set<Long> hashes = new HashSet<>();
long collisions = 0;
while (keys.hasNext())
{
hashes.clear();
FilterKey buf = FilterTestHelper.wrap(keys.next());
- BloomFilter bf = (BloomFilter) FilterFactory.getFilter(10, 1, false);
+ BloomFilter bf = (BloomFilter) FilterFactory.getFilter(10, 1, false, oldBfHashOrder);
for (long hashIndex : bf.getHashBuckets(buf, MAX_HASH_COUNT, 1024 * 1024))
{
hashes.add(hashIndex);
@@ -143,28 +175,48 @@ public class BloomFilterTest
collisions += (MAX_HASH_COUNT - hashes.size());
bf.close();
}
- assert collisions <= 100;
+ Assert.assertTrue("collisions=" + collisions, collisions <= 100);
}
- @Test
- public void testManyRandom()
+ @Test(expected = UnsupportedOperationException.class)
+ public void testOffHeapException()
{
- testManyHashes(FilterTestHelper.randomKeys());
+ long numKeys = ((long)Integer.MAX_VALUE) * 64L + 1L; // approx 128 Billion
+ FilterFactory.getFilter(numKeys, 0.01d, true, true).close();
}
- @Test(expected = UnsupportedOperationException.class)
- public void testOffHeapException()
+ @Test
+ public void compareCachedKeyOldHashOrder()
{
- long numKeys = (Integer.MAX_VALUE * 64) + 1; // approx 128 Billion
- FilterFactory.getFilter(numKeys, 0.01d, true);
+ BloomFilter bf1 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE, false, true);
+ BloomFilter bf2 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE, false, true);
+ BloomFilter bf3 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE, false, true);
+
+ RandomStringGenerator gen1 = new KeyGenerator.RandomStringGenerator(new Random().nextInt(), FilterTestHelper.ELEMENTS);
+
+ // make sure all bitsets are empty.
+ BitSetTest.compare(bf1.bitset, bf2.bitset);
+ BitSetTest.compare(bf1.bitset, bf3.bitset);
+
+ while (gen1.hasNext())
+ {
+ ByteBuffer key = gen1.next();
+ FilterKey cached = FilterTestHelper.wrapCached(key);
+ bf1.add(FilterTestHelper.wrap(key));
+ bf2.add(cached);
+ bf3.add(cached);
+ }
+
+ BitSetTest.compare(bf1.bitset, bf2.bitset);
+ BitSetTest.compare(bf1.bitset, bf3.bitset);
}
@Test
- public void compareCachedKey()
+ public void compareCachedKeyNewHashOrder()
{
- try (BloomFilter bf1 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE, false);
- BloomFilter bf2 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE, false);
- BloomFilter bf3 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE, false))
+ try (BloomFilter bf1 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE, false, false);
+ BloomFilter bf2 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE, false, false);
+ BloomFilter bf3 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE, false, false))
{
RandomStringGenerator gen1 = new KeyGenerator.RandomStringGenerator(new Random().nextInt(), FilterTestHelper.ELEMENTS);
@@ -190,10 +242,16 @@ public class BloomFilterTest
@Ignore
public void testHugeBFSerialization() throws IOException
{
+ hugeBFSerialization(false);
+ hugeBFSerialization(true);
+ }
+
+ static void hugeBFSerialization(boolean oldBfHashOrder) throws IOException
+ {
ByteBuffer test = ByteBuffer.wrap(new byte[] {0, 1});
File file = FileUtils.createTempFile("bloomFilterTest-", ".dat");
- BloomFilter filter = (BloomFilter) FilterFactory.getFilter(((long) Integer.MAX_VALUE / 8) + 1, 0.01d, true);
+ BloomFilter filter = (BloomFilter) FilterFactory.getFilter(((long) Integer.MAX_VALUE / 8) + 1, 0.01d, true, oldBfHashOrder);
filter.add(FilterTestHelper.wrap(test));
DataOutputStreamPlus out = new BufferedDataOutputStreamPlus(new FileOutputStream(file));
FilterFactory.serialize(filter, out);
@@ -202,7 +260,7 @@ public class BloomFilterTest
filter.close();
DataInputStream in = new DataInputStream(new FileInputStream(file));
- BloomFilter filter2 = (BloomFilter) FilterFactory.deserialize(in, true);
+ BloomFilter filter2 = (BloomFilter) FilterFactory.deserialize(in, true, oldBfHashOrder);
Assert.assertTrue(filter2.isPresent(FilterTestHelper.wrap(test)));
FileUtils.closeQuietly(in);
filter2.close();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/23fd75f2/test/unit/org/apache/cassandra/utils/SerializationsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/SerializationsTest.java b/test/unit/org/apache/cassandra/utils/SerializationsTest.java
index f3809b3..c50f400 100644
--- a/test/unit/org/apache/cassandra/utils/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/utils/SerializationsTest.java
@@ -24,22 +24,42 @@ import java.io.IOException;
import org.junit.Assert;
import org.junit.Test;
import org.apache.cassandra.AbstractSerializationsTester;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.marshal.Int32Type;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
import org.apache.cassandra.io.util.DataOutputStreamPlus;
import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+
+import java.io.File;
+import java.io.FileInputStream;
public class SerializationsTest extends AbstractSerializationsTester
{
- private void testBloomFilterWrite(boolean offheap) throws IOException
+ private static void testBloomFilterWrite(boolean offheap, boolean oldBfHashOrder) throws IOException
{
IPartitioner partitioner = StorageService.getPartitioner();
- try (IFilter bf = FilterFactory.getFilter(1000000, 0.0001, offheap))
+ try (IFilter bf = FilterFactory.getFilter(1000000, 0.0001, offheap, oldBfHashOrder))
{
for (int i = 0; i < 100; i++)
bf.add(partitioner.decorateKey(partitioner.getTokenFactory().toByteArray(partitioner.getRandomToken())));
- try (DataOutputStreamPlus out = getOutput("utils.BloomFilter.bin"))
+ try (DataOutputStreamPlus out = getOutput(oldBfHashOrder ? "2.1" : "3.0", "utils.BloomFilter.bin"))
+ {
+ FilterFactory.serialize(bf, out);
+ }
+ }
+ }
+
+ private static void testBloomFilterWrite1000(boolean offheap, boolean oldBfHashOrder) throws IOException
+ {
+ IPartitioner partitioner = StorageService.getPartitioner();
+ try (IFilter bf = FilterFactory.getFilter(1000000, 0.0001, offheap, oldBfHashOrder))
+ {
+ for (int i = 0; i < 1000; i++)
+ bf.add(partitioner.decorateKey(Int32Type.instance.decompose(i)));
+ try (DataOutputStreamPlus out = getOutput(oldBfHashOrder ? "2.1" : "3.0", "utils.BloomFilter1000.bin"))
{
FilterFactory.serialize(bf, out);
}
@@ -47,19 +67,133 @@ public class SerializationsTest extends AbstractSerializationsTester
}
@Test
+ public void testBloomFilterRead1000() throws IOException
+ {
+ if (EXECUTE_WRITES)
+ {
+ testBloomFilterWrite1000(true, false);
+ testBloomFilterWrite1000(true, true);
+ }
+
+ IPartitioner partitioner = StorageService.getPartitioner();
+ try (DataInputStream in = getInput("3.0", "utils.BloomFilter1000.bin");
+ IFilter filter = FilterFactory.deserialize(in, true, false))
+ {
+ boolean present;
+ for (int i = 0 ; i < 1000 ; i++)
+ {
+ present = filter.isPresent(partitioner.decorateKey(Int32Type.instance.decompose(i)));
+ Assert.assertTrue(present);
+ }
+ for (int i = 1000 ; i < 2000 ; i++)
+ {
+ present = filter.isPresent(partitioner.decorateKey(Int32Type.instance.decompose(i)));
+ Assert.assertFalse(present);
+ }
+ }
+
+ try (DataInputStream in = getInput("2.1", "utils.BloomFilter1000.bin");
+ IFilter filter = FilterFactory.deserialize(in, true, true))
+ {
+ boolean present;
+ for (int i = 0 ; i < 1000 ; i++)
+ {
+ present = filter.isPresent(partitioner.decorateKey(Int32Type.instance.decompose(i)));
+ Assert.assertTrue(present);
+ }
+ for (int i = 1000 ; i < 2000 ; i++)
+ {
+ present = filter.isPresent(partitioner.decorateKey(Int32Type.instance.decompose(i)));
+ Assert.assertFalse(present);
+ }
+ }
+
+ // eh - reading version version 'ka' (2.1) with 3.0 BloomFilter
+ int falsePositive = 0;
+ int falseNegative = 0;
+ try (DataInputStream in = getInput("2.1", "utils.BloomFilter1000.bin");
+ IFilter filter = FilterFactory.deserialize(in, true, false))
+ {
+ boolean present;
+ for (int i = 0 ; i < 1000 ; i++)
+ {
+ present = filter.isPresent(partitioner.decorateKey(Int32Type.instance.decompose(i)));
+ if (!present)
+ falseNegative ++;
+ }
+ for (int i = 1000 ; i < 2000 ; i++)
+ {
+ present = filter.isPresent(partitioner.decorateKey(Int32Type.instance.decompose(i)));
+ if (present)
+ falsePositive ++;
+ }
+ }
+ Assert.assertEquals(1000, falseNegative);
+ Assert.assertEquals(0, falsePositive);
+ }
+
+ @Test
+ public void testBloomFilterTable() throws Exception
+ {
+ testBloomFilterTable("test/data/bloom-filter/ka/foo/foo-atable-ka-1-Filter.db", true);
+ testBloomFilterTable("test/data/bloom-filter/la/foo/la-1-big-Filter.db", false);
+ }
+
+ private static void testBloomFilterTable(String file, boolean oldBfHashOrder) throws Exception
+ {
+ Murmur3Partitioner partitioner = new Murmur3Partitioner();
+
+ try (DataInputStream in = new DataInputStream(new FileInputStream(new File(file)));
+ IFilter filter = FilterFactory.deserialize(in, true, oldBfHashOrder))
+ {
+ for (int i = 1; i <= 10; i++)
+ {
+ DecoratedKey decoratedKey = partitioner.decorateKey(Int32Type.instance.decompose(i));
+ boolean present = filter.isPresent(decoratedKey);
+ Assert.assertTrue(present);
+ }
+
+ int positives = 0;
+ for (int i = 11; i <= 1000010; i++)
+ {
+ DecoratedKey decoratedKey = partitioner.decorateKey(Int32Type.instance.decompose(i));
+ boolean present = filter.isPresent(decoratedKey);
+ if (present)
+ positives++;
+ }
+ double fpr = positives;
+ fpr /= 1000000;
+ Assert.assertTrue(fpr <= 0.011d);
+ }
+ }
+
+ @Test
public void testBloomFilterReadMURMUR3() throws IOException
{
if (EXECUTE_WRITES)
- testBloomFilterWrite(true);
+ testBloomFilterWrite(true, true);
+
+ try (DataInputStream in = getInput("3.0", "utils.BloomFilter.bin");
+ IFilter filter = FilterFactory.deserialize(in, true, true))
+ {
+ Assert.assertNotNull(filter);
+ }
+ }
+
+ @Test
+ public void testBloomFilterReadMURMUR3pre30() throws IOException
+ {
+ if (EXECUTE_WRITES)
+ testBloomFilterWrite(true, false);
- try (DataInputStream in = getInput("utils.BloomFilter.bin");
- IFilter filter = FilterFactory.deserialize(in, true))
+ try (DataInputStream in = getInput("2.1", "utils.BloomFilter.bin");
+ IFilter filter = FilterFactory.deserialize(in, true, false))
{
Assert.assertNotNull(filter);
}
}
- private void testEstimatedHistogramWrite() throws IOException
+ private static void testEstimatedHistogramWrite() throws IOException
{
EstimatedHistogram hist0 = new EstimatedHistogram();
EstimatedHistogram hist1 = new EstimatedHistogram(5000);