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 2011/04/28 15:36:07 UTC

svn commit: r1097455 - in /cassandra/branches/cassandra-0.7: CHANGES.txt src/java/org/apache/cassandra/service/DatacenterReadCallback.java src/java/org/apache/cassandra/service/ReadCallback.java

Author: jbellis
Date: Thu Apr 28 13:36:07 2011
New Revision: 1097455

URL: http://svn.apache.org/viewvc?rev=1097455&view=rev
Log:
trigger read repair correctly forLOCAL_QUORUM reads
patch by jbellis; reviewed by slebresne for CASSANDRA-2556

Modified:
    cassandra/branches/cassandra-0.7/CHANGES.txt
    cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/DatacenterReadCallback.java
    cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/ReadCallback.java

Modified: cassandra/branches/cassandra-0.7/CHANGES.txt
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/CHANGES.txt?rev=1097455&r1=1097454&r2=1097455&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/CHANGES.txt (original)
+++ cassandra/branches/cassandra-0.7/CHANGES.txt Thu Apr 28 13:36:07 2011
@@ -3,6 +3,7 @@
  * move gossip heartbeat back to its own thread (CASSANDRA-2554)
  * fix incorrect use of NBHM.size in ReadCallback that could cause
    reads to time out even when responses were received (CASSAMDRA-2552)
+ * trigger read repair correctly for LOCAL_QUORUM reads (CASSANDRA-2556)
 
 
 0.7.5

Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/DatacenterReadCallback.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/DatacenterReadCallback.java?rev=1097455&r1=1097454&r2=1097455&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/DatacenterReadCallback.java (original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/DatacenterReadCallback.java Thu Apr 28 13:36:07 2011
@@ -48,31 +48,17 @@ public class DatacenterReadCallback<T> e
     }
 
     @Override
-    public void response(Message message)
+    protected boolean waitingFor(Message message)
     {
-        resolver.preprocess(message);
-
-        int n = localdc.equals(snitch.getDatacenter(message.getFrom()))
-              ? received.incrementAndGet()
-              : received.get();
-
-        if (n == blockfor && resolver.isDataPresent())
-        {
-            condition.signal();
-            maybeResolveForRepair();
-        }
+        return localdc.equals(snitch.getDatacenter(message.getFrom()));
     }
-    
+
     @Override
-    public void response(ReadResponse result)
+    protected boolean waitingFor(ReadResponse response)
     {
-        ((RowDigestResolver) resolver).injectPreProcessed(result);
-
-        if (received.incrementAndGet() == blockfor && resolver.isDataPresent())
-        {
-            condition.signal();
-            maybeResolveForRepair();
-        }
+        // cheat and leverage our knowledge that a local read is the only way the ReadResponse
+        // version of this method gets called
+        return true;
     }
     
     @Override

Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/ReadCallback.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/ReadCallback.java?rev=1097455&r1=1097454&r2=1097455&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/ReadCallback.java (original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/ReadCallback.java Thu Apr 28 13:36:07 2011
@@ -128,17 +128,40 @@ public class ReadCallback<T> implements 
     public void response(Message message)
     {
         resolver.preprocess(message);
-        if (received.incrementAndGet() >= blockfor && resolver.isDataPresent())
+        int n = waitingFor(message)
+              ? received.incrementAndGet()
+              : received.get();
+        if (n >= blockfor && resolver.isDataPresent())
         {
             condition.signal();
             maybeResolveForRepair();
         }
     }
 
+    /**
+     * @return true if the message counts towards the blockfor threshold
+     * TODO turn the Message into a response so we don't need two versions of this method
+     */
+    protected boolean waitingFor(Message message)
+    {
+        return true;
+    }
+
+    /**
+     * @return true if the response counts towards the blockfor threshold
+     */
+    protected boolean waitingFor(ReadResponse response)
+    {
+        return true;
+    }
+
     public void response(ReadResponse result)
     {
         ((RowDigestResolver) resolver).injectPreProcessed(result);
-        if (received.incrementAndGet() >= blockfor && resolver.isDataPresent())
+        int n = waitingFor(result)
+              ? received.incrementAndGet()
+              : received.get();
+        if (n >= blockfor && resolver.isDataPresent())
         {
             condition.signal();
             maybeResolveForRepair();