You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by yu...@apache.org on 2015/08/06 00:07:12 UTC

[01/10] cassandra git commit: Fix nodetool info error when the node is not joined

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 1a2c1bcdc -> 20f12e974
  refs/heads/cassandra-2.2 5c59d5af7 -> 32bc8b0b1
  refs/heads/cassandra-3.0 90e001312 -> c3ed25b0a
  refs/heads/trunk 760dbd957 -> c1aff4fa6


Fix nodetool info error when the node is not joined

patch by yukim; reviewed by stefania for CASSANDRA-9031


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

Branch: refs/heads/cassandra-2.1
Commit: 20f12e97446eee55461a8d3512a94389a67e79ee
Parents: 1a2c1bc
Author: Yuki Morishita <yu...@apache.org>
Authored: Wed Aug 5 15:58:36 2015 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Wed Aug 5 16:01:53 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/tools/NodeProbe.java   | 16 ++-------------
 .../org/apache/cassandra/tools/NodeTool.java    | 21 ++++++++++++++------
 3 files changed, 18 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/20f12e97/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index de7cfa8..9a475ea 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -12,6 +12,7 @@
  * Remove repair snapshot leftover on startup (CASSANDRA-7357)
  * Use random nodes for batch log when only 2 racks (CASSANDRA-8735)
  * Ensure atomicity inside thrift and stream session (CASSANDRA-7757)
+ * Fix nodetool info error when the node is not joined (CASSANDRA-9031)
 Merged from 2.0:
  * Don't cast expected bf size to an int (CASSANDRA-9959)
  * Log when messages are dropped due to cross_node_timeout (CASSANDRA-9793)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/20f12e97/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index d3bce4d..caa12c3 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -807,20 +807,8 @@ public class NodeProbe implements AutoCloseable
 
     public String getEndpoint()
     {
-        // Try to find the endpoint using the local token, doing so in a crazy manner
-        // to maintain backwards compatibility with the MBean interface
-        String stringToken = ssProxy.getTokens().get(0);
-        Map<String, String> tokenToEndpoint = ssProxy.getTokenToEndpointMap();
-
-        for (Map.Entry<String, String> pair : tokenToEndpoint.entrySet())
-        {
-            if (pair.getKey().equals(stringToken))
-            {
-                return pair.getValue();
-            }
-        }
-
-        throw new RuntimeException("Could not find myself in the endpoint list, something is very wrong!  Is the Cassandra node fully started?");
+        Map<String, String> hostIdToEndpoint = ssProxy.getHostIdMap();
+        return hostIdToEndpoint.get(ssProxy.getLocalHostId());
     }
 
     public String getDataCenter()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/20f12e97/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index a2d4ead..6a7a930 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -463,13 +463,22 @@ public class NodeTool
                     probe.getCacheMetric("CounterCache", "HitRate"),
                     cacheService.getCounterCacheSavePeriodInSeconds());
 
-            // Tokens
-            List<String> tokens = probe.getTokens();
-            if (tokens.size() == 1 || this.tokens)
-                for (String token : tokens)
-                    System.out.printf("%-23s: %s%n", "Token", token);
+            // check if node is already joined, before getting tokens, since it throws exception if not.
+            if (probe.isJoined())
+            {
+                // Tokens
+                List<String> tokens = probe.getTokens();
+                if (tokens.size() == 1 || this.tokens)
+                    for (String token : tokens)
+                        System.out.printf("%-23s: %s%n", "Token", token);
+                else
+                    System.out.printf("%-23s: (invoke with -T/--tokens to see all %d tokens)%n", "Token",
+                                      tokens.size());
+            }
             else
-                System.out.printf("%-23s: (invoke with -T/--tokens to see all %d tokens)%n", "Token", tokens.size());
+            {
+                System.out.printf("%-23s: (node is not joined to the cluster)%n", "Token");
+            }
         }
 
         /**


[07/10] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by yu...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2


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

Branch: refs/heads/trunk
Commit: 32bc8b0b182176f0132522f821a1b13919efc63a
Parents: 5c59d5a 20f12e9
Author: Yuki Morishita <yu...@apache.org>
Authored: Wed Aug 5 16:10:22 2015 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Wed Aug 5 16:10:22 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/tools/NodeProbe.java   | 16 ++------------
 .../apache/cassandra/tools/nodetool/Info.java   | 23 ++++++++++++++------
 3 files changed, 19 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/32bc8b0b/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 66e5a0c,9a475ea..72ad3cd
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,27 -1,8 +1,28 @@@
 -2.1.9
 +2.2.1
 + * Log warning when using an aggregate without partition key (CASSANDRA-9737)
 + * Avoid grouping sstables for anticompaction with DTCS (CASSANDRA-9900)
 + * UDF / UDA execution time in trace (CASSANDRA-9723)
 + * Fix broken internode SSL (CASSANDRA-9884)
 +Merged from 2.1:
   * Cannot replace token does not exist - DN node removed as Fat Client (CASSANDRA-9871)
   * Fix handling of enable/disable autocompaction (CASSANDRA-9899)
 - * Commit log segment recycling is disabled by default (CASSANDRA-9896)
   * Add consistency level to tracing ouput (CASSANDRA-9827)
 + * Remove repair snapshot leftover on startup (CASSANDRA-7357)
 + * Use random nodes for batch log when only 2 racks (CASSANDRA-8735)
 + * Ensure atomicity inside thrift and stream session (CASSANDRA-7757)
++ * Fix nodetool info error when the node is not joined (CASSANDRA-9031)
 +Merged from 2.0:
 + * Log when messages are dropped due to cross_node_timeout (CASSANDRA-9793)
 + * Don't track hotness when opening from snapshot for validation (CASSANDRA-9382)
 +
 +
 +2.2.0
 + * Allow the selection of columns together with aggregates (CASSANDRA-9767)
 + * Fix cqlsh copy methods and other windows specific issues (CASSANDRA-9795)
 + * Don't wrap byte arrays in SequentialWriter (CASSANDRA-9797)
 + * sum() and avg() functions missing for smallint and tinyint types (CASSANDRA-9671)
 + * Revert CASSANDRA-9542 (allow native functions in UDA) (CASSANDRA-9771)
 +Merged from 2.1:
   * Fix MarshalException when upgrading superColumn family (CASSANDRA-9582)
   * Fix broken logging for "empty" flushes in Memtable (CASSANDRA-9837)
   * Handle corrupt files on startup (CASSANDRA-9686)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/32bc8b0b/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/32bc8b0b/src/java/org/apache/cassandra/tools/nodetool/Info.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Info.java
index 5852fc7,0000000..0d9bd73
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Info.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Info.java
@@@ -1,153 -1,0 +1,162 @@@
 +/*
 + * 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;
 +
 +import io.airlift.command.Command;
 +import io.airlift.command.Option;
 +
 +import java.lang.management.MemoryUsage;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +
 +import javax.management.InstanceNotFoundException;
 +
 +import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 +import org.apache.cassandra.io.util.FileUtils;
 +import org.apache.cassandra.service.CacheServiceMBean;
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
 +
 +@Command(name = "info", description = "Print node information (uptime, load, ...)")
 +public class Info extends NodeToolCmd
 +{
 +    @Option(name = {"-T", "--tokens"}, description = "Display all tokens")
 +    private boolean tokens = false;
 +
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        boolean gossipInitialized = probe.isInitialized();
 +
 +        System.out.printf("%-23s: %s%n", "ID", probe.getLocalHostId());
 +        System.out.printf("%-23s: %s%n", "Gossip active", gossipInitialized);
 +        System.out.printf("%-23s: %s%n", "Thrift active", probe.isThriftServerRunning());
 +        System.out.printf("%-23s: %s%n", "Native Transport active", probe.isNativeTransportRunning());
 +        System.out.printf("%-23s: %s%n", "Load", probe.getLoadString());
 +        if (gossipInitialized)
 +            System.out.printf("%-23s: %s%n", "Generation No", probe.getCurrentGenerationNumber());
 +        else
 +            System.out.printf("%-23s: %s%n", "Generation No", 0);
 +
 +        // Uptime
 +        long secondsUp = probe.getUptime() / 1000;
 +        System.out.printf("%-23s: %d%n", "Uptime (seconds)", secondsUp);
 +
 +        // Memory usage
 +        MemoryUsage heapUsage = probe.getHeapMemoryUsage();
 +        double memUsed = (double) heapUsage.getUsed() / (1024 * 1024);
 +        double memMax = (double) heapUsage.getMax() / (1024 * 1024);
 +        System.out.printf("%-23s: %.2f / %.2f%n", "Heap Memory (MB)", memUsed, memMax);
 +        try
 +        {
 +            System.out.printf("%-23s: %.2f%n", "Off Heap Memory (MB)", getOffHeapMemoryUsed(probe));
 +        }
 +        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;
 +        }
 +
 +        // Data Center/Rack
 +        System.out.printf("%-23s: %s%n", "Data Center", probe.getDataCenter());
 +        System.out.printf("%-23s: %s%n", "Rack", probe.getRack());
 +
 +        // Exceptions
 +        System.out.printf("%-23s: %s%n", "Exceptions", probe.getStorageMetric("Exceptions"));
 +
 +        CacheServiceMBean cacheService = probe.getCacheServiceMBean();
 +
 +        // Key Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
 +        System.out.printf("%-23s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
 +                "Key Cache",
 +                probe.getCacheMetric("KeyCache", "Entries"),
 +                FileUtils.stringifyFileSize((long) probe.getCacheMetric("KeyCache", "Size")),
 +                FileUtils.stringifyFileSize((long) probe.getCacheMetric("KeyCache", "Capacity")),
 +                probe.getCacheMetric("KeyCache", "Hits"),
 +                probe.getCacheMetric("KeyCache", "Requests"),
 +                probe.getCacheMetric("KeyCache", "HitRate"),
 +                cacheService.getKeyCacheSavePeriodInSeconds());
 +
 +        // Row Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
 +        System.out.printf("%-23s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
 +                "Row Cache",
 +                probe.getCacheMetric("RowCache", "Entries"),
 +                FileUtils.stringifyFileSize((long) probe.getCacheMetric("RowCache", "Size")),
 +                FileUtils.stringifyFileSize((long) probe.getCacheMetric("RowCache", "Capacity")),
 +                probe.getCacheMetric("RowCache", "Hits"),
 +                probe.getCacheMetric("RowCache", "Requests"),
 +                probe.getCacheMetric("RowCache", "HitRate"),
 +                cacheService.getRowCacheSavePeriodInSeconds());
 +
 +        // Counter Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
 +        System.out.printf("%-23s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
 +                "Counter Cache",
 +                probe.getCacheMetric("CounterCache", "Entries"),
 +                FileUtils.stringifyFileSize((long) probe.getCacheMetric("CounterCache", "Size")),
 +                FileUtils.stringifyFileSize((long) probe.getCacheMetric("CounterCache", "Capacity")),
 +                probe.getCacheMetric("CounterCache", "Hits"),
 +                probe.getCacheMetric("CounterCache", "Requests"),
 +                probe.getCacheMetric("CounterCache", "HitRate"),
 +                cacheService.getCounterCacheSavePeriodInSeconds());
 +
-         // Tokens
-         List<String> tokens = probe.getTokens();
-         if (tokens.size() == 1 || this.tokens)
-             for (String token : tokens)
-                 System.out.printf("%-23s: %s%n", "Token", token);
++        // check if node is already joined, before getting tokens, since it throws exception if not.
++        if (probe.isJoined())
++        {
++            // Tokens
++            List<String> tokens = probe.getTokens();
++            if (tokens.size() == 1 || this.tokens)
++                for (String token : tokens)
++                    System.out.printf("%-23s: %s%n", "Token", token);
++            else
++                System.out.printf("%-23s: (invoke with -T/--tokens to see all %d tokens)%n", "Token",
++                                  tokens.size());
++        }
 +        else
-             System.out.printf("%-23s: (invoke with -T/--tokens to see all %d tokens)%n", "Token", tokens.size());
++        {
++            System.out.printf("%-23s: (node is not joined to the cluster)%n", "Token");
++        }
 +    }
 +
 +    /**
 +     * Returns the total off heap memory used in MB.
 +     * @return the total off heap memory used in MB.
 +     */
 +    private static double getOffHeapMemoryUsed(NodeProbe probe)
 +    {
 +        long offHeapMemUsedInBytes = 0;
 +        // get a list of column family stores
 +        Iterator<Map.Entry<String, ColumnFamilyStoreMBean>> cfamilies = probe.getColumnFamilyStoreMBeanProxies();
 +
 +        while (cfamilies.hasNext())
 +        {
 +            Entry<String, ColumnFamilyStoreMBean> entry = cfamilies.next();
 +            String keyspaceName = entry.getKey();
 +            String cfName = entry.getValue().getColumnFamilyName();
 +
 +            offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "MemtableOffHeapSize");
 +            offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "BloomFilterOffHeapMemoryUsed");
 +            offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "IndexSummaryOffHeapMemoryUsed");
 +            offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "CompressionMetadataOffHeapMemoryUsed");
 +        }
 +
 +        return offHeapMemUsedInBytes / (1024d * 1024);
 +    }
- }
++}


[03/10] cassandra git commit: Fix nodetool info error when the node is not joined

Posted by yu...@apache.org.
Fix nodetool info error when the node is not joined

patch by yukim; reviewed by stefania for CASSANDRA-9031


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

Branch: refs/heads/cassandra-3.0
Commit: 20f12e97446eee55461a8d3512a94389a67e79ee
Parents: 1a2c1bc
Author: Yuki Morishita <yu...@apache.org>
Authored: Wed Aug 5 15:58:36 2015 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Wed Aug 5 16:01:53 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/tools/NodeProbe.java   | 16 ++-------------
 .../org/apache/cassandra/tools/NodeTool.java    | 21 ++++++++++++++------
 3 files changed, 18 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/20f12e97/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index de7cfa8..9a475ea 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -12,6 +12,7 @@
  * Remove repair snapshot leftover on startup (CASSANDRA-7357)
  * Use random nodes for batch log when only 2 racks (CASSANDRA-8735)
  * Ensure atomicity inside thrift and stream session (CASSANDRA-7757)
+ * Fix nodetool info error when the node is not joined (CASSANDRA-9031)
 Merged from 2.0:
  * Don't cast expected bf size to an int (CASSANDRA-9959)
  * Log when messages are dropped due to cross_node_timeout (CASSANDRA-9793)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/20f12e97/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index d3bce4d..caa12c3 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -807,20 +807,8 @@ public class NodeProbe implements AutoCloseable
 
     public String getEndpoint()
     {
-        // Try to find the endpoint using the local token, doing so in a crazy manner
-        // to maintain backwards compatibility with the MBean interface
-        String stringToken = ssProxy.getTokens().get(0);
-        Map<String, String> tokenToEndpoint = ssProxy.getTokenToEndpointMap();
-
-        for (Map.Entry<String, String> pair : tokenToEndpoint.entrySet())
-        {
-            if (pair.getKey().equals(stringToken))
-            {
-                return pair.getValue();
-            }
-        }
-
-        throw new RuntimeException("Could not find myself in the endpoint list, something is very wrong!  Is the Cassandra node fully started?");
+        Map<String, String> hostIdToEndpoint = ssProxy.getHostIdMap();
+        return hostIdToEndpoint.get(ssProxy.getLocalHostId());
     }
 
     public String getDataCenter()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/20f12e97/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index a2d4ead..6a7a930 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -463,13 +463,22 @@ public class NodeTool
                     probe.getCacheMetric("CounterCache", "HitRate"),
                     cacheService.getCounterCacheSavePeriodInSeconds());
 
-            // Tokens
-            List<String> tokens = probe.getTokens();
-            if (tokens.size() == 1 || this.tokens)
-                for (String token : tokens)
-                    System.out.printf("%-23s: %s%n", "Token", token);
+            // check if node is already joined, before getting tokens, since it throws exception if not.
+            if (probe.isJoined())
+            {
+                // Tokens
+                List<String> tokens = probe.getTokens();
+                if (tokens.size() == 1 || this.tokens)
+                    for (String token : tokens)
+                        System.out.printf("%-23s: %s%n", "Token", token);
+                else
+                    System.out.printf("%-23s: (invoke with -T/--tokens to see all %d tokens)%n", "Token",
+                                      tokens.size());
+            }
             else
-                System.out.printf("%-23s: (invoke with -T/--tokens to see all %d tokens)%n", "Token", tokens.size());
+            {
+                System.out.printf("%-23s: (node is not joined to the cluster)%n", "Token");
+            }
         }
 
         /**


[08/10] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by yu...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


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

Branch: refs/heads/cassandra-3.0
Commit: c3ed25b0ad43aad0deaade1b915ff8310c9ca3fc
Parents: 90e0013 32bc8b0
Author: Yuki Morishita <yu...@apache.org>
Authored: Wed Aug 5 16:10:33 2015 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Wed Aug 5 16:10:33 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/tools/NodeProbe.java   | 16 ++------------
 .../apache/cassandra/tools/nodetool/Info.java   | 23 ++++++++++++++------
 3 files changed, 19 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3ed25b0/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3ed25b0/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------


[02/10] cassandra git commit: Fix nodetool info error when the node is not joined

Posted by yu...@apache.org.
Fix nodetool info error when the node is not joined

patch by yukim; reviewed by stefania for CASSANDRA-9031


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

Branch: refs/heads/cassandra-2.2
Commit: 20f12e97446eee55461a8d3512a94389a67e79ee
Parents: 1a2c1bc
Author: Yuki Morishita <yu...@apache.org>
Authored: Wed Aug 5 15:58:36 2015 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Wed Aug 5 16:01:53 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/tools/NodeProbe.java   | 16 ++-------------
 .../org/apache/cassandra/tools/NodeTool.java    | 21 ++++++++++++++------
 3 files changed, 18 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/20f12e97/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index de7cfa8..9a475ea 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -12,6 +12,7 @@
  * Remove repair snapshot leftover on startup (CASSANDRA-7357)
  * Use random nodes for batch log when only 2 racks (CASSANDRA-8735)
  * Ensure atomicity inside thrift and stream session (CASSANDRA-7757)
+ * Fix nodetool info error when the node is not joined (CASSANDRA-9031)
 Merged from 2.0:
  * Don't cast expected bf size to an int (CASSANDRA-9959)
  * Log when messages are dropped due to cross_node_timeout (CASSANDRA-9793)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/20f12e97/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index d3bce4d..caa12c3 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -807,20 +807,8 @@ public class NodeProbe implements AutoCloseable
 
     public String getEndpoint()
     {
-        // Try to find the endpoint using the local token, doing so in a crazy manner
-        // to maintain backwards compatibility with the MBean interface
-        String stringToken = ssProxy.getTokens().get(0);
-        Map<String, String> tokenToEndpoint = ssProxy.getTokenToEndpointMap();
-
-        for (Map.Entry<String, String> pair : tokenToEndpoint.entrySet())
-        {
-            if (pair.getKey().equals(stringToken))
-            {
-                return pair.getValue();
-            }
-        }
-
-        throw new RuntimeException("Could not find myself in the endpoint list, something is very wrong!  Is the Cassandra node fully started?");
+        Map<String, String> hostIdToEndpoint = ssProxy.getHostIdMap();
+        return hostIdToEndpoint.get(ssProxy.getLocalHostId());
     }
 
     public String getDataCenter()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/20f12e97/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index a2d4ead..6a7a930 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -463,13 +463,22 @@ public class NodeTool
                     probe.getCacheMetric("CounterCache", "HitRate"),
                     cacheService.getCounterCacheSavePeriodInSeconds());
 
-            // Tokens
-            List<String> tokens = probe.getTokens();
-            if (tokens.size() == 1 || this.tokens)
-                for (String token : tokens)
-                    System.out.printf("%-23s: %s%n", "Token", token);
+            // check if node is already joined, before getting tokens, since it throws exception if not.
+            if (probe.isJoined())
+            {
+                // Tokens
+                List<String> tokens = probe.getTokens();
+                if (tokens.size() == 1 || this.tokens)
+                    for (String token : tokens)
+                        System.out.printf("%-23s: %s%n", "Token", token);
+                else
+                    System.out.printf("%-23s: (invoke with -T/--tokens to see all %d tokens)%n", "Token",
+                                      tokens.size());
+            }
             else
-                System.out.printf("%-23s: (invoke with -T/--tokens to see all %d tokens)%n", "Token", tokens.size());
+            {
+                System.out.printf("%-23s: (node is not joined to the cluster)%n", "Token");
+            }
         }
 
         /**


[09/10] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by yu...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


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

Branch: refs/heads/trunk
Commit: c3ed25b0ad43aad0deaade1b915ff8310c9ca3fc
Parents: 90e0013 32bc8b0
Author: Yuki Morishita <yu...@apache.org>
Authored: Wed Aug 5 16:10:33 2015 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Wed Aug 5 16:10:33 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/tools/NodeProbe.java   | 16 ++------------
 .../apache/cassandra/tools/nodetool/Info.java   | 23 ++++++++++++++------
 3 files changed, 19 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3ed25b0/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c3ed25b0/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------


[04/10] cassandra git commit: Fix nodetool info error when the node is not joined

Posted by yu...@apache.org.
Fix nodetool info error when the node is not joined

patch by yukim; reviewed by stefania for CASSANDRA-9031


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

Branch: refs/heads/trunk
Commit: 20f12e97446eee55461a8d3512a94389a67e79ee
Parents: 1a2c1bc
Author: Yuki Morishita <yu...@apache.org>
Authored: Wed Aug 5 15:58:36 2015 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Wed Aug 5 16:01:53 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/tools/NodeProbe.java   | 16 ++-------------
 .../org/apache/cassandra/tools/NodeTool.java    | 21 ++++++++++++++------
 3 files changed, 18 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/20f12e97/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index de7cfa8..9a475ea 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -12,6 +12,7 @@
  * Remove repair snapshot leftover on startup (CASSANDRA-7357)
  * Use random nodes for batch log when only 2 racks (CASSANDRA-8735)
  * Ensure atomicity inside thrift and stream session (CASSANDRA-7757)
+ * Fix nodetool info error when the node is not joined (CASSANDRA-9031)
 Merged from 2.0:
  * Don't cast expected bf size to an int (CASSANDRA-9959)
  * Log when messages are dropped due to cross_node_timeout (CASSANDRA-9793)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/20f12e97/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index d3bce4d..caa12c3 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -807,20 +807,8 @@ public class NodeProbe implements AutoCloseable
 
     public String getEndpoint()
     {
-        // Try to find the endpoint using the local token, doing so in a crazy manner
-        // to maintain backwards compatibility with the MBean interface
-        String stringToken = ssProxy.getTokens().get(0);
-        Map<String, String> tokenToEndpoint = ssProxy.getTokenToEndpointMap();
-
-        for (Map.Entry<String, String> pair : tokenToEndpoint.entrySet())
-        {
-            if (pair.getKey().equals(stringToken))
-            {
-                return pair.getValue();
-            }
-        }
-
-        throw new RuntimeException("Could not find myself in the endpoint list, something is very wrong!  Is the Cassandra node fully started?");
+        Map<String, String> hostIdToEndpoint = ssProxy.getHostIdMap();
+        return hostIdToEndpoint.get(ssProxy.getLocalHostId());
     }
 
     public String getDataCenter()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/20f12e97/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index a2d4ead..6a7a930 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -463,13 +463,22 @@ public class NodeTool
                     probe.getCacheMetric("CounterCache", "HitRate"),
                     cacheService.getCounterCacheSavePeriodInSeconds());
 
-            // Tokens
-            List<String> tokens = probe.getTokens();
-            if (tokens.size() == 1 || this.tokens)
-                for (String token : tokens)
-                    System.out.printf("%-23s: %s%n", "Token", token);
+            // check if node is already joined, before getting tokens, since it throws exception if not.
+            if (probe.isJoined())
+            {
+                // Tokens
+                List<String> tokens = probe.getTokens();
+                if (tokens.size() == 1 || this.tokens)
+                    for (String token : tokens)
+                        System.out.printf("%-23s: %s%n", "Token", token);
+                else
+                    System.out.printf("%-23s: (invoke with -T/--tokens to see all %d tokens)%n", "Token",
+                                      tokens.size());
+            }
             else
-                System.out.printf("%-23s: (invoke with -T/--tokens to see all %d tokens)%n", "Token", tokens.size());
+            {
+                System.out.printf("%-23s: (node is not joined to the cluster)%n", "Token");
+            }
         }
 
         /**


[06/10] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by yu...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2


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

Branch: refs/heads/cassandra-2.2
Commit: 32bc8b0b182176f0132522f821a1b13919efc63a
Parents: 5c59d5a 20f12e9
Author: Yuki Morishita <yu...@apache.org>
Authored: Wed Aug 5 16:10:22 2015 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Wed Aug 5 16:10:22 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/tools/NodeProbe.java   | 16 ++------------
 .../apache/cassandra/tools/nodetool/Info.java   | 23 ++++++++++++++------
 3 files changed, 19 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/32bc8b0b/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 66e5a0c,9a475ea..72ad3cd
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,27 -1,8 +1,28 @@@
 -2.1.9
 +2.2.1
 + * Log warning when using an aggregate without partition key (CASSANDRA-9737)
 + * Avoid grouping sstables for anticompaction with DTCS (CASSANDRA-9900)
 + * UDF / UDA execution time in trace (CASSANDRA-9723)
 + * Fix broken internode SSL (CASSANDRA-9884)
 +Merged from 2.1:
   * Cannot replace token does not exist - DN node removed as Fat Client (CASSANDRA-9871)
   * Fix handling of enable/disable autocompaction (CASSANDRA-9899)
 - * Commit log segment recycling is disabled by default (CASSANDRA-9896)
   * Add consistency level to tracing ouput (CASSANDRA-9827)
 + * Remove repair snapshot leftover on startup (CASSANDRA-7357)
 + * Use random nodes for batch log when only 2 racks (CASSANDRA-8735)
 + * Ensure atomicity inside thrift and stream session (CASSANDRA-7757)
++ * Fix nodetool info error when the node is not joined (CASSANDRA-9031)
 +Merged from 2.0:
 + * Log when messages are dropped due to cross_node_timeout (CASSANDRA-9793)
 + * Don't track hotness when opening from snapshot for validation (CASSANDRA-9382)
 +
 +
 +2.2.0
 + * Allow the selection of columns together with aggregates (CASSANDRA-9767)
 + * Fix cqlsh copy methods and other windows specific issues (CASSANDRA-9795)
 + * Don't wrap byte arrays in SequentialWriter (CASSANDRA-9797)
 + * sum() and avg() functions missing for smallint and tinyint types (CASSANDRA-9671)
 + * Revert CASSANDRA-9542 (allow native functions in UDA) (CASSANDRA-9771)
 +Merged from 2.1:
   * Fix MarshalException when upgrading superColumn family (CASSANDRA-9582)
   * Fix broken logging for "empty" flushes in Memtable (CASSANDRA-9837)
   * Handle corrupt files on startup (CASSANDRA-9686)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/32bc8b0b/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/32bc8b0b/src/java/org/apache/cassandra/tools/nodetool/Info.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Info.java
index 5852fc7,0000000..0d9bd73
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Info.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Info.java
@@@ -1,153 -1,0 +1,162 @@@
 +/*
 + * 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;
 +
 +import io.airlift.command.Command;
 +import io.airlift.command.Option;
 +
 +import java.lang.management.MemoryUsage;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +
 +import javax.management.InstanceNotFoundException;
 +
 +import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 +import org.apache.cassandra.io.util.FileUtils;
 +import org.apache.cassandra.service.CacheServiceMBean;
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
 +
 +@Command(name = "info", description = "Print node information (uptime, load, ...)")
 +public class Info extends NodeToolCmd
 +{
 +    @Option(name = {"-T", "--tokens"}, description = "Display all tokens")
 +    private boolean tokens = false;
 +
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        boolean gossipInitialized = probe.isInitialized();
 +
 +        System.out.printf("%-23s: %s%n", "ID", probe.getLocalHostId());
 +        System.out.printf("%-23s: %s%n", "Gossip active", gossipInitialized);
 +        System.out.printf("%-23s: %s%n", "Thrift active", probe.isThriftServerRunning());
 +        System.out.printf("%-23s: %s%n", "Native Transport active", probe.isNativeTransportRunning());
 +        System.out.printf("%-23s: %s%n", "Load", probe.getLoadString());
 +        if (gossipInitialized)
 +            System.out.printf("%-23s: %s%n", "Generation No", probe.getCurrentGenerationNumber());
 +        else
 +            System.out.printf("%-23s: %s%n", "Generation No", 0);
 +
 +        // Uptime
 +        long secondsUp = probe.getUptime() / 1000;
 +        System.out.printf("%-23s: %d%n", "Uptime (seconds)", secondsUp);
 +
 +        // Memory usage
 +        MemoryUsage heapUsage = probe.getHeapMemoryUsage();
 +        double memUsed = (double) heapUsage.getUsed() / (1024 * 1024);
 +        double memMax = (double) heapUsage.getMax() / (1024 * 1024);
 +        System.out.printf("%-23s: %.2f / %.2f%n", "Heap Memory (MB)", memUsed, memMax);
 +        try
 +        {
 +            System.out.printf("%-23s: %.2f%n", "Off Heap Memory (MB)", getOffHeapMemoryUsed(probe));
 +        }
 +        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;
 +        }
 +
 +        // Data Center/Rack
 +        System.out.printf("%-23s: %s%n", "Data Center", probe.getDataCenter());
 +        System.out.printf("%-23s: %s%n", "Rack", probe.getRack());
 +
 +        // Exceptions
 +        System.out.printf("%-23s: %s%n", "Exceptions", probe.getStorageMetric("Exceptions"));
 +
 +        CacheServiceMBean cacheService = probe.getCacheServiceMBean();
 +
 +        // Key Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
 +        System.out.printf("%-23s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
 +                "Key Cache",
 +                probe.getCacheMetric("KeyCache", "Entries"),
 +                FileUtils.stringifyFileSize((long) probe.getCacheMetric("KeyCache", "Size")),
 +                FileUtils.stringifyFileSize((long) probe.getCacheMetric("KeyCache", "Capacity")),
 +                probe.getCacheMetric("KeyCache", "Hits"),
 +                probe.getCacheMetric("KeyCache", "Requests"),
 +                probe.getCacheMetric("KeyCache", "HitRate"),
 +                cacheService.getKeyCacheSavePeriodInSeconds());
 +
 +        // Row Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
 +        System.out.printf("%-23s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
 +                "Row Cache",
 +                probe.getCacheMetric("RowCache", "Entries"),
 +                FileUtils.stringifyFileSize((long) probe.getCacheMetric("RowCache", "Size")),
 +                FileUtils.stringifyFileSize((long) probe.getCacheMetric("RowCache", "Capacity")),
 +                probe.getCacheMetric("RowCache", "Hits"),
 +                probe.getCacheMetric("RowCache", "Requests"),
 +                probe.getCacheMetric("RowCache", "HitRate"),
 +                cacheService.getRowCacheSavePeriodInSeconds());
 +
 +        // Counter Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
 +        System.out.printf("%-23s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
 +                "Counter Cache",
 +                probe.getCacheMetric("CounterCache", "Entries"),
 +                FileUtils.stringifyFileSize((long) probe.getCacheMetric("CounterCache", "Size")),
 +                FileUtils.stringifyFileSize((long) probe.getCacheMetric("CounterCache", "Capacity")),
 +                probe.getCacheMetric("CounterCache", "Hits"),
 +                probe.getCacheMetric("CounterCache", "Requests"),
 +                probe.getCacheMetric("CounterCache", "HitRate"),
 +                cacheService.getCounterCacheSavePeriodInSeconds());
 +
-         // Tokens
-         List<String> tokens = probe.getTokens();
-         if (tokens.size() == 1 || this.tokens)
-             for (String token : tokens)
-                 System.out.printf("%-23s: %s%n", "Token", token);
++        // check if node is already joined, before getting tokens, since it throws exception if not.
++        if (probe.isJoined())
++        {
++            // Tokens
++            List<String> tokens = probe.getTokens();
++            if (tokens.size() == 1 || this.tokens)
++                for (String token : tokens)
++                    System.out.printf("%-23s: %s%n", "Token", token);
++            else
++                System.out.printf("%-23s: (invoke with -T/--tokens to see all %d tokens)%n", "Token",
++                                  tokens.size());
++        }
 +        else
-             System.out.printf("%-23s: (invoke with -T/--tokens to see all %d tokens)%n", "Token", tokens.size());
++        {
++            System.out.printf("%-23s: (node is not joined to the cluster)%n", "Token");
++        }
 +    }
 +
 +    /**
 +     * Returns the total off heap memory used in MB.
 +     * @return the total off heap memory used in MB.
 +     */
 +    private static double getOffHeapMemoryUsed(NodeProbe probe)
 +    {
 +        long offHeapMemUsedInBytes = 0;
 +        // get a list of column family stores
 +        Iterator<Map.Entry<String, ColumnFamilyStoreMBean>> cfamilies = probe.getColumnFamilyStoreMBeanProxies();
 +
 +        while (cfamilies.hasNext())
 +        {
 +            Entry<String, ColumnFamilyStoreMBean> entry = cfamilies.next();
 +            String keyspaceName = entry.getKey();
 +            String cfName = entry.getValue().getColumnFamilyName();
 +
 +            offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "MemtableOffHeapSize");
 +            offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "BloomFilterOffHeapMemoryUsed");
 +            offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "IndexSummaryOffHeapMemoryUsed");
 +            offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "CompressionMetadataOffHeapMemoryUsed");
 +        }
 +
 +        return offHeapMemUsedInBytes / (1024d * 1024);
 +    }
- }
++}


[10/10] cassandra git commit: Merge branch 'cassandra-3.0' into trunk

Posted by yu...@apache.org.
Merge branch 'cassandra-3.0' into trunk


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

Branch: refs/heads/trunk
Commit: c1aff4fa61e09396de56cfa365c56dbe256393ee
Parents: 760dbd9 c3ed25b
Author: Yuki Morishita <yu...@apache.org>
Authored: Wed Aug 5 16:10:42 2015 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Wed Aug 5 16:10:42 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/tools/NodeProbe.java   | 16 ++------------
 .../apache/cassandra/tools/nodetool/Info.java   | 23 ++++++++++++++------
 3 files changed, 19 insertions(+), 21 deletions(-)
----------------------------------------------------------------------



[05/10] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by yu...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2


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

Branch: refs/heads/cassandra-3.0
Commit: 32bc8b0b182176f0132522f821a1b13919efc63a
Parents: 5c59d5a 20f12e9
Author: Yuki Morishita <yu...@apache.org>
Authored: Wed Aug 5 16:10:22 2015 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Wed Aug 5 16:10:22 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/tools/NodeProbe.java   | 16 ++------------
 .../apache/cassandra/tools/nodetool/Info.java   | 23 ++++++++++++++------
 3 files changed, 19 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/32bc8b0b/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 66e5a0c,9a475ea..72ad3cd
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,27 -1,8 +1,28 @@@
 -2.1.9
 +2.2.1
 + * Log warning when using an aggregate without partition key (CASSANDRA-9737)
 + * Avoid grouping sstables for anticompaction with DTCS (CASSANDRA-9900)
 + * UDF / UDA execution time in trace (CASSANDRA-9723)
 + * Fix broken internode SSL (CASSANDRA-9884)
 +Merged from 2.1:
   * Cannot replace token does not exist - DN node removed as Fat Client (CASSANDRA-9871)
   * Fix handling of enable/disable autocompaction (CASSANDRA-9899)
 - * Commit log segment recycling is disabled by default (CASSANDRA-9896)
   * Add consistency level to tracing ouput (CASSANDRA-9827)
 + * Remove repair snapshot leftover on startup (CASSANDRA-7357)
 + * Use random nodes for batch log when only 2 racks (CASSANDRA-8735)
 + * Ensure atomicity inside thrift and stream session (CASSANDRA-7757)
++ * Fix nodetool info error when the node is not joined (CASSANDRA-9031)
 +Merged from 2.0:
 + * Log when messages are dropped due to cross_node_timeout (CASSANDRA-9793)
 + * Don't track hotness when opening from snapshot for validation (CASSANDRA-9382)
 +
 +
 +2.2.0
 + * Allow the selection of columns together with aggregates (CASSANDRA-9767)
 + * Fix cqlsh copy methods and other windows specific issues (CASSANDRA-9795)
 + * Don't wrap byte arrays in SequentialWriter (CASSANDRA-9797)
 + * sum() and avg() functions missing for smallint and tinyint types (CASSANDRA-9671)
 + * Revert CASSANDRA-9542 (allow native functions in UDA) (CASSANDRA-9771)
 +Merged from 2.1:
   * Fix MarshalException when upgrading superColumn family (CASSANDRA-9582)
   * Fix broken logging for "empty" flushes in Memtable (CASSANDRA-9837)
   * Handle corrupt files on startup (CASSANDRA-9686)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/32bc8b0b/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/32bc8b0b/src/java/org/apache/cassandra/tools/nodetool/Info.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Info.java
index 5852fc7,0000000..0d9bd73
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Info.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Info.java
@@@ -1,153 -1,0 +1,162 @@@
 +/*
 + * 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;
 +
 +import io.airlift.command.Command;
 +import io.airlift.command.Option;
 +
 +import java.lang.management.MemoryUsage;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +
 +import javax.management.InstanceNotFoundException;
 +
 +import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 +import org.apache.cassandra.io.util.FileUtils;
 +import org.apache.cassandra.service.CacheServiceMBean;
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
 +
 +@Command(name = "info", description = "Print node information (uptime, load, ...)")
 +public class Info extends NodeToolCmd
 +{
 +    @Option(name = {"-T", "--tokens"}, description = "Display all tokens")
 +    private boolean tokens = false;
 +
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        boolean gossipInitialized = probe.isInitialized();
 +
 +        System.out.printf("%-23s: %s%n", "ID", probe.getLocalHostId());
 +        System.out.printf("%-23s: %s%n", "Gossip active", gossipInitialized);
 +        System.out.printf("%-23s: %s%n", "Thrift active", probe.isThriftServerRunning());
 +        System.out.printf("%-23s: %s%n", "Native Transport active", probe.isNativeTransportRunning());
 +        System.out.printf("%-23s: %s%n", "Load", probe.getLoadString());
 +        if (gossipInitialized)
 +            System.out.printf("%-23s: %s%n", "Generation No", probe.getCurrentGenerationNumber());
 +        else
 +            System.out.printf("%-23s: %s%n", "Generation No", 0);
 +
 +        // Uptime
 +        long secondsUp = probe.getUptime() / 1000;
 +        System.out.printf("%-23s: %d%n", "Uptime (seconds)", secondsUp);
 +
 +        // Memory usage
 +        MemoryUsage heapUsage = probe.getHeapMemoryUsage();
 +        double memUsed = (double) heapUsage.getUsed() / (1024 * 1024);
 +        double memMax = (double) heapUsage.getMax() / (1024 * 1024);
 +        System.out.printf("%-23s: %.2f / %.2f%n", "Heap Memory (MB)", memUsed, memMax);
 +        try
 +        {
 +            System.out.printf("%-23s: %.2f%n", "Off Heap Memory (MB)", getOffHeapMemoryUsed(probe));
 +        }
 +        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;
 +        }
 +
 +        // Data Center/Rack
 +        System.out.printf("%-23s: %s%n", "Data Center", probe.getDataCenter());
 +        System.out.printf("%-23s: %s%n", "Rack", probe.getRack());
 +
 +        // Exceptions
 +        System.out.printf("%-23s: %s%n", "Exceptions", probe.getStorageMetric("Exceptions"));
 +
 +        CacheServiceMBean cacheService = probe.getCacheServiceMBean();
 +
 +        // Key Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
 +        System.out.printf("%-23s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
 +                "Key Cache",
 +                probe.getCacheMetric("KeyCache", "Entries"),
 +                FileUtils.stringifyFileSize((long) probe.getCacheMetric("KeyCache", "Size")),
 +                FileUtils.stringifyFileSize((long) probe.getCacheMetric("KeyCache", "Capacity")),
 +                probe.getCacheMetric("KeyCache", "Hits"),
 +                probe.getCacheMetric("KeyCache", "Requests"),
 +                probe.getCacheMetric("KeyCache", "HitRate"),
 +                cacheService.getKeyCacheSavePeriodInSeconds());
 +
 +        // Row Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
 +        System.out.printf("%-23s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
 +                "Row Cache",
 +                probe.getCacheMetric("RowCache", "Entries"),
 +                FileUtils.stringifyFileSize((long) probe.getCacheMetric("RowCache", "Size")),
 +                FileUtils.stringifyFileSize((long) probe.getCacheMetric("RowCache", "Capacity")),
 +                probe.getCacheMetric("RowCache", "Hits"),
 +                probe.getCacheMetric("RowCache", "Requests"),
 +                probe.getCacheMetric("RowCache", "HitRate"),
 +                cacheService.getRowCacheSavePeriodInSeconds());
 +
 +        // Counter Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
 +        System.out.printf("%-23s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
 +                "Counter Cache",
 +                probe.getCacheMetric("CounterCache", "Entries"),
 +                FileUtils.stringifyFileSize((long) probe.getCacheMetric("CounterCache", "Size")),
 +                FileUtils.stringifyFileSize((long) probe.getCacheMetric("CounterCache", "Capacity")),
 +                probe.getCacheMetric("CounterCache", "Hits"),
 +                probe.getCacheMetric("CounterCache", "Requests"),
 +                probe.getCacheMetric("CounterCache", "HitRate"),
 +                cacheService.getCounterCacheSavePeriodInSeconds());
 +
-         // Tokens
-         List<String> tokens = probe.getTokens();
-         if (tokens.size() == 1 || this.tokens)
-             for (String token : tokens)
-                 System.out.printf("%-23s: %s%n", "Token", token);
++        // check if node is already joined, before getting tokens, since it throws exception if not.
++        if (probe.isJoined())
++        {
++            // Tokens
++            List<String> tokens = probe.getTokens();
++            if (tokens.size() == 1 || this.tokens)
++                for (String token : tokens)
++                    System.out.printf("%-23s: %s%n", "Token", token);
++            else
++                System.out.printf("%-23s: (invoke with -T/--tokens to see all %d tokens)%n", "Token",
++                                  tokens.size());
++        }
 +        else
-             System.out.printf("%-23s: (invoke with -T/--tokens to see all %d tokens)%n", "Token", tokens.size());
++        {
++            System.out.printf("%-23s: (node is not joined to the cluster)%n", "Token");
++        }
 +    }
 +
 +    /**
 +     * Returns the total off heap memory used in MB.
 +     * @return the total off heap memory used in MB.
 +     */
 +    private static double getOffHeapMemoryUsed(NodeProbe probe)
 +    {
 +        long offHeapMemUsedInBytes = 0;
 +        // get a list of column family stores
 +        Iterator<Map.Entry<String, ColumnFamilyStoreMBean>> cfamilies = probe.getColumnFamilyStoreMBeanProxies();
 +
 +        while (cfamilies.hasNext())
 +        {
 +            Entry<String, ColumnFamilyStoreMBean> entry = cfamilies.next();
 +            String keyspaceName = entry.getKey();
 +            String cfName = entry.getValue().getColumnFamilyName();
 +
 +            offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "MemtableOffHeapSize");
 +            offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "BloomFilterOffHeapMemoryUsed");
 +            offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "IndexSummaryOffHeapMemoryUsed");
 +            offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "CompressionMetadataOffHeapMemoryUsed");
 +        }
 +
 +        return offHeapMemUsedInBytes / (1024d * 1024);
 +    }
- }
++}