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 2022/05/13 17:30:35 UTC

[cassandra] branch cassandra-4.1 updated: jvm-dtest unhandled exceptions failing the test no longer work

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

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


The following commit(s) were added to refs/heads/cassandra-4.1 by this push:
     new f451c2b028 jvm-dtest unhandled exceptions failing the test no longer work
f451c2b028 is described below

commit f451c2b02827fd1db34a14e7603a52ffe4fb188e
Author: David Capwell <dc...@apache.org>
AuthorDate: Thu May 12 16:25:04 2022 -0700

    jvm-dtest unhandled exceptions failing the test no longer work
    
    patch by David Capwell; reviewed by Benedict Elliott Smith for CASSANDRA-17549
---
 src/java/org/apache/cassandra/concurrent/ExecutionFailure.java     | 2 +-
 src/java/org/apache/cassandra/repair/RepairJob.java                | 2 +-
 .../org/apache/cassandra/distributed/impl/Instance.java            | 7 ++++---
 .../apache/cassandra/distributed/test/FailingTruncationTest.java   | 4 ++--
 .../org/apache/cassandra/distributed/test/GossipTest.java          | 2 +-
 .../test/JVMStabilityInspectorCorruptSSTableExceptionTest.java     | 2 ++
 .../distributed/test/metrics/HintsServiceMetricsTest.java          | 2 ++
 7 files changed, 13 insertions(+), 8 deletions(-)

diff --git a/src/java/org/apache/cassandra/concurrent/ExecutionFailure.java b/src/java/org/apache/cassandra/concurrent/ExecutionFailure.java
index 61bdae0578..7fa7dcbd54 100644
--- a/src/java/org/apache/cassandra/concurrent/ExecutionFailure.java
+++ b/src/java/org/apache/cassandra/concurrent/ExecutionFailure.java
@@ -61,7 +61,7 @@ public class ExecutionFailure
             {
                 Thread thread = Thread.currentThread();
                 Thread.UncaughtExceptionHandler handler = thread.getUncaughtExceptionHandler();
-                if (handler == null || handler.equals(thread.getThreadGroup()))
+                if (handler == null)
                     handler = JVMStabilityInspector::uncaughtException;
                 handler.uncaughtException(thread, t);
             }
diff --git a/src/java/org/apache/cassandra/repair/RepairJob.java b/src/java/org/apache/cassandra/repair/RepairJob.java
index 1d5a988561..aba8bd8c85 100644
--- a/src/java/org/apache/cassandra/repair/RepairJob.java
+++ b/src/java/org/apache/cassandra/repair/RepairJob.java
@@ -366,7 +366,7 @@ public class RepairJob extends AsyncFuture<RepairResult> implements Runnable
         }
         catch (NoSuchRepairSessionException e)
         {
-            throw new NoSuchRepairSessionExceptionWrapper(e);
+            return ImmediateFuture.failure(new NoSuchRepairSessionExceptionWrapper(e));
         }
     }
 
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
index d03f2f1b5e..20935907ea 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
@@ -51,6 +51,7 @@ import org.apache.cassandra.Util;
 import org.apache.cassandra.auth.AuthCache;
 import org.apache.cassandra.batchlog.Batch;
 import org.apache.cassandra.batchlog.BatchlogManager;
+import org.apache.cassandra.concurrent.ExecutorFactory;
 import org.apache.cassandra.concurrent.ExecutorLocals;
 import org.apache.cassandra.concurrent.ExecutorPlus;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
