[ 
https://issues.apache.org/jira/browse/CASSANDRA-13213?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Zachary Girouard updated CASSANDRA-13213:
-----------------------------------------
    Since Version: 3.9  (was: 3.10)

> compactionstats not available, node eventually OOMs due to pending mutations
> ----------------------------------------------------------------------------
>
>                 Key: CASSANDRA-13213
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13213
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Compaction, Local Write-Read Paths
>            Reporter: Zachary Girouard
>
> I'm seeing semi-frequent instances of nodetool compactionstats hanging 
> forever. While this is occurring none of the compaction metrics are available 
> via jmx/jconsole.
> Sometimes the node will eventually recover, but I'm seeing a pattern where a 
> node will exhibit this behavior, and then eventually pending mutations start 
> piling up and the node dies due to OOM. Sometimes pending gossip operations 
> starting piling up too, but I think this is due to the impending OOM causing 
> everything to bog down.
> As an experiment I turned auto compaction off on all the nodes and I haven't 
> seen this issue occur since I did that. Additionally, I'm running 
> relocatesstables on some nodes with unthrottled compaction and so far none of 
> them have had any issues handling it.
> I managed to get some stack traces from a dying node:
> All MutationStage threads look similar to this:
> {noformat}
> Name: MutationStage-10                                                        
>   
> State: WAITING
> Total blocked: 9  Total waited: 1,959,850
> Stack trace: 
> sun.misc.Unsafe.park(Native Method)
> java.util.concurrent.locks.LockSupport.park(Unknown Source)
> org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:279)
> org.apache.cassandra.utils.memory.MemtableAllocator$SubAllocator.allocate(MemtableAllocator.java:162)
> org.apache.cassandra.utils.memory.SlabAllocator.allocate(SlabAllocator.java:89)
> org.apache.cassandra.utils.memory.ContextAllocator.allocate(ContextAllocator.java:57)
> org.apache.cassandra.utils.memory.ContextAllocator.clone(ContextAllocator.java:47)
> org.apache.cassandra.db.rows.BufferCell.copy(BufferCell.java:122)
> org.apache.cassandra.utils.memory.AbstractAllocator$CloningBTreeRowBuilder.addCell(AbstractAllocator.java:72)
> org.apache.cassandra.db.rows.Rows.copy(Rows.java:51)
> org.apache.cassandra.db.partitions.AtomicBTreePartition$RowUpdater.apply(AtomicBTreePartition.java:332)
> org.apache.cassandra.db.partitions.AtomicBTreePartition$RowUpdater.apply(AtomicBTreePartition.java:295)
> org.apache.cassandra.utils.btree.NodeBuilder.addNewKey(NodeBuilder.java:323)
> org.apache.cassandra.utils.btree.NodeBuilder.update(NodeBuilder.java:184)
> org.apache.cassandra.utils.btree.TreeBuilder.update(TreeBuilder.java:95)
> org.apache.cassandra.utils.btree.BTree.update(BTree.java:182)
> org.apache.cassandra.db.partitions.AtomicBTreePartition.addAllWithSizeDelta(AtomicBTreePartition.java:156)
> org.apache.cassandra.db.Memtable.put(Memtable.java:284)org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:1316)
> org.apache.cassandra.db.Keyspace.applyInternal(Keyspace.java:618)
> org.apache.cassandra.db.Keyspace.applyFuture(Keyspace.java:425)
> org.apache.cassandra.db.Mutation.applyFuture(Mutation.java:222)
> org.apache.cassandra.db.MutationVerbHandler.doVerb(MutationVerbHandler.java:68)
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:66)
> java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162)
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:134)
> org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109)
> java.lang.Thread.run(Unknown Source)
> {noformat}
> The Compaction threads
> {noformat}
> Name: CompactionExecutor:4
> State: RUNNABLE
> Total blocked: 32,781,277  Total waited: 549
> Stack trace: 
> org.apache.cassandra.io.sstable.format.SSTableReader.getTotalBytes(SSTableReader.java:661)
> org.apache.cassandra.db.compaction.LeveledManifest.getCandidatesFor(LeveledManifest.java:669)
> org.apache.cassandra.db.compaction.LeveledManifest.getCompactionCandidates(LeveledManifest.java:385)
>    - locked org.apache.cassandra.db.compaction.LeveledManifest@55c79600
>    
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getNextBackgroundTask(LeveledCompactionStrategy.java:119)
>    
> org.apache.cassandra.db.compaction.CompactionStrategyManager.getNextBackgroundTask(CompactionStrategyManager.java:119)
>    
> org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:261)
>    java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
>    java.util.concurrent.FutureTask.run(Unknown Source)
>    java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
>    java.util.concurrent.FutureTask.run(Unknown Source)
>    java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
>    java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>    
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
>    
> org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$4/45023307.run(Unknown
>  Source)
>    java.lang.Thread.run(Unknown Source)
> {noformat}
> {noformat}
>    Name: CompactionExecutor:1
>    State: BLOCKED on 
> org.apache.cassandra.db.compaction.LeveledManifest@55c79600 owned by: 
> CompactionExecutor:2
>    Total blocked: 116,196,349  Total waited: 4,771
>    Stack trace: 
>    
> org.apache.cassandra.db.compaction.LeveledManifest.getCompactionCandidates(LeveledManifest.java:310)
>    
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getNextBackgroundTask(LeveledCompactionStrategy.java:119)
>    
> org.apache.cassandra.db.compaction.CompactionStrategyManager.getNextBackgroundTask(CompactionStrategyManager.java:119)
>    
> org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:261)
>    java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
>    java.util.concurrent.FutureTask.run(Unknown Source)
>    java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
>    java.util.concurrent.FutureTask.run(Unknown Source)
>    java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
>    java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>    
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
>    
> org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$4/45023307.run(Unknown
>  Source)
>    java.lang.Thread.run(Unknown Source)
> {noformat}
> {noformat}
>    Name: CompactionExecutor:2
>    State: BLOCKED on 
> org.apache.cassandra.db.compaction.LeveledManifest@55c79600 owned by: 
> CompactionExecutor:1
>    Total blocked: 26,542,909  Total waited: 4,373
>    Stack trace: 
>    
> org.apache.cassandra.db.compaction.LeveledManifest.getCompactionCandidates(LeveledManifest.java:310)
>    
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getNextBackgroundTask(LeveledCompactionStrategy.java:119)
>    
> org.apache.cassandra.db.compaction.CompactionStrategyManager.getNextBackgroundTask(CompactionStrategyManager.java:119)
>    
> org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:261)
>    java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
>    java.util.concurrent.FutureTask.run(Unknown Source)
>    java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
>    java.util.concurrent.FutureTask.run(Unknown Source)
>    java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
>    java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>    
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
>    
> org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$4/45023307.run(Unknown
>  Source)
>    java.lang.Thread.run(Unknown Source)
> {noformat}
> Tp stats from the node
> {noformat}
>   Pool Name                    Active   Pending      Completed   Blocked  All 
> time blocked
>   MutationStage                    32    762545       53122841         0      
>            0
>   ViewMutationStage                 0         0              0         0      
>            0
>   ReadStage                         0         0         247792         0      
>            0
>   RequestResponseStage              0         0         200621         0      
>            0
>   ReadRepairStage                   0         0           2489         0      
>            0
>   CounterMutationStage              0         0              0         0      
>            0
>   MiscStage                         0         0              0         0      
>            0
>   CompactionExecutor                3        58          26816         0      
>            0
>   MemtableReclaimMemory             0         0            176         0      
>            0
>   PendingRangeCalculator            0         0             84         0      
>            0
>   GossipStage                       0         0         235500         0      
>            0
>   SecondaryIndexManagement          0         0              0         0      
>            0
>   HintsDispatcher                   0         0            749         0      
>            0
>   PerDiskMemtableFlushWriter_1         0         0            156         0   
>               0
>   PerDiskMemtableFlushWriter_2         0         0            156         0   
>               0
>   MigrationStage                    1        25             73         0      
>            0
>   MemtablePostFlush                 1        25           1953         0      
>            0
>   PerDiskMemtableFlushWriter_0         0         0            176         0   
>               0
>   ValidationExecutor                0         0           1320         0      
>            0
>   Sampler                           0         0              0         0      
>            0
>   MemtableFlushWriter               1        18            176         0      
>            0
>   InternalResponseStage             0         0           3030         0      
>            0
>   AntiEntropyStage                  0         0           4087         0      
>            0
>   CacheCleanupExecutor              0         0              0         0      
>            0
>   Native-Transport-Requests         0         0         670925         0      
>            0 
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to