Author: jbellis
Date: Sat Sep  4 20:34:46 2010
New Revision: 992672

URL: http://svn.apache.org/viewvc?rev=992672&view=rev
Log:
remove failed bootstrap attempt from pending ranges when gossip times it out.  
patch by jbellis; reviewed by brandonwilliams for CASSANDRA-1463

Modified:
    cassandra/branches/cassandra-0.6/CHANGES.txt
    
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/gms/FailureDetector.java
    
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/gms/Gossiper.java
    
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/gms/IEndPointStateChangeSubscriber.java
    
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/locator/TokenMetadata.java
    
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageLoadBalancer.java
    
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageService.java

Modified: cassandra/branches/cassandra-0.6/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/CHANGES.txt?rev=992672&r1=992671&r2=992672&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.6/CHANGES.txt (original)
+++ cassandra/branches/cassandra-0.6/CHANGES.txt Sat Sep  4 20:34:46 2010
@@ -7,6 +7,8 @@
    to default JVM options
  * decrease jvm heap for cassandra-cli (CASSANDRA-1446)
  * document DoConsistencyChecksBoolean option to disable Read Repair
+ * remove failed bootstrap attempt from pending ranges when gossip times
+   it out after 1h (CASSANDRA-1463)
 
 
 0.6.5

Modified: 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/gms/FailureDetector.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/gms/FailureDetector.java?rev=992672&r1=992671&r2=992672&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/gms/FailureDetector.java
 (original)
+++ 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/gms/FailureDetector.java
 Sat Sep  4 20:34:46 2010
@@ -125,14 +125,16 @@ public class FailureDetector implements 
     
     public boolean isAlive(InetAddress ep)
     {
-       /* If the endpoint in question is the local endpoint return true. */
-        InetAddress localHost = FBUtilities.getLocalAddress();
-        if (localHost.equals(ep))
+        if (ep.equals(FBUtilities.getLocalAddress()))
             return true;
 
-       /* Incoming port is assumed to be the Storage port. We need to change 
it to the control port */
         EndPointState epState = 
Gossiper.instance.getEndPointStateForEndPoint(ep);
-        return epState.isAlive();
+        // we could assert not-null, but having isAlive fail screws a node 
over so badly that
+        // it's worth being defensive here so minor bugs don't cause 
disproportionate
+        // badness.  (See CASSANDRA-1463 for an example).
+        if (epState == null)
+            logger_.error("unknown endpoint " + ep);
+        return epState != null && epState.isAlive();
     }
     
     public void report(InetAddress ep)

Modified: 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/gms/Gossiper.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/gms/Gossiper.java?rev=992672&r1=992671&r2=992672&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/gms/Gossiper.java
 (original)
+++ 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/gms/Gossiper.java
 Sat Sep  4 20:34:46 2010
@@ -216,6 +216,10 @@ public class Gossiper implements IFailur
      */
     public void removeEndPoint(InetAddress endpoint)
     {
+        // do subscribers first so anything in the subscriber that depends on 
gossiper state won't get confused
+        for (IEndPointStateChangeSubscriber subscriber : subscribers_)
+            subscriber.onRemove(endpoint);
+
         liveEndpoints_.remove(endpoint);
         unreachableEndpoints_.remove(endpoint);
         endPointStateMap_.remove(endpoint);

Modified: 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/gms/IEndPointStateChangeSubscriber.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/gms/IEndPointStateChangeSubscriber.java?rev=992672&r1=992671&r2=992672&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/gms/IEndPointStateChangeSubscriber.java
 (original)
+++ 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/gms/IEndPointStateChangeSubscriber.java
 Sat Sep  4 20:34:46 2010
@@ -45,4 +45,6 @@ public interface IEndPointStateChangeSub
     public void onAlive(InetAddress endpoint, EndPointState state);
 
     public void onDead(InetAddress endpoint, EndPointState state);
+
+    public void onRemove(InetAddress endpoint);
 }

Modified: 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/locator/TokenMetadata.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/locator/TokenMetadata.java?rev=992672&r1=992671&r2=992672&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/locator/TokenMetadata.java
 (original)
+++ 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/locator/TokenMetadata.java
 Sat Sep  4 20:34:46 2010
@@ -172,27 +172,13 @@ public class TokenMetadata
         }
     }
 
-    public void removeLeavingEndPoint(InetAddress endpoint)
+    public void removeEndpoint(InetAddress endpoint)
     {
         assert endpoint != null;
 
         lock.writeLock().lock();
         try
         {
-            leavingEndPoints.remove(endpoint);
-        }
-        finally
-        {
-            lock.writeLock().unlock();
-        }
-    }
-
-    public void removeEndpoint(InetAddress endpoint)
-    {
-        assert tokenToEndPointMap.containsValue(endpoint);
-        lock.writeLock().lock();
-        try
-        {
             bootstrapTokens.inverse().remove(endpoint);
             tokenToEndPointMap.inverse().remove(endpoint);
             leavingEndPoints.remove(endpoint);

Modified: 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageLoadBalancer.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageLoadBalancer.java?rev=992672&r1=992671&r2=992672&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageLoadBalancer.java
 (original)
+++ 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageLoadBalancer.java
 Sat Sep  4 20:34:46 2010
@@ -223,6 +223,8 @@ public class StorageLoadBalancer impleme
 
     public void onDead(InetAddress endpoint, EndPointState state) {}
 
+    public void onRemove(InetAddress endpoint) {}
+
 /*
     private boolean isMoveable()
     {

Modified: 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageService.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageService.java?rev=992672&r1=992671&r2=992672&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageService.java
 (original)
+++ 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageService.java
 Sat Sep  4 20:34:46 2010
@@ -657,7 +657,7 @@ public class StorageService implements I
     {
         restoreReplicaCount(endPoint);
         Gossiper.instance.removeEndPoint(endPoint);
-        tokenMetadata_.removeEndpoint(endPoint);
+        // gossiper onRemove will take care of TokenMetadata
     }
 
     /**
@@ -874,7 +874,13 @@ public class StorageService implements I
             deliverHints(endpoint);
     }
 
-    public void onDead(InetAddress endpoint, EndPointState state) 
+    public void onRemove(InetAddress endpoint)
+    {
+        tokenMetadata_.removeEndpoint(endpoint);
+        calculatePendingRanges();
+    }
+
+    public void onDead(InetAddress endpoint, EndPointState state)
     {
         MessagingService.instance.convict(endpoint);
     }


Reply via email to