Add LWTRetryPolicy: retry CAS timeouts on same host with backoff#783
Add LWTRetryPolicy: retry CAS timeouts on same host with backoff#783mykaul wants to merge 1 commit intoscylladb:masterfrom
Conversation
|
CC @calebxyz |
@calebxyz - it's pointless to compare the different drivers' performance - they differ greatly. What is important is the correct and optimized behavior - and there we still have gaps. I think we are very far from testing the correct behavior - we need many more system level tests on one hand (and on the other hand, I'm against testing it in full setup - which is why I've created scylladb/scylla-ccm#731 (that is probably not ready yet , but that's a different issue) |
This is sad, the amount of unpredictability is horrible |
That's one of the major reasons to move some to be Rust based - Rust, CPP-over-Rust, NodeJS-over-Rust, Python-over-Rust. (and we'll stay with Java and Go, I reckon). |
LWT queries use Paxos consensus where the coordinator is the Paxos leader. Retrying on a different host causes Paxos contention — the new coordinator must compete with the original one, potentially causing cascading timeouts. LWTRetryPolicy (extends ExponentialBackoffRetryPolicy) handles this by: - CAS write timeouts: retry on SAME host with exponential backoff - Serial consistency read timeouts: retry on SAME host with backoff - Serial consistency unavailable: retry on NEXT host (paxos quorum lost) - Non-CAS operations: delegate to base ExponentialBackoffRetryPolicy Modeled after gocql's LWTRetryPolicy interface.
f1a865b to
d2a8538
Compare
Summary
LWT queries use Paxos consensus where the first replica (Paxos coordinator/leader) drives the consensus rounds. When a CAS write times out, retrying on a different host causes Paxos contention — the new coordinator must compete with the original, potentially causing cascading timeouts across the cluster.
Currently, no built-in retry policy retries CAS write timeouts at all — they are all RETHROWN immediately:
RetryPolicy.on_write_timeout: CAS → RETHROWExponentialBackoffRetryPolicy.on_write_timeout: CAS → RETHROWDowngradingConsistencyRetryPolicy.on_write_timeout: CAS → RETHROWThis PR adds
LWTRetryPolicy, a new retry policy that extendsExponentialBackoffRetryPolicywith LWT-aware behavior:ExponentialBackoffRetryPolicybehaviorThis is modeled after gocql's
LWTRetryPolicyinterface, which retries LWT queries on the same host to avoid Paxos contention. The key comment from gocql (line 188):Usage
Changes
cassandra/policies.py: AddedLWTRetryPolicyclass (extendsExponentialBackoffRetryPolicy)tests/unit/test_policies.py: AddedLWTRetryPolicyTestwith 21 testsTests
21 new tests covering:
All 103 tests in
tests/unit/test_policies.pypass.Related
TokenAwarePolicy): routing to the Paxos leader + retrying on the same host = optimal LWT latency path.