[Discuss] Generic Purpose Rate Limiter in Cassandra

2024-01-16 Thread Jaydeep Chovatia
Hi,

Happy New Year!

I would like to discuss the following idea:

Open-source Cassandra (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


Please let me know your thoughts.

Jaydeep


Re: [Discuss] Generic Purpose Rate Limiter in Cassandra

2024-01-16 Thread Štefan Miklošovič
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 
wrote:

> Hi,
>
> Happy New Year!
>
> I would like to discuss the following idea:
>
> Open-source Cassandra (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
> 
>
> Please let me know your thoughts.
>
> Jaydeep
>


Re: [Discuss] Generic Purpose Rate Limiter in Cassandra

2024-01-16 Thread Jon Haddad
Server side rate limiting can be useful, but imo if we were to focus effort
into a single place, time would be much better spent adding adaptive rate
limiting to the drivers.

Rate limiting at the driver level can be done based on 2 simple feedback
mechanisms - error rate and latency.  When a node is throwing errors (or
exceeds the defined latency SLO), requests to that node can be rate
limited.  It does a better job of preventing issues than server side rate
limiting as we don't get the DDOS effect in addition to whatever issue the
node is dealing with at the time.  Netflix has a good post on it here [1],
and I've incorporated the latency version into my fork of tlp-stress [2]
for benchmarking.

Adding this to the driver means the DS Spark Connector can also take
advantage of it, which is nice because tuning it to get the
optimal throughput is a bit of a headache.

Regarding the server side, I think the proposal to use various system
metrics is overly complicated.  The only metrics that matter are latency
and error rate.  As long as you're within acceptable thresholds, you don't
need to rate limit.

Jon

[1] https://netflixtechblog.medium.com/performance-under-load-3e6fa9a60581
[2]
https://rustyrazorblade.com/post/2023/2023-10-31-tlp-stress-adaptive-scheduler/

On Tue, Jan 16, 2024 at 10:02 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
>> ) 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
>> 
>>
>> Please let me know your thoughts.
>>
>> Jaydeep
>>
>


Re: [Discuss] Generic Purpose Rate Limiter in Cassandra

2024-01-16 Thread C. Scott Andreas

Jaydeep, thanks for reaching out and for sharing this proposal. I like the direction. Please also take a look at 
https://issues.apache.org/jira/browse/CASSANDRA-16663 , which adds coordinator-level rate limiting on request rate. This ticket 
introduced a lockless rate limiter patterned on an approach in Guava, adapted for Cassandra. This will likely be the rate 
limiter you'd like to actuate via changes in the database metrics your proposal describes. One challenge we've had with the 
rate limiter introduced in C-16663 is that it's difficult to determine what value to set it to. Another aspect of C-16663 that 
I find valuable is that the rate limiter can dispatch notifications to clients that rate limiting is taking effect, implemented 
via client warnings. Users should/need to be able to know that their request rate is being artificially reduced due to 
server-side throttling. Appreciate your thinking along the lines of different metrics that operators may want to trigger on, as 
this will vary per-workload. There are probably extremely read-heavy workloads that push CPU hard but which don't incur 
compaction backlog/other balance-of-humors issues and are comfortable running at high CPU, as an example. I like the idea of 
adaptive request volumes in clients as well and don't see this and Jon's proposal as mutually exclusive. There are many 
different Cassandra clients, including some outside the project written in Go and Rust that are beyond the scope of our direct 
control. Any server should have mechanisms to defend itself against any client; official or non–. Jon's also right that an 
implementation in the Spark Cassandra Connector may help jobs make progress at the maximum safe pace. Thanks for sharing this 
thoughtful document. – Scott On Jan 16, 2024, at 10:46 AM, Jon Haddad  wrote: Server side rate 
limiting can be useful, but imo if we were to focus effort into a single place, time would be much better spent adding adaptive 
rate limiting to the drivers. Rate limiting at the driver level can be done based on 2 simple feedback mechanisms - error rate 
and latency. When a node is throwing errors (or exceeds the defined latency SLO), requests to that node can be rate limited. It 
does a better job of preventing issues than server side rate limiting as we don't get the DDOS effect in addition to whatever 
issue the node is dealing with at the time. Netflix has a good post on it here [1], and I've incorporated the latency version 
into my fork of tlp-stress [2] for benchmarking. Adding this to the driver means the DS Spark Connector can also take advantage 
of it, which is nice because tuning it to get the optimal throughput is a bit of a headache. Regarding the server side, I think 
the proposal to use various system metrics is overly complicated. The only metrics that matter are latency and error rate. As 
long as you're within acceptable thresholds, you don't need to rate limit. Jon [1] 
https://netflixtechblog.medium.com/performance-under-load-3e6fa9a60581 [2] 
https://rustyrazorblade.com/post/2023/2023-10-31-tlp-stress-adaptive-scheduler/ On Tue, Jan 16, 2024 at 10:02 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 

Re: [Discuss] Generic Purpose Rate Limiter in Cassandra

2024-01-16 Thread Jaydeep Chovatia
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
>> ) 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
>> 
>>
>> Please let me know your thoughts.
>>
>> Jaydeep
>>
>


