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

Peter Schuller commented on CASSANDRA-1955:
-------------------------------------------

Heh, I'm aware of that ;) Let me try to make myself clearer. Maybe I'm missing 
something which means this is not a problem, but to establish that I first have 
to make sure I'm making myself understandable.

You definitely want to limit the amount of heap space used for memtables, no 
problem there. The problem, if I have understood things correctly, is that the 
amount of memtables pending flush (i.e., switched away from + actively being 
written) is limited by the flush writer concurrency and the queue length. The 
flush writer concurrency being tweaked with respect to I/O concerns.

So, given a keyspace with many column families the problem is one of timing. 
Suppose you have 10 column families that are all written at a reasonable pace, 
but they all end up triggering a memtable switch at the same time (this is what 
I interpreted the OP's situation as on the mailing list), you get a sudden 
spike of memtable flushes that is independent of the actual write throughput. 
If this spike or peak in the number of pending memtables is higher than queue 
length + concurrency, you suddenly block on writes, even though you were never 
even close to saturating the write capacity of the node/cluster.

Does this make sense or have I misunderstood how the flush writer executor 
interacts with the memtable flushing process?



> memtable flushing can block writes due to queue size limitations even though 
> overall write throughput is below capacity
> -----------------------------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-1955
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1955
>             Project: Cassandra
>          Issue Type: Improvement
>            Reporter: Peter Schuller
>
> It seems that someone ran into this (see cassandra-user thread "Question re: 
> the use of multiple ColumnFamilies").
> If my interpretation is correct, the queue size is set to the concurrency in 
> the case of the flushSorter, and set to memtable_flush_writers in the case of 
> flushWriter in ColumnFamilyStore.
> While the choice of concurrency for the two executors makes perfect sense, 
> the queue sizing does not. As a user, I would expect, and did expect, that 
> for a given memtable independently tuned (w.r.t. flushing thresholds etc), 
> writes to the CF would not block until there is at least one other memtable 
> *for that CF* waiting to be flushed.
> With the current behavior, if I am not misinterpreting, whether or not writes 
> will inappropriately block is very much dependent on not just the overall 
> write throughput, but also the incidental timing of memtable flushes across 
> multiple column families.
> The simplest way to mitigate (but not fix) this is probably to set the queue 
> size to be equal to the number of column families if that is higher than the 
> number of CPU cores. But that is only a mitigation because nothing prevents 
> e.g. a large number of memtable flushes for a small column family under 
> temporary write load, can still block a large (possibly more important) 
> memtable flush for another CF. Such a shared-but-larger queue would also not 
> prevent heap usage spikes resulting from some a single cf with very large 
> memtable thresholds being rapidly written to, with a queue sized for lots of 
> cf:s that are in practice not used. In other words, this mitigation technique 
> would effectively negate the backpressure mechanism in some cases and likely 
> lead to more people having OOM issues when saturating a CF with writes.
> A more involved change is to make each CF have it's own queue through which 
> flushes go prior to being submitted to flushSorter, which would guarantee 
> that at least one memtable can always be in pending flush state for a given 
> CF. The global queue could effectively have size 1 hard-coded since the queue 
> is no longer really used as if it were a queue.
> The flushWriter is unaffected since it is a separate concern that is supposed 
> to be I/O bound. The current behavior would not be perfect if there is a huge 
> discrepancy between memtable flush thresholds of different memtables, but it 
> does not seem high priority to make a change here in practice.
> So, I propose either:
> (a) changing the flushSorter queue size to be max(num cores, num cfs)
> (b) creating a per-cf queue
> I'll volunteer to work on it as a nice bite sized change, assuming there is 
> agreement on what needs to be done. Given the concerns with (a), I think (b) 
> is the right solution unless it turns out to cause major complexity. Worth 
> noting is that these are not performance sensitive given the low frequency of 
> memtable flushes, so an extra queue:ing step should not be an issue.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to