cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jbel...@apache.org
Subject [1/3] git commit: Reduce garbage generated by bloom filter lookups patch by Benedict Elliott Smith; reviewed by jbellis for CASSANDRA-6609
Date Mon, 27 Jan 2014 23:00:31 GMT
Updated Branches:
  refs/heads/cassandra-2.0 8bbb6eda6 -> 20c2adc87
  refs/heads/trunk 714c42336 -> 82735e096


Reduce garbage generated by bloom filter lookups
patch by Benedict Elliott Smith; reviewed by jbellis for CASSANDRA-6609


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

Branch: refs/heads/cassandra-2.0
Commit: 20c2adc87102963836a59a5e9626005fd9ee08bc
Parents: 8bbb6ed
Author: Jonathan Ellis <jbellis@apache.org>
Authored: Mon Jan 27 17:00:08 2014 -0600
Committer: Jonathan Ellis <jbellis@apache.org>
Committed: Mon Jan 27 17:00:08 2014 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/dht/Murmur3Partitioner.java       |  5 +-
 .../org/apache/cassandra/utils/BloomFilter.java | 79 ++++++++++++++------
 .../org/apache/cassandra/utils/FBUtilities.java |  6 ++
 .../cassandra/utils/Murmur3BloomFilter.java     |  4 +-
 .../org/apache/cassandra/utils/MurmurHash.java  |  6 +-
 .../cassandra/utils/obs/OffHeapBitSet.java      |  2 +-
 7 files changed, 72 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/20c2adc8/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 6acbc87..68727dc 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.0.5
+ * Reduce garbage generated by bloom filter lookups (CASSANDRA-6609)
  * Add ks.cf names to tombstone logging (CASSANDRA-6597)
  * Use LOCAL_QUORUM for LWT operations at LOCAL_SERIAL (CASSANDRA-6495)
  * Wait for gossip to settle before accepting client connections (CASSANDRA-4288)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/20c2adc8/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
index 9ff635e..3a045d7 100644
--- a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
+++ b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
@@ -89,8 +89,9 @@ public class Murmur3Partitioner extends AbstractPartitioner<LongToken>
         if (key.remaining() == 0)
             return MINIMUM;
 
-        long hash = MurmurHash.hash3_x64_128(key, key.position(), key.remaining(), 0)[0];
-        return new LongToken(normalize(hash));
+        long[] hash = new long[2];
+        MurmurHash.hash3_x64_128(key, key.position(), key.remaining(), 0, hash);
+        return new LongToken(normalize(hash[0]));
     }
 
     public LongToken getRandomToken()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/20c2adc8/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 b134b3c..9fbb38e 100644
--- a/src/java/org/apache/cassandra/utils/BloomFilter.java
+++ b/src/java/org/apache/cassandra/utils/BloomFilter.java
@@ -20,10 +20,20 @@ package org.apache.cassandra.utils;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.cassandra.utils.obs.IBitSet;
 
 public abstract class BloomFilter implements IFilter
 {
+    private static final ThreadLocal<long[]> reusableIndexes = new ThreadLocal<long[]>()
+    {
+        protected long[] initialValue()
+        {
+            return new long[21];
+        }
+    };
+
     public final IBitSet bitset;
     public final int hashCount;
 
@@ -33,47 +43,68 @@ public abstract class BloomFilter implements IFilter
         this.bitset = bitset;
     }
 
-    private long[] getHashBuckets(ByteBuffer key)
-    {
-        return getHashBuckets(key, hashCount, bitset.capacity());
-    }
-
-    protected abstract long[] hash(ByteBuffer b, int position, int remaining, long seed);
-
     // Murmur is faster than an SHA-based approach and provides as-good collision
     // resistance.  The combinatorial generation approach described in
     // http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf
     // does prove to work in actual tests, and is obviously faster
     // than performing further iterations of murmur.
