You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2016/03/17 14:46:08 UTC

cassandra git commit: Have keyspace stats in nodetool tablestats always include all tables

Repository: cassandra
Updated Branches:
  refs/heads/trunk 645829214 -> 07ddb8a21


Have keyspace stats in nodetool tablestats always include all tables

patch by slebresne; reviewed by krummas for CASSANDRA-11226


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

Branch: refs/heads/trunk
Commit: 07ddb8a2198185bff4aa1215acdbb07cf3b18b66
Parents: 6458292
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Mar 17 12:04:13 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Mar 17 14:45:46 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/tools/nodetool/TableStats.java    | 179 +++++++++++--------
 2 files changed, 106 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/07ddb8a2/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index bec51ab..15cceaa 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.6
+ * Fix nodetool tablestats keyspace level metrics (CASSANDRA-11226)
  * Store repair options in parent_repair_history (CASSANDRA-11244)
  * Print current leveling in sstableofflinerelevel (CASSANDRA-9588)
  * Change repair message for keyspaces with RF 1 (CASSANDRA-11203)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07ddb8a2/src/java/org/apache/cassandra/tools/nodetool/TableStats.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/TableStats.java b/src/java/org/apache/cassandra/tools/nodetool/TableStats.java
index 681af5b..f082466 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/TableStats.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/TableStats.java
@@ -21,14 +21,11 @@ import io.airlift.command.Arguments;
 import io.airlift.command.Command;
 import io.airlift.command.Option;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
+import java.util.*;
 import javax.management.InstanceNotFoundException;
 
+import com.google.common.collect.ArrayListMultimap;
+
 import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.metrics.CassandraMetricsRegistry;
@@ -53,80 +50,56 @@ public class TableStats extends NodeToolCmd
     public void execute(NodeProbe probe)
     {
         TableStats.OptionFilter filter = new OptionFilter(ignore, tableNames);
-        Map<String, List<ColumnFamilyStoreMBean>> tableStoreMap = new HashMap<>();
+        ArrayListMultimap<String, ColumnFamilyStoreMBean> selectedTableMbeans = ArrayListMultimap.create();
+        Map<String, KeyspaceStats> keyspaceStats = new HashMap<>();
 
         // get a list of column family stores
-        Iterator<Map.Entry<String, ColumnFamilyStoreMBean>> tables = probe.getColumnFamilyStoreMBeanProxies();
+        Iterator<Map.Entry<String, ColumnFamilyStoreMBean>> tableMBeans = probe.getColumnFamilyStoreMBeanProxies();
 
-        while (tables.hasNext())
+        while (tableMBeans.hasNext())
         {
-            Map.Entry<String, ColumnFamilyStoreMBean> entry = tables.next();
+            Map.Entry<String, ColumnFamilyStoreMBean> entry = tableMBeans.next();
             String keyspaceName = entry.getKey();
             ColumnFamilyStoreMBean tableProxy = entry.getValue();
 
-            if (!tableStoreMap.containsKey(keyspaceName) && filter.isColumnFamilyIncluded(entry.getKey(), tableProxy.getColumnFamilyName()))
-            {
-                List<ColumnFamilyStoreMBean> columnFamilies = new ArrayList<>();
-                columnFamilies.add(tableProxy);
-                tableStoreMap.put(keyspaceName, columnFamilies);
-            } else if (filter.isColumnFamilyIncluded(entry.getKey(), tableProxy.getColumnFamilyName()))
+            if (filter.isKeyspaceIncluded(keyspaceName))
             {
-                tableStoreMap.get(keyspaceName).add(tableProxy);
+                KeyspaceStats stats = keyspaceStats.get(keyspaceName);
+                if (stats == null)
+                {
+                    stats = new KeyspaceStats(probe, keyspaceName);
+                    keyspaceStats.put(keyspaceName, stats);
+                }
+                stats.add(tableProxy);
+
+                if (filter.isTableIncluded(keyspaceName, tableProxy.getTableName()))
+                    selectedTableMbeans.put(keyspaceName, tableProxy);
             }
         }
 
         // make sure all specified keyspace and tables exist
         filter.verifyKeyspaces(probe.getKeyspaces());
-        filter.verifyColumnFamilies();
+        filter.verifyTables();
 
         // print out the table statistics
-        for (Map.Entry<String, List<ColumnFamilyStoreMBean>> entry : tableStoreMap.entrySet())
+        for (Map.Entry<String, Collection<ColumnFamilyStoreMBean>> entry : selectedTableMbeans.asMap().entrySet())
         {
             String keyspaceName = entry.getKey();
-            List<ColumnFamilyStoreMBean> columnFamilies = entry.getValue();
-            long keyspaceReadCount = 0;
-            long keyspaceWriteCount = 0;
-            int keyspacePendingFlushes = 0;
-            double keyspaceTotalReadTime = 0.0f;
-            double keyspaceTotalWriteTime = 0.0f;
+            Collection<ColumnFamilyStoreMBean> tables = entry.getValue();
 
             System.out.println("Keyspace: " + keyspaceName);
-            for (ColumnFamilyStoreMBean table : columnFamilies)
-            {
-                String tableName = table.getColumnFamilyName();
-                long writeCount = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "WriteLatency")).getCount();
-                long readCount = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "ReadLatency")).getCount();
-
-                if (readCount > 0)
-                {
-                    keyspaceReadCount += readCount;
-                    keyspaceTotalReadTime += (long) probe.getColumnFamilyMetric(keyspaceName, tableName, "ReadTotalLatency");
-                }
-                if (writeCount > 0)
-                {
-                    keyspaceWriteCount += writeCount;
-                    keyspaceTotalWriteTime += (long) probe.getColumnFamilyMetric(keyspaceName, tableName, "WriteTotalLatency");
-                }
-                keyspacePendingFlushes += (long) probe.getColumnFamilyMetric(keyspaceName, tableName, "PendingFlushes");
-            }
-
-            double keyspaceReadLatency = keyspaceReadCount > 0
-                                         ? keyspaceTotalReadTime / keyspaceReadCount / 1000
-                                         : Double.NaN;
-            double keyspaceWriteLatency = keyspaceWriteCount > 0
-                                          ? keyspaceTotalWriteTime / keyspaceWriteCount / 1000
-                                          : Double.NaN;
+            KeyspaceStats stats = keyspaceStats.get(keyspaceName);
 
