You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2010/06/25 00:12:43 UTC
svn commit: r957734 - in /cassandra/trunk: ./
src/java/org/apache/cassandra/service/ src/java/org/apache/cassandra/tools/
Author: jbellis
Date: Thu Jun 24 22:12:43 2010
New Revision: 957734
URL: http://svn.apache.org/viewvc?rev=957734&view=rev
Log:
add joining/leaving status to nodetool ring. patch by mdennis; reviewed by jbellis for CASSANDRA-1115
Modified:
cassandra/trunk/CHANGES.txt
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/tools/NodeCmd.java
cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java
Modified: cassandra/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/cassandra/trunk/CHANGES.txt?rev=957734&r1=957733&r2=957734&view=diff
==============================================================================
--- cassandra/trunk/CHANGES.txt (original)
+++ cassandra/trunk/CHANGES.txt Thu Jun 24 22:12:43 2010
@@ -35,6 +35,7 @@ dev
* avoid allocating a new byte[] for each mutation on replay (CASSANDRA-1219)
* revise HH schema to be per-endpoint (CASSANDRA-1142)
* remove gossip message size limit (CASSANDRA-1138)
+ * add joining/leaving status to nodetool ring (CASSANDRA-1115)
0.6.3
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=957734&r1=957733&r2=957734&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java Thu Jun 24 22:12:43 2010
@@ -495,6 +495,22 @@ public class StorageService implements I
return map;
}
+ public Map<Range, List<String>> getPendingRangeToEndpointMap(String keyspace)
+ {
+ // some people just want to get a visual representation of things. Allow null and set it to the first
+ // non-system table.
+ if (keyspace == null)
+ keyspace = DatabaseDescriptor.getNonSystemTables().get(0);
+
+ Map<Range, List<String>> map = new HashMap<Range, List<String>>();
+ for (Map.Entry<Range, Collection<InetAddress>> entry : tokenMetadata_.getPendingRanges(keyspace).entrySet())
+ {
+ List<InetAddress> l = new ArrayList<InetAddress>(entry.getValue());
+ map.put(entry.getKey(), stringify(l));
+ }
+ return map;
+ }
+
public Map<Range, List<InetAddress>> getRangeToAddressMap(String keyspace)
{
List<Range> ranges = getAllRanges(tokenMetadata_.sortedTokens());
@@ -981,6 +997,16 @@ public class StorageService implements I
return getLocalToken().toString();
}
+ public Set<String> getLeavingNodes()
+ {
+ return stringify(tokenMetadata_.getLeavingEndpoints());
+ }
+
+ public Set<String> getJoiningNodes()
+ {
+ return stringify(tokenMetadata_.getBootstrapTokens().values());
+ }
+
public Set<String> getLiveNodes()
{
return stringify(Gossiper.instance.getLiveMembers());
@@ -991,7 +1017,7 @@ public class StorageService implements I
return stringify(Gossiper.instance.getUnreachableMembers());
}
- private Set<String> stringify(Set<InetAddress> endpoints)
+ private Set<String> stringify(Collection<InetAddress> endpoints)
{
Set<String> stringEndpoints = new HashSet<String>();
for (InetAddress ep : endpoints)
@@ -1475,7 +1501,7 @@ public class StorageService implements I
if (logger_.isDebugEnabled())
logger_.debug("Leaving: old token was " + getLocalToken());
startLeaving();
- setMode("Leaving: sleeping " + RING_DELAY + " ms for pending range setup", true);
+ setMode("Leaving: sleeping " + RING_DELAY + " ms for pending range setup", true);
Thread.sleep(RING_DELAY);
Runnable finishMoving = new WrappedRunnable()
@@ -1483,11 +1509,11 @@ public class StorageService implements I
public void runMayThrow() throws IOException
{
Token bootstrapToken = token;
- if (bootstrapToken == null)
- {
- StorageLoadBalancer.instance.waitForLoadInfo();
- bootstrapToken = BootStrapper.getBalancedToken(tokenMetadata_, StorageLoadBalancer.instance.getLoadInfo());
- }
+ if (bootstrapToken == null)
+ {
+ StorageLoadBalancer.instance.waitForLoadInfo();
+ bootstrapToken = BootStrapper.getBalancedToken(tokenMetadata_, StorageLoadBalancer.instance.getLoadInfo());
+ }
logger_.info("re-bootstrapping to new token {}", bootstrapToken);
startBootstrap(bootstrapToken);
}
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=957734&r1=957733&r2=957734&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/service/StorageServiceMBean.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/service/StorageServiceMBean.java Thu Jun 24 22:12:43 2010
@@ -52,6 +52,20 @@ public interface StorageServiceMBean
public Set<String> getUnreachableNodes();
/**
+ * Retrieve the list of nodes currently bootstrapping into the ring.
+ *
+ * @return set of IP addresses, as Strings
+ */
+ public Set<String> getJoiningNodes();
+
+ /**
+ * Retrieve the list of nodes currently leaving the ring.
+ *
+ * @return set of IP addresses, as Strings
+ */
+ public Set<String> getLeavingNodes();
+
+ /**
* Fetch a string representation of the token.
*
* @return a string token
@@ -67,6 +81,13 @@ public interface StorageServiceMBean
public Map<Range, List<String>> getRangeToEndpointMap(String keyspace);
/**
+ * Retrieve a map of pending ranges to endpoints that describe the ring topology
+ * @param keyspace the keyspace to get the pending range map for.
+ * @return a map of pending ranges to endpoints
+ */
+ public Map<Range, List<String>> getPendingRangeToEndpointMap(String keyspace);
+
+ /**
* Numeric load value.
*/
public double getLoad();
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=957734&r1=957733&r2=957734&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/tools/NodeCmd.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/tools/NodeCmd.java Thu Jun 24 22:12:43 2010
@@ -43,7 +43,6 @@ import org.apache.cassandra.db.Compactio
import org.apache.cassandra.dht.Range;
import org.apache.commons.cli.*;
-import org.apache.commons.lang.StringUtils;
public class NodeCmd {
private static final String HOST_OPT_LONG = "host";
@@ -93,67 +92,58 @@ public class NodeCmd {
public void printRing(PrintStream outs)
{
Map<Range, List<String>> rangeMap = probe.getRangeToEndpointMap(null);
- List<Range> ranges = new ArrayList<Range>(rangeMap.keySet());
+ Map<Range, List<String>> pendingRangeMap = probe.getPendingRangeToEndpoingMap(null);
+ Map<Range, List<String>> rangesToIterate = new HashMap<Range, List<String>>();
+
+ rangesToIterate.putAll(pendingRangeMap);
+ rangesToIterate.putAll(rangeMap);
+
+ List<Range> ranges = new ArrayList<Range>(rangesToIterate.keySet());
Collections.sort(ranges);
Set<String> liveNodes = probe.getLiveNodes();
Set<String> deadNodes = probe.getUnreachableNodes();
+ Set<String> joiningNodes = probe.getJoiningNodes();
+ Set<String> leavingNodes = probe.getLeavingNodes();
Map<String, String> loadMap = probe.getLoadMap();
- // Print range-to-endpoint mapping
- int counter = 0;
- outs.print(String.format("%-14s", "Address"));
- outs.print(String.format("%-11s", "Status"));
- outs.print(String.format("%-14s", "Load"));
- outs.print(String.format("%-43s", "Range"));
- outs.println("Ring");
- // emphasize that we're showing the right part of each range
+ outs.print(String.format("%-16s", "Address"));
+ outs.print(String.format("%-7s", "Status"));
+ outs.print(String.format("%-8s", "State"));
+ outs.print(String.format("%-16s", "Load"));
+ outs.print(String.format("%-44s", "Token"));
+ outs.println();
+
+ // 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));
- }
- // normal range & node info
+
for (Range range : ranges) {
- List<String> endpoints = rangeMap.get(range);
+ List<String> endpoints = rangesToIterate.get(range);
+
String primaryEndpoint = endpoints.get(0);
+ outs.print(String.format("%-16s", primaryEndpoint));
- outs.print(String.format("%-14s", primaryEndpoint));
-
- String status = liveNodes.contains(primaryEndpoint)
- ? "Up"
- : deadNodes.contains(primaryEndpoint)
- ? "Down"
- : "?";
- outs.print(String.format("%-11s", status));
+ String status =
+ liveNodes.contains(primaryEndpoint) ? "Up" :
+ deadNodes.contains(primaryEndpoint) ? "Down" :
+ "?";
+ outs.print(String.format("%-7s", status));
+
+ String state =
+ joiningNodes.contains(primaryEndpoint) ? "Joining" :
+ leavingNodes.contains(primaryEndpoint) ? "Leaving" :
+ "Normal";
+ outs.print(String.format("%-8s", state));
- String load = loadMap.containsKey(primaryEndpoint) ? loadMap.get(primaryEndpoint) : "?";
- outs.print(String.format("%-14s", load));
+ outs.print(String.format("%-16s", loadMap.containsKey(primaryEndpoint) ? loadMap.get(primaryEndpoint) : "?"));
- outs.print(String.format("%-43s", range.right));
+ outs.print(String.format("%-44s", range.right));
- String asciiRingArt;
- if (counter == 0)
- {
- asciiRingArt = "|<--|";
- }
- else if (counter == (rangeMap.size() - 1))
- {
- asciiRingArt = "|-->|";
- }
- else
- {
- if ((rangeMap.size() > 4) && ((counter % 2) == 0))
- asciiRingArt = "v |";
- else if ((rangeMap.size() > 4) && ((counter % 2) != 0))
- asciiRingArt = "| ^";
- else
- asciiRingArt = "| |";
- }
- outs.println(asciiRingArt);
-
- counter++;
+ outs.println();
}
}
-
+
public void printThreadPoolStats(PrintStream outs)
{
outs.print(String.format("%-25s", "Pool Name"));
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=957734&r1=957733&r2=957734&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java Thu Jun 24 22:12:43 2010
@@ -159,79 +159,25 @@ public class NodeProbe
{
return ssProxy.getRangeToEndpointMap(tableName);
}
-
+
+ public Map<Range, List<String>> getPendingRangeToEndpoingMap(String tableName)
+ {
+ return ssProxy.getPendingRangeToEndpointMap(tableName);
+ }
+
public Set<String> getLiveNodes()
{
return ssProxy.getLiveNodes();
}
- /**
- * Write a textual representation of the Cassandra ring.
- *
- * @param outs the stream to write to
- */
- public void printRing(PrintStream outs)
+ public Set<String> getJoiningNodes()
{
- Map<Range, List<String>> rangeMap = ssProxy.getRangeToEndpointMap(null);
- List<Range> ranges = new ArrayList<Range>(rangeMap.keySet());
- Collections.sort(ranges);
- Set<String> liveNodes = ssProxy.getLiveNodes();
- Set<String> deadNodes = ssProxy.getUnreachableNodes();
- Map<String, String> loadMap = ssProxy.getLoadMap();
-
- // Print range-to-endpoint mapping
- int counter = 0;
- outs.print(String.format("%-14s", "Address"));
- outs.print(String.format("%-11s", "Status"));
- outs.print(String.format("%-14s", "Load"));
- outs.print(String.format("%-43s", "Range"));
- outs.println("Ring");
- // emphasize that we're showing the right part of each range
- if (ranges.size() > 1)
- {
- outs.println(String.format("%-14s%-11s%-14s%-43s", "", "", "", ranges.get(0).left));
- }
- // normal range & node info
- for (Range range : ranges) {
- List<String> endpoints = rangeMap.get(range);
- String primaryEndpoint = endpoints.get(0);
-
- outs.print(String.format("%-14s", primaryEndpoint));
-
- String status = liveNodes.contains(primaryEndpoint)
- ? "Up"
- : deadNodes.contains(primaryEndpoint)
- ? "Down"
- : "?";
- outs.print(String.format("%-11s", status));
-
- String load = loadMap.containsKey(primaryEndpoint) ? loadMap.get(primaryEndpoint) : "?";
- outs.print(String.format("%-14s", load));
-
- outs.print(String.format("%-43s", range.right));
-
- String asciiRingArt;
- if (counter == 0)
- {
- asciiRingArt = "|<--|";
- }
- else if (counter == (rangeMap.size() - 1))
- {
- asciiRingArt = "|-->|";
- }
- else
- {
- if ((rangeMap.size() > 4) && ((counter % 2) == 0))
- asciiRingArt = "v |";
- else if ((rangeMap.size() > 4) && ((counter % 2) != 0))
- asciiRingArt = "| ^";
- else
- asciiRingArt = "| |";
- }
- outs.println(asciiRingArt);
-
- counter++;
- }
+ return ssProxy.getJoiningNodes();
+ }
+
+ public Set<String> getLeavingNodes()
+ {
+ return ssProxy.getLeavingNodes();
}
public Set<String> getUnreachableNodes()