Author: brandonwilliams
Date: Tue Jan 25 16:49:31 2011
New Revision: 1063345

URL: http://svn.apache.org/viewvc?rev=1063345&view=rev
Log:
Allow removetoken to be called on nodes already leaving the ring.
Patch by brandonwilliams, reviewed by jbellis for CASSANDRA-1900

Modified:
    
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageService.java

Modified: 
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageService.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageService.java?rev=1063345&r1=1063344&r2=1063345&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageService.java
 (original)
+++ 
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageService.java
 Tue Jan 25 16:49:31 2011
@@ -169,7 +169,7 @@ public class StorageService implements I
     /* This abstraction maintains the token/endpoint metadata information */
     private TokenMetadata tokenMetadata_ = new TokenMetadata();
 
-    private Set<InetAddress> replicatingNodes;
+    private Set<InetAddress> replicatingNodes = new 
Collections.synchronizedSet(new HashSet<InetAddress>());
     private InetAddress removingNode;
 
     /* Are we starting this node in bootstrap mode? */
@@ -734,9 +734,10 @@ public class StorageService implements I
     }
 
     /**
-     * Handle node being actively removed from the ring.
+     * Handle notification that a node being actively removed from the ring 
via 'removetoken'
      *
      * @param endpoint node
+     * @param state either REMOVED_TOKEN (node is gone) or REMOVING_TOKEN 
(replicas need to be restored)
      */
     private void handleStateRemoving(InetAddress endpoint, Token removeToken, 
String state)
     {
@@ -1676,17 +1677,28 @@ public class StorageService implements I
 
     /**
      * Force a remove operation to complete. This may be necessary if a remove 
operation
-     * blocks forever due to node/stream failure.
+     * blocks forever due to node/stream failure. removeToken() must be called
+     * first, this is a last resort measure.  No further attempt will be made 
to restore replicas.
      */
     public void forceRemoveCompletion()
     {
         if (!replicatingNodes.isEmpty())
+        {
             logger_.warn("Removal not confirmed for for " + 
StringUtils.join(this.replicatingNodes, ","));
-        replicatingNodes.clear();
+            replicatingNodes.clear();
+        }
+        else
+        {
+            throw new UnsupportedOperationException("No tokens to force 
removal on, call 'removetoken' first");
+        }
     }
 
     /**
-     * Remove a node that has died.
+     * Remove a node that has died, attempting to restore the replica count.
+     * If the node is alive, decommission should be attempted.  If decommission
+     * fails, then removeToken should be called.  If we fail while trying to
+     * restore the replica count, finally forceRemoveCompleteion should be
+     * called to forcibly remove the node without regard to replica count.
      *
      * @param tokenString token for the node
      */
@@ -1707,14 +1719,13 @@ public class StorageService implements I
             throw new UnsupportedOperationException("Node " + endpoint + " is 
alive and owns this token. Use decommission command to remove it from the 
ring");
 
         // A leaving endpoint that is dead is already being removed.
-        if (tokenMetadata_.isLeaving(endpoint)) 
-            throw new UnsupportedOperationException("Node " + endpoint + " is 
already being removed.");
+        if (tokenMetadata_.isLeaving(endpoint))
+            logger_.warn("Node " + endpoint + " is already being removed, 
continuing removal anyway");
 
         if (replicatingNodes != null)
-            throw new UnsupportedOperationException("This node is already 
processing a removal. Wait for it to complete.");
+            throw new UnsupportedOperationException("This node is already 
processing a removal. Wait for it to complete, or use 'removetoken force' if 
this has failed.");
 
         // Find the endpoints that are going to become responsible for data
-        replicatingNodes = Collections.synchronizedSet(new 
HashSet<InetAddress>());
         for (String table : DatabaseDescriptor.getNonSystemTables())
         {
             // if the replication factor is 1 the data is lost so we shouldn't 
wait for confirmation


Reply via email to