Hi Abe,

The com.datastax.oss.cdc.agent thing in the thread dump is the code forked from https://github.com/datastax/cdc-apache-cassandra/ . It sends the CDC data to Kafka instead of Pulsar, and some other additional features/changes.

Due to the lack of documentation and examples on how to read the CDC commitlog files, our developers couldn't figure out how to do it independently, so they built on top of the Datastax CDC agent, and used that to process the files. I don't like the idea of Java agent either. It makes my life hard too. Each time they make changes and release a new version of the jar file, I'll have to do a full cluster rolling restart. But this is the only thing we have, so we don't really have a choice.

Since this issue only happens on our production environment, and the CDC agent is actively being used, I can't remove it without impacting the production workload.

The pool-5-thread-1 thread you mentioned is waiting on this line <https://github.com/datastax/cdc-apache-cassandra/blob/cd886ceb2ab9669a04ded2c7742b357846e8fefa/agent/src/main/java/com/datastax/oss/cdc/agent/CommitLogReaderService.java#L96>:

    @Override
    public void run() {
        while(true) {
            try {
                File file = commitLogQueue.take();  // <--- here
                submitCommitLog(file);
            } catch (InterruptedException e) {
                log.error("error:", e);
            }
        }
    }

Note that the line number in thread dump won't match exactly, because our developers forked it and made changes to the code.

Cheers,
Bowen


On 08/11/2024 18:35, Abe Ratnofsky wrote:
Hey Bowen,

Are you able to reproduce the issue without running com.datastax.oss.cdc.agent? I don't see any glaringly obvious bugs there indicated by the thread dump, but it would be useful to rule that out, particularly because it runs as an agent and manages new instances of C*-defined classes, potentially on executors that aren't expecting to run those classes.

I'm curious what threads like this one are waiting on:

"pool-5-thread-1" prio=5 tid=37 WAITING
    at jdk.internal.misc.Unsafe.park(Native Method)
    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:194)
       local variable: java.lang.Thread#25
    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2081)        local variable: java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject#16        local variable: java.util.concurrent.locks.AbstractQueuedSynchronizer$Node#62     at java.util.concurrent.PriorityBlockingQueue.take(PriorityBlockingQueue.java:546)
       local variable: java.util.concurrent.PriorityBlockingQueue#1
       local variable: java.util.concurrent.locks.ReentrantLock#24
    at com.datastax.oss.cdc.agent.CommitLogReaderService.run(CommitLogReaderService.java:95)        local variable: com.datastax.oss.cdc.agent.CommitLogReaderServiceImpl#1

On Nov 8, 2024, at 12:22, Dmitry Konstantinov <netud...@gmail.com> wrote:

Hi Bowen,

>> protected volatile long lastSyncedAt = currentTimeMillis();". It looks like the code is inconsistent on whether lastSyncedAt is a relative monotonic clock or absolute epoch time.
Yes, I think so, I am going to create a defect regarding it.

>>  I've also attempted to search for all RegisteredSignal objects in the heap, and found 15 of them, but none of them is related to the read-hotness-tracker:1 thread. It is very strange because the stack trace for this thread waits within the object and read-hotness-tracker:1 thread has active "WaitQueue.Signal signal" local variable.. What tool do you use for the heap dump inspection? I use Eclipse Memory Analyzer. It has "threads_overview" view and you can check the values for local variables within a thread stack.
Like this:<image.png>


Regards,
Dmitry



