This is an automated email from the ASF dual-hosted git repository. brandonwilliams pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/cassandra.git
commit 6dad6600393f99c3d413dffefa1bf7f6ee12b79c Author: bryn <brynco...@gmail.com> AuthorDate: Mon Jun 8 11:02:42 2020 +0100 CASSANDRA-15677 Add shutdown to JMX thread pool to avoid metaspace errors associated with thread leaks. There are still other thread shutdown issues remaining, but this seems to be the most serious one. --- .../cassandra/utils/progress/jmx/JMXBroadcastExecutor.java | 6 ++++-- .../apache/cassandra/distributed/impl/AbstractCluster.java | 12 ++++++++++++ .../org/apache/cassandra/distributed/impl/Instance.java | 4 ++++ 3 files changed, 20 insertions(+), 2 deletions(-) diff --git a/src/java/org/apache/cassandra/utils/progress/jmx/JMXBroadcastExecutor.java b/src/java/org/apache/cassandra/utils/progress/jmx/JMXBroadcastExecutor.java index f545f0f..f28609c 100644 --- a/src/java/org/apache/cassandra/utils/progress/jmx/JMXBroadcastExecutor.java +++ b/src/java/org/apache/cassandra/utils/progress/jmx/JMXBroadcastExecutor.java @@ -18,9 +18,11 @@ package org.apache.cassandra.utils.progress.jmx; -import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import org.apache.cassandra.concurrent.NamedThreadFactory; + /** * Holds dedicated executor for JMX event handling. Events will internally queued by ArrayNotificationBuffer, * synchronized by an exclusive write lock, which makes a shared single thread executor desirable. @@ -30,6 +32,6 @@ public final class JMXBroadcastExecutor private JMXBroadcastExecutor() { } - public final static Executor executor = Executors.newSingleThreadExecutor(); + public final static ExecutorService executor = Executors.newSingleThreadExecutor(new NamedThreadFactory("JMX")); } diff --git a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java index a164e3b..0c8421a 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java @@ -667,9 +667,21 @@ public abstract class AbstractCluster<I extends IInstance> implements ICluster<I Thread.setDefaultUncaughtExceptionHandler(previousHandler); previousHandler = null; + //checkForThreadLeaks(); //withThreadLeakCheck(futures); } + private void checkForThreadLeaks() + { + //This is an alternate version of the thread leak check that just checks to see if any threads are still alive + // with the context classloader. + Set<Thread> threadSet = Thread.getAllStackTraces().keySet(); + threadSet.stream().filter(t->t.getContextClassLoader() instanceof InstanceClassLoader).forEach(t->{ + t.setContextClassLoader(null); + throw new RuntimeException("Unterminated thread detected " + t.getName() + " in group " + t.getThreadGroup().getName()); + }); + } + // We do not want this check to run every time until we fix problems with tread stops private void withThreadLeakCheck(List<Future<?>> futures) { diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java index c3e9982..989bf6e 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java @@ -108,6 +108,7 @@ import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.Throwables; import org.apache.cassandra.utils.concurrent.Ref; import org.apache.cassandra.utils.memory.BufferPool; +import org.apache.cassandra.utils.progress.jmx.JMXBroadcastExecutor; import static java.util.concurrent.TimeUnit.MINUTES; import static org.apache.cassandra.distributed.api.Feature.GOSSIP; @@ -566,6 +567,9 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance () -> Stage.shutdownAndWait(1L, MINUTES), () -> SharedExecutorPool.SHARED.shutdownAndWait(1L, MINUTES) ); + error = parallelRun(error, executor, + () -> shutdownAndWait(Collections.singletonList(JMXBroadcastExecutor.executor)) + ); Throwables.maybeFail(error); }).apply(isolatedExecutor); --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org