Upon further studying the thread dump, I think there is a deadlock between
CompactionExecutor (Thread1) & MemtableReclaimMemory (Thread2) in the stack
trace I have mentioned in my email below.

Please find the deadlock details here:

   1. *Thread1:* It has invoked *readOrdering.start()* on *baseCf* and
   *indexCf* as part of the "*ReadExecutionController controller =
   cmd.executionController()*" at a line no
   
<https://github.com/apache/cassandra/blob/cassandra-4.1/src/java/org/apache/cassandra/index/SecondaryIndexManager.java#L921>
   2. *Thread1:* It wants to apply the change locally to Memtable as part
   of indexer.insertRow(row) at a line no
   
<https://github.com/apache/cassandra/blob/cassandra-4.1/src/java/org/apache/cassandra/index/SecondaryIndexManager.java#L962>
   3. *Thread1:* There is not enough space available in Memtable; hence,
   Thread1 is blocked on space to become available
   4. *Thread2:* It wants to free up the space in Memtable, but before it
   does that, it is waiting
   
<https://github.com/apache/cassandra/blob/cassandra-4.1/src/java/org/apache/cassandra/db/ColumnFamilyStore.java#L1350>
    on all the *readOrdering*for a given Cf to finish.
   5. *Thread2**:* But Thread2's indefinite wait cannot succeed because
   Thread1 has already invoked "*.start()*" on it, so unless and until
   Thread1 calls *".close()"* Threa2 cannot proceed. And here is the
   *deadlock*.

Here are the stack traces for Thread1 and Thread2 to prove the deadlock, as
mentioned above.

Thread1 (*CompactionExecutor*)

"CompactionExecutor:5" - Thread t@118
   java.lang.Thread.State: WAITING
        at java.base@11.0.20.1/jdk.internal.misc.Unsafe.park(Native Method)
        at 
java.base@11.0.20.1/java.util.concurrent.locks.LockSupport.park(LockSupport.java:323)
        at 
app//org.apache.cassandra.utils.concurrent.WaitQueue$Standard$AbstractSignal.await(WaitQueue.java:289)
       at 
app//org.apache.cassandra.utils.concurrent.WaitQueue$Standard$AbstractSignal.await(WaitQueue.java:282)
       at 
app//org.apache.cassandra.utils.concurrent.Awaitable$Defaults.awaitThrowUncheckedOnInterrupt(Awaitable.java:131)
       at 
app//org.apache.cassandra.utils.concurrent.Awaitable$AbstractAwaitable.awaitThrowUncheckedOnInterrupt(Awaitable.java:235)
       at 
app//org.apache.cassandra.utils.memory.MemtableAllocator$SubAllocator.allocate(MemtableAllocator.java:195)
       at 
app//org.apache.cassandra.utils.memory.SlabAllocator.allocate(SlabAllocator.java:89)
       at 
app//org.apache.cassandra.utils.memory.MemtableBufferAllocator$1.allocate(MemtableBufferAllocator.java:40)
       at 
app//org.apache.cassandra.utils.memory.ByteBufferCloner.clone(ByteBufferCloner.java:77)
       at 
app//org.apache.cassandra.utils.memory.ByteBufferCloner.clone(ByteBufferCloner.java:63)
       at app//org.apache.cassandra.db.Clustering.clone(Clustering.java:54)
       at 
app//org.apache.cassandra.utils.memory.ByteBufferCloner.clone(ByteBufferCloner.java:52)
       at app//org.apache.cassandra.db.rows.BTreeRow.clone(BTreeRow.java:517)
       at 
app//org.apache.cassandra.db.partitions.AtomicBTreePartition$RowUpdater.insert(AtomicBTreePartition.java:375)
       at 
app//org.apache.cassandra.db.partitions.AtomicBTreePartition$RowUpdater.insert(AtomicBTreePartition.java:351)
       at 
app//org.apache.cassandra.utils.btree.BTree.updateLeaves(BTree.java:461)
       at app//org.apache.cassandra.utils.btree.BTree.update(BTree.java:368)
       at 
app//org.apache.cassandra.db.partitions.AtomicBTreePartition.addAllWithSizeDeltaInternal(AtomicBTreePartition.java:146)
       at 
app//org.apache.cassandra.db.partitions.AtomicBTreePartition.addAllWithSizeDelta(AtomicBTreePartition.java:190)
       at 
app//org.apache.cassandra.db.memtable.SkipListMemtable.put(SkipListMemtable.java:134)
       at 
app//org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:1431)
       at 
app//org.apache.cassandra.db.CassandraTableWriteHandler.write(CassandraTableWriteHandler.java:40)
       at 
app//org.apache.cassandra.index.internal.CassandraIndex.insert(CassandraIndex.java:529)
       at 
app//org.apache.cassandra.index.internal.CassandraIndex$1.indexCell(CassandraIndex.java:445)
       at 
app//org.apache.cassandra.index.internal.CassandraIndex$1.indexCells(CassandraIndex.java:437)
       at 
app//org.apache.cassandra.index.internal.CassandraIndex$1.insertRow(CassandraIndex.java:387)
       at 
app//org.apache.cassandra.index.SecondaryIndexManager.lambda$indexPartition$25(SecondaryIndexManager.java:985)
       at 
app//org.apache.cassandra.index.SecondaryIndexManager$$Lambda$1396/0x00007f00861e2cb0.accept(Unknown
Source)        at
java.base@11.0.20.1/java.lang.Iterable.forEach(Iterable.java:75)
        at 
app//org.apache.cassandra.index.SecondaryIndexManager.indexPartition(SecondaryIndexManager.java:985)
       at 
app//org.apache.cassandra.index.internal.CollatedViewIndexBuilder.build(CollatedViewIndexBuilder.java:78)
       at 
app//org.apache.cassandra.db.compaction.CompactionManager$13.run(CompactionManager.java:1888)
       at 
app//org.apache.cassandra.concurrent.FutureTask$3.call(FutureTask.java:141)
       at 
app//org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
       at 
app//org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
       at 
java.base@11.0.20.1/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        at 
java.base@11.0.20.1/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        at 
app//io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
       at java.base@11.0.20.1/java.lang.Thread.run(Thread.java:829)



Thread2 (*MemtableReclaimMemory*)

"MemtableReclaimMemory:1" - Thread t@56
   java.lang.Thread.State: WAITING
        at java.base@11.0.20.1/jdk.internal.misc.Unsafe.park(Native Method)
        at 
java.base@11.0.20.1/java.util.concurrent.locks.LockSupport.park(LockSupport.java:323)
        at 
app//org.apache.cassandra.utils.concurrent.WaitQueue$Standard$AbstractSignal.await(WaitQueue.java:289)
       at 
app//org.apache.cassandra.utils.concurrent.WaitQueue$Standard$AbstractSignal.await(WaitQueue.java:282)
       at 
app//org.apache.cassandra.utils.concurrent.Awaitable$Defaults.awaitUninterruptibly(Awaitable.java:186)
       at 
app//org.apache.cassandra.utils.concurrent.Awaitable$AbstractAwaitable.awaitUninterruptibly(Awaitable.java:259)
       at 
app//org.apache.cassandra.utils.concurrent.OpOrder$Group.await(OpOrder.java:267)
       at 
app//org.apache.cassandra.utils.concurrent.OpOrder$Barrier.await(OpOrder.java:425)
       at 
app//org.apache.cassandra.db.ColumnFamilyStore$Flush$1.runMayThrow(ColumnFamilyStore.java:1357)
       at 
app//org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
       at 
app//org.apache.cassandra.concurrent.ExecutionFailure$1.run(ExecutionFailure.java:133)
       at 
java.base@11.0.20.1/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        at 
java.base@11.0.20.1/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        at 
app//io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
       at java.base@11.0.20.1/java.lang.Thread.run(Thread.java:829)

Jaydeep

On Wed, Nov 6, 2024 at 8:29 AM Jaydeep Chovatia <chovatia.jayd...@gmail.com>
wrote:

> We have also seen this issue a few times in our production (4.1). My
> teammates added a thread dump here
> <https://issues.apache.org/jira/browse/CASSANDRA-19564?focusedCommentId=17893987&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17893987>
> .
>
> One of my theories is that the MemtableReclaimMemory thread has been
> unable to make progress:
>
> "MemtableReclaimMemory:1" - Thread t@56
>
>    java.lang.Thread.State: WAITING
>
>         at java.base@11.0.20.1/jdk.internal.misc.Unsafe.park(Native
> Method)
>
>         at
> java.base@11.0.20.1/java.util.concurrent.locks.LockSupport.park(LockSupport.java:323)
>
>         at
> app//org.apache.cassandra.utils.concurrent.WaitQueue$Standard$AbstractSignal.await(WaitQueue.java:289)
>
>         at
> app//org.apache.cassandra.utils.concurrent.WaitQueue$Standard$AbstractSignal.await(WaitQueue.java:282)
>
>         at
> app//org.apache.cassandra.utils.concurrent.Awaitable$Defaults.awaitUninterruptibly(Awaitable.java:186)
>
>         at
> app//org.apache.cassandra.utils.concurrent.Awaitable$AbstractAwaitable.awaitUninterruptibly(Awaitable.java:259)
>
>         at
> app//org.apache.cassandra.utils.concurrent.OpOrder$Group.await(OpOrder.java:267)
>
>         at
> app//org.apache.cassandra.utils.concurrent.OpOrder$Barrier.await(OpOrder.java:425)
>
>         at
> app//org.apache.cassandra.db.ColumnFamilyStore$Flush$1.runMayThrow(ColumnFamilyStore.java:1357)
>
>         at
> app//org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>
>         at
> app//org.apache.cassandra.concurrent.ExecutionFailure$1.run(ExecutionFailure.java:133)
>
>         at
> java.base@11.0.20.1/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>
>         at
> java.base@11.0.20.1/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>
>         at
> app//io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>
>         at java.base@11.0.20.1/java.lang.Thread.run(Thread.java:829)
>
>
>
> One of the possible bugs I could see is a line
> <https://github.com/apache/cassandra/blob/cassandra-4.1/src/java/org/apache/cassandra/db/ReadExecutionController.java#L141C77-L141C97>
> *            indexController = new ReadExecutionController(command,
> indexCfs.readOrdering.start(), indexCfs.metadata(), null, null,
> NO_SAMPLING, false);*
> If "*indexCfs.readOrdering.start()*" succeeded but the constructor "*new
> ReadExecutionController*", then we are not closing "
> *indexCfs.readOrdering*", which means it will remain
> inaccurate forever.
>
>
> Jaydeep
>
> On Wed, Nov 6, 2024 at 5:51 AM Bowen Song via user <
> user@cassandra.apache.org> wrote:
>
>> I think I'm getting really close now. This seems to have something to do
>> with the "read-hotness-tracker:1" thread. The thread dump is:
>>
>> "read-hotness-tracker:1" daemon prio=5 tid=93 WAITING
>>     at jdk.internal.misc.Unsafe.park(Native Method)
>>     at java.util.concurrent.locks.LockSupport.park(LockSupport.java:323)
>>     at
>> org.apache.cassandra.utils.concurrent.WaitQueue$Standard$AbstractSignal.await(WaitQueue.java:289)
>>        local variable:
>> org.apache.cassandra.utils.concurrent.WaitQueue$Standard$SignalWithListener#2086139
>>     at
>> org.apache.cassandra.utils.concurrent.WaitQueue$Standard$AbstractSignal.await(WaitQueue.java:282)
>>        local variable:
>> org.apache.cassandra.utils.concurrent.WaitQueue$Standard$SignalWithListener#2086139
>>     at
>> org.apache.cassandra.utils.concurrent.Awaitable$Defaults.awaitUninterruptibly(Awaitable.java:186)
>>        local variable:
>> org.apache.cassandra.utils.concurrent.WaitQueue$Standard$SignalWithListener#2086139
>>     at
>> org.apache.cassandra.utils.concurrent.Awaitable$AbstractAwaitable.awaitUninterruptibly(Awaitable.java:259)
>>     at
>> org.apache.cassandra.db.commitlog.AbstractCommitLogService.awaitSyncAt(AbstractCommitLogService.java:324)
>> <------ here
>>        local variable:
>> org.apache.cassandra.db.commitlog.PeriodicCommitLogService#1
>>        local variable: com.codahale.metrics.Timer$Context#2086795
>>     at
>> org.apache.cassandra.db.commitlog.PeriodicCommitLogService.maybeWaitForSync(PeriodicCommitLogService.java:42)
>>        local variable:
>> org.apache.cassandra.db.commitlog.PeriodicCommitLogService#1
>>     at
>> org.apache.cassandra.db.commitlog.AbstractCommitLogService.finishWriteFor(AbstractCommitLogService.java:284)
>>        local variable:
>> org.apache.cassandra.db.commitlog.PeriodicCommitLogService#1
>>     at org.apache.cassandra.db.commitlog.CommitLog.add(CommitLog.java:307)
>>        local variable: org.apache.cassandra.db.commitlog.CommitLog#1
>>        local variable:
>> org.apache.cassandra.io.util.DataOutputBuffer$1$1#61
>>        local variable:
>> org.apache.cassandra.db.commitlog.CommitLogSegment$Allocation#1
>>     at
>> org.apache.cassandra.db.CassandraKeyspaceWriteHandler.addToCommitLog(CassandraKeyspaceWriteHandler.java:100)
>>     at
>> org.apache.cassandra.db.CassandraKeyspaceWriteHandler.beginWrite(CassandraKeyspaceWriteHandler.java:54)
>> <------ and here
>>        local variable:
>> org.apache.cassandra.utils.concurrent.OpOrder$Group#8162
>>     at org.apache.cassandra.db.Keyspace.applyInternal(Keyspace.java:628)
>>        local variable: org.apache.cassandra.db.Keyspace#8
>>        local variable: org.apache.cassandra.db.Mutation#54491
>>     at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:512)
>>     at org.apache.cassandra.db.Mutation.apply(Mutation.java:228)
>>     at org.apache.cassandra.db.Mutation.apply(Mutation.java:248)
>>     at
>> org.apache.cassandra.cql3.statements.ModificationStatement.executeInternalWithoutCondition(ModificationStatement.java:675)
>>     at
>> org.apache.cassandra.cql3.statements.ModificationStatement.executeLocally(ModificationStatement.java:666)
>>     at
>> org.apache.cassandra.cql3.QueryProcessor.executeInternal(QueryProcessor.java:447)
>>     at
>> org.apache.cassandra.db.SystemKeyspace.persistSSTableReadMeter(SystemKeyspace.java:1488)
>>        local variable: java.lang.String#42806
>>        local variable: java.lang.String#47659
>>        local variable:
>> org.apache.cassandra.io.sstable.UUIDBasedSSTableId#1389
>>        local variable: org.apache.cassandra.metrics.RestorableMeter#752
>>     at
>> org.apache.cassandra.io.sstable.format.SSTableReader$GlobalTidy$1.run(SSTableReader.java:2170)
>>     at
>> org.apache.cassandra.concurrent.ExecutionFailure$1.run(ExecutionFailure.java:124)
>>        local variable:
>> org.apache.cassandra.concurrent.ExecutionFailure$1#122029
>>        local variable:
>> org.apache.cassandra.utils.WithResources$None$$Lambda$223#1
>>     at
>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
>>        local variable: java.util.concurrent.Executors$RunnableAdapter#934
>>     at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:305)
>>        local variable:
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask#931
>>     at
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:305)
>>        local variable:
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask#931
>>     at
>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>>        local variable:
>> org.apache.cassandra.concurrent.ScheduledThreadPoolExecutorPlus#6
>>        local variable: java.util.concurrent.ThreadPoolExecutor$Worker#7
>>        local variable: io.netty.util.concurrent.FastThreadLocalThread#7
>>        local variable:
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask#931
>>     at
>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>>        local variable: java.util.concurrent.ThreadPoolExecutor$Worker#7
>>     at
>> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>>        local variable: io.netty.util.concurrent.FastThreadLocalRunnable#7
>>     at java.lang.Thread.run(Thread.java:829)
>>        local variable: io.netty.util.concurrent.FastThreadLocalThread#7
>>
>>
>> In the CassandraKeyspaceWriteHandler.beginWrite() method:
>>
>> public WriteContext beginWrite(Mutation mutation, boolean makeDurable)
>> throws RequestExecutionException
>> {
>>     OpOrder.Group group = null;
>>     try
>>     {
>>         group = Keyspace.writeOrder.start();  // <--- here
>>
>>         // write the mutation to the commitlog and memtables
>>         CommitLogPosition position = null;
>>         if (makeDurable)
>>         {
>>             position = addToCommitLog(mutation);  // <--- and here
>>         }
>>         return new CassandraWriteContext(group, position);
>>     }
>>     catch (Throwable t)
>>     {
>>         if (group != null)
>>         {
>>             group.close();
>>         }
>>         throw t;
>>     }
>> }
>>
>> The Keyspace.writeOrder.start() method call blocks the
>> MemtableFlushWriter thread, and then it calls addToCommitLog(mutation) which
>> indirectly calls AbstractCommitLogService.awaitSyncAt()
>>
>>     void awaitSyncAt(long syncTime, Context context)
>>     {
>>         do
>>         {
>>             WaitQueue.Signal signal = context != null ?
>> syncComplete.register(context, Context::stop) : syncComplete.register();
>>             if (lastSyncedAt < syncTime)
>>                 signal.awaitUninterruptibly();  // <--- here
>>             else
>>                 signal.cancel();
>>         }
>>         while (lastSyncedAt < syncTime);
>>     }
>>
>> This then got stuck on the signal.awaitUninterruptibly()
>>
>> Now I know what is blocking the memtable flushing, but I haven't been
>> able to figure out is why it got stuck on waiting for that signal.
>>
>> I would appreciate it if anyone can offer some insight here.
>>
>>
>> On 05/11/2024 17:48, Bowen Song via user wrote:
>>
>> I will give it a try and see what I can find. I plan to go down the
>> rabbit hole tomorrow. Will keep you updated.
>> On 05/11/2024 17:34, Jeff Jirsa wrote:
>>
>>
>>
>> On Nov 5, 2024, at 4:12 AM, Bowen Song via user
>> <user@cassandra.apache.org> <user@cassandra.apache.org> wrote:
>>
>> 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.
>>
>>
>> Wouldn’t be completely surprised if  CDC  or repair somehow has a
>> barrier, I’ve also seen similar behavior pre-3.0 with “very long running
>> read commands” that have a barrier on the memtable that prevent release.
>>
>> You’ve got the heap (great, way better than most people debugging), are
>> you able to navigate through it and look for references to that memtable or
>> other things holding a barrier?
>>
>>
>>
>>
>>

Reply via email to