On Fri, 8 Nov 2024 at 11:01, Bowen Song via user <user@cassandra.apache.org> wrote:
Hi Dmitry,
Thank you for helping me.
You are right, that's a monotonic clock. I was definitely wrongly interpreting its value. I got misled by this line of code: "protected volatile long lastSyncedAt = currentTimeMillis();". It looks like the code is inconsistent on whether lastSyncedAt is a relative monotonic clock or absolute epoch time. Please see the attached cassandra.yaml with sensitive information masked out with asterisks. There's no ERROR level log in the system.log other than some guardrail violations (items_per_collection, if it matters) and random and rare network issues such as connection timeout in the few days run up to and during the event. We have the stdout and stderr captured in the systemd journal, and they have been quiet in during that period as well, not a single line got logged. There's 196 CommitLogSegment (more specifically, MemoryMappedSegment) objects in the heap dump. Of these, 194 has cdcState = CONTAINS lastSyncedOffset = 33554432 lastMarkerOffset = 33554432. The other two has cdcState = CONTAINS lastSyncedOffset = 27007818 lastMarkerOffset = 27007818 and cdcState = PERMITTED lastSyncedOffset = 20 lastMarkerOffset = 20 respectively. The CDC index files have been deleted by the CDC consumer, and we don't keep a copy of these files elsewhere, so I cannot check the offsets in them unfortunately. Regarding the WaitQueue, I found the queue to be empty in the heap dump, which means there's no RegisteredSignal objects in the queue at all. The queue is a ConcurrentLinkedQueue object, which has a head and a tail. The head has item=null and next=null, and the tail has item=null and next=head. I've also attempted to search for all RegisteredSignal objects in the heap, and found 15 of them, but none of them is related to the read-hotness-tracker:1 thread. They are for the following threads:
• MemtablePostFlush:1
• AntiEntropyStage:1
• MemtableFlushWriter:910
• COMMIT-LOG-ALLOCATOR
• NativePoolCleaner
• MemtableFlushWriter:911
• Native-Transport-Requests-135
• Native-Transport-Requests-139
• Native-Transport-Requests-132
• Native-Transport-Requests-143
• Native-Transport-Requests-129
• Native-Transport-Requests-140
• Native-Transport-Requests-153
• Native-Transport-Requests-141
• Native-Transport-Requests-137
This looks rather suspicious to me, as the read-hotness-tracker:1 thread is blocked and waiting for a non-existence registered signal.
Cheers,
Bowen
On 07/11/2024 21:30, Dmitry Konstantinov wrote:
Hi Bowen,

lastSyncedAt is updated by taking pollStarted: lastSyncedAt = pollStarted;
where long pollStarted = clock.now();
the logic uses clock from: SyncRunnable sync = new SyncRunnable(preciseTime)
preciseTime by default is SystemClock
clock.now() -> org.apache.cassandra.utils.Clock.Default#nanoTime -> System.nanoTime() so, the field is populated from System.nanoTime() which returns a relative timestamp, which cannot be treated as an absolute time like linux epoch (it is calculated like this: https://github.com/openjdk/jdk/blob/jdk-11%2B20/src/hotspot/os/linux/os_linux.cpp#L1301 )

Can you share your cassandra.yaml file? (with removed IPs, paths and other security sensitive things, of course)
Do you have any exceptions in your system.log related to commit log?
Do you have JVM stdout/stderr captured to a file? Does it have any exceptions? (which can be present there if Cassandra does not catch them properly by itself)

---
How many CommitLogSegment objects do you have in your heap dump?
What are values for the following fields of CommitLogSegment objects?
lastSyncedOffset
lastMarkerOffset
cdcState

Do you have CDC index files written by org.apache.cassandra.db.commitlog.CommitLogSegment#writeCDCIndexFile ?
If you have them - what are the last offsets written there?

Regards,
Dmitry

On Thu, 7 Nov 2024 at 16:56, Bowen Song via user <user@cassandra.apache.org> wrote: The syncComplete is an instance of WaitQueue, and the syncComplete.queue is an instance of ConcurrentLinkedQueue. Surprisingly, the queue is empty. There is no item in the queue's linked list, only the head and tail nodes, each has item=null. The usage of the WaitQueue within the AbstractCommitLogService looks correct to me:
        do
        {
            WaitQueue.Signal signal = context != null ? syncComplete.register(context, Context::stop) : syncComplete.register();
            if (lastSyncedAt < syncTime)
                signal.awaitUninterruptibly();
            else
                signal.cancel();
        }
        while (lastSyncedAt < syncTime);
This matches the example given in the WaitQueue's documentation (which is definitely missing a pair of curly brackets and has bad indentation):
while (!conditionMet())
    Signal s = q.register();
        if (!conditionMet())    // or, perhaps more correctly, !conditionChanged()
            s.await();
        else
            s.cancel();
