You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by br...@apache.org on 2014/06/24 20:09:12 UTC

[15/16] git commit: Expose global CF metrics

Expose global CF metrics

Patch by Chris Lohfink, reviewed by brandonwilliams for CASSANDRA-7273


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

Branch: refs/heads/cassandra-2.1
Commit: 5b8ac8cbc5ff9e0dffb6ebc0485e0e2bc2a679fe
Parents: cc20907
Author: Brandon Williams <br...@apache.org>
Authored: Tue Jun 24 13:06:53 2014 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Tue Jun 24 13:06:53 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |   4 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |   2 +-
 .../cassandra/metrics/ColumnFamilyMetrics.java  | 345 +++++++++++++++----
 .../cassandra/metrics/KeyspaceMetrics.java      | 248 +++++++------
 .../cassandra/metrics/LatencyMetrics.java       |  28 +-
 .../org/apache/cassandra/db/KeyspaceTest.java   |  16 +-
 7 files changed, 437 insertions(+), 207 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b8ac8cb/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8650b7a..4b97875 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -30,6 +30,7 @@ Merged from 2.0:
  * Improve sub range repair validation (CASSANDRA-7317)
  * Accept subtypes for function results, type casts (CASSANDRA-6766)
 Merged from 1.2:
+ * Expose global ColumnFamily metrics (CASSANDRA-7273)
  * Handle possible integer overflow in FastByteArrayOutputStream (CASSANDRA-7373)
  * cqlsh: 'ascii' values weren't formatted as text (CASSANDRA-7407)
  * cqlsh: ignore .cassandra permission errors (CASSANDRA-7266)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b8ac8cb/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 e2d9248..fb66c34 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -2674,12 +2674,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public double getTombstonesPerSlice()
     {
-        return metric.tombstoneScannedHistogram.getSnapshot().getMedian();
+        return metric.tombstoneScannedHistogram.cf.getSnapshot().getMedian();
     }
 
     public double getLiveCellsPerSlice()
     {
-        return metric.liveScannedHistogram.getSnapshot().getMedian();
+        return metric.liveScannedHistogram.cf.getSnapshot().getMedian();
     }
 
     // End JMX get/set.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b8ac8cb/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index 54054c5..ceeac70 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -273,12 +273,12 @@ public class Keyspace
         assert metadata != null : "Unknown keyspace " + keyspaceName;
         createReplicationStrategy(metadata);
 
+        this.metric = new KeyspaceMetrics(this);
         for (CFMetaData cfm : new ArrayList<CFMetaData>(metadata.cfMetaData().values()))
         {
             logger.debug("Initializing {}.{}", getName(), cfm.cfName);
             initCf(cfm.cfId, cfm.cfName, loadSSTables);
         }
-        this.metric = new KeyspaceMetrics(this);
     }
 
     public void createReplicationStrategy(KSMetaData ksm)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b8ac8cb/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java b/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
index f4682da..75a21dc 100644
--- a/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
@@ -17,17 +17,23 @@
  */
 package org.apache.cassandra.metrics;
 
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
 
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.*;
-import com.yammer.metrics.util.RatioGauge;
-
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.utils.EstimatedHistogram;
 
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import com.yammer.metrics.Metrics;
+import com.yammer.metrics.core.*;
+import com.yammer.metrics.util.RatioGauge;
+
 /**
  * Metrics for {@link ColumnFamilyStore}.
  */
@@ -56,7 +62,7 @@ public class ColumnFamilyMetrics
     /** Histogram of estimated number of columns. */
     public final Gauge<long[]> estimatedColumnCountHistogram;
     /** Histogram of the number of sstable data files accessed per read */
-    public final Histogram sstablesPerReadHistogram;
+    public final ColumnFamilyHistogram sstablesPerReadHistogram;
     /** (Local) read metrics */
     public final LatencyMetrics readLatency;
     /** (Local) range slice metrics */
@@ -92,9 +98,9 @@ public class ColumnFamilyMetrics
     /** Key cache hit rate  for this CF */
     public final Gauge<Double> keyCacheHitRate;
     /** Tombstones scanned in queries on this CF */
-    public final Histogram tombstoneScannedHistogram;
+    public final ColumnFamilyHistogram tombstoneScannedHistogram;
     /** Live cells scanned in queries on this CF */
-    public final Histogram liveScannedHistogram;
+    public final ColumnFamilyHistogram liveScannedHistogram;
     /** Disk space used by snapshot files which */
     public final Gauge<Long> trueSnapshotsSize;
     /** Row cache hits, but result out of range */
@@ -111,14 +117,28 @@ public class ColumnFamilyMetrics
     public final Timer waitingOnFreeMemtableSpace;
 
     private final MetricNameFactory factory;
+    private static final MetricNameFactory globalNameFactory = new AllColumnFamilyMetricNameFactory();;
 
     public final Counter speculativeRetries;
 
     // for backward compatibility
     @Deprecated public final EstimatedHistogram sstablesPerRead = new EstimatedHistogram(35);
     @Deprecated public final EstimatedHistogram recentSSTablesPerRead = new EstimatedHistogram(35);
