You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by br...@apache.org on 2014/09/14 01:04:50 UTC

[1/3] git commit: Don't output nonsense ownership without a keyspace

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 58c9a4c44 -> d002c7edc
  refs/heads/trunk 4191967e3 -> 080816637


Don't output nonsense ownership without a keyspace

Patch by Steven Nelson, reviewed by brandonwilliams for CASSANDRA-7173


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

Branch: refs/heads/cassandra-2.1
Commit: d002c7edc58b3a01e57392eb4f5aab367b1a5512
Parents: 58c9a4c
Author: Brandon Williams <br...@apache.org>
Authored: Sat Sep 13 18:04:05 2014 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Sat Sep 13 18:04:05 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/service/StorageService.java       | 33 ++++++++++++----
 .../org/apache/cassandra/tools/NodeTool.java    | 40 ++++++++++++++++----
 3 files changed, 59 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d002c7ed/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7f10ca5..6e029ab 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.1
+ * nodetool won't output bogus ownership info without a keyspace (CASSANDRA-7173)
  * Add human readable option to nodetool commands (CASSANDRA-5433)
  * Don't try to set repairedAt on old sstables (CASSANDRA-7913)
  * Add metrics for tracking PreparedStatement use (CASSANDRA-7719)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d002c7ed/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 788d425..86412ba 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -51,10 +51,8 @@ import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.Uninterruptibles;
 
 import org.apache.commons.lang3.StringUtils;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.cassandra.auth.Auth;
 import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.Stage;
@@ -3610,11 +3608,32 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     public LinkedHashMap<InetAddress, Float> effectiveOwnership(String keyspace) throws IllegalStateException
     {
-        if (Schema.instance.getNonSystemKeyspaces().size() <= 0)
-            throw new IllegalStateException("Couldn't find any Non System Keyspaces to infer replication topology");
-        if (keyspace == null && !hasSameReplication(Schema.instance.getNonSystemKeyspaces()))
-            throw new IllegalStateException("Non System keyspaces doesnt have the same topology");
-
+    	
+    	if (keyspace != null)
+    	{
+    		Keyspace keyspaceInstance = Schema.instance.getKeyspaceInstance(keyspace);
+			if(keyspaceInstance == null)
+				throw new IllegalArgumentException("The keyspace " + keyspace + ", does not exist");
+    		
+    		if(keyspaceInstance.getReplicationStrategy() instanceof LocalStrategy)
+				throw new IllegalStateException("Ownership values for keyspaces with LocalStrategy are meaningless");
+    	}
+    	else
+    	{
+        	List<String> nonSystemKeyspaces = Schema.instance.getNonSystemKeyspaces();
+        	
+        	//system_traces is a non-system keyspace however it needs to be counted as one for this process
+        	int specialTableCount = 0;
+        	if (nonSystemKeyspaces.contains("system_traces"))
+			{
+        		specialTableCount += 1;
+			}
+        	if (nonSystemKeyspaces.size() > specialTableCount) 	   		
+        		throw new IllegalStateException("Non-system keyspaces don't have the same replication settings, effective ownership information is meaningless");
+        	
+        	keyspace = "system_traces";
+    	}
+    	
         TokenMetadata metadata = tokenMetadata.cloneOnlyTokenMap();
 
         if (keyspace == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d002c7ed/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 cc38107..e7d1404 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -39,12 +39,14 @@ import com.yammer.metrics.reporting.JmxReporter;
 import io.airlift.command.*;
 
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.CompactionManagerMBean;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
+import org.apache.cassandra.locator.LocalStrategy;
 import org.apache.cassandra.net.MessagingServiceMBean;
 import org.apache.cassandra.service.CacheServiceMBean;
 import org.apache.cassandra.streaming.ProgressInfo;
@@ -450,31 +452,44 @@ public class NodeTool
             String formatPlaceholder = "%%-%ds  %%-12s%%-7s%%-8s%%-16s%%-20s%%-44s%%n";
             String format = format(formatPlaceholder, maxAddressLength);
 
+            StringBuffer errors = new StringBuffer();
+            boolean showEffectiveOwnership = true;
             // Calculate per-token ownership of the ring
             Map<InetAddress, Float> ownerships;
             try
             {
                 ownerships = probe.effectiveOwnership(keyspace);
-            } catch (IllegalStateException ex)
+            } 
+            catch (IllegalStateException ex)
             {
                 ownerships = probe.getOwnership();
-                System.out.printf("Note: Ownership information does not include topology; for complete information, specify a keyspace%n");
+                errors.append("Note: " + ex.getMessage() + "%n");
+                showEffectiveOwnership = false;
+            } 
+            catch (IllegalArgumentException ex)
+            {
+                System.out.printf("%nError: " + ex.getMessage() + "%n");
+                return;
             }
+
+            
             System.out.println();
             for (Entry<String, SetHostStat> entry : getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships).entrySet())
