Author: gdusbabek
Date: Mon Aug 2 20:19:42 2010
New Revision: 981673
URL: http://svn.apache.org/viewvc?rev=981673&view=rev
Log:
consult tokenmetadata directly for ring topology. patch by Sylvain Lebresne,
reviewed by Gary Dusbabek. CASSANDRA-1286
Modified:
cassandra/trunk/src/java/org/apache/cassandra/locator/TokenMetadata.java
cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
cassandra/trunk/src/java/org/apache/cassandra/service/StorageServiceMBean.java
cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java
cassandra/trunk/src/java/org/apache/cassandra/tools/NodeCmd.java
cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java
Modified:
cassandra/trunk/src/java/org/apache/cassandra/locator/TokenMetadata.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/locator/TokenMetadata.java?rev=981673&r1=981672&r2=981673&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/locator/TokenMetadata.java
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/locator/TokenMetadata.java
Mon Aug 2 20:19:42 2010
@@ -567,4 +567,15 @@ public class TokenMetadata
return endpoints;
}
+
+ /**
+ * Return the Token to Endpoint map for all the node in the cluster,
including bootstrapping ones.
+ */
+ public Map<Token, InetAddress> getTokenToEndpointMap()
+ {
+ Map<Token, InetAddress> map = new HashMap<Token,
InetAddress>(tokenToEndpointMap.size() + bootstrapTokens.size());
+ map.putAll(tokenToEndpointMap);
+ map.putAll(bootstrapTokens);
+ return map;
+ }
}
Modified:
cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java?rev=981673&r1=981672&r2=981673&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
Mon Aug 2 20:19:42 2010
@@ -532,6 +532,17 @@ public class StorageService implements I
return constructRangeToEndpointMap(keyspace, ranges);
}
+ public Map<Token, String> getTokenToEndpointMap()
+ {
+ Map<Token, InetAddress> mapInetAddress =
tokenMetadata_.getTokenToEndpointMap();
+ Map<Token, String> mapString = new HashMap<Token,
String>(mapInetAddress.size());
+ for (Map.Entry<Token, InetAddress> entry : mapInetAddress.entrySet())
+ {
+ mapString.put(entry.getKey(), entry.getValue().getHostAddress());
+ }
+ return mapString;
+ }
+
/**
* Construct the range to endpoint mapping based on the true view
* of the world.
Modified:
cassandra/trunk/src/java/org/apache/cassandra/service/StorageServiceMBean.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/service/StorageServiceMBean.java?rev=981673&r1=981672&r2=981673&view=diff
==============================================================================
---
cassandra/trunk/src/java/org/apache/cassandra/service/StorageServiceMBean.java
(original)
+++
cassandra/trunk/src/java/org/apache/cassandra/service/StorageServiceMBean.java
Mon Aug 2 20:19:42 2010
@@ -28,6 +28,7 @@ import java.util.concurrent.TimeoutExcep
import org.apache.cassandra.config.ConfigurationException;
import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
import org.apache.cassandra.thrift.UnavailableException;
import java.net.InetAddress;
@@ -94,6 +95,14 @@ public interface StorageServiceMBean
public Map<Range, List<String>> getPendingRangeToEndpointMap(String
keyspace);
/**
+ * Retrieve a map of tokens to endpoints, including the bootstrapping
+ * ones.
+ *
+ * @return a map of tokens to endpoints
+ */
+ public Map<Token, String> getTokenToEndpointMap();
+
+ /**
* Numeric load value.
*/
public double getLoad();
Modified:
cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java?rev=981673&r1=981672&r2=981673&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java
Mon Aug 2 20:19:42 2010
@@ -666,7 +666,7 @@ public class CassandraServer implements
public List<TokenRange> describe_ring(String keyspace)throws
InvalidRequestException
{
- if (!DatabaseDescriptor.getNonSystemTables().contains(keyspace))
+ if (keyspace == null ||
!DatabaseDescriptor.getNonSystemTables().contains(keyspace))
throw new InvalidRequestException("There is no ring for the
keyspace: " + keyspace);
List<TokenRange> ranges = new ArrayList<TokenRange>();
Token.TokenFactory tf =
StorageService.getPartitioner().getTokenFactory();
Modified: cassandra/trunk/src/java/org/apache/cassandra/tools/NodeCmd.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/tools/NodeCmd.java?rev=981673&r1=981672&r2=981673&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/tools/NodeCmd.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/tools/NodeCmd.java Mon Aug 2
20:19:42 2010
@@ -40,7 +40,7 @@ import org.apache.cassandra.cache.JMXIns
import org.apache.cassandra.concurrent.IExecutorMBean;
import org.apache.cassandra.db.ColumnFamilyStoreMBean;
import org.apache.cassandra.db.CompactionManager;
-import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
import org.apache.commons.cli.*;
@@ -91,22 +91,10 @@ public class NodeCmd {
*/
public void printRing(PrintStream outs)
{
-
- Map<Range, List<String>> rangesToIterate = new HashMap<Range,
List<String>>();
- try
- {
- rangesToIterate.putAll(probe.getPendingRangeToEndpoingMap(null));
- rangesToIterate.putAll(probe.getRangeToEndpointMap(null));
- }
- catch (IllegalStateException ise)
- {
- outs.println(String.format("Ring information unavailable:
%s",ise.getMessage()));
- return;
- }
-
-
- List<Range> ranges = new ArrayList<Range>(rangesToIterate.keySet());
- Collections.sort(ranges);
+ Map<Token, String> tokenToEndpoint = probe.getTokenToEndpointMap();
+ List<Token> sortedTokens = new
ArrayList<Token>(tokenToEndpoint.keySet());
+ Collections.sort(sortedTokens);
+
Set<String> liveNodes = probe.getLiveNodes();
Set<String> deadNodes = probe.getUnreachableNodes();
Set<String> joiningNodes = probe.getJoiningNodes();
@@ -122,13 +110,11 @@ public class NodeCmd {
// show pre-wrap token twice so you can always read a node's range as
// (previous line token, current line token]
- if (ranges.size() > 1)
- outs.println(String.format("%-14s%-11s%-14s%-43s", "", "", "",
ranges.get(0).left));
+ if (sortedTokens.size() > 1)
+ outs.println(String.format("%-14s%-11s%-14s%-43s", "", "", "",
sortedTokens.get(sortedTokens.size() - 1)));
- for (Range range : ranges) {
- List<String> endpoints = rangesToIterate.get(range);
-
- String primaryEndpoint = endpoints.get(0);
+ for (Token token : sortedTokens) {
+ String primaryEndpoint = tokenToEndpoint.get(token);
outs.print(String.format("%-16s", primaryEndpoint));
String status =
@@ -145,7 +131,7 @@ public class NodeCmd {
outs.print(String.format("%-16s",
loadMap.containsKey(primaryEndpoint) ? loadMap.get(primaryEndpoint) : "?"));
- outs.print(String.format("%-44s", range.right));
+ outs.print(String.format("%-44s", token));
outs.println();
}
Modified: cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java?rev=981673&r1=981672&r2=981673&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java Mon Aug
2 20:19:42 2010
@@ -44,6 +44,7 @@ import org.apache.cassandra.db.ColumnFam
import org.apache.cassandra.db.CompactionManager;
import org.apache.cassandra.db.CompactionManagerMBean;
import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
import org.apache.cassandra.service.StorageServiceMBean;
import org.apache.cassandra.streaming.StreamingService;
import org.apache.cassandra.streaming.StreamingServiceMBean;
@@ -152,14 +153,9 @@ public class NodeProbe
ssProxy.drain();
}
- public Map<Range, List<String>> getRangeToEndpointMap(String tableName)
+ public Map<Token, String> getTokenToEndpointMap()
{
- return ssProxy.getRangeToEndpointMap(tableName);
- }
-
- public Map<Range, List<String>> getPendingRangeToEndpoingMap(String
tableName)
- {
- return ssProxy.getPendingRangeToEndpointMap(tableName);
+ return ssProxy.getTokenToEndpointMap();
}
public Set<String> getLiveNodes()