-    long[] getHashBuckets(ByteBuffer b, int hashCount, long max)
+    protected abstract void hash(ByteBuffer b, int position, int remaining, long seed, long[]
result);
+
+    // tests ask for ridiculous numbers of hashes so here is a special case for them
+    // rather than using the threadLocal like we do in production
+    @VisibleForTesting
+    public long[] getHashBuckets(ByteBuffer key, int hashCount, long max)
     {
-        long[] result = new long[hashCount];
-        long[] hash = this.hash(b, b.position(), b.remaining(), 0L);
-        for (int i = 0; i < hashCount; ++i)
+        long[] hash = new long[2];
+        hash(key, key.position(), key.remaining(), 0L, hash);
+        long[] indexes = new long[hashCount];
+        setIndexes(hash[0], hash[1], hashCount, max, indexes);
+        return indexes;
+    }
+
+    // note that this method uses the threadLocal that may be longer than hashCount
+    // to avoid generating a lot of garbage since stack allocation currently does not support
stores
+    // (CASSANDRA-6609).  it returns the array so that the caller does not need to perform
+    // a second threadlocal lookup.
+    private long[] indexes(ByteBuffer key)
+    {
+        // we use the same array both for storing the hash result, and for storing the indexes
we return,
+        // so that we do not need to allocate two arrays.
+        long[] indexes = reusableIndexes.get();
+        hash(key, key.position(), key.remaining(), 0L, indexes);
+        setIndexes(indexes[0], indexes[1], hashCount, bitset.capacity(), indexes);
+        return indexes;
+    }
+
+    private void setIndexes(long base, long inc, int count, long max, long[] results)
+    {
+        for (int i = 0; i < count; i++)
         {
-            result[i] = Math.abs((hash[0] + (long)i * hash[1]) % max);
+            results[i] = FBUtilities.abs(base % max);
+            base += inc;
         }
-        return result;
     }
 
     public void add(ByteBuffer key)
     {
-        for (long bucketIndex : getHashBuckets(key))
+        long[] indexes = indexes(key);
+        for (int i = 0; i < hashCount; i++)
         {
-            bitset.set(bucketIndex);
+            bitset.set(indexes[i]);
         }
     }
 
-    public boolean isPresent(ByteBuffer key)
+    public final boolean isPresent(ByteBuffer key)
     {
-      for (long bucketIndex : getHashBuckets(key))
-      {
-          if (!bitset.get(bucketIndex))
-          {
-              return false;
-          }
-      }
-      return true;
+        long[] indexes = indexes(key);
+        for (int i = 0; i < hashCount; i++)
+        {
+            if (!bitset.get(indexes[i]))
+            {
+                return false;
+            }
+        }
+        return true;
     }
 
     public void clear()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/20c2adc8/src/java/org/apache/cassandra/utils/FBUtilities.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java
index 7eb735e..579f5fa 100644
--- a/src/java/org/apache/cassandra/utils/FBUtilities.java
+++ b/src/java/org/apache/cassandra/utils/FBUtilities.java
@@ -618,6 +618,12 @@ public class FBUtilities
         checksum.update((v >>> 0) & 0xFF);
     }
 
+    public static long abs(long index)
+    {
+        long negbit = index >> 63;
+        return (index ^ negbit) - negbit;
+    }
+
     private static final class WrappedCloseableIterator<T>
         extends AbstractIterator<T> implements CloseableIterator<T>
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/20c2adc8/src/java/org/apache/cassandra/utils/Murmur3BloomFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/Murmur3BloomFilter.java b/src/java/org/apache/cassandra/utils/Murmur3BloomFilter.java
index 2c0c45b..3c2a47e 100644
--- a/src/java/org/apache/cassandra/utils/Murmur3BloomFilter.java
+++ b/src/java/org/apache/cassandra/utils/Murmur3BloomFilter.java
@@ -36,9 +36,9 @@ public class Murmur3BloomFilter extends BloomFilter
         return serializer.serializedSize(this, TypeSizes.NATIVE);
     }
 
-    protected long[] hash(ByteBuffer b, int position, int remaining, long seed)
+    protected void hash(ByteBuffer b, int position, int remaining, long seed, long[] result)
     {
-        return MurmurHash.hash3_x64_128(b, b.position(), b.remaining(), seed);
+        MurmurHash.hash3_x64_128(b, b.position(), b.remaining(), seed, result);
     }
 
     public static class Murmur3BloomFilterSerializer extends BloomFilterSerializer

http://git-wip-us.apache.org/repos/asf/cassandra/blob/20c2adc8/src/java/org/apache/cassandra/utils/MurmurHash.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/MurmurHash.java b/src/java/org/apache/cassandra/utils/MurmurHash.java
index eb7b75a..9dcde6d 100644
--- a/src/java/org/apache/cassandra/utils/MurmurHash.java
+++ b/src/java/org/apache/cassandra/utils/MurmurHash.java
@@ -170,7 +170,7 @@ public class MurmurHash
         return k;
     }
 
-    public static long[] hash3_x64_128(ByteBuffer key, int offset, int length, long seed)
+    public static void hash3_x64_128(ByteBuffer key, int offset, int length, long seed, long[]
result)
     {
         final int nblocks = length >> 4; // Process as 128-bit blocks.
 
@@ -242,6 +242,8 @@ public class MurmurHash
         h1 += h2;
         h2 += h1;
 
-        return(new long[] {h1, h2});
+        result[0] = h1;
+        result[1] = h2;
     }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/20c2adc8/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 74b7b71..29dd848 100644
--- a/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java
+++ b/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java
@@ -67,7 +67,7 @@ public class OffHeapBitSet implements IBitSet
         long i = index >> 3;
         long bit = index & 0x7;
         int bitmask = 0x1 << bit;
-        return ((bytes.getByte(i) & 0xFF) & bitmask) != 0;
+        return (bytes.getByte(i) & bitmask) != 0;
     }
 
     public void set(long index)


Mime
View raw message