You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ee...@apache.org on 2012/07/18 20:49:39 UTC

[2/5] git commit: jmx / nodetool support for virtual nodes

jmx / nodetool support for virtual nodes

Patch by eevans; reviewed by Brandon Williams for CASSANDRA-4125


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

Branch: refs/heads/trunk
Commit: 4f9fd76db3b63cea98426097e424524dc07237ad
Parents: 8c09e87
Author: Eric Evans <ee...@apache.org>
Authored: Wed Jul 18 13:39:29 2012 -0500
Committer: Eric Evans <ee...@apache.org>
Committed: Wed Jul 18 13:39:29 2012 -0500

----------------------------------------------------------------------
 .../apache/cassandra/service/StorageService.java   |   54 ++--
 .../cassandra/service/StorageServiceMBean.java     |   14 +-
 src/java/org/apache/cassandra/tools/NodeCmd.java   |  282 ++++++++++++---
 src/java/org/apache/cassandra/tools/NodeProbe.java |   18 +-
 4 files changed, 283 insertions(+), 85 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4f9fd76d/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 3875054..b5d6d20 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -1726,9 +1726,22 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
     /* These methods belong to the MBean interface */
 
-    public String getToken()
+    public List<String> getTokens()
     {
-        return getLocalTokens().iterator().next().toString();
+        return getTokens(FBUtilities.getBroadcastAddress());
+    }
+
+    public List<String> getTokens(String endpoint) throws UnknownHostException
+    {
+        return getTokens(InetAddress.getByName(endpoint));
+    }
+
+    private List<String> getTokens(InetAddress endpoint)
+    {
+        List<String> strTokens = new ArrayList<String>();
+        for (Token tok : getTokenMetadata().getTokens(endpoint))
+            strTokens.add(tok.toString());
+        return strTokens;
     }
 
     public String getReleaseVersion()
@@ -2432,6 +2445,14 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
 
         // address of the current node
         InetAddress localAddress = FBUtilities.getBroadcastAddress();
+
+        // This doesn't make any sense in a vnodes environment.
+        if (getTokenMetadata().getTokens(localAddress).size() > 1)
+        {
+            logger.error("Invalid request to move(Token); This node has more than one token and cannot be moved thusly.");
+            throw new UnsupportedOperationException("This node has more than one token and cannot be moved thusly.");
+        }
+        
         List<String> tablesToProcess = Schema.instance.getNonSystemTables();
 
         // checking if data is moving to this node