-            System.out.println("\tRead Count: " + keyspaceReadCount);
-            System.out.println("\tRead Latency: " + String.format("%s", keyspaceReadLatency) + " ms.");
-            System.out.println("\tWrite Count: " + keyspaceWriteCount);
-            System.out.println("\tWrite Latency: " + String.format("%s", keyspaceWriteLatency) + " ms.");
-            System.out.println("\tPending Flushes: " + keyspacePendingFlushes);
+            System.out.println("\tRead Count: " + stats.readCount);
+            System.out.println("\tRead Latency: " + String.format("%s", stats.readLatency()) + " ms.");
+            System.out.println("\tWrite Count: " + stats.writeCount);
+            System.out.println("\tWrite Latency: " + String.format("%s", stats.writeLatency()) + " ms.");
+            System.out.println("\tPending Flushes: " + stats.pendingFlushes);
 
             // print out column family statistics for this keyspace
-            for (ColumnFamilyStoreMBean table : columnFamilies)
+            for (ColumnFamilyStoreMBean table : tables)
             {
-                String tableName = table.getColumnFamilyName();
+                String tableName = table.getTableName();
                 if (tableName.contains("."))
                     System.out.println("\t\tTable (index): " + tableName);
                 else
@@ -227,7 +200,8 @@ public class TableStats extends NodeToolCmd
         }
     }
 
