You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ma...@apache.org on 2022/07/14 16:43:52 UTC

[cassandra] branch cassandra-4.0 updated: Clean up ScheduledExecutors, CommitLog, and MessagingService shutdown for in-JVM dtests

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

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


The following commit(s) were added to refs/heads/cassandra-4.0 by this push:
     new d51f90201b Clean up ScheduledExecutors, CommitLog, and MessagingService shutdown for in-JVM dtests
d51f90201b is described below

commit d51f90201b2d127166612384a75b4a49da9776d1
Author: Caleb Rackliffe <ca...@gmail.com>
AuthorDate: Tue Jul 12 12:06:56 2022 -0500

    Clean up ScheduledExecutors, CommitLog, and MessagingService shutdown for in-JVM dtests
    
    patch by Caleb Rackliffe; reviewed by David Capwell for CASSANDRA-17731
---
 CHANGES.txt                                            |  1 +
 .../apache/cassandra/distributed/impl/Instance.java    | 18 ++++++++++--------
 .../cassandra/distributed/test/RepairErrorsTest.java   |  4 +---
 3 files changed, 12 insertions(+), 11 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 8f8dbf20da..20c9b4d560 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0.5
+ * Clean up ScheduledExecutors, CommitLog, and MessagingService shutdown for in-JVM dtests (CASSANDRA-17731)
  * Utilise BTree improvements to reduce garbage and improve throughput (CASSANDRA-15511)
  * Make sure existing delayed tasks in StreamTransferTask cannot prevent clean shutdown (CASSANDRA-17706)
  * SSL storage port in sstableloader is deprecated (CASSANDRA-17602)
diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
index 97dd45ec77..3155c063e0 100644
--- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
+++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java
@@ -787,27 +787,29 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance
                                 () -> Ref.shutdownReferenceReaper(1L, MINUTES),
                                 () -> Memtable.MEMORY_POOL.shutdownAndWait(1L, MINUTES),
                                 () -> DiagnosticSnapshotService.instance.shutdownAndWait(1L, MINUTES),
-                                () -> ScheduledExecutors.shutdownAndWait(1L, MINUTES),
                                 () -> SSTableReader.shutdownBlocking(1L, MINUTES),
                                 () -> shutdownAndWait(Collections.singletonList(ActiveRepairService.repairCommandExecutor()))
             );
 
-            error = parallelRun(error, executor, () -> ScheduledExecutors.shutdownAndWait(1L, MINUTES));
-
             internodeMessagingStarted = false;
             error = parallelRun(error, executor,
-                                CommitLog.instance::shutdownBlocking,
                                 // can only shutdown message once, so if the test shutsdown an instance, then ignore the failure
                                 (IgnoreThrowingRunnable) () -> MessagingService.instance().shutdown(1L, MINUTES, false, true)
             );
+
             error = parallelRun(error, executor,
-                                () -> GlobalEventExecutor.INSTANCE.awaitInactivity(1l, MINUTES),
+                                () -> GlobalEventExecutor.INSTANCE.awaitInactivity(1L, MINUTES),
                                 () -> Stage.shutdownAndWait(1L, MINUTES),
                                 () -> SharedExecutorPool.SHARED.shutdownAndWait(1L, MINUTES)
             );
-            error = parallelRun(error, executor,
-                                () -> shutdownAndWait(Collections.singletonList(JMXBroadcastExecutor.executor))
-            );
+
+            // CommitLog must shut down after Stage, or threads from the latter may attempt to use the former.
+            // (ex. A Mutation stage thread may attempt to add a mutation to the CommitLog.)
+            error = parallelRun(error, executor, CommitLog.instance::shutdownBlocking);
+            error = parallelRun(error, executor, () -> shutdownAndWait(Collections.singletonList(JMXBroadcastExecutor.executor)));
+            
+            // ScheduledExecutors shuts down after MessagingService, as MessagingService may issue tasks to it.
+            error = parallelRun(error, executor, () -> ScheduledExecutors.shutdownAndWait(1L, MINUTES));
 
             Throwables.maybeFail(error);
         }).apply(isolatedExecutor);
diff --git a/test/distributed/org/apache/cassandra/distributed/test/RepairErrorsTest.java b/test/distributed/org/apache/cassandra/distributed/test/RepairErrorsTest.java
index 37c9171542..b3de7db94c 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/RepairErrorsTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/RepairErrorsTest.java
@@ -145,9 +145,7 @@ public class RepairErrorsTest extends TestBaseImpl
     @SuppressWarnings("Convert2MethodRef")
     private void assertNoActiveRepairSessions(IInvokableInstance instance)
     {
-        // Make sure we've cleaned up sessions and parent sessions:
-        Integer parents = instance.callOnInstance(() -> ActiveRepairService.instance.parentRepairSessionCount());
-        assertEquals(0, parents.intValue());
+        // Make sure we've cleaned up local sessions:
         Integer sessions = instance.callOnInstance(() -> ActiveRepairService.instance.sessionCount());
         assertEquals(0, sessions.intValue());
     }


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