[
https://issues.apache.org/jira/browse/CASSANDRA-9882?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14681893#comment-14681893
]
Sean Thornton edited comment on CASSANDRA-9882 at 8/11/15 5:18 PM:
-------------------------------------------------------------------
Edit: trying to remember formatting - haven't posted anything for a while
Also have seen this behavior under LCS. Again with a cluster that had 150-200K
sstables. First thread below executed for a very, very long time and held onto
a lock that blocked flushes and compactions (and some jmx methods). Glanced at
the code and saw some nested loops each going through all sstables but didn't
look deeper than that.
{panel}
{noformat}
"CompactionExecutor:12" daemon prio=10 tid=0x00002ad7735244a0 nid=0x81f6
runnable [0x00002ad7beb60000]
java.lang.Thread.State: RUNNABLE
at org.apache.cassandra.dht.Bounds.intersects(Bounds.java:67)
at
org.apache.cassandra.db.compaction.LeveledManifest.overlapping(LeveledManifest.java:459)
at
org.apache.cassandra.db.compaction.LeveledManifest.overlapping(LeveledManifest.java:445)
at
org.apache.cassandra.db.compaction.LeveledManifest.getCandidatesFor(LeveledManifest.java:520)
at
org.apache.cassandra.db.compaction.LeveledManifest.getCompactionCandidates(LeveledManifest.java:307)
- locked <0x00000006b02c9a30> (a
org.apache.cassandra.db.compaction.LeveledManifest)
at
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getMaximalTask(LeveledCompactionStrategy.java:121)
at
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getNextBackgroundTask(LeveledCompactionStrategy.java:113)
- locked <0x00000006b0315410> (a
org.apache.cassandra.db.compaction.LeveledCompactionStrategy)
at
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:192)
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
Locked ownable synchronizers:
- <0x0000000739a196c0> (a
java.util.concurrent.ThreadPoolExecutor$Worker)
"FlushWriter:26" daemon prio=10 tid=0x00002ad697d84210 nid=0x941f waiting for
monitor entry [0x00002ad7bfb5f000]
java.lang.Thread.State: BLOCKED (on object monitor)
at
org.apache.cassandra.db.compaction.LeveledManifest.add(LeveledManifest.java:118)
- waiting to lock <0x00000006b02c9a30> (a
org.apache.cassandra.db.compaction.LeveledManifest)
at
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:181)
at org.apache.cassandra.db.DataTracker.notifyAdded(DataTracker.java:461)
at
org.apache.cassandra.db.DataTracker.replaceFlushed(DataTracker.java:179)
at
org.apache.cassandra.db.compaction.AbstractCompactionStrategy.replaceFlushed(AbstractCompactionStrategy.java:231)
at
org.apache.cassandra.db.ColumnFamilyStore.replaceFlushed(ColumnFamilyStore.java:1151)
at
org.apache.cassandra.db.Memtable$FlushRunnable.runMayThrow(Memtable.java:346)
at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
Locked ownable synchronizers:
- <0x00000006c8c12568> (a
java.util.concurrent.ThreadPoolExecutor$Worker)
"CompactionExecutor:15" daemon prio=10 tid=0x00002ad697eb9640 nid=0x81fa
waiting for monitor entry [0x00002ad7bea5c000]
java.lang.Thread.State: BLOCKED (on object monitor)
at
org.apache.cassandra.db.compaction.LeveledManifest.replace(LeveledManifest.java:128)
- waiting to lock <0x00000006b02c9a30> (a
org.apache.cassandra.db.compaction.LeveledManifest)
at
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:186)
at
org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:454)
at
org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:267)
at
org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:1146)
at
org.apache.cassandra.db.compaction.CompactionTask.replaceCompactedSSTables(CompactionTask.java:330)
at
org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:254)
at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
at
org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
at
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
at
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:198)
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
Locked ownable synchronizers:
- <0x00000007399e4110> (a
java.util.concurrent.ThreadPoolExecutor$Worker)
"CompactionExecutor:11" daemon prio=10 tid=0x00002ad773523ed0 nid=0x81f5
waiting on condition [0x00002ad7beba1000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00000007976adaf0> (a
java.util.concurrent.FutureTask)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:425)
at java.util.concurrent.FutureTask.get(FutureTask.java:187)
at
org.apache.cassandra.utils.FBUtilities.waitOnFuture(FBUtilities.java:409)
at
org.apache.cassandra.db.SystemKeyspace.forceBlockingFlush(SystemKeyspace.java:459)
at
org.apache.cassandra.db.SystemKeyspace.finishCompaction(SystemKeyspace.java:205)
at
org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:237)
at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
at
org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
at
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
at
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:198)
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
Locked ownable synchronizers:
- <0x00000007399c4a60> (a
java.util.concurrent.ThreadPoolExecutor$Worker)
"RMI TCP Connection(130)-10.129.124.20" daemon prio=10 tid=0x00002ad5d8598750
nid=0xc054 waiting for monitor entry [0x00002ad7bdce5000]
java.lang.Thread.State: BLOCKED (on object monitor)
at
org.apache.cassandra.db.compaction.LeveledManifest.getEstimatedTasks(LeveledManifest.java:620)
- waiting to lock <0x00000006b02c9a30> (a
org.apache.cassandra.db.compaction.LeveledManifest)
at
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getEstimatedRemainingTasks(LeveledCompactionStrategy.java:173)
at
org.apache.cassandra.metrics.CompactionMetrics$1.value(CompactionMetrics.java:64)
at
org.apache.cassandra.metrics.CompactionMetrics$1.value(CompactionMetrics.java:57)
....
{noformat}
{panel}
was (Author: thorntsg):
Also have seen this behavior under LCS. Again with a cluster that had 150-200K
sstables. First thread below executed for a very, very long time and held onto
a lock that blocked flushes and compactions (and some jmx methods). Glanced at
the code and saw some nested loops each going through all sstables but didn't
look deeper than that.
*Cut down stack* -
"CompactionExecutor:12" daemon prio=10 tid=0x00002ad7735244a0 nid=0x81f6
runnable [0x00002ad7beb60000]
java.lang.Thread.State: RUNNABLE
at org.apache.cassandra.dht.Bounds.intersects(Bounds.java:67)
at
org.apache.cassandra.db.compaction.LeveledManifest.overlapping(LeveledManifest.java:459)
at
org.apache.cassandra.db.compaction.LeveledManifest.overlapping(LeveledManifest.java:445)
at
org.apache.cassandra.db.compaction.LeveledManifest.getCandidatesFor(LeveledManifest.java:520)
at
org.apache.cassandra.db.compaction.LeveledManifest.getCompactionCandidates(LeveledManifest.java:307)
- locked <0x00000006b02c9a30> (a
org.apache.cassandra.db.compaction.LeveledManifest)
at
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getMaximalTask(LeveledCompactionStrategy.java:121)
at
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getNextBackgroundTask(LeveledCompactionStrategy.java:113)
- locked <0x00000006b0315410> (a
org.apache.cassandra.db.compaction.LeveledCompactionStrategy)
at
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:192)
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
Locked ownable synchronizers:
- <0x0000000739a196c0> (a
java.util.concurrent.ThreadPoolExecutor$Worker)
"FlushWriter:26" daemon prio=10 tid=0x00002ad697d84210 nid=0x941f waiting for
monitor entry [0x00002ad7bfb5f000]
java.lang.Thread.State: BLOCKED (on object monitor)
at
org.apache.cassandra.db.compaction.LeveledManifest.add(LeveledManifest.java:118)
- waiting to lock <0x00000006b02c9a30> (a
org.apache.cassandra.db.compaction.LeveledManifest)
at
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:181)
at org.apache.cassandra.db.DataTracker.notifyAdded(DataTracker.java:461)
at
org.apache.cassandra.db.DataTracker.replaceFlushed(DataTracker.java:179)
at
org.apache.cassandra.db.compaction.AbstractCompactionStrategy.replaceFlushed(AbstractCompactionStrategy.java:231)
at
org.apache.cassandra.db.ColumnFamilyStore.replaceFlushed(ColumnFamilyStore.java:1151)
at
org.apache.cassandra.db.Memtable$FlushRunnable.runMayThrow(Memtable.java:346)
at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
Locked ownable synchronizers:
- <0x00000006c8c12568> (a
java.util.concurrent.ThreadPoolExecutor$Worker)
"CompactionExecutor:15" daemon prio=10 tid=0x00002ad697eb9640 nid=0x81fa
waiting for monitor entry [0x00002ad7bea5c000]
java.lang.Thread.State: BLOCKED (on object monitor)
at
org.apache.cassandra.db.compaction.LeveledManifest.replace(LeveledManifest.java:128)
- waiting to lock <0x00000006b02c9a30> (a
org.apache.cassandra.db.compaction.LeveledManifest)
at
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:186)
at
org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:454)
at
org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:267)
at
org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:1146)
at
org.apache.cassandra.db.compaction.CompactionTask.replaceCompactedSSTables(CompactionTask.java:330)
at
org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:254)
at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
at
org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
at
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
at
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:198)
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
Locked ownable synchronizers:
- <0x00000007399e4110> (a
java.util.concurrent.ThreadPoolExecutor$Worker)
"CompactionExecutor:11" daemon prio=10 tid=0x00002ad773523ed0 nid=0x81f5
waiting on condition [0x00002ad7beba1000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00000007976adaf0> (a
java.util.concurrent.FutureTask)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:425)
at java.util.concurrent.FutureTask.get(FutureTask.java:187)
at
org.apache.cassandra.utils.FBUtilities.waitOnFuture(FBUtilities.java:409)
at
org.apache.cassandra.db.SystemKeyspace.forceBlockingFlush(SystemKeyspace.java:459)
at
org.apache.cassandra.db.SystemKeyspace.finishCompaction(SystemKeyspace.java:205)
at
org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:237)
at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
at
org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
at
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
at
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:198)
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
Locked ownable synchronizers:
- <0x00000007399c4a60> (a
java.util.concurrent.ThreadPoolExecutor$Worker)
"RMI TCP Connection(130)-10.129.124.20" daemon prio=10 tid=0x00002ad5d8598750
nid=0xc054 waiting for monitor entry [0x00002ad7bdce5000]
java.lang.Thread.State: BLOCKED (on object monitor)
at
org.apache.cassandra.db.compaction.LeveledManifest.getEstimatedTasks(LeveledManifest.java:620)
- waiting to lock <0x00000006b02c9a30> (a
org.apache.cassandra.db.compaction.LeveledManifest)
at
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getEstimatedRemainingTasks(LeveledCompactionStrategy.java:173)
at
org.apache.cassandra.metrics.CompactionMetrics$1.value(CompactionMetrics.java:64)
at
org.apache.cassandra.metrics.CompactionMetrics$1.value(CompactionMetrics.java:57)
....
> DTCS (maybe other strategies) can block flushing when there are lots of
> sstables
> --------------------------------------------------------------------------------
>
> Key: CASSANDRA-9882
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9882
> Project: Cassandra
> Issue Type: Bug
> Components: Core
> Reporter: Jeremiah Jordan
> Assignee: Marcus Eriksson
> Labels: dtcs
> Fix For: 2.1.x, 2.2.x
>
>
> MemtableFlushWriter tasks can get blocked by Compaction
> getNextBackgroundTask. This is in a wonky cluster with 200k sstables in the
> CF, but seems bad for flushing to be blocked by getNextBackgroundTask when we
> are trying to make these new "smart" strategies that may take some time to
> calculate what to do.
> {noformat}
> "MemtableFlushWriter:21" daemon prio=10 tid=0x00007ff7ad965000 nid=0x6693
> waiting for monitor entry [0x00007ff78a667000]
> java.lang.Thread.State: BLOCKED (on object monitor)
> at
> org.apache.cassandra.db.compaction.WrappingCompactionStrategy.handleNotification(WrappingCompactionStrategy.java:237)
> - waiting to lock <0x00000006fcdbbf60> (a
> org.apache.cassandra.db.compaction.WrappingCompactionStrategy)
> at org.apache.cassandra.db.DataTracker.notifyAdded(DataTracker.java:518)
> at
> org.apache.cassandra.db.DataTracker.replaceFlushed(DataTracker.java:178)
> at
> org.apache.cassandra.db.compaction.AbstractCompactionStrategy.replaceFlushed(AbstractCompactionStrategy.java:234)
> at
> org.apache.cassandra.db.ColumnFamilyStore.replaceFlushed(ColumnFamilyStore.java:1475)
> at
> org.apache.cassandra.db.Memtable$FlushRunnable.runMayThrow(Memtable.java:336)
> at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
> at
> com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
> at
> org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1127)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> at java.lang.Thread.run(Thread.java:745)
> Locked ownable synchronizers:
> - <0x0000000743b3ac38> (a
> java.util.concurrent.ThreadPoolExecutor$Worker)
> "MemtableFlushWriter:19" daemon prio=10 tid=0x00007ff7ac57a000 nid=0x649b
> waiting for monitor entry [0x00007ff78b8ee000]
> java.lang.Thread.State: BLOCKED (on object monitor)
> at
> org.apache.cassandra.db.compaction.WrappingCompactionStrategy.handleNotification(WrappingCompactionStrategy.java:237)
> - waiting to lock <0x00000006fcdbbf60> (a
> org.apache.cassandra.db.compaction.WrappingCompactionStrategy)
> at org.apache.cassandra.db.DataTracker.notifyAdded(DataTracker.java:518)
> at
> org.apache.cassandra.db.DataTracker.replaceFlushed(DataTracker.java:178)
> at
> org.apache.cassandra.db.compaction.AbstractCompactionStrategy.replaceFlushed(AbstractCompactionStrategy.java:234)
> at
> org.apache.cassandra.db.ColumnFamilyStore.replaceFlushed(ColumnFamilyStore.java:1475)
> at
> org.apache.cassandra.db.Memtable$FlushRunnable.runMayThrow(Memtable.java:336)
> at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
> at
> com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
> at
> org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1127)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> at java.lang.Thread.run(Thread.java:745)
> "CompactionExecutor:14" daemon prio=10 tid=0x00007ff7ad359800 nid=0x4d59
> runnable [0x00007fecce3ea000]
> java.lang.Thread.State: RUNNABLE
> at
> org.apache.cassandra.io.sstable.SSTableReader.equals(SSTableReader.java:628)
> at
> com.google.common.collect.ImmutableSet.construct(ImmutableSet.java:206)
> at
> com.google.common.collect.ImmutableSet.construct(ImmutableSet.java:220)
> at
> com.google.common.collect.ImmutableSet.access$000(ImmutableSet.java:74)
> at
> com.google.common.collect.ImmutableSet$Builder.build(ImmutableSet.java:531)
> at com.google.common.collect.Sets$1.immutableCopy(Sets.java:606)
> at
> org.apache.cassandra.db.ColumnFamilyStore.getOverlappingSSTables(ColumnFamilyStore.java:1352)
> at
> org.apache.cassandra.db.compaction.DateTieredCompactionStrategy.getNextBackgroundSSTables(DateTieredCompactionStrategy.java:88)
> at
> org.apache.cassandra.db.compaction.DateTieredCompactionStrategy.getNextBackgroundTask(DateTieredCompactionStrategy.java:65)
> - locked <0x00000006fcdbbf00> (a
> org.apache.cassandra.db.compaction.DateTieredCompactionStrategy)
> at
> org.apache.cassandra.db.compaction.WrappingCompactionStrategy.getNextBackgroundTask(WrappingCompactionStrategy.java:72)
> - locked <0x00000006fcdbbf60> (a
> org.apache.cassandra.db.compaction.WrappingCompactionStrategy)
> at
> org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:238)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at java.util.concurrent.FutureTask.run(FutureTask.java:262)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> at java.lang.Thread.run(Thread.java:745)
> {noformat}
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)