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 2015/07/16 20:48:35 UTC

[2/3] cassandra git commit: Metrics should use up to date nomenclature

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0bd5170c/src/java/org/apache/cassandra/metrics/TableMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java b/src/java/org/apache/cassandra/metrics/TableMetrics.java
new file mode 100644
index 0000000..d708ac4
--- /dev/null
+++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java
@@ -0,0 +1,837 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.metrics;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.ConcurrentMap;
+
+import com.google.common.collect.Maps;
+
+import com.codahale.metrics.*;
+import com.codahale.metrics.Timer;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.utils.EstimatedHistogram;
+import org.apache.cassandra.utils.TopKSampler;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
+
+/**
+ * Metrics for {@link ColumnFamilyStore}.
+ */
+public class TableMetrics
+{
+
+    /** Total amount of data stored in the memtable that resides on-heap, including column related overhead and partitions overwritten. */
+    public final Gauge<Long> memtableOnHeapSize;
+    /** Total amount of data stored in the memtable that resides off-heap, including column related overhead and partitions overwritten. */
+    public final Gauge<Long> memtableOffHeapSize;
+    /** Total amount of live data stored in the memtable, excluding any data structure overhead */
+    public final Gauge<Long> memtableLiveDataSize;
+    /** Total amount of data stored in the memtables (2i and pending flush memtables included) that resides on-heap. */
+    public final Gauge<Long> allMemtablesOnHeapSize;
+    /** Total amount of data stored in the memtables (2i and pending flush memtables included) that resides off-heap. */
+    public final Gauge<Long> allMemtablesOffHeapSize;
+    /** Total amount of live data stored in the memtables (2i and pending flush memtables included) that resides off-heap, excluding any data structure overhead */
+    public final Gauge<Long> allMemtablesLiveDataSize;
+    /** Total number of columns present in the memtable. */
+    public final Gauge<Long> memtableColumnsCount;
+    /** Number of times flush has resulted in the memtable being switched out. */
+    public final Counter memtableSwitchCount;
+    /** Current compression ratio for all SSTables */
+    public final Gauge<Double> compressionRatio;
+    /** Histogram of estimated partition size (in bytes). */
+    public final Gauge<long[]> estimatedPartitionSizeHistogram;
+    /** Approximate number of keys in table. */
+    public final Gauge<Long> estimatedPartitionCount;
+    /** Histogram of estimated number of columns. */
+    public final Gauge<long[]> estimatedColumnCountHistogram;
+    /** Histogram of the number of sstable data files accessed per read */
+    public final TableHistogram sstablesPerReadHistogram;
+    /** (Local) read metrics */
+    public final LatencyMetrics readLatency;
+    /** (Local) range slice metrics */
+    public final LatencyMetrics rangeLatency;
+    /** (Local) write metrics */
+    public final LatencyMetrics writeLatency;
+    /** Estimated number of tasks pending for this table */
+    public final Counter pendingFlushes;
+    /** Estimate of number of pending compactios for this table */
+    public final Gauge<Integer> pendingCompactions;
+    /** Number of SSTables on disk for this CF */
+    public final Gauge<Integer> liveSSTableCount;
+    /** Disk space used by SSTables belonging to this table */
+    public final Counter liveDiskSpaceUsed;
+    /** Total disk space used by SSTables belonging to this table, including obsolete ones waiting to be GC'd */
+    public final Counter totalDiskSpaceUsed;
+    /** Size of the smallest compacted partition */
+    public final Gauge<Long> minPartitionSize;
+    /** Size of the largest compacted partition */
+    public final Gauge<Long> maxPartitionSize;
+    /** Size of the smallest compacted partition */
+    public final Gauge<Long> meanPartitionSize;
+    /** Number of false positives in bloom filter */
+    public final Gauge<Long> bloomFilterFalsePositives;
+    /** Number of false positives in bloom filter from last read */
+    public final Gauge<Long> recentBloomFilterFalsePositives;
+    /** False positive ratio of bloom filter */
+    public final Gauge<Double> bloomFilterFalseRatio;
+    /** False positive ratio of bloom filter from last read */
+    public final Gauge<Double> recentBloomFilterFalseRatio;
+    /** Disk space used by bloom filter */
+    public final Gauge<Long> bloomFilterDiskSpaceUsed;
+    /** Off heap memory used by bloom filter */
+    public final Gauge<Long> bloomFilterOffHeapMemoryUsed;
+    /** Off heap memory used by index summary */
+    public final Gauge<Long> indexSummaryOffHeapMemoryUsed;
+    /** Off heap memory used by compression meta data*/
+    public final Gauge<Long> compressionMetadataOffHeapMemoryUsed;
+    /** Key cache hit rate  for this CF */
+    public final Gauge<Double> keyCacheHitRate;
+    /** Tombstones scanned in queries on this CF */
+    public final TableHistogram tombstoneScannedHistogram;
+    /** Live cells scanned in queries on this CF */
+    public final TableHistogram liveScannedHistogram;
+    /** Column update time delta on this CF */
+    public final TableHistogram colUpdateTimeDeltaHistogram;
+    /** Disk space used by snapshot files which */
+    public final Gauge<Long> trueSnapshotsSize;
+    /** Partition cache hits, but result out of range */
+    public final Counter partitionCacheHitOutOfRange;
+    /** Number of partition cache hits */
+    public final Counter partitionCacheHit;
+    /** Number of partition cache misses */
+    public final Counter partitionCacheMiss;
+    /** CAS Prepare metrics */
+    public final LatencyMetrics casPrepare;
+    /** CAS Propose metrics */
+    public final LatencyMetrics casPropose;
+    /** CAS Commit metrics */
+    public final LatencyMetrics casCommit;
+
+    public final Timer coordinatorReadLatency;
+    public final Timer coordinatorScanLatency;
+
+    /** Time spent waiting for free memtable space, either on- or off-heap */
+    public final Histogram waitingOnFreeMemtableSpace;
+
+    private final MetricNameFactory factory;
+    private final MetricNameFactory aliasFactory;
+    private static final MetricNameFactory globalFactory = new AllTableMetricNameFactory("Table");
+    private static final MetricNameFactory globalAliasFactory = new AllTableMetricNameFactory("ColumnFamily");
+
+    public final Counter speculativeRetries;
+
+    public final static LatencyMetrics globalReadLatency = new LatencyMetrics(globalFactory, globalAliasFactory, "Read");
+    public final static LatencyMetrics globalWriteLatency = new LatencyMetrics(globalFactory, globalAliasFactory, "Write");
+    public final static LatencyMetrics globalRangeLatency = new LatencyMetrics(globalFactory, globalAliasFactory, "Range");
+
+    public final Map<Sampler, TopKSampler<ByteBuffer>> samplers;
+    /**
+     * stores metrics that will be rolled into a single global metric
+     */
+    public final static ConcurrentMap<String, Set<Metric>> allTableMetrics = Maps.newConcurrentMap();
+
+    /**
+     * Stores all metric names created that can be used when unregistering, optionally mapped to an alias name.
+     */
+    public final static Map<String, String> all = Maps.newHashMap();
+
+    private interface GetHistogram
+    {
+        EstimatedHistogram getHistogram(SSTableReader reader);
+    }
+
+    private static long[] combineHistograms(Iterable<SSTableReader> sstables, GetHistogram getHistogram)
+    {
+        Iterator<SSTableReader> iterator = sstables.iterator();
+        if (!iterator.hasNext())
+        {
+            return new long[0];
+        }
+        long[] firstBucket = getHistogram.getHistogram(iterator.next()).getBuckets(false);
+        long[] values = new long[firstBucket.length];
+        System.arraycopy(firstBucket, 0, values, 0, values.length);
+
+        while (iterator.hasNext())
+        {
+            long[] nextBucket = getHistogram.getHistogram(iterator.next()).getBuckets(false);
+            if (nextBucket.length > values.length)
+            {
+                long[] newValues = new long[nextBucket.length];
+                System.arraycopy(firstBucket, 0, newValues, 0, firstBucket.length);
+                for (int i = 0; i < newValues.length; i++)
+                {
+                    newValues[i] += nextBucket[i];
+                }
+                values = newValues;
+            }
+            else
+            {
+                for (int i = 0; i < values.length; i++)
+                {
+                    values[i] += nextBucket[i];
+                }
+            }
+        }
+        return values;
+    }
+
+    /**
+     * Creates metrics for given {@link ColumnFamilyStore}.
+     *
+     * @param cfs ColumnFamilyStore to measure metrics
+     */
+    public TableMetrics(final ColumnFamilyStore cfs)
+    {
+        factory = new TableMetricNameFactory(cfs, "Table");
+        aliasFactory = new TableMetricNameFactory(cfs, "ColumnFamily");
+
+        samplers = Maps.newHashMap();
+        for (Sampler sampler : Sampler.values())
+        {
+            samplers.put(sampler, new TopKSampler<>());
+        }
+
+        memtableColumnsCount = createTableGauge("MemtableColumnsCount", new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                return cfs.getTracker().getView().getCurrentMemtable().getOperations();
+            }
+        });
+        memtableOnHeapSize = createTableGauge("MemtableOnHeapSize", new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                return cfs.getTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
+            }
+        });
+        memtableOffHeapSize = createTableGauge("MemtableOffHeapSize", new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                return cfs.getTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
+            }
+        });
+        memtableLiveDataSize = createTableGauge("MemtableLiveDataSize", new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                return cfs.getTracker().getView().getCurrentMemtable().getLiveDataSize();
+            }
+        });
+        allMemtablesOnHeapSize = createTableGauge("AllMemtablesHeapSize", new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                long size = 0;
+                for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
+                    size += cfs2.getTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
+                return size;
+            }
+        });
+        allMemtablesOffHeapSize = createTableGauge("AllMemtablesOffHeapSize", new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                long size = 0;
+                for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
+                    size += cfs2.getTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
+                return size;
+            }
+        });
+        allMemtablesLiveDataSize = createTableGauge("AllMemtablesLiveDataSize", new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                long size = 0;
+                for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
+                    size += cfs2.getTracker().getView().getCurrentMemtable().getLiveDataSize();
+                return size;
+            }
+        });
+        memtableSwitchCount = createTableCounter("MemtableSwitchCount");
+        estimatedPartitionSizeHistogram = Metrics.register(factory.createMetricName("EstimatedPartitionSizeHistogram"),
+                                                           aliasFactory.createMetricName("EstimatedRowSizeHistogram"),
+                                                           new Gauge<long[]>()
+                                                           {
+                                                               public long[] getValue()
+                                                               {
+                                                                   return combineHistograms(cfs.getSSTables(), new GetHistogram()
+                                                                   {
+                                                                       public EstimatedHistogram getHistogram(SSTableReader reader)
+                                                                       {
+                                                                           return reader.getEstimatedPartitionSize();
+                                                                       }
+                                                                   });
+                                                               }
+                                                           });
+        estimatedPartitionCount = Metrics.register(factory.createMetricName("EstimatedPartitionCount"),
+                                                   aliasFactory.createMetricName("EstimatedRowCount"),
+                                                   new Gauge<Long>()
+                                                   {
+                                                       public Long getValue()
+                                                       {
+                                                           long memtablePartitions = 0;
+                                                           for (Memtable memtable : cfs.getTracker().getView().getAllMemtables())
+                                                               memtablePartitions += memtable.partitionCount();
+                                                           return SSTableReader.getApproximateKeyCount(cfs.getSSTables()) + memtablePartitions;
+                                                       }
+                                                   });
+        estimatedColumnCountHistogram = Metrics.register(factory.createMetricName("EstimatedColumnCountHistogram"),
+                                                         aliasFactory.createMetricName("EstimatedColumnCountHistogram"),
+                                                         new Gauge<long[]>()
+                                                         {
+                                                             public long[] getValue()
+                                                             {
+                                                                 return combineHistograms(cfs.getSSTables(), new GetHistogram()
+                                                                 {
+                                                                     public EstimatedHistogram getHistogram(SSTableReader reader)
+                                                                     {
+                                                                         return reader.getEstimatedColumnCount();
+                                                                     }
+                                                                 });
+            }
+        });
+        sstablesPerReadHistogram = createTableHistogram("SSTablesPerReadHistogram", cfs.keyspace.metric.sstablesPerReadHistogram);
+        compressionRatio = createTableGauge("CompressionRatio", new Gauge<Double>()
+        {
+            public Double getValue()
+            {
+                double sum = 0;
+                int total = 0;
+                for (SSTableReader sstable : cfs.getSSTables())
+                {
+                    if (sstable.getCompressionRatio() != MetadataCollector.NO_COMPRESSION_RATIO)
+                    {
+                        sum += sstable.getCompressionRatio();
+                        total++;
+                    }
+                }
+                return total != 0 ? sum / total : 0;
+            }
+        }, new Gauge<Double>() // global gauge
+        {
+            public Double getValue()
+            {
+                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", 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 = createTableCounter("PendingFlushes");
+        pendingCompactions = createTableGauge("PendingCompactions", new Gauge<Integer>()
+        {
+            public Integer getValue()
+            {
+                return cfs.getCompactionStrategyManager().getEstimatedRemainingTasks();
+            }
+        });
+        liveSSTableCount = createTableGauge("LiveSSTableCount", new Gauge<Integer>()
+        {
+            public Integer getValue()
+            {
+                return cfs.getTracker().getSSTables().size();
+            }
+        });
+        liveDiskSpaceUsed = createTableCounter("LiveDiskSpaceUsed");
+        totalDiskSpaceUsed = createTableCounter("TotalDiskSpaceUsed");
+        minPartitionSize = createTableGauge("MinPartitionSize", "MinRowSize", new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                long min = 0;
+                for (SSTableReader sstable : cfs.getSSTables())
+                {
+                    if (min == 0 || sstable.getEstimatedPartitionSize().min() < min)
+                        min = sstable.getEstimatedPartitionSize().min();
+                }
+                return min;
+            }
+        }, new Gauge<Long>() // global gauge
+        {
+            public Long getValue()
+            {
+                long min = Long.MAX_VALUE;
+                for (Metric cfGauge : allTableMetrics.get("MinPartitionSize"))
+                {
+                    min = Math.min(min, ((Gauge<? extends Number>) cfGauge).getValue().longValue());
+                }
+                return min;
+            }
+        });
+        maxPartitionSize = createTableGauge("MaxPartitionSize", "MaxRowSize", new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                long max = 0;
+                for (SSTableReader sstable : cfs.getSSTables())
+                {
+                    if (sstable.getEstimatedPartitionSize().max() > max)
+                        max = sstable.getEstimatedPartitionSize().max();
+                }
+                return max;
+            }
+        }, new Gauge<Long>() // global gauge
+        {
+            public Long getValue()
+            {
+                long max = 0;
+                for (Metric cfGauge : allTableMetrics.get("MaxPartitionSize"))
+                {
+                    max = Math.max(max, ((Gauge<? extends Number>) cfGauge).getValue().longValue());
+                }
+                return max;
+            }
+        });
+        meanPartitionSize = createTableGauge("MeanPartitionSize", "MeanRowSize", new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                long sum = 0;
+                long count = 0;
+                for (SSTableReader sstable : cfs.getSSTables())
+                {
+                    long n = sstable.getEstimatedPartitionSize().count();
+                    sum += sstable.getEstimatedPartitionSize().mean() * n;
+                    count += n;
+                }
+                return count > 0 ? sum / count : 0;
+            }
+        }, new Gauge<Long>() // global gauge
+        {
+            public Long getValue()
+            {
+                long sum = 0;
+                long count = 0;
+                for (Keyspace keyspace : Keyspace.all())
+                {
+                    for (SSTableReader sstable : keyspace.getAllSSTables())
+                    {
+                        long n = sstable.getEstimatedPartitionSize().count();
+                        sum += sstable.getEstimatedPartitionSize().mean() * n;
+                        count += n;
+                    }
+                }
+                return count > 0 ? sum / count : 0;
+            }
+        });
+        bloomFilterFalsePositives = createTableGauge("BloomFilterFalsePositives", new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                long count = 0L;
+                for (SSTableReader sstable : cfs.getSSTables())
+                    count += sstable.getBloomFilterFalsePositiveCount();
+                return count;
+            }
+        });
+        recentBloomFilterFalsePositives = createTableGauge("RecentBloomFilterFalsePositives", new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                long count = 0L;
+                for (SSTableReader sstable : cfs.getSSTables())
+                    count += sstable.getRecentBloomFilterFalsePositiveCount();
+                return count;
+            }
+        });
+        bloomFilterFalseRatio = createTableGauge("BloomFilterFalseRatio", new Gauge<Double>()
+        {
+            public Double getValue()
+            {
+                long falseCount = 0L;
+                long trueCount = 0L;
+                for (SSTableReader sstable : cfs.getSSTables())
+                {
+                    falseCount += sstable.getBloomFilterFalsePositiveCount();
+                    trueCount += sstable.getBloomFilterTruePositiveCount();
+                }
+                if (falseCount == 0L && trueCount == 0L)
+                    return 0d;
+                return (double) falseCount / (trueCount + falseCount);
+            }
+        }, new Gauge<Double>() // global gauge
+        {
+            public Double getValue()
+            {
+                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 = createTableGauge("RecentBloomFilterFalseRatio", new Gauge<Double>()
+        {
+            public Double getValue()
+            {
+                long falseCount = 0L;
+                long trueCount = 0L;
+                for (SSTableReader sstable : cfs.getSSTables())
+                {
+                    falseCount += sstable.getRecentBloomFilterFalsePositiveCount();
+                    trueCount += sstable.getRecentBloomFilterTruePositiveCount();
+                }
+                if (falseCount == 0L && trueCount == 0L)
+                    return 0d;
+                return (double) falseCount / (trueCount + falseCount);
+            }
+        }, new Gauge<Double>() // global gauge
+        {
+            public Double getValue()
+            {
+                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 = createTableGauge("BloomFilterDiskSpaceUsed", new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                long total = 0;
+                for (SSTableReader sst : cfs.getSSTables())
+                    total += sst.getBloomFilterSerializedSize();
+                return total;
+            }
+        });
+        bloomFilterOffHeapMemoryUsed = createTableGauge("BloomFilterOffHeapMemoryUsed", new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                long total = 0;
+                for (SSTableReader sst : cfs.getSSTables())
+                    total += sst.getBloomFilterOffHeapSize();
+                return total;
+            }
+        });
+        indexSummaryOffHeapMemoryUsed = createTableGauge("IndexSummaryOffHeapMemoryUsed", new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                long total = 0;
+                for (SSTableReader sst : cfs.getSSTables())
+                    total += sst.getIndexSummaryOffHeapSize();
+                return total;
+            }
+        });
+        compressionMetadataOffHeapMemoryUsed = createTableGauge("CompressionMetadataOffHeapMemoryUsed", new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                long total = 0;
+                for (SSTableReader sst : cfs.getSSTables())
+                    total += sst.getCompressionMetadataOffHeapSize();
+                return total;
+            }
+        });
+        speculativeRetries = createTableCounter("SpeculativeRetries");
+        keyCacheHitRate = Metrics.register(factory.createMetricName("KeyCacheHitRate"),
+                                           aliasFactory.createMetricName("KeyCacheHitRate"),
+                                           new RatioGauge()
+        {
+            @Override
+            public Ratio getRatio()
+            {
+                return Ratio.of(getNumerator(), getDenominator());
+            }
+
+            protected double getNumerator()
+            {
+                long hits = 0L;
+                for (SSTableReader sstable : cfs.getSSTables())
+                    hits += sstable.getKeyCacheHit();
+                return hits;
+            }
+
+            protected double getDenominator()
+            {
+                long requests = 0L;
+                for (SSTableReader sstable : cfs.getSSTables())
+                    requests += sstable.getKeyCacheRequest();
+                return Math.max(requests, 1); // to avoid NaN.
+            }
+        });
+        tombstoneScannedHistogram = createTableHistogram("TombstoneScannedHistogram", cfs.keyspace.metric.tombstoneScannedHistogram);
+        liveScannedHistogram = createTableHistogram("LiveScannedHistogram", cfs.keyspace.metric.liveScannedHistogram);
+        colUpdateTimeDeltaHistogram = createTableHistogram("ColUpdateTimeDeltaHistogram", cfs.keyspace.metric.colUpdateTimeDeltaHistogram);
+        coordinatorReadLatency = Metrics.timer(factory.createMetricName("CoordinatorReadLatency"));
+        coordinatorScanLatency = Metrics.timer(factory.createMetricName("CoordinatorScanLatency"));
+        waitingOnFreeMemtableSpace = Metrics.histogram(factory.createMetricName("WaitingOnFreeMemtableSpace"));
+
+        trueSnapshotsSize = createTableGauge("SnapshotsSize", new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                return cfs.trueSnapshotsSize();
+            }
+        });
+        partitionCacheHitOutOfRange = createTableCounter("PartitionCacheHitOutOfRange", "RowCacheHitOutOfRange");
+        partitionCacheHit = createTableCounter("PartitionCacheHit", "RowCacheHit");
+        partitionCacheMiss = createTableCounter("PartitionCacheMiss", "RowCacheMiss");
+
+        casPrepare = new LatencyMetrics(factory, "CasPrepare", cfs.keyspace.metric.casPrepare);
+        casPropose = new LatencyMetrics(factory, "CasPropose", cfs.keyspace.metric.casPropose);
+        casCommit = new LatencyMetrics(factory, "CasCommit", cfs.keyspace.metric.casCommit);
+    }
+
+    public void updateSSTableIterated(int count)
+    {
+        sstablesPerReadHistogram.update(count);
+    }
+
+    /**
+     * Release all associated metrics.
+     */
+    public void release()
+    {
+        for(Map.Entry<String, String> entry : all.entrySet())
+        {
+            CassandraMetricsRegistry.MetricName name = factory.createMetricName(entry.getKey());
+            CassandraMetricsRegistry.MetricName alias = aliasFactory.createMetricName(entry.getValue());
+            allTableMetrics.get(entry.getKey()).remove(Metrics.getMetrics().get(name.getMetricName()));
+            Metrics.remove(name, alias);
+        }
+        readLatency.release();
+        writeLatency.release();
+        rangeLatency.release();
+        Metrics.remove(factory.createMetricName("EstimatedPartitionSizeHistogram"), aliasFactory.createMetricName("EstimatedRowSizeHistogram"));
+        Metrics.remove(factory.createMetricName("EstimatedPartitionCount"), aliasFactory.createMetricName("EstimatedRowCount"));
+        Metrics.remove(factory.createMetricName("EstimatedColumnCountHistogram"), aliasFactory.createMetricName("EstimatedColumnCountHistogram"));
+        Metrics.remove(factory.createMetricName("KeyCacheHitRate"), aliasFactory.createMetricName("KeyCacheHitRate"));
+        Metrics.remove(factory.createMetricName("CoordinatorReadLatency"), aliasFactory.createMetricName("CoordinatorReadLatency"));
+        Metrics.remove(factory.createMetricName("CoordinatorScanLatency"), aliasFactory.createMetricName("CoordinatorScanLatency"));
+        Metrics.remove(factory.createMetricName("WaitingOnFreeMemtableSpace"), aliasFactory.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> createTableGauge(final String name, Gauge<T> gauge)
+    {
+        return createTableGauge(name, gauge, new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                long total = 0;
+                for (Metric cfGauge : allTableMetrics.get(name))
+                {
+                    total = total + ((Gauge<? extends Number>) cfGauge).getValue().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> createTableGauge(String name, Gauge<T> gauge, Gauge<G> globalGauge)
+    {
+        return createTableGauge(name, name, gauge, globalGauge);
+    }
+
+    protected <G,T> Gauge<T> createTableGauge(String name, String alias, Gauge<T> gauge, Gauge<G> globalGauge)
+    {
+        Gauge<T> cfGauge = Metrics.register(factory.createMetricName(name), aliasFactory.createMetricName(alias), gauge);
+        if (register(name, alias, cfGauge))
+        {
+            Metrics.register(globalFactory.createMetricName(name), globalAliasFactory.createMetricName(alias), 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 createTableCounter(final String name)
+    {
+        return createTableCounter(name, name);
+    }
+
+    protected Counter createTableCounter(final String name, final String alias)
+    {
+        Counter cfCounter = Metrics.counter(factory.createMetricName(name), aliasFactory.createMetricName(alias));
+        if (register(name, alias, cfCounter))
+        {
+            Metrics.register(globalFactory.createMetricName(name),
+                             globalAliasFactory.createMetricName(alias),
+                             new Gauge<Long>()
+            {
+                public Long getValue()
+                {
+                    long total = 0;
+                    for (Metric cfGauge : allTableMetrics.get(name))
+                    {
+                        total += ((Counter) cfGauge).getCount();
+                    }
+                    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 TableHistogram createTableHistogram(String name, Histogram keyspaceHistogram)
+    {
+        return createTableHistogram(name, name, keyspaceHistogram);
+    }
+
+    protected TableHistogram createTableHistogram(String name, String alias, Histogram keyspaceHistogram)
+    {
+        Histogram cfHistogram = Metrics.histogram(factory.createMetricName(name), aliasFactory.createMetricName(alias));
+        register(name, alias, cfHistogram);
+        return new TableHistogram(cfHistogram,
+                                  keyspaceHistogram,
+                                  Metrics.histogram(globalFactory.createMetricName(name),
+                                                    globalAliasFactory.createMetricName(alias)));
+    }
+
+    /**
+     * 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, String alias, Metric metric)
+    {
+        boolean ret = allTableMetrics.putIfAbsent(name,  new HashSet<>()) == null;
+        allTableMetrics.get(name).add(metric);
+        all.put(name, alias);
+        return ret;
+    }
+
+    public static class TableHistogram
+    {
+        public final Histogram[] all;
+        public final Histogram cf;
+        private TableHistogram(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);
+            }
+        }
+    }
+
+    static class TableMetricNameFactory implements MetricNameFactory
+    {
+        private final String keyspaceName;
+        private final String tableName;
+        private final boolean isIndex;
+        private final String type;
+
+        TableMetricNameFactory(ColumnFamilyStore cfs, String type)
+        {
+            this.keyspaceName = cfs.keyspace.getName();
+            this.tableName = cfs.name;
+            this.isIndex = cfs.isIndex();
+            this.type = type;
+        }
+
+        public CassandraMetricsRegistry.MetricName createMetricName(String metricName)
+        {
+            String groupName = TableMetrics.class.getPackage().getName();
+            String type = isIndex ? "Index" + this.type : this.type;
+
+            StringBuilder mbeanName = new StringBuilder();
+            mbeanName.append(groupName).append(":");
+            mbeanName.append("type=").append(type);
+            mbeanName.append(",keyspace=").append(keyspaceName);
+            mbeanName.append(",scope=").append(tableName);
+            mbeanName.append(",name=").append(metricName);
+
+            return new CassandraMetricsRegistry.MetricName(groupName, type, metricName, keyspaceName + "." + tableName, mbeanName.toString());
+        }
+    }
+
+    static class AllTableMetricNameFactory implements MetricNameFactory
+    {
+        private final String type;
+        public AllTableMetricNameFactory(String type)
+        {
+            this.type = type;
+        }
+
+        public CassandraMetricsRegistry.MetricName createMetricName(String metricName)
+        {
+            String groupName = TableMetrics.class.getPackage().getName();
+            StringBuilder mbeanName = new StringBuilder();
+            mbeanName.append(groupName).append(":");
+            mbeanName.append("type=" + type);
+            mbeanName.append(",name=").append(metricName);
+            return new CassandraMetricsRegistry.MetricName(groupName, type, metricName, "all", mbeanName.toString());
+        }
+    }
+
+    public enum Sampler
+    {
+        READS, WRITES
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0bd5170c/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index cb97fd3..236cde6 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -2476,13 +2476,13 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return Gossiper.instance.getCurrentGenerationNumber(FBUtilities.getBroadcastAddress());
     }
 
-    public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int forceKeyspaceCleanup(String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
     {
         if (Schema.isSystemKeyspace(keyspaceName))
             throw new RuntimeException("Cleanup of the system keyspace is neither necessary nor wise");
 
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
-        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
+        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, tables))
         {
             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.forceCleanup();
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
@@ -2491,15 +2491,15 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return status.statusCode;
     }
 
-    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
     {
-        return scrub(disableSnapshot, skipCorrupted, true, keyspaceName, columnFamilies);
+        return scrub(disableSnapshot, skipCorrupted, true, keyspaceName, tables);
     }
 
-    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
     {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
-        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
+        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, tables))
         {
             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
@@ -2508,10 +2508,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return status.statusCode;
     }
 
-    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int verify(boolean extendedVerify, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
     {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
-        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
+        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, tableNames))
         {
             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.verify(extendedVerify);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
@@ -2520,10 +2520,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return status.statusCode;
     }
 
-    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... tableNames) throws IOException, ExecutionException, InterruptedException
     {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
-        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, keyspaceName, columnFamilies))
+        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, keyspaceName, tableNames))
         {
             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.sstablesRewrite(excludeCurrentVersion);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
@@ -2532,9 +2532,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return status.statusCode;
     }
 
-    public void forceKeyspaceCompaction(boolean splitOutput, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void forceKeyspaceCompaction(boolean splitOutput, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
     {
-        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
+        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, tableNames))
         {
             cfStore.forceMajorCompaction(splitOutput);
         }
@@ -2577,29 +2577,29 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     }
 
     /**
-     * Takes the snapshot of a specific column family. A snapshot name must be specified.
+     * Takes the snapshot of a specific table. A snapshot name must be specified.
      *
-     * @param keyspaceName the keyspace which holds the specified column family
-     * @param columnFamilyName the column family to snapshot
+     * @param keyspaceName the keyspace which holds the specified table
+     * @param tableName the table to snapshot
      * @param tag the tag given to the snapshot; may not be null or empty
      */
-    public void takeColumnFamilySnapshot(String keyspaceName, String columnFamilyName, String tag) throws IOException
+    public void takeTableSnapshot(String keyspaceName, String tableName, String tag) throws IOException
     {
         if (keyspaceName == null)
             throw new IOException("You must supply a keyspace name");
         if (operationMode == Mode.JOINING)
             throw new IOException("Cannot snapshot until bootstrap completes");
 
-        if (columnFamilyName == null)
+        if (tableName == null)
             throw new IOException("You must supply a table name");
-        if (columnFamilyName.contains("."))
+        if (tableName.contains("."))
             throw new IllegalArgumentException("Cannot take a snapshot of a secondary index by itself. Run snapshot on the table that owns the index.");
 
         if (tag == null || tag.equals(""))
             throw new IOException("You must supply a snapshot name.");
 
         Keyspace keyspace = getValidKeyspace(keyspaceName);
-        ColumnFamilyStore columnFamilyStore = keyspace.getColumnFamilyStore(columnFamilyName);
+        ColumnFamilyStore columnFamilyStore = keyspace.getColumnFamilyStore(tableName);
         if (columnFamilyStore.snapshotExists(tag))
             throw new IOException("Snapshot " + tag + " already exists.");
 
@@ -2612,34 +2612,34 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * 
      * @param tag
      *            the tag given to the snapshot; may not be null or empty
-     * @param columnFamilyList
-     *            list of columnfamily from different keyspace in the form of ks1.cf1 ks2.cf2
+     * @param tableList
+     *            list of tables from different keyspace in the form of ks1.cf1 ks2.cf2
      */
     @Override
-    public void takeMultipleColumnFamilySnapshot(String tag, String... columnFamilyList)
+    public void takeMultipleTableSnapshot(String tag, String... tableList)
             throws IOException
     {
         Map<Keyspace, List<String>> keyspaceColumnfamily = new HashMap<Keyspace, List<String>>();
-        for (String columnFamily : columnFamilyList)
+        for (String table : tableList)
         {
-            String splittedString[] = columnFamily.split("\\.");
+            String splittedString[] = table.split("\\.");
             if (splittedString.length == 2)
             {
                 String keyspaceName = splittedString[0];
-                String columnFamilyName = splittedString[1];
+                String tableName = splittedString[1];
 
                 if (keyspaceName == null)
                     throw new IOException("You must supply a keyspace name");
                 if (operationMode.equals(Mode.JOINING))
                     throw new IOException("Cannot snapshot until bootstrap completes");
 
-                if (columnFamilyName == null)
-                    throw new IOException("You must supply a column family name");
+                if (tableName == null)
+                    throw new IOException("You must supply a table name");
                 if (tag == null || tag.equals(""))
                     throw new IOException("You must supply a snapshot name.");
 
                 Keyspace keyspace = getValidKeyspace(keyspaceName);
-                ColumnFamilyStore columnFamilyStore = keyspace.getColumnFamilyStore(columnFamilyName);
+                ColumnFamilyStore columnFamilyStore = keyspace.getColumnFamilyStore(tableName);
                 // As there can be multiple column family from same keyspace check if snapshot exist for that specific
                 // columnfamily and not for whole keyspace
 
@@ -2652,7 +2652,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
                 // Add Keyspace columnfamily to map in order to support atomicity for snapshot process.
                 // So no snapshot should happen if any one of the above conditions fail for any keyspace or columnfamily
-                keyspaceColumnfamily.get(keyspace).add(columnFamilyName);
+                keyspaceColumnfamily.get(keyspace).add(tableName);
 
             }
             else
@@ -2664,8 +2664,8 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
         for (Entry<Keyspace, List<String>> entry : keyspaceColumnfamily.entrySet())
         {
-            for (String columnFamily : entry.getValue())
-                entry.getKey().snapshot(tag, columnFamily);
+            for (String table : entry.getValue())
+                entry.getKey().snapshot(tag, table);
         }
 
     }
@@ -2766,12 +2766,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     /**
      * Flush all memtables for a keyspace and column families.
      * @param keyspaceName
-     * @param columnFamilies
+     * @param tableNames
      * @throws IOException
      */
-    public void forceKeyspaceFlush(String keyspaceName, String... columnFamilies) throws IOException
+    public void forceKeyspaceFlush(String keyspaceName, String... tableNames) throws IOException
     {
-        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
+        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, tableNames))
         {
             logger.debug("Forcing flush on keyspace {}, CF {}", keyspaceName, cfStore.name);
             cfStore.forceBlockingFlush();
@@ -2810,9 +2810,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                                 Collection<String> hosts,
                                 boolean primaryRange,
                                 boolean fullRepair,
-                                String... columnFamilies)
+                                String... tableNames)
     {
-        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL.ordinal() : RepairParallelism.PARALLEL.ordinal(), dataCenters, hosts, primaryRange, fullRepair, columnFamilies);
+        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL.ordinal() : RepairParallelism.PARALLEL.ordinal(), dataCenters, hosts, primaryRange, fullRepair, tableNames);
     }
 
     @Deprecated
@@ -2822,7 +2822,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                                 Collection<String> hosts,
                                 boolean primaryRange,
                                 boolean fullRepair,
-                                String... columnFamilies)
+                                String... tableNames)
     {
         if (parallelismDegree < 0 || parallelismDegree > RepairParallelism.values().length - 1)
         {
@@ -2859,11 +2859,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         {
             options.getRanges().addAll(getLocalRanges(keyspace));
         }
-        if (columnFamilies != null)
+        if (tableNames != null)
         {
-            for (String columnFamily : columnFamilies)
+            for (String table : tableNames)
             {
-                options.getColumnFamilies().add(columnFamily);
+                options.getColumnFamilies().add(table);
             }
         }
         return forceRepairAsync(keyspace, options);
@@ -2874,14 +2874,14 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                                 boolean isLocal,
                                 boolean primaryRange,
                                 boolean fullRepair,
-                                String... columnFamilies)
+                                String... tableNames)
     {
         Set<String> dataCenters = null;
         if (isLocal)
         {
             dataCenters = Sets.newHashSet(DatabaseDescriptor.getLocalDataCenter());
         }
-        return forceRepairAsync(keyspace, isSequential, dataCenters, null, primaryRange, fullRepair, columnFamilies);
+        return forceRepairAsync(keyspace, isSequential, dataCenters, null, primaryRange, fullRepair, tableNames);
     }
 
     public int forceRepairRangeAsync(String beginToken,
@@ -2891,11 +2891,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                                      Collection<String> dataCenters,
                                      Collection<String> hosts,
                                      boolean fullRepair,
-                                     String... columnFamilies)
+                                     String... tableNames)
     {
         return forceRepairRangeAsync(beginToken, endToken, keyspaceName,
                                      isSequential ? RepairParallelism.SEQUENTIAL.ordinal() : RepairParallelism.PARALLEL.ordinal(),
-                                     dataCenters, hosts, fullRepair, columnFamilies);
+                                     dataCenters, hosts, fullRepair, tableNames);
     }
 
     public int forceRepairRangeAsync(String beginToken,
@@ -2905,7 +2905,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                                      Collection<String> dataCenters,
                                      Collection<String> hosts,
                                      boolean fullRepair,
-                                     String... columnFamilies)
+                                     String... tableNames)
     {
         if (parallelismDegree < 0 || parallelismDegree > RepairParallelism.values().length - 1)
         {
@@ -2925,16 +2925,16 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         {
             options.getHosts().addAll(hosts);
         }
-        if (columnFamilies != null)
+        if (tableNames != null)
         {
-            for (String columnFamily : columnFamilies)
+            for (String table : tableNames)
             {
-                options.getColumnFamilies().add(columnFamily);
+                options.getColumnFamilies().add(table);
             }
         }
 
         logger.info("starting user-requested repair of range {} for keyspace {} and column families {}",
-                    repairingRange, keyspaceName, columnFamilies);
+                    repairingRange, keyspaceName, tableNames);
         return forceRepairAsync(keyspaceName, options);
     }
 
@@ -2944,14 +2944,14 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                                      boolean isSequential,
                                      boolean isLocal,
                                      boolean fullRepair,
-                                     String... columnFamilies)
+                                     String... tableNames)
     {
         Set<String> dataCenters = null;
         if (isLocal)
         {
             dataCenters = Sets.newHashSet(DatabaseDescriptor.getLocalDataCenter());
         }
-        return forceRepairRangeAsync(beginToken, endToken, keyspaceName, isSequential, dataCenters, null, fullRepair, columnFamilies);
+        return forceRepairRangeAsync(beginToken, endToken, keyspaceName, isSequential, dataCenters, null, fullRepair, tableNames);
     }
 
     /**
@@ -3909,11 +3909,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return old;
     }
 
-    public void truncate(String keyspace, String columnFamily) throws TimeoutException, IOException
+    public void truncate(String keyspace, String table) throws TimeoutException, IOException
     {
         try
         {
-            StorageProxy.truncateBlocking(keyspace, columnFamily);
+            StorageProxy.truncateBlocking(keyspace, table);
         }
         catch (UnavailableException e)
         {
@@ -4266,17 +4266,17 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return traceProbability;
     }
 
-    public void disableAutoCompaction(String ks, String... columnFamilies) throws IOException
+    public void disableAutoCompaction(String ks, String... tables) throws IOException
     {
-        for (ColumnFamilyStore cfs : getValidColumnFamilies(true, true, ks, columnFamilies))
+        for (ColumnFamilyStore cfs : getValidColumnFamilies(true, true, ks, tables))
         {
             cfs.disableAutoCompaction();
         }
     }
 
-    public void enableAutoCompaction(String ks, String... columnFamilies) throws IOException
+    public void enableAutoCompaction(String ks, String... tables) throws IOException
     {
-        for (ColumnFamilyStore cfs : getValidColumnFamilies(true, true, ks, columnFamilies))
+        for (ColumnFamilyStore cfs : getValidColumnFamilies(true, true, ks, tables))
         {
             cfs.enableAutoCompaction();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0bd5170c/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 2bbc999..67b7002 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -198,20 +198,20 @@ public interface StorageServiceMBean extends NotificationEmitter
      * Takes the snapshot of a specific column family. A snapshot name must be specified.
      *
      * @param keyspaceName the keyspace which holds the specified column family
-     * @param columnFamilyName the column family to snapshot
+     * @param tableName the table to snapshot
      * @param tag the tag given to the snapshot; may not be null or empty
      */
-    public void takeColumnFamilySnapshot(String keyspaceName, String columnFamilyName, String tag) throws IOException;
+    public void takeTableSnapshot(String keyspaceName, String tableName, String tag) throws IOException;
 
     /**
      * Takes the snapshot of a multiple column family from different keyspaces. A snapshot name must be specified.
      * 
      * @param tag
      *            the tag given to the snapshot; may not be null or empty
-     * @param columnFamilyList
-     *            list of columnfamily from different keyspace in the form of ks1.cf1 ks2.cf2
+     * @param tableList
+     *            list of tables from different keyspace in the form of ks1.cf1 ks2.cf2
      */
-    public void takeMultipleColumnFamilySnapshot(String tag, String... columnFamilyList) throws IOException;
+    public void takeMultipleTableSnapshot(String tag, String... tableList) throws IOException;
 
     /**
      * Remove the snapshot with the given name from the given keyspaces.
@@ -234,45 +234,45 @@ public interface StorageServiceMBean extends NotificationEmitter
     /**
      * Forces major compaction of a single keyspace
      */
-    public void forceKeyspaceCompaction(boolean splitOutput, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public void forceKeyspaceCompaction(boolean splitOutput, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Trigger a cleanup of keys on a single keyspace
      */
-    public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int forceKeyspaceCleanup(String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Scrub (deserialize + reserialize at the latest version, skipping bad rows if any) the given keyspace.
-     * If columnFamilies array is empty, all CFs are scrubbed.
+     * If tableNames array is empty, all CFs are scrubbed.
      *
      * Scrubbed CFs will be snapshotted first, if disableSnapshot is false
      */
     @Deprecated
-    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
-    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Verify (checksums of) the given keyspace.
-     * If columnFamilies array is empty, all CFs are verified.
+     * If tableNames array is empty, all CFs are verified.
      *
      * The entire sstable will be read to ensure each cell validates if extendedVerify is true
      */
-    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int verify(boolean extendedVerify, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Rewrite all sstables to the latest version.
      * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
      */
-    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... tableNames) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Flush all memtables for the given column families, or all columnfamilies for the given keyspace
      * if none are explicitly listed.
      * @param keyspaceName
-     * @param columnFamilies
+     * @param tableNames
      * @throws IOException
      */
-    public void forceKeyspaceFlush(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public void forceKeyspaceFlush(String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Invoke repair asynchronously.
@@ -288,7 +288,7 @@ public interface StorageServiceMBean extends NotificationEmitter
     public int repairAsync(String keyspace, Map<String, String> options);
 
     @Deprecated
-    public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts,  boolean primaryRange, boolean repairedAt, String... columnFamilies) throws IOException;
+    public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts,  boolean primaryRange, boolean repairedAt, String... tableNames) throws IOException;
 
     /**
      * Invoke repair asynchronously.
@@ -301,10 +301,10 @@ public interface StorageServiceMBean extends NotificationEmitter
      * @return Repair command number, or 0 if nothing to repair
      */
     @Deprecated
-    public int forceRepairAsync(String keyspace, int parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies);
+    public int forceRepairAsync(String keyspace, int parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... tableNames);
 
     @Deprecated
-    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean repairedAt, String... columnFamilies) throws IOException;
+    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean repairedAt, String... tableNames) throws IOException;
 
     /**
      * Same as forceRepairAsync, but handles a specified range
@@ -312,13 +312,13 @@ public interface StorageServiceMBean extends NotificationEmitter
      * @param parallelismDegree 0: sequential, 1: parallel, 2: DC parallel
      */
     @Deprecated
-    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, int parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean fullRepair, String... columnFamilies);
+    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, int parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean fullRepair, String... tableNames);
 
     @Deprecated
-    public int forceRepairAsync(String keyspace, boolean isSequential, boolean isLocal, boolean primaryRange, boolean fullRepair, String... columnFamilies);
+    public int forceRepairAsync(String keyspace, boolean isSequential, boolean isLocal, boolean primaryRange, boolean fullRepair, String... tableNames);
 
     @Deprecated
-    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, boolean isLocal, boolean repairedAt, String... columnFamilies);
+    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, boolean isLocal, boolean repairedAt, String... tableNames);
 
     public void forceTerminateAllRepairSessions();
 
@@ -382,16 +382,16 @@ public interface StorageServiceMBean extends NotificationEmitter
     public void drain() throws IOException, InterruptedException, ExecutionException;
 
     /**
-     * Truncates (deletes) the given columnFamily from the provided keyspace.
+     * Truncates (deletes) the given table from the provided keyspace.
      * Calling truncate results in actual deletion of all data in the cluster
-     * under the given columnFamily and it will fail unless all hosts are up.
+     * under the given table and it will fail unless all hosts are up.
      * All data in the given column family will be deleted, but its definition
      * will not be affected.
      *
      * @param keyspace The keyspace to delete from
-     * @param columnFamily The column family to delete data from.
+     * @param table The column family to delete data from.
      */
-    public void truncate(String keyspace, String columnFamily)throws TimeoutException, IOException;
+    public void truncate(String keyspace, String table)throws TimeoutException, IOException;
 
     /**
      * given a list of tokens (representing the nodes in the cluster), returns
@@ -484,12 +484,12 @@ public interface StorageServiceMBean extends NotificationEmitter
     public void rescheduleFailedDeletions();
 
     /**
-     * Load new SSTables to the given keyspace/columnFamily
+     * Load new SSTables to the given keyspace/table
      *
      * @param ksName The parent keyspace name
      * @param cfName The ColumnFamily name where SSTables belong
      */
-    public void loadNewSSTables(String ksName, String cfName);
+    public void loadNewSSTables(String ksName, String tableName);
 
     /**
      * Return a List of Tokens representing a sample of keys across all ColumnFamilyStores.
@@ -522,8 +522,8 @@ public interface StorageServiceMBean extends NotificationEmitter
      */
     public double getTraceProbability();
 
-    void disableAutoCompaction(String ks, String ... columnFamilies) throws IOException;
-    void enableAutoCompaction(String ks, String ... columnFamilies) throws IOException;
+    void disableAutoCompaction(String ks, String ... tables) throws IOException;
+    void enableAutoCompaction(String ks, String ... tables) throws IOException;
 
     public void deliverHints(String host) throws UnknownHostException;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0bd5170c/src/java/org/apache/cassandra/service/pager/MultiPartitionPager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/pager/MultiPartitionPager.java b/src/java/org/apache/cassandra/service/pager/MultiPartitionPager.java
index 4fb1429..e329fd5 100644
--- a/src/java/org/apache/cassandra/service/pager/MultiPartitionPager.java
+++ b/src/java/org/apache/cassandra/service/pager/MultiPartitionPager.java
@@ -40,7 +40,7 @@ import org.apache.cassandra.service.ClientState;
  *
  * For now, we keep it simple (somewhat) and just do one command at a time. Provided that we make sure to not
  * create a pager unless we need to, this is probably fine. Though if we later want to get fancy, we could use the
- * cfs meanRowSize to decide if parallelizing some of the command might be worth it while being confident we don't
+ * cfs meanPartitionSize to decide if parallelizing some of the command might be worth it while being confident we don't
  * blow out memory.
  */
 public class MultiPartitionPager implements QueryPager

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0bd5170c/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java
index 1dc1527..4399a80 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -680,7 +680,7 @@ public class CassandraServer implements Cassandra.Iface
             // request by page if this is a large row
             if (cfs.getMeanColumns() > 0)
             {
-                int averageColumnSize = (int) (cfs.metric.meanRowSize.getValue() / cfs.getMeanColumns());
+                int averageColumnSize = (int) (cfs.metric.meanPartitionSize.getValue() / cfs.getMeanColumns());
                 pageSize = Math.min(COUNT_PAGE_SIZE, 4 * 1024 * 1024 / averageColumnSize);
                 pageSize = Math.max(2, pageSize);
                 logger.debug("average row column size is {}; using pageSize of {}", averageColumnSize, pageSize);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0bd5170c/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 929820d..19c4815 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -27,7 +27,6 @@ import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.rmi.server.RMIClientSocketFactory;
 import java.rmi.server.RMISocketFactory;
-import java.text.SimpleDateFormat;
 import java.util.AbstractMap;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -68,8 +67,9 @@ import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.gms.GossiperMBean;
 import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
 import org.apache.cassandra.metrics.CassandraMetricsRegistry;
-import org.apache.cassandra.metrics.ColumnFamilyMetrics.Sampler;
+import org.apache.cassandra.metrics.TableMetrics.Sampler;
 import org.apache.cassandra.metrics.StorageMetrics;
+import org.apache.cassandra.metrics.TableMetrics;
 import org.apache.cassandra.metrics.ThreadPoolMetrics;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.MessagingServiceMBean;
@@ -236,47 +236,47 @@ public class NodeProbe implements AutoCloseable
         jmxc.close();
     }
 
-    public int forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int forceKeyspaceCleanup(String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.forceKeyspaceCleanup(keyspaceName, columnFamilies);
+        return ssProxy.forceKeyspaceCleanup(keyspaceName, tables);
     }
 
-    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies);
+        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, tables);
     }
 
-    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int verify(boolean extendedVerify, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.verify(extendedVerify, keyspaceName, columnFamilies);
+        return ssProxy.verify(extendedVerify, keyspaceName, tableNames);
     }
 
-    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... tableNames) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies);
+        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, tableNames);
     }
 
-    public void forceKeyspaceCleanup(PrintStream out, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void forceKeyspaceCleanup(PrintStream out, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
     {
-        if (forceKeyspaceCleanup(keyspaceName, columnFamilies) != 0)
+        if (forceKeyspaceCleanup(keyspaceName, tableNames) != 0)
         {
             failed = true;
             out.println("Aborted cleaning up at least one table in keyspace "+keyspaceName+", check server logs for more information.");
         }
     }
 
-    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
     {
-        if (scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, columnFamilies) != 0)
+        if (scrub(disableSnapshot, skipCorrupted, checkData, keyspaceName, tables) != 0)
         {
             failed = true;
             out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
         }
     }
 
-    public void verify(PrintStream out, boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void verify(PrintStream out, boolean extendedVerify, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
     {
-        if (verify(extendedVerify, keyspaceName, columnFamilies) != 0)
+        if (verify(extendedVerify, keyspaceName, tableNames) != 0)
         {
             failed = true;
             out.println("Aborted verifying at least one table in keyspace "+keyspaceName+", check server logs for more information.");
@@ -284,9 +284,9 @@ public class NodeProbe implements AutoCloseable
     }
 
 
-    public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, String... tableNames) throws IOException, ExecutionException, InterruptedException
     {
-        if (upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies) != 0)
+        if (upgradeSSTables(keyspaceName, excludeCurrentVersion, tableNames) != 0)
         {
             failed = true;
             out.println("Aborted upgrading sstables for atleast one table in keyspace "+keyspaceName+", check server logs for more information.");
@@ -294,14 +294,14 @@ public class NodeProbe implements AutoCloseable
     }
 
 
-    public void forceKeyspaceCompaction(boolean splitOutput, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void forceKeyspaceCompaction(boolean splitOutput, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
     {
-        ssProxy.forceKeyspaceCompaction(splitOutput, keyspaceName, columnFamilies);
+        ssProxy.forceKeyspaceCompaction(splitOutput, keyspaceName, tableNames);
     }
 
-    public void forceKeyspaceFlush(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void forceKeyspaceFlush(String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
     {
-        ssProxy.forceKeyspaceFlush(keyspaceName, columnFamilies);
+        ssProxy.forceKeyspaceFlush(keyspaceName, tableNames);
     }
 
     public void repairAsync(final PrintStream out, final String keyspace, Map<String, String> options) throws IOException
@@ -508,18 +508,18 @@ public class NodeProbe implements AutoCloseable
      * Take a snapshot of all the keyspaces, optionally specifying only a specific column family.
      *
      * @param snapshotName the name of the snapshot.
-     * @param columnFamily the column family to snapshot or all on null
+     * @param table the table to snapshot or all on null
      * @param keyspaces the keyspaces to snapshot
      */
-    public void takeSnapshot(String snapshotName, String columnFamily, String... keyspaces) throws IOException
+    public void takeSnapshot(String snapshotName, String table, String... keyspaces) throws IOException
     {
-        if (columnFamily != null)
+        if (table != null)
         {
             if (keyspaces.length != 1)
             {
                 throw new IOException("When specifying the table for a snapshot, you must specify one and only one keyspace");
             }
-            ssProxy.takeColumnFamilySnapshot(keyspaces[0], columnFamily, snapshotName);
+            ssProxy.takeTableSnapshot(keyspaces[0], table, snapshotName);
         }
         else
             ssProxy.takeSnapshot(snapshotName, keyspaces);
@@ -530,15 +530,15 @@ public class NodeProbe implements AutoCloseable
      * 
      * @param snapshotName
      *            the name of the snapshot.
-     * @param columnFamilyList
+     * @param tableList
      *            list of columnfamily from different keyspace in the form of ks1.cf1 ks2.cf2
      */
-    public void takeMultipleColumnFamilySnapshot(String snapshotName, String... columnFamilyList)
+    public void takeMultipleTableSnapshot(String snapshotName, String... tableList)
             throws IOException
     {
-        if (null != columnFamilyList && columnFamilyList.length != 0)
+        if (null != tableList && tableList.length != 0)
         {
-            ssProxy.takeMultipleColumnFamilySnapshot(snapshotName, columnFamilyList);
+            ssProxy.takeMultipleTableSnapshot(snapshotName, tableList);
         }
         else
         {
@@ -617,14 +617,14 @@ public class NodeProbe implements AutoCloseable
         cfsProxy.setCompactionThresholds(minimumCompactionThreshold, maximumCompactionThreshold);
     }
 
-    public void disableAutoCompaction(String ks, String ... columnFamilies) throws IOException
+    public void disableAutoCompaction(String ks, String ... tables) throws IOException
     {
-        ssProxy.disableAutoCompaction(ks, columnFamilies);
+        ssProxy.disableAutoCompaction(ks, tables);
     }
 
-    public void enableAutoCompaction(String ks, String ... columnFamilies) throws IOException
+    public void enableAutoCompaction(String ks, String ... tableNames) throws IOException
     {
-        ssProxy.enableAutoCompaction(ks, columnFamilies);
+        ssProxy.enableAutoCompaction(ks, tableNames);
     }
 
     public void setIncrementalBackupsEnabled(boolean enabled)
@@ -706,11 +706,11 @@ public class NodeProbe implements AutoCloseable
         return ssProxy.isStarting();
     }
 
-    public void truncate(String keyspaceName, String cfName)
+    public void truncate(String keyspaceName, String tableName)
     {
         try
         {
-            ssProxy.truncate(keyspaceName, cfName);
+            ssProxy.truncate(keyspaceName, tableName);
         }
         catch (TimeoutException e)
         {
@@ -1091,13 +1091,13 @@ public class NodeProbe implements AutoCloseable
      * 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}.
+     * @param metricName View {@link TableMetrics}.
      */
     public Object getColumnFamilyMetric(String ks, String cf, String metricName)
     {
         try
         {
-            String type = cf.contains(".") ? "IndexColumnFamily": "ColumnFamily";
+            String type = cf.contains(".") ? "IndexTable" : "Table";
             ObjectName oName = new ObjectName(String.format("org.apache.cassandra.metrics:type=%s,keyspace=%s,scope=%s,name=%s", type, ks, cf, metricName));
             switch(metricName)
             {
@@ -1109,16 +1109,16 @@ public class NodeProbe implements AutoCloseable
                 case "CompressionMetadataOffHeapMemoryUsed":
                 case "CompressionRatio":
                 case "EstimatedColumnCountHistogram":
-                case "EstimatedRowSizeHistogram":
-                case "EstimatedRowCount":
+                case "EstimatedPartitionSizeHistogram":
+                case "EstimatedPartitionCount":
                 case "KeyCacheHitRate":
                 case "LiveSSTableCount":
-                case "MaxRowSize":
-                case "MeanRowSize":
+                case "MaxPartitionSize":
+                case "MeanPartitionSize":
                 case "MemtableColumnsCount":
                 case "MemtableLiveDataSize":
                 case "MemtableOffHeapSize":
-                case "MinRowSize":
+                case "MinPartitionSize":
                 case "RecentBloomFilterFalsePositives":
                 case "RecentBloomFilterFalseRatio":
                 case "SnapshotsSize":
@@ -1141,7 +1141,7 @@ public class NodeProbe implements AutoCloseable
                 case "TombstoneScannedHistogram":
                     return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxHistogramMBean.class);
                 default:
-                    throw new RuntimeException("Unknown table metric.");
+                    throw new RuntimeException("Unknown table metric " + metricName);
             }
         }
         catch (MalformedObjectNameException e)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0bd5170c/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index ed8b296..20fa0c9 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -333,7 +333,7 @@ public class NodeTool
             return Collections.unmodifiableList(keyspaces);
         }
 
-        protected String[] parseOptionalColumnFamilies(List<String> cmdArgs)
+        protected String[] parseOptionalTables(List<String> cmdArgs)
         {
             return cmdArgs.size() <= 1 ? EMPTY_STRING_ARRAY : toArray(cmdArgs.subList(1, cmdArgs.size()), String.class);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0bd5170c/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
index 6330915..a644f49 100644
--- a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
+++ b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
@@ -92,8 +92,8 @@ public class SSTableMetadataViewer
 
     private static void printHistograms(StatsMetadata metadata, PrintStream out)
     {
-        long[] offsets = metadata.estimatedRowSize.getBucketOffsets();
-        long[] ersh = metadata.estimatedRowSize.getBuckets(false);
+        long[] offsets = metadata.estimatedPartitionSize.getBucketOffsets();
+        long[] ersh = metadata.estimatedPartitionSize.getBuckets(false);
         long[] ecch = metadata.estimatedColumnCount.getBuckets(false);
 
         out.println(String.format("%-10s%18s%18s",

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0bd5170c/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java b/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
index d5c505e..cbfd4d6 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
@@ -38,7 +38,7 @@ public class Cleanup extends NodeToolCmd
     public void execute(NodeProbe probe)
     {
         List<String> keyspaces = parseOptionalKeyspace(args, probe);
-        String[] cfnames = parseOptionalColumnFamilies(args);
+        String[] tableNames = parseOptionalTables(args);
 
         for (String keyspace : keyspaces)
         {
@@ -47,7 +47,7 @@ public class Cleanup extends NodeToolCmd
 
             try
             {
-                probe.forceKeyspaceCleanup(System.out, keyspace, cfnames);
+                probe.forceKeyspaceCleanup(System.out, keyspace, tableNames);
             }
             catch (Exception e)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0bd5170c/src/java/org/apache/cassandra/tools/nodetool/Compact.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Compact.java b/src/java/org/apache/cassandra/tools/nodetool/Compact.java
index 4d04ae7..002541d 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Compact.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Compact.java
@@ -40,13 +40,13 @@ public class Compact extends NodeToolCmd
     public void execute(NodeProbe probe)
     {
         List<String> keyspaces = parseOptionalKeyspace(args, probe);
-        String[] cfnames = parseOptionalColumnFamilies(args);
+        String[] tableNames = parseOptionalTables(args);
 
         for (String keyspace : keyspaces)
         {
             try
             {
-                probe.forceKeyspaceCompaction(splitOutput, keyspace, cfnames);
+                probe.forceKeyspaceCompaction(splitOutput, keyspace, tableNames);
             } catch (Exception e)
             {
                 throw new RuntimeException("Error occurred during compaction", e);