-    private String format(long bytes, boolean humanReadable) {
+    private String format(long bytes, boolean humanReadable)
+    {
         return humanReadable ? FileUtils.stringifyFileSize(bytes) : Long.toString(bytes);
     }
 
@@ -237,7 +211,8 @@ public class TableStats extends NodeToolCmd
     private static class OptionFilter
     {
         private Map<String, List<String>> filter = new HashMap<>();
-        private Map<String, List<String>> verifier = new HashMap<>();
+        private Map<String, List<String>> verifier = new HashMap<>(); // Same as filter initially, but we remove tables every time we've checked them for inclusion
+                                                                      // in isTableIncluded() so that we detect if those table requested don't exist (verifyTables())
         private List<String> filterList = new ArrayList<>();
         private boolean ignoreMode;
 
@@ -255,24 +230,17 @@ public class TableStats extends NodeToolCmd
                 {
                     filter.put(keyValues[0], new ArrayList<String>());
                     verifier.put(keyValues[0], new ArrayList<String>());
+                }
 
-                    if (keyValues.length == 2)
-                    {
-                        filter.get(keyValues[0]).add(keyValues[1]);
-                        verifier.get(keyValues[0]).add(keyValues[1]);
-                    }
-                } else
+                if (keyValues.length == 2)
                 {
-                    if (keyValues.length == 2)
-                    {
-                        filter.get(keyValues[0]).add(keyValues[1]);
-                        verifier.get(keyValues[0]).add(keyValues[1]);
-                    }
+                    filter.get(keyValues[0]).add(keyValues[1]);
+                    verifier.get(keyValues[0]).add(keyValues[1]);
                 }
             }
         }
 
-        public boolean isColumnFamilyIncluded(String keyspace, String columnFamily)
+        public boolean isTableIncluded(String keyspace, String table)
         {
             // supplying empty params list is treated as wanting to display all keyspaces and tables
             if (filterList.isEmpty())
@@ -289,8 +257,20 @@ public class TableStats extends NodeToolCmd
                 return !ignoreMode;
 
             // keyspace exists, and it contains specific table
-            verifier.get(keyspace).remove(columnFamily);
-            return ignoreMode ^ tables.contains(columnFamily);
+            verifier.get(keyspace).remove(table);
+            return ignoreMode ^ tables.contains(table);
+        }
+
+        public boolean isKeyspaceIncluded(String keyspace)
+        {
+            // supplying empty params list is treated as wanting to display all keyspaces and tables
+            if (filterList.isEmpty())
+                return !ignoreMode;
+
+            // Note that if there is any table for the keyspace, we want to include the keyspace irregarding
+            // of the ignoreMode, since the ignoreMode then apply to the table inside the keyspace but the
+            // keyspace itself is not ignored
+            return filter.get(keyspace) != null || ignoreMode;
         }
 
         public void verifyKeyspaces(List<String> keyspaces)
@@ -300,11 +280,62 @@ public class TableStats extends NodeToolCmd
                     throw new IllegalArgumentException("Unknown keyspace: " + ks);
         }
 
-        public void verifyColumnFamilies()
+        public void verifyTables()
         {
             for (String ks : filter.keySet())
                 if (verifier.get(ks).size() > 0)
                     throw new IllegalArgumentException("Unknown tables: " + verifier.get(ks) + " in keyspace: " + ks);
         }
     }
+
+    private static class KeyspaceStats
+    {
+        private final NodeProbe probe;
+        private final String keyspaceName;
+
+        public long readCount;
+        public long writeCount;
+        public int pendingFlushes;
+        private double totalReadTime;
+        private double totalWriteTime;
+
+        public KeyspaceStats(NodeProbe probe, String keyspaceName)
+        {
+            this.probe = probe;
+            this.keyspaceName = keyspaceName;
+        }
+
+        public void add(ColumnFamilyStoreMBean table)
+        {
+            String tableName = table.getTableName();
+            long tableWriteCount = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "WriteLatency")).getCount();
+            long tableReadCount = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "ReadLatency")).getCount();
+
+            if (tableReadCount > 0)
+            {
+                readCount += tableReadCount;
+                totalReadTime += (long) probe.getColumnFamilyMetric(keyspaceName, tableName, "ReadTotalLatency");
+            }
+            if (tableWriteCount > 0)
+            {
+                writeCount += tableWriteCount;
+                totalWriteTime += (long) probe.getColumnFamilyMetric(keyspaceName, tableName, "WriteTotalLatency");
+            }
+            pendingFlushes += (long) probe.getColumnFamilyMetric(keyspaceName, tableName, "PendingFlushes");
+        }
+
+        public double readLatency()
+        {
+            return readCount > 0
+                 ? totalReadTime / readCount / 1000
+                 : Double.NaN;
+        }
+
+        public double writeLatency()
+        {
+            return writeCount > 0
+                 ? totalWriteTime / writeCount / 1000
+                 : Double.NaN;
+        }
+    }
 }