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 14:45:19 UTC

[6/8] git commit: Track metrics at a keyspace level

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/bbd130eb
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/bbd130eb
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/bbd130eb

Branch: refs/heads/trunk
Commit: bbd130ebbf2c1ebfe4f8514dedb20c9dabe4fbd4
Parents: c8f596a
Author: Brandon Williams <br...@apache.org>
Authored: Thu Jun 12 18:49:09 2014 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Thu Jun 12 18:49:09 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |   5 +
 .../cassandra/metrics/KeyspaceMetrics.java      | 202 +++++++++++++++++++
 3 files changed, 208 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/bbd130eb/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a2a978e..576355c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -16,7 +16,7 @@
  * Fix infinite loop on exception while streaming (CASSANDRA-7330)
  * Reference sstables before populating key cache (CASSANDRA-7234)
 Merged from 1.2:
-1.2.17
+ * Tracke 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/bbd130eb/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index 714956a..d92bea7 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -27,6 +27,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
+import org.apache.cassandra.metrics.KeyspaceMetrics;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -62,6 +63,8 @@ public class Keyspace
      */
     public static final ReentrantReadWriteLock switchLock = new ReentrantReadWriteLock();
 
+    public final KeyspaceMetrics metric;
+
     // It is possible to call Keyspace.open without a running daemon, so it makes sense to ensure
     // proper directories here as well as in CassandraDaemon.
     static
@@ -133,6 +136,7 @@ public class Keyspace
             {
                 for (ColumnFamilyStore cfs : t.getColumnFamilyStores())
                     t.unloadCf(cfs);
+                t.metric.release();
             }
             return t;
         }
@@ -259,6 +263,7 @@ public class Keyspace
         metadata = Schema.instance.getKSMetaData(keyspaceName);
         assert metadata != null : "Unknown keyspace " + keyspaceName;
         createReplicationStrategy(metadata);
+        metric = new KeyspaceMetrics(this);
 
         for (CFMetaData cfm : new ArrayList<CFMetaData>(metadata.cfMetaData().values()))
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bbd130eb/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());
+        }
+    }
+}