Hi Jon and Jeff,

First of all sorry for the late reply as I got a bit busy with a production
bug we have discovered in 4.0
https://issues.apache.org/jira/browse/CASSANDRA-17401, please see if you or
someone could prioritize the review of the submitted PR.

Thanks for your valuable feedback!

I just think there's a ton more value in the drivers being able to throttle
requests to deal than server side.

There is no denying that we can do a ton of things on the client side as
far as the rate limiter is concerned, but the server has more fine-grained
indicators, so if we tackle the server-side first, then we will get a lot
of benefits. In an ideal world, we would need rate limiters at various
layers, i.e., at server layer and at the client layers.

Yes, I agree using dropped metrics (errors) is useful, as well as queue
length.  I can't remember offhand all the details of the request queue and
how load shedding works there, I need to go back and look.  If we don't
already have load shedding based on queue depth that seems like an easy
thing to do immediately, and is a high quality signal.  Maybe someone can
remind me if we have that already?

The Cassandra queues, especially, pending reads, mutations, and transport
queues usually indicate some serious slowdowns.

My issue with using CPU to rate limit clients is that I think it's a very
low quality signal, and I suspect it'll trigger a ton of false positives.

In the rate limiter, CPU is not only going to be the only source, but it
will be one of the key indicators. Along with the CPU, we will consider
Cassandra internal queues as well and then make a decision on whether a
server node is running hot or not.

Again, the "Generic" name I have given of this project might be
misleading. This is not the full and final rate limiter implementation for
Cassandra, but one step towards protecting the Cassandra server due to the
slowdowns caused by the clients, such as pushing 10x load, reading/writing
large partitions, tombstones, etc. On purpose, in this design, we are not
touching system activities such as compaction, system traffic, etc. because
they are essential for Cassandra server to stay alive. For sure, we can
enhance this framework in future and make it more smarter and smarter so
that it will be able to predict the cause of the slowdowns and then apply
back pressure depending upon the situations, such as user traffic or
compaction or repair, etc.
But in the first phase, the target is the slowdowns caused by client
traffic, and especially avoid ripple effect on the entire Cassandra server
ring, e.g., a few hot nodes eventually slowing down the entire cluster.

Jaydeep



On Thu, Jan 18, 2024 at 8:24 PM Jon Haddad <j...@jonhaddad.com> wrote:

> > The problem with generalizing things is if you’re behind on compaction,
> reads get expensive, so you pause compaction completely, you’re SOL and
> you’ll eventually have to throttle traffic to recover
>
> Yeah - there's definitely quite a few ways this can go sideways, and this
> is a good example that won't be consistent across deployments.  There's a
> lot of variables to consider.  I agree that building the machinery for
> operators to make adjustments is the right first step.  Assuming we get all
> the rate limiting options available over CQL and stats available via
> virtual tables, operators can make whatever decisions they feel is best,
> and we'd hopefully get some good feedback about what works well and what
> doesn't.
>
> Jon
>
>
>
>
> On Thu, Jan 18, 2024 at 4:16 PM Jeff Jirsa <jji...@gmail.com> wrote:
>
>> The problem with generalizing things is if you’re behind on compaction,
>> reads get expensive, so you pause compaction completely, you’re SOL and
>> you’ll eventually have to throttle traffic to recover
>>
>> The SEDA model is bad at back pressure and deferred cost makes it
>> non-obvious which resource to slow to ensure stability
>>
>> Just start by exposing it instead of pretending we can outsmart the very
>> complex system
>>
>> On Jan 18, 2024, at 4:56 PM, Jon Haddad <j...@jonhaddad.com> wrote:
>>
>> 
>> I am definitely +1 on the ability to rate limit operations to tables and
>> keyspaces, and if we can do it at a granular level per user I'm +1 to that
>> as well.  I think this would need to be exposed to the operator regardless
>> of any automatic rate limiter.
>>
>> Thinking about the bigger picture for a minute, I think there's a few
>> things we could throttle dynamically on the server before limiting the
>> client requests.  I've long wanted to see a dynamic rate limiter with
>> compaction and any streaming operation - using resources when they're
>> available but slowing down to allow an influx of requests.  Being able to
>> throttle background operations to free up resources to ensure the DB stays
>> online and healthy would be a big win.
>>
>> > The major challenge with latency based rate limiters is that the
>> latency is subjective from one workload to another.
>>
>> You're absolutely right.  This goes to my other suggestion that
>> client-side rate limiting would be a higher priority (on my list at least)
>> as it is perfectly suited for multiple varying workloads.  Of course, if
>> you're not interested in working on the drivers and only on C* itself, this
>> is a moot point.  You're free to work on whatever you want - I just think
>> there's a ton more value in the drivers being able to throttle requests to
>> deal than server side.
>>
>> > And if these two are +ve then consider the server under pressure. And
>> once it is under the pressure, then shed the traffic from less aggressive
>> to more aggressive, etc. The idea is to prevent Cassandra server from
>> melting (by considering the above two signals to begin with and add any
>> more based on the learnings)
>>
>> Yes, I agree using dropped metrics (errors) is useful, as well as queue
>> length.  I can't remember offhand all the details of the request queue and
>> how load shedding works there, I need to go back and look.  If we don't
>> already have load shedding based on queue depth that seems like an easy
>> thing to do immediately, and is a high quality signal.  Maybe someone can
>> remind me if we have that already?
>>
>> My issue with using CPU to rate limit clients is that I think it's a very
>> low quality signal, and I suspect it'll trigger a ton of false positives.
>> For example, there's a big difference from performance being impacted by
>> repair vs large reads vs backing up a snapshot to an object store, but they
>> have similar effects on the CPU - high I/O, high CPU usage, both sustained
>> over time.  Imo it would be a pretty bad decision to throttle clients when
>> we should be throttling repair instead, and we should only do so if it's
>> actually causing an issue for the client, something CPU usage can't tell
>> us, only the response time and error rates can.
>>
>> In the case of a backup, throttling might make sense, or might not, it
>> really depends on the environment and if backups are happening
>> concurrently.  If a backup's configured with nice +19 (as it should be),
>> I'd consider throttling user requests to be a false positive, potentially
>> one that does more harm than good to the cluster, since the OS should be
>> deprioritizing the backup for us rather than us deprioritizing C*.
>>
>> In my ideal world, if C* detected problematic response times (possibly
>> violating a per-table, target latency time) or query timeouts, it would
>> start by throttling back compactions, repairs, and streaming to ensure
>> client requests can be serviced.  I think we'd need to define the latency
>> targets in order for this to work optimally, b/c you might not want to wait
>> for query timeouts before you throttle.  I think there's a lot of value in
>> dynamically adaptive compaction, repair, and streaming since it would
>> prioritize user requests, but again, if you're not willing to work on that,
>> it's your call.
>>
>> Anyways - I like the idea of putting more safeguards in the database
>> itself, we're fundamentally in agreement there.  I see a ton of value in
>> having flexible rate limiters, whether it be per-table, keyspace, or
>> user+table combination.  I'd also like to ensure the feature doesn't cause
>> more disruptions than it solves, which I think would be the case from using
>> CPU usage as a signal.
>>
>> Jon
>>
>>
>> On Wed, Jan 17, 2024 at 10:26 AM Jaydeep Chovatia <
>> chovatia.jayd...@gmail.com> wrote:
>>
>>> Jon,
>>>
>>> The major challenge with latency based rate limiters is that the latency
>>> is subjective from one workload to another. As a result, in the proposal I
>>> have described, the idea is to make decision on the following combinations:
>>>
>>>    1. System parameters (such as CPU usage, etc.)
>>>    2. Cassandra thread pools health (are they dropping requests, etc.)
>>>
>>> And if these two are +ve then consider the server under pressure. And
>>> once it is under the pressure, then shed the traffic from less aggressive
>>> to more aggressive, etc. The idea is to prevent Cassandra server from
>>> melting (by considering the above two signals to begin with and add any
>>> more based on the learnings)
>>>
>>> Scott,
>>>
>>> Yes, I did look at some of the implementations, but they are all great
>>> systems and helping quite a lot. But they are still not relying on system
>>> health, etc. and also not in the generic coordinator/replication read/write
>>> path. The idea here is on the similar lines as the existing
>>> implementations, but making it a bit more generic and trying to cover as
>>> many paths as possible.
>>>
>>> German,
>>>
>>> Sure, let's first continue the discussions here. If it turns out that
>>> there is no widespread interest in the idea then we can do 1:1 and see how
>>> we can help each other on a private fork, etc.
>>>
>>> Jaydeep
>>>
>>> On Wed, Jan 17, 2024 at 7:57 AM German Eichberger via dev <
>>> dev@cassandra.apache.org> wrote:
>>>
>>>> Jaydeep,
>>>>
>>>> I concur with Stefan that extensibility of this  should be a design
>>>> goal:
>>>>
>>>>    - It should be easy to add additional metrics (e.g. write queue
>>>>    depth) and decision logic
>>>>    - There should be a way to interact with other systems to signal a
>>>>    resource need  which then could kick off things like scaling
>>>>
>>>>
>>>> Super interested in this and we have been thinking about siimilar
>>>> things internally 😉
>>>>
>>>> Thanks,
>>>> German
>>>> ------------------------------
>>>> *From:* Jaydeep Chovatia <chovatia.jayd...@gmail.com>
>>>> *Sent:* Tuesday, January 16, 2024 1:16 PM
>>>> *To:* dev@cassandra.apache.org <dev@cassandra.apache.org>
>>>> *Subject:* [EXTERNAL] Re: [Discuss] Generic Purpose Rate Limiter in
>>>> Cassandra
>>>>
>>>> You don't often get email from chovatia.jayd...@gmail.com. Learn why
>>>> this is important <https://aka.ms/LearnAboutSenderIdentification>
>>>> Hi Stefan,
>>>>
>>>> Please find my response below:
>>>> 1) Currently, I am keeping the signals as interface, so one can
>>>> override with a different implementation, but a point noted that even the
>>>> interface APIs could be also made dynamic so one can define APIs and its
>>>> implementation, if they wish to override.
>>>> 2) I've not looked into that yet, but I will look into it and see if it
>>>> can be easily integrated into the Guardrails framework.
>>>> 3) On the server side, when the framework detects that a node is
>>>> overloaded, then it will throw *OverloadedException* back to the
>>>> client. Because if the node while busy continues to serve additional
>>>> requests, then it will slow down other peer nodes due to dependencies on
>>>> meeting the QUORUM, etc. In this, we are at least preventing server nodes
>>>> from melting down, and giving the control to the client via
>>>> *OverloadedException.* Now, it will be up to the client policy, if
>>>> client wishes to retry immediately on a different server node then
>>>> eventually that server node might be impacted, but if client wishes to do
>>>> exponential back off or throw exception back to the application then that
>>>> server node will not be impacted.
>>>>
>>>>
>>>> Jaydeep
>>>>
>>>> On Tue, Jan 16, 2024 at 10:03 AM Štefan Miklošovič <
>>>> stefan.mikloso...@gmail.com> wrote:
>>>>
>>>> Hi Jaydeep,
>>>>
>>>> That seems quite interesting. Couple points though:
>>>>
>>>> 1) It would be nice if there is a way to "subscribe" to decisions your
>>>> detection framework comes up with. Integration with e.g. diagnostics
>>>> subsystem would be beneficial. This should be pluggable - just coding up an
>>>> interface to dump / react on the decisions how I want. This might also act
>>>> as a notifier to other systems, e-mail, slack channels ...
>>>>
>>>> 2) Have you tried to incorporate this with the Guardrails framework? I
>>>> think that if something is detected to be throttled or rejected (e.g
>>>> writing to a table), there might be a guardrail which would be triggered
>>>> dynamically in runtime. Guardrails are useful as such but here we might
>>>> reuse them so we do not need to code it twice.
>>>>
>>>> 3) I am curious how complex this detection framework would be, it can
>>>> be complicated pretty fast I guess. What would be desirable is to act on it
>>>> in such a way that you will not put that node under even more pressure. In
>>>> other words, your detection system should work in such a way that there
>>>> will not be any "doom loop" whereby mere throttling of various parts of
>>>> Cassandra you make it even worse for other nodes in the cluster. For
>>>> example, if a particular node starts to be overwhelmed and you detect this
>>>> and requests start to be rejected, is it not possible that Java driver
>>>> would start to see this node as "erroneous" with delayed response time etc
>>>> and it would start to prefer other nodes in the cluster when deciding what
>>>> node to contact for query coordination? So you would put more load on other
>>>> nodes, making them more susceptible to be throttled as well ...
>>>>
>>>> Regards
>>>>
>>>> Stefan Miklosovic
>>>>
>>>> On Tue, Jan 16, 2024 at 6:41 PM Jaydeep Chovatia <
>>>> chovatia.jayd...@gmail.com> wrote:
>>>>
>>>> Hi,
>>>>
>>>> Happy New Year!
>>>>
>>>> I would like to discuss the following idea:
>>>>
>>>> Open-source Cassandra (CASSANDRA-15013
>>>> <https://issues.apache.org/jira/browse/CASSANDRA-15013>) has an
>>>> elementary built-in memory rate limiter based on the incoming payload from
>>>> user requests. This rate limiter activates if any incoming user request’s
>>>> payload exceeds certain thresholds. However, the existing rate limiter only
>>>> solves limited-scope issues. Cassandra's server-side meltdown due to
>>>> overload is a known problem. Often we see that a couple of busy nodes take
>>>> down the entire Cassandra ring due to the ripple effect. The following
>>>> document proposes a generic purpose comprehensive rate limiter that works
>>>> considering system signals, such as CPU, and internal signals, such as
>>>> thread pools. The rate limiter will have knobs to filter out internal
>>>> traffic, system traffic, replication traffic, and furthermore based on the
>>>> types of queries.
>>>>
>>>> More design details to this doc: [OSS] Cassandra Generic Purpose Rate
>>>> Limiter - Google Docs
>>>> <https://docs.google.com/document/d/1w-A3fnoeBS6tS1ffBda_R0QR90olzFoMqLE7znFEUrQ/edit>
>>>>
>>>> Please let me know your thoughts.
>>>>
>>>> Jaydeep
>>>>
>>>>

Reply via email to