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

Jon Haddad updated CASSANDRA-19564:
-----------------------------------
    Description: 
I've run into an issue on a 4.1.4 cluster where an entire node has locked up 
due to what I believe is a deadlock in memtable flushing. Here's what I know so 
far.  I've stitched together what happened based on conversations, logs, and 
some flame graphs.

*Log reports memtable flushing*

The last successful flush happens at 12:19. 
{noformat}
INFO  [NativePoolCleaner] 2024-04-16 12:19:53,634 
AbstractAllocatorMemtable.java:286 - Flushing largest CFS(Keyspace='ks', 
ColumnFamily='version') to free up room. Used total: 0.24/0.33, live: 
0.16/0.20, flushing: 0.09/0.13, this: 0.13/0.15
INFO  [NativePoolCleaner] 2024-04-16 12:19:53,634 ColumnFamilyStore.java:1012 - 
Enqueuing flush of ks.version, Reason: MEMTABLE_LIMIT, Usage: 660.521MiB (13%) 
on-heap, 790.606MiB (15%) off-heap
{noformat}
*MemtablePostFlush appears to be blocked*

At this point, MemtablePostFlush completed tasks stops incrementing, active 
stays at 1 and pending starts to rise.
{noformat}
MemtablePostFlush   1    1   3446   0   0{noformat}
 

