Uploaded image for project: 'Apache Cassandra'
  1. Apache Cassandra
  2. CASSANDRA-19564

MemtablePostFlush deadlock leads to stuck nodes and crashes

    XMLWordPrintableJSON

Details

    • Availability - Process Crash
    • Critical
    • Normal
    • User Report
    • All
    • None

    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. 

      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
      

      MemtablePostFlush appears to be blocked

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

      MemtablePostFlush   1    1   3446   0   0
      

       
      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:

      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.

      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()

       MutationStage    48    828425      980253369      0    0

       

      Compaction Stops

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

       

       

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

      Attachments

        1. screenshot-1.png
          183 kB
          Jon Haddad
        2. image-2024-04-17-19-14-34-344.png
          403 kB
          Jon Haddad
        3. image-2024-04-17-19-13-06-769.png
          157 kB
          Jon Haddad
        4. image-2024-04-17-18-46-29-474.png
          217 kB
          Jon Haddad
        5. image-2024-04-16-13-53-24-455.png
          245 kB
          Jon Haddad
        6. image-2024-04-16-13-43-11-064.png
          96 kB
          Jon Haddad
        7. image-2024-04-16-12-29-15-386.png
          98 kB
          Jon Haddad
        8. image-2024-04-16-11-55-54-750.png
          427 kB
          Jon Haddad

        Activity

          People

            curlylrt Runtian Liu
            rustyrazorblade Jon Haddad
            Runtian Liu
            Votes:
            0 Vote for this issue
            Watchers:
            13 Start watching this issue

            Dates

              Created:
              Updated: