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

[01/12] git commit: Track metrics at a keyspace level

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-1.2 7065b1bdf -> 593bba94f
  refs/heads/cassandra-2.0 cc276fb19 -> cee1f674a
  refs/heads/cassandra-2.1 a8f5278c8 -> a9da3fda5
  refs/heads/trunk b47d8a8c4 -> 305132946


Track metrics at a keyspace level

Patch by brandonwilliams, reviewed by yukim for CASSANDRA-6539


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

Branch: refs/heads/cassandra-2.0
Commit: 7065b1bdf119e07f2e64735f60ab3e3d036c00de
Parents: d509fe1
Author: Brandon Williams <br...@apache.org>
Authored: Thu Jun 12 18:42:21 2014 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Thu Jun 12 18:48:11 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 src/java/org/apache/cassandra/db/Table.java     |   4 +
 .../cassandra/metrics/KeyspaceMetrics.java      | 202 +++++++++++++++++++
 3 files changed, 207 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7065b1bd/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f37eda4..28b5f29 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 1.2.17
+ * Track metrics at a keyspace level (CASSANDRA-6539)
  * Add replace_address_first_boot flag to only replace if not bootstrapped (CASSANDRA-7356)
  * Enable keepalive for native protocol (CASSANDRA-7380)
  * Check internal addresses for seeds (CASSANDRA-6523)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7065b1bd/src/java/org/apache/cassandra/db/Table.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Table.java b/src/java/org/apache/cassandra/db/Table.java