The flame graph reveals that PostFlush.call is stuck.  I don't have the line 
number, but I know we're stuck in 
{{org.apache.cassandra.db.ColumnFamilyStore.PostFlush#call}} given the visual 
below:

 

*!image-2024-04-16-13-43-11-064.png!*

 

*Memtable flushing is now blocked.*

All MemtableFlushWriter threads are Parked waiting on 
{{{}OpOrder.Barrier.await{}}}. A wall clock profile of 30s reveals all time is 
spent here.  Presumably we're waiting on the single threaded Post Flush.

!image-2024-04-16-12-29-15-386.png!

*Memtable allocations start to block*

Eventually it looks like the NativeAllocator stops successfully allocating 
memory. I assume it's waiting on memory to be freed, but since memtable flushes 
are blocked, we wait indefinitely.

Looking at a wall clock flame graph, all writer threads have reached the 
allocation failure path of {{MemtableAllocator.allocate().  I believe we're 
waiting on }}{{signal.awaitThrowUncheckedOnInterrupt()}}
{noformat}
 MutationStage    48    828425      980253369      0    0{noformat}
!image-2024-04-16-11-55-54-750.png!

 

*Compaction Stops*

Since we write to the compaction history table, and that requires memtables, 
compactions are now blocked as well.

 

!image-2024-04-16-13-53-24-455.png!

 

The node is now doing basically nothing and must be restarted.

  was:
I've run into an issue on a 4.1.4 cluster where an entire node has locked up 
due to what I believe is a deadlock in memtable flushing. Here's what I know so 
far.  I've stitched together what happened based on conversations, logs, and 
some flame graphs.

*Log reports memtable flushing*

The last successful flush happens at 12:19. 
{noformat}
INFO  [NativePoolCleaner] 2024-04-16 12:19:53,634 
AbstractAllocatorMemtable.java:286 - Flushing largest CFS(Keyspace='ks', 
ColumnFamily='version') to free up room. Used total: 0.24/0.33, live: 
0.16/0.20, flushing: 0.09/0.13, this: 0.13/0.15
INFO  [NativePoolCleaner] 2024-04-16 12:19:53,634 ColumnFamilyStore.java:1012 - 
Enqueuing flush of ks.version, Reason: MEMTABLE_LIMIT, Usage: 660.521MiB (13%) 
on-heap, 790.606MiB (15%) off-heap
{noformat}
*MemtablePostFlush appears to be blocked*

At this point, MemtablePostFlush completed tasks stops incrementing, active 
stays at 1 and pending starts to rise.

 
{noformat}
MemtablePostFlush   1    1   3446   0   0{noformat}
 

 

The flame graph reveals that PostFlush.call is stuck.  I don't have the line 
number, but I know we're stuck in 
{{org.apache.cassandra.db.ColumnFamilyStore.PostFlush#call}} given the visual 
below:

 

*!image-2024-04-16-13-43-11-064.png!*

 

*Memtable flushing is now blocked.* 

All MemtableFlushWriter threads are Parked waiting on 
{{{}OpOrder.Barrier.await{}}}. A wall clock profile of 30s reveals all time is 
spent here.  Presumably we're waiting on the single threaded Post Flush.

!image-2024-04-16-12-29-15-386.png!

*Memtable allocations start to block*

Eventually it looks like the NativeAllocator stops successfully allocating 
memory. I assume it's waiting on memory to be freed, but since memtable flushes 
are blocked, we wait indefinitely.

Looking at a wall clock flame graph, all writer threads have reached the 
allocation failure path of {{MemtableAllocator.allocate().  I believe we're 
waiting on }}{{signal.awaitThrowUncheckedOnInterrupt()}}
{noformat}
 MutationStage    48    828425      980253369      0    0{noformat}
!image-2024-04-16-11-55-54-750.png!

 

*Compaction Stops*

Since we write to the compaction history table, and that requires memtables, 
compactions are now blocked as well.

 

!image-2024-04-16-13-53-24-455.png!

 

The node is now doing basically nothing and must be restarted.


> MemtablePostFlush deadlock leads to stuck nodes
> -----------------------------------------------
>
>                 Key: CASSANDRA-19564
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-19564
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Local/Compaction
>            Reporter: Jon Haddad
>            Priority: Normal
>         Attachments: image-2024-04-16-11-55-54-750.png, 
> image-2024-04-16-12-29-15-386.png, image-2024-04-16-13-43-11-064.png, 
> image-2024-04-16-13-53-24-455.png
>
>
> I've run into an issue on a 4.1.4 cluster where an entire node has locked up 
> due to what I believe is a deadlock in memtable flushing. Here's what I know 
> so far.  I've stitched together what happened based on conversations, logs, 
> and some flame graphs.
> *Log reports memtable flushing*
> The last successful flush happens at 12:19. 
> {noformat}
> INFO  [NativePoolCleaner] 2024-04-16 12:19:53,634 
> AbstractAllocatorMemtable.java:286 - Flushing largest CFS(Keyspace='ks', 
> ColumnFamily='version') to free up room. Used total: 0.24/0.33, live: 
> 0.16/0.20, flushing: 0.09/0.13, this: 0.13/0.15
> INFO  [NativePoolCleaner] 2024-04-16 12:19:53,634 ColumnFamilyStore.java:1012 
> - Enqueuing flush of ks.version, Reason: MEMTABLE_LIMIT, Usage: 660.521MiB 
> (13%) on-heap, 790.606MiB (15%) off-heap
> {noformat}
> *MemtablePostFlush appears to be blocked*
> At this point, MemtablePostFlush completed tasks stops incrementing, active 
> stays at 1 and pending starts to rise.
> {noformat}
> MemtablePostFlush   1    1   3446   0   0{noformat}
>  
> The flame graph reveals that PostFlush.call is stuck.  I don't have the line 
> number, but I know we're stuck in 
> {{org.apache.cassandra.db.ColumnFamilyStore.PostFlush#call}} given the visual 
> below:
>  
> *!image-2024-04-16-13-43-11-064.png!*
>  
> *Memtable flushing is now blocked.*
> All MemtableFlushWriter threads are Parked waiting on 
> {{{}OpOrder.Barrier.await{}}}. A wall clock profile of 30s reveals all time 
> is spent here.  Presumably we're waiting on the single threaded Post Flush.
> !image-2024-04-16-12-29-15-386.png!
> *Memtable allocations start to block*
> Eventually it looks like the NativeAllocator stops successfully allocating 
> memory. I assume it's waiting on memory to be freed, but since memtable 
> flushes are blocked, we wait indefinitely.
> Looking at a wall clock flame graph, all writer threads have reached the 
> allocation failure path of {{MemtableAllocator.allocate().  I believe we're 
> waiting on }}{{signal.awaitThrowUncheckedOnInterrupt()}}
> {noformat}
>  MutationStage    48    828425      980253369      0    0{noformat}
> !image-2024-04-16-11-55-54-750.png!
>  
> *Compaction Stops*
> Since we write to the compaction history table, and that requires memtables, 
> compactions are now blocked as well.
>  
> !image-2024-04-16-13-53-24-455.png!
>  
> The node is now doing basically nothing and must be restarted.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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

Reply via email to