You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ja...@apache.org on 2018/01/17 22:18:05 UTC

cassandra git commit: Remove unused on-heap Bloomfilter implementation

Repository: cassandra
Updated Branches:
  refs/heads/trunk 6d324f9d7 -> 7533ff089


Remove unused on-heap Bloomfilter implementation

patch by Jay Zhuang; reviewed by jasobrown for CASSANDRA-14152


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/7533ff08
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/7533ff08
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/7533ff08

Branch: refs/heads/trunk
Commit: 7533ff089ab913262e7a540cc778d7f67ce17ce1
Parents: 6d324f9
Author: Jay Zhuang <ja...@yahoo.com>
Authored: Sat Jan 13 16:56:16 2018 -0800
Committer: Jason Brown <ja...@gmail.com>
Committed: Wed Jan 17 14:16:31 2018 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../io/sstable/format/SSTableReader.java        |   4 +-
 .../io/sstable/format/big/BigTableWriter.java   |   2 +-
 .../cassandra/utils/BloomFilterSerializer.java  |  10 +-
 .../apache/cassandra/utils/FilterFactory.java   |  17 +-
 .../cassandra/utils/obs/OffHeapBitSet.java      |   4 +-
 .../apache/cassandra/utils/obs/OpenBitSet.java  | 494 -------------------
 .../apache/cassandra/utils/LongBitSetTest.java  | 134 -----
 .../cassandra/utils/LongBloomFilterTest.java    |  10 +-
 .../org/apache/cassandra/utils/BitSetTest.java  | 133 -----
 .../apache/cassandra/utils/BloomFilterTest.java |  37 +-
 .../cassandra/utils/SerializationsTest.java     |  24 +-
 .../cassandra/utils/obs/OffHeapBitSetTest.java  | 134 +++++
 13 files changed, 183 insertions(+), 821 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7533ff08/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0cda459..0d6d486 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Remove unused on-heap BloomFilter implementation (CASSANDRA-14152)
  * Delete temp test files on exit (CASSANDRA-14153)
  * Make PartitionUpdate and Mutation immutable (CASSANDRA-13867)
  * Fix CommitLogReplayer exception for CDC data (CASSANDRA-14066)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7533ff08/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 5ab3964..6722e1b 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -741,7 +741,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
     {
         try (DataInputStream stream = new DataInputStream(new BufferedInputStream(Files.newInputStream(Paths.get(descriptor.filenameFor(Component.FILTER))))))
         {
-            bf = FilterFactory.deserialize(stream, true);
+            bf = FilterFactory.deserialize(stream);
         }
     }
 
@@ -827,7 +827,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().params.bloomFilterFpChance, true);
+                bf = FilterFactory.getFilter(estimatedKeys, metadata().params.bloomFilterFpChance);
 
             try (IndexSummaryBuilder summaryBuilder = summaryLoaded ? null : new IndexSummaryBuilder(estimatedKeys, metadata().params.minIndexInterval, samplingLevel))
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7533ff08/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 4ae4331..04c7bbf 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
@@ -439,7 +439,7 @@ public class BigTableWriter extends SSTableWriter
             builder = new FileHandle.Builder(descriptor.filenameFor(Component.PRIMARY_INDEX)).mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap);
             chunkCache.ifPresent(builder::withChunkCache);
             summary = new IndexSummaryBuilder(keyCount, metadata().params.minIndexInterval, Downsampling.BASE_SAMPLING_LEVEL);
