cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Dan Kinder (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (CASSANDRA-13948) Avoid deadlock when not able to acquire references for compaction
Date Fri, 13 Oct 2017 01:05:01 GMT

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

Dan Kinder edited comment on CASSANDRA-13948 at 10/13/17 1:04 AM:
------------------------------------------------------------------

Just a heads up, I have been seeing these deadlocks happen easily, so I am running your patch
[~pauloricardomg] in addition to Marcus's[patch|https://github.com/krummas/cassandra/commits/marcuse/13215]
from CASSANDRA-13215.

I do see a large number of "Could not acquire references for compacting SSTable" happening,
in bursts. Will upload a log file.

I also see some of this:
{noformat}
java.lang.AssertionError: Memory was freed
        at org.apache.cassandra.io.util.Memory.checkBounds(Memory.java:344) ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.io.util.Memory.getInt(Memory.java:291) ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.io.sstable.IndexSummary.getPositionInSummary(IndexSummary.java:148)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.io.sstable.IndexSummary.fillTemporaryKey(IndexSummary.java:162)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.io.sstable.IndexSummary.binarySearch(IndexSummary.java:121)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.io.sstable.format.SSTableReader.getSampleIndexesForRanges(SSTableReader.java:1370)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.io.sstable.format.SSTableReader.estimatedKeysForRanges(SSTableReader.java:1326)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.db.compaction.AbstractCompactionStrategy.worthDroppingTombstones(AbstractCompactionStrategy.java:441)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.db.compaction.LeveledCompactionStrategy.findDroppableSSTable(LeveledCompactionStrategy.java:503)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getNextBackgroundTask(LeveledCompactionStrategy.java:121)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.db.compaction.CompactionStrategyManager.getNextBackgroundTask(CompactionStrategyManager.java:124)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:262)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_144]
        at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_144]
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
~[na:1.8.0_144]
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
[na:1.8.0_144]
        at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_144]
{noformat}

UPDATE: the node does successfully complete compactions for a while but gradually does fewer
and fewer and then stops compacting altogether, even though compactionstats says there are
pending compactions:

{noformat}
dkinder@seu-walker-fs01:~$ nodetool compactionstats
pending tasks: 102
- walker.links: 102

dkinder@seu-walker-fs01:~$
{noformat}
It stays this way and periodically prints out some "Could not acquire references for compacting
SSTable" messages. It is able to compact some more if I restart the node.

tablestats for walker.links:
{noformat}
Keyspace : walker
        Read Count: 16952
        Read Latency: 10.388668062765454 ms.
        Write Count: 277291
        Write Latency: 0.0186555207345352 ms.
        Pending Flushes: 0
                Table: links
                SSTable count: 8507
                SSTables in each level: [73/4, 81/10, 297/100, 2078/1000, 2402, 3635, 0, 0,
0]
                Space used (live): 4902698702556
                Space used (total): 4902698702556
                Space used by snapshots (total): 13788057680993
                Off heap memory used (total): 9835235
                SSTable Compression Ratio: -1.0
                Number of partitions (estimate): 19996043
                Memtable cell count: 360248
                Memtable data size: 36729792
                Memtable off heap memory used: 0
                Memtable switch count: 0
                Local read count: 0
                Local read latency: NaN ms
                Local write count: 360248
                Local write latency: 0.017 ms
                Pending flushes: 0
                Percent repaired: 0.0
                Bloom filter false positives: 0
                Bloom filter false ratio: 0.00000
                Bloom filter space used: 8538560
                Bloom filter off heap memory used: 8470504
                Index summary off heap memory used: 1364731
                Compression metadata off heap memory used: 0
                Compacted partition minimum bytes: 43
                Compacted partition maximum bytes: 190420296972
                Compacted partition mean bytes: 247808
                Average live cells per slice (last five minutes): NaN
                Maximum live cells per slice (last five minutes): 0
                Average tombstones per slice (last five minutes): NaN
                Maximum tombstones per slice (last five minutes): 0
                Dropped Mutations: 2
{noformat}


was (Author: dkinder):
Just a heads up, I have been seeing these deadlocks happen easily, so I am running your patch
[~pauloricardomg] in addition to Marcus's[patch|https://github.com/krummas/cassandra/commits/marcuse/13215]
from CASSANDRA-13215.

I do see a large number of "Could not acquire references for compacting SSTables [BigTableReader(path='/srv/disk2/cassandra-data/walker/links/mc-6566879-big-Data.db')]
which is not a problem per se,unless it happens frequently, in which case it must be reported.
Will retry later." happening, in bursts. Will upload a log file.

I also see some of this:
{noformat}
java.lang.AssertionError: Memory was freed
        at org.apache.cassandra.io.util.Memory.checkBounds(Memory.java:344) ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.io.util.Memory.getInt(Memory.java:291) ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.io.sstable.IndexSummary.getPositionInSummary(IndexSummary.java:148)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.io.sstable.IndexSummary.fillTemporaryKey(IndexSummary.java:162)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.io.sstable.IndexSummary.binarySearch(IndexSummary.java:121)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.io.sstable.format.SSTableReader.getSampleIndexesForRanges(SSTableReader.java:1370)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.io.sstable.format.SSTableReader.estimatedKeysForRanges(SSTableReader.java:1326)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.db.compaction.AbstractCompactionStrategy.worthDroppingTombstones(AbstractCompactionStrategy.java:441)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.db.compaction.LeveledCompactionStrategy.findDroppableSSTable(LeveledCompactionStrategy.java:503)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getNextBackgroundTask(LeveledCompactionStrategy.java:121)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.db.compaction.CompactionStrategyManager.getNextBackgroundTask(CompactionStrategyManager.java:124)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:262)
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_144]
        at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_144]
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
~[na:1.8.0_144]
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
[na:1.8.0_144]
        at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
        at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_144]
{noformat}

