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/04/13 18:48:04 UTC
[cassandra] branch trunk updated: Periodic failures in
*RepairCoordinator*Test caused by race condition with nodetool repair
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 5cbdb2e Periodic failures in *RepairCoordinator*Test caused by race condition with nodetool repair
5cbdb2e is described below
commit 5cbdb2e58e870535af61204898a1e2bbf6cb5f64
Author: David Capwell <dc...@apache.org>
AuthorDate: Tue Apr 13 09:46:49 2021 -0700
Periodic failures in *RepairCoordinator*Test caused by race condition with nodetool repair
patch by David Capwell; reviewed by Marcus Eriksson for CASSANDRA-16585
---
CHANGES.txt | 1 +
src/java/org/apache/cassandra/repair/RepairRunnable.java | 8 +++++++-
.../distributed/test/RepairCoordinatorNeighbourDown.java | 14 ++------------
3 files changed, 10 insertions(+), 13 deletions(-)
diff --git a/CHANGES.txt b/CHANGES.txt
index c22d6a6..8551edf 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
4.0-rc1
+ * Fixed a race condition issue in nodetool repair where we poll for the error before seeing the error notification, leading to a less meaningful message (CASSANDRA-16585)
* Fix mixed cluster GROUP BY queries (CASSANDRA-16582)
* Upgrade jflex to 1.8.2 (CASSANDRA-16576)
* Binary releases no longer bundle the apidocs (javadoc) (CASSANDRA-15561)
diff --git a/src/java/org/apache/cassandra/repair/RepairRunnable.java b/src/java/org/apache/cassandra/repair/RepairRunnable.java
index 793d2f2..5ada116 100644
--- a/src/java/org/apache/cassandra/repair/RepairRunnable.java
+++ b/src/java/org/apache/cassandra/repair/RepairRunnable.java
@@ -32,6 +32,7 @@ import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
@@ -112,6 +113,7 @@ public class RepairRunnable implements Runnable, ProgressEventNotifier
private final List<ProgressListener> listeners = new ArrayList<>();
private static final AtomicInteger threadCounter = new AtomicInteger(1);
+ private final AtomicReference<Throwable> firstError = new AtomicReference<>(null);
private TraceState traceState;
@@ -178,6 +180,7 @@ public class RepairRunnable implements Runnable, ProgressEventNotifier
StorageMetrics.repairExceptions.inc();
String errorMessage = String.format("Repair command #%d failed with error %s", cmd, error.getMessage());
fireProgressEvent(new ProgressEvent(ProgressEventType.ERROR, progressCounter.get(), totalProgress, errorMessage));
+ firstError.compareAndSet(null, error);
// since this can fail, update table only after updating in-memory and notification state
maybeStoreParentRepairFailure(error);
@@ -186,7 +189,10 @@ public class RepairRunnable implements Runnable, ProgressEventNotifier
private void fail(String reason)
{
if (reason == null)
- reason = "Some repair failed";
+ {
+ Throwable error = firstError.get();
+ reason = error != null ? error.getMessage() : "Some repair failed";
+ }
String completionMessage = String.format("Repair command #%d finished with error", cmd);
// Note we rely on the first message being the reason for the failure
diff --git a/test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorNeighbourDown.java b/test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorNeighbourDown.java
index aa367d4..4228806 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorNeighbourDown.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorNeighbourDown.java
@@ -166,24 +166,14 @@ public abstract class RepairCoordinatorNeighbourDown extends RepairCoordinatorBa
NodeToolResult result = repair(1, KEYSPACE, table);
recovered.join(); // if recovery didn't happen then the results are not what are being tested, so block here first
result.asserts()
- .failure();
+ .failure()
+ .errorContains("/127.0.0.2:7012 died");
if (withNotifications)
{
result.asserts()
- .errorContains("/127.0.0.2:7012 died")
.notificationContains(NodeToolResult.ProgressEventType.ERROR, "/127.0.0.2:7012 died")
.notificationContains(NodeToolResult.ProgressEventType.COMPLETE, "finished with error");
}
- else
- {
- // Right now coordination doesn't propgate the first exception, so we only know "there exists a issue".
- // With notifications on nodetool will see the error then complete, so the cmd state (what nodetool
- // polls on) is ignored. With notifications off, the poll await fails and queries cmd state, and that
- // will have the below error.
- // NOTE: this isn't desireable, would be good to propgate
- result.asserts()
- .errorContains("Some repair failed");
- }
Assert.assertEquals(repairExceptions + 1, getRepairExceptions(CLUSTER, 1));
if (repairType != RepairType.PREVIEW)
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org