So, why is the queue empty while the thread is blocked on waiting for the signal? Further digging showed that PeriodicCommitLogService.lastSyncedAt = 1573038452910796, which translates to 6 November 2019 11:07:32.910796, about 5 years ago. Is this suppose to be a wall clock time in milliseconds since 1 January 1970 00:00? Or is it a monotonic clock time that has no meaning out of the context, and I'm just wrongly interpreting its value as a wall clock time?

I've found that the syncComplete.queue is empty, meaning the WaitQueue object believes that there's nothing waiting for the signal, yet the "read-hotness-tracker:1" thread is clearly waiting for it.
On 06/11/2024 13:49, Bowen Song via user 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> 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 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 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?






--
Dmitry Konstantinov



--
Dmitry Konstantinov


On Nov 8, 2024, at 12:22, Dmitry Konstantinov <netud...@gmail.com> wrote:

Hi Bowen,

>> protected volatile long lastSyncedAt = currentTimeMillis();". It looks like the code is inconsistent on whether lastSyncedAt is a relative monotonic clock or absolute epoch time.
Yes, I think so, I am going to create a defect regarding it.

>>  I've also attempted to search for all RegisteredSignal objects in the heap, and found 15 of them, but none of them is related to the read-hotness-tracker:1 thread. It is very strange because the stack trace for this thread waits within the object and read-hotness-tracker:1 thread has active "WaitQueue.Signal signal" local variable.. What tool do you use for the heap dump inspection? I use Eclipse Memory Analyzer. It has "threads_overview" view and you can check the values for local variables within a thread stack.
Like this:
<image.png>

Regards,
Dmitry



On Fri, 8 Nov 2024 at 11:01, Bowen Song via user <user@cassandra.apache.org> wrote:

    Hi Dmitry,

    Thank you for helping me.

    You are right, that's a monotonic clock. I was definitely wrongly
    interpreting its value. I got misled bythis line of code
    
