Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 5CA0BD424 for ; Fri, 27 Jul 2012 15:19:48 +0000 (UTC) Received: (qmail 90547 invoked by uid 500); 27 Jul 2012 15:19:40 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 90487 invoked by uid 500); 27 Jul 2012 15:19:40 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 89262 invoked by uid 99); 27 Jul 2012 15:19:39 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 27 Jul 2012 15:19:39 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 456361963A; Fri, 27 Jul 2012 15:19:39 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: yukim@apache.org To: commits@cassandra.apache.org X-Mailer: ASF-Git Admin Mailer Subject: [35/50] [abbrv] git commit: jmx / nodetool support for virtual nodes Message-Id: <20120727151939.456361963A@tyr.zones.apache.org> Date: Fri, 27 Jul 2012 15:19:39 +0000 (UTC) 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/cassandra-1.1 Commit: 4f9fd76db3b63cea98426097e424524dc07237ad Parents: 8c09e87 Author: Eric Evans Authored: Wed Jul 18 13:39:29 2012 -0500 Committer: Eric Evans 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 getTokens() { - return getLocalTokens().iterator().next().toString(); + return getTokens(FBUtilities.getBroadcastAddress()); + } + + public List getTokens(String endpoint) throws UnknownHostException + { + return getTokens(InetAddress.getByName(endpoint)); + } + + private List getTokens(InetAddress endpoint) + { + List strTokens = new ArrayList(); + 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 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 endpoints : endpointsGroupedByDc) { - // sort the endpoints by their tokens - List sortedEndpoints = Lists.newArrayListWithExpectedSize(endpoints.size()); - sortedEndpoints.addAll(endpoints); - - Collections.sort(sortedEndpoints, new Comparator() - { - 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 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 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 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 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 USERNAME_OPT = new Pair("u", "username"); private static final Pair PASSWORD_OPT = new Pair("pw", "password"); private static final Pair TAG_OPT = new Pair("t", "tag"); + private static final Pair TOKENS_OPT = new Pair("T", "tokens"); private static final Pair PRIMARY_RANGE_OPT = new Pair("pr", "partitioner-range"); private static final Pair SNAPSHOT_REPAIR_OPT = new Pair("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 endpointsToTokens = ImmutableBiMap.copyOf(probe.getTokenToEndpointMap()).inverse(); + Map tokensToEndpoints = probe.getTokenToEndpointMap(); + LinkedHashMultimap endpointsToTokens = LinkedHashMultimap.create(); + for (Map.Entry 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 endpointsToTokens, + private void printDc(PrintStream outs, String format, String dc, LinkedHashMultimap endpointsToTokens, boolean keyspaceSelected, Map filteredOwnerships) { Collection 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 tokens = new ArrayList(); float totalReplicas = 0f; String lastToken = ""; + for (Map.Entry 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 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 entry : probe.getHostIdMap().entrySet()) + String kSpace = null, format = null; + Collection joiningNodes, leavingNodes, movingNodes, liveNodes, unreachableNodes; + Map 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> getOwnershipByDc(Map ownerships) + throws UnknownHostException + { + Map> ownershipByDc = Maps.newLinkedHashMap(); + EndpointSnitchInfoMBean epSnitchInfo = probe.getEndpointSnitchInfoProxy(); + + for (Map.Entry ownership : ownerships.entrySet()) + { + String dc = epSnitchInfo.getDatacenter(ownership.getKey().getHostAddress()); + if (!ownershipByDc.containsKey(dc)) + ownershipByDc.put(dc, new LinkedHashMap()); + 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 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 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(tokensToEndpoints.values()).size() < tokensToEndpoints.keySet().size()) + isTokenPerNode = false; + + // Datacenters + for (Map.Entry> 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 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 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 getTokens() { - return ssProxy.getToken(); + return ssProxy.getTokens(); + } + + public List 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 tokenToEndpoint = ssProxy.getTokenToEndpointMap(); for (Map.Entry pair : tokenToEndpoint.entrySet())