-                printDc(probe, format, entry.getKey(), endpointsToTokens, entry.getValue());
+                printDc(probe, format, entry.getKey(), endpointsToTokens, entry.getValue(),showEffectiveOwnership);
 
             if (haveVnodes)
             {
                 System.out.println("  Warning: \"nodetool ring\" is used to output all the tokens of a node.");
                 System.out.println("  To view status related info of a node use \"nodetool status\" instead.\n");
             }
+
+            System.out.printf("%n  " + errors.toString());
         }
 
         private void printDc(NodeProbe probe, String format,
                              String dc,
                              LinkedHashMultimap<String, String> endpointsToTokens,
-                             SetHostStat hoststats)
+                             SetHostStat hoststats,boolean showEffectiveOwnership)
         {
             Collection<String> liveNodes = probe.getLiveNodes();
             Collection<String> deadNodes = probe.getUnreachableNodes();
@@ -534,7 +549,7 @@ public class NodeTool
                 String load = loadMap.containsKey(endpoint)
                         ? loadMap.get(endpoint)
                         : "?";
-                String owns = stat.owns != null ? new DecimalFormat("##0.00%").format(stat.owns) : "?";
+                String owns = stat.owns != null && showEffectiveOwnership? new DecimalFormat("##0.00%").format(stat.owns) : "?";
                 System.out.printf(format, stat.ipOrDns(), rack, status, state, load, owns, stat.token);
             }
             System.out.println();
@@ -1886,6 +1901,8 @@ public class NodeTool
             unreachableNodes = probe.getUnreachableNodes();
             hostIDMap = probe.getHostIdMap();
             epSnitchInfo = probe.getEndpointSnitchInfoProxy();
+            
+            StringBuffer errors = new StringBuffer();
 
             Map<InetAddress, Float> ownerships;
             try
@@ -1895,7 +1912,12 @@ public class NodeTool
             } catch (IllegalStateException e)
             {
                 ownerships = probe.getOwnership();
-                System.out.printf("Note: Ownership information does not include topology; for complete information, specify a keyspace%n");
+                errors.append("Note: " + e.getMessage() + "%n");
+            }
+            catch (IllegalArgumentException ex)
+            {
+                System.out.printf("%nError: " + ex.getMessage() + "%n");
+                return;
             }
 
             Map<String, SetHostStat> dcs = getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships);
@@ -1931,7 +1953,9 @@ public class NodeTool
                     printNode(endpoint.getHostAddress(), owns, tokens, hasEffectiveOwns, isTokenPerNode);
                 }
             }
-
+            
+            System.out.printf("%n" + errors.toString());
+            
         }
 
         private void findMaxAddressLength(Map<String, SetHostStat> dcs)
@@ -1970,7 +1994,7 @@ public class NodeTool
             else state = "N";
 
             load = loadMap.containsKey(endpoint) ? loadMap.get(endpoint) : "?";
-            strOwns = owns != null ? new DecimalFormat("##0.0%").format(owns) : "?";
+            strOwns = owns != null && hasEffectiveOwns ? new DecimalFormat("##0.0%").format(owns) : "?";
             hostID = hostIDMap.get(endpoint);
 
             try


[2/3] git commit: Don't output nonsense ownership without a keyspace

Posted by br...@apache.org.
Don't output nonsense ownership without a keyspace

Patch by Steven Nelson, reviewed by brandonwilliams for CASSANDRA-7173


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

Branch: refs/heads/trunk
Commit: d002c7edc58b3a01e57392eb4f5aab367b1a5512
Parents: 58c9a4c
Author: Brandon Williams <br...@apache.org>
Authored: Sat Sep 13 18:04:05 2014 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Sat Sep 13 18:04:05 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/service/StorageService.java       | 33 ++++++++++++----
 .../org/apache/cassandra/tools/NodeTool.java    | 40 ++++++++++++++++----
 3 files changed, 59 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d002c7ed/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7f10ca5..6e029ab 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.1
+ * nodetool won't output bogus ownership info without a keyspace (CASSANDRA-7173)
  * Add human readable option to nodetool commands (CASSANDRA-5433)
  * Don't try to set repairedAt on old sstables (CASSANDRA-7913)
  * Add metrics for tracking PreparedStatement use (CASSANDRA-7719)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d002c7ed/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 788d425..86412ba 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -51,10 +51,8 @@ import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.Uninterruptibles;
 
 import org.apache.commons.lang3.StringUtils;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.cassandra.auth.Auth;
 import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.Stage;
