You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Alex Petrov (JIRA)" <ji...@apache.org> on 2019/07/12 12:22:00 UTC

[jira] [Comment Edited] (CASSANDRA-15170) Reduce the time needed to release in-JVM dtest cluster resources after close

    [ https://issues.apache.org/jira/browse/CASSANDRA-15170?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16883755#comment-16883755 ] 

Alex Petrov edited comment on CASSANDRA-15170 at 7/12/19 12:21 PM:
-------------------------------------------------------------------

[~jmeredithco] thank you for the patch. I have several minor nits:

  * {{numClusterNodes}} seems to be unused in {{ResourceLeakTest}}
  * I'm not 100% sure why we need changes to logging to remove instance IDs from some log messages and adding {{INSTANCE}} prefix to logger names.
  * We have a [shutdown hook|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L641], which should be using the instance class loader, but because we're running it after the instance class loader is already shut down, we get the exception [1]. The error message it throws is unclear, and I would probably override {InstanceClassLoader#close} to make it more obvious what's going on: if class loader is already closed, we should thrown with a message that it's been already shut down. In addition to this, I'd probably avoid adding a JVM shutdown hook, and close this explicitly. I think this was existing prior to this patch. 
 * On multiple runs, I've also seen the exceptions [2], [3], and [4]. I'm not claiming that this patch has caused them.  
 * We're seemingly logging each log message twice right now. I think this is also pre-existing, and this can be resolved by using only one of the two console appenders.
 
[1]
{code}
java.lang.NoClassDefFoundError: org/apache/cassandra/utils/logging/LoggingSupportFactory
        at org.apache.cassandra.service.StorageService$1.runMayThrow(StorageService.java:638)
        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.ClassNotFoundException: org.apache.cassandra.utils.logging.LoggingSupportFactory
        at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
        at org.apache.cassandra.distributed.impl.InstanceClassLoader.loadClassInternal(InstanceClassLoader.java:95)
        at org.apache.cassandra.distributed.impl.InstanceClassLoader.loadClass(InstanceClassLoader.java:84)
        ... 4 more
{code}

[2]
{code}
java.util.concurrent.RejectedExecutionException: ThreadPoolExecutor has shut down
	at org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor$1.rejectedExecution(DebuggableThreadPoolExecutor.java:58)
	at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:830)
	at java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1379)
	at org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.execute(DebuggableThreadPoolExecutor.java:162)
	at org.apache.cassandra.db.ColumnFamilyStore.waitForFlushes(ColumnFamilyStore.java:907)
	at org.apache.cassandra.db.ColumnFamilyStore.forceFlush(ColumnFamilyStore.java:873)
	at org.apache.cassandra.schema.SchemaKeyspace.lambda$flush$19(SchemaKeyspace.java:348)
	at com.google.common.collect.ImmutableList.forEach(ImmutableList.java:407)
	at org.apache.cassandra.schema.SchemaKeyspace.flush(SchemaKeyspace.java:348)
	at org.apache.cassandra.schema.SchemaKeyspace.applyChanges(SchemaKeyspace.java:1282)
	at org.apache.cassandra.schema.Schema.merge(Schema.java:653)
	at org.apache.cassandra.schema.Schema.mergeAndAnnounceVersion(Schema.java:586)
	at org.apache.cassandra.schema.MigrationTask.lambda$runMayThrow$0(MigrationTask.java:91)
	at org.apache.cassandra.net.ResponseVerbHandler.doVerb(ResponseVerbHandler.java:58)
	at org.apache.cassandra.net.InboundSink.lambda$new$0(InboundSink.java:77)
	at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:93)
	at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:44)
	at org.apache.cassandra.net.InboundMessageHandler$ProcessMessage.run(InboundMessageHandler.java:885)
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	at java.lang.Thread.run(Thread.java:748)
{code}

[3]
{code}
SEVERE: RuntimeException while executing runnable org.apache.cassandra.db.ColumnFamilyStore$Flush$1@46975039 with executor org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor@7616fb13[Terminated, pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 21]
java.util.concurrent.RejectedExecutionException: ThreadPoolExecutor has shut down
	at org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor$1.rejectedExecution(DebuggableThreadPoolExecutor.java:58)
	at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:830)
	at java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1379)
	at org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.execute(DebuggableThreadPoolExecutor.java:162)
	at com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:141)
	at com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:130)
	at com.google.common.util.concurrent.ListenableFutureTask.done(ListenableFutureTask.java:88)
	at java.util.concurrent.FutureTask.finishCompletion(FutureTask.java:384)
	at java.util.concurrent.FutureTask.set(FutureTask.java:233)
	at java.util.concurrent.FutureTask.run(FutureTask.java:274)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	at java.lang.Thread.run(Thread.java:748)
{code}

[4]
{code}
Dumping resources for exception: org.apache.cassandra.exceptions.UnavailableException: Cannot achieve consistency level ALL
java.lang.RuntimeException: org.apache.cassandra.exceptions.UnavailableException: Cannot achieve consistency level ALL
	at org.apache.cassandra.distributed.impl.IsolatedExecutor.waitOn(IsolatedExecutor.java:183)
	at org.apache.cassandra.distributed.impl.IsolatedExecutor.lambda$sync$4(IsolatedExecutor.java:83)
	at org.apache.cassandra.distributed.impl.Coordinator.execute(Coordinator.java:82)
	at org.apache.cassandra.distributed.test.ResourceLeakTest.doTest(ResourceLeakTest.java:148)
	at org.apache.cassandra.distributed.test.ResourceLeakTest.looperTest(ResourceLeakTest.java:172)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
	at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
	at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:68)
	at com.intellij.rt.execution.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:47)
	at com.intellij.rt.execution.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:242)
	at com.intellij.rt.execution.junit.JUnitStarter.main(JUnitStarter.java:70)
Caused by: org.apache.cassandra.exceptions.UnavailableException: Cannot achieve consistency level ALL
	at org.apache.cassandra.exceptions.UnavailableException.create(UnavailableException.java:37)
	at org.apache.cassandra.locator.ReplicaPlans.assureSufficientLiveReplicas(ReplicaPlans.java:170)
	at org.apache.cassandra.locator.ReplicaPlans.assureSufficientLiveReplicasForWrite(ReplicaPlans.java:113)
	at org.apache.cassandra.locator.ReplicaPlans.forWrite(ReplicaPlans.java:349)
	at org.apache.cassandra.locator.ReplicaPlans.forWrite(ReplicaPlans.java:343)
	at org.apache.cassandra.locator.ReplicaPlans.forWrite(ReplicaPlans.java:336)
	at org.apache.cassandra.locator.ReplicaPlans.forWrite(ReplicaPlans.java:325)
	at org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:1081)
	at org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:621)
	at org.apache.cassandra.service.StorageProxy.mutateWithTriggers(StorageProxy.java:859)
	at org.apache.cassandra.cql3.statements.ModificationStatement.executeWithoutCondition(ModificationStatement.java:475)
	at org.apache.cassandra.cql3.statements.ModificationStatement.execute(ModificationStatement.java:453)
	at org.apache.cassandra.distributed.impl.Coordinator.lambda$execute$0(Coordinator.java:63)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
{code}


was (Author: ifesdjeen):
[~jmeredithco] thank you for the patch. I have several minor nits:

  * {{numClusterNodes}} seems to be unused in {{ResourceLeakTest}}
  * I'm not 100% sure why we need changes to logging to remove instance IDs from some log messages and adding {{INSTANCE}} prefix to logger names.
  * We have a [shutdown hook|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L641], which should be using the instance class loader, but because we're running it after the instance class loader is already shut down, we get the exception [1]. The error message it throws is unclear, and I would probably override {InstanceClassLoader#close} to make it more obvious what's going on: if class loader is already closed, we should thrown with a message that it's been already shut down. In addition to this, I'd probably avoid adding a JVM shutdown hook, and close this explicitly. I think this was existing prior to this patch. 
 * On multiple runs, I've also seen the exceptions [2] and [3]. I'm not claiming that this patch has caused them.  
 * We're seemingly logging each log message twice right now. I think this is also pre-existing, and this can be resolved by using only one of the two console appenders.
 
[1]
{code}
java.lang.NoClassDefFoundError: org/apache/cassandra/utils/logging/LoggingSupportFactory
        at org.apache.cassandra.service.StorageService$1.runMayThrow(StorageService.java:638)
        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.ClassNotFoundException: org.apache.cassandra.utils.logging.LoggingSupportFactory
        at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
        at org.apache.cassandra.distributed.impl.InstanceClassLoader.loadClassInternal(InstanceClassLoader.java:95)
        at org.apache.cassandra.distributed.impl.InstanceClassLoader.loadClass(InstanceClassLoader.java:84)
        ... 4 more
{code}

[2]
{code}
java.util.concurrent.RejectedExecutionException: ThreadPoolExecutor has shut down
	at org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor$1.rejectedExecution(DebuggableThreadPoolExecutor.java:58)
	at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:830)
	at java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1379)
	at org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.execute(DebuggableThreadPoolExecutor.java:162)
	at org.apache.cassandra.db.ColumnFamilyStore.waitForFlushes(ColumnFamilyStore.java:907)
	at org.apache.cassandra.db.ColumnFamilyStore.forceFlush(ColumnFamilyStore.java:873)
	at org.apache.cassandra.schema.SchemaKeyspace.lambda$flush$19(SchemaKeyspace.java:348)
	at com.google.common.collect.ImmutableList.forEach(ImmutableList.java:407)
	at org.apache.cassandra.schema.SchemaKeyspace.flush(SchemaKeyspace.java:348)
	at org.apache.cassandra.schema.SchemaKeyspace.applyChanges(SchemaKeyspace.java:1282)
	at org.apache.cassandra.schema.Schema.merge(Schema.java:653)
	at org.apache.cassandra.schema.Schema.mergeAndAnnounceVersion(Schema.java:586)
	at org.apache.cassandra.schema.MigrationTask.lambda$runMayThrow$0(MigrationTask.java:91)
	at org.apache.cassandra.net.ResponseVerbHandler.doVerb(ResponseVerbHandler.java:58)
	at org.apache.cassandra.net.InboundSink.lambda$new$0(InboundSink.java:77)
	at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:93)
	at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:44)
	at org.apache.cassandra.net.InboundMessageHandler$ProcessMessage.run(InboundMessageHandler.java:885)
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	at java.lang.Thread.run(Thread.java:748)
{code}

[3]
{code}
SEVERE: RuntimeException while executing runnable org.apache.cassandra.db.ColumnFamilyStore$Flush$1@46975039 with executor org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor@7616fb13[Terminated, pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 21]
java.util.concurrent.RejectedExecutionException: ThreadPoolExecutor has shut down
	at org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor$1.rejectedExecution(DebuggableThreadPoolExecutor.java:58)
	at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:830)
	at java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1379)
	at org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.execute(DebuggableThreadPoolExecutor.java:162)
	at com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:141)
	at com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:130)
	at com.google.common.util.concurrent.ListenableFutureTask.done(ListenableFutureTask.java:88)
	at java.util.concurrent.FutureTask.finishCompletion(FutureTask.java:384)
	at java.util.concurrent.FutureTask.set(FutureTask.java:233)
	at java.util.concurrent.FutureTask.run(FutureTask.java:274)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	at java.lang.Thread.run(Thread.java:748)
{code}


> Reduce the time needed to release in-JVM dtest cluster resources after close
> ----------------------------------------------------------------------------
>
>                 Key: CASSANDRA-15170
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-15170
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Test/dtest
>            Reporter: Jon Meredith
>            Assignee: Jon Meredith
>            Priority: Normal
>
> There are a few issues that slow the in-JVM dtests from reclaiming metaspace once the cluster is closed.
> IsolatedExecutor issues the shutdown on a SingleExecutorThreadPool, sometimes this thread was still running 10s after the dtest cluster was closed.  Instead, switch to a ThreadPoolExecutor with a core pool size of 0 so that the thread executing the class loader close executes sooner.
> If an OutboundTcpConnection is waiting to connect() and the endpoint is not answering, it has to wait for a timeout before it exits. Instead it should check the isShutdown flag and terminate early if shutdown has been requested.
> In 3.0 and above, HintsCatalog.load uses java.nio.Files.list outside of a try-with-resources construct and leaks a file handle for the directory.  This doesn't matter for normal usage, it leaks a file handle for each dtest Instance created.
> On trunk, Netty global event executor threads are still running and delay GC for the instance class loader.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

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