[jira] [Commented] (CASSANDRA-7937) Apply backpressure gently when overloaded with writes
[ https://issues.apache.org/jira/browse/CASSANDRA-7937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15344961#comment-15344961 ] Sergio Bossa commented on CASSANDRA-7937: - [~weideng], see: https://issues.apache.org/jira/browse/CASSANDRA-9318?focusedCommentId=15344958&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15344958 > Apply backpressure gently when overloaded with writes > - > > Key: CASSANDRA-7937 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7937 > Project: Cassandra > Issue Type: Improvement > 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)
[jira] [Commented] (CASSANDRA-7937) Apply backpressure gently when overloaded with writes
[ https://issues.apache.org/jira/browse/CASSANDRA-7937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15328674#comment-15328674 ] Ariel Weisberg commented on CASSANDRA-7937: --- Looking at this a little harder it looks like 3.0 might be better off than later versions because of changes that were part of CASSANDRA-6696 which appear to be what reduced the number of concurrent flushes that could occur. [See this change|https://github.com/apache/cassandra/commit/e2c6341898fa43b0e262ef031f267587050b8d0f#diff-98f5acb96aa6d684781936c141132e2aR121] which was actually a surprise to me because I thought it worked the old way. [~krummas] > Apply backpressure gently when overloaded with writes > - > > Key: CASSANDRA-7937 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7937 > Project: Cassandra > Issue Type: Improvement > 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)
[jira] [Commented] (CASSANDRA-7937) Apply backpressure gently when overloaded with writes
[ https://issues.apache.org/jira/browse/CASSANDRA-7937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15328373#comment-15328373 ] Ariel Weisberg commented on CASSANDRA-7937: --- I think we can make this situation better, and I mentioned some ideas at NGCC and in CASSANDRA-11327. There are two issues. The first is that if flushing falls behind throughput falls to zero instead of progressing at the rate at which flushing progresses which is usually not zero. Right now it looks like it is zero because flushing doesn't release any memory as it progresses and is all or nothing. Aleksey mentioned we could do something like early opening for flushing so that memory is made available sooner. Alternatively we could overcommit and then gradually release memory as flushing progresses. The second is that flushing falls behind in several reasonable configurations. Ingest has gotten faster and I don't think flushing has as much so it's easier for it to fall behind if it's driven by a single thread against a busy device (even a fast SSD). I haven't tested this yet, but I suspect that if you use multiple JBOD paths for a fast device like an SSD and increase memtable_flush_writers you will get enough additional flushing throughput to keep up with ingest. Right now flushing is single threaded for a single path and only one flush can occur at any time. Flushing falling behind is more noticeable when you let compaction have more threads and a bigger rate limit because it can dirty enough memory in the filesystem cache that when it flushes it causes a temporally localized slowdown in flushing that is enough to cause timeouts when there is no more free memory because flushing didn't finish soon enough. I think the long term solution is that the further flushing falls behind the more concurrent flush threads we start deploying kind of like compaction up to the configured limit. [Right now there is a single thread scheduling them and waiting on the result.|https://github.com/apache/cassandra/blob/cassandra-3.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java#L1130] memtable_flush_writers doesn't help due to this code here that only [generates more flush runnables for a memtable if there are multiple directories|https://github.com/apache/cassandra/blob/cassandra-3.7/src/java/org/apache/cassandra/db/Memtable.java#L278]. C* is already divvying up the heap using memtable_cleanup_threshold which would allow for concurrent flushing it's just not actually flushing concurrently. > Apply backpressure gently when overloaded with writes > - > > Key: CASSANDRA-7937 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7937 > Project: Cassandra > Issue Type: Improvement > 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
[jira] [Commented] (CASSANDRA-7937) Apply backpressure gently when overloaded with writes
[ https://issues.apache.org/jira/browse/CASSANDRA-7937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15328071#comment-15328071 ] Jonathan Ellis commented on CASSANDRA-7937: --- Here is where 9318 ended up: bq. I found some ways to OOM the server (CASSANDRA-10971 and CASSANDRA-10972) and have patches out for those. bq. The # of in flight requests already has bounds depending on the bottleneck that prevent the server from crashing so adding an explicit one isn't useful right now. When TPC is implemented we will have to implement a bound since there is no thread pool to exhaust, but that is later work. I don't think adding more band-aids to SEDA is going to be useful. > Apply backpressure gently when overloaded with writes > - > > Key: CASSANDRA-7937 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7937 > Project: Cassandra > Issue Type: Improvement > 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)
[jira] [Commented] (CASSANDRA-7937) Apply backpressure gently when overloaded with writes
[ https://issues.apache.org/jira/browse/CASSANDRA-7937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15328023#comment-15328023 ] Wei Deng commented on CASSANDRA-7937: - [~jbellis] Since CASSANDRA-9318 turned out to be an ineffective approach (and closed as WONT-FIX), should we reopen this JIRA? > Apply backpressure gently when overloaded with writes > - > > Key: CASSANDRA-7937 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7937 > Project: Cassandra > Issue Type: Improvement > 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)
[jira] [Commented] (CASSANDRA-7937) Apply backpressure gently when overloaded with writes
[ https://issues.apache.org/jira/browse/CASSANDRA-7937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14638908#comment-14638908 ] Jonathan Ellis commented on CASSANDRA-7937: --- I think we can do as well with a simpler approach by using MessagingService queues as a proxy for target's load. (If the target is overwhelmed it will read slower from the socket and our queue will not drain; if it is not more-than-usually-overwhelmed but clients are sending us so many requests for that target that we still can't drain it fast enough, then we should also pause accepting extra requests.) See CASSANDRA-9318 and in particular my summary [here|https://issues.apache.org/jira/browse/CASSANDRA-9318?focusedCommentId=14604649&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14604649]. (NB feel free to reassign to Jacek if he has free cycles.) > Apply backpressure gently when overloaded with writes > - > > Key: CASSANDRA-7937 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7937 > Project: Cassandra > Issue Type: Improvement > Components: Core > Environment: Cassandra 2.0 >Reporter: Piotr Kołaczkowski >Assignee: Jacek Lewandowski > 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)
[jira] [Commented] (CASSANDRA-7937) Apply backpressure gently when overloaded with writes
[ https://issues.apache.org/jira/browse/CASSANDRA-7937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14259967#comment-14259967 ] Piotr Kołaczkowski commented on CASSANDRA-7937: --- I like the idea of using parallelism level as a backpressure mechanism. That would have a nice positive effect of automatically reducing the amount of memory used for queuing the requests. However, my biggest concern is, that even limiting a single client to one write at a time (window size = 1), might still be too fast, for some fast clients, if only row sizes are big enough, particularly when writing big cells of data, where big = hundreds of kB / single MBs per cell. Cassandra is extremely efficient at ingesting data into memtables. If it was faster than we're able to flush, then we still have a problem. So I guess if, after going down to parallelism level = 1 and still being too fast (e.g. flush queue full, last memtable almost full), we could tell the client a message "please do not send data faster than X MB/s now" and the client (driver) could do some artificial delay before processing the next request. > 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)
[jira] [Commented] (CASSANDRA-7937) Apply backpressure gently when overloaded with writes
[ 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)
[jira] [Commented] (CASSANDRA-7937) Apply backpressure gently when overloaded with writes
[ https://issues.apache.org/jira/browse/CASSANDRA-7937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14243181#comment-14243181 ] Jonathan Ellis commented on CASSANDRA-7937: --- bq. So in this situation the backpressure information received by the client could be used properly, as it would just be understood by the client as a request to slow down for this particular replica, it could therefore pick another replica. 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. > 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)
[jira] [Commented] (CASSANDRA-7937) Apply backpressure gently when overloaded with writes
[ https://issues.apache.org/jira/browse/CASSANDRA-7937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14242880#comment-14242880 ] Michaël Figuière commented on CASSANDRA-7937: - The StreamIDs, introduced in the native protocol to multiplex several pending requests on a single connection, could actually serve as a backpressure mechanism. Before protocol v2 we had just 128 IDs per connection with drivers typically allowing just a few connection per node. This therefore already acts as a throttling mechanism on the client side. With protocol v3 we've increased this limit but the driver still let the user define a value for the max requests per host that will have the same effect. A simple way the handle backpressure could therefore be to introduce a Window (similar as TCP Window) of the currently allowed concurrent requests for each client. Just like in TCP, the Window Size could be included in each response header to the client. This Window Size could then be adjusted using a magic formula to define, probably based on the load of each Stage of the Cassandra architecture, state of compaction, etc... I agree with [~jbellis]'s point: backpressure in a distributed system like Cassandra, with a coordinator fowarding traffic to replicas, is confusing. But in practice, most recent CQL Drivers now do Token Aware Balancing by default (since 2.0.2 in the Java Driver), which will send the queries to the replicas any PreparedStatement (expected to be used under the high pressure condition described here). So in this situation the backpressure information received by the client could be used properly, as it would just be understood by the client as a request to slow down for *this* particular replica, it could therefore pick another replica. Thus we end up with a system in which we avoid doing Load Shedding (which is a waste of time, bandwidth and workload) and that, I believe, could behave more smoothly when the cluster is overloaded. Note that this StreamID Window could be considered as a "mandatory" limit or just as a "hint" in the protocol specification. The driver could then adjust its strategy to use it or not depending on the settings or type of request. > 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)
[jira] [Commented] (CASSANDRA-7937) Apply backpressure gently when overloaded with writes
[ https://issues.apache.org/jira/browse/CASSANDRA-7937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14136794#comment-14136794 ] Benedict commented on CASSANDRA-7937: - Under these workload conditions, CASSANDRA-3852 is unlikely to help; it is more for reads. CASSANDRA-6812 will help for writes. It in no way affects the concept of load shedding or backpressure, just write latency spikes and hence timeouts. There are some issues with load shedding by itself, though, at least as it stands in Cassandra. Take your example cluster: A..E, with RF=3. Let's say B _and_ C are now load shedding. The result is that we see 1/5th of requests queue up and block in A,D and E, which within a short window results in all requests to those servers being blocked, as all rpc threads are waiting for results that will never come. Now, with explicit back pressure (whether it be blocking consumption of network messages or signalling the coordinator we are load shedding) we could detect this problem and start dropping those requests on the floor in A, D and E and continue to serve other requests. FTR, we do already have an accidental back pressure system built into Incoming/OutboundTcpConnection, with only negative consequences. If the remote server is blocked due to GC (or some other event), it will not consume its IncomingTcpConnection which will cause our coordinator's queue to that node to back up. This is something we should deal with, and introducing explicit back pressure of the same kind when a node cannot cope with its load would allow us to deal with both situations using the same mechanism. > 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)
[jira] [Commented] (CASSANDRA-7937) Apply backpressure gently when overloaded with writes
[ https://issues.apache.org/jira/browse/CASSANDRA-7937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14136389#comment-14136389 ] Jonathan Ellis commented on CASSANDRA-7937: --- Now, I do agree that it would be nice to make progress on some requests instead of dropping all of them when overloaded. Improved timeout can help (CASSANDRA-2858, see also CASSANDRA-4812). CASSANDRA-3852 may also help. > 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)
[jira] [Commented] (CASSANDRA-7937) Apply backpressure gently when overloaded with writes
[ https://issues.apache.org/jira/browse/CASSANDRA-7937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14136381#comment-14136381 ] Jonathan Ellis commented on CASSANDRA-7937: --- How do you backpressure in a distributed system? Suppose a cluster has 5 nodes, A..E. Node B is overloaded. Client is talking to node A. We do not want to restrict requests to CDE simply because B is overloaded. I conclude that load shedding (dropping requests) is a better solution than backpressure for us. > 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)
[jira] [Commented] (CASSANDRA-7937) Apply backpressure gently when overloaded with writes
[ https://issues.apache.org/jira/browse/CASSANDRA-7937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14133861#comment-14133861 ] Piotr Kołaczkowski commented on CASSANDRA-7937: --- Indeed, but it is not yet supported in Spark, also not everyone uses it with Hive/Hadoop. > 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 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 happen 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)
[jira] [Commented] (CASSANDRA-7937) Apply backpressure gently when overloaded with writes
[ https://issues.apache.org/jira/browse/CASSANDRA-7937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14133862#comment-14133862 ] Benedict commented on CASSANDRA-7937: - This should certainly be dealt with by the cluster. We cannot rely on well behaved clients, and clients cannot easily calculate a safe data-rate cross cluster, so any client change would at best help direct writes only, which with RF>1 is not much help. Nor could it be as responsive. My preferred solution to this is CASSANDRA-6812, which should keep the server responding to writes within the timeout window even as it blocks for lengthy flushes, but during these windows writes would be acked much more slowly, at a steady drip. This solution won't make it into 2.0 or 2.1, and possibly not even 3.0, though. > 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 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 happen 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)
[jira] [Commented] (CASSANDRA-7937) Apply backpressure gently when overloaded with writes
[ https://issues.apache.org/jira/browse/CASSANDRA-7937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14133859#comment-14133859 ] Brandon Williams commented on CASSANDRA-7937: - One easy way around this is to use the BulkOutputFormat so it doesn't go through the write path. > 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 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 happen 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)