Repository: cassandra
Updated Branches:
  refs/heads/trunk f00e7a773 -> 27d25c6df


Add json and yaml formatting for tpstats output

Patch by Hiroyuki Nishi; reviewed by Alex Petrov for CASSANDRA-12035


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

Branch: refs/heads/trunk
Commit: 27d25c6df3930bb173d8d811f50d7643a7cacdf5
Parents: f00e7a7
Author: Hiroyuki Nishi <hni...@yahoo-corp.jp>
Authored: Fri Jul 15 15:54:07 2016 +0200
Committer: Aleksey Yeschenko <alek...@apache.org>
Committed: Fri Jul 15 15:39:50 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +-
 .../cassandra/tools/nodetool/TableStats.java    | 259 +------------
 .../cassandra/tools/nodetool/TpStats.java       |  34 +-
 .../tools/nodetool/stats/StatsHolder.java       |  91 +----
 .../tools/nodetool/stats/StatsPrinter.java      |  41 +-
 .../tools/nodetool/stats/TableStatsHolder.java  | 373 +++++++++++++++++++
 .../tools/nodetool/stats/TableStatsPrinter.java |  53 +--
 .../tools/nodetool/stats/TpStatsHolder.java     |  63 ++++
 .../tools/nodetool/stats/TpStatsPrinter.java    |  73 ++++
 9 files changed, 581 insertions(+), 409 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/27d25c6d/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index caa0625..f8c9964 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.10
+ * Support json/yaml output in nodetool tpstats (CASSANDRA-12035)
  * Expose metrics for successful/failed authentication attempts 
(CASSANDRA-10635)
  * Prepend snapshot name with "truncated" or "dropped" when a snapshot
    is taken before truncating or dropping a table (CASSANDRA-12178)
@@ -149,7 +150,7 @@ Merged from 2.1:
  * Add support to rebuild from specific range (CASSANDRA-10406)
  * Optimize the overlapping lookup by calculating all the
    bounds in advance (CASSANDRA-11571)
- * Support json/yaml output in noetool tablestats (CASSANDRA-5977)
+ * Support json/yaml output in nodetool tablestats (CASSANDRA-5977)
  * (stress) Add datacenter option to -node options (CASSANDRA-11591)
  * Fix handling of empty slices (CASSANDRA-11513)
  * Make number of cores used by cqlsh COPY visible to testing code 
(CASSANDRA-11437)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/27d25c6d/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 d948be3..53a3066 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/TableStats.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/TableStats.java
@@ -30,10 +30,7 @@ import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.metrics.CassandraMetricsRegistry;
 import org.apache.cassandra.tools.NodeProbe;
 import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
-import org.apache.cassandra.tools.nodetool.stats.StatsHolder;
-import org.apache.cassandra.tools.nodetool.stats.StatsKeyspace;
-import org.apache.cassandra.tools.nodetool.stats.StatsTable;
-import org.apache.cassandra.tools.nodetool.stats.TableStatsPrinter;
+import org.apache.cassandra.tools.nodetool.stats.*;
 
 @Command(name = "tablestats", description = "Print statistics on tables")
 public class TableStats extends NodeToolCmd
@@ -62,260 +59,10 @@ public class TableStats extends NodeToolCmd
             throw new IllegalArgumentException("arguments for -F are json,yaml 
only.");
         }
 
-        TableStats.OptionFilter filter = new OptionFilter(ignore, tableNames);
-        ArrayListMultimap<String, ColumnFamilyStoreMBean> selectedTableMbeans 
= ArrayListMultimap.create();
-        Map<String, StatsKeyspace> keyspaceStats = new HashMap<>();
-
-        // get a list of table stores
-        Iterator<Map.Entry<String, ColumnFamilyStoreMBean>> tableMBeans = 
probe.getColumnFamilyStoreMBeanProxies();
-
-        while (tableMBeans.hasNext())
-        {
-            Map.Entry<String, ColumnFamilyStoreMBean> entry = 
tableMBeans.next();
-            String keyspaceName = entry.getKey();
-            ColumnFamilyStoreMBean tableProxy = entry.getValue();
-
-            if (filter.isKeyspaceIncluded(keyspaceName))
-            {
-                StatsKeyspace stats = keyspaceStats.get(keyspaceName);
-                if (stats == null)
-                {
-                    stats = new StatsKeyspace(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.verifyTables();
-
-        // get metrics of keyspace
-        StatsHolder holder = new StatsHolder(probe.getNumberOfTables());
-        for (Map.Entry<String, Collection<ColumnFamilyStoreMBean>> entry : 
selectedTableMbeans.asMap().entrySet())
-        {
-            String keyspaceName = entry.getKey();
-            Collection<ColumnFamilyStoreMBean> tables = entry.getValue();
-            StatsKeyspace statsKeyspace = keyspaceStats.get(keyspaceName);
-
-            // get metrics of table statistics for this keyspace
-            for (ColumnFamilyStoreMBean table : tables)
-            {
-                String tableName = table.getTableName();
-                StatsTable statsTable = new StatsTable();
-                statsTable.name = tableName;
-                statsTable.isIndex = tableName.contains(".");
-                statsTable.sstableCount = 
probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveSSTableCount");
-                int[] leveledSStables = table.getSSTableCountPerLevel();
-                if (leveledSStables != null)
-                {
-                    statsTable.isLeveledSstable = true;
-
-                    for (int level = 0; level < leveledSStables.length; 
level++)
-                    {
-                        int count = leveledSStables[level];
-                        long maxCount = 4L; // for L0
-                        if (level > 0)
-                            maxCount = (long) Math.pow(10, level);
-                        // show max threshold for level when exceeded
-                        statsTable.sstablesInEachLevel.add(count + ((count > 
maxCount) ? "/" + maxCount : ""));
-                    }
-                }
-
-                Long memtableOffHeapSize = null;
-                Long bloomFilterOffHeapSize = null;
-                Long indexSummaryOffHeapSize = null;
-                Long compressionMetadataOffHeapSize = null;
-                Long offHeapSize = null;
-                Double percentRepaired = null;
-
-                try
-                {
-                    memtableOffHeapSize = (Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableOffHeapSize");
-                    bloomFilterOffHeapSize = (Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, 
"BloomFilterOffHeapMemoryUsed");
-                    indexSummaryOffHeapSize = (Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, 
"IndexSummaryOffHeapMemoryUsed");
-                    compressionMetadataOffHeapSize = (Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, 
"CompressionMetadataOffHeapMemoryUsed");
-                    offHeapSize = memtableOffHeapSize + bloomFilterOffHeapSize 
+ indexSummaryOffHeapSize + compressionMetadataOffHeapSize;
-                    percentRepaired = (Double) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "PercentRepaired");
-                }
-                catch (RuntimeException e)
-                {
-                    // offheap-metrics introduced in 2.1.3 - older versions do 
not have the appropriate mbeans
-                    if (!(e.getCause() instanceof InstanceNotFoundException))
-                        throw e;
-                }
-
-                statsTable.spaceUsedLive = format((Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveDiskSpaceUsed"), 
humanReadable);
-                statsTable.spaceUsedTotal = format((Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "TotalDiskSpaceUsed"), 
humanReadable);
-                statsTable.spaceUsedBySnapshotsTotal = format((Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "SnapshotsSize"), 
humanReadable);
-                if (offHeapSize != null)
-                {
-                    statsTable.offHeapUsed = true;
-                    statsTable.offHeapMemoryUsedTotal = format(offHeapSize, 
humanReadable);
-
-                }
-                if (percentRepaired != null)
-                {
-                    statsTable.percentRepaired = Math.round(100 * 
percentRepaired) / 100.0;
-                }
-                statsTable.sstableCompressionRatio = 
probe.getColumnFamilyMetric(keyspaceName, tableName, "CompressionRatio");
-                Object estimatedPartitionCount = 
probe.getColumnFamilyMetric(keyspaceName, tableName, "EstimatedPartitionCount");
-                if (Long.valueOf(-1L).equals(estimatedPartitionCount))
-                {
-                    estimatedPartitionCount = 0L;
-                }
-                statsTable.numberOfKeysEstimate = estimatedPartitionCount;
-
-                statsTable.memtableCellCount = 
probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableColumnsCount");
-                statsTable.memtableDataSize = format((Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableLiveDataSize"), 
humanReadable);
-                if (memtableOffHeapSize != null)
-                {
-                    statsTable.memtableOffHeapUsed = true;
-                    statsTable.memtableOffHeapMemoryUsed = 
format(memtableOffHeapSize, humanReadable);
-                }
-                statsTable.memtableSwitchCount = 
probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableSwitchCount");
-                statsTable.localReadCount = 
((CassandraMetricsRegistry.JmxTimerMBean) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "ReadLatency")).getCount();
-
-                double localReadLatency = 
((CassandraMetricsRegistry.JmxTimerMBean) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "ReadLatency")).getMean() 
/ 1000;
-                double localRLatency = localReadLatency > 0 ? localReadLatency 
: Double.NaN;
-                statsTable.localReadLatencyMs = localRLatency;
-                statsTable.localWriteCount = 
((CassandraMetricsRegistry.JmxTimerMBean) 
probe.getColumnFamilyMetric(keyspaceName, tableName, 
"WriteLatency")).getCount();
-
-                double localWriteLatency = 
((CassandraMetricsRegistry.JmxTimerMBean) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "WriteLatency")).getMean() 
/ 1000;
-                double localWLatency = localWriteLatency > 0 ? 
localWriteLatency : Double.NaN;
-                statsTable.localWriteLatencyMs = localWLatency;
-                statsTable.pendingFlushes = 
probe.getColumnFamilyMetric(keyspaceName, tableName, "PendingFlushes");
-
-                statsTable.bloomFilterFalsePositives = 
probe.getColumnFamilyMetric(keyspaceName, tableName, 
"BloomFilterFalsePositives");
-                statsTable.bloomFilterFalseRatio = 
probe.getColumnFamilyMetric(keyspaceName, tableName, 
"RecentBloomFilterFalseRatio");
-                statsTable.bloomFilterSpaceUsed = format((Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, 
"BloomFilterDiskSpaceUsed"), humanReadable);
-
-                if (bloomFilterOffHeapSize != null)
-                {
-                    statsTable.bloomFilterOffHeapUsed = true;
-                    statsTable.bloomFilterOffHeapMemoryUsed = 
format(bloomFilterOffHeapSize, humanReadable);
-                }
-
-                if (indexSummaryOffHeapSize != null)
-                {
-                    statsTable.indexSummaryOffHeapUsed = true;
-                    statsTable.indexSummaryOffHeapMemoryUsed = 
format(indexSummaryOffHeapSize, humanReadable);
-                }
-                if (compressionMetadataOffHeapSize != null)
-                {
-                    statsTable.compressionMetadataOffHeapUsed = true;
-                    statsTable.compressionMetadataOffHeapMemoryUsed = 
format(compressionMetadataOffHeapSize, humanReadable);
-                }
-                statsTable.compactedPartitionMinimumBytes = (Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "MinPartitionSize");
-                statsTable.compactedPartitionMaximumBytes = (Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "MaxPartitionSize");
-                statsTable.compactedPartitionMeanBytes = (Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "MeanPartitionSize");
-
-                CassandraMetricsRegistry.JmxHistogramMBean histogram = 
(CassandraMetricsRegistry.JmxHistogramMBean) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveScannedHistogram");
-                statsTable.averageLiveCellsPerSliceLastFiveMinutes = 
histogram.getMean();
-                statsTable.maximumLiveCellsPerSliceLastFiveMinutes = 
histogram.getMax();
-
-                histogram = (CassandraMetricsRegistry.JmxHistogramMBean) 
probe.getColumnFamilyMetric(keyspaceName, tableName, 
"TombstoneScannedHistogram");
-                statsTable.averageTombstonesPerSliceLastFiveMinutes = 
histogram.getMean();
-                statsTable.maximumTombstonesPerSliceLastFiveMinutes = 
histogram.getMax();
-                statsTable.droppedMutations = format((Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "DroppedMutations"), 
humanReadable);
-                statsKeyspace.tables.add(statsTable);
-            }
-            holder.keyspaces.add(statsKeyspace);
-        }
+        StatsHolder holder = new TableStatsHolder(probe, humanReadable, 
ignore, tableNames);
         // print out the keyspace and table statistics
-        TableStatsPrinter printer = TableStatsPrinter.from(outputFormat);
+        StatsPrinter printer = TableStatsPrinter.from(outputFormat);
         printer.print(holder, System.out);
     }
 
-    private String format(long bytes, boolean humanReadable)
-    {
-        return humanReadable ? FileUtils.stringifyFileSize(bytes) : 
Long.toString(bytes);
-    }
-
-    /**
-     * Used for filtering keyspaces and tables to be displayed using the 
tablestats command.
-     */
-    private static class OptionFilter
-    {
-        private final Map<String, List<String>> filter = new HashMap<>();
-        private final 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 final List<String> filterList = new ArrayList<>();
-        private final boolean ignoreMode;
-
-        OptionFilter(boolean ignoreMode, List<String> filterList)
-        {
-            this.filterList.addAll(filterList);
-            this.ignoreMode = ignoreMode;
-
-            for (String s : filterList)
-            {
-                String[] keyValues = s.split("\\.", 2);
-
-                // build the map that stores the keyspaces and tables to use
-                if (!filter.containsKey(keyValues[0]))
-                {
-                    filter.put(keyValues[0], new ArrayList<>());
-                    verifier.put(keyValues[0], new ArrayList<>());
-                }
-
-                if (keyValues.length == 2)
-                {
-                    filter.get(keyValues[0]).add(keyValues[1]);
-                    verifier.get(keyValues[0]).add(keyValues[1]);
-                }
-            }
-        }
-
-        public boolean isTableIncluded(String keyspace, String table)
-        {
-            // supplying empty params list is treated as wanting to display 
all keyspaces and tables
-            if (filterList.isEmpty())
-                return !ignoreMode;
-
-            List<String> tables = filter.get(keyspace);
-
-            // no such keyspace is in the map
-            if (tables == null)
-                return ignoreMode;
-                // only a keyspace with no tables was supplied
-                // so ignore or include (based on the flag) every column 
family in specified keyspace
-            else if (tables.isEmpty())
-                return !ignoreMode;
-
-            // keyspace exists, and it contains specific table
-            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)
-        {
-            for (String ks : verifier.keySet())
-                if (!keyspaces.contains(ks))
-                    throw new IllegalArgumentException("Unknown keyspace: " + 
ks);
-        }
-
-        public void verifyTables()
-        {
-            for (String ks : filter.keySet())
-                if (!verifier.get(ks).isEmpty())
-                    throw new IllegalArgumentException("Unknown tables: " + 
verifier.get(ks) + " in keyspace: " + ks);
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/27d25c6d/src/java/org/apache/cassandra/tools/nodetool/TpStats.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/TpStats.java 
b/src/java/org/apache/cassandra/tools/nodetool/TpStats.java
index 5d3eab7..0cf8e50 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/TpStats.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/TpStats.java
@@ -19,36 +19,32 @@ package org.apache.cassandra.tools.nodetool;
 
 import io.airlift.command.Command;
 
-import java.util.Map;
-
-import com.google.common.collect.Multimap;
-
+import io.airlift.command.Option;
 import org.apache.cassandra.tools.NodeProbe;
 import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+import org.apache.cassandra.tools.nodetool.stats.TpStatsHolder;
+import org.apache.cassandra.tools.nodetool.stats.TpStatsPrinter;
+import org.apache.cassandra.tools.nodetool.stats.*;
+
 
 @Command(name = "tpstats", description = "Print usage statistics of thread 
pools")
 public class TpStats extends NodeToolCmd
 {
+    @Option(title = "format",
+            name = {"-F", "--format"},
+            description = "Output format (json, yaml)")
+    private String outputFormat = "";
+
     @Override
     public void execute(NodeProbe probe)
     {
-        System.out.printf("%-25s%10s%10s%15s%10s%18s%n", "Pool Name", 
"Active", "Pending", "Completed", "Blocked", "All time blocked");
-
-
-        Multimap<String, String> threadPools = probe.getThreadPools();
-        for (Map.Entry<String, String> tpool : threadPools.entries())
+        if (!outputFormat.isEmpty() && !"json".equals(outputFormat) && 
!"yaml".equals(outputFormat))
         {
-            System.out.printf("%-25s%10s%10s%15s%10s%18s%n",
-                              tpool.getValue(),
-                              probe.getThreadPoolMetric(tpool.getKey(), 
tpool.getValue(), "ActiveTasks"),
-                              probe.getThreadPoolMetric(tpool.getKey(), 
tpool.getValue(), "PendingTasks"),
-                              probe.getThreadPoolMetric(tpool.getKey(), 
tpool.getValue(), "CompletedTasks"),
-                              probe.getThreadPoolMetric(tpool.getKey(), 
tpool.getValue(), "CurrentlyBlockedTasks"),
-                              probe.getThreadPoolMetric(tpool.getKey(), 
tpool.getValue(), "TotalBlockedTasks"));
+            throw new IllegalArgumentException("arguments for -F are json,yaml 
only.");
         }
 
-        System.out.printf("%n%-20s%10s%n", "Message type", "Dropped");
-        for (Map.Entry<String, Integer> entry : 
probe.getDroppedMessages().entrySet())
-            System.out.printf("%-20s%10s%n", entry.getKey(), entry.getValue());
+        StatsHolder data = new TpStatsHolder(probe);
+        StatsPrinter printer = TpStatsPrinter.from(outputFormat);
+        printer.print(data, System.out);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/27d25c6d/src/java/org/apache/cassandra/tools/nodetool/stats/StatsHolder.java
----------------------------------------------------------------------
diff --git 
a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsHolder.java 
b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsHolder.java
index 28768e6..c35e1fe 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsHolder.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsHolder.java
@@ -18,91 +18,12 @@
 
 package org.apache.cassandra.tools.nodetool.stats;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
-public class StatsHolder
+/**
+ * Interface for the Stats property bag
+ */
+public interface StatsHolder
 {
-    public List<StatsKeyspace> keyspaces;
-    public final int numberOfTables;
-
-    public StatsHolder(int numberOfTables)
-    {
-        keyspaces = new ArrayList<>();
-        this.numberOfTables = numberOfTables;
-    }
-
-    public Map<String, Object> convert2Map()
-    {
-        HashMap<String, Object> mpRet = new HashMap<>();
-        mpRet.put("total_number_of_tables", numberOfTables);
-        for (StatsKeyspace keyspace : keyspaces)
-        {
-            // store each keyspace's metrics to map
-            HashMap<String, Object> mpKeyspace = new HashMap<>();
-            mpKeyspace.put("read_latency", keyspace.readLatency());
-            mpKeyspace.put("read_count", keyspace.readCount);
-            mpKeyspace.put("read_latency_ms", keyspace.readLatency());
-            mpKeyspace.put("write_count", keyspace.writeCount);
-            mpKeyspace.put("write_latency_ms", keyspace.writeLatency());
-            mpKeyspace.put("pending_flushes", keyspace.pendingFlushes);
-
-            // store each table's metrics to map
-            List<StatsTable> tables = keyspace.tables;
-            Map<String, Map<String, Object>> mpTables = new HashMap<>();
-            for (StatsTable table : tables)
-            {
-                Map<String, Object> mpTable = new HashMap<>();
-
-                mpTable.put("sstables_in_each_level", 
table.sstablesInEachLevel);
-                mpTable.put("space_used_live", table.spaceUsedLive);
-                mpTable.put("space_used_total", table.spaceUsedTotal);
-                mpTable.put("space_used_by_snapshots_total", 
table.spaceUsedBySnapshotsTotal);
-                if (table.offHeapUsed)
-                    mpTable.put("off_heap_memory_used_total", 
table.offHeapMemoryUsedTotal);
-                mpTable.put("sstable_compression_ratio", 
table.sstableCompressionRatio);
-                mpTable.put("number_of_keys_estimate", 
table.numberOfKeysEstimate);
-                mpTable.put("memtable_cell_count", table.memtableCellCount);
-                mpTable.put("memtable_data_size", table.memtableDataSize);
-                if (table.memtableOffHeapUsed)
-                    mpTable.put("memtable_off_heap_memory_used", 
table.memtableOffHeapMemoryUsed);
-                mpTable.put("memtable_switch_count", 
table.memtableSwitchCount);
-                mpTable.put("local_read_count", table.localReadCount);
-                mpTable.put("local_read_latency_ms", String.format("%01.3f", 
table.localReadLatencyMs));
-                mpTable.put("local_write_count", table.localWriteCount);
-                mpTable.put("local_write_latency_ms", String.format("%01.3f", 
table.localWriteLatencyMs));
-                mpTable.put("pending_flushes", table.pendingFlushes);
-                mpTable.put("percent_repaired", table.percentRepaired);
-                mpTable.put("bloom_filter_false_positives", 
table.bloomFilterFalsePositives);
-                mpTable.put("bloom_filter_false_ratio", 
String.format("%01.5f", table.bloomFilterFalseRatio));
-                mpTable.put("bloom_filter_space_used", 
table.bloomFilterSpaceUsed);
-                if (table.bloomFilterOffHeapUsed)
-                    mpTable.put("bloom_filter_off_heap_memory_used", 
table.bloomFilterOffHeapMemoryUsed);
-                if (table.indexSummaryOffHeapUsed)
-                    mpTable.put("index_summary_off_heap_memory_used", 
table.indexSummaryOffHeapMemoryUsed);
-                if (table.compressionMetadataOffHeapUsed)
-                    mpTable.put("compression_metadata_off_heap_memory_used",
-                                table.compressionMetadataOffHeapMemoryUsed);
-                mpTable.put("compacted_partition_minimum_bytes", 
table.compactedPartitionMinimumBytes);
-                mpTable.put("compacted_partition_maximum_bytes", 
table.compactedPartitionMaximumBytes);
-                mpTable.put("compacted_partition_mean_bytes", 
table.compactedPartitionMeanBytes);
-                mpTable.put("average_live_cells_per_slice_last_five_minutes",
-                            table.averageLiveCellsPerSliceLastFiveMinutes);
-                mpTable.put("maximum_live_cells_per_slice_last_five_minutes",
-                            table.maximumLiveCellsPerSliceLastFiveMinutes);
-                mpTable.put("average_tombstones_per_slice_last_five_minutes",
-                            table.averageTombstonesPerSliceLastFiveMinutes);
-                mpTable.put("maximum_tombstones_per_slice_last_five_minutes",
-                            table.maximumTombstonesPerSliceLastFiveMinutes);
-                mpTable.put("dropped_mutations", table.droppedMutations);
-
-                mpTables.put(table.name, mpTable);
-            }
-            mpKeyspace.put("tables", mpTables);
-            mpRet.put(keyspace.name, mpKeyspace);
-        }
-        return mpRet;
-    }
-}
+    public Map<String, Object> convert2Map();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/27d25c6d/src/java/org/apache/cassandra/tools/nodetool/stats/StatsPrinter.java
----------------------------------------------------------------------
diff --git 
a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsPrinter.java 
b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsPrinter.java
index 2d98781..e67f33a 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsPrinter.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsPrinter.java
@@ -15,11 +15,46 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.cassandra.tools.nodetool.stats;
 
 import java.io.PrintStream;
 
-public interface StatsPrinter<T>
+import org.json.simple.JSONObject;
+import org.yaml.snakeyaml.DumperOptions;
+import org.yaml.snakeyaml.Yaml;
+
+/**
+ * Interface for the Stats printer, that'd output statistics
+ * given the {@code StatsHolder}
+ *
+ * @param <T> Stats property bad type
+ */
+public interface StatsPrinter<T extends StatsHolder>
 {
-    void printFormat(T data, PrintStream out);
-}
+    void print(T data, PrintStream out);
+
+    static class JsonPrinter<T extends StatsHolder> implements StatsPrinter<T>
+    {
+        @Override
+        public void print(T data, PrintStream out)
+        {
+            JSONObject json = new JSONObject();
+            json.putAll(data.convert2Map());
+            out.println(json.toString());
+        }
+    }
+
+    static class YamlPrinter<T extends StatsHolder> implements StatsPrinter<T>
+    {
+        @Override
+        public void print(T data, PrintStream out)
+        {
+            DumperOptions options = new DumperOptions();
+            options.setDefaultFlowStyle(DumperOptions.FlowStyle.BLOCK);
+
+            Yaml yaml = new Yaml(options);
+            out.println(yaml.dump(data.convert2Map()));
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/27d25c6d/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java
----------------------------------------------------------------------
diff --git 
a/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java 
b/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java
new file mode 100644
index 0000000..dd56d8a
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java
@@ -0,0 +1,373 @@
+/*
+ * 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.tools.nodetool.stats;
+
+import java.util.*;
+
+import javax.management.InstanceNotFoundException;
+
+import com.google.common.collect.ArrayListMultimap;
+
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.io.util.*;
+import org.apache.cassandra.metrics.*;
+import org.apache.cassandra.tools.*;
+import org.apache.cassandra.tools.nodetool.*;
+import org.apache.cassandra.tools.nodetool.stats.*;
+
+public class TableStatsHolder implements StatsHolder
+{
+    public final List<StatsKeyspace> keyspaces;
+    public final int numberOfTables;
+
+    public TableStatsHolder(NodeProbe probe, boolean humanReadable, boolean 
ignore, List<String> tableNames)
+    {
+        this.keyspaces = new ArrayList<>();
+        this.numberOfTables = probe.getNumberOfTables();
+        this.initializeKeyspaces(probe, humanReadable, ignore, tableNames);
+    }
+
+    @Override
+    public Map<String, Object> convert2Map()
+    {
+        HashMap<String, Object> mpRet = new HashMap<>();
+        mpRet.put("total_number_of_tables", numberOfTables);
+        for (StatsKeyspace keyspace : keyspaces)
+        {
+            // store each keyspace's metrics to map
+            HashMap<String, Object> mpKeyspace = new HashMap<>();
+            mpKeyspace.put("read_latency", keyspace.readLatency());
+            mpKeyspace.put("read_count", keyspace.readCount);
+            mpKeyspace.put("read_latency_ms", keyspace.readLatency());
+            mpKeyspace.put("write_count", keyspace.writeCount);
+            mpKeyspace.put("write_latency_ms", keyspace.writeLatency());
+            mpKeyspace.put("pending_flushes", keyspace.pendingFlushes);
+
+            // store each table's metrics to map
+            List<StatsTable> tables = keyspace.tables;
+            Map<String, Map<String, Object>> mpTables = new HashMap<>();
+            for (StatsTable table : tables)
+            {
+                Map<String, Object> mpTable = new HashMap<>();
+
+                mpTable.put("sstables_in_each_level", 
table.sstablesInEachLevel);
+                mpTable.put("space_used_live", table.spaceUsedLive);
+                mpTable.put("space_used_total", table.spaceUsedTotal);
+                mpTable.put("space_used_by_snapshots_total", 
table.spaceUsedBySnapshotsTotal);
+                if (table.offHeapUsed)
+                    mpTable.put("off_heap_memory_used_total", 
table.offHeapMemoryUsedTotal);
+                mpTable.put("sstable_compression_ratio", 
table.sstableCompressionRatio);
+                mpTable.put("number_of_keys_estimate", 
table.numberOfKeysEstimate);
+                mpTable.put("memtable_cell_count", table.memtableCellCount);
+                mpTable.put("memtable_data_size", table.memtableDataSize);
+                if (table.memtableOffHeapUsed)
+                    mpTable.put("memtable_off_heap_memory_used", 
table.memtableOffHeapMemoryUsed);
+                mpTable.put("memtable_switch_count", 
table.memtableSwitchCount);
+                mpTable.put("local_read_count", table.localReadCount);
+                mpTable.put("local_read_latency_ms", String.format("%01.3f", 
table.localReadLatencyMs));
+                mpTable.put("local_write_count", table.localWriteCount);
+                mpTable.put("local_write_latency_ms", String.format("%01.3f", 
table.localWriteLatencyMs));
+                mpTable.put("pending_flushes", table.pendingFlushes);
+                mpTable.put("percent_repaired", table.percentRepaired);
+                mpTable.put("bloom_filter_false_positives", 
table.bloomFilterFalsePositives);
+                mpTable.put("bloom_filter_false_ratio", 
String.format("%01.5f", table.bloomFilterFalseRatio));
+                mpTable.put("bloom_filter_space_used", 
table.bloomFilterSpaceUsed);
+                if (table.bloomFilterOffHeapUsed)
+                    mpTable.put("bloom_filter_off_heap_memory_used", 
table.bloomFilterOffHeapMemoryUsed);
+                if (table.indexSummaryOffHeapUsed)
+                    mpTable.put("index_summary_off_heap_memory_used", 
table.indexSummaryOffHeapMemoryUsed);
+                if (table.compressionMetadataOffHeapUsed)
+                    mpTable.put("compression_metadata_off_heap_memory_used",
+                                table.compressionMetadataOffHeapMemoryUsed);
+                mpTable.put("compacted_partition_minimum_bytes", 
table.compactedPartitionMinimumBytes);
+                mpTable.put("compacted_partition_maximum_bytes", 
table.compactedPartitionMaximumBytes);
+                mpTable.put("compacted_partition_mean_bytes", 
table.compactedPartitionMeanBytes);
+                mpTable.put("average_live_cells_per_slice_last_five_minutes",
+                            table.averageLiveCellsPerSliceLastFiveMinutes);
+                mpTable.put("maximum_live_cells_per_slice_last_five_minutes",
+                            table.maximumLiveCellsPerSliceLastFiveMinutes);
+                mpTable.put("average_tombstones_per_slice_last_five_minutes",
+                            table.averageTombstonesPerSliceLastFiveMinutes);
+                mpTable.put("maximum_tombstones_per_slice_last_five_minutes",
+                            table.maximumTombstonesPerSliceLastFiveMinutes);
+                mpTable.put("dropped_mutations", table.droppedMutations);
+
+                mpTables.put(table.name, mpTable);
+            }
+            mpKeyspace.put("tables", mpTables);
+            mpRet.put(keyspace.name, mpKeyspace);
+        }
+        return mpRet;
+    }
+
+    private void initializeKeyspaces(NodeProbe probe, boolean humanReadable, 
boolean ignore, List<String> tableNames)
+    {
+        OptionFilter filter = new OptionFilter(ignore, tableNames);
+        ArrayListMultimap<String, ColumnFamilyStoreMBean> selectedTableMbeans 
= ArrayListMultimap.create();
+        Map<String, StatsKeyspace> keyspaceStats = new HashMap<>();
+
+        // get a list of table stores
+        Iterator<Map.Entry<String, ColumnFamilyStoreMBean>> tableMBeans = 
probe.getColumnFamilyStoreMBeanProxies();
+
+        while (tableMBeans.hasNext())
+        {
+            Map.Entry<String, ColumnFamilyStoreMBean> entry = 
tableMBeans.next();
+            String keyspaceName = entry.getKey();
+            ColumnFamilyStoreMBean tableProxy = entry.getValue();
+
+            if (filter.isKeyspaceIncluded(keyspaceName))
+            {
+                StatsKeyspace stats = keyspaceStats.get(keyspaceName);
+                if (stats == null)
+                {
+                    stats = new StatsKeyspace(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.verifyTables();
+
+        // get metrics of keyspace
+        for (Map.Entry<String, Collection<ColumnFamilyStoreMBean>> entry : 
selectedTableMbeans.asMap().entrySet())
+        {
+            String keyspaceName = entry.getKey();
+            Collection<ColumnFamilyStoreMBean> tables = entry.getValue();
+            StatsKeyspace statsKeyspace = keyspaceStats.get(keyspaceName);
+
+            // get metrics of table statistics for this keyspace
+            for (ColumnFamilyStoreMBean table : tables)
+            {
+                String tableName = table.getTableName();
+                StatsTable statsTable = new StatsTable();
+                statsTable.name = tableName;
+                statsTable.isIndex = tableName.contains(".");
+                statsTable.sstableCount = 
probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveSSTableCount");
+                int[] leveledSStables = table.getSSTableCountPerLevel();
+                if (leveledSStables != null)
+                {
+                    statsTable.isLeveledSstable = true;
+
+                    for (int level = 0; level < leveledSStables.length; 
level++)
+                    {
+                        int count = leveledSStables[level];
+                        long maxCount = 4L; // for L0
+                        if (level > 0)
+                            maxCount = (long) Math.pow(10, level);
+                        // show max threshold for level when exceeded
+                        statsTable.sstablesInEachLevel.add(count + ((count > 
maxCount) ? "/" + maxCount : ""));
+                    }
+                }
+
+                Long memtableOffHeapSize = null;
+                Long bloomFilterOffHeapSize = null;
+                Long indexSummaryOffHeapSize = null;
+                Long compressionMetadataOffHeapSize = null;
+                Long offHeapSize = null;
+                Double percentRepaired = null;
+
+                try
+                {
+                    memtableOffHeapSize = (Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableOffHeapSize");
+                    bloomFilterOffHeapSize = (Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, 
"BloomFilterOffHeapMemoryUsed");
+                    indexSummaryOffHeapSize = (Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, 
"IndexSummaryOffHeapMemoryUsed");
+                    compressionMetadataOffHeapSize = (Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, 
"CompressionMetadataOffHeapMemoryUsed");
+                    offHeapSize = memtableOffHeapSize + bloomFilterOffHeapSize 
+ indexSummaryOffHeapSize + compressionMetadataOffHeapSize;
+                    percentRepaired = (Double) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "PercentRepaired");
+                }
+                catch (RuntimeException e)
+                {
+                    // offheap-metrics introduced in 2.1.3 - older versions do 
not have the appropriate mbeans
+                    if (!(e.getCause() instanceof InstanceNotFoundException))
+                        throw e;
+                }
+
+                statsTable.spaceUsedLive = format((Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveDiskSpaceUsed"), 
humanReadable);
+                statsTable.spaceUsedTotal = format((Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "TotalDiskSpaceUsed"), 
humanReadable);
+                statsTable.spaceUsedBySnapshotsTotal = format((Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "SnapshotsSize"), 
humanReadable);
+                if (offHeapSize != null)
+                {
+                    statsTable.offHeapUsed = true;
+                    statsTable.offHeapMemoryUsedTotal = format(offHeapSize, 
humanReadable);
+
+                }
+                if (percentRepaired != null)
+                {
+                    statsTable.percentRepaired = Math.round(100 * 
percentRepaired) / 100.0;
+                }
+                statsTable.sstableCompressionRatio = 
probe.getColumnFamilyMetric(keyspaceName, tableName, "CompressionRatio");
+                Object estimatedPartitionCount = 
probe.getColumnFamilyMetric(keyspaceName, tableName, "EstimatedPartitionCount");
+                if (Long.valueOf(-1L).equals(estimatedPartitionCount))
+                {
+                    estimatedPartitionCount = 0L;
+                }
+                statsTable.numberOfKeysEstimate = estimatedPartitionCount;
+
+                statsTable.memtableCellCount = 
probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableColumnsCount");
+                statsTable.memtableDataSize = format((Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableLiveDataSize"), 
humanReadable);
+                if (memtableOffHeapSize != null)
+                {
+                    statsTable.memtableOffHeapUsed = true;
+                    statsTable.memtableOffHeapMemoryUsed = 
format(memtableOffHeapSize, humanReadable);
+                }
+                statsTable.memtableSwitchCount = 
probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableSwitchCount");
+                statsTable.localReadCount = 
((CassandraMetricsRegistry.JmxTimerMBean) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "ReadLatency")).getCount();
+
+                double localReadLatency = 
((CassandraMetricsRegistry.JmxTimerMBean) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "ReadLatency")).getMean() 
/ 1000;
+                double localRLatency = localReadLatency > 0 ? localReadLatency 
: Double.NaN;
+                statsTable.localReadLatencyMs = localRLatency;
+                statsTable.localWriteCount = 
((CassandraMetricsRegistry.JmxTimerMBean) 
probe.getColumnFamilyMetric(keyspaceName, tableName, 
"WriteLatency")).getCount();
+
+                double localWriteLatency = 
((CassandraMetricsRegistry.JmxTimerMBean) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "WriteLatency")).getMean() 
/ 1000;
+                double localWLatency = localWriteLatency > 0 ? 
localWriteLatency : Double.NaN;
+                statsTable.localWriteLatencyMs = localWLatency;
+                statsTable.pendingFlushes = 
probe.getColumnFamilyMetric(keyspaceName, tableName, "PendingFlushes");
+
+                statsTable.bloomFilterFalsePositives = 
probe.getColumnFamilyMetric(keyspaceName, tableName, 
"BloomFilterFalsePositives");
+                statsTable.bloomFilterFalseRatio = 
probe.getColumnFamilyMetric(keyspaceName, tableName, 
"RecentBloomFilterFalseRatio");
+                statsTable.bloomFilterSpaceUsed = format((Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, 
"BloomFilterDiskSpaceUsed"), humanReadable);
+
+                if (bloomFilterOffHeapSize != null)
+                {
+                    statsTable.bloomFilterOffHeapUsed = true;
+                    statsTable.bloomFilterOffHeapMemoryUsed = 
format(bloomFilterOffHeapSize, humanReadable);
+                }
+
+                if (indexSummaryOffHeapSize != null)
+                {
+                    statsTable.indexSummaryOffHeapUsed = true;
+                    statsTable.indexSummaryOffHeapMemoryUsed = 
format(indexSummaryOffHeapSize, humanReadable);
+                }
+                if (compressionMetadataOffHeapSize != null)
+                {
+                    statsTable.compressionMetadataOffHeapUsed = true;
+                    statsTable.compressionMetadataOffHeapMemoryUsed = 
format(compressionMetadataOffHeapSize, humanReadable);
+                }
+                statsTable.compactedPartitionMinimumBytes = (Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "MinPartitionSize");
+                statsTable.compactedPartitionMaximumBytes = (Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "MaxPartitionSize");
+                statsTable.compactedPartitionMeanBytes = (Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "MeanPartitionSize");
+
+                CassandraMetricsRegistry.JmxHistogramMBean histogram = 
(CassandraMetricsRegistry.JmxHistogramMBean) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveScannedHistogram");
+                statsTable.averageLiveCellsPerSliceLastFiveMinutes = 
histogram.getMean();
+                statsTable.maximumLiveCellsPerSliceLastFiveMinutes = 
histogram.getMax();
+
+                histogram = (CassandraMetricsRegistry.JmxHistogramMBean) 
probe.getColumnFamilyMetric(keyspaceName, tableName, 
"TombstoneScannedHistogram");
+                statsTable.averageTombstonesPerSliceLastFiveMinutes = 
histogram.getMean();
+                statsTable.maximumTombstonesPerSliceLastFiveMinutes = 
histogram.getMax();
+                statsTable.droppedMutations = format((Long) 
probe.getColumnFamilyMetric(keyspaceName, tableName, "DroppedMutations"), 
humanReadable);
+                statsKeyspace.tables.add(statsTable);
+            }
+            keyspaces.add(statsKeyspace);
+        }
+    }
+
+    private String format(long bytes, boolean humanReadable)
+    {
+        return humanReadable ? FileUtils.stringifyFileSize(bytes) : 
Long.toString(bytes);
+    }
+
+    /**
+     * Used for filtering keyspaces and tables to be displayed using the 
tablestats command.
+     */
+    private static class OptionFilter
+    {
+        private final Map<String, List<String>> filter = new HashMap<>();
+        private final 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 final List<String> filterList = new ArrayList<>();
+        private final boolean ignoreMode;
+
+        OptionFilter(boolean ignoreMode, List<String> filterList)
+        {
+            this.filterList.addAll(filterList);
+            this.ignoreMode = ignoreMode;
+
+            for (String s : filterList)
+            {
+                String[] keyValues = s.split("\\.", 2);
+
+                // build the map that stores the keyspaces and tables to use
+                if (!filter.containsKey(keyValues[0]))
+                {
+                    filter.put(keyValues[0], new ArrayList<>());
+                    verifier.put(keyValues[0], new ArrayList<>());
+                }
+
+                if (keyValues.length == 2)
+                {
+                    filter.get(keyValues[0]).add(keyValues[1]);
+                    verifier.get(keyValues[0]).add(keyValues[1]);
+                }
+            }
+        }
+
+        public boolean isTableIncluded(String keyspace, String table)
+        {
+            // supplying empty params list is treated as wanting to display 
all keyspaces and tables
+            if (filterList.isEmpty())
+                return !ignoreMode;
+
+            List<String> tables = filter.get(keyspace);
+
+            // no such keyspace is in the map
+            if (tables == null)
+                return ignoreMode;
+                // only a keyspace with no tables was supplied
+                // so ignore or include (based on the flag) every column 
family in specified keyspace
+            else if (tables.isEmpty())
+                return !ignoreMode;
+
+            // keyspace exists, and it contains specific table
+            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)
+        {
+            for (String ks : verifier.keySet())
+                if (!keyspaces.contains(ks))
+                    throw new IllegalArgumentException("Unknown keyspace: " + 
ks);
+        }
+
+        public void verifyTables()
+        {
+            for (String ks : filter.keySet())
+                if (!verifier.get(ks).isEmpty())
+                    throw new IllegalArgumentException("Unknown tables: " + 
verifier.get(ks) + " in keyspace: " + ks);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/27d25c6d/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinter.java
----------------------------------------------------------------------
diff --git 
a/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinter.java 
b/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinter.java
index c901da7..4c86d27 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinter.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinter.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.cassandra.tools.nodetool.stats;
 
 import java.io.PrintStream;
@@ -23,41 +24,25 @@ import java.util.List;
 import org.json.simple.JSONObject;
 import org.yaml.snakeyaml.Yaml;
 
-public enum TableStatsPrinter
+public class TableStatsPrinter
 {
-    DEFAULT(new DefaultPrinter()),
-    JSON(new JsonPrinter()),
-    YAML(new YamlPrinter()),;
-
-    private final StatsPrinter<StatsHolder> printer;
-
-    TableStatsPrinter(StatsPrinter<StatsHolder> printer)
-    {
-        this.printer = printer;
-    }
-
-    public void print(StatsHolder stats, PrintStream out)
-    {
-        printer.printFormat(stats, out);
-    }
-
-    public static TableStatsPrinter from(String format)
+    public static StatsPrinter from(String format)
     {
         switch (format)
         {
             case "json":
-                return JSON;
+                return new StatsPrinter.JsonPrinter();
             case "yaml":
-                return YAML;
+                return new StatsPrinter.YamlPrinter();
             default:
-                return DEFAULT;
+                return new DefaultPrinter();
         }
     }
 
-    private static class DefaultPrinter implements StatsPrinter<StatsHolder>
+    private static class DefaultPrinter implements 
StatsPrinter<TableStatsHolder>
     {
         @Override
-        public void printFormat(StatsHolder data, PrintStream out)
+        public void print(TableStatsHolder data, PrintStream out)
         {
             out.println("Total number of tables: " + data.numberOfTables);
             out.println("----------------");
@@ -128,26 +113,4 @@ public enum TableStatsPrinter
             }
         }
     }
-
-    private static class JsonPrinter implements StatsPrinter<StatsHolder>
-    {
-        @Override
-        public void printFormat(StatsHolder data, PrintStream out)
-        {
-            JSONObject json = new JSONObject();
-            json.putAll(data.convert2Map());
-            out.println(json.toString());
-        }
-    }
-
-    private static class YamlPrinter implements StatsPrinter<StatsHolder>
-    {
-        @Override
-        public void printFormat(StatsHolder data, PrintStream out)
-        {
-            Yaml yaml = new Yaml();
-            out.println(yaml.dump(data.convert2Map()));
-        }
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/27d25c6d/src/java/org/apache/cassandra/tools/nodetool/stats/TpStatsHolder.java
----------------------------------------------------------------------
diff --git 
a/src/java/org/apache/cassandra/tools/nodetool/stats/TpStatsHolder.java 
b/src/java/org/apache/cassandra/tools/nodetool/stats/TpStatsHolder.java
new file mode 100644
index 0000000..df2f135
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/stats/TpStatsHolder.java
@@ -0,0 +1,63 @@
+/*
+ * 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.tools.nodetool.stats;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import com.google.common.collect.Multimap;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.nodetool.stats.StatsHolder;
+
+public class TpStatsHolder implements StatsHolder
+{
+    public final NodeProbe probe;
+
+    public TpStatsHolder(NodeProbe probe)
+    {
+        this.probe = probe;
+    }
+
+    @Override
+    public Map<String, Object> convert2Map()
+    {
+        HashMap<String, Object> result = new HashMap<>();
+        HashMap<String, Map<String, Object>> threadPools = new HashMap<>();
+        HashMap<String, Object> droppedMessage = new HashMap<>();
+
+        for (Map.Entry<String, String> tp : probe.getThreadPools().entries())
+        {
+            HashMap<String, Object> threadPool = new HashMap<>();
+            threadPool.put("ActiveTasks", 
probe.getThreadPoolMetric(tp.getKey(), tp.getValue(), "ActiveTasks"));
+            threadPool.put("PendingTasks", 
probe.getThreadPoolMetric(tp.getKey(), tp.getValue(), "PendingTasks"));
+            threadPool.put("CompletedTasks", 
probe.getThreadPoolMetric(tp.getKey(), tp.getValue(), "CompletedTasks"));
+            threadPool.put("CurrentlyBlockedTasks", 
probe.getThreadPoolMetric(tp.getKey(), tp.getValue(), "CurrentlyBlockedTasks"));
+            threadPool.put("TotalBlockedTasks", 
probe.getThreadPoolMetric(tp.getKey(), tp.getValue(), "TotalBlockedTasks"));
+            threadPools.put(tp.getValue(), threadPool);
+        }
+        result.put("ThreadPools", threadPools);
+
+        for (Map.Entry<String, Integer> entry : 
probe.getDroppedMessages().entrySet())
+            droppedMessage.put(entry.getKey(), entry.getValue());
+        result.put("DroppedMessage", droppedMessage);
+
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/27d25c6d/src/java/org/apache/cassandra/tools/nodetool/stats/TpStatsPrinter.java
----------------------------------------------------------------------
diff --git 
a/src/java/org/apache/cassandra/tools/nodetool/stats/TpStatsPrinter.java 
b/src/java/org/apache/cassandra/tools/nodetool/stats/TpStatsPrinter.java
new file mode 100644
index 0000000..83f2373
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/stats/TpStatsPrinter.java
@@ -0,0 +1,73 @@
+/*
+ * 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.tools.nodetool.stats;
+
+import java.io.PrintStream;
+import java.util.HashMap;
+import java.util.Map;
+
+public class TpStatsPrinter
+{
+    public static StatsPrinter from(String format)
+    {
+        switch (format)
+        {
+            case "json":
+                return new StatsPrinter.JsonPrinter();
+            case "yaml":
+                return new StatsPrinter.YamlPrinter();
+            default:
+                return new DefaultPrinter();
+        }
+
+    }
+
+    public static class DefaultPrinter implements StatsPrinter<TpStatsHolder>
+    {
+        @Override
+        public void print(TpStatsHolder data, PrintStream out)
+        {
+            Map<String, Object> convertData = data.convert2Map();
+
+            out.printf("%-30s%10s%10s%15s%10s%18s%n", "Pool Name", "Active", 
"Pending", "Completed", "Blocked", "All time blocked");
+
+            Map<Object, Object> threadPools = convertData.get("ThreadPools") 
instanceof HashMap<?, ?> ? (HashMap)convertData.get("ThreadPools") : new 
HashMap();
+            for (Map.Entry<Object, Object> entry : threadPools.entrySet())
+            {
+                HashMap values = entry.getValue() instanceof HashMap<?, ?> ? 
(HashMap)entry.getValue() : new HashMap();
+                out.printf("%-30s%10s%10s%15s%10s%18s%n",
+                           entry.getKey(),
+                           values.get("ActiveTasks"),
+                           values.get("PendingTasks"),
+                           values.get("CompletedTasks"),
+                           values.get("CurrentlyBlockedTasks"),
+                           values.get("TotalBlockedTasks"),
+                           values.get("ActiveTasks"));
+            }
+
+            out.printf("%n%-20s%10s%n", "Message type", "Dropped");
+
+            Map<Object, Object> droppedMessages = 
convertData.get("DroppedMessage") instanceof HashMap<?, ?> ? 
(HashMap)convertData.get("DroppedMessage") : new HashMap();
+            for (Map.Entry<Object, Object> entry : droppedMessages.entrySet())
+            {
+                out.printf("%-20s%10s%n", entry.getKey(), entry.getValue());
+            }
+        }
+    }
+}

Reply via email to