Hi Bowen, would it be possible to share a full thread dump? Regards, Dmitry
On Tue, 5 Nov 2024 at 12:12, Bowen Song via user <user@cassandra.apache.org> wrote: > Hi all, > > We have a cluster running Cassandra 4.1.1. We are seeing the memtable > flush randomly getting stuck. This has happened twice in the last 10 days, > to two different nodes in the same cluster. This started to happen after we > enabled CDC, and each time it got stuck, there was at least one repair > running involving the affected node. > > The signs of the stuck memtable flush is most obvious from the > "StatusLogger" logs. > > At the beginning, the MemtablePostFlush and MemtableFlushWriter got stuck, > they have 1 and 2 active tasks each, and a small number of pending tasks. > > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:65 - > Pool Name Active Pending Completed Blocked > All Time Blocked > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > ReadStage 0 0 34052333 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > CompactionExecutor 0 0 1019777 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > MutationStage 0 0 14930764 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > MemtableReclaimMemory 0 0 21877 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > PendingRangeCalculator 0 0 177 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > Repair#61 0 0 1344 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > GossipStage 0 0 889452 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > SecondaryIndexManagement 0 0 1 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > HintsDispatcher 0 0 19 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > Repair-Task 0 0 65 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > RequestResponseStage 0 0 44447834 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > Native-Transport-Requests 0 0 8967921 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > MigrationStage 0 0 5 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > MemtableFlushWriter 2 10 21781 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > MemtablePostFlush 1 11 47856 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > PerDiskMemtableFlushWriter_0 0 0 21769 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > Sampler 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > ValidationExecutor 0 0 36651 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > ViewBuildExecutor 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > InternalResponseStage 0 0 240 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > AntiEntropyStage 1 1773 120067 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - > CacheCleanupExecutor 0 0 0 > 0 0 > > The number of pending tasks slowly grows larger over time, and the number > of completed tasks do not increase at all. > > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:65 - > Pool Name Active Pending Completed Blocked > All Time Blocked > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > ReadStage 0 0 39905462 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > CompactionExecutor 0 0 1170100 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > MutationStage 0 0 16976992 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > Repair#76 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > Repair#74 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > Repair#72 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > GossipStage 0 0 1026877 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > Repair-Task 0 0 83 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > RequestResponseStage 0 0 51122191 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > ValidationExecutor 0 0 36651 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > MemtablePostFlush 1 54 47856 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > MemtableFlushWriter 2 53 21781 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > CacheCleanupExecutor 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > Repair#67 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > MemtableReclaimMemory 0 0 21877 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > PendingRangeCalculator 0 0 177 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > Repair#82 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > Repair#61 0 0 1344 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > SecondaryIndexManagement 0 0 1 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > HintsDispatcher 0 0 20 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > Native-Transport-Requests 0 0 10145020 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > MigrationStage 0 0 5 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > Sampler 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - > PerDiskMemtableFlushWriter_0 0 0 21769 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,942 StatusLogger.java:69 - > ViewBuildExecutor 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,942 StatusLogger.java:69 - > InternalResponseStage 0 0 996 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 16:33:05,942 StatusLogger.java:69 - > AntiEntropyStage 1 9694 120067 > 0 0 > > Then, things start to get worse, the MutationStage also got stuck. > > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:65 - > Pool Name Active Pending Completed Blocked > All Time Blocked > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > ReadStage 0 0 40186426 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > CompactionExecutor 0 0 1171479 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > MutationStage 128 201414 16988212 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > Repair#76 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > Repair#74 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > Repair#72 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > GossipStage 0 0 1032988 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > Repair-Task 0 0 85 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > RequestResponseStage 0 0 51307497 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > ValidationExecutor 0 0 36651 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > MemtablePostFlush 1 54 47856 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > MemtableFlushWriter 2 53 21781 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > CacheCleanupExecutor 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > Repair#67 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > MemtableReclaimMemory 0 0 21877 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > Repair#84 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > PendingRangeCalculator 0 0 177 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > Repair#82 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > Repair#61 0 0 1344 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > SecondaryIndexManagement 0 0 1 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > HintsDispatcher 0 0 20 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > Native-Transport-Requests 0 0 10194549 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > MigrationStage 0 0 5 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > Sampler 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > PerDiskMemtableFlushWriter_0 0 0 21769 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > ViewBuildExecutor 0 0 0 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > InternalResponseStage 0 0 1087 > 0 0 > INFO [ScheduledTasks:1] 2024-11-03 17:05:40,974 StatusLogger.java:69 - > AntiEntropyStage 1 10364 120067 > 0 0 > > Writes on this node starts to timeout and fail. But if left untouched, > it's only gonna get worse, and eventually lead to JVM OOM and crash. > > By inspecting the heap dump created at OOM, we can see that both of the > MemtableFlushWriter threads are stuck on line 1190 > <https://github.com/apache/cassandra/blob/8d91b469afd3fcafef7ef85c10c8acc11703ba2d/src/java/org/apache/cassandra/db/ColumnFamilyStore.java#L1190> > in the ColumnFamilyStore.java: > > // mark writes older than the barrier as blocking progress, > permitting them to exceed our memory limit > // if they are stuck waiting on it, then wait for them all to > complete > writeBarrier.markBlocking(); > writeBarrier.await(); // <----------- stuck here > > And the MemtablePostFlush thread is stuck on line 1094 > <https://github.com/apache/cassandra/blob/8d91b469afd3fcafef7ef85c10c8acc11703ba2d/src/java/org/apache/cassandra/db/ColumnFamilyStore.java#L1094> > in the same file. > > try > { > // we wait on the latch for the commitLogUpperBound to be > set, and so that waiters > // on this task can rely on all prior flushes being > complete > latch.await(); // <----------- stuck here > } > Our top suspect is CDC interacting with repair, since this started to > happen shortly after we enabled CDC on the nodes, and each time repair was > running. But we have not been able to reproduce this in a testing cluster, > and don't know what's the next step to troubleshoot this issue. So I'm > posting it in the mailing lists and hoping someone may know something about > it or point me to the right direction. > > p.s.: sorry about posting this to both the user & dev mailing lists. It's > an end-user related issue but involves Cassandra internals, so I can't > decide which one is best suited. > > Cheers, > Bowen > > > -- Dmitry Konstantinov