index e6df982..611fd9a 100644
--- a/src/java/org/apache/cassandra/db/Table.java
+++ b/src/java/org/apache/cassandra/db/Table.java
@@ -42,6 +42,7 @@ import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.metrics.KeyspaceMetrics;
 
 /**
  * It represents a Keyspace.
@@ -74,6 +75,7 @@ public class Table
     private final ConcurrentMap<UUID, ColumnFamilyStore> columnFamilyStores = new ConcurrentHashMap<UUID, ColumnFamilyStore>();
     private final Object[] indexLocks;
     private volatile AbstractReplicationStrategy replicationStrategy;
+    public final KeyspaceMetrics metric;
     public static final Function<String,Table> tableTransformer = new Function<String, Table>()
     {
         public Table apply(String tableName)
@@ -132,6 +134,7 @@ public class Table
             {
                 for (ColumnFamilyStore cfs : t.getColumnFamilyStores())
                     t.unloadCf(cfs);
+                t.metric.release();
             }
             return t;
         }
@@ -271,6 +274,7 @@ public class Table
             logger.debug("Initializing {}.{}", name, cfm.cfName);
             initCf(cfm.cfId, cfm.cfName, loadSSTables);
         }
+        this.metric = new KeyspaceMetrics(this);
     }
 
     public void createReplicationStrategy(KSMetaData ksm)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7065b1bd/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
new file mode 100644
index 0000000..4a0980f
--- /dev/null
+++ b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
@@ -0,0 +1,202 @@
+/*
+ * 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.util.ArrayList;
+import java.util.List;
+
+import com.yammer.metrics.Metrics;
+import com.yammer.metrics.core.*;
+import com.yammer.metrics.stats.Snapshot;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+
+/**
+ * Metrics for {@link ColumnFamilyStore}.
+ */
+public class KeyspaceMetrics
+{
+    /** Total amount of data stored in the memtable, including column related overhead. */
+    public final Gauge<Long> memtableDataSize;
+    /** Total amount of data stored in the memtables (2i and pending flush memtables included). */
+    public final Gauge<Long> allMemtablesDataSize;
+    /** 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 Gauge<Long> memtableSwitchCount;
+    /** Estimated number of tasks pending for this column family */
+    public final Gauge<Integer> pendingTasks;
+    /** Estimate of number of pending compactios for this CF */
+    public final Gauge<Integer> pendingCompactions;
+    /** Disk space used by SSTables belonging to this CF */
+    public final Gauge<Long> liveDiskSpaceUsed;
+    /** Total disk space used by SSTables belonging to this CF, including obsolete ones waiting to be GC'd */
+    public final Gauge<Long> totalDiskSpaceUsed;
+    /** Disk space used by bloom filter */
+    public final Gauge<Long> bloomFilterDiskSpaceUsed;
+
+    private final MetricNameFactory factory;
+
+    /**
+     * Creates metrics for given {@link ColumnFamilyStore}.
+     *
+     * @param ks Keyspace to measure metrics
+     */
+    public KeyspaceMetrics(final Keyspace ks)
+    {
+        factory = new KeyspaceMetricNameFactory(ks);
+
+        memtableColumnsCount = Metrics.newGauge(factory.createMetricName("MemtableColumnsCount"), new Gauge<Long>()
+        {
+            public Long value()
+            {
+                long total = 0;
+                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
+                {
+                    total += cf.metric.memtableColumnsCount.value();
+                }
+                return total;
+            }
+        });
+        memtableDataSize = Metrics.newGauge(factory.createMetricName("MemtableDataSize"), new Gauge<Long>()
+        {
+            public Long value()
+            {
+                long total = 0;
+                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
+                {
+                    total += cf.metric.memtableDataSize.value();
+                }
+                return total;
+            }
+        });
+        allMemtablesDataSize = Metrics.newGauge(factory.createMetricName("AllMemtablesDataSize"), new Gauge<Long>()
+        {
+            public Long value()
+            {
+                long total = 0;
+                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
+                {
+                    total += cf.metric.allMemtablesDataSize.value();
+                }
+                return total;
+            }
+        });
+        memtableSwitchCount = Metrics.newGauge(factory.createMetricName("MemtableSwitchCount"), new Gauge<Long>()
+        {
+            public Long value()
+            {
+                long sum = 0;
+                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
+                    sum += cf.metric.memtableSwitchCount.count();
+                return sum;
+            }
+        });
+        pendingCompactions = Metrics.newGauge(factory.createMetricName("PendingCompactions"), new Gauge<Integer>()
+        {
+            public Integer value()
+            {
+                int sum = 0;
+                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
+                {
+                    sum += cf.metric.pendingCompactions.value();
+                }
+                return sum;
+            }
+        });
+        pendingTasks = Metrics.newGauge(factory.createMetricName("PendingTasks"), new Gauge<Integer>()
+        {
+            public Integer value()
+            {
+                return Keyspace.switchLock.getQueueLength();
+            }
+        });
+        liveDiskSpaceUsed = Metrics.newGauge(factory.createMetricName("LiveDiskSpaceUsed"), new Gauge<Long>()
+        {
+            public Long value()
+            {
+                long sum = 0;
+                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
+                {
+                    sum += cf.metric.liveDiskSpaceUsed.count();
+                }
+                return sum;
+            }
+        });
+        totalDiskSpaceUsed = Metrics.newGauge(factory.createMetricName("TotalDiskSpaceUsed"), new Gauge<Long>()
+        {
+            public Long value()
+            {
+                long sum = 0;
+                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
+                {
+                    sum += cf.metric.totalDiskSpaceUsed.count();
+                }
+                return sum;
+            }
+        });
+        bloomFilterDiskSpaceUsed = Metrics.newGauge(factory.createMetricName("BloomFilterDiskSpaceUsed"), new Gauge<Long>()
+        {
+            public Long value()
+            {
+                long total = 0;
+                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
+                    total += cf.metric.bloomFilterDiskSpaceUsed.value();
+                return total;
+            }
+        });
+    }
+
+    /**
+     * Release all associated metrics.
+     */
+    public void release()
+    {
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("AllMemtablesDataSize"));
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableDataSize"));
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableSwitchCount"));
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("PendingTasks"));
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("LiveDiskSpaceUsed"));
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("TotalDiskSpaceUsed"));
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("BloomFilterDiskSpaceUsed"));
+    }
+
+    class KeyspaceMetricNameFactory implements MetricNameFactory
+    {
+        private final String keyspaceName;
+
+        KeyspaceMetricNameFactory(Keyspace ks)
+        {
+            this.keyspaceName = ks.getName();
+        }
+
+        public MetricName createMetricName(String metricName)
+        {
+            String groupName = ColumnFamilyMetrics.class.getPackage().getName();
+
+            StringBuilder mbeanName = new StringBuilder();
+            mbeanName.append(groupName).append(":");
+            mbeanName.append("type=Keyspace");
+            mbeanName.append(",keyspace=").append(keyspaceName);
+            mbeanName.append(",name=").append(metricName);
+
+            return new MetricName(groupName, "keyspace", metricName, keyspaceName, mbeanName.toString());
+        }
+    }
+}


[10/12] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by br...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1


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

Branch: refs/heads/cassandra-2.1
Commit: a9da3fda5a83d8fbe3b5316f03a97dd2da6227e1
Parents: a8f5278 cee1f67
Author: Brandon Williams <br...@apache.org>
Authored: Fri Jun 13 13:35:29 2014 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Fri Jun 13 13:35:29 2014 -0500

----------------------------------------------------------------------

----------------------------------------------------------------------



[08/12] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0

Posted by br...@apache.org.
Merge branch 'cassandra-1.2' into cassandra-2.0


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

Branch: refs/heads/cassandra-2.1
Commit: cee1f674aa9bf50710dc3874c2e77698236e4734
Parents: cc276fb 593bba9
Author: Brandon Williams <br...@apache.org>
Authored: Fri Jun 13 13:35:19 2014 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Fri Jun 13 13:35:19 2014 -0500

----------------------------------------------------------------------

----------------------------------------------------------------------



[02/12] git commit: Track metrics at a keyspace level

Posted by br...@apache.org.
Track metrics at a keyspace level

Patch by brandonwilliams, reviewed by yukim for CASSANDRA-6539


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

Branch: refs/heads/cassandra-2.1
Commit: 7065b1bdf119e07f2e64735f60ab3e3d036c00de
Parents: d509fe1
Author: Brandon Williams <br...@apache.org>
Authored: Thu Jun 12 18:42:21 2014 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Thu Jun 12 18:48:11 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 src/java/org/apache/cassandra/db/Table.java     |   4 +
 .../cassandra/metrics/KeyspaceMetrics.java      | 202 +++++++++++++++++++
 3 files changed, 207 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7065b1bd/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f37eda4..28b5f29 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 1.2.17
+ * Track metrics at a keyspace level (CASSANDRA-6539)
  * Add replace_address_first_boot flag to only replace if not bootstrapped (CASSANDRA-7356)
  * Enable keepalive for native protocol (CASSANDRA-7380)
  * Check internal addresses for seeds (CASSANDRA-6523)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7065b1bd/src/java/org/apache/cassandra/db/Table.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Table.java b/src/java/org/apache/cassandra/db/Table.java
index e6df982..611fd9a 100644
--- a/src/java/org/apache/cassandra/db/Table.java
+++ b/src/java/org/apache/cassandra/db/Table.java
@@ -42,6 +42,7 @@ import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.metrics.KeyspaceMetrics;
 
 /**
  * It represents a Keyspace.
@@ -74,6 +75,7 @@ public class Table
     private final ConcurrentMap<UUID, ColumnFamilyStore> columnFamilyStores = new ConcurrentHashMap<UUID, ColumnFamilyStore>();
     private final Object[] indexLocks;
     private volatile AbstractReplicationStrategy replicationStrategy;
+    public final KeyspaceMetrics metric;
     public static final Function<String,Table> tableTransformer = new Function<String, Table>()
     {
         public Table apply(String tableName)
@@ -132,6 +134,7 @@ public class Table
             {
                 for (ColumnFamilyStore cfs : t.getColumnFamilyStores())
                     t.unloadCf(cfs);
+                t.metric.release();
             }
             return t;
         }
@@ -271,6 +274,7 @@ public class Table
             logger.debug("Initializing {}.{}", name, cfm.cfName);
             initCf(cfm.cfId, cfm.cfName, loadSSTables);
         }
+        this.metric = new KeyspaceMetrics(this);
     }
 
     public void createReplicationStrategy(KSMetaData ksm)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7065b1bd/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
new file mode 100644
index 0000000..4a0980f
--- /dev/null
+++ b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
@@ -0,0 +1,202 @@
+/*
+ * 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.util.ArrayList;
+import java.util.List;
+
+import com.yammer.metrics.Metrics;
+import com.yammer.metrics.core.*;
+import com.yammer.metrics.stats.Snapshot;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+
+/**
+ * Metrics for {@link ColumnFamilyStore}.
+ */
+public class KeyspaceMetrics
+{
+    /** Total amount of data stored in the memtable, including column related overhead. */
+    public final Gauge<Long> memtableDataSize;
+    /** Total amount of data stored in the memtables (2i and pending flush memtables included). */
+    public final Gauge<Long> allMemtablesDataSize;
+    /** 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 Gauge<Long> memtableSwitchCount;
+    /** Estimated number of tasks pending for this column family */
+    public final Gauge<Integer> pendingTasks;
+    /** Estimate of number of pending compactios for this CF */
+    public final Gauge<Integer> pendingCompactions;
+    /** Disk space used by SSTables belonging to this CF */
+    public final Gauge<Long> liveDiskSpaceUsed;
+    /** Total disk space used by SSTables belonging to this CF, including obsolete ones waiting to be GC'd */
+    public final Gauge<Long> totalDiskSpaceUsed;
+    /** Disk space used by bloom filter */
+    public final Gauge<Long> bloomFilterDiskSpaceUsed;
+
+    private final MetricNameFactory factory;
+
+    /**
+     * Creates metrics for given {@link ColumnFamilyStore}.
+     *
+     * @param ks Keyspace to measure metrics
+     */
+    public KeyspaceMetrics(final Keyspace ks)
+    {
+        factory = new KeyspaceMetricNameFactory(ks);
+
+        memtableColumnsCount = Metrics.newGauge(factory.createMetricName("MemtableColumnsCount"), new Gauge<Long>()
+        {
+            public Long value()
+            {
+                long total = 0;
+                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
+                {
+                    total += cf.metric.memtableColumnsCount.value();
+                }
+                return total;
+            }
+        });
+        memtableDataSize = Metrics.newGauge(factory.createMetricName("MemtableDataSize"), new Gauge<Long>()
+        {
+            public Long value()
+            {
+                long total = 0;
+                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
+                {
+                    total += cf.metric.memtableDataSize.value();
+                }
+                return total;
+            }
+        });
+        allMemtablesDataSize = Metrics.newGauge(factory.createMetricName("AllMemtablesDataSize"), new Gauge<Long>()
+        {
+            public Long value()
+            {
+                long total = 0;
+                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
+                {
+                    total += cf.metric.allMemtablesDataSize.value();
+                }
+                return total;
+            }
+        });
+        memtableSwitchCount = Metrics.newGauge(factory.createMetricName("MemtableSwitchCount"), new Gauge<Long>()
+        {
+            public Long value()
+            {
+                long sum = 0;
+                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
+                    sum += cf.metric.memtableSwitchCount.count();
+                return sum;
+            }
+        });
+        pendingCompactions = Metrics.newGauge(factory.createMetricName("PendingCompactions"), new Gauge<Integer>()
+        {
+            public Integer value()
+            {
+                int sum = 0;
+                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
+                {
+                    sum += cf.metric.pendingCompactions.value();
+                }
+                return sum;
+            }
+        });
+        pendingTasks = Metrics.newGauge(factory.createMetricName("PendingTasks"), new Gauge<Integer>()
+        {
+            public Integer value()
+            {
+                return Keyspace.switchLock.getQueueLength();
+            }
+        });
+        liveDiskSpaceUsed = Metrics.newGauge(factory.createMetricName("LiveDiskSpaceUsed"), new Gauge<Long>()
+        {
+            public Long value()
+            {
+                long sum = 0;
+                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
+                {
+                    sum += cf.metric.liveDiskSpaceUsed.count();
+                }
+                return sum;
+            }
+        });
+        totalDiskSpaceUsed = Metrics.newGauge(factory.createMetricName("TotalDiskSpaceUsed"), new Gauge<Long>()
+        {
+            public Long value()
+            {
+                long sum = 0;
+                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
+                {
+                    sum += cf.metric.totalDiskSpaceUsed.count();
+                }
+                return sum;
+            }
+        });
+        bloomFilterDiskSpaceUsed = Metrics.newGauge(factory.createMetricName("BloomFilterDiskSpaceUsed"), new Gauge<Long>()
+        {
+            public Long value()
+            {
+                long total = 0;
+                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
+                    total += cf.metric.bloomFilterDiskSpaceUsed.value();
+                return total;
+            }
+        });
+    }
+
+    /**
+     * Release all associated metrics.
+     */
+    public void release()
+    {
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("AllMemtablesDataSize"));
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableDataSize"));
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableSwitchCount"));
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("PendingTasks"));
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("LiveDiskSpaceUsed"));
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("TotalDiskSpaceUsed"));
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("BloomFilterDiskSpaceUsed"));
+    }
+
+    class KeyspaceMetricNameFactory implements MetricNameFactory
+    {
+        private final String keyspaceName;
+
+        KeyspaceMetricNameFactory(Keyspace ks)
+        {
+            this.keyspaceName = ks.getName();
+        }
+
+        public MetricName createMetricName(String metricName)
+        {
+            String groupName = ColumnFamilyMetrics.class.getPackage().getName();
+
+            StringBuilder mbeanName = new StringBuilder();
+            mbeanName.append(groupName).append(":");
+            mbeanName.append("type=Keyspace");
+            mbeanName.append(",keyspace=").append(keyspaceName);
+            mbeanName.append(",name=").append(metricName);
+
+            return new MetricName(groupName, "keyspace", metricName, keyspaceName, mbeanName.toString());
+        }
+    }
+}


[07/12] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0

Posted by br...@apache.org.
Merge branch 'cassandra-1.2' into cassandra-2.0


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

Branch: refs/heads/trunk
Commit: cee1f674aa9bf50710dc3874c2e77698236e4734
Parents: cc276fb 593bba9
Author: Brandon Williams <br...@apache.org>
Authored: Fri Jun 13 13:35:19 2014 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Fri Jun 13 13:35:19 2014 -0500

----------------------------------------------------------------------

----------------------------------------------------------------------



[06/12] git commit: fix wrong patch commit

Posted by br...@apache.org.
fix wrong patch commit


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

Branch: refs/heads/cassandra-1.2
Commit: 593bba94f68ce6fe5220264d5323693ce80ef6fb
Parents: 7065b1b
Author: Brandon Williams <br...@apache.org>
Authored: Fri Jun 13 13:35:08 2014 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Fri Jun 13 13:35:08 2014 -0500

----------------------------------------------------------------------
 .../cassandra/metrics/KeyspaceMetrics.java      | 42 +++-----------------
 1 file changed, 6 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/593bba94/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
index 4a0980f..936187c 100644
--- a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
@@ -17,15 +17,12 @@
  */
 package org.apache.cassandra.metrics;
 
-import java.util.ArrayList;
-import java.util.List;
 
 import com.yammer.metrics.Metrics;
 import com.yammer.metrics.core.*;
-import com.yammer.metrics.stats.Snapshot;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.Table;
 
 /**
  * Metrics for {@link ColumnFamilyStore}.
@@ -34,16 +31,12 @@ public class KeyspaceMetrics
 {
     /** Total amount of data stored in the memtable, including column related overhead. */
     public final Gauge<Long> memtableDataSize;
-    /** Total amount of data stored in the memtables (2i and pending flush memtables included). */
-    public final Gauge<Long> allMemtablesDataSize;
     /** 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 Gauge<Long> memtableSwitchCount;
     /** Estimated number of tasks pending for this column family */
     public final Gauge<Integer> pendingTasks;
-    /** Estimate of number of pending compactios for this CF */
-    public final Gauge<Integer> pendingCompactions;
     /** Disk space used by SSTables belonging to this CF */
     public final Gauge<Long> liveDiskSpaceUsed;
     /** Total disk space used by SSTables belonging to this CF, including obsolete ones waiting to be GC'd */
@@ -58,7 +51,7 @@ public class KeyspaceMetrics
      *
      * @param ks Keyspace to measure metrics
      */
-    public KeyspaceMetrics(final Keyspace ks)
+    public KeyspaceMetrics(final Table ks)
     {
         factory = new KeyspaceMetricNameFactory(ks);
 
@@ -86,18 +79,6 @@ public class KeyspaceMetrics
                 return total;
             }
         });
-        allMemtablesDataSize = Metrics.newGauge(factory.createMetricName("AllMemtablesDataSize"), new Gauge<Long>()
-        {
-            public Long value()
-            {
-                long total = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    total += cf.metric.allMemtablesDataSize.value();
-                }
-                return total;
-            }
-        });
         memtableSwitchCount = Metrics.newGauge(factory.createMetricName("MemtableSwitchCount"), new Gauge<Long>()
         {
             public Long value()
@@ -108,23 +89,11 @@ public class KeyspaceMetrics
                 return sum;
             }
         });
-        pendingCompactions = Metrics.newGauge(factory.createMetricName("PendingCompactions"), new Gauge<Integer>()
-        {
-            public Integer value()
-            {
-                int sum = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    sum += cf.metric.pendingCompactions.value();
-                }
-                return sum;
-            }
-        });
         pendingTasks = Metrics.newGauge(factory.createMetricName("PendingTasks"), new Gauge<Integer>()
         {
             public Integer value()
             {
-                return Keyspace.switchLock.getQueueLength();
+                return Table.switchLock.getQueueLength();
             }
         });
         liveDiskSpaceUsed = Metrics.newGauge(factory.createMetricName("LiveDiskSpaceUsed"), new Gauge<Long>()
@@ -169,6 +138,7 @@ public class KeyspaceMetrics
     public void release()
     {
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("AllMemtablesDataSize"));
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableColumnsCount"));
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableDataSize"));
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableSwitchCount"));
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("PendingTasks"));
@@ -181,9 +151,9 @@ public class KeyspaceMetrics
     {
         private final String keyspaceName;
 
-        KeyspaceMetricNameFactory(Keyspace ks)
+        KeyspaceMetricNameFactory(Table ks)
         {
-            this.keyspaceName = ks.getName();
+            this.keyspaceName = ks.name;
         }
 
         public MetricName createMetricName(String metricName)


[09/12] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0

Posted by br...@apache.org.
Merge branch 'cassandra-1.2' into cassandra-2.0


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

Branch: refs/heads/cassandra-2.0
Commit: cee1f674aa9bf50710dc3874c2e77698236e4734
Parents: cc276fb 593bba9
Author: Brandon Williams <br...@apache.org>
Authored: Fri Jun 13 13:35:19 2014 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Fri Jun 13 13:35:19 2014 -0500

----------------------------------------------------------------------

----------------------------------------------------------------------



[05/12] git commit: fix wrong patch commit

Posted by br...@apache.org.
fix wrong patch commit


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

Branch: refs/heads/trunk
Commit: 593bba94f68ce6fe5220264d5323693ce80ef6fb
Parents: 7065b1b
Author: Brandon Williams <br...@apache.org>
Authored: Fri Jun 13 13:35:08 2014 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Fri Jun 13 13:35:08 2014 -0500

----------------------------------------------------------------------
 .../cassandra/metrics/KeyspaceMetrics.java      | 42 +++-----------------
 1 file changed, 6 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/593bba94/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
index 4a0980f..936187c 100644
--- a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
@@ -17,15 +17,12 @@
  */
 package org.apache.cassandra.metrics;
 
-import java.util.ArrayList;
-import java.util.List;
 
 import com.yammer.metrics.Metrics;
 import com.yammer.metrics.core.*;
-import com.yammer.metrics.stats.Snapshot;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.Table;
 
 /**
  * Metrics for {@link ColumnFamilyStore}.
@@ -34,16 +31,12 @@ public class KeyspaceMetrics
 {
     /** Total amount of data stored in the memtable, including column related overhead. */
     public final Gauge<Long> memtableDataSize;
-    /** Total amount of data stored in the memtables (2i and pending flush memtables included). */
-    public final Gauge<Long> allMemtablesDataSize;
     /** 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 Gauge<Long> memtableSwitchCount;
     /** Estimated number of tasks pending for this column family */
     public final Gauge<Integer> pendingTasks;
-    /** Estimate of number of pending compactios for this CF */
-    public final Gauge<Integer> pendingCompactions;
     /** Disk space used by SSTables belonging to this CF */
     public final Gauge<Long> liveDiskSpaceUsed;
     /** Total disk space used by SSTables belonging to this CF, including obsolete ones waiting to be GC'd */
@@ -58,7 +51,7 @@ public class KeyspaceMetrics
      *
      * @param ks Keyspace to measure metrics
      */
-    public KeyspaceMetrics(final Keyspace ks)
+    public KeyspaceMetrics(final Table ks)
     {
         factory = new KeyspaceMetricNameFactory(ks);
 
@@ -86,18 +79,6 @@ public class KeyspaceMetrics
                 return total;
             }
         });
-        allMemtablesDataSize = Metrics.newGauge(factory.createMetricName("AllMemtablesDataSize"), new Gauge<Long>()
-        {
-            public Long value()
-            {
-                long total = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    total += cf.metric.allMemtablesDataSize.value();
-                }
-                return total;
-            }
-        });
         memtableSwitchCount = Metrics.newGauge(factory.createMetricName("MemtableSwitchCount"), new Gauge<Long>()
         {
             public Long value()
@@ -108,23 +89,11 @@ public class KeyspaceMetrics
                 return sum;
             }
         });
