[
https://issues.apache.org/jira/browse/CASSANDRA-10766?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15038185#comment-15038185
]
Sylvain Lebresne commented on CASSANDRA-10766:
----------------------------------------------
bq. If we throw an exception while trying to flush a memtable, we will never
signal the PostFlush runnable, and since the post flush executor is
single-threaded, a single blocked PostFlush runnable will stop that executor.
I don't seem to be able to find the ticket (and if someone remembers it, please
do share) where that debate happened previously, but just so you know, this is
know, has been debated extensively, and so far there is no clear agreement on
how we should handle that (including whether we should change anything).
The short version is that not being able to flush a memtable is a serious
problem that we don't want to ignore. For instance, we can't consider the
memtable on which the exception happen flushed and we shouldn't reclaim it, but
we should *not* under any circumstance flush another memtable for the same
table, as that would confuse the commit log and we would potentially lose data.
We could theoretically allow other tables to flush, but unless the table on
which the exception occurred is very low traffic, you'll still end up OOMing so
we'd be adding complexity for unclear benefits. So the only other option would
be to completely crash the server when that happens, which has been suggested
but hasn't reached full consensus when it was (I really need to find that
ticket ...).
So really, we need to make sure no exceptions happen during flush but when that
happens, the node is hosed.
> OOM from 22000 memtables
> -------------------------
>
> Key: CASSANDRA-10766
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10766
> Project: Cassandra
> Issue Type: Bug
> Environment: * 32 cores, physical machines
> * 8 SSD drives in raid 0
> * hotspot java 8.60
> * Linux version 3.8.13-98.el6uek.x86_64 (mockbuild@x86-ol6-builder-04) (gcc
> version 4.4.7 20120313 (Red Hat 4.4.7-11)
> * G1 heap with 26 GB heap, 24 concurrent and 24 parallel threads, 500 ms
> pause target
> Reporter: amorton
> Assignee: amorton
> Fix For: 2.1.x, 2.2.x, 3.0.x, 3.x
>
> Attachments: MemtableFlushWriter.txt, SlabPoolCleaner.txt,
> lots-of-memtables.png, thread-dump-flush.txt, thread-dump-full.txt
>
>
> Hi folks, we observed this on a new cassandra 2.1.11 cluster.
> The node ran out of memory, and when it did it had 22,000 + memtables in
> memory with a corresponding number of pending MemtableFlushWriter and
> MemtablePostFlushWriter tasks. We have seen other nodes going OOM, but have
> not looked at them as deeply as this one.
> We also noticed that while there was 8 flush writers configured, there were
> times when only one (or maybe two) were making progress. And that there was
> times when the SlabPoolCleaner was flushing memtables 50 times a second.
> I'll try to put as much info below that makes sense. I have the full system
> logs and a heap dump and can go through them with someone if needed. Please
> ping me on IRC or the usual email if I am not on.
> Below is part of the Heap Dump showing the huge amount of memtables in ram.
> I have also attached copied of the same thread dump:
> * thread-dump-full.txt is the full dump, node that is includes approximately
> 300 shared worker threads.
> * thread-dump-flush.txt is the MemtableFlushWriter and MemtablePostFlush
> threads.
> Note that all the MemtableFlushWriter threads are trying to update the
> DataTracker, and that the MemtablePostFlush is waiting on the countdown latch
> to be set. The PostFlush object in the MemtablePostFlush thread is associated
> with the MemtableFlushWriter:744 thread.
> Looking at the MemtableFlushWriter threads they are trying to flush the
> following tables:
> {noformat}
> java.lang.Thread [Thread, Stack Local] "MemtableFlushWriter:750" 266120 120
> system.hints
> java.lang.Thread [Thread, Stack Local] "MemtableFlushWriter:744" 223864 120
> system.sstable_activity
> java.lang.Thread [Thread, Stack Local] "MemtableFlushWriter:752" 220848 120
> system.sstable_activity
> java.lang.Thread [Thread, Stack Local] "MemtableFlushWriter:754" 219736 120
> system.sstable_activity
> java.lang.Thread [Thread, Stack Local] "MemtableFlushWriter:749" 214336 120
> system.sstable_activity
> java.lang.Thread [Thread, Stack Local] "MemtableFlushWriter:753" 211632 120
> system.sstable_activity
> java.lang.Thread [Thread, Stack Local] "MemtableFlushWriter:748" 149616 120
> system.sstable_activity
> java.lang.Thread [Thread, Stack Local] "MemtableFlushWriter:751" 143480 120
> system.sstable_activity
> {noformat}
> When I saw that these were all on the same table I was wondering if the CAS
> in {{DataTracker.replaceFlushed()}} was starving some threads.
> The attached SlabPoolCleaner.txt file is the lines from the SlabPoolCleaner
> thread from a couple of system.log files. Notes:
> * The machine went OOM at 08:38:25,463 server time.
> * The server was flushing user tables between 100 and 200 MB until approx
> 2015-11-22 07:40:39,669 server time.
> * After that the server was flushing files that were 10's of bytes.
> The attached MemtableFlushWriter.txt includes log lines from the
> MemtableFlushWriter threads. Notes:
> * from approx 2015-11-22 07:41:18 it often looks like only one
> MemtableFlushWriter thread progresses at a time.
> A few other notes:
> * This machine has logged a few dozen LEAK DETECTED messages.
> * This has logged messages about corrupted SSTables.
> At this point I am unsure if the SlabPoolCleaner was reacting to low memory
> and correctly flushing, or it was incorrectly flushing a lot which overloaded
> the flush system. We are also unsure what role the CAS in
> {{DataTracker.replaceFlushed()}} played, all the MemtableFlushWriter threads
> with the same progress seems odd.
> As a test for the CAS theory we ran the node with {{memtable_flush_writers}}
> set to 2, it has been running for approx 12 hours under load and has not
> OOM'd yet. This is a good sign.
> Any thoughts on what could be happening ?
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)