Re: [Discuss] CQLSH should left-align numbers, right-align text (CASSANDRA-19150)

2024-01-16 Thread Brad
Hi Shailaja,

In the case of machine readable output, CQL uses delimited output ('|')
with whitespace on either side of the data values.

To better support machine readable output, it might be useful to allow
user specified delimiters (in a separate Jira). E.g.:

cqlsh *-s","* -e"CAPTURE '/tmp/props.csv';select * from
system_views.system_properties limit 7"

We could make something like that a precondition here. But it is just
whitespace. I'd agree this shouldn't go into a patch release.

On Mon, Jan 15, 2024 at 1:34 PM  wrote:

> Hi Brad,
>
> While I prefer the indentation style that Postgres following for better
> readability of text, if we are changing it, this may break existing scripts
> of users/operators if tightly coupled with the current format/spaces etc
> (Ideally shouldn’t be, but as Cassandra being used all over the world, such
> scenarios are possible). To avoid breaking such existing scripts, I believe
> either these changes need to happen in a major release or under a feature
> flag (which can be deprecated over the time), for existing scripts to
> continue without breaking until they are fixed.
>
> Thanks,
> Shailaja
>
>
> On Jan 9, 2024, at 5:23 PM, Derek Chen-Becker 
> wrote:
>
> Actually, now that I'm looking at the original email on my browser and not
> my phone (and can see the formatting properly), I think we have the
> nomenclature backward here. Left-alignment in the printing world means that
> text in each cell starts at the left-most column for the cell, but in your
> examples you're calling that right-aligned (and vice-versa). Along the
> lines of what Stefan said, I think this probably came about more as a
> "we'll just keep things simple and use the same alignment everywhere"
> rather than an intentional right-alignment of text for a specific purpose.
> I would actually be fine with left-aligning text to fit what appears to be
> standard practice in other systems.
>
> Cheers,
>
> Derek
>
> On Tue, Jan 9, 2024 at 7:34 AM Brad  wrote:
>
>> CQLSH currently left-aligns all output, affecting both numbers and text.
>> While this works well for numbers, a better approach adopted by many is to
>> left align numbers and right align text.
>>
>> For example, both Excel and Postgres shell use the later:
>>
>> psql
>> # select * from employee;
>>  empid |  name   |dept
>> ---+-+
>>  1 | Clark   | Sales
>>200 | Dave| Accounting
>> 33 | Johnson | Sales
>>
>>
>> while CQLSH simply left aligns all the columns
>>
>> cqlsh> select * from employee;
>>
>>  empid | dept   | name
>> ---++-
>> 33 |  Sales | Johnson
>>  1 |  Sales |   Clark
>>200 | Accounting |Dave
>>
>>
>>
>> Left aligned text looks much worse on text values which share common
>> prefixes
>>
>> cqlsh> select * from system_views.system_properties limit 7 ;
>>
>>
>>  name   | value
>>
>> +
>>   JAVA_HOME |
>>   /Users/brad/.jenv/versions/17
>>cassandra.jmx.local.port |
>>   7199
>>cassandra.logdir |
>> /usr/local/cassandra-5.0-beta1/bin/../logs
>>cassandra.storagedir |
>> /usr/local/cassandra-5.0-beta1/bin/../data
>>   com.sun.management.jmxremote.authenticate |
>>   false
>>  com.sun.management.jmxremote.password.file |
>>   /etc/cassandra/jmxremote.password
>> io.netty.transport.estimateSizeOnSubmit |
>>   false
>>
>>
>>
>> The Jira CASSANDRA-19150
>>  discusses this
>> in further detail with some additional examples.
>>
>> I wanted to raise the issue here to propose changing CQLSH to right-align
>> text while continue to left-align numbers.
>>
>> Regards,
>>
>> Brad Schoening
>>
>> ReplyForward
>> Add reaction
>>
>
>
> --
> +---+
> | Derek Chen-Becker |
> | GPG Key available at https://keybase.io/dchenbecker and   |
> | https://pgp.mit.edu/pks/lookup?search=derek%40chen-becker.org |
> | Fngrprnt: EB8A 6480 F0A3 C8EB C1E7  7F42 AFC5 AFEE 96E4 6ACC  |
> +---+
>
>
>