-            bf = FilterFactory.getFilter(keyCount, metadata().params.bloomFilterFpChance, true);
+            bf = FilterFactory.getFilter(keyCount, metadata().params.bloomFilterFpChance);
             // register listeners to be alerted when the data files are flushed
             indexFile.setPostFlushListener(() -> summary.markIndexSynced(indexFile.getLastFlushOffset()));
             dataFile.setPostFlushListener(() -> summary.markDataSynced(dataFile.getLastFlushOffset()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7533ff08/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 17ab123..9e9d15a 100644
--- a/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
+++ b/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
@@ -24,7 +24,6 @@ import org.apache.cassandra.db.TypeSizes;
 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;
 
 final class BloomFilterSerializer
 {
@@ -38,16 +37,11 @@ final class BloomFilterSerializer
         bf.bitset.serialize(out);
     }
 
-    public static BloomFilter deserialize(DataInput in) throws IOException
-    {
-        return deserialize(in, false);
-    }
-
     @SuppressWarnings("resource")
-    public static BloomFilter deserialize(DataInput in, boolean offheap) throws IOException
+    public static BloomFilter deserialize(DataInput in) throws IOException
     {
         int hashes = in.readInt();
-        IBitSet bs = offheap ? OffHeapBitSet.deserialize(in) : OpenBitSet.deserialize(in);
+        IBitSet bs = OffHeapBitSet.deserialize(in);
 
         return new BloomFilter(hashes, bs);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7533ff08/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 f79f720..9479452 100644
--- a/src/java/org/apache/cassandra/utils/FilterFactory.java
+++ b/src/java/org/apache/cassandra/utils/FilterFactory.java
@@ -26,7 +26,6 @@ import org.slf4j.LoggerFactory;
 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;
 
 public class FilterFactory
 {
@@ -40,16 +39,16 @@ public class FilterFactory
         BloomFilterSerializer.serialize((BloomFilter) bf, output);
     }
 
-    public static IFilter deserialize(DataInput input, boolean offheap) throws IOException
+    public static IFilter deserialize(DataInput input) throws IOException
     {
-        return BloomFilterSerializer.deserialize(input, offheap);
+        return BloomFilterSerializer.deserialize(input);
     }
 
     /**
      * @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)
     {
         int maxBucketsPerElement = Math.max(1, BloomCalculations.maxBucketsPerElement(numElements));
         int bucketsPerElement = Math.min(targetBucketsPerElem, maxBucketsPerElement);
@@ -58,7 +57,7 @@ public class FilterFactory
             logger.warn("Cannot provide an optimal BloomFilter for {} elements ({}/{} 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);
     }
 
     /**
@@ -68,21 +67,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)
     {
         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);
     }
 
     @SuppressWarnings("resource")
-    private static IFilter createFilter(int hash, long numElements, int bucketsPer, boolean offheap)
+    private static IFilter createFilter(int hash, long numElements, int bucketsPer)
     {
         long numBits = (numElements * bucketsPer) + BITSET_EXCESS;
-        IBitSet bitset = offheap ? new OffHeapBitSet(numBits) : new OpenBitSet(numBits);
+        IBitSet bitset = new OffHeapBitSet(numBits);
         return new BloomFilter(hash, bitset);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7533ff08/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java b/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java
index 8593a11..d2c15ca 100644
--- a/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java
+++ b/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java
@@ -35,8 +35,8 @@ public class OffHeapBitSet implements IBitSet
 
     public OffHeapBitSet(long numBits)
     {
-        // OpenBitSet.bits2words calculation is there for backward compatibility.
-        long wordCount = OpenBitSet.bits2words(numBits);
+        /** returns the number of 64 bit words it would take to hold numBits */
+        long wordCount = (((numBits - 1) >>> 6) + 1);
         if (wordCount > Integer.MAX_VALUE)
             throw new UnsupportedOperationException("Bloom filter size is > 16GB, reduce the bloom_filter_fp_chance");
         try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7533ff08/src/java/org/apache/cassandra/utils/obs/OpenBitSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/obs/OpenBitSet.java b/src/java/org/apache/cassandra/utils/obs/OpenBitSet.java
deleted file mode 100644
index a21729a..0000000
--- a/src/java/org/apache/cassandra/utils/obs/OpenBitSet.java
+++ /dev/null
@@ -1,494 +0,0 @@
-/*
- * 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.cassandra.utils.obs;
-
-import java.util.Arrays;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.utils.concurrent.Ref;
-
-/**
- * <p>
- * An "open" BitSet implementation that allows direct access to the arrays of words
- * storing the bits.  Derived from Lucene's OpenBitSet, but with a paged backing array
- * (see bits delaration, below).
- * </p>
- * <p>
- * Unlike java.util.bitset, the fact that bits are packed into an array of longs
- * is part of the interface.  This allows efficient implementation of other algorithms
- * by someone other than the author.  It also allows one to efficiently implement
- * alternate serialization or interchange formats.
- * </p>
- * <p>
- * <code>OpenBitSet</code> is faster than <code>java.util.BitSet</code> in most operations
- * and *much* faster at calculating cardinality of sets and results of set operations.
- * It can also handle sets of larger cardinality (up to 64 * 2**32-1)
- * </p>
- * <p>
- * The goals of <code>OpenBitSet</code> are the fastest implementation possible, and
- * maximum code reuse.  Extra safety and encapsulation
- * may always be built on top, but if that's built in, the cost can never be removed (and
- * hence people re-implement their own version in order to get better performance).
- * If you want a "safe", totally encapsulated (and slower and limited) BitSet
- * class, use <code>java.util.BitSet</code>.
- * </p>
- */
-
-public class OpenBitSet implements IBitSet
-{
-  /**
-   * We break the bitset up into multiple arrays to avoid promotion failure caused by attempting to allocate
-   * large, contiguous arrays (CASSANDRA-2466).  All sub-arrays but the last are uniformly PAGE_SIZE words;
-   * to avoid waste in small bloom filters (of which Cassandra has many: one per row) the last sub-array
-   * is sized to exactly the remaining number of words required to achieve the desired set size (CASSANDRA-3618).
-   */
-  private final long[][] bits;
-  private int wlen; // number of words (elements) used in the array
-  private final int pageCount;
-  private static final int PAGE_SIZE = 4096;
-
-  /**
-   * Constructs an OpenBitSet large enough to hold numBits.
-   * @param numBits
-   */
-  public OpenBitSet(long numBits)
-  {
-      wlen = (int) bits2words(numBits);
-      int lastPageSize = wlen % PAGE_SIZE;
-      int fullPageCount = wlen / PAGE_SIZE;
-      pageCount = fullPageCount + (lastPageSize == 0 ? 0 : 1);
-
-      bits = new long[pageCount][];
-
-      for (int i = 0; i < fullPageCount; ++i)
-          bits[i] = new long[PAGE_SIZE];
-
-      if (lastPageSize != 0)
-          bits[bits.length - 1] = new long[lastPageSize];
-  }
-
-  public OpenBitSet()
-  {
-    this(64);
-  }
-
-  /**
-   * @return the pageSize
-   */
-  public int getPageSize()
-  {
-      return PAGE_SIZE;
-  }
-
-  public int getPageCount()
-  {
-      return pageCount;
-  }
-
-  public long[] getPage(int pageIdx)
-  {
-      return bits[pageIdx];
-  }
-
-  /** Returns the current capacity in bits (1 greater than the index of the last bit) */
-  public long capacity() { return ((long)wlen) << 6; }
-
-  @Override
-  public long offHeapSize()
-  {
-      return 0;
-  }
-
-    public void addTo(Ref.IdentityCollection identities)
-    {
-    }
-
-    /**
-  * Returns the current capacity of this set.  Included for
-  * compatibility.  This is *not* equal to {@link #cardinality}
-  */
-  public long size()
-  {
-      return capacity();
-  }
-
-  // @Override -- not until Java 1.6
-  public long length()
-  {
-    return capacity();
-  }
-
-  /** Returns true if there are no set bits */
-  public boolean isEmpty() { return cardinality()==0; }
-
-
-  /** Expert: gets the number of longs in the array that are in use */
-  public int getNumWords() { return wlen; }
-
-
-  /**
-   * Returns true or false for the specified bit index.
-   * The index should be less than the OpenBitSet size
-   */
-  public boolean get(int index)
-  {
-    int i = index >> 6;               // div 64
-    // signed shift will keep a negative index and force an
-    // array-index-out-of-bounds-exception, removing the need for an explicit check.
-    int bit = index & 0x3f;           // mod 64
-    long bitmask = 1L << bit;
-    // TODO perfectionist one can implement this using bit operations
-    return (bits[i / PAGE_SIZE][i % PAGE_SIZE ] & bitmask) != 0;
-  }
-
-  /**
-   * Returns true or false for the specified bit index.
-   * The index should be less than the OpenBitSet size.
-   */
-  public boolean get(long index)
-  {
-    int i = (int)(index >> 6);               // div 64
-    int bit = (int)index & 0x3f;           // mod 64
-    long bitmask = 1L << bit;
-    // TODO perfectionist one can implement this using bit operations
-    return (bits[i / PAGE_SIZE][i % PAGE_SIZE ] & bitmask) != 0;
-  }
-
-  /**
-   * Sets the bit at the specified index.
-   * The index should be less than the OpenBitSet size.
-   */
-  public void set(long index)
-  {
-    int wordNum = (int)(index >> 6);
-    int bit = (int)index & 0x3f;
-    long bitmask = 1L << bit;
-    bits[ wordNum / PAGE_SIZE ][ wordNum % PAGE_SIZE ] |= bitmask;
-  }
-
-  /**
-   * Sets the bit at the specified index.
-   * The index should be less than the OpenBitSet size.
-   */
-  public void set(int index)
-  {
-    int wordNum = index >> 6;      // div 64
-    int bit = index & 0x3f;     // mod 64
-    long bitmask = 1L << bit;
-    bits[ wordNum / PAGE_SIZE ][ wordNum % PAGE_SIZE ] |= bitmask;
-  }
-
-  /**
-   * clears a bit.
-   * The index should be less than the OpenBitSet size.
-   */
-  public void clear(int index)
-  {
-    int wordNum = index >> 6;
-    int bit = index & 0x03f;
-    long bitmask = 1L << bit;
-    bits[wordNum / PAGE_SIZE][wordNum % PAGE_SIZE] &= ~bitmask;
-    // hmmm, it takes one more instruction to clear than it does to set... any
-    // way to work around this?  If there were only 63 bits per word, we could
-    // use a right shift of 10111111...111 in binary to position the 0 in the
-    // correct place (using sign extension).
-    // Could also use Long.rotateRight() or rotateLeft() *if* they were converted
-    // by the JVM into a native instruction.
-    // bits[word] &= Long.rotateLeft(0xfffffffe,bit);
-  }
-
-  /**
-   * clears a bit.
-   * The index should be less than the OpenBitSet size.
-   */
-  public void clear(long index)
-  {
-    int wordNum = (int)(index >> 6); // div 64
-    int bit = (int)index & 0x3f;     // mod 64
-    long bitmask = 1L << bit;
-    bits[wordNum / PAGE_SIZE][wordNum % PAGE_SIZE] &= ~bitmask;
-  }
-
-  /**
-   * Clears a range of bits.  Clearing past the end does not change the size of the set.
-   *
-   * @param startIndex lower index
-   * @param endIndex one-past the last bit to clear
-   */
-  public void clear(int startIndex, int endIndex)
-  {
-    if (endIndex <= startIndex) return;
-
-    int startWord = (startIndex>>6);
-    if (startWord >= wlen) return;
-
-    // since endIndex is one past the end, this is index of the last
-    // word to be changed.
-    int endWord   = ((endIndex-1)>>6);
-
-    long startmask = -1L << startIndex;
-    long endmask = -1L >>> -endIndex;  // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
-
-    // invert masks since we are clearing
-    startmask = ~startmask;
-    endmask = ~endmask;
-
-    if (startWord == endWord)
-    {
-      bits[startWord / PAGE_SIZE][startWord % PAGE_SIZE] &= (startmask | endmask);
-      return;
-    }
-
-
-    bits[startWord / PAGE_SIZE][startWord % PAGE_SIZE]  &= startmask;
-
-    int middle = Math.min(wlen, endWord);
-    if (startWord / PAGE_SIZE == middle / PAGE_SIZE)
-    {
-        Arrays.fill(bits[startWord/PAGE_SIZE], (startWord+1) % PAGE_SIZE, middle % PAGE_SIZE, 0L);
-    } else
-    {
-        while (++startWord<middle)
-            bits[startWord / PAGE_SIZE][startWord % PAGE_SIZE] = 0L;
-    }
-    if (endWord < wlen)
-    {
-      bits[endWord / PAGE_SIZE][endWord % PAGE_SIZE] &= endmask;
-    }
-  }
-
-
-  /** Clears a range of bits.  Clearing past the end does not change the size of the set.
-   *
-   * @param startIndex lower index
-   * @param endIndex one-past the last bit to clear
-   */
-  public void clear(long startIndex, long endIndex)
-  {
-    if (endIndex <= startIndex) return;
-
-    int startWord = (int)(startIndex>>6);
-    if (startWord >= wlen) return;
-
-    // since endIndex is one past the end, this is index of the last
-    // word to be changed.
-    int endWord   = (int)((endIndex-1)>>6);
-
-    long startmask = -1L << startIndex;
-    long endmask = -1L >>> -endIndex;  // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
-
-    // invert masks since we are clearing
-    startmask = ~startmask;
-    endmask = ~endmask;
-
-    if (startWord == endWord)
-{
-        bits[startWord / PAGE_SIZE][startWord % PAGE_SIZE] &= (startmask | endmask);
-        return;
-    }
-
-    bits[startWord / PAGE_SIZE][startWord % PAGE_SIZE]  &= startmask;
-
-    int middle = Math.min(wlen, endWord);
-    if (startWord / PAGE_SIZE == middle / PAGE_SIZE)
-    {
-        Arrays.fill(bits[startWord/PAGE_SIZE], (startWord+1) % PAGE_SIZE, middle % PAGE_SIZE, 0L);
-    } else
-    {
-        while (++startWord<middle)
-            bits[startWord / PAGE_SIZE][startWord % PAGE_SIZE] = 0L;
-    }
-    if (endWord < wlen)
-    {
-        bits[endWord / PAGE_SIZE][endWord % PAGE_SIZE] &= endmask;
-    }
-  }
-
-  /** @return the number of set bits */
-  public long cardinality()
-  {
-    long bitCount = 0L;
-    for (int i=getPageCount();i-->0;)
-        bitCount+=BitUtil.pop_array(bits[i],0,wlen);
-
-    return bitCount;
-  }
-
-  /** this = this AND other */
-  public void intersect(OpenBitSet other)
-  {
-    int newLen= Math.min(this.wlen,other.wlen);
-    long[][] thisArr = this.bits;
-    long[][] otherArr = other.bits;
-    int thisPageSize = PAGE_SIZE;
-    int otherPageSize = OpenBitSet.PAGE_SIZE;
-    // testing against zero can be more efficient
-    int pos=newLen;
-    while(--pos>=0)
-    {
-      thisArr[pos / thisPageSize][ pos % thisPageSize] &= otherArr[pos / otherPageSize][pos % otherPageSize];
-    }
-
-    if (this.wlen > newLen)
-    {
-      // fill zeros from the new shorter length to the old length
-      for (pos=wlen;pos-->newLen;)
-          thisArr[pos / thisPageSize][ pos % thisPageSize] =0;
-    }
-    this.wlen = newLen;
-  }
-
-  // some BitSet compatability methods
-
-  //** see {@link intersect} */
-  public void and(OpenBitSet other)
-  {
-    intersect(other);
-  }
-
-  /** Lowers numWords, the number of words in use,
-   * by checking for trailing zero words.
-   */
-  public void trimTrailingZeros()
-  {
-    int idx = wlen-1;
-    while (idx>=0 && bits[idx / PAGE_SIZE][idx % PAGE_SIZE]==0) idx--;
-    wlen = idx+1;
-  }
-
-  /** returns the number of 64 bit words it would take to hold numBits */
-  public static long bits2words(long numBits)
-  {
-   return (((numBits-1)>>>6)+1);
-  }
-
-  /** returns true if both sets have the same bits set */
-  @Override
-  public boolean equals(Object o)
-  {
-    if (this == o) return true;
-    if (!(o instanceof OpenBitSet)) return false;
-    OpenBitSet a;
-    OpenBitSet b = (OpenBitSet)o;
-    // make a the larger set.
-    if (b.wlen > this.wlen)
-    {
-      a = b; b=this;
-    }
-    else
-    {
-      a=this;
-    }
-
-    int aPageSize = OpenBitSet.PAGE_SIZE;
-    int bPageSize = OpenBitSet.PAGE_SIZE;
-
-    // check for any set bits out of the range of b
-    for (int i=a.wlen-1; i>=b.wlen; i--)
-    {
-      if (a.bits[i/aPageSize][i % aPageSize]!=0) return false;
-    }
-
-    for (int i=b.wlen-1; i>=0; i--)
-    {
-      if (a.bits[i/aPageSize][i % aPageSize] != b.bits[i/bPageSize][i % bPageSize]) return false;
-    }
-
-    return true;
-  }
-
-
-  @Override
-  public int hashCode()
-  {
-    // Start with a zero hash and use a mix that results in zero if the input is zero.
-    // This effectively truncates trailing zeros without an explicit check.
-    long h = 0;
-    for (int i = wlen; --i>=0;)
-    {
-      h ^= bits[i / PAGE_SIZE][i % PAGE_SIZE];
-      h = (h << 1) | (h >>> 63); // rotate left
-    }
-    // fold leftmost bits into right and add a constant to prevent
-    // empty sets from returning 0, which is too common.
-    return (int)((h>>32) ^ h) + 0x98761234;
-  }
-
-  public void close()
-  {
-    // noop, let GC do the cleanup.
-  }
-
-  public void serialize(DataOutput out) throws IOException
-  {
-    int bitLength = getNumWords();
-    int pageSize = getPageSize();
-    int pageCount = getPageCount();
-
-    out.writeInt(bitLength);
-    for (int p = 0; p < pageCount; p++)
-    {
-      long[] bits = getPage(p);
-      for (int i = 0; i < pageSize && bitLength-- > 0; i++)
-      {
-        out.writeLong(bits[i]);
-      }
-    }
-}
-
-  public long serializedSize()
-  {
-    int bitLength = getNumWords();
-    int pageSize = getPageSize();
-    int pageCount = getPageCount();
-
-    long size = TypeSizes.sizeof(bitLength); // length
-    for (int p = 0; p < pageCount; p++)
-    {
-      long[] bits = getPage(p);
-      for (int i = 0; i < pageSize && bitLength-- > 0; i++)
-        size += TypeSizes.sizeof(bits[i]); // bucket
-    }
-    return size;
-  }
-
-  public void clear()
-  {
-    clear(0, capacity());
-  }
-
-  public static OpenBitSet deserialize(DataInput in) throws IOException
-  {
-    long bitLength = in.readInt();
-
-    OpenBitSet bs = new OpenBitSet(bitLength << 6);
-    int pageSize = bs.getPageSize();
-    int pageCount = bs.getPageCount();
-
-    for (int p = 0; p < pageCount; p++)
-    {
-      long[] bits = bs.getPage(p);
-      for (int i = 0; i < pageSize && bitLength-- > 0; i++)
-        bits[i] = in.readLong();
-    }
-    return bs;
-  }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7533ff08/test/long/org/apache/cassandra/utils/LongBitSetTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/utils/LongBitSetTest.java b/test/long/org/apache/cassandra/utils/LongBitSetTest.java
deleted file mode 100644
index f20a4f8..0000000
--- a/test/long/org/apache/cassandra/utils/LongBitSetTest.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/*
- * 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.cassandra.utils;
-
-import java.util.Random;
-import java.util.concurrent.TimeUnit;
-
-import org.junit.Assert;
-
-import org.apache.cassandra.utils.obs.OffHeapBitSet;
-import org.apache.cassandra.utils.obs.OpenBitSet;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class LongBitSetTest
-{
-    private static final Logger logger = LoggerFactory.getLogger(LongBitSetTest.class);
-    private static final Random random = new Random();
-
-    public void populateRandom(OffHeapBitSet offbs, OpenBitSet obs, long index)
-    {
-        if (random.nextBoolean())
-        {
-            offbs.set(index);
-            obs.set(index);
-        }
-    }
-
-    public void compare(OffHeapBitSet offbs, OpenBitSet obs, long index)
-    {
-        if (offbs.get(index) != obs.get(index))
-            throw new RuntimeException();
-        Assert.assertEquals(offbs.get(index), obs.get(index));
-    }
-
-    @Test
-    public void testBitSetOperations()
-    {
-        long size_to_test = Integer.MAX_VALUE / 40;
-        long size_and_excess = size_to_test + 20;
-        OffHeapBitSet offbs = new OffHeapBitSet(size_and_excess);
-        OpenBitSet obs = new OpenBitSet(size_and_excess);
-        for (long i = 0; i < size_to_test; i++)
-            populateRandom(offbs, obs, i);
-
-        for (long i = 0; i < size_to_test; i++)
-            compare(offbs, obs, i);
-    }
-
-    @Test
-    public void timeit()
-    {
-        long size_to_test = Integer.MAX_VALUE / 10; // about 214 million
-        long size_and_excess = size_to_test + 20;
-
-        OpenBitSet obs = new OpenBitSet(size_and_excess);
-        OffHeapBitSet offbs = new OffHeapBitSet(size_and_excess);
-        logger.info("||Open BS set's|Open BS get's|Open BS clear's|Offheap BS set's|Offheap BS get's|Offheap BS clear's|");
-        // System.out.println("||Open BS set's|Open BS get's|Open BS clear's|Offheap BS set's|Offheap BS get's|Offheap BS clear's|");
-        loopOnce(obs, offbs, size_to_test);
-    }
-
-    public void loopOnce(OpenBitSet obs, OffHeapBitSet offbs, long size_to_test)
-    {
-        StringBuffer buffer = new StringBuffer();
-        // start off fresh.
-        System.gc();
-        long start = System.nanoTime();
-        for (long i = 0; i < size_to_test; i++)
-            obs.set(i);
-        buffer.append("||").append(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
-
-        start = System.nanoTime();
-        for (long i = 0; i < size_to_test; i++)
-            obs.get(i);
-        buffer.append("|").append(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
-
-        start = System.nanoTime();
-        for (long i = 0; i < size_to_test; i++)
-            obs.clear(i);
-        buffer.append("|").append(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
-
-        System.gc();
-        start = System.nanoTime();
-        for (long i = 0; i < size_to_test; i++)
-            offbs.set(i);
-        buffer.append("|").append(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
-
-        start = System.nanoTime();
-        for (long i = 0; i < size_to_test; i++)
-            offbs.get(i);
-        buffer.append("|").append(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
-
-        start = System.nanoTime();
-        for (long i = 0; i < size_to_test; i++)
-            offbs.clear(i);
-        buffer.append("|").append(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start)).append("|");
-        logger.info(buffer.toString());
-        // System.out.println(buffer.toString());
-    }
-
-    /**
-     * Just to make sure JIT doesn't come on our way
-     */
-    @Test
-    // @Ignore
-    public void loopIt()
-    {
-        long size_to_test = Integer.MAX_VALUE / 10; // about 214 million
-        long size_and_excess = size_to_test + 20;
-
-        OpenBitSet obs = new OpenBitSet(size_and_excess);
-        OffHeapBitSet offbs = new OffHeapBitSet(size_and_excess);
-        for (int i = 0; i < 10; i++)
-            // 10 times to do approx 2B keys each.
-            loopOnce(obs, offbs, size_to_test);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7533ff08/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 10dd5a6..d998e4d 100644
--- a/test/long/org/apache/cassandra/utils/LongBloomFilterTest.java
+++ b/test/long/org/apache/cassandra/utils/LongBloomFilterTest.java
@@ -39,7 +39,7 @@ public class LongBloomFilterTest
     public void testBigInt()
     {
         int size = 10 * 1000 * 1000;
-        IFilter bf = getFilter(size, FilterTestHelper.spec.bucketsPerElement, false);
+        IFilter bf = getFilter(size, FilterTestHelper.spec.bucketsPerElement);
         double fp = testFalsePositives(bf,
                                        new KeyGenerator.IntGenerator(size),
                                        new KeyGenerator.IntGenerator(size, size * 2));
@@ -50,7 +50,7 @@ public class LongBloomFilterTest
     public void testBigRandom()
     {
         int size = 10 * 1000 * 1000;
-        IFilter bf = getFilter(size, FilterTestHelper.spec.bucketsPerElement, false);
+        IFilter bf = getFilter(size, FilterTestHelper.spec.bucketsPerElement);
         double fp = testFalsePositives(bf,
                                        new KeyGenerator.RandomStringGenerator(new Random().nextInt(), size),
                                        new KeyGenerator.RandomStringGenerator(new Random().nextInt(), size));
@@ -64,7 +64,7 @@ public class LongBloomFilterTest
     public void testConstrained()
     {
         int size = 10 * 1000 * 1000;
-        try (IFilter bf = getFilter(size, 0.01, false))
+        try (IFilter bf = getFilter(size, 0.01))
         {
             double fp = testFalsePositives(bf,
                                            new KeyGenerator.IntGenerator(size),
@@ -77,7 +77,7 @@ public class LongBloomFilterTest
     {
         for (long bits : staticBits)
         {
-            try (IFilter bf = getFilter(elements, targetFp, false);)
+            try (IFilter bf = getFilter(elements, targetFp))
             {
                 SequentialHashGenerator gen = new SequentialHashGenerator(staticBitCount, bits);
                 long[] hash = new long[2];
@@ -160,7 +160,7 @@ public class LongBloomFilterTest
     public void timeit()
     {
         int size = 300 * FilterTestHelper.ELEMENTS;
-        IFilter bf = getFilter(size, FilterTestHelper.spec.bucketsPerElement, false);
+        IFilter bf = getFilter(size, FilterTestHelper.spec.bucketsPerElement);
         double sumfp = 0;
         for (int i = 0; i < 10; i++)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7533ff08/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
deleted file mode 100644
index 4dab17e..0000000
--- a/test/unit/org/apache/cassandra/utils/BitSetTest.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * 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.cassandra.utils;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.util.List;
-import java.util.Random;
-
-import com.google.common.collect.Lists;
-import org.junit.Assert;
-import org.junit.Test;
-
-import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.utils.IFilter.FilterKey;
-import org.apache.cassandra.utils.KeyGenerator.RandomStringGenerator;
-import org.apache.cassandra.utils.obs.IBitSet;
-import org.apache.cassandra.utils.obs.OffHeapBitSet;
-import org.apache.cassandra.utils.obs.OpenBitSet;
-
-import static org.junit.Assert.assertEquals;
-
-public class BitSetTest
-{
-    /**
-     * Test bitsets in a "real-world" environment, i.e., bloom filters
-     */
-    @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);
-
-        RandomStringGenerator gen1 = new KeyGenerator.RandomStringGenerator(new Random().nextInt(), FilterTestHelper.ELEMENTS);
-
-        // make sure both bitsets are empty.
-        compare(bf2.bitset, bf3.bitset);
-
-        while (gen1.hasNext())
-        {
-            FilterKey key = FilterTestHelper.wrap(gen1.next());
-            bf2.add(key);
-            bf3.add(key);
-        }
-
-        compare(bf2.bitset, bf3.bitset);
-    }
-
-    private static final Random random = new Random();
-
-    /**
-     * Test serialization and de-serialization in-memory
-     */
-    @Test
-    public void testOffHeapSerialization() throws IOException
-    {
-        try (OffHeapBitSet bs = new OffHeapBitSet(100000))
-        {
-            populateAndReserialize(bs);
-        }
-    }
-
-    @Test
-    public void testOffHeapCompatibility() throws IOException
-    {
-        try (OpenBitSet bs = new OpenBitSet(100000))
-        {
-            populateAndReserialize(bs);
-        }
-    }
-
-    private static void populateAndReserialize(IBitSet bs) throws IOException
-    {
-        for (long i = 0; i < bs.capacity(); i++)
-            if (random.nextBoolean())
-                bs.set(i);
-
-        DataOutputBuffer out = new DataOutputBuffer();
-        bs.serialize(out);
-        DataInputStream in = new DataInputStream(new ByteArrayInputStream(out.getData()));
-        try (OffHeapBitSet newbs = OffHeapBitSet.deserialize(in))
-        {
-            compare(bs, newbs);
-        }
-    }
-
-    static void compare(IBitSet bs, IBitSet newbs)
-    {
-        assertEquals(bs.capacity(), newbs.capacity());
-        for (long i = 0; i < bs.capacity(); i++)
-            Assert.assertEquals(bs.get(i), newbs.get(i));
-    }
-
-    @Test
-    public void testBitClear()
-    {
-        int size = Integer.MAX_VALUE / 4000;
-        try (OffHeapBitSet bitset = new OffHeapBitSet(size))
-        {
-            List<Integer> randomBits = Lists.newArrayList();
-            for (int i = 0; i < 10; i++)
-                randomBits.add(random.nextInt(size));
-    
-            for (long randomBit : randomBits)
-                bitset.set(randomBit);
-    
-            for (long randomBit : randomBits)
-                Assert.assertEquals(true, bitset.get(randomBit));
-    
-            for (long randomBit : randomBits)
-                bitset.clear(randomBit);
-    
-            for (long randomBit : randomBits)
-                Assert.assertEquals(false, bitset.get(randomBit));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7533ff08/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 337e387..581248c 100644
--- a/test/unit/org/apache/cassandra/utils/BloomFilterTest.java
+++ b/test/unit/org/apache/cassandra/utils/BloomFilterTest.java
@@ -35,6 +35,9 @@ import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.IFilter.FilterKey;
 import org.apache.cassandra.utils.KeyGenerator.RandomStringGenerator;
+import org.apache.cassandra.utils.obs.IBitSet;
+
+import static org.junit.Assert.assertEquals;
 
 public class BloomFilterTest
 {
@@ -52,18 +55,24 @@ public class BloomFilterTest
         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));
 
         assert f2.isPresent(FilterTestHelper.bytes("a"));
         assert !f2.isPresent(FilterTestHelper.bytes("b"));
         return f2;
     }
 
+    static void compare(IBitSet bs, IBitSet newbs)
+    {
+        assertEquals(bs.capacity(), newbs.capacity());
+        for (long i = 0; i < bs.capacity(); i++)
+            assertEquals(bs.get(i), newbs.get(i));
+    }
 
     @Before
     public void setup()
     {
-        bfInvHashes = FilterFactory.getFilter(10000L, FilterTestHelper.MAX_FAILURE_RATE, true);
+        bfInvHashes = FilterFactory.getFilter(10000L, FilterTestHelper.MAX_FAILURE_RATE);
     }
 
     @After
@@ -112,7 +121,7 @@ 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);
         int skipEven = KeyGenerator.WordGenerator.WORDS % 2 == 0 ? 0 : 2;
         FilterTestHelper.testFalsePositives(bf2,
                                             new KeyGenerator.WordGenerator(skipEven, 2),
@@ -142,7 +151,7 @@ public class BloomFilterTest
         {
             hashes.clear();
             FilterKey buf = FilterTestHelper.wrap(keys.next());
-            BloomFilter bf = (BloomFilter) FilterFactory.getFilter(10, 1, false);
+            BloomFilter bf = (BloomFilter) FilterFactory.getFilter(10, 1);
             for (long hashIndex : bf.getHashBuckets(buf, MAX_HASH_COUNT, 1024 * 1024))
             {
                 hashes.add(hashIndex);
@@ -157,21 +166,21 @@ public class BloomFilterTest
     public void testOffHeapException()
     {
         long numKeys = ((long)Integer.MAX_VALUE) * 64L + 1L; // approx 128 Billion
-        FilterFactory.getFilter(numKeys, 0.01d, true).close();
+        FilterFactory.getFilter(numKeys, 0.01d).close();
     }
 
     @Test
     public void compareCachedKey()
     {
-        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);
+             BloomFilter bf2 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE);
+             BloomFilter bf3 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE))
         {
             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);
+            compare(bf1.bitset, bf2.bitset);
+            compare(bf1.bitset, bf3.bitset);
 
             while (gen1.hasNext())
             {
@@ -182,8 +191,8 @@ public class BloomFilterTest
                 bf3.add(cached);
             }
 
-            BitSetTest.compare(bf1.bitset, bf2.bitset);
-            BitSetTest.compare(bf1.bitset, bf3.bitset);
+            compare(bf1.bitset, bf2.bitset);
+            compare(bf1.bitset, bf3.bitset);
         }
     }
 
@@ -194,7 +203,7 @@ public class BloomFilterTest
         ByteBuffer test = ByteBuffer.wrap(new byte[] {0, 1});
 
         File file = FileUtils.createDeletableTempFile("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);
         filter.add(FilterTestHelper.wrap(test));
         DataOutputStreamPlus out = new BufferedDataOutputStreamPlus(new FileOutputStream(file));
         FilterFactory.serialize(filter, out);
@@ -203,7 +212,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);
         Assert.assertTrue(filter2.isPresent(FilterTestHelper.wrap(test)));
         FileUtils.closeQuietly(in);
         filter2.close();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7533ff08/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 7973964..8da4a92 100644
--- a/test/unit/org/apache/cassandra/utils/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/utils/SerializationsTest.java
@@ -46,23 +46,9 @@ public class SerializationsTest extends AbstractSerializationsTester
         DatabaseDescriptor.daemonInitialization();
     }
 
-    private static void testBloomFilterWrite(boolean offheap) throws IOException
+    private static void testBloomFilterWrite1000() throws IOException
     {
-        IPartitioner partitioner = Util.testPartitioner();
-        try (IFilter bf = FilterFactory.getFilter(1000000, 0.0001, offheap))
-        {
-            for (int i = 0; i < 100; i++)
-                bf.add(partitioner.decorateKey(partitioner.getTokenFactory().toByteArray(partitioner.getRandomToken())));
-            try (DataOutputStreamPlus out = getOutput("3.0", "utils.BloomFilter.bin"))
-            {
-                FilterFactory.serialize(bf, out);
-            }
-        }
-    }
-
-    private static void testBloomFilterWrite1000(boolean offheap) throws IOException
-    {
-        try (IFilter bf = FilterFactory.getFilter(1000000, 0.0001, offheap))
+        try (IFilter bf = FilterFactory.getFilter(1000000, 0.0001))
         {
             for (int i = 0; i < 1000; i++)
                 bf.add(Util.dk(Int32Type.instance.decompose(i)));
@@ -77,10 +63,10 @@ public class SerializationsTest extends AbstractSerializationsTester
     public void testBloomFilterRead1000() throws IOException
     {
         if (EXECUTE_WRITES)
-            testBloomFilterWrite1000(true);
+            testBloomFilterWrite1000();
 
         try (DataInputStream in = getInput("3.0", "utils.BloomFilter1000.bin");
-             IFilter filter = FilterFactory.deserialize(in, true))
+             IFilter filter = FilterFactory.deserialize(in))
         {
             boolean present;
             for (int i = 0 ; i < 1000 ; i++)
@@ -107,7 +93,7 @@ public class SerializationsTest extends AbstractSerializationsTester
         Murmur3Partitioner partitioner = new Murmur3Partitioner();
 
         try (DataInputStream in = new DataInputStream(new FileInputStream(new File(file)));
-             IFilter filter = FilterFactory.deserialize(in, true))
+             IFilter filter = FilterFactory.deserialize(in))
         {
             for (int i = 1; i <= 10; i++)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7533ff08/test/unit/org/apache/cassandra/utils/obs/OffHeapBitSetTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/obs/OffHeapBitSetTest.java b/test/unit/org/apache/cassandra/utils/obs/OffHeapBitSetTest.java
new file mode 100644
index 0000000..f0325da
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/obs/OffHeapBitSetTest.java
@@ -0,0 +1,134 @@
+/*
+ * 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.cassandra.utils.obs;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+import com.google.common.collect.Lists;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import org.apache.cassandra.io.util.DataOutputBuffer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class OffHeapBitSetTest
+{
+    private static final Random random = new Random();
+
+    static void compare(IBitSet bs, IBitSet newbs)
+    {
+        assertEquals(bs.capacity(), newbs.capacity());
+        for (long i = 0; i < bs.capacity(); i++)
+            Assert.assertEquals(bs.get(i), newbs.get(i));
+    }
+
+    @Test
+    public void testOffHeapSerialization() throws IOException
+    {
+        try (OffHeapBitSet bs = new OffHeapBitSet(100000))
+        {
+            for (long i = 0; i < bs.capacity(); i++)
+                if (random.nextBoolean())
+                    bs.set(i);
+
+            DataOutputBuffer out = new DataOutputBuffer();
+            bs.serialize(out);
+
+            DataInputStream in = new DataInputStream(new ByteArrayInputStream(out.getData()));
+            try (OffHeapBitSet newbs = OffHeapBitSet.deserialize(in))
+            {
+                compare(bs, newbs);
+            }
+        }
+    }
+
+    @Test
+    public void testBitSetGetClear()
+    {
+        int size = Integer.MAX_VALUE / 4000;
+        try (OffHeapBitSet bs = new OffHeapBitSet(size))
+        {
+            List<Integer> randomBits = Lists.newArrayList();
+            for (int i = 0; i < 10; i++)
+                randomBits.add(random.nextInt(size));
+
+            for (long randomBit : randomBits)
+                bs.set(randomBit);
+
+            for (long randomBit : randomBits)
+                assertEquals(true, bs.get(randomBit));
+
+            for (long randomBit : randomBits)
+                bs.clear(randomBit);
+
+            for (long randomBit : randomBits)
+                assertEquals(false, bs.get(randomBit));
+        }
+    }
+
+    @Test(expected = UnsupportedOperationException.class)
+    public void testUnsupportedLargeSize()
+    {
+        long size = 64L * Integer.MAX_VALUE + 1; // Max size 16G * 8 bits
+        OffHeapBitSet bs = new OffHeapBitSet(size);
+    }
+
+    @Test
+    public void testInvalidIndex()
+    {
+        OffHeapBitSet bs = new OffHeapBitSet(10);
+        int invalidIdx[] = {-1, 64, 1000};
+
+        for (int i : invalidIdx)
+        {
+            try
+            {
+                bs.set(i);
+            }
+            catch (AssertionError e)
+            {
+                assertTrue(e.getMessage().startsWith("Illegal bounds"));
+                continue;
+            }
+            fail(String.format("expect exception for index %d", i));
+        }
+
+        for (int i : invalidIdx)
+        {
+            try
+            {
+                bs.get(i);
+            }
+            catch (AssertionError e)
+            {
+                assertTrue(e.getMessage().startsWith("Illegal bounds"));
+                continue;
+            }
+            fail(String.format("expect exception for index %d", i));
+        }
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org