@@ -2861,39 +2882,14 @@ public class StorageService implements IEndpointStateChangeSubscriber, StorageSe
         // calculate ownership per dc
         for (Collection<InetAddress> endpoints : endpointsGroupedByDc)
         {
-            // sort the endpoints by their tokens
-            List<InetAddress> sortedEndpoints = Lists.newArrayListWithExpectedSize(endpoints.size());
-            sortedEndpoints.addAll(endpoints);
-
-            Collections.sort(sortedEndpoints, new Comparator<InetAddress>()
-            {
-                public int compare(InetAddress o1, InetAddress o2)
-                {
-                    byte[] b1 = o1.getAddress();
-                    byte[] b2 = o2.getAddress();
-
-                    if(b1.length < b2.length) return -1;
-                    if(b1.length > b2.length) return 1;
-
-                    for(int i = 0; i < b1.length; i++)
-                    {
-                        int left = (int)b1[i] & 0xFF;
-                        int right = (int)b2[i] & 0xFF;
-                        if (left < right)       return -1;
-                        else if (left > right)  return 1;
-                    }
-                    return 0;
-                }
-            });
-
             // calculate the ownership with replication and add the endpoint to the final ownership map
             for (InetAddress endpoint : endpoints)
             {
                 float ownership = 0.0f;
                 for (Range<Token> range : getRangesForEndpoint(keyspace, endpoint))
                 {
-                    if (tokenOwnership.containsKey(range.left))
-                        ownership += tokenOwnership.get(range.left);
+                    if (tokenOwnership.containsKey(range.right))
+                        ownership += tokenOwnership.get(range.right);
                 }
                 finalOwnership.put(endpoint, ownership);
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4f9fd76d/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 80ba05f..4201542 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -72,11 +72,19 @@ public interface StorageServiceMBean
     public List<String> getMovingNodes();
 
     /**
-     * Fetch a string representation of the token.
+     * Fetch string representations of the tokens for this node.
      *
-     * @return a string token
+     * @return a collection of tokens formatted as strings
      */
-    public String getToken();
+    public List<String> getTokens();
+
+    /**
+     * Fetch string representations of the tokens for a specified node.
+     *
+     * @param endpoint string representation of an node
+     * @return a collection of tokens formatted as strings
+     */
+    public List<String> getTokens(String endpoint) throws UnknownHostException;
 
     /**
      * Fetch a string representation of the Cassandra version.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4f9fd76d/src/java/org/apache/cassandra/tools/NodeCmd.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeCmd.java b/src/java/org/apache/cassandra/tools/NodeCmd.java
index 2a732cf..9fff6c5 100644
--- a/src/java/org/apache/cassandra/tools/NodeCmd.java
+++ b/src/java/org/apache/cassandra/tools/NodeCmd.java
@@ -28,8 +28,9 @@ import java.util.*;
 import java.util.Map.Entry;
 import java.util.concurrent.ExecutionException;
 
-import com.google.common.collect.ImmutableBiMap;
+import com.google.common.collect.LinkedHashMultimap;
 import com.google.common.collect.Maps;
+
 import org.apache.commons.cli.*;
 
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
@@ -37,12 +38,15 @@ import org.apache.cassandra.config.ConfigurationException;
 import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 import org.apache.cassandra.db.compaction.CompactionManagerMBean;
 import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.locator.EndpointSnitchInfo;
+import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
 import org.apache.cassandra.net.MessagingServiceMBean;
 import org.apache.cassandra.service.CacheServiceMBean;
 import org.apache.cassandra.service.StorageProxyMBean;
 import org.apache.cassandra.thrift.InvalidRequestException;
 import org.apache.cassandra.utils.EstimatedHistogram;
 import org.apache.cassandra.utils.Pair;
+import org.apache.rat.document.UnreadableArchiveException;
 
 public class NodeCmd
 {
@@ -52,6 +56,7 @@ public class NodeCmd
     private static final Pair<String, String> USERNAME_OPT = new Pair<String, String>("u",  "username");
     private static final Pair<String, String> PASSWORD_OPT = new Pair<String, String>("pw", "password");
     private static final Pair<String, String> TAG_OPT = new Pair<String, String>("t", "tag");
+    private static final Pair<String, String> TOKENS_OPT = new Pair<String, String>("T", "tokens");
     private static final Pair<String, String> PRIMARY_RANGE_OPT = new Pair<String, String>("pr", "partitioner-range");
     private static final Pair<String, String> SNAPSHOT_REPAIR_OPT = new Pair<String, String>("snapshot", "with-snapshot");
 
@@ -70,6 +75,7 @@ public class NodeCmd
         options.addOption(USERNAME_OPT, true, "remote jmx agent username");
         options.addOption(PASSWORD_OPT, true, "remote jmx agent password");
         options.addOption(TAG_OPT,      true, "optional name to give a snapshot");
+        options.addOption(TOKENS_OPT,   false, "display all tokens");
         options.addOption(PRIMARY_RANGE_OPT, false, "only repair the first range returned by the partitioner for the node");
         options.addOption(SNAPSHOT_REPAIR_OPT, false, "repair one node at a time using snapshots");
     }
@@ -98,7 +104,6 @@ public class NodeCmd
         GETENDPOINTS,
         GETSSTABLES,
         GOSSIPINFO,
-        IDS,
         INFO,
         INVALIDATEKEYCACHE,
         INVALIDATEROWCACHE,
@@ -118,6 +123,7 @@ public class NodeCmd
         SETCOMPACTIONTHROUGHPUT,
         SETSTREAMTHROUGHPUT,
         SNAPSHOT,
+        STATUS,
         STATUSTHRIFT,
         STOP,
         TPSTATS,
@@ -141,9 +147,9 @@ public class NodeCmd
         // No args
         addCmdHelp(header, "ring", "Print information about the token ring");
         addCmdHelp(header, "join", "Join the ring");
-        addCmdHelp(header, "info", "Print node information (uptime, load, ...)");
+        addCmdHelp(header, "info [-T/--tokens]", "Print node information (uptime, load, ...)");
+        addCmdHelp(header, "status", "Print cluster information (state, load, IDs, ...)");
         addCmdHelp(header, "cfstats", "Print statistics on column families");
-        addCmdHelp(header, "ids", "Print list of unique host IDs");
         addCmdHelp(header, "version", "Print cassandra version");
         addCmdHelp(header, "tpstats", "Print usage statistics of thread pools");
         addCmdHelp(header, "proxyhistograms", "Print statistic histograms for network operations");
@@ -216,7 +222,11 @@ public class NodeCmd
      */
     public void printRing(PrintStream outs, String keyspace)
     {
-        Map<String, String> endpointsToTokens = ImmutableBiMap.copyOf(probe.getTokenToEndpointMap()).inverse();
+        Map<String, String> tokensToEndpoints = probe.getTokenToEndpointMap();
+        LinkedHashMultimap<String, String> endpointsToTokens = LinkedHashMultimap.create();
+        for (Map.Entry<String, String> entry : tokensToEndpoints.entrySet())
+            endpointsToTokens.put(entry.getValue(), entry.getKey());
+
         String format = "%-16s%-12s%-7s%-8s%-16s%-20s%-44s%n";
 
         // Calculate per-token ownership of the ring
@@ -254,7 +264,7 @@ public class NodeCmd
         }
     }
     
-    private void printDc(PrintStream outs, String format, String dc, Map<String, String> endpointsToTokens,
+    private void printDc(PrintStream outs, String format, String dc, LinkedHashMultimap<String, String> endpointsToTokens,
             boolean keyspaceSelected, Map<InetAddress, Float> filteredOwnerships)
     {
         Collection<String> liveNodes = probe.getLiveNodes();
@@ -268,11 +278,14 @@ public class NodeCmd
         outs.println("==========");
 
         // get the total amount of replicas for this dc and the last token in this dc's ring
+        List<String> tokens = new ArrayList<String>();
         float totalReplicas = 0f;
         String lastToken = "";
+
         for (Map.Entry<InetAddress, Float> entry : filteredOwnerships.entrySet())
         {
-            lastToken = endpointsToTokens.get(entry.getKey().getHostAddress());
+            tokens.addAll(endpointsToTokens.get(entry.getKey().getHostAddress()));
+            lastToken = tokens.get(tokens.size() - 1);
             totalReplicas += entry.getValue();
         }
         
@@ -290,54 +303,196 @@ public class NodeCmd
         for (Map.Entry<InetAddress, Float> entry : filteredOwnerships.entrySet())
         {
             String endpoint = entry.getKey().getHostAddress();
-            String token = endpointsToTokens.get(entry.getKey().getHostAddress());
-            String rack;
-            try
-            {
-                rack = probe.getEndpointSnitchInfoProxy().getRack(endpoint);
-            }
-            catch (UnknownHostException e)
+            for (String token : endpointsToTokens.get(endpoint))
             {
-                rack = "Unknown";
+                String rack;
+                try
+                {
+                    rack = probe.getEndpointSnitchInfoProxy().getRack(endpoint);
+                }
+                catch (UnknownHostException e)
+                {
+                    rack = "Unknown";
+                }
+    
+                String status = liveNodes.contains(endpoint)
+                        ? "Up"
+                        : deadNodes.contains(endpoint)
+                                ? "Down"
+                                : "?";
+    
+                String state = "Normal";
+    
+                if (joiningNodes.contains(endpoint))
+                    state = "Joining";
+                else if (leavingNodes.contains(endpoint))
+                    state = "Leaving";
+                else if (movingNodes.contains(endpoint))
+                    state = "Moving";
+    
+                String load = loadMap.containsKey(endpoint)
+                        ? loadMap.get(endpoint)
+                        : "?";
+                String owns = new DecimalFormat("##0.00%").format(entry.getValue());
+                outs.printf(format, endpoint, rack, status, state, load, owns, token);
             }
-            String status = liveNodes.contains(endpoint)
-                    ? "Up"
-                    : deadNodes.contains(endpoint)
-                            ? "Down"
-                            : "?";
-
-            String state = "Normal";
-
-            if (joiningNodes.contains(endpoint))
-                state = "Joining";
-            else if (leavingNodes.contains(endpoint))
-                state = "Leaving";
-            else if (movingNodes.contains(endpoint))
-                state = "Moving";
-
-            String load = loadMap.containsKey(endpoint)
-                    ? loadMap.get(endpoint)
-                    : "?";
-            String owns = new DecimalFormat("##0.00%").format(entry.getValue());
-            outs.printf(format, endpoint, rack, status, state, load, owns, token);
         }
         outs.println();
     }
 
-    /** Writes a table of host IDs to a PrintStream */
-    public void printHostIds(PrintStream outs)
+    private class ClusterStatus
     {
-        System.out.print(String.format("%-16s %-7s %s%n", "Address", "Status", "Host ID"));
-        for (Map.Entry<String, String> entry : probe.getHostIdMap().entrySet())
+        String kSpace = null, format = null;
+        Collection<String> joiningNodes, leavingNodes, movingNodes, liveNodes, unreachableNodes;
+        Map<String, String> loadMap, hostIDMap, tokensToEndpoints;
+        EndpointSnitchInfoMBean epSnitchInfo;
+        PrintStream outs;
+
+        ClusterStatus(PrintStream outs, String kSpace)
+        {
+            this.kSpace = kSpace;
+            this.outs = outs;
+            joiningNodes = probe.getJoiningNodes();
+            leavingNodes = probe.getLeavingNodes();
+            movingNodes = probe.getMovingNodes();
+            loadMap = probe.getLoadMap();
+            tokensToEndpoints = probe.getTokenToEndpointMap();
+            liveNodes = probe.getLiveNodes();
+            unreachableNodes = probe.getUnreachableNodes();
+            hostIDMap = probe.getHostIdMap();
+            epSnitchInfo = probe.getEndpointSnitchInfoProxy();
+        }
+
+        private void printStatusLegend()
         {
-            String status;
-            if      (probe.getLiveNodes().contains(entry.getKey()))        status = "Up";
-            else if (probe.getUnreachableNodes().contains(entry.getKey())) status = "Down";
-            else                                                           status = "?";
-            System.out.print(String.format("%-16s %-7s %s%n", entry.getKey(), status, entry.getValue()));
+            outs.println("Status=Up/Down");
+            outs.println("|/ State=Normal/Leaving/Joining/Moving");
+        }
+
+        private Map<String, Map<InetAddress, Float>> getOwnershipByDc(Map<InetAddress, Float> ownerships)
+        throws UnknownHostException
+        {
+            Map<String, Map<InetAddress, Float>> ownershipByDc = Maps.newLinkedHashMap();
+            EndpointSnitchInfoMBean epSnitchInfo = probe.getEndpointSnitchInfoProxy();
+
+            for (Map.Entry<InetAddress, Float> ownership : ownerships.entrySet())
+            {
+                String dc = epSnitchInfo.getDatacenter(ownership.getKey().getHostAddress());
+                if (!ownershipByDc.containsKey(dc))
+                    ownershipByDc.put(dc, new LinkedHashMap<InetAddress, Float>());
+                ownershipByDc.get(dc).put(ownership.getKey(), ownership.getValue());
+            }
+
+            return ownershipByDc;
+        }
+
+        private String getFormat(boolean hasEffectiveOwns, boolean isTokenPerNode)
+        {
+            if (format == null)
+            {
+                StringBuffer buf = new StringBuffer();
+                buf.append("%s%s  %-16s  %-9s  ");            // status, address, and load
+                if (!isTokenPerNode)  buf.append("%-6s  ");   // "Tokens"
+                if (hasEffectiveOwns) buf.append("%-16s  ");  // "Owns (effective)"
+                else                  buf.append("%-5s  ");   // "Owns
+                buf.append("%-36s  ");                        // Host ID
+                if (isTokenPerNode)   buf.append("%-39s  ");  // token
+                buf.append("%s%n");                           // "Rack"
+
+                format = buf.toString();
+            }
+
+            return format;
+        }
+
+        private void printNode(String endpoint, Float owns, Map<InetAddress, Float> ownerships,
+                boolean hasEffectiveOwns, boolean isTokenPerNode) throws UnknownHostException
+        {
+            String status, state, load, strOwns, hostID, rack, fmt;
+            fmt = getFormat(hasEffectiveOwns, isTokenPerNode);
+
+            if      (liveNodes.contains(endpoint))        status = "U";
+            else if (unreachableNodes.contains(endpoint)) status = "D";
+            else                                          status = "?";
+            if      (joiningNodes.contains(endpoint))     state = "J";
+            else if (leavingNodes.contains(endpoint))     state = "L";
+            else if (movingNodes.contains(endpoint))      state = "M";
+            else                                          state = "N";
+
+            load = loadMap.containsKey(endpoint) ? loadMap.get(endpoint) : "?";
+            strOwns = new DecimalFormat("##0.0%").format(ownerships.get(InetAddress.getByName(endpoint)));
+            hostID = hostIDMap.get(endpoint);
+            rack = epSnitchInfo.getRack(endpoint);
+
+            if (isTokenPerNode)
+            {
+                outs.printf(fmt, status, state, endpoint, load, strOwns, hostID, probe.getTokens(endpoint).get(0), rack);
+            }
+            else
+            {
+                int tokens = probe.getTokens(endpoint).size();
+                outs.printf(fmt, status, state, endpoint, load, tokens, strOwns, hostID, rack);
+            }
+        }
+
+        private void printNodesHeader(boolean hasEffectiveOwns, boolean isTokenPerNode)
+        {
+            String fmt = getFormat(hasEffectiveOwns, isTokenPerNode);
+            String owns = hasEffectiveOwns ? "Owns (effective)" : "Owns";
+
+            if (isTokenPerNode)
+                outs.printf(fmt, "-", "-", "Address", "Load", owns, "Host ID", "Token", "Rack");
+            else
+                outs.printf(fmt, "-", "-", "Address", "Load", "Tokens", owns, "Host ID", "Rack");
+        }
+
+        void print() throws UnknownHostException
+        {
+            Map<InetAddress, Float> ownerships;
+            boolean hasEffectiveOwns = false, isTokenPerNode = true;
+            try
+            {
+                ownerships = probe.effectiveOwnership(kSpace);
+                hasEffectiveOwns = true;
+            }
+            catch (ConfigurationException e)
+            {
+                ownerships = probe.getOwnership();
+            }
+
+            // More tokens then nodes (aka vnodes)?
+            if (new HashSet<String>(tokensToEndpoints.values()).size() < tokensToEndpoints.keySet().size())
+                isTokenPerNode = false;
+
+            // Datacenters
+            for (Map.Entry<String, Map<InetAddress, Float>> dc : getOwnershipByDc(ownerships).entrySet())
+            {
+                String dcHeader = String.format("Datacenter: %s%n", dc.getKey());
+                outs.printf(dcHeader);
+                for (int i=0; i < (dcHeader.length() - 1); i++) outs.print('=');
+                outs.println();
+
+                printStatusLegend();
+                printNodesHeader(hasEffectiveOwns, isTokenPerNode);
+
+                // Nodes
+                for (Map.Entry<InetAddress, Float> entry : dc.getValue().entrySet())
+                    printNode(entry.getKey().getHostAddress(),
+                              entry.getValue(),
+                              ownerships,
+                              hasEffectiveOwns,
+                              isTokenPerNode);
+            }
         }
     }
 
+    /** Writes a table of cluster-wide node information to a PrintStream
+     * @throws UnknownHostException */
+    public void printClusterStatus(PrintStream outs, String keyspace) throws UnknownHostException
+    {
+        new ClusterStatus(outs, keyspace).print();
+    }
+
     public void printThreadPoolStats(PrintStream outs)
     {
         outs.printf("%-25s%10s%10s%15s%10s%18s%n", "Pool Name", "Active", "Pending", "Completed", "Blocked", "All time blocked");
@@ -367,10 +522,18 @@ public class NodeCmd
      *
      * @param outs the stream to write to
      */
-    public void printInfo(PrintStream outs)
+    public void printInfo(PrintStream outs, ToolCommandLine cmd)
     {
         boolean gossipInitialized = probe.isInitialized();
-        outs.printf("%-17s: %s%n", "Token", probe.getToken());
+        List<String> toks = probe.getTokens();
+
+        // If there is just 1 token, print it now like we always have, otherwise,
+        // require that -T/--tokens be passed (that output is potentially verbose).
+        if (toks.size() == 1)
+            outs.printf("%-17s: %s%n", "Token", toks.get(0));
+        else if (!cmd.hasOption(TOKENS_OPT.left))
+            outs.printf("%-17s: (invoke with -T/--tokens to see all %d tokens)%n", "Token", toks.size());
+
         outs.printf("%-17s: %s%n", "ID", probe.getLocalHostId());
         outs.printf("%-17s: %s%n", "Gossip active", gossipInitialized);
         outs.printf("%-17s: %s%n", "Thrift active", probe.isThriftServerRunning());
@@ -418,6 +581,12 @@ public class NodeCmd
                     cacheService.getRowCacheRequests(),
                     cacheService.getRowCacheRecentHitRate(),
                     cacheService.getRowCacheSavePeriodInSeconds());
+
+        if (toks.size() > 1 && cmd.hasOption(TOKENS_OPT.left))
+        {
+            for (String tok : toks)
+                outs.printf("%-17s: %s%n", "Token", tok);
+        }
     }
 
     public void printReleaseVersion(PrintStream outs)
@@ -789,7 +958,7 @@ public class NodeCmd
                     else                      { nodeCmd.printRing(System.out, null); };
                     break;
 
-                case INFO            : nodeCmd.printInfo(System.out); break;
+                case INFO            : nodeCmd.printInfo(System.out, cmd); break;
                 case CFSTATS         : nodeCmd.printColumnFamilyStats(System.out); break;
                 case TPSTATS         : nodeCmd.printThreadPoolStats(System.out); break;
                 case VERSION         : nodeCmd.printReleaseVersion(System.out); break;
@@ -800,7 +969,11 @@ public class NodeCmd
                 case ENABLETHRIFT    : probe.startThriftServer(); break;
                 case STATUSTHRIFT    : nodeCmd.printIsThriftServerRunning(System.out); break;
                 case RESETLOCALSCHEMA: probe.resetLocalSchema(); break;
-                case IDS             : nodeCmd.printHostIds(System.out); break;
+
+                case STATUS :
+                    if (arguments.length > 0) nodeCmd.printClusterStatus(System.out, arguments[0]);
+                    else                      nodeCmd.printClusterStatus(System.out, null);
+                    break;
 
                 case DECOMMISSION :
                     if (arguments.length > 0)
@@ -831,7 +1004,15 @@ public class NodeCmd
 
                 case MOVE :
                     if (arguments.length != 1) { badUse("Missing token argument for move."); }
-                    probe.move(arguments[0]);
+                    try
+                    {
+                        probe.move(arguments[0]);
+                    }
+                    catch (UnsupportedOperationException uoerror)
+                    {
+                        System.err.println(uoerror.getMessage());
+                        System.exit(1);
+                    }
                     break;
 
                 case JOIN:
@@ -859,8 +1040,9 @@ public class NodeCmd
                     probe.rebuild(arguments.length == 1 ? arguments[0] : null);
                     break;
 
-                case REMOVENODE  :
                 case REMOVETOKEN :
+                    System.err.println("Warn: removetoken is deprecated, please use removenode instead");
+                case REMOVENODE  :
                     if (arguments.length != 1) { badUse("Missing an argument for removenode (either status, force, or an ID)"); }
                     else if (arguments[0].equals("status")) { nodeCmd.printRemovalStatus(System.out); }
                     else if (arguments[0].equals("force"))  { nodeCmd.printRemovalStatus(System.out); probe.forceRemoveCompletion(); }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4f9fd76d/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 8ad8b92..93e0a22 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -308,9 +308,21 @@ public class NodeProbe
       return compactionProxy;
     }
 
-    public String getToken()
+    public List<String> getTokens()
     {
-        return ssProxy.getToken();
+        return ssProxy.getTokens();
+    }
+
+    public List<String> getTokens(String endpoint)
+    {
+        try
+        {
+            return ssProxy.getTokens(endpoint);
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
     }
 
     public String getLocalHostId()
@@ -572,7 +584,7 @@ public class NodeProbe
     {
         // 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.getToken();
+        String stringToken = ssProxy.getTokens().get(0);
         Map<String, String> tokenToEndpoint = ssProxy.getTokenToEndpointMap();
 
         for (Map.Entry<String, String> pair : tokenToEndpoint.entrySet())