<https://github.com/apache/cassandra/blob/8d91b469afd3fcafef7ef85c10c8acc11703ba2d/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java#L63>:
    "protected volatile long lastSyncedAt = currentTimeMillis();". It
    looks like the code is inconsistent on whether lastSyncedAt is a
    relative monotonic clock or absolute epoch time.

    Please see the attached cassandra.yaml with sensitive information
    masked out with asterisks.

    There's no ERROR level log in the system.log other than some
    guardrail violations (items_per_collection, if it matters) and
    random and rare network issues such as connection timeout in the
    few days run up to and during the event.

    We have the stdout and stderr captured in the systemd journal,
    and they have been quiet in during that period as well, not a
    single line got logged.

    There's 196 CommitLogSegment (more specifically,
    MemoryMappedSegment) objects in the heap dump. Of these, 194 has
    cdcState = CONTAINS lastSyncedOffset = 33554432 lastMarkerOffset
    = 33554432. The other two has cdcState = CONTAINS
    lastSyncedOffset = 27007818 lastMarkerOffset = 27007818 and
    cdcState = PERMITTED lastSyncedOffset = 20 lastMarkerOffset = 20
    respectively.

    The CDC index files have been deleted by the CDC consumer, and we
    don't keep a copy of these files elsewhere, so I cannot check the
    offsets in them unfortunately.

    Regarding the WaitQueue, I found the queue to be empty in the
    heap dump, which means there's no RegisteredSignal objects in the
    queue at all. The queue is a ConcurrentLinkedQueue object, which
    has a head and a tail. The head has item=null and next=null, and
    the tail has item=null and next=head. I've also attempted to
    search for all RegisteredSignal objects in the heap, and found 15
    of them, but none of them is related to the
    read-hotness-tracker:1 thread. They are for the following threads:

      * MemtablePostFlush:1
      * AntiEntropyStage:1
      * MemtableFlushWriter:910
      * COMMIT-LOG-ALLOCATOR
      * NativePoolCleaner
      * MemtableFlushWriter:911
      * Native-Transport-Requests-135
      * Native-Transport-Requests-139
      * Native-Transport-Requests-132
      * Native-Transport-Requests-143
      * Native-Transport-Requests-129
      * Native-Transport-Requests-140
      * Native-Transport-Requests-153
      * Native-Transport-Requests-141
      * Native-Transport-Requests-137

    This looks rather suspicious to me, as the read-hotness-tracker:1
    thread is blocked and waiting for a non-existence registered signal.

    Cheers,
    Bowen

    On 07/11/2024 21:30, Dmitry Konstantinov wrote:
    Hi Bowen,

    lastSyncedAt is updated by taking pollStarted: lastSyncedAt =
    pollStarted;
    where long pollStarted = clock.now();
    the logic uses clock from: SyncRunnable sync = new
    SyncRunnable(preciseTime)
    preciseTime by default is SystemClock
    clock.now() -> org.apache.cassandra.utils.Clock.Default#nanoTime
    -> System.nanoTime()
    so, the field is populated from System.nanoTime() which returns
    a relative timestamp, which cannot be treated as an absolute
    time like linux epoch (it is calculated like this:
    
https://github.com/openjdk/jdk/blob/jdk-11%2B20/src/hotspot/os/linux/os_linux.cpp#L1301)


    Can you share your cassandra.yaml file? (with removed IPs, paths
    and other security sensitive things, of course)
    Do you have any exceptions in your system.log related to commit log?
    Do you have JVM stdout/stderr captured to a file? Does it have
    any exceptions? (which can be present there if Cassandra does
    not catch them properly by itself)

    ---
    How many CommitLogSegment objects do you have in your heap dump?
    What are values for the following fields of CommitLogSegment
    objects?
    lastSyncedOffset
    lastMarkerOffset
    cdcState

    Do you have CDC index files written by
    org.apache.cassandra.db.commitlog.CommitLogSegment#writeCDCIndexFile
    ?
    If you have them - what are the last offsets written there?

    Regards,
    Dmitry

    On Thu, 7 Nov 2024 at 16:56, Bowen Song via user
    <user@cassandra.apache.org> wrote:

        The syncComplete is an instance of WaitQueue, and the
        syncComplete.queue is an instance of ConcurrentLinkedQueue.
        Surprisingly, the queue is empty. There is no item in the
        queue's linked list, only the head and tail nodes, each has
        item=null.

        The usage of the WaitQueue within the
        AbstractCommitLogService looks correct to me:

        do
        {
        WaitQueue.Signal signal = context != null ?
        syncComplete.register(context, Context::stop) :
        syncComplete.register();
        if (lastSyncedAt < syncTime)
        signal.awaitUninterruptibly();
        else
        signal.cancel();
        }
        while (lastSyncedAt < syncTime);

        This matches the example given in the WaitQueue's
        documentation (which is definitely missing a pair of curly
        brackets and has bad indentation):

        while (!conditionMet())
        Signal s = q.register();
        if (!conditionMet())    // or, perhaps more correctly,
        !conditionChanged()
        s.await();
        else
        s.cancel();

        So, why is the queue empty while the thread is blocked on
        waiting for the signal?

        Further digging showed that
        PeriodicCommitLogService.lastSyncedAt = 1573038452910796,
        which translates to 6 November 2019 11:07:32.910796, about 5
        years ago.

        Is this suppose to be a wall clock time in milliseconds
        since 1 January 1970 00:00? Or is it a monotonic clock time
        that has no meaning out of the context, and I'm just wrongly
        interpreting its value as a wall clock time?


        I've found that the syncComplete.queue is empty, meaning the
        WaitQueue object believes that there's nothing waiting for
        the signal, yet the "read-hotness-tracker:1" thread is
        clearly waiting for it.

        On 06/11/2024 13:49, Bowen Song via user 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 theCassandraKeyspaceWriteHandler.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;
        }
        }

        TheKeyspace.writeOrder.start()method call blocks the
        MemtableFlushWriter thread, and then it
        callsaddToCommitLog(mutation)which indirectly
        callsAbstractCommitLogService.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 thesignal.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>
        <mailto: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
        online 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 online 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?






    --
    Dmitry Konstantinov



--
Dmitry Konstantinov

Reply via email to