You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by dc...@apache.org on 2021/11/19 22:33:25 UTC

[cassandra] branch trunk updated: repair prepare message would produce a wrong error message if network timeout happened rather than reply wait timeout

This is an automated email from the ASF dual-hosted git repository.

dcapwell pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new dcd7b0d  repair prepare message would produce a wrong error message if network timeout happened rather than reply wait timeout
dcd7b0d is described below

commit dcd7b0d35f8d0a8a6182919a6beae7274325d818
Author: David Capwell <dc...@apache.org>
AuthorDate: Fri Nov 19 12:44:44 2021 -0800

    repair prepare message would produce a wrong error message if network timeout happened rather than reply wait timeout
    
    patch by David Capwell; reviewed by Berenguer Blasi for CASSANDRA-16992
---
 CHANGES.txt                                                    | 1 +
 src/java/org/apache/cassandra/service/ActiveRepairService.java | 6 +++++-
 2 files changed, 6 insertions(+), 1 deletion(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 316d14b..a362f78 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.1
+ * repair prepare message would produce a wrong error message if network timeout happened rather than reply wait timeout (CASSANDRA-16992)
  * Log queries that fail on timeout or unavailable errors up to once per minute by default (CASSANDRA-17159)
  * Refactor normal/preview/IR repair to standardize repair cleanup and error handling of failed RepairJobs (CASSANDRA-17069)
  * Log missing peers in StartupClusterConnectivityChecker (CASSANDRA-17130)
diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java
index fbcb745..322fd18 100644
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@ -26,6 +26,7 @@ import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import javax.management.openmbean.CompositeData;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Predicate;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -523,6 +524,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
         final CountDownLatch prepareLatch = newCountDownLatch(endpoints.size());
         final AtomicBoolean status = new AtomicBoolean(true);
         final Set<String> failedNodes = synchronizedSet(new HashSet<String>());
+        final AtomicInteger timeouts = new AtomicInteger(0);
         RequestCallback callback = new RequestCallback()
         {
             @Override
@@ -536,6 +538,8 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
             {
                 status.set(false);
                 failedNodes.add(from.toString());
+                if (failureReason == RequestFailureReason.TIMEOUT)
+                    timeouts.incrementAndGet();
                 prepareLatch.decrement();
             }
 
@@ -576,7 +580,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
         }
         try
         {
-            if (!prepareLatch.await(getRpcTimeout(MILLISECONDS), MILLISECONDS))
+            if (!prepareLatch.await(getRpcTimeout(MILLISECONDS), MILLISECONDS) || timeouts.get() > 0)
                 failRepair(parentRepairSession, "Did not get replies from all endpoints.");
         }
         catch (InterruptedException e)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org