[ 
https://issues.apache.org/jira/browse/CASSANDRA-19564?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17898081#comment-17898081
 ] 

Jaydeepkumar Chovatia edited comment on CASSANDRA-19564 at 11/14/24 1:03 AM:
-----------------------------------------------------------------------------

[~benedict] [~gauravapiscean]

Copying Memtable is going to be a very tricky thing and might lead us to more 
future corner cases, in my opinion. How about a third option, which is not to 
wait indefinitely in {_}MemtableAllocator.java{_}? The major problem is that 
the _MemtableAllocator.java_ thread has been waiting indefinitely to acquire 
the memory.  Instead of having it stay forever, we can introduce a timeout, and 
if it is unsuccessful, then let that operation fail.

That way, when _Memtable_ is almost full, the _MutationStage_ and _Compaction_ 
threads will eventually timeout and unblock *MemtableReclaimMemory.* Once the 
*_MemtableReclaimMemory_*  is unblocked, it will free up more space, and then 
the future _MutationStage_ and _Compaction_ threads will succeed.

In short, bail out the compaction and mutation tasks after some interval to 
break the deadlock. If so, the change to the MemtableAllocator.java would look 
as follows: 
{code:java}
--- a/src/java/org/apache/cassandra/utils/memory/MemtableAllocator.java
+++ b/src/java/org/apache/cassandra/utils/memory/MemtableAllocator.java
@@ -25,9 +25,13 @@ import org.slf4j.LoggerFactory;
 
 import com.codahale.metrics.Timer;
 
+import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
 
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
 public abstract class MemtableAllocator
 {
     private static final Logger logger = 
LoggerFactory.getLogger(MemtableAllocator.class);
@@ -192,7 +196,11 @@ public abstract class MemtableAllocator
                     return;
                 }
                 else
-                    signal.awaitThrowUncheckedOnInterrupt();
+                    if (!signal.awaitUntilThrowUncheckedOnInterrupt(nanoTime() 
+ SECONDS.toNanos(5))) //TODO: introduce a new timeout configuration or use the 
write timeout
+                    {
+                        throw new RuntimeException("Timed out waiting for free 
memory");
+                    }
+
             }
         }
 {code}
wdyt?


was (Author: [email protected]):
[~benedict] [~gauravapiscean]

Copying Memtable is going to be a very tricky thing and might lead us to more 
future corner cases, in my opinion. How about a third option, which is not to 
wait indefinitely in {_}MemtableAllocator.java{_}? The major problem is that 
the _MemtableAllocator.java_ thread has been waiting indefinitely to acquire 
the memory.  Instead of having it stay forever, we can introduce a timeout, and 
if it is unsuccessful, then let that operation fail.

That way, when _Memtable_ is almost full, the _MutationStage_ and _Compaction_ 
threads will eventually timeout and unblock *MemtableReclaimMemory.* Once the 
*_MemtableReclaimMemory_*  is unblocked, it will free up more space, and then 
the future _MutationStage_ and _Compaction_ threads will succeed.

In short, bail out the compaction and mutation tasks after some interval to 
break the deadlock. If so, the change to the MemtableAllocator.java would look 
as follows: 
{code:java}
--- a/src/java/org/apache/cassandra/utils/memory/MemtableAllocator.java
+++ b/src/java/org/apache/cassandra/utils/memory/MemtableAllocator.java
@@ -28,6 +28,10 @@ import com.codahale.metrics.Timer;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
 
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.utils.Clock.Global.nanoTime;
+
 public abstract class MemtableAllocator
 {
     private static final Logger logger = 
LoggerFactory.getLogger(MemtableAllocator.class);
@@ -192,7 +196,7 @@ public abstract class MemtableAllocator
                     return;
                 }
                 else
-                    signal.awaitThrowUncheckedOnInterrupt();
+                    signal.awaitUntilThrowUncheckedOnInterrupt(nanoTime() + 
SECONDS.toNanos(5)); //TODO: introduce a new timeout configuration or use the 
write timeout
             }
         }
{code}
wdyt?

> MemtablePostFlush deadlock leads to stuck nodes and crashes
> -----------------------------------------------------------
>
>                 Key: CASSANDRA-19564
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-19564
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Local/Compaction, Local/Memtable
>            Reporter: Jon Haddad
>            Priority: Urgent
>             Fix For: 4.1.x
>
>         Attachments: image-2024-04-16-11-55-54-750.png, 
> image-2024-04-16-12-29-15-386.png, image-2024-04-16-13-43-11-064.png, 
> image-2024-04-16-13-53-24-455.png, image-2024-04-17-18-46-29-474.png, 
> image-2024-04-17-19-13-06-769.png, image-2024-04-17-19-14-34-344.png, 
> screenshot-1.png
>
>
> 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. 
> {noformat}
> 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
> {noformat}
> *MemtablePostFlush appears to be blocked*
> At this point, MemtablePostFlush completed tasks stops incrementing, active 
> stays at 1 and pending starts to rise.
> {noformat}
> MemtablePostFlush   1    1   3446   0   0
> {noformat}
>  
> 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:
> *!image-2024-04-16-13-43-11-064.png!*
> *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.
> !image-2024-04-16-12-29-15-386.png!
> *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()}}
> {noformat}
>  MutationStage    48    828425      980253369      0    0{noformat}
> !image-2024-04-16-11-55-54-750.png!
>  
> *Compaction Stops*
> Since we write to the compaction history table, and that requires memtables, 
> compactions are now blocked as well.
>  
> !image-2024-04-16-13-53-24-455.png!
>  
> The node is now doing basically nothing and must be restarted.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to