You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Yuki Morishita (JIRA)" <ji...@apache.org> on 2015/09/03 00:50:45 UTC

[jira] [Commented] (CASSANDRA-10079) LEAK DETECTED, after nodetool drain

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

Yuki Morishita commented on CASSANDRA-10079:
--------------------------------------------

Reproduced with the following:

# Create SSTables using {{cassaandra-stress}} with size tiered compaction disabled.
# {{nodetool compact}}
# During major compaction, trigger drain.
# Force GC
# LEAK DETECTED

I found that compaction keeps running after drain, and it can get stuck when updating {{compaction_in_progress}} because commit log is stopped during drain.
Stack trace where compaction is hanging is:

{code}
CompactionExecutor:1 [WAITING] [DAEMON]
sun.misc.Unsafe.park(boolean, long) Unsafe.java (native)
java.util.concurrent.locks.LockSupport.park() LockSupport.java:304
org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly() WaitQueue.java:283
org.apache.cassandra.db.commitlog.PeriodicCommitLogService.maybeWaitForSync(CommitLogSegment$Allocation) PeriodicCommitLogService.java:44
org.apache.cassandra.db.commitlog.AbstractCommitLogService.finishWriteFor(CommitLogSegment$Allocation) AbstractCommitLogService.java:152
org.apache.cassandra.db.commitlog.CommitLog.add(Mutation) CommitLog.java:252
org.apache.cassandra.db.Keyspace.apply(Mutation, boolean, boolean) Keyspace.java:383
org.apache.cassandra.db.Keyspace.apply(Mutation, boolean) Keyspace.java:363
org.apache.cassandra.db.Mutation.apply() Mutation.java:214
org.apache.cassandra.cql3.statements.ModificationStatement.executeInternalWithoutCondition(QueryState, QueryOptions) ModificationStatement.java:641
org.apache.cassandra.cql3.statements.ModificationStatement.executeInternal(QueryState, QueryOptions) ModificationStatement.java:631
org.apache.cassandra.cql3.QueryProcessor.executeInternal(String, Object[]) QueryProcessor.java:317
org.apache.cassandra.db.SystemKeyspace.finishCompaction(UUID) SystemKeyspace.java:241
org.apache.cassandra.db.compaction.CompactionTask.runMayThrow() CompactionTask.java:233
org.apache.cassandra.utils.WrappedRunnable.run() WrappedRunnable.java:28
org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionManager$CompactionExecutorStatsCollector) CompactionTask.java:73
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(CompactionManager$CompactionExecutorStatsCollector) AbstractCompactionTask.java:59
org.apache.cassandra.db.compaction.CompactionManager$7.runMayThrow() CompactionManager.java:510
org.apache.cassandra.utils.WrappedRunnable.run() WrappedRunnable.java:28
java.util.concurrent.Executors$RunnableAdapter.call() Executors.java:511
java.util.concurrent.FutureTask.run() FutureTask.java:266
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor$Worker) ThreadPoolExecutor.java:1142
java.util.concurrent.ThreadPoolExecutor$Worker.run() ThreadPoolExecutor.java:617
java.lang.Thread.run() Thread.java:745
{code}

Since compaction task relies on accessing system tables ({{compaction_in_progress}} (not in 3.0 anymore) and {{compaction_history}}) we should shutdown compaction before shutting down commit log.

> LEAK DETECTED, after nodetool drain
> -----------------------------------
>
>                 Key: CASSANDRA-10079
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-10079
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Sebastian Estevez
>            Assignee: Yuki Morishita
>             Fix For: 2.1.x
>
>
> 6 node cluster running 2.1.8
> Sequence of events:
> {quote}
> 2015-08-14 13:37:07,049 - Drain the node
> 2015-08-14 13:37:11,943 - Drained
> 2015-08-14 13:37:37,055 Ref.java:179 - LEAK DETECTED:
> {quote}
> {code}
> ERROR [Reference-Reaper:1] 2015-08-14 13:37:37,055 Ref.java:179 - LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@5534701) to class org.apache.cassandra.utils.concurrent.WrappedSharedCloseable$1@194296283:[[OffHeapBitSet]] was not released before the reference was garbage collected
> ERROR [Reference-Reaper:1] 2015-08-14 13:37:37,057 Ref.java:179 - LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@fab2c71) to class org.apache.cassandra.io.util.MmappedSegmentedFile$Cleanup@1252635616:/var/lib/cassandra/data/metric/metric-811fa5402a3b11e5a2c0870545c0f352/metric-metric-ka-6883-Index.db was not released before the reference was garbage collected
> ERROR [Reference-Reaper:1] 2015-08-14 13:37:37,057 Ref.java:179 - LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@555d8efb) to class org.apache.cassandra.io.util.MmappedSegmentedFile$Cleanup@1252635616:/var/lib/cassandra/data/metric/metric-811fa5402a3b11e5a2c0870545c0f352/metric-metric-ka-6883-Index.db was not released before the reference was garbage collected
> ERROR [Reference-Reaper:1] 2015-08-14 13:37:37,057 Ref.java:179 - LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@7b29bfea) to class org.apache.cassandra.io.util.MmappedSegmentedFile$Cleanup@1252635616:/var/lib/cassandra/data/metric/metric-811fa5402a3b11e5a2c0870545c0f352/metric-metric-ka-6883-Index.db was not released before the reference was garbage collected
> ERROR [Reference-Reaper:1] 2015-08-14 13:37:37,057 Ref.java:179 - LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@2d37dc5a) to class org.apache.cassandra.utils.concurrent.WrappedSharedCloseable$1@713444527:[[OffHeapBitSet]] was not released before the reference was garbage collected
> ERROR [Reference-Reaper:1] 2015-08-14 13:37:37,057 Ref.java:179 - LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@13153552) to class org.apache.cassandra.utils.concurrent.WrappedSharedCloseable$1@713444527:[[OffHeapBitSet]] was not released before the reference was garbage collected
> ERROR [Reference-Reaper:1] 2015-08-14 13:37:37,057 Ref.java:179 - LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@25f51e35) to class org.apache.cassandra.utils.concurrent.WrappedSharedCloseable$1@713444527:[[OffHeapBitSet]] was not released before the reference was garbage collected
> ERROR [Reference-Reaper:1] 2015-08-14 13:37:37,057 Ref.java:179 - LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@3633d3dd) to class org.apache.cassandra.utils.concurrent.WrappedSharedCloseable$1@194296283:[[OffHeapBitSet]] was not released before the reference was garbage collected
> ERROR [Reference-Reaper:1] 2015-08-14 13:37:37,057 Ref.java:179 - LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@2ec81280) to class org.apache.cassandra.utils.concurrent.WrappedSharedCloseable$1@194296283:[[OffHeapBitSet]] was not released before the reference was garbage collected
> ERROR [Reference-Reaper:1] 2015-08-14 13:37:37,058 Ref.java:179 - LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@144d1dae) to class org.apache.cassandra.utils.concurrent.WrappedSharedCloseable$1@194296283:[[OffHeapBitSet]] was not released before the reference was garbage collected
> ERROR [Reference-Reaper:1] 2015-08-14 13:37:37,058 Ref.java:179 - LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@1944bda4) to class org.apache.cassandra.utils.concurrent.WrappedSharedCloseable$1@194296283:[[OffHeapBitSet]] was not released before the reference was garbage collected
> ERROR [Reference-Reaper:1] 2015-08-14 13:37:37,058 Ref.java:179 - LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@31c1386a) to class org.apache.cassandra.io.util.MmappedSegmentedFile$Cleanup@1601396928:/var/lib/cassandra/data/metric/metric-811fa5402a3b11e5a2c0870545c0f352/metric-metric-ka-8229-Index.db was not released before the reference was garbage collected
> {code}
> See full log here:
> https://dl.dropboxusercontent.com/u/4179566/cassandra-system.log



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)