-        pendingCompactions = Metrics.newGauge(factory.createMetricName("PendingCompactions"), new Gauge<Integer>()
-        {
-            public Integer value()
-            {
-                int sum = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    sum += cf.metric.pendingCompactions.value();
-                }
-                return sum;
-            }
-        });
         pendingTasks = Metrics.newGauge(factory.createMetricName("PendingTasks"), new Gauge<Integer>()
         {
             public Integer value()
             {
-                return Keyspace.switchLock.getQueueLength();
+                return Table.switchLock.getQueueLength();
             }
         });
         liveDiskSpaceUsed = Metrics.newGauge(factory.createMetricName("LiveDiskSpaceUsed"), new Gauge<Long>()
@@ -169,6 +138,7 @@ public class KeyspaceMetrics
     public void release()
     {
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("AllMemtablesDataSize"));
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableColumnsCount"));
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableDataSize"));
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableSwitchCount"));
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("PendingTasks"));
@@ -181,9 +151,9 @@ public class KeyspaceMetrics
     {
         private final String keyspaceName;
 
-        KeyspaceMetricNameFactory(Keyspace ks)
+        KeyspaceMetricNameFactory(Table ks)
         {
-            this.keyspaceName = ks.getName();
+            this.keyspaceName = ks.name;
         }
 
         public MetricName createMetricName(String metricName)


[11/12] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by br...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1


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

Branch: refs/heads/trunk
Commit: a9da3fda5a83d8fbe3b5316f03a97dd2da6227e1
Parents: a8f5278 cee1f67
Author: Brandon Williams <br...@apache.org>
Authored: Fri Jun 13 13:35:29 2014 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Fri Jun 13 13:35:29 2014 -0500

----------------------------------------------------------------------

----------------------------------------------------------------------



[04/12] git commit: fix wrong patch commit

Posted by br...@apache.org.
fix wrong patch commit


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

Branch: refs/heads/cassandra-2.1
Commit: 593bba94f68ce6fe5220264d5323693ce80ef6fb
Parents: 7065b1b
Author: Brandon Williams <br...@apache.org>
Authored: Fri Jun 13 13:35:08 2014 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Fri Jun 13 13:35:08 2014 -0500

----------------------------------------------------------------------
 .../cassandra/metrics/KeyspaceMetrics.java      | 42 +++-----------------
 1 file changed, 6 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/593bba94/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
index 4a0980f..936187c 100644
--- a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
@@ -17,15 +17,12 @@
  */
 package org.apache.cassandra.metrics;
 
-import java.util.ArrayList;
-import java.util.List;
 
 import com.yammer.metrics.Metrics;
 import com.yammer.metrics.core.*;
-import com.yammer.metrics.stats.Snapshot;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.Table;
 
 /**
  * Metrics for {@link ColumnFamilyStore}.
@@ -34,16 +31,12 @@ public class KeyspaceMetrics
 {
     /** Total amount of data stored in the memtable, including column related overhead. */
     public final Gauge<Long> memtableDataSize;
-    /** Total amount of data stored in the memtables (2i and pending flush memtables included). */
-    public final Gauge<Long> allMemtablesDataSize;
     /** 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 Gauge<Long> memtableSwitchCount;
     /** Estimated number of tasks pending for this column family */
     public final Gauge<Integer> pendingTasks;
-    /** Estimate of number of pending compactios for this CF */
-    public final Gauge<Integer> pendingCompactions;
     /** Disk space used by SSTables belonging to this CF */
     public final Gauge<Long> liveDiskSpaceUsed;
     /** Total disk space used by SSTables belonging to this CF, including obsolete ones waiting to be GC'd */
@@ -58,7 +51,7 @@ public class KeyspaceMetrics
      *
      * @param ks Keyspace to measure metrics
      */
-    public KeyspaceMetrics(final Keyspace ks)
+    public KeyspaceMetrics(final Table ks)
     {
         factory = new KeyspaceMetricNameFactory(ks);
 
@@ -86,18 +79,6 @@ public class KeyspaceMetrics
                 return total;
             }
         });
-        allMemtablesDataSize = Metrics.newGauge(factory.createMetricName("AllMemtablesDataSize"), new Gauge<Long>()
-        {
-            public Long value()
-            {
-                long total = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    total += cf.metric.allMemtablesDataSize.value();
-                }
-                return total;
-            }
-        });
         memtableSwitchCount = Metrics.newGauge(factory.createMetricName("MemtableSwitchCount"), new Gauge<Long>()
         {
             public Long value()
@@ -108,23 +89,11 @@ public class KeyspaceMetrics
                 return sum;
             }
         });
