You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by yu...@apache.org on 2013/11/05 23:00:36 UTC

git commit: Use of o.a.c.metrics in nodetool

Updated Branches:
  refs/heads/trunk 82c0094a8 -> a4fc13c05


Use of o.a.c.metrics in nodetool

patch by Lyuben Todorov; reviewed by yukim for CASSANDRA-5871


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

Branch: refs/heads/trunk
Commit: a4fc13c052fd816585bcb15793495ca74643aa2c
Parents: 82c0094
Author: Lyuben Todorov <lt...@datastax.com>
Authored: Tue Nov 5 12:15:07 2013 -0600
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Nov 5 15:54:55 2013 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +-
 .../org/apache/cassandra/tools/NodeCmd.java     | 173 ++++++++++--------
 .../org/apache/cassandra/tools/NodeProbe.java   | 179 +++++++++++++++++++
 3 files changed, 282 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4fc13c0/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f1be530..045fb34 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,7 +7,8 @@
  * Remove leveled json manifest migration code (CASSANDRA-5996)
  * Remove CFDefinition (CASSANDRA-6253)
  * Use AtomicIntegerFieldUpdater in RefCountedMemory (CASSANDRA-6278)
- * User-defined types for CQL3 (5590)
+ * User-defined types for CQL3 (CASSANDRA-5590)
+ * Use of o.a.c.metrics in nodetool (CASSANDRA-5871)
 
 
 2.0.3

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4fc13c0/src/java/org/apache/cassandra/tools/NodeCmd.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeCmd.java b/src/java/org/apache/cassandra/tools/NodeCmd.java
index 034ff29..c629e0d 100644
--- a/src/java/org/apache/cassandra/tools/NodeCmd.java
+++ b/src/java/org/apache/cassandra/tools/NodeCmd.java
@@ -32,7 +32,10 @@ import javax.management.openmbean.TabularData;
 import com.google.common.base.Joiner;
 import com.google.common.collect.LinkedHashMultimap;
 import com.google.common.collect.Maps;
+import com.yammer.metrics.reporting.JmxReporter;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.metrics.StorageMetrics;
+import org.apache.cassandra.utils.EstimatedHistogram;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.commons.cli.*;
 import org.yaml.snakeyaml.Yaml;
@@ -47,11 +50,9 @@ import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
 import org.apache.cassandra.net.MessagingServiceMBean;
 import org.apache.cassandra.service.CacheServiceMBean;
-import org.apache.cassandra.service.StorageProxyMBean;
 import org.apache.cassandra.streaming.StreamState;
 import org.apache.cassandra.streaming.ProgressInfo;
 import org.apache.cassandra.streaming.SessionInfo;
-import org.apache.cassandra.utils.EstimatedHistogram;
 import org.apache.cassandra.utils.Pair;
 
 public class NodeCmd
@@ -605,28 +606,30 @@ public class NodeCmd
         outs.printf("%-17s: %s%n", "Rack", probe.getRack());
 
         // Exceptions
-        outs.printf("%-17s: %s%n", "Exceptions", probe.getExceptionCount());
+        outs.printf("%-17s: %s%n", "Exceptions", probe.getStorageMetric("Exceptions"));
 
         CacheServiceMBean cacheService = probe.getCacheServiceMBean();
 
         // Key Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
-        outs.printf("%-17s: size %d (bytes), capacity %d (bytes), %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
+        outs.printf("%-17s: entries %d, size %d (bytes), capacity %d (bytes), %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
                     "Key Cache",
-                    cacheService.getKeyCacheSize(),
-                    cacheService.getKeyCacheCapacityInBytes(),
-                    cacheService.getKeyCacheHits(),
-                    cacheService.getKeyCacheRequests(),
-                    cacheService.getKeyCacheRecentHitRate(),
+                    probe.getCacheMetric("KeyCache", "Entries"),
+                    probe.getCacheMetric("KeyCache", "Size"),
+                    probe.getCacheMetric("KeyCache", "Capacity"),
+                    probe.getCacheMetric("KeyCache", "Hits"),
+                    probe.getCacheMetric("KeyCache", "Requests"),
+                    probe.getCacheMetric("KeyCache", "HitRate"),
                     cacheService.getKeyCacheSavePeriodInSeconds());
 
         // Row Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
-        outs.printf("%-17s: size %d (bytes), capacity %d (bytes), %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
+        outs.printf("%-17s: entries %d, size %d (bytes), capacity %d (bytes), %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
                     "Row Cache",
-                    cacheService.getRowCacheSize(),
-                    cacheService.getRowCacheCapacityInBytes(),
-                    cacheService.getRowCacheHits(),
-                    cacheService.getRowCacheRequests(),
-                    cacheService.getRowCacheRecentHitRate(),
+                    probe.getCacheMetric("RowCache", "Entries"),
+                    probe.getCacheMetric("RowCache", "Size"),
+                    probe.getCacheMetric("RowCache", "Capacity"),
+                    probe.getCacheMetric("RowCache", "Hits"),
+                    probe.getCacheMetric("RowCache", "Requests"),
+                    probe.getCacheMetric("RowCache", "HitRate"),
                     cacheService.getRowCacheSavePeriodInSeconds());
 
         if (toks.size() > 1 && cmd.hasOption(TOKENS_OPT.left))
@@ -704,7 +707,7 @@ public class NodeCmd
     {
         int compactionThroughput = probe.getCompactionThroughput();
         CompactionManagerMBean cm = probe.getCompactionManagerProxy();
-        outs.println("pending tasks: " + cm.getPendingTasks());
+        outs.println("pending tasks: " + probe.getCompactionMetric("PendingTasks"));
         if (cm.getCompactions().size() > 0)
             outs.printf("%25s%16s%16s%16s%16s%10s%10s%n", "compaction type", "keyspace", "table", "completed", "total", "unit", "progress");
         long remainingBytes = 0;
@@ -827,20 +830,21 @@ public class NodeCmd
             outs.println("Keyspace: " + keyspaceName);
             for (ColumnFamilyStoreMBean cfstore : columnFamilies)
             {
-                long writeCount = cfstore.getWriteCount();
-                long readCount = cfstore.getReadCount();
+                String cfName = cfstore.getColumnFamilyName();
+                long writeCount = ((JmxReporter.TimerMBean)probe.getColumnFamilyMetric(keyspaceName, cfName, "WriteLatency")).getCount();
+                long readCount = ((JmxReporter.TimerMBean)probe.getColumnFamilyMetric(keyspaceName, cfName, "ReadLatency")).getCount();
 
                 if (readCount > 0)
                 {
                     keyspaceReadCount += readCount;
-                    keyspaceTotalReadTime += cfstore.getTotalReadLatencyMicros();
+                    keyspaceTotalReadTime += (long)probe.getColumnFamilyMetric(keyspaceName, cfName, "ReadTotalLatency");
                 }
                 if (writeCount > 0)
                 {
                     keyspaceWriteCount += writeCount;
-                    keyspaceTotalWriteTime += cfstore.getTotalWriteLatencyMicros();
+                    keyspaceTotalWriteTime += (long)probe.getColumnFamilyMetric(keyspaceName, cfName, "WriteTotalLatency");
                 }
-                keyspacePendingTasks += cfstore.getPendingTasks();
+                keyspacePendingTasks += (int)probe.getColumnFamilyMetric(keyspaceName, cfName, "PendingTasks");
             }
 
             double keyspaceReadLatency = keyspaceReadCount > 0 ? keyspaceTotalReadTime / keyspaceReadCount / 1000 : Double.NaN;
@@ -861,7 +865,8 @@ public class NodeCmd
                 else
                     outs.println("\t\tTable: " + cfName);
 
-                outs.println("\t\tSSTable count: " + cfstore.getLiveSSTableCount());
+                outs.println("\t\tSSTable count: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "LiveSSTableCount"));
+
                 int[] leveledSStables = cfstore.getSSTableCountPerLevel();
                 if (leveledSStables != null)
                 {
@@ -883,26 +888,29 @@ public class NodeCmd
                             outs.println("]");
                     }
                 }
-                outs.println("\t\tSpace used (live), bytes: " + cfstore.getLiveDiskSpaceUsed());
-                outs.println("\t\tSpace used (total), bytes: " + cfstore.getTotalDiskSpaceUsed());
-                outs.println("\t\tSSTable Compression Ratio: " + cfstore.getCompressionRatio());
-                outs.println("\t\tNumber of keys (estimate): " + cfstore.estimateKeys());
-                outs.println("\t\tMemtable cell count: " + cfstore.getMemtableColumnsCount());
-                outs.println("\t\tMemtable data size, bytes: " + cfstore.getMemtableDataSize());
-                outs.println("\t\tMemtable switch count: " + cfstore.getMemtableSwitchCount());
-                outs.println("\t\tLocal read count: " + cfstore.getReadCount());
-                outs.printf("\t\tLocal read latency: %01.3f ms%n", cfstore.getRecentReadLatencyMicros() / 1000);
-                outs.println("\t\tLocal write count: " + cfstore.getWriteCount());
-                outs.printf("\t\tLocal write latency: %01.3f ms%n", cfstore.getRecentWriteLatencyMicros() / 1000);
-                outs.println("\t\tPending tasks: " + cfstore.getPendingTasks());
-                outs.println("\t\tBloom filter false positives: " + cfstore.getBloomFilterFalsePositives());
-                outs.println("\t\tBloom filter false ratio: " + String.format("%01.5f", cfstore.getRecentBloomFilterFalseRatio()));
-                outs.println("\t\tBloom filter space used, bytes: " + cfstore.getBloomFilterDiskSpaceUsed());
-                outs.println("\t\tCompacted partition minimum bytes: " + cfstore.getMinRowSize());
-                outs.println("\t\tCompacted partition maximum bytes: " + cfstore.getMaxRowSize());
-                outs.println("\t\tCompacted partition mean bytes: " + cfstore.getMeanRowSize());
-                outs.println("\t\tAverage live cells per slice (last five minutes): " + cfstore.getLiveCellsPerSlice());
-                outs.println("\t\tAverage tombstones per slice (last five minutes): " + cfstore.getTombstonesPerSlice());
+                outs.println("\t\tSpace used (live), bytes: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "LiveDiskSpaceUsed"));
+                outs.println("\t\tSpace used (total), bytes: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "TotalDiskSpaceUsed"));
+                outs.println("\t\tSSTable Compression Ratio: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "CompressionRatio"));
+                outs.println("\t\tMemtable cell count: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "MemtableColumnsCount"));
+                outs.println("\t\tMemtable data size, bytes: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "MemtableDataSize"));
+                outs.println("\t\tMemtable switch count: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "MemtableSwitchCount"));
+                outs.println("\t\tLocal read count: " + ((JmxReporter.TimerMBean)probe.getColumnFamilyMetric(keyspaceName, cfName, "ReadLatency")).getCount());
+                double localReadLatency = ((JmxReporter.TimerMBean)probe.getColumnFamilyMetric(keyspaceName, cfName, "ReadLatency")).getMean() / 1000;
+                double localRLatency = localReadLatency > 0 ? localReadLatency : Double.NaN;
+                outs.printf("\t\tLocal read latency: %01.3f ms%n", localRLatency);
+                outs.println("\t\tLocal write count: " + ((JmxReporter.TimerMBean)probe.getColumnFamilyMetric(keyspaceName, cfName, "WriteLatency")).getCount());
+                double localWriteLatency = ((JmxReporter.TimerMBean)probe.getColumnFamilyMetric(keyspaceName, cfName, "WriteLatency")).getMean() / 1000;
+                double localWLatency = localWriteLatency > 0 ? localWriteLatency : Double.NaN;
+                outs.printf("\t\tLocal write latency: %01.3f ms%n", localWLatency);
+                outs.println("\t\tPending tasks: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "PendingTasks"));
+                outs.println("\t\tBloom filter false positives: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "BloomFilterFalsePositives"));
+                outs.println("\t\tBloom filter false ratio: " + String.format("%01.5f", probe.getColumnFamilyMetric(keyspaceName, cfName, "RecentBloomFilterFalseRatio")));
+                outs.println("\t\tBloom filter space used, bytes: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "BloomFilterDiskSpaceUsed"));
+                outs.println("\t\tCompacted partition minimum bytes: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "MinRowSize"));
+                outs.println("\t\tCompacted partition maximum bytes: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "MaxRowSize"));
+                outs.println("\t\tCompacted partition mean bytes: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "MeanRowSize"));
+                outs.println("\t\tAverage live cells per slice (last five minutes): " + ((JmxReporter.HistogramMBean)probe.getColumnFamilyMetric(keyspaceName, cfName, "LiveScannedHistogram")).getMean());
+                outs.println("\t\tAverage tombstones per slice (last five minutes): " + ((JmxReporter.HistogramMBean)probe.getColumnFamilyMetric(keyspaceName, cfName, "TombstoneScannedHistogram")).getMean());
 
                 outs.println("");
             }
@@ -917,55 +925,76 @@ public class NodeCmd
 
     private void printCfHistograms(String keySpace, String columnFamily, PrintStream output)
     {
-        ColumnFamilyStoreMBean store = this.probe.getCfsProxy(keySpace, columnFamily);
+        // calculate percentile of row size and column count
+        long[] estimatedRowSize = (long[]) probe.getColumnFamilyMetric(keySpace, columnFamily, "EstimatedRowSizeHistogram");
+        long[] estimatedColumnCount = (long[]) probe.getColumnFamilyMetric(keySpace, columnFamily, "EstimatedColumnCountHistogram");
+
+        long[] bucketOffsets = new EstimatedHistogram().getBucketOffsets();
+        EstimatedHistogram rowSizeHist = new EstimatedHistogram(bucketOffsets, estimatedRowSize);
+        EstimatedHistogram columnCountHist = new EstimatedHistogram(bucketOffsets, estimatedColumnCount);
+
+        // build arrays to store percentile values
+        double[] estimatedRowSizePercentiles = new double[7];
+        double[] estimatedColumnCountPercentiles = new double[7];
+        double[] offsetPercentiles = new double[]{0.5, 0.75, 0.95, 0.98, 0.99};
+        for (int i = 0; i < offsetPercentiles.length; i++)
+        {
+            estimatedRowSizePercentiles[i] = rowSizeHist.percentile(offsetPercentiles[i]);
+            estimatedColumnCountPercentiles[i] = columnCountHist.percentile(offsetPercentiles[i]);
+        }
 
-        // default is 90 offsets
-        long[] offsets = new EstimatedHistogram().getBucketOffsets();
+        // min value
+        estimatedRowSizePercentiles[5] = rowSizeHist.min();
+        estimatedColumnCountPercentiles[5] = columnCountHist.min();
+        // max value
+        estimatedRowSizePercentiles[6] = rowSizeHist.max();
+        estimatedColumnCountPercentiles[6] = columnCountHist.max();
 
-        long[] rrlh = store.getRecentReadLatencyHistogramMicros();
-        long[] rwlh = store.getRecentWriteLatencyHistogramMicros();
-        long[] sprh = store.getRecentSSTablesPerReadHistogram();
-        long[] ersh = store.getEstimatedRowSizeHistogram();
-        long[] ecch = store.getEstimatedColumnCountHistogram();
+        String[] percentiles = new String[]{ "50%", "75%", "95%", "98%", "99%", "Min", "Max" };
+        double[] readLatency = probe.metricPercentilesAsArray((JmxReporter.HistogramMBean)probe.getColumnFamilyMetric(keySpace, columnFamily, "ReadLatency"));
+        double[] writeLatency = probe.metricPercentilesAsArray((JmxReporter.TimerMBean)probe.getColumnFamilyMetric(keySpace, columnFamily, "WriteLatency"));
+        double[] sstablesPerRead = probe.metricPercentilesAsArray((JmxReporter.HistogramMBean)probe.getColumnFamilyMetric(keySpace, columnFamily, "SSTablesPerReadHistogram"));
 
         output.println(String.format("%s/%s histograms", keySpace, columnFamily));
-
         output.println(String.format("%-10s%10s%18s%18s%18s%18s",
-                                     "Offset", "SSTables", "Write Latency", "Read Latency", "Partition Size", "Cell Count"));
+                                     "Percentile", "SSTables", "Write Latency", "Read Latency", "Partition Size", "Cell Count"));
         output.println(String.format("%-10s%10s%18s%18s%18s%18s",
                                      "", "", "(micros)", "(micros)", "(bytes)", ""));
 
-        for (int i = 0; i < offsets.length; i++)
+        for (int i = 0; i < percentiles.length; i++)
         {
-            output.println(String.format("%-10d%10s%18s%18s%18s%18s",
-                                         offsets[i],
-                                         (i < sprh.length ? sprh[i] : "0"),
-                                         (i < rwlh.length ? rwlh[i] : "0"),
-                                         (i < rrlh.length ? rrlh[i] : "0"),
-                                         (i < ersh.length ? ersh[i] : "0"),
-                                         (i < ecch.length ? ecch[i] : "0")));
+            output.println(String.format("%-10s%10.2f%18.2f%18.2f%18.0f%18.0f",
+                                         percentiles[i],
+                                         sstablesPerRead[i],
+                                         writeLatency[i],
+                                         readLatency[i],
+                                         estimatedRowSizePercentiles[i],
+                                         estimatedColumnCountPercentiles[i]));
         }
+        output.println();
     }
 
     private void printProxyHistograms(PrintStream output)
     {
-        StorageProxyMBean sp = this.probe.getSpProxy();
-        long[] offsets = new EstimatedHistogram().getBucketOffsets();
-        long[] rrlh = sp.getRecentReadLatencyHistogramMicros();
-        long[] rwlh = sp.getRecentWriteLatencyHistogramMicros();
-        long[] rrnglh = sp.getRecentRangeLatencyHistogramMicros();
+        String[] percentiles = new String[]{ "50%", "75%", "95%", "98%", "99%", "Min", "Max" };
+        double[] readLatency = probe.metricPercentilesAsArray(probe.getProxyMetric("Read"));
+        double[] writeLatency = probe.metricPercentilesAsArray(probe.getProxyMetric("Write"));
+        double[] rangeLatency = probe.metricPercentilesAsArray(probe.getProxyMetric("RangeSlice"));
 
         output.println("proxy histograms");
         output.println(String.format("%-10s%18s%18s%18s",
-                                    "Offset", "Read Latency", "Write Latency", "Range Latency"));
-        for (int i = 0; i < offsets.length; i++)
+                                     "Percentile", "Read Latency", "Write Latency", "Range Latency"));
+        output.println(String.format("%-10s%18s%18s%18s",
+                                     "", "(micros)", "(micros)", "(micros)"));
+        for (int i = 0; i < percentiles.length; i++)
         {
-            output.println(String.format("%-10d%18s%18s%18s",
-                                        offsets[i],
-                                        (i < rrlh.length ? rrlh[i] : "0"),
-                                        (i < rwlh.length ? rwlh[i] : "0"),
-                                        (i < rrnglh.length ? rrnglh[i] : "0")));
+            output.println(String.format("%-10s%18.2f%18.2f%18.2f",
+                                         percentiles[i],
+                                         readLatency[i],
+                                         writeLatency[i],
+                                         rangeLatency[i]));
         }
+        output.println();
     }
 
     private void printEndPoints(String keySpace, String cf, String key, PrintStream output)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a4fc13c0/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 0008325..48d2013 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -43,6 +43,7 @@ import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 
+import com.yammer.metrics.reporting.JmxReporter;
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
 import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 import org.apache.cassandra.db.HintedHandOffManager;
@@ -871,6 +872,184 @@ public class NodeProbe
         return spProxy.getReadRepairRepairedBackground();
     }
 
+    // JMX getters for the o.a.c.metrics API below.
+    /**
+     * Retrieve cache metrics based on the cache type (KeyCache or RowCache)
+     * @param cacheType KeyCache or RowCache
+     * @param metricName Capacity, Entries, HitRate, Size, Requests or Hits.
+     */
+    public Object getCacheMetric(String cacheType, String metricName)
+    {
+        try
+        {
+            switch(metricName)
+            {
+                case "Capacity":
+                case "Entries":
+                case "HitRate":
+                case "Size":
+                    return JMX.newMBeanProxy(mbeanServerConn,
+                            new ObjectName("org.apache.cassandra.metrics:type=Cache,scope=" + cacheType + ",name=" + metricName),
+                            JmxReporter.GaugeMBean.class).getValue();
+                case "Requests":
+                case "Hits":
+                    return JMX.newMBeanProxy(mbeanServerConn,
+                            new ObjectName("org.apache.cassandra.metrics:type=Cache,scope=" + cacheType + ",name=" + metricName),
+                            JmxReporter.MeterMBean.class).getCount();
+                default:
+                    throw new RuntimeException("Unknown cache metric name.");
+
+            }
+        }
+        catch (MalformedObjectNameException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Retrieve ColumnFamily metrics
+     * @param ks Keyspace for which stats are to be displayed.
+     * @param cf ColumnFamily for which stats are to be displayed.
+     * @param metricName View {@link org.apache.cassandra.metrics.ColumnFamilyMetrics}.
+     */
+    public Object getColumnFamilyMetric(String ks, String cf, String metricName)
+    {
+        try
+        {
+            switch(metricName)
+            {
+                case "BloomFilterDiskSpaceUsed":
+                case "BloomFilterFalsePositives":
+                case "BloomFilterFalseRatio":
+                case "CompressionRatio":
+                case "EstimatedColumnCountHistogram":
+                case "EstimatedRowSizeHistogram":
+                case "KeyCacheHitRate":
+                case "LiveSSTableCount":
+                case "MaxRowSize":
+                case "MeanRowSize":
+                case "MemtableColumnsCount":
+                case "MemtableDataSize":
+                case "MinRowSize":
+                case "PendingTasks":
+                case "RecentBloomFilterFalsePositives":
+                case "RecentBloomFilterFalseRatio":
+                    return JMX.newMBeanProxy(mbeanServerConn,
+                            new ObjectName("org.apache.cassandra.metrics:type=ColumnFamily,keyspace=" + ks + ",scope=" + cf + ",name=" + metricName),
+                            JmxReporter.GaugeMBean.class).getValue();
+                case "LiveDiskSpaceUsed":
+                case "MemtableSwitchCount":
+                case "SpeculativeRetries":
+                case "TotalDiskSpaceUsed":
+                case "WriteTotalLatency":
+                case "ReadTotalLatency":
+                    return JMX.newMBeanProxy(mbeanServerConn,
+                            new ObjectName("org.apache.cassandra.metrics:type=ColumnFamily,keyspace=" + ks + ",scope=" + cf + ",name=" + metricName),
+                            JmxReporter.CounterMBean.class).getCount();
+                case "ReadLatency":
+                case "CoordinatorReadLatency":
+                case "CoordinatorScanLatency":
+                case "WriteLatency":
+                    return JMX.newMBeanProxy(mbeanServerConn,
+                            new ObjectName("org.apache.cassandra.metrics:type=ColumnFamily,keyspace=" + ks + ",scope=" + cf + ",name=" + metricName),
+                            JmxReporter.TimerMBean.class);
+                case "LiveScannedHistogram":
+                case "SSTablesPerReadHistogram":
+                case "TombstoneScannedHistogram":
+                    return JMX.newMBeanProxy(mbeanServerConn,
+                            new ObjectName("org.apache.cassandra.metrics:type=ColumnFamily,keyspace=" + ks + ",scope=" + cf + ",name=" + metricName),
+                            JmxReporter.HistogramMBean.class);
+                default:
+                    throw new RuntimeException("Unknown column family metric.");
+            }
+        }
+        catch (MalformedObjectNameException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Retrieve Proxy metrics
+     * @param scope; RangeSlice, Read or Write
+     */
+    public JmxReporter.TimerMBean getProxyMetric(String scope)
+    {
+        try
+        {
+            return JMX.newMBeanProxy(mbeanServerConn,
+                    new ObjectName("org.apache.cassandra.metrics:type=ClientRequest,scope=" + scope + ",name=Latency"),
+                    JmxReporter.TimerMBean.class);
+        }
+        catch (MalformedObjectNameException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Retrieve Proxy metrics
+     * @param metricName; CompletedTasks, PendingTasks, BytesCompacted or TotalCompactionsCompleted.
+     */
+    public Object getCompactionMetric(String metricName)
+    {
+        try
+        {
+            switch(metricName)
+            {
+                case "BytesCompacted":
+                    return JMX.newMBeanProxy(mbeanServerConn,
+                            new ObjectName("org.apache.cassandra.metrics:type=Compaction,name=" + metricName),
+                            JmxReporter.CounterMBean.class);
+                case "CompletedTasks":
+                case "PendingTasks":
+                    return JMX.newMBeanProxy(mbeanServerConn,
+                            new ObjectName("org.apache.cassandra.metrics:type=Compaction,name=" + metricName),
+                            JmxReporter.GaugeMBean.class).getValue();
+                case "TotalCompactionsCompleted":
+                    return JMX.newMBeanProxy(mbeanServerConn,
+                            new ObjectName("org.apache.cassandra.metrics:type=Compaction,name=" + metricName),
+                            JmxReporter.MeterMBean.class);
+                default:
+                    throw new RuntimeException("Unknown compaction metric.");
+            }
+        }
+        catch (MalformedObjectNameException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Retrieve Proxy metrics
+     * @param metricName; Exceptions, Load, TotalHints or TotalHintsInProgress.
+     */
+    public long getStorageMetric(String metricName)
+    {
+        try
+        {
+            return JMX.newMBeanProxy(mbeanServerConn,
+                    new ObjectName("org.apache.cassandra.metrics:type=Storage,name=" + metricName),
+                    JmxReporter.CounterMBean.class).getCount();
+        }
+        catch (MalformedObjectNameException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public double[] metricPercentilesAsArray(JmxReporter.HistogramMBean metric)
+    {
+        return new double[]{ metric.get50thPercentile(),
+                metric.get75thPercentile(),
+                metric.get95thPercentile(),
+                metric.get98thPercentile(),
+                metric.get99thPercentile(),
+                metric.getMin(),
+                metric.getMax()};
+    }
+
     public TabularData getCompactionHistory()
     {
         return compactionProxy.getCompactionHistory();