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
<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?