You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by gd...@apache.org on 2010/07/20 19:11:47 UTC

svn commit: r965913 - in /cassandra/trunk: CHANGES.txt src/java/org/apache/cassandra/net/MessagingService.java src/java/org/apache/cassandra/net/OutboundTcpConnection.java src/java/org/apache/cassandra/service/StorageService.java

Author: gdusbabek
Date: Tue Jul 20 17:11:46 2010
New Revision: 965913

URL: http://svn.apache.org/viewvc?rev=965913&view=rev
Log:
failure detection wasn't closing sockets. patch by gdusbabek, reviewed by jbellis. CASSANDRA-1221

Modified:
    cassandra/trunk/CHANGES.txt
    cassandra/trunk/src/java/org/apache/cassandra/net/MessagingService.java
    cassandra/trunk/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
    cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java

Modified: cassandra/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/cassandra/trunk/CHANGES.txt?rev=965913&r1=965912&r2=965913&view=diff
==============================================================================
--- cassandra/trunk/CHANGES.txt (original)
+++ cassandra/trunk/CHANGES.txt Tue Jul 20 17:11:46 2010
@@ -61,6 +61,7 @@ dev
  * log errors in gossip instead of re-throwing (CASSANDRA-1289)
  * avoid aborting commitlog replay prematurely if a flushed-but-
    not-removed commitlog segment is encountered (CASSANDRA-1297)
+ * failure detection wasn't closing command sockets (CASSANDRA-1221)
 
 
 0.6.3

Modified: cassandra/trunk/src/java/org/apache/cassandra/net/MessagingService.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/net/MessagingService.java?rev=965913&r1=965912&r2=965913&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/net/MessagingService.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/net/MessagingService.java Tue Jul 20 17:11:46 2010
@@ -55,7 +55,7 @@ import java.util.concurrent.LinkedBlocki
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
-public class MessagingService implements IFailureDetectionEventListener
+public class MessagingService
 {
     private static int version_ = 1;
     //TODO: make this parameter dynamic somehow.  Not sure if config is appropriate.
@@ -142,7 +142,7 @@ public class MessagingService implements
         return result;
     }
 
-    /** called by failure detection code to notify that housekeeping should be performed on downed sockets. */
+    /** called from gossiper when it notices a node is not responding. */
     public void convict(InetAddress ep)
     {
         logger_.trace("Resetting pool for " + ep);

Modified: cassandra/trunk/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/net/OutboundTcpConnection.java?rev=965913&r1=965912&r2=965913&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/net/OutboundTcpConnection.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/net/OutboundTcpConnection.java Tue Jul 20 17:11:46 2010
@@ -67,7 +67,7 @@ public class OutboundTcpConnection exten
         }
     }
 
-    public void closeSocket()
+    void closeSocket()
     {
         queue.clear();
         write(CLOSE_SENTINEL);
@@ -85,6 +85,9 @@ public class OutboundTcpConnection exten
             }
             if (socket != null || connect())
                 writeConnected(bb);
+            else
+                // clear out the queue, else gossip messages back up.
+                queue.clear();
         }
     }
 

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=965913&r1=965912&r2=965913&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java Tue Jul 20 17:11:46 2010
@@ -944,7 +944,10 @@ public class StorageService implements I
             deliverHints(endpoint);
     }
 
-    public void onDead(InetAddress endpoint, EndpointState state) {}
+    public void onDead(InetAddress endpoint, EndpointState state) 
+    {
+        MessagingService.instance.convict(endpoint);
+    }
 
     /** raw load value */
     public double getLoad()