You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2015/03/04 05:55:49 UTC

cassandra git commit: Fix top partitions reporting wrong cardinality

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 9499f7cb9 -> 0127b6963


Fix top partitions reporting wrong cardinality

patch by Chris Lohfink; reviewed by Aleksey Yeschenko for CASSANDRA-8834


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

Branch: refs/heads/cassandra-2.1
Commit: 0127b6963ebf1c4fa407c71c12f10748b509b189
Parents: 9499f7c
Author: Chris Lohfink <Ch...@datastax.com>
Authored: Tue Mar 3 22:31:50 2015 -0600
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Tue Mar 3 20:37:17 2015 -0800

----------------------------------------------------------------------
 CHANGES.txt                                               | 1 +
 src/java/org/apache/cassandra/db/ColumnFamilyStore.java   | 4 ++--
 src/java/org/apache/cassandra/utils/TopKSampler.java      | 6 +++---
 test/unit/org/apache/cassandra/utils/TopKSamplerTest.java | 8 ++++++--
 4 files changed, 12 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0127b696/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a90dd48..7ce6200 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.4
+ * Fix top partitions reporting wrong cardinality (CASSANDRA-8834)
  * Fix rare NPE in KeyCacheSerializer (CASSANDRA-8067)
  * Pick sstables for validation as late as possible inc repairs (CASSANDRA-8366)
  * Fix commitlog getPendingTasks to not increment (CASSANDRA-8856)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0127b696/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index e4531f2..9b792b6 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1179,7 +1179,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         Memtable mt = data.getMemtableFor(opGroup, replayPosition);
         final long timeDelta = mt.put(key, columnFamily, indexer, opGroup);
         maybeUpdateRowCache(key);
-        metric.samplers.get(Sampler.WRITES).addSample(key.getKey());
+        metric.samplers.get(Sampler.WRITES).addSample(key.getKey(), key.hashCode(), 1);
         metric.writeLatency.addNano(System.nanoTime() - start);
         if(timeDelta < Long.MAX_VALUE)
             metric.colUpdateTimeDeltaHistogram.update(timeDelta);
@@ -1915,7 +1915,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             columns = controller.getTopLevelColumns(Memtable.MEMORY_POOL.needToCopyOnHeap());
         }
         if (columns != null)
-            metric.samplers.get(Sampler.READS).addSample(filter.key.getKey());
+            metric.samplers.get(Sampler.READS).addSample(filter.key.getKey(), filter.key.hashCode(), 1);
         metric.updateSSTableIterated(controller.getSstablesIterated());
         return columns;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0127b696/src/java/org/apache/cassandra/utils/TopKSampler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/TopKSampler.java b/src/java/org/apache/cassandra/utils/TopKSampler.java
index 29d46286..a8bd602 100644
--- a/src/java/org/apache/cassandra/utils/TopKSampler.java
+++ b/src/java/org/apache/cassandra/utils/TopKSampler.java
@@ -81,7 +81,7 @@ public class TopKSampler<T>
 
     public void addSample(T item)
     {
-        addSample(item, 1);
+        addSample(item, item.hashCode(), 1);
     }
 
     /**
@@ -89,7 +89,7 @@ public class TopKSampler<T>
      * use the "Sampler" thread pool to record results if the sampler is enabled.  If not
      * sampling this is a NOOP
      */
-    public void addSample(final T item, final int value)
+    public void addSample(final T item, final long hash, final int value)
     {
         if (enabled)
         {
@@ -107,7 +107,7 @@ public class TopKSampler<T>
                             try
                             {
                                 summary.offer(item, value);
-                                hll.offer(item);
+                                hll.offerHashed(hash);
                             } catch (Exception e)
                             {
                                 logger.debug("Failure to offer sample", e);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0127b696/test/unit/org/apache/cassandra/utils/TopKSamplerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/TopKSamplerTest.java b/test/unit/org/apache/cassandra/utils/TopKSamplerTest.java
index dc3b91c..085fa76 100644
--- a/test/unit/org/apache/cassandra/utils/TopKSamplerTest.java
+++ b/test/unit/org/apache/cassandra/utils/TopKSamplerTest.java
@@ -12,6 +12,7 @@ import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.utils.TopKSampler.SamplerResult;
 import org.junit.Test;
 
+import com.clearspring.analytics.hash.MurmurHash;
 import com.clearspring.analytics.stream.Counter;
 import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.Uninterruptibles;
@@ -32,10 +33,12 @@ public class TopKSamplerTest
         sampler2.beginSampling(10);
         for(int i = 1; i <= 10; i++)
         {
-           sampler2.addSample("item" + i, i);
+           String key = "item" + i;
+           sampler2.addSample(key, MurmurHash.hash64(key), i);
         }
         waitForEmpty(1000);
         Assert.assertEquals(countMap(single.topK), countMap(sampler2.finishSampling(10).topK));
+        Assert.assertEquals(sampler2.hll.cardinality(), 10);
         Assert.assertEquals(sampler.hll.cardinality(), sampler2.hll.cardinality());
     }
 
@@ -116,7 +119,8 @@ public class TopKSamplerTest
         {
             for(int j = 0; j < i; j++)
             {
-                sampler.addSample("item" + i);
+                String key = "item" + i;
+                sampler.addSample(key, MurmurHash.hash64(key), 1);
             }
         }
     }