[ 
https://issues.apache.org/jira/browse/CASSANDRA-16663?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Caleb Rackliffe updated CASSANDRA-16663:
----------------------------------------
    Test and Documentation Plan: 
There are two new `cassandra.yaml` options in this patch:

{{native_transport_rate_limiting_enabled}} - Whether or not to apply a rate 
limit on CQL requests. (Default: false)

{{native_transport_requests_per_second}} - The limit itself. (Default: 
Double.MAX_VALUE)

In terms of testing, there is a new suite of overload tests in 
{{RateLimitingTest}}, and this provides at least basic coverage along the major 
axes of message size, back-pressure vs. throw-on-overload, and behavior in the 
face of configuration changes at runtime (although part of that last bit is in 
{{ClientResourceLimitsTest}}).

As this moves into review, I've also used {{SimpleClientPerfTest}} across 
small/large messages and all 4 active protocol versions to make sure the 
limiter maintains throughput in a reasonably tight band around the configured 
limit. More stress testing, likely w/ tlp-stress and with multi-node clusters, 
will happen concurrently w/ review.
                         Status: Patch Available  (was: In Progress)

Moving this into review.

[trunk patch|https://github.com/apache/cassandra/pull/1045]
[j8 
tests|https://app.circleci.com/pipelines/github/maedhroz/cassandra/271/workflows/c54f4796-22ec-4619-acdb-8c9f8409b37f]
[j11 
tests|https://app.circleci.com/pipelines/github/maedhroz/cassandra/271/workflows/9caf2b96-d0ce-4e59-a6c0-caba28b561f0]

In addition to the test and documentation notes above and some minor inline 
notes in the PR, there are a couple things that could qualify as outstanding 
questions. First, for V5+, we attempt to acquire from the rate limiter before 
asking for resource from the "concurrent bytes in flight" pool. When we are 
below the rate limiting threshold but somehow above the bytes threshold, the 
patch as it now stands does not "give a permit back" to the rate limiter. This 
means we count dropped requests against the rate limit, and perhaps isn't ideal 
if we'd rather maintain the rate of _successful_ requests at the configured 
limit instead. (This may be a pretty rare case though.)

Second, there is a subtle difference in how V4 and V5 deal with messages that 
breach the back-pressure threshold. In V4 (and below) requests that tip over 
the threshold are allowed to  complete, but in V5+ those requests are not 
processed until we fall back below the threshold (i.e. in the rate-limiting 
case, a permit is available). This is the reason for some [minor 
differences|https://github.com/apache/cassandra/pull/1045/files#diff-85cc40243dd7805ced11a47e02222406d3d2cd56a9b72f3752884599f48f080bR173]
 in the structure of the tests, but in production code, it means in V4 we force 
acquire a permit, while in V5, we acquire only once it is available. (This has 
the effect of leaving the conditions for unpausing/reactivation the same in all 
versions.) In short, I've not modified the existing behavior, but only tailored 
the permit acquisition behavior to fit.

I don't think they should block review, but I'll hopefully be posting some 
results from some basic performance regression testing and testing around the 
limits soon.

> Request-Based Native Transport Rate-Limiting
> --------------------------------------------
>
>                 Key: CASSANDRA-16663
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-16663
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Messaging/Client
>            Reporter: Caleb Rackliffe
>            Assignee: Caleb Rackliffe
>            Priority: Normal
>             Fix For: 4.x
>
>          Time Spent: 2h 50m
>  Remaining Estimate: 0h
>
> Together, CASSANDRA-14855, CASSANDRA-15013, and CASSANDRA-15519 added support 
> for a runtime-configurable, per-coordinator limit on the number of bytes 
> allocated for concurrent requests over the native protocol. It supports 
> channel back-pressure by default, and optionally supports throwing 
> OverloadedException if that is requested in the relevant connection’s STARTUP 
> message.
> This can be an effective tool to prevent the coordinator from running out of 
> memory, but it may not correspond to how expensive a queries are or provide a 
> direct conceptual mapping to how users think about request capacity. I 
> propose adding the option of request-based (or perhaps more correctly 
> message-based) back-pressure, coexisting with (and reusing the logic that 
> supports) the current bytes-based back-pressure.
> _We can roll this forward in phases_, where the server’s cost accounting 
> becomes more accurate, we segment limits by operation type/keyspace/etc., and 
> the client/driver reacts more intelligently to (especially non-back-pressure) 
> overload, _but something minimally viable could look like this_:
> 1.) Reuse most of the existing logic in Limits, et al. to support a simple 
> per-coordinator limit only on native transport requests per second. Under 
> this limit will be CQL reads and writes, but also auth requests, prepare 
> requests, and batches. This is obviously simplistic, and it does not account 
> for the variation in cost between individual queries, but even a fixed cost 
> model should be useful in aggregate.
>  * If the client specifies THROW_ON_OVERLOAD in its STARTUP message at 
> connection time, a breach of the per-node limit will result in an 
> OverloadedException being propagated to the client, and the server will 
> discard the request.
>  * If THROW_ON_OVERLOAD is not specified, the server will stop consuming 
> messages from the channel/socket, which should back-pressure the client, 
> while the message continues to be processed.
> 2.) This limit is infinite by default (or simply disabled), and can be 
> enabled via the YAML config or JMX at runtime. (It might be cleaner to have a 
> no-op rate limiter that's used when the feature is disabled entirely.)
> 3.) The current value of the limit is available via JMX, and metrics around 
> coordinator operations/second are already available to compare against it.
> 4.) Any interaction with existing byte-based limits will intersect. (i.e. A 
> breach of any limit, bytes or request-based, will actuate back-pressure or 
> OverloadedExceptions.)
> In this first pass, explicitly out of scope would be any work on the 
> client/driver side.
> In terms of validation/testing, our biggest concern with anything that adds 
> overhead on a very hot path is performance. In particular, we want to fully 
> understand how the client and server perform along two axes constituting 4 
> scenarios. Those are a.) whether or not we are breaching the request limit 
> and b.) whether the server is throwing on overload at the behest of the 
> client. Having said that, query execution should dwarf the cost of limit 
> accounting.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to