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<mailto: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<mailto: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