kvserver: simplify lease forwarding constraints #74506
Labels
A-kv-replication
Relating to Raft, consensus, and coordination.
C-enhancement
Solution expected to add code/behavior + preserve backward-compat (pg compat issues are exception)
no-issue-activity
X-stale
To solve #37906, we landed #55148 (and refined it a few times since) which essentially tries to pin lease acquisitions to the raft leader (thus preventing followers that are behind on the log from sending ill-advised lease requests which lead to high request latencies).
I am currently working on per-Replica circuit breakers in #33007 and I wonder if our current solution for these slow followers could be simplified. We could essentially start the replica in a circuit-broken state (which means nobody gets stuck on the lease since everyone fails fast) and let the circuit breaker's probe deal with making sure that the follower is caught up (it does this by sending a
ProbeRequest
through the replication layer and waiting for it to apply). That way, by the time the follower first attempts a lease acquisition, it is known to have made it through the initial catch-up.Some elbow grease would be required; we'd probably want #74504 addressed so that requests that DistSender sent to this replica to discover the lease don't bubble up to the client (maybe the breaker is initially in a
NotLeaseholderError
state instead?)The upshot is that we could let everyone propose lease requests again. My feeling is that we initially thought that the current approach (trying to have only the Raft leader request a lease) was a simplification, but due to the presence of demoting/outgoing voters (which can be raft leaders but will cease to be voters soon) everything is complicated again. We are on a follower that is potentially behind on a lot of raft log, which means we can't be too sure who else is a) around (the leadership might be held by a replica not even in our descriptor) and b) what kind of voter they are. And yet, that's exactly what we need to know and we seem to just hope that we're up to date:
cockroach/pkg/kv/kvserver/replica_proposal_buf.go
Lines 1034 to 1049 in 95a7671
Note that in particular we assume that if the leader is not in the descriptor, we assume it can hold the lease & we will thus not grab the lease ourselves. But the leader could be a
VOTER_OUTGOING
and thus not eligible to hold the lease.All of this makes me critical about the current approach. It seems hard to understand and get right; it seems to err in both directions in various scenarios. I would thus be in favor of exploring somethings simpler, though of course there is also the argument that the status quo seems to have solved the worst parts of the problem. I am not pushing to necessarily do anything here in the short term.
Jira issue: CRDB-12124
The text was updated successfully, but these errors were encountered: