[
https://issues.apache.org/jira/browse/CASSANDRA-7937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14243239#comment-14243239
]
Michaël Figuière commented on CASSANDRA-7937:
---------------------------------------------
bq. That is a good point. However, it's only really useful for reads, since
writes are always sent to all replicas. And unfortunately writes are by far a
bigger problem because of the memory pressure they generate (in queues, as well
as in the memtable). I've never seen a node OOM and fall over from too many
reads.
Indeed for Reads with CL=1 this will bring an appropriate backpressure for each
replica.
For Writes the appropriate backpressure that you'd want to see is the clients
to slow down their rate for all the replicas, that is for the entire partition,
as you don't want to loose it. And we could actually have it with this
mechanism at the Window Size of each of the replicas would be reduced due to
the heavy load they experience, and when Token Awareness is enabled on the
client, it could avoid balancing to another node when reaching the maximum
allowed concurrent requests threshold for each Replica, if configured to do so.
Now if the entire cluster starts to be overloaded, this mechanism would make
sure that the clients slow down their traffic, as there's no point in hammering
an already overloaded cluster.
> Apply backpressure gently when overloaded with writes
> -----------------------------------------------------
>
> Key: CASSANDRA-7937
> URL: https://issues.apache.org/jira/browse/CASSANDRA-7937
> Project: Cassandra
> Issue Type: Bug
> Components: Core
> Environment: Cassandra 2.0
> Reporter: Piotr Kołaczkowski
> Labels: performance
>
> When writing huge amounts of data into C* cluster from analytic tools like
> Hadoop or Apache Spark, we can see that often C* can't keep up with the load.
> This is because analytic tools typically write data "as fast as they can" in
> parallel, from many nodes and they are not artificially rate-limited, so C*
> is the bottleneck here. Also, increasing the number of nodes doesn't really
> help, because in a collocated setup this also increases number of
> Hadoop/Spark nodes (writers) and although possible write performance is
> higher, the problem still remains.
> We observe the following behavior:
> 1. data is ingested at an extreme fast pace into memtables and flush queue
> fills up
> 2. the available memory limit for memtables is reached and writes are no
> longer accepted
> 3. the application gets hit by "write timeout", and retries repeatedly, in
> vain
> 4. after several failed attempts to write, the job gets aborted
> Desired behaviour:
> 1. data is ingested at an extreme fast pace into memtables and flush queue
> fills up
> 2. after exceeding some memtable "fill threshold", C* applies adaptive rate
> limiting to writes - the more the buffers are filled-up, the less writes/s
> are accepted, however writes still occur within the write timeout.
> 3. thanks to slowed down data ingestion, now flush can finish before all the
> memory gets used
> Of course the details how rate limiting could be done are up for a discussion.
> It may be also worth considering putting such logic into the driver, not C*
> core, but then C* needs to expose at least the following information to the
> driver, so we could calculate the desired maximum data rate:
> 1. current amount of memory available for writes before they would completely
> block
> 2. total amount of data queued to be flushed and flush progress (amount of
> data to flush remaining for the memtable currently being flushed)
> 3. average flush write speed
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)