You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by br...@apache.org on 2011/10/22 00:21:54 UTC

svn commit: r1187578 - in /cassandra/branches/cassandra-0.8: CHANGES.txt src/java/org/apache/cassandra/gms/Gossiper.java

Author: brandonwilliams
Date: Fri Oct 21 22:21:54 2011
New Revision: 1187578

URL: http://svn.apache.org/viewvc?rev=1187578&view=rev
Log:
Prevent nodes that failed to join from being stuck in the joining state
indefinitely.
Patch by brandonwilliams, reviewed by Paul Cannon for CASSANDRA-3351

Modified:
    cassandra/branches/cassandra-0.8/CHANGES.txt
    cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/gms/Gossiper.java

Modified: cassandra/branches/cassandra-0.8/CHANGES.txt
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/CHANGES.txt?rev=1187578&r1=1187577&r2=1187578&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.8/CHANGES.txt (original)
+++ cassandra/branches/cassandra-0.8/CHANGES.txt Fri Oct 21 22:21:54 2011
@@ -29,6 +29,7 @@
  * fix assertionError during repair with ordered partitioners (CASSANDRA-3369)
  * correctly serialize key_validation_class for avro (CASSANDRA-3391)
  * don't expire counter tombstone after streaming (CASSANDRA-3394)
+ * prevent nodes that failed to join from hanging around forever (CASSANDRA-3351)
 
 
 0.8.7

Modified: cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/gms/Gossiper.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/gms/Gossiper.java?rev=1187578&r1=1187577&r2=1187578&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/gms/Gossiper.java (original)
+++ cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/gms/Gossiper.java Fri Oct 21 22:21:54 2011
@@ -541,15 +541,18 @@ public class Gossiper implements IFailur
             {
                 long duration = now - epState.getUpdateTimestamp();
 
-                if (StorageService.instance.getTokenMetadata().isMember(endpoint))
-                    epState.setHasToken(true);
                 // check if this is a fat client. fat clients are removed automatically from
                 // gosip after FatClientTimeout
                 if (!epState.hasToken() && !epState.isAlive() && !justRemovedEndpoints.containsKey(endpoint) && (duration > FatClientTimeout))
                 {
-                    logger.info("FatClient " + endpoint + " has been silent for " + FatClientTimeout + "ms, removing from gossip");
-                    removeEndpoint(endpoint); // will put it in justRemovedEndpoints to respect quarantine delay
-                    evictFromMembership(endpoint); // can get rid of the state immediately
+                    if (StorageService.instance.getTokenMetadata().isMember(endpoint))
+                        epState.setHasToken(true);
+                    else
+                    {
+                        logger.info("FatClient " + endpoint + " has been silent for " + FatClientTimeout + "ms, removing from gossip");
+                        removeEndpoint(endpoint); // will put it in justRemovedEndpoints to respect quarantine delay
+                        evictFromMembership(endpoint); // can get rid of the state immediately
+                    }
                 }
 
                 if ( !epState.isAlive() && (duration > aVeryLongTime) && (!StorageService.instance.getTokenMetadata().isMember(endpoint)))
@@ -728,7 +731,7 @@ public class Gossiper implements IFailur
      */
     private void handleMajorStateChange(InetAddress ep, EndpointState epState)
     {
-        if (epState.getApplicationState(ApplicationState.STATUS) != null && !isDeadState(epState.getApplicationState(ApplicationState.STATUS).value))
+        if (!isDeadState(epState))
         {
             if (endpointStateMap.get(ep) != null)
                 logger.info("Node {} has restarted, now UP again", ep);
@@ -743,7 +746,7 @@ public class Gossiper implements IFailur
         for (IEndpointStateChangeSubscriber subscriber : subscribers)
             subscriber.onRestart(ep, epState);
 
-        if (epState.getApplicationState(ApplicationState.STATUS) != null && !isDeadState(epState.getApplicationState(ApplicationState.STATUS).value))
+        if (!isDeadState(epState))
             markAlive(ep, epState);
         else
         {
@@ -755,8 +758,11 @@ public class Gossiper implements IFailur
             subscriber.onJoin(ep, epState);
     }
 
-    private Boolean isDeadState(String value)
+    private Boolean isDeadState(EndpointState epState)
     {
+        if (epState.getApplicationState(ApplicationState.STATUS) == null)
+            return false;
+        String value = epState.getApplicationState(ApplicationState.STATUS).value;
         String[] pieces = value.split(VersionedValue.DELIMITER_STR, -1);
         assert (pieces.length > 0);
         String state = pieces[0];