@@ -547,6 +548,9 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
         sync(() -> {
             try
             {
+                // org.apache.cassandra.distributed.impl.AbstractCluster.startup sets the exception handler for the thread
+                // so extract it to populate ExecutorFactory.Global
+                ExecutorFactory.Global.unsafeSet(new ExecutorFactory.Default(Thread.currentThread().getContextClassLoader(), null, Thread.getDefaultUncaughtExceptionHandler()));
                 if (config.has(GOSSIP))
                 {
                     // TODO: hacky
@@ -734,9 +738,6 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
     @Override
     public Future<Void> shutdown(boolean graceful)
     {
-        if (!graceful && config.has(NETWORK))
-            MessagingService.instance().shutdown(1L, MINUTES, false, true);
-
         Future<?> future = async((ExecutorService executor) -> {
             Throwable error = null;
 
diff --git a/test/distributed/org/apache/cassandra/distributed/test/FailingTruncationTest.java b/test/distributed/org/apache/cassandra/distributed/test/FailingTruncationTest.java
index 9c6818923a..ee573c4315 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/FailingTruncationTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/FailingTruncationTest.java
@@ -45,7 +45,7 @@ public class FailingTruncationTest extends TestBaseImpl
                                            .withInstanceInitializer(BBFailHelper::install)
                                            .start()))
         {
-
+            cluster.setUncaughtExceptionsFilter(t -> "truncateBlocking".equals(t.getMessage()));
             System.setProperty(BB_FAIL_HELPER_PROP, "true");
             cluster.schemaChange("create table " + KEYSPACE + ".tbl (id int primary key, t int)");
             try
@@ -78,7 +78,7 @@ public class FailingTruncationTest extends TestBaseImpl
         public static void truncateBlocking()
         {
             if (Boolean.getBoolean(BB_FAIL_HELPER_PROP))
-                throw new RuntimeException();
+                throw new RuntimeException("truncateBlocking");
         }
     }
 }
diff --git a/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java b/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java
index 3aab549652..d10c9f60fa 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java
@@ -112,7 +112,7 @@ public class GossipTest extends TestBaseImpl
                 }).accept(failAddress);
             }
 
-            cluster.get(fail).shutdown(false).get();
+            ClusterUtils.stopAbrupt(cluster, cluster.get(fail));
             cluster.get(late).startup();
             cluster.get(late).acceptsOnInstance((InetSocketAddress address) -> {
                 EndpointState ep;
diff --git a/test/distributed/org/apache/cassandra/distributed/test/JVMStabilityInspectorCorruptSSTableExceptionTest.java b/test/distributed/org/apache/cassandra/distributed/test/JVMStabilityInspectorCorruptSSTableExceptionTest.java
index 10e62944ac..1bf2d22d1e 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/JVMStabilityInspectorCorruptSSTableExceptionTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/JVMStabilityInspectorCorruptSSTableExceptionTest.java
@@ -48,6 +48,7 @@ import org.apache.cassandra.io.sstable.format.SSTableReadsListener;
 import org.apache.cassandra.io.util.FileDataInput;
 import org.apache.cassandra.service.CassandraDaemon;
 import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.Throwables;
 
 import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
 import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
@@ -72,6 +73,7 @@ public class JVMStabilityInspectorCorruptSSTableExceptionTest extends TestBaseIm
         String table = policy.name();
         try (final Cluster cluster = init(getCluster(policy).start()))
         {
+            cluster.setUncaughtExceptionsFilter(t -> Throwables.anyCauseMatches(t, t2 -> t2.getClass().getCanonicalName().equals(CorruptSSTableException.class.getCanonicalName())));
             IInvokableInstance node = cluster.get(1);
             boolean[] setup = node.callOnInstance(() -> {
                 CassandraDaemon instanceForTesting = CassandraDaemon.getInstanceForTesting();
diff --git a/test/distributed/org/apache/cassandra/distributed/test/metrics/HintsServiceMetricsTest.java b/test/distributed/org/apache/cassandra/distributed/test/metrics/HintsServiceMetricsTest.java
index 730cc450ec..0fc9bff174 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/metrics/HintsServiceMetricsTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/metrics/HintsServiceMetricsTest.java
@@ -69,6 +69,7 @@ public class HintsServiceMetricsTest extends TestBaseImpl
                                         .withInstanceInitializer(FailHints::install)
                                         .start())
         {
+            cluster.setUncaughtExceptionsFilter(t -> "Injected failure".equals(t.getMessage()));
             // setup a message filter to drop some of the hint request messages from node1
             AtomicInteger hintsNode2 = new AtomicInteger();
             AtomicInteger hintsNode3 = new AtomicInteger();
@@ -152,6 +153,7 @@ public class HintsServiceMetricsTest extends TestBaseImpl
         await().atMost(5, MINUTES)
                .pollDelay(0, SECONDS)
                .pollInterval(1, SECONDS)
+               .dontCatchUncaughtExceptions()
                .untilAsserted(assertion);
     }
 


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