-
-
+    
+    public final static LatencyMetrics globalReadLatency = new LatencyMetrics(globalNameFactory, "Read");
+    public final static LatencyMetrics globalWriteLatency = new LatencyMetrics(globalNameFactory, "Write");
+    public final static LatencyMetrics globalRangeLatency = new LatencyMetrics(globalNameFactory, "Range");
+    
+    /**
+     * stores metrics that will be rolled into a single global metric
+     */
+    public final static ConcurrentMap<String, Set<Metric>> allColumnFamilyMetrics = Maps.newConcurrentMap();
+    
+    /**
+     * Stores all metric names created that can be used when unregistering
+     */
+    public final static Set<String> all = Sets.newHashSet();
+    
     /**
      * Creates metrics for given {@link ColumnFamilyStore}.
      *
@@ -128,35 +148,35 @@ public class ColumnFamilyMetrics
     {
         factory = new ColumnFamilyMetricNameFactory(cfs);
 
-        memtableColumnsCount = Metrics.newGauge(factory.createMetricName("MemtableColumnsCount"), new Gauge<Long>()
+        memtableColumnsCount = createColumnFamilyGauge("MemtableColumnsCount", new Gauge<Long>()
         {
             public Long value()
             {
                 return cfs.getDataTracker().getView().getCurrentMemtable().getOperations();
             }
         });
-        memtableOnHeapSize = Metrics.newGauge(factory.createMetricName("MemtableHeapSize"), new Gauge<Long>()
+        memtableOnHeapSize = createColumnFamilyGauge("MemtableOnHeapSize", new Gauge<Long>()
         {
             public Long value()
             {
                 return cfs.getDataTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
             }
         });
-        memtableOffHeapSize = Metrics.newGauge(factory.createMetricName("MemtableHeapSize"), new Gauge<Long>()
+        memtableOffHeapSize = createColumnFamilyGauge("MemtableOffHeapSize", new Gauge<Long>()
         {
             public Long value()
             {
                 return cfs.getDataTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
             }
         });
-        memtableLiveDataSize = Metrics.newGauge(factory.createMetricName("MemtableLiveDataSize"), new Gauge<Long>()
+        memtableLiveDataSize = createColumnFamilyGauge("MemtableLiveDataSize", new Gauge<Long>()
         {
             public Long value()
             {
                 return cfs.getDataTracker().getView().getCurrentMemtable().getLiveDataSize();
             }
         });
-        allMemtablesOnHeapSize = Metrics.newGauge(factory.createMetricName("AllMemtablesHeapSize"), new Gauge<Long>()
+        allMemtablesOnHeapSize = createColumnFamilyGauge("AllMemtablesHeapSize", new Gauge<Long>()
         {
             public Long value()
             {
@@ -166,7 +186,7 @@ public class ColumnFamilyMetrics
                 return size;
             }
         });
-        allMemtablesOffHeapSize = Metrics.newGauge(factory.createMetricName("AllMemtablesHeapSize"), new Gauge<Long>()
+        allMemtablesOffHeapSize = createColumnFamilyGauge("AllMemtablesOffHeapSize", new Gauge<Long>()
         {
             public Long value()
             {
@@ -176,7 +196,7 @@ public class ColumnFamilyMetrics
                 return size;
             }
         });
-        allMemtablesLiveDataSize = Metrics.newGauge(factory.createMetricName("AllMemtablesLiveDataSize"), new Gauge<Long>()
+        allMemtablesLiveDataSize = createColumnFamilyGauge("AllMemtablesLiveDataSize", new Gauge<Long>()
         {
             public Long value()
             {
@@ -186,7 +206,7 @@ public class ColumnFamilyMetrics
                 return size;
             }
         });
-        memtableSwitchCount = Metrics.newCounter(factory.createMetricName("MemtableSwitchCount"));
+        memtableSwitchCount = createColumnFamilyCounter("MemtableSwitchCount");
         estimatedRowSizeHistogram = Metrics.newGauge(factory.createMetricName("EstimatedRowSizeHistogram"), new Gauge<long[]>()
         {
             public long[] value()
@@ -215,8 +235,8 @@ public class ColumnFamilyMetrics
                 return histogram;
             }
         });
-        sstablesPerReadHistogram = Metrics.newHistogram(factory.createMetricName("SSTablesPerReadHistogram"), true);
-        compressionRatio = Metrics.newGauge(factory.createMetricName("CompressionRatio"), new Gauge<Double>()
+        sstablesPerReadHistogram = createColumnFamilyHistogram("SSTablesPerReadHistogram", cfs.keyspace.metric.sstablesPerReadHistogram);
+        compressionRatio = createColumnFamilyGauge("CompressionRatio", new Gauge<Double>()
         {
             public Double value()
             {
@@ -232,28 +252,47 @@ public class ColumnFamilyMetrics
                 }
                 return total != 0 ? sum / total : 0;
             }
+        }, new Gauge<Double>() // global gauge
+        {
+            public Double value()
+            {
+                double sum = 0;
+                int total = 0;
+                for (Keyspace keyspace : Keyspace.all())
+                {
+                    for (SSTableReader sstable : keyspace.getAllSSTables())
+                    {
+                        if (sstable.getCompressionRatio() != MetadataCollector.NO_COMPRESSION_RATIO)
+                        {
+                            sum += sstable.getCompressionRatio();
+                            total++;
+                        }
+                    }
+                }
+                return total != 0 ? sum / total : 0;
+            }
         });
-        readLatency = new LatencyMetrics(factory, "Read");
-        writeLatency = new LatencyMetrics(factory, "Write");
-        rangeLatency = new LatencyMetrics(factory, "Range");
-        pendingFlushes = Metrics.newCounter(factory.createMetricName("PendingFlushes"));
-        pendingCompactions = Metrics.newGauge(factory.createMetricName("PendingCompactions"), new Gauge<Integer>()
+        readLatency = new LatencyMetrics(factory, "Read", cfs.keyspace.metric.readLatency, globalReadLatency);
+        writeLatency = new LatencyMetrics(factory, "Write", cfs.keyspace.metric.writeLatency, globalWriteLatency);
+        rangeLatency = new LatencyMetrics(factory, "Range", cfs.keyspace.metric.rangeLatency, globalRangeLatency);
+        pendingFlushes = createColumnFamilyCounter("PendingFlushes");
+        pendingCompactions = createColumnFamilyGauge("PendingCompactions", new Gauge<Integer>()
         {
             public Integer value()
             {
                 return cfs.getCompactionStrategy().getEstimatedRemainingTasks();
             }
         });
-        liveSSTableCount = Metrics.newGauge(factory.createMetricName("LiveSSTableCount"), new Gauge<Integer>()
+        liveSSTableCount = createColumnFamilyGauge("LiveSSTableCount", new Gauge<Integer>()
         {
             public Integer value()
             {
                 return cfs.getDataTracker().getSSTables().size();
             }
         });
-        liveDiskSpaceUsed = Metrics.newCounter(factory.createMetricName("LiveDiskSpaceUsed"));
-        totalDiskSpaceUsed = Metrics.newCounter(factory.createMetricName("TotalDiskSpaceUsed"));
-        minRowSize = Metrics.newGauge(factory.createMetricName("MinRowSize"), new Gauge<Long>()
+        liveDiskSpaceUsed = createColumnFamilyCounter("LiveDiskSpaceUsed");
+        totalDiskSpaceUsed = createColumnFamilyCounter("TotalDiskSpaceUsed");
+        minRowSize = createColumnFamilyGauge("MinRowSize", new Gauge<Long>()
         {
             public Long value()
             {
@@ -265,8 +304,19 @@ public class ColumnFamilyMetrics
                 }
                 return min;
             }
+        }, new Gauge<Long>() // global gauge
+        {
+            public Long value()
+            {
+                long min = Long.MAX_VALUE;
+                for (Metric cfGauge : allColumnFamilyMetrics.get("MinRowSize"))
+                {
+                    min = Math.min(min, ((Gauge<? extends Number>) cfGauge).value().longValue());
+                }
+                return min;
+            }
         });
-        maxRowSize = Metrics.newGauge(factory.createMetricName("MaxRowSize"), new Gauge<Long>()
+        maxRowSize = createColumnFamilyGauge("MaxRowSize", new Gauge<Long>()
         {
             public Long value()
             {
@@ -278,8 +328,19 @@ public class ColumnFamilyMetrics
                 }
                 return max;
             }
+        }, new Gauge<Long>() // global gauge
+        {
+            public Long value()
+            {
+                long max = 0;
+                for (Metric cfGauge : allColumnFamilyMetrics.get("MaxRowSize"))
+                {
+                    max = Math.max(max, ((Gauge<? extends Number>) cfGauge).value().longValue());
+                }
+                return max;
+            }
         });
-        meanRowSize = Metrics.newGauge(factory.createMetricName("MeanRowSize"), new Gauge<Long>()
+        meanRowSize = createColumnFamilyGauge("MeanRowSize", new Gauge<Long>()
         {
             public Long value()
             {
@@ -293,8 +354,25 @@ public class ColumnFamilyMetrics
                 }
                 return count > 0 ? sum / count : 0;
             }
+        }, new Gauge<Long>() // global gauge
+        {
+            public Long value()
+            {
+                long sum = 0;
+                long count = 0;
+                for (Keyspace keyspace : Keyspace.all())
+                {
+                    for (SSTableReader sstable : keyspace.getAllSSTables())
+                    {
+                        long n = sstable.getEstimatedRowSize().count();
+                        sum += sstable.getEstimatedRowSize().mean() * n;
+                        count += n;
+                    }
+                }
+                return count > 0 ? sum / count : 0;
+            }
         });
-        bloomFilterFalsePositives = Metrics.newGauge(factory.createMetricName("BloomFilterFalsePositives"), new Gauge<Long>()
+        bloomFilterFalsePositives = createColumnFamilyGauge("BloomFilterFalsePositives", new Gauge<Long>()
         {
             public Long value()
             {
@@ -304,7 +382,7 @@ public class ColumnFamilyMetrics
                 return count;
             }
         });
-        recentBloomFilterFalsePositives = Metrics.newGauge(factory.createMetricName("RecentBloomFilterFalsePositives"), new Gauge<Long>()
+        recentBloomFilterFalsePositives = createColumnFamilyGauge("RecentBloomFilterFalsePositives", new Gauge<Long>()
         {
             public Long value()
             {
@@ -314,7 +392,7 @@ public class ColumnFamilyMetrics
                 return count;
             }
         });
-        bloomFilterFalseRatio = Metrics.newGauge(factory.createMetricName("BloomFilterFalseRatio"), new Gauge<Double>()
+        bloomFilterFalseRatio = createColumnFamilyGauge("BloomFilterFalseRatio", new Gauge<Double>()
         {
             public Double value()
             {
@@ -329,8 +407,26 @@ public class ColumnFamilyMetrics
                     return 0d;
                 return (double) falseCount / (trueCount + falseCount);
             }
+        }, new Gauge<Double>() // global gauge
+        {
+            public Double value()
+            {
+                long falseCount = 0L;
+                long trueCount = 0L;
+                for (Keyspace keyspace : Keyspace.all())
+                {
+                    for (SSTableReader sstable : keyspace.getAllSSTables())
+                    {
+                        falseCount += sstable.getBloomFilterFalsePositiveCount();
+                        trueCount += sstable.getBloomFilterTruePositiveCount();
+                    }
+                }
+                if (falseCount == 0L && trueCount == 0L)
+                    return 0d;
+                return (double) falseCount / (trueCount + falseCount);
+            }
         });
-        recentBloomFilterFalseRatio = Metrics.newGauge(factory.createMetricName("RecentBloomFilterFalseRatio"), new Gauge<Double>()
+        recentBloomFilterFalseRatio = createColumnFamilyGauge("RecentBloomFilterFalseRatio", new Gauge<Double>()
         {
             public Double value()
             {
@@ -345,8 +441,26 @@ public class ColumnFamilyMetrics
                     return 0d;
                 return (double) falseCount / (trueCount + falseCount);
             }
+        }, new Gauge<Double>() // global gauge
+        {
+            public Double value()
+            {
+                long falseCount = 0L;
+                long trueCount = 0L;
+                for (Keyspace keyspace : Keyspace.all())
+                {
+                    for (SSTableReader sstable : keyspace.getAllSSTables())
+                    {
+                        falseCount += sstable.getRecentBloomFilterFalsePositiveCount();
+                        trueCount += sstable.getRecentBloomFilterTruePositiveCount();
+                    }
+                }
+                if (falseCount == 0L && trueCount == 0L)
+                    return 0d;
+                return (double) falseCount / (trueCount + falseCount);
+            }
         });
-        bloomFilterDiskSpaceUsed = Metrics.newGauge(factory.createMetricName("BloomFilterDiskSpaceUsed"), new Gauge<Long>()
+        bloomFilterDiskSpaceUsed = createColumnFamilyGauge("BloomFilterDiskSpaceUsed", new Gauge<Long>()
         {
             public Long value()
             {
@@ -356,7 +470,7 @@ public class ColumnFamilyMetrics
                 return total;
             }
         });
-        speculativeRetries = Metrics.newCounter(factory.createMetricName("SpeculativeRetries"));
+        speculativeRetries = createColumnFamilyCounter("SpeculativeRetries");
         keyCacheHitRate = Metrics.newGauge(factory.createMetricName("KeyCacheHitRate"), new RatioGauge()
         {
             protected double getNumerator()
@@ -375,22 +489,22 @@ public class ColumnFamilyMetrics
                 return Math.max(requests, 1); // to avoid NaN.
             }
         });
-        tombstoneScannedHistogram = Metrics.newHistogram(factory.createMetricName("TombstoneScannedHistogram"), true);
-        liveScannedHistogram = Metrics.newHistogram(factory.createMetricName("LiveScannedHistogram"), true);
+        tombstoneScannedHistogram = createColumnFamilyHistogram("TombstoneScannedHistogram", cfs.keyspace.metric.tombstoneScannedHistogram);
+        liveScannedHistogram = createColumnFamilyHistogram("LiveScannedHistogram", cfs.keyspace.metric.liveScannedHistogram);
         coordinatorReadLatency = Metrics.newTimer(factory.createMetricName("CoordinatorReadLatency"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
         coordinatorScanLatency = Metrics.newTimer(factory.createMetricName("CoordinatorScanLatency"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
         waitingOnFreeMemtableSpace = Metrics.newTimer(factory.createMetricName("WaitingOnFreeMemtableSpace"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
 
-        trueSnapshotsSize = Metrics.newGauge(factory.createMetricName("SnapshotsSize"), new Gauge<Long>()
+        trueSnapshotsSize = createColumnFamilyGauge("SnapshotsSize", new Gauge<Long>()
         {
             public Long value()
             {
                 return cfs.trueSnapshotsSize();
             }
         });
-        rowCacheHitOutOfRange = Metrics.newCounter(factory.createMetricName("RowCacheHitOutOfRange"));
-        rowCacheHit = Metrics.newCounter(factory.createMetricName("RowCacheHit"));
-        rowCacheMiss = Metrics.newCounter(factory.createMetricName("RowCacheMiss"));
+        rowCacheHitOutOfRange = createColumnFamilyCounter("RowCacheHitOutOfRange");
+        rowCacheHit = createColumnFamilyCounter("RowCacheHit");
+        rowCacheMiss = createColumnFamilyCounter("RowCacheMiss");
     }
 
     public void updateSSTableIterated(int count)
@@ -405,41 +519,124 @@ public class ColumnFamilyMetrics
      */
     public void release()
     {
+        for(String name : all)
+        {
+            allColumnFamilyMetrics.get(name).remove(Metrics.defaultRegistry().allMetrics().get(factory.createMetricName(name)));
+            Metrics.defaultRegistry().removeMetric(factory.createMetricName(name));
+        }
         readLatency.release();
         writeLatency.release();
         rangeLatency.release();
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("AllMemtablesDataSize"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableColumnsCount"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableDataSize"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableSwitchCount"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("CompressionRatio"));
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("EstimatedRowSizeHistogram"));
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("EstimatedColumnCountHistogram"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("SSTablesPerReadHistogram"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("PendingFlushes"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("LiveSSTableCount"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("LiveDiskSpaceUsed"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("TotalDiskSpaceUsed"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MinRowSize"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MaxRowSize"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MeanRowSize"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("BloomFilterFalsePositives"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("RecentBloomFilterFalsePositives"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("BloomFilterFalseRatio"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("RecentBloomFilterFalseRatio"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("BloomFilterDiskSpaceUsed"));
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("KeyCacheHitRate"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("SpeculativeRetries"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("TombstoneScannedHistogram"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("LiveScannedHistogram"));
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("CoordinatorReadLatency"));
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("CoordinatorScanLatency"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("SnapshotsSize"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("RowCacheHitOutOfRange"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("RowCacheHit"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("RowCacheHitMiss"));
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("WaitingOnFreeMemtableSpace"));
     }
 
+
+    /**
+     * Create a gauge that will be part of a merged version of all column families.  The global gauge
+     * will merge each CF gauge by adding their values 
+     */
+    protected <T extends Number> Gauge<T> createColumnFamilyGauge(final String name, Gauge<T> gauge)
+    {
+        return createColumnFamilyGauge(name, gauge, new Gauge<Long>()
+        {
+            public Long value()
+            {
+                long total = 0;
+                for (Metric cfGauge : allColumnFamilyMetrics.get(name))
+                {
+                    total = total + ((Gauge<? extends Number>) cfGauge).value().longValue();
+                }
+                return total;
+            }
+        });
+    }
+    
+    /**
+     * Create a gauge that will be part of a merged version of all column families.  The global gauge
+     * is defined as the globalGauge parameter
+     */
+    protected <G,T> Gauge<T> createColumnFamilyGauge(String name, Gauge<T> gauge, Gauge<G> globalGauge)
+    {
+        Gauge<T> cfGauge = Metrics.newGauge(factory.createMetricName(name), gauge);
+        if (register(name, cfGauge))
+        {
+            Metrics.newGauge(globalNameFactory.createMetricName(name), globalGauge);
+        }
+        return cfGauge;
+    }
+    
+    /**
+     * Creates a counter that will also have a global counter thats the sum of all counters across 
+     * different column families
+     */
+    protected Counter createColumnFamilyCounter(final String name)
+    {
+        Counter cfCounter = Metrics.newCounter(factory.createMetricName(name));
+        if (register(name, cfCounter))
+        {
+            Metrics.newGauge(globalNameFactory.createMetricName(name), new Gauge<Long>()
+            {
+                public Long value()
+                {
+                    long total = 0;
+                    for (Metric cfGauge : allColumnFamilyMetrics.get(name))
+                    {
+                        total += ((Counter) cfGauge).count();
+                    }
+                    return total;
+                }
+            });
+        }
+        return cfCounter;
+    }
+    
+    /**
+     * Create a histogram-like interface that will register both a CF, keyspace and global level
+     * histogram and forward any updates to both
+     */
+    protected ColumnFamilyHistogram createColumnFamilyHistogram(String name, Histogram keyspaceHistogram) 
+    {
+        Histogram cfHistogram = Metrics.newHistogram(factory.createMetricName(name), true);  
+        register(name, cfHistogram);
+        return new ColumnFamilyHistogram(cfHistogram, keyspaceHistogram, Metrics.newHistogram(globalNameFactory.createMetricName(name), true));
+    }
+    
+    /**
+     * Registers a metric to be removed when unloading CF.
+     * @return true if first time metric with that name has been registered
+     */
+    private boolean register(String name, Metric metric)
+    { 
+        boolean ret = allColumnFamilyMetrics.putIfAbsent(name,  new HashSet<Metric>()) == null;
+        allColumnFamilyMetrics.get(name).add(metric);
+        all.add(name);
+        return ret;
+    }
+    
+    public class ColumnFamilyHistogram
+    {
+        public final Histogram[] all;
+        public final Histogram cf;
+        private ColumnFamilyHistogram(Histogram cf, Histogram keyspace, Histogram global) 
+        {
+            this.cf = cf;
+            this.all = new Histogram[]{cf, keyspace, global};
+        }
+        
+        public void update(long i) 
+        {
+            for(Histogram histo : all)
+            {
+                histo.update(i);
+            }
+        }
+    }
+    
     class ColumnFamilyMetricNameFactory implements MetricNameFactory
     {
         private final String keyspaceName;
@@ -468,5 +665,17 @@ public class ColumnFamilyMetrics
             return new MetricName(groupName, type, metricName, keyspaceName + "." + columnFamilyName, mbeanName.toString());
         }
     }
-
+    
+    static class AllColumnFamilyMetricNameFactory implements MetricNameFactory
+    {
+        public MetricName createMetricName(String metricName)
+        {
+            String groupName = ColumnFamilyMetrics.class.getPackage().getName(); 
+            StringBuilder mbeanName = new StringBuilder();
+            mbeanName.append(groupName).append(":");
+            mbeanName.append("type=ColumnFamily");
+            mbeanName.append(",name=").append(metricName);
+            return new MetricName(groupName, "ColumnFamily", metricName, "all", mbeanName.toString());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b8ac8cb/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
index 718e6f4..92fabf1 100644
--- a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
@@ -17,16 +17,16 @@
  */
 package org.apache.cassandra.metrics;
 
-import java.util.ArrayList;
-import java.util.List;
-
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.*;
-import com.yammer.metrics.stats.Snapshot;
+import java.util.Set;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.yammer.metrics.Metrics;
+import com.yammer.metrics.core.*;
+
 /**
  * Metrics for {@link ColumnFamilyStore}.
  */
@@ -49,18 +49,34 @@ public class KeyspaceMetrics
     /** Number of times flush has resulted in the memtable being switched out. */
     public final Gauge<Long> memtableSwitchCount;
     /** Estimated number of tasks pending for this column family */
-    public final Gauge<Integer> pendingFlushes;
+    public final Gauge<Long> pendingFlushes;
     /** Estimate of number of pending compactios for this CF */
-    public final Gauge<Integer> pendingCompactions;
+    public final Gauge<Long> pendingCompactions;
     /** Disk space used by SSTables belonging to this CF */
     public final Gauge<Long> liveDiskSpaceUsed;
     /** Total disk space used by SSTables belonging to this CF, including obsolete ones waiting to be GC'd */
     public final Gauge<Long> totalDiskSpaceUsed;
     /** Disk space used by bloom filter */
     public final Gauge<Long> bloomFilterDiskSpaceUsed;
-
-    private final MetricNameFactory factory;
-
+    /** (Local) read metrics */
+    public final LatencyMetrics readLatency;
+    /** (Local) range slice metrics */
+    public final LatencyMetrics rangeLatency;
+    /** (Local) write metrics */
+    public final LatencyMetrics writeLatency;
+    /** Histogram of the number of sstable data files accessed per read */
+    public final Histogram sstablesPerReadHistogram;
+    /** Tombstones scanned in queries on this Keyspace */
+    public final Histogram tombstoneScannedHistogram;
+    /** Live cells scanned in queries on this Keyspace */
+    public final Histogram liveScannedHistogram;
+    
+    public final MetricNameFactory factory;
+    private Keyspace keyspace;
+    
+    /** set containing names of all the metrics stored here, for releasing later */
+    private Set<String> allMetrics = Sets.newHashSet();
+    
     /**
      * Creates metrics for given {@link ColumnFamilyStore}.
      *
@@ -69,159 +85,108 @@ public class KeyspaceMetrics
     public KeyspaceMetrics(final Keyspace ks)
     {
         factory = new KeyspaceMetricNameFactory(ks);
-
-        memtableColumnsCount = Metrics.newGauge(factory.createMetricName("MemtableColumnsCount"), new Gauge<Long>()
+        keyspace = ks;
+        memtableColumnsCount = createKeyspaceGauge("MemtableColumnsCount", new MetricValue()
         {
-            public Long value()
+            public Long getValue(ColumnFamilyMetrics metric)
             {
-                long total = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    total += cf.metric.memtableColumnsCount.value();
-                }
-                return total;
+                return metric.memtableColumnsCount.value();
             }
         });
-        memtableLiveDataSize = Metrics.newGauge(factory.createMetricName("MemtableLiveDataSize"), new Gauge<Long>()
+        memtableLiveDataSize = createKeyspaceGauge("MemtableLiveDataSize", new MetricValue()
         {
-            public Long value()
+            public Long getValue(ColumnFamilyMetrics metric)
             {
-                long total = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    total += cf.metric.memtableLiveDataSize.value();
-                }
-                return total;
+                return metric.memtableLiveDataSize.value();
             }
-        });
-        memtableOnHeapDataSize = Metrics.newGauge(factory.createMetricName("MemtableOnHeapDataSize"), new Gauge<Long>()
+        }); 
+        memtableOnHeapDataSize = createKeyspaceGauge("MemtableOnHeapDataSize", new MetricValue()
         {
-            public Long value()
+            public Long getValue(ColumnFamilyMetrics metric)
             {
-                long total = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    total += cf.metric.memtableOnHeapSize.value();
-                }
-                return total;
+                return metric.memtableOnHeapSize.value();
             }
         });
-        memtableOffHeapDataSize = Metrics.newGauge(factory.createMetricName("MemtableOffHeapDataSize"), new Gauge<Long>()
+        memtableOffHeapDataSize = createKeyspaceGauge("MemtableOffHeapDataSize", new MetricValue()
         {
-            public Long value()
+            public Long getValue(ColumnFamilyMetrics metric)
             {
-                long total = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    total += cf.metric.memtableOffHeapSize.value();
-                }
-                return total;
+                return metric.memtableOffHeapSize.value();
             }
         });
-        allMemtablesLiveDataSize = Metrics.newGauge(factory.createMetricName("AllMemtablesLiveDataSize"), new Gauge<Long>()
+        allMemtablesLiveDataSize = createKeyspaceGauge("AllMemtablesLiveDataSize", new MetricValue()
         {
-            public Long value()
+            public Long getValue(ColumnFamilyMetrics metric)
             {
-                long total = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    total += cf.metric.allMemtablesLiveDataSize.value();
-                }
-                return total;
+                return metric.allMemtablesLiveDataSize.value();
             }
         });
-        allMemtablesOnHeapDataSize = Metrics.newGauge(factory.createMetricName("AllMemtablesOnHeapDataSize"), new Gauge<Long>()
+        allMemtablesOnHeapDataSize = createKeyspaceGauge("AllMemtablesOnHeapDataSize", new MetricValue()
         {
-            public Long value()
+            public Long getValue(ColumnFamilyMetrics metric)
             {
-                long total = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    total += cf.metric.allMemtablesOnHeapSize.value();
-                }
-                return total;
+                return metric.allMemtablesOnHeapSize.value();
             }
         });
-        allMemtablesOffHeapDataSize = Metrics.newGauge(factory.createMetricName("AllMemtablesOffHeapDataSize"), new Gauge<Long>()
+        allMemtablesOffHeapDataSize = createKeyspaceGauge("AllMemtablesOffHeapDataSize", new MetricValue()
         {
-            public Long value()
+            public Long getValue(ColumnFamilyMetrics metric)
             {
-                long total = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    total += cf.metric.allMemtablesOffHeapSize.value();
-                }
-                return total;
+                return metric.allMemtablesOffHeapSize.value();
             }
         });
-        memtableSwitchCount = Metrics.newGauge(factory.createMetricName("MemtableSwitchCount"), new Gauge<Long>()
+        memtableSwitchCount = createKeyspaceGauge("MemtableSwitchCount", new MetricValue()
         {
-            public Long value()
+            public Long getValue(ColumnFamilyMetrics metric)
             {
-                long sum = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                    sum += cf.metric.memtableSwitchCount.count();
-                return sum;
+                return metric.memtableSwitchCount.count();
             }
         });
-        pendingCompactions = Metrics.newGauge(factory.createMetricName("PendingCompactions"), new Gauge<Integer>()
+        pendingCompactions = createKeyspaceGauge("PendingCompactions", new MetricValue()
         {
-            public Integer value()
+            public Long getValue(ColumnFamilyMetrics metric)
             {
-                int sum = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    sum += cf.metric.pendingCompactions.value();
-                }
-                return sum;
+                return (long) metric.pendingCompactions.value();
             }
         });
-        pendingFlushes = Metrics.newGauge(factory.createMetricName("PendingFlushes"), new Gauge<Integer>()
+        pendingFlushes = createKeyspaceGauge("PendingFlushes", new MetricValue()
         {
-            public Integer value()
+            public Long getValue(ColumnFamilyMetrics metric)
             {
-                int sum = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    sum += cf.metric.pendingFlushes.count();
-                }
-                return sum;
+                return (long) metric.pendingFlushes.count();
             }
         });
-        liveDiskSpaceUsed = Metrics.newGauge(factory.createMetricName("LiveDiskSpaceUsed"), new Gauge<Long>()
+        liveDiskSpaceUsed = createKeyspaceGauge("LiveDiskSpaceUsed", new MetricValue()
         {
-            public Long value()
+            public Long getValue(ColumnFamilyMetrics metric)
             {
-                long sum = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    sum += cf.metric.liveDiskSpaceUsed.count();
-                }
-                return sum;
+                return metric.liveDiskSpaceUsed.count();
             }
         });
-        totalDiskSpaceUsed = Metrics.newGauge(factory.createMetricName("TotalDiskSpaceUsed"), new Gauge<Long>()
+        totalDiskSpaceUsed = createKeyspaceGauge("TotalDiskSpaceUsed", new MetricValue()
         {
-            public Long value()
+            public Long getValue(ColumnFamilyMetrics metric)
             {
-                long sum = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    sum += cf.metric.totalDiskSpaceUsed.count();
-                }
-                return sum;
+                return metric.totalDiskSpaceUsed.count();
             }
         });
-        bloomFilterDiskSpaceUsed = Metrics.newGauge(factory.createMetricName("BloomFilterDiskSpaceUsed"), new Gauge<Long>()
+        bloomFilterDiskSpaceUsed = createKeyspaceGauge("BloomFilterDiskSpaceUsed", new MetricValue()
         {
-            public Long value()
+            public Long getValue(ColumnFamilyMetrics metric)
             {
-                long total = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                    total += cf.metric.bloomFilterDiskSpaceUsed.value();
-                return total;
+                return metric.bloomFilterDiskSpaceUsed.value();
             }
         });
+        // latency metrics for ColumnFamilyMetrics to update
+        readLatency = new LatencyMetrics(factory, "Read");
+        writeLatency = new LatencyMetrics(factory, "Write");
+        rangeLatency = new LatencyMetrics(factory, "Range");
+        // create histograms for ColumnFamilyMetrics to replicate updates to
+        sstablesPerReadHistogram = Metrics.newHistogram(factory.createMetricName("SSTablesPerReadHistogram"), true);
+        tombstoneScannedHistogram = Metrics.newHistogram(factory.createMetricName("TombstoneScannedHistogram"), true);
+        liveScannedHistogram = Metrics.newHistogram(factory.createMetricName("LiveScannedHistogram"), true);
+        // add manually since histograms do not use createKeyspaceGauge method
+        allMetrics.addAll(Lists.newArrayList("SSTablesPerReadHistogram", "TombstoneScannedHistogram", "LiveScannedHistogram"));
     }
 
     /**
@@ -229,19 +194,50 @@ public class KeyspaceMetrics
      */
     public void release()
     {
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("AllMemtablesLiveDataSize"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("AllMemtablesOnHeapDataSize"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("AllMemtablesOffHeapDataSize"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableLiveDataSize"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableOnHeapDataSize"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableOffHeapDataSize"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableColumnsCount"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableSwitchCount"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("PendingFlushes"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("PendingCompactions"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("LiveDiskSpaceUsed"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("TotalDiskSpaceUsed"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("BloomFilterDiskSpaceUsed"));
+        for(String name : allMetrics) 
+        {
+            Metrics.defaultRegistry().removeMetric(factory.createMetricName(name));
+        }
+        // latency metrics contain multiple metrics internally and need to be released manually
+        readLatency.release();
+        writeLatency.release();
+        rangeLatency.release();
+    }
+    
+    /**
+     * Represents a column family metric value.
+     */
+    private interface MetricValue
+    {
+        /**
+         * get value of a metric
+         * @param columnfamilymetrics of a column family in this keyspace
+         * @return current value of a metric
+         */
+        public Long getValue(ColumnFamilyMetrics metric);
+    }
+
+    /**
+     * Creates a gauge that will sum the current value of a metric for all column families in this keyspace
+     * @param name
+     * @param MetricValue 
+     * @return Gauge&gt;Long> that computes sum of MetricValue.getValue()
+     */
+    private <T extends Number> Gauge<Long> createKeyspaceGauge(String name, final MetricValue extractor)
+    {
+        allMetrics.add(name);
+        return Metrics.newGauge(factory.createMetricName(name), new Gauge<Long>()
+        {
+            public Long value()
+            {
+                long sum = 0;
+                for (ColumnFamilyStore cf : keyspace.getColumnFamilyStores())
+                {
+                    sum += extractor.getValue(cf.metric);
+                }
+                return sum;
+            }
+        });
     }
 
     class KeyspaceMetricNameFactory implements MetricNameFactory

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b8ac8cb/src/java/org/apache/cassandra/metrics/LatencyMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/LatencyMetrics.java b/src/java/org/apache/cassandra/metrics/LatencyMetrics.java
index b456e2a..28256fe 100644
--- a/src/java/org/apache/cassandra/metrics/LatencyMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/LatencyMetrics.java
@@ -17,14 +17,17 @@
  */
 package org.apache.cassandra.metrics;
 
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.cassandra.utils.EstimatedHistogram;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 import com.yammer.metrics.Metrics;
 import com.yammer.metrics.core.Counter;
 import com.yammer.metrics.core.Timer;
 
-import org.apache.cassandra.utils.EstimatedHistogram;
-
 /**
  * Metrics about latencies
  */
@@ -35,6 +38,9 @@ public class LatencyMetrics
     /** Total latency in micro sec */
     public final Counter totalLatency;
 
+    /** parent metrics to replicate any updates to **/
+    private List<LatencyMetrics> parents = Lists.newArrayList();
+    
     protected final MetricNameFactory factory;
     protected final String namePrefix;
 
@@ -80,6 +86,20 @@ public class LatencyMetrics
         latency = Metrics.newTimer(factory.createMetricName(namePrefix + "Latency"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
         totalLatency = Metrics.newCounter(factory.createMetricName(namePrefix + "TotalLatency"));
     }
+    
+    /**
+     * Create LatencyMetrics with given group, type, prefix to append to each metric name, and scope.  Any updates
+     * to this will also run on parent
+     *
+     * @param factory MetricName factory to use
+     * @param namePrefix Prefix to append to each metric name
+     * @param parents... any amount of parents to replicate updates to
+     */
+    public LatencyMetrics(MetricNameFactory factory, String namePrefix, LatencyMetrics ... parents)
+    {
+        this(factory, namePrefix);
+        this.parents.addAll(ImmutableList.copyOf(parents));
+    }
 
     /** takes nanoseconds **/
     public void addNano(long nanos)
@@ -89,6 +109,10 @@ public class LatencyMetrics
         totalLatency.inc(nanos / 1000);
         totalLatencyHistogram.add(nanos / 1000);
         recentLatencyHistogram.add(nanos / 1000);
+        for(LatencyMetrics parent : parents)
+        {
+            parent.addNano(nanos);
+        }
     }
 
     public void release()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5b8ac8cb/test/unit/org/apache/cassandra/db/KeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyspaceTest.java b/test/unit/org/apache/cassandra/db/KeyspaceTest.java
index 12e6d9f..d610563 100644
--- a/test/unit/org/apache/cassandra/db/KeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyspaceTest.java
@@ -445,18 +445,18 @@ public class KeyspaceTest extends SchemaLoader
             rm.apply();
             cfStore.forceBlockingFlush();
         }
-        cfStore.metric.sstablesPerReadHistogram.clear();
+        cfStore.metric.sstablesPerReadHistogram.cf.clear();
         ColumnFamily cf = cfStore.getColumnFamily(key, Composites.EMPTY, cellname("col1499"), false, 1000, System.currentTimeMillis());
-        assertEquals(cfStore.metric.sstablesPerReadHistogram.max(), 5, 0.1);
+        assertEquals(cfStore.metric.sstablesPerReadHistogram.cf.max(), 5, 0.1);
         int i = 0;
         for (Cell c : cf.getSortedColumns())
         {
             assertEquals(ByteBufferUtil.string(c.name().toByteBuffer()), "col" + (1000 + i++));
         }
         assertEquals(i, 500);
-        cfStore.metric.sstablesPerReadHistogram.clear();
+        cfStore.metric.sstablesPerReadHistogram.cf.clear();
         cf = cfStore.getColumnFamily(key, cellname("col1500"), cellname("col2000"), false, 1000, System.currentTimeMillis());
-        assertEquals(cfStore.metric.sstablesPerReadHistogram.max(), 5, 0.1);
+        assertEquals(cfStore.metric.sstablesPerReadHistogram.cf.max(), 5, 0.1);
 
         for (Cell c : cf.getSortedColumns())
         {
@@ -465,9 +465,9 @@ public class KeyspaceTest extends SchemaLoader
         assertEquals(i, 1000);
 
         // reverse
-        cfStore.metric.sstablesPerReadHistogram.clear();
+        cfStore.metric.sstablesPerReadHistogram.cf.clear();
         cf = cfStore.getColumnFamily(key, cellname("col2000"), cellname("col1500"), true, 1000, System.currentTimeMillis());
-        assertEquals(cfStore.metric.sstablesPerReadHistogram.max(), 5, 0.1);
+        assertEquals(cfStore.metric.sstablesPerReadHistogram.cf.max(), 5, 0.1);
         i = 500;
         for (Cell c : cf.getSortedColumns())
         {
@@ -514,13 +514,13 @@ public class KeyspaceTest extends SchemaLoader
         }
         Composite start = type.builder().add(ByteBufferUtil.bytes("a5")).add(ByteBufferUtil.bytes(85)).build();
         Composite finish = type.builder().add(ByteBufferUtil.bytes("a5")).build().end();
-        cfs.metric.sstablesPerReadHistogram.clear();
+        cfs.metric.sstablesPerReadHistogram.cf.clear();
         ColumnFamily cf = cfs.getColumnFamily(key, start, finish, false, 1000, System.currentTimeMillis());
         int colCount = 0;
         for (Cell c : cf)
             colCount++;
         assertEquals(2, colCount);
-        assertEquals(2, cfs.metric.sstablesPerReadHistogram.max(), 0.1);
+        assertEquals(2, cfs.metric.sstablesPerReadHistogram.cf.max(), 0.1);
     }
 
     private void validateSliceLarge(ColumnFamilyStore cfStore) throws IOException