-        pendingCompactions = Metrics.newGauge(factory.createMetricName("PendingCompactions"), new Gauge<Integer>()
-        {
-            public Integer value()
-            {
-                int sum = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    sum += cf.metric.pendingCompactions.value();
-                }
-                return sum;
-            }
-        });
         pendingTasks = Metrics.newGauge(factory.createMetricName("PendingTasks"), new Gauge<Integer>()
         {
             public Integer value()
             {
-                return Keyspace.switchLock.getQueueLength();
+                return Table.switchLock.getQueueLength();
             }
         });
         liveDiskSpaceUsed = Metrics.newGauge(factory.createMetricName("LiveDiskSpaceUsed"), new Gauge<Long>()
@@ -169,6 +138,7 @@ public class KeyspaceMetrics
     public void release()
     {
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("AllMemtablesDataSize"));
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableColumnsCount"));
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableDataSize"));
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableSwitchCount"));
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("PendingTasks"));
@@ -181,9 +151,9 @@ public class KeyspaceMetrics
     {
         private final String keyspaceName;
 
-        KeyspaceMetricNameFactory(Keyspace ks)
+        KeyspaceMetricNameFactory(Table ks)
         {
-            this.keyspaceName = ks.getName();
+            this.keyspaceName = ks.name;
         }
 
         public MetricName createMetricName(String metricName)


[12/12] git commit: Merge branch 'cassandra-2.1' into trunk

Posted by br...@apache.org.
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 305132946d20c7c2e81f2eb80da50f7131031061
Parents: b47d8a8 a9da3fd
Author: Brandon Williams <br...@apache.org>
Authored: Fri Jun 13 13:36:01 2014 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Fri Jun 13 13:36:01 2014 -0500

----------------------------------------------------------------------

----------------------------------------------------------------------



[03/12] git commit: fix wrong patch commit

Posted by br...@apache.org.
fix wrong patch commit


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

Branch: refs/heads/cassandra-2.0
Commit: 593bba94f68ce6fe5220264d5323693ce80ef6fb
Parents: 7065b1b
Author: Brandon Williams <br...@apache.org>
Authored: Fri Jun 13 13:35:08 2014 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Fri Jun 13 13:35:08 2014 -0500

----------------------------------------------------------------------
 .../cassandra/metrics/KeyspaceMetrics.java      | 42 +++-----------------
 1 file changed, 6 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/593bba94/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
index 4a0980f..936187c 100644
--- a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
@@ -17,15 +17,12 @@
  */
 package org.apache.cassandra.metrics;
 
-import java.util.ArrayList;
-import java.util.List;
 
 import com.yammer.metrics.Metrics;
 import com.yammer.metrics.core.*;
-import com.yammer.metrics.stats.Snapshot;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.Table;
 
 /**
  * Metrics for {@link ColumnFamilyStore}.
@@ -34,16 +31,12 @@ public class KeyspaceMetrics
 {
     /** Total amount of data stored in the memtable, including column related overhead. */
     public final Gauge<Long> memtableDataSize;
-    /** Total amount of data stored in the memtables (2i and pending flush memtables included). */
-    public final Gauge<Long> allMemtablesDataSize;
     /** 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 Gauge<Long> memtableSwitchCount;
     /** Estimated number of tasks pending for this column family */
     public final Gauge<Integer> pendingTasks;
-    /** Estimate of number of pending compactios for this CF */
-    public final Gauge<Integer> pendingCompactions;
     /** Disk space used by SSTables belonging to this CF */
     public final Gauge<Long> liveDiskSpaceUsed;
     /** Total disk space used by SSTables belonging to this CF, including obsolete ones waiting to be GC'd */
@@ -58,7 +51,7 @@ public class KeyspaceMetrics
      *
      * @param ks Keyspace to measure metrics
      */
-    public KeyspaceMetrics(final Keyspace ks)
+    public KeyspaceMetrics(final Table ks)
     {
         factory = new KeyspaceMetricNameFactory(ks);
 
@@ -86,18 +79,6 @@ public class KeyspaceMetrics
                 return total;
             }
         });
-        allMemtablesDataSize = Metrics.newGauge(factory.createMetricName("AllMemtablesDataSize"), new Gauge<Long>()
-        {
-            public Long value()
-            {
-                long total = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    total += cf.metric.allMemtablesDataSize.value();
-                }
-                return total;
-            }
-        });
         memtableSwitchCount = Metrics.newGauge(factory.createMetricName("MemtableSwitchCount"), new Gauge<Long>()
         {
             public Long value()
@@ -108,23 +89,11 @@ public class KeyspaceMetrics
                 return sum;
             }
         });
-        pendingCompactions = Metrics.newGauge(factory.createMetricName("PendingCompactions"), new Gauge<Integer>()
-        {
-            public Integer value()
-            {
-                int sum = 0;
-                for (ColumnFamilyStore cf : ks.getColumnFamilyStores())
-                {
-                    sum += cf.metric.pendingCompactions.value();
-                }
-                return sum;
-            }
-        });
         pendingTasks = Metrics.newGauge(factory.createMetricName("PendingTasks"), new Gauge<Integer>()
         {
             public Integer value()
             {
-                return Keyspace.switchLock.getQueueLength();
+                return Table.switchLock.getQueueLength();
             }
         });
         liveDiskSpaceUsed = Metrics.newGauge(factory.createMetricName("LiveDiskSpaceUsed"), new Gauge<Long>()
@@ -169,6 +138,7 @@ public class KeyspaceMetrics
     public void release()
     {
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("AllMemtablesDataSize"));
+        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableColumnsCount"));
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableDataSize"));
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("MemtableSwitchCount"));
         Metrics.defaultRegistry().removeMetric(factory.createMetricName("PendingTasks"));
@@ -181,9 +151,9 @@ public class KeyspaceMetrics
     {
         private final String keyspaceName;
 
-        KeyspaceMetricNameFactory(Keyspace ks)
+        KeyspaceMetricNameFactory(Table ks)
         {
-            this.keyspaceName = ks.getName();
+            this.keyspaceName = ks.name;
         }
 
         public MetricName createMetricName(String metricName)