> Avoid deadlock when not able to acquire references for compaction
> -----------------------------------------------------------------
>
>                 Key: CASSANDRA-13948
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13948
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Compaction
>            Reporter: Paulo Motta
>            Assignee: Paulo Motta
>             Fix For: 3.11.x, 4.x
>
>         Attachments: debug.log
>
>
> The thread dump below shows a race between an sstable replacement by the {{IndexSummaryRedistribution}}
and {{AbstractCompactionTask.getNextBackgroundTask}}:
> {noformat}
> Thread 94580: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, line=175 (Compiled
frame)
>  - java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt() @bci=1,
line=836 (Compiled frame)
>  - java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(java.util.concurrent.locks.AbstractQueuedSynchronizer$Node,
int) @bci=67, line=870 (Compiled frame)
>  - java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(int) @bci=17, line=1199
(Compiled frame)
>  - java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock() @bci=5, line=943
(Compiled frame)
>  - org.apache.cassandra.db.compaction.CompactionStrategyManager.handleListChangedNotification(java.lang.Iterable,
java.lang.Iterable) @bci=359, line=483 (Interpreted frame)
>  - org.apache.cassandra.db.compaction.CompactionStrategyManager.handleNotification(org.apache.cassandra.notifications.INotification,
java.lang.Object) @bci=53, line=555 (Interpreted frame)
>  - org.apache.cassandra.db.lifecycle.Tracker.notifySSTablesChanged(java.util.Collection,
java.util.Collection, org.apache.cassandra.db.compaction.OperationType, java.lang.Throwable)
@bci=50, line=409 (Interpreted frame)
>  - org.apache.cassandra.db.lifecycle.LifecycleTransaction.doCommit(java.lang.Throwable)
@bci=157, line=227 (Interpreted frame)
>  - org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.commit(java.lang.Throwable)
@bci=61, line=116 (Compiled frame)
>  - org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.commit()
@bci=2, line=200 (Interpreted frame)
>  - org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.finish()
@bci=5, line=185 (Interpreted frame)
>  - org.apache.cassandra.io.sstable.IndexSummaryRedistribution.redistributeSummaries()
@bci=559, line=130 (Interpreted frame)
>  - org.apache.cassandra.db.compaction.CompactionManager.runIndexSummaryRedistribution(org.apache.cassandra.io.sstable.IndexSummaryRedistribution)
@bci=9, line=1420 (Interpreted frame)
>  - org.apache.cassandra.io.sstable.IndexSummaryManager.redistributeSummaries(org.apache.cassandra.io.sstable.IndexSummaryRedistribution)
@bci=4, line=250 (Interpreted frame)
>  - org.apache.cassandra.io.sstable.IndexSummaryManager.redistributeSummaries() @bci=30,
line=228 (Interpreted frame)
>  - org.apache.cassandra.io.sstable.IndexSummaryManager$1.runMayThrow() @bci=4, line=125
(Interpreted frame)
>  - org.apache.cassandra.utils.WrappedRunnable.run() @bci=1, line=28 (Interpreted frame)
>  - org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run()
@bci=4, line=118 (Compiled frame)
>  - java.util.concurrent.Executors$RunnableAdapter.call() @bci=4, line=511 (Compiled frame)
>  - java.util.concurrent.FutureTask.runAndReset() @bci=47, line=308 (Compiled frame)
>  - java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask)
@bci=1, line=180 (Compiled frame)
>  - java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run() @bci=37,
line=294 (Compiled frame)
>  - java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
@bci=95, line=1149 (Compiled frame)
>  - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=624 (Interpreted
frame)
>  - org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(java.lang.Runnable)
@bci=1, line=81 (Interpreted frame)
>  - org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$8.run() @bci=4 (Interpreted
frame)
>  - java.lang.Thread.run() @bci=11, line=748 (Compiled frame)
> {noformat}
> {noformat}
> Thread 94573: (state = IN_JAVA)
>  - java.util.HashMap$HashIterator.nextNode() @bci=95, line=1441 (Compiled frame; information
may be imprecise)
>  - java.util.HashMap$KeyIterator.next() @bci=1, line=1461 (Compiled frame)
>  - org.apache.cassandra.db.lifecycle.View$3.apply(org.apache.cassandra.db.lifecycle.View)
@bci=20, line=268 (Compiled frame)
>  - org.apache.cassandra.db.lifecycle.View$3.apply(java.lang.Object) @bci=5, line=265
(Compiled frame)
>  - org.apache.cassandra.db.lifecycle.Tracker.apply(com.google.common.base.Predicate,
com.google.common.base.Function) @bci=13, line=133 (Compiled frame)
>  - org.apache.cassandra.db.lifecycle.Tracker.tryModify(java.lang.Iterable, org.apache.cassandra.db.compaction.OperationType)
@bci=31, line=99 (Compiled frame)
>  - org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getNextBackgroundTask(int)
@bci=84, line=139 (Compiled frame)
>  - org.apache.cassandra.db.compaction.CompactionStrategyManager.getNextBackgroundTask(int)
@bci=105, line=119 (Interpreted frame)
>  - org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run()
@bci=84, line=265 (Interpreted frame)
>  - java.util.concurrent.Executors$RunnableAdapter.call() @bci=4, line=511 (Compiled frame)
>  - java.util.concurrent.FutureTask.run() @bci=42, line=266 (Compiled frame)
>  - java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
@bci=95, line=1149 (Compiled frame)
>  - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=624 (Interpreted
frame)
>  - org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(java.lang.Runnable)
@bci=1, line=81 (Interpreted frame)
>  - org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$8.run() @bci=4 (Interpreted
frame)
>  - java.lang.Thread.run() @bci=11, line=748 (Compiled frame)
> {noformat}
> This particular node remain in this state forever, indicating {{LeveledCompactionStrategyTask.getNextBackgroundTask}}
was looping indefinitely.
> What happened is that sstable references were replaced on the tracker by the {{IndexSummaryRedistribution}}
thread, so the {{AbstractCompactionStrategy.getNextBackgroundTask}} could not create the transaction
with the old references, and the {{IndexSummaryRedistribution}} could not update the sstable
reference in the compaction strategy because {{AbstractCompactionStrategy.getNextBackgroundTask}}
was holding the {{CompactionStrategyManager}} lock.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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


Mime
View raw message