Details
-
Bug
-
Status: Resolved
-
Normal
-
Resolution: Fixed
-
Normal
Description
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.
"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)