@@ -3610,11 +3608,32 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     public LinkedHashMap<InetAddress, Float> effectiveOwnership(String keyspace) throws IllegalStateException
     {
-        if (Schema.instance.getNonSystemKeyspaces().size() <= 0)
-            throw new IllegalStateException("Couldn't find any Non System Keyspaces to infer replication topology");
-        if (keyspace == null && !hasSameReplication(Schema.instance.getNonSystemKeyspaces()))
-            throw new IllegalStateException("Non System keyspaces doesnt have the same topology");
-
+    	
+    	if (keyspace != null)
+    	{
+    		Keyspace keyspaceInstance = Schema.instance.getKeyspaceInstance(keyspace);
+			if(keyspaceInstance == null)
+				throw new IllegalArgumentException("The keyspace " + keyspace + ", does not exist");
+    		
+    		if(keyspaceInstance.getReplicationStrategy() instanceof LocalStrategy)
+				throw new IllegalStateException("Ownership values for keyspaces with LocalStrategy are meaningless");
+    	}
+    	else
+    	{
+        	List<String> nonSystemKeyspaces = Schema.instance.getNonSystemKeyspaces();
+        	
+        	//system_traces is a non-system keyspace however it needs to be counted as one for this process
+        	int specialTableCount = 0;
+        	if (nonSystemKeyspaces.contains("system_traces"))
+			{
+        		specialTableCount += 1;
+			}
+        	if (nonSystemKeyspaces.size() > specialTableCount) 	   		
+        		throw new IllegalStateException("Non-system keyspaces don't have the same replication settings, effective ownership information is meaningless");
+        	
+        	keyspace = "system_traces";
+    	}
+    	
         TokenMetadata metadata = tokenMetadata.cloneOnlyTokenMap();
 
         if (keyspace == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d002c7ed/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 cc38107..e7d1404 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -39,12 +39,14 @@ import com.yammer.metrics.reporting.JmxReporter;
 import io.airlift.command.*;
 
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.CompactionManagerMBean;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
+import org.apache.cassandra.locator.LocalStrategy;
 import org.apache.cassandra.net.MessagingServiceMBean;
 import org.apache.cassandra.service.CacheServiceMBean;
 import org.apache.cassandra.streaming.ProgressInfo;
@@ -450,31 +452,44 @@ public class NodeTool
             String formatPlaceholder = "%%-%ds  %%-12s%%-7s%%-8s%%-16s%%-20s%%-44s%%n";
             String format = format(formatPlaceholder, maxAddressLength);
 
+            StringBuffer errors = new StringBuffer();
+            boolean showEffectiveOwnership = true;
             // Calculate per-token ownership of the ring
             Map<InetAddress, Float> ownerships;
             try
             {
                 ownerships = probe.effectiveOwnership(keyspace);
-            } catch (IllegalStateException ex)
+            } 
+            catch (IllegalStateException ex)
             {
                 ownerships = probe.getOwnership();
-                System.out.printf("Note: Ownership information does not include topology; for complete information, specify a keyspace%n");
+                errors.append("Note: " + ex.getMessage() + "%n");
+                showEffectiveOwnership = false;
+            } 
+            catch (IllegalArgumentException ex)
+            {
+                System.out.printf("%nError: " + ex.getMessage() + "%n");
+                return;
             }
+
+            
             System.out.println();
             for (Entry<String, SetHostStat> entry : getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships).entrySet())
-                printDc(probe, format, entry.getKey(), endpointsToTokens, entry.getValue());
+                printDc(probe, format, entry.getKey(), endpointsToTokens, entry.getValue(),showEffectiveOwnership);
 
             if (haveVnodes)
             {
                 System.out.println("  Warning: \"nodetool ring\" is used to output all the tokens of a node.");
                 System.out.println("  To view status related info of a node use \"nodetool status\" instead.\n");
             }
+
+            System.out.printf("%n  " + errors.toString());
         }
 
         private void printDc(NodeProbe probe, String format,
                              String dc,
                              LinkedHashMultimap<String, String> endpointsToTokens,
-                             SetHostStat hoststats)
+                             SetHostStat hoststats,boolean showEffectiveOwnership)
         {
             Collection<String> liveNodes = probe.getLiveNodes();
             Collection<String> deadNodes = probe.getUnreachableNodes();
@@ -534,7 +549,7 @@ public class NodeTool
                 String load = loadMap.containsKey(endpoint)
                         ? loadMap.get(endpoint)
                         : "?";
-                String owns = stat.owns != null ? new DecimalFormat("##0.00%").format(stat.owns) : "?";
+                String owns = stat.owns != null && showEffectiveOwnership? new DecimalFormat("##0.00%").format(stat.owns) : "?";
                 System.out.printf(format, stat.ipOrDns(), rack, status, state, load, owns, stat.token);
             }
             System.out.println();
@@ -1886,6 +1901,8 @@ public class NodeTool
             unreachableNodes = probe.getUnreachableNodes();
             hostIDMap = probe.getHostIdMap();
             epSnitchInfo = probe.getEndpointSnitchInfoProxy();
+            
+            StringBuffer errors = new StringBuffer();
 
             Map<InetAddress, Float> ownerships;
             try
@@ -1895,7 +1912,12 @@ public class NodeTool
             } catch (IllegalStateException e)
             {
                 ownerships = probe.getOwnership();
-                System.out.printf("Note: Ownership information does not include topology; for complete information, specify a keyspace%n");
+                errors.append("Note: " + e.getMessage() + "%n");
+            }
+            catch (IllegalArgumentException ex)
+            {
+                System.out.printf("%nError: " + ex.getMessage() + "%n");
+                return;
             }
 
             Map<String, SetHostStat> dcs = getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships);
@@ -1931,7 +1953,9 @@ public class NodeTool
                     printNode(endpoint.getHostAddress(), owns, tokens, hasEffectiveOwns, isTokenPerNode);
                 }
             }
-
+            
+            System.out.printf("%n" + errors.toString());
+            
         }
 
         private void findMaxAddressLength(Map<String, SetHostStat> dcs)
@@ -1970,7 +1994,7 @@ public class NodeTool
             else state = "N";
 
             load = loadMap.containsKey(endpoint) ? loadMap.get(endpoint) : "?";
-            strOwns = owns != null ? new DecimalFormat("##0.0%").format(owns) : "?";
+            strOwns = owns != null && hasEffectiveOwns ? new DecimalFormat("##0.0%").format(owns) : "?";
             hostID = hostIDMap.get(endpoint);
 
             try


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

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


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

Branch: refs/heads/trunk
Commit: 0808166375ba3fc08f09d1a7637de3cb0094ddcd
Parents: 4191967 d002c7e
Author: Brandon Williams <br...@apache.org>
Authored: Sat Sep 13 18:04:41 2014 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Sat Sep 13 18:04:41 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/service/StorageService.java       | 33 ++++++++++++----
 .../org/apache/cassandra/tools/NodeTool.java    | 40 ++++++++++++++++----
 3 files changed, 59 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/08081663/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 488f4fd,6e029ab..c35e5b2
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,26 -1,5 +1,27 @@@
 +3.0
 + * Remove YamlFileNetworkTopologySnitch (CASSANDRA-7917)
 + * Support Java source code for user-defined functions (CASSANDRA-7562)
 + * Require arg types to disambiguate UDF drops (CASSANDRA-7812)
 + * Do anticompaction in groups (CASSANDRA-6851)
 + * Verify that UDF class methods are static (CASSANDRA-7781)
 + * Support pure user-defined functions (CASSANDRA-7395, 7740)
 + * Permit configurable timestamps with cassandra-stress (CASSANDRA-7416)
 + * Move sstable RandomAccessReader to nio2, which allows using the
 +   FILE_SHARE_DELETE flag on Windows (CASSANDRA-4050)
 + * Remove CQL2 (CASSANDRA-5918)
 + * Add Thrift get_multi_slice call (CASSANDRA-6757)
 + * Optimize fetching multiple cells by name (CASSANDRA-6933)
 + * Allow compilation in java 8 (CASSANDRA-7028)
 + * Make incremental repair default (CASSANDRA-7250)
 + * Enable code coverage thru JaCoCo (CASSANDRA-7226)
 + * Switch external naming of 'column families' to 'tables' (CASSANDRA-4369) 
 + * Shorten SSTable path (CASSANDRA-6962)
 + * Use unsafe mutations for most unit tests (CASSANDRA-6969)
 + * Fix race condition during calculation of pending ranges (CASSANDRA-7390)
 +
 +
  2.1.1
+  * nodetool won't output bogus ownership info without a keyspace (CASSANDRA-7173)
   * Add human readable option to nodetool commands (CASSANDRA-5433)
   * Don't try to set repairedAt on old sstables (CASSANDRA-7913)
   * Add metrics for tracking PreparedStatement use (CASSANDRA-7719)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/08081663/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/08081663/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeTool.java
index 8f0dcef,e7d1404..0bfbc13
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@@ -1892,11 -1909,15 +1909,16 @@@ public class NodeToo
              {
                  ownerships = probe.effectiveOwnership(keyspace);
                  hasEffectiveOwns = true;
 -            } catch (IllegalStateException e)
 +            }
 +            catch (IllegalStateException e)
              {
                  ownerships = probe.getOwnership();
-                 System.out.printf("Note: Ownership information does not include topology; for complete information, specify a keyspace%n");
+                 errors.append("Note: " + e.getMessage() + "%n");
+             }
+             catch (IllegalArgumentException ex)
+             {
+                 System.out.printf("%nError: " + ex.getMessage() + "%n");
+                 return;
              }
  
              Map<String, SetHostStat> dcs = getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships);