[WIP] storage: proposal quota pool#13869
[WIP] storage: proposal quota pool#13869petermattis wants to merge 2 commits intocockroachdb:masterfrom
Conversation
|
@spencerkimball Per our discussion yesterday. This isn't fully fleshed out right now, but gives us something concrete to argue over. |
dd3f1cf to
6e031a0
Compare
|
This seems to work in simple testing. A node crash will cause a 2 second blip of unavailability if the range is very active. In normal operation, we'll limit proposals to the speed of the slowest replica, but with a buffer to avoid imposing the latency of the slower replicas on the commit latency. Cc @bdarnell |
spencerkimball
left a comment
There was a problem hiding this comment.
Overall this is pretty simple, which is positive. As mentioned in the review comments, I think the biggest concern would be a replica coming back online and wreaking havoc.
My earlier thought to avoid the 2s timeout seems less useful now that I'm looking at this. It would just push the problem I was looking to solve elsewhere – we'd be relying on node liveness instead of this 2s timeout.
Need unittests too, obviously.
pkg/storage/quota_pool.go
Outdated
| ) | ||
|
|
||
| const ( | ||
| leaderProposalQuota = 1000 |
There was a problem hiding this comment.
Do we need something more thoughtful here?
There was a problem hiding this comment.
Probably. We'd like something that avoided Raft log truncation. This achieves that for the workload I'm using which involves small requests. More directly estimating the affect on Raft log size would be nice, though I'm not sure how to achieve that.
pkg/storage/quota_pool.go
Outdated
| type quotaPool struct { | ||
| c chan int64 | ||
|
|
||
| mu sync.Mutex |
There was a problem hiding this comment.
Any reason not to use atomic primitives?
There was a problem hiding this comment.
This code was lifted from grpc. I haven't thought through the difficulties with using atomics here.
There was a problem hiding this comment.
yep, could be tricky. You'll have to scrutinize the code carefully.
pkg/storage/quota_pool.go
Outdated
| } | ||
|
|
||
| func newQuotaPool(q int64) *quotaPool { | ||
| qb := "aPool{ |
There was a problem hiding this comment.
Cause I stole it from the grpc code base which uses the name qb. I'll change it eventually.
pkg/storage/replica.go
Outdated
| r.mu.proposalQuotaBase = r.mu.lastIndex | ||
| r.proposalQuota.reset(leaderProposalQuota) | ||
| } else { | ||
| r.proposalQuota.reset(followerProposalQuota) |
There was a problem hiding this comment.
Why do followers require a proposal quota?
There was a problem hiding this comment.
So they can propose requests which get forwarded to the leader. For example, followers propose lease requests which is how they determine who holds the lease. We could probably change this so that we don't acquire proposal quota when a follower proposes a Raft command.
There was a problem hiding this comment.
I'd just as soon eliminate the follower's use of a quota pool and the associated constant and just set the pool to nil on a leader -> follower transition, and allow followers to always propose and forward to the leader.
pkg/storage/replica.go
Outdated
| // Computed checksum at a snapshot UUID. | ||
| checksums map[uuid.UUID]replicaChecksum | ||
|
|
||
| proposalQuotaBase uint64 |
There was a problem hiding this comment.
s/proposalQuotaBase/proposalQuotaBaseIndex/
pkg/storage/store.go
Outdated
| if req.Message.Type == raftpb.MsgApp { | ||
| r.setEstimatedCommitIndexLocked(req.Message.Commit) | ||
| } | ||
| r.setLastActivityLocked(req.FromReplica.ReplicaID) |
There was a problem hiding this comment.
Is it possible for a hopelessly lagged replica to come back online and require a snapshot, but still report its log index and cause the quota pool to be immediately drained to a negative value?
There was a problem hiding this comment.
No, that's not possible. The purpose of Replica.mu.proposalQuotaBaseIndex is that the index used for quota calculations ratchets up. If a replica comes back on line and has a commit index before the quota base index, it will be ignored for quota purposes until it catches back up.
There was a problem hiding this comment.
Actually, this was broken before, but should be fixed now.
pkg/storage/replica.go
Outdated
| for _, rep := range r.mu.state.Desc.Replicas { | ||
| // Only consider followers that we've received a message from in the last 2 | ||
| // seconds. | ||
| const activeTime = 2 * time.Second |
There was a problem hiding this comment.
Probably need to put this in StoreConfig.
There was a problem hiding this comment.
Yep. I'll add a TODO for now.
petermattis
left a comment
There was a problem hiding this comment.
Yes, unit tests are definitely needed before merging.
pkg/storage/quota_pool.go
Outdated
| ) | ||
|
|
||
| const ( | ||
| leaderProposalQuota = 1000 |
There was a problem hiding this comment.
Probably. We'd like something that avoided Raft log truncation. This achieves that for the workload I'm using which involves small requests. More directly estimating the affect on Raft log size would be nice, though I'm not sure how to achieve that.
pkg/storage/quota_pool.go
Outdated
| } | ||
|
|
||
| func newQuotaPool(q int64) *quotaPool { | ||
| qb := "aPool{ |
There was a problem hiding this comment.
Cause I stole it from the grpc code base which uses the name qb. I'll change it eventually.
pkg/storage/replica.go
Outdated
| // Computed checksum at a snapshot UUID. | ||
| checksums map[uuid.UUID]replicaChecksum | ||
|
|
||
| proposalQuotaBase uint64 |
pkg/storage/replica.go
Outdated
| for _, rep := range r.mu.state.Desc.Replicas { | ||
| // Only consider followers that we've received a message from in the last 2 | ||
| // seconds. | ||
| const activeTime = 2 * time.Second |
There was a problem hiding this comment.
Yep. I'll add a TODO for now.
pkg/storage/replica.go
Outdated
| r.mu.proposalQuotaBase = r.mu.lastIndex | ||
| r.proposalQuota.reset(leaderProposalQuota) | ||
| } else { | ||
| r.proposalQuota.reset(followerProposalQuota) |
There was a problem hiding this comment.
So they can propose requests which get forwarded to the leader. For example, followers propose lease requests which is how they determine who holds the lease. We could probably change this so that we don't acquire proposal quota when a follower proposes a Raft command.
pkg/storage/store.go
Outdated
| if req.Message.Type == raftpb.MsgApp { | ||
| r.setEstimatedCommitIndexLocked(req.Message.Commit) | ||
| } | ||
| r.setLastActivityLocked(req.FromReplica.ReplicaID) |
There was a problem hiding this comment.
No, that's not possible. The purpose of Replica.mu.proposalQuotaBaseIndex is that the index used for quota calculations ratchets up. If a replica comes back on line and has a commit index before the quota base index, it will be ignored for quota purposes until it catches back up.
pkg/storage/quota_pool.go
Outdated
| type quotaPool struct { | ||
| c chan int64 | ||
|
|
||
| mu sync.Mutex |
There was a problem hiding this comment.
This code was lifted from grpc. I haven't thought through the difficulties with using atomics here.
|
Aside from unittests, LGTM |
ec6f53a to
e5c6396
Compare
|
I reworked the handling of |
|
Review status: 0 of 3 files reviewed at latest revision, 7 unresolved discussions, some commit checks pending. pkg/storage/replica.go, line 2802 at r1 (raw file): Previously, spencerkimball (Spencer Kimball) wrote…
Done. Comments from Reviewable |
|
Review status: 0 of 3 files reviewed at latest revision, 7 unresolved discussions, some commit checks pending. pkg/storage/quota_pool.go, line 18 at r1 (raw file): Previously, spencerkimball (Spencer Kimball) wrote…
Reworked to remove the mutex. pkg/storage/quota_pool.go, line 23 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
Done. Comments from Reviewable |
7044091 to
fd1947d
Compare
|
Review status: 0 of 3 files reviewed at latest revision, 7 unresolved discussions, some commit checks pending. pkg/storage/quota_pool.go, line 18 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
Doh! Removing the mutex clearly wasn't safe. Reintroduced for now. Comments from Reviewable |
fd1947d to
813d7bc
Compare
|
Reviewed 1 of 3 files at r1, 2 of 2 files at r2. pkg/storage/quota_pool.go, line 75 at r2 (raw file):
Is pkg/storage/quota_pool.go, line 101 at r2 (raw file):
Document what is required of the caller: if acquire() returns without error, the caller must call add(1) after doing its work. pkg/storage/replica.go, line 825 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
We have a lot of different failure detectors, with different thresholds and even different mechanisms. Can this piggyback on another (probably NodeLiveness) instead of introducing a new way for a node to be "down"? A failing node already triggers a blip of up to 9s (?) for ranges it leads, so it doesn't seem bad to me to lengthen the impact of a failing node on ranges where it is a follower which are also maxing out their proposal quota. pkg/storage/replica.go, line 812 at r2 (raw file):
We should also clear from this map at some point - at least when removing a replica, and maybe also empty the whole map when leadership changes hands? pkg/storage/replica.go, line 825 at r2 (raw file):
s/refresh/update/? pkg/storage/replica.go, line 840 at r2 (raw file):
Add "still" to this comment to distinguish it from the early return above when we're becoming the leader. pkg/storage/replica.go, line 842 at r2 (raw file):
I assume your concern is for the allocations this performs? We could probably add a method to query this without additional allocations. We could also maintain our own map by tracking MsgAppResps as they pass through the transport, but that probably duplicates too much raft logic. pkg/storage/replica.go, line 844 at r2 (raw file):
s/committed/acknowledged/ pkg/storage/replica.go, line 857 at r2 (raw file):
pkg/storage/replica.go, line 870 at r2 (raw file):
Add a comment pointing out this subtlety: Raft may propose commands itself (specifically the empty commands when leadership changes), and these commands don't go through the code paths where we acquire quota from the pool. We avoid releasing quota here that we never acquired by resetting the quota pool whenever leadership changes hands. Comments from Reviewable |
813d7bc to
7369586
Compare
|
I still need to add tests. Review status: 1 of 3 files reviewed at latest revision, 16 unresolved discussions. pkg/storage/quota_pool.go, line 75 at r2 (raw file): Previously, bdarnell (Ben Darnell) wrote…
We always pass in a positive value to pkg/storage/quota_pool.go, line 101 at r2 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Done. pkg/storage/replica.go, line 825 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
I initially piggybacked this on node liveness, but then got worried that the node liveness ranges would have to be exempted in order to avoid deadlock. Seemed simpler to have a different mechanism here. pkg/storage/replica.go, line 812 at r2 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Done. We now only initialize this map on the leader and clear it when a node becomes a follower. pkg/storage/replica.go, line 825 at r2 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Done. pkg/storage/replica.go, line 840 at r2 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Done. pkg/storage/replica.go, line 842 at r2 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Yeah, the allocation is the concern. It is a small concern, though. Trying to track this ourselves via MsgAppResp seems like overkill. I've enhanced the TODO comment. pkg/storage/replica.go, line 844 at r2 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Done. pkg/storage/replica.go, line 857 at r2 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Done. pkg/storage/replica.go, line 870 at r2 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Done. Comments from Reviewable |
|
Review status: 1 of 3 files reviewed at latest revision, 16 unresolved discussions, some commit checks pending. pkg/storage/replica.go, line 825 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
Rather than introducing a new configurable here, perhaps this time should be the same as the Raft election timeout. The only way for a leader to have not heard for a Replica for longer than the Raft election timeout is for the Range to be quiescent in which case there are no concerns about proposal quota. Comments from Reviewable |
|
Review status: 1 of 3 files reviewed at latest revision, 16 unresolved discussions, some commit checks pending. pkg/storage/replica.go, line 825 at r1 (raw file): Previously, petermattis (Peter Mattis) wrote…
Err, Comments from Reviewable |
|
Reviewed 2 of 2 files at r3. pkg/storage/replica.go, line 825 at r1 (raw file):
OK, what about RPC heartbeats then? I'd really like to avoid introducing a new notion of node activity, especially one with subtle interactions with quiescence and leadership changes. pkg/storage/replica.go, line 812 at r2 (raw file): Previously, petermattis (Peter Mattis) wrote…
So a brand-new leader will see all the followers as inactive. That seems less than ideal, although since we're also resetting the quota to its maximum limit when becoming leader I don't see a specific problem with it. Comments from Reviewable |
7369586 to
ddc6ef8
Compare
|
Review status: 1 of 2 files reviewed at latest revision, 8 unresolved discussions. pkg/storage/replica.go, line 825 at r1 (raw file): Previously, bdarnell (Ben Darnell) wrote…
My instinct is that the new mechanism would be less fragile than reusing an existing one like RPC heartbeats. Not sure why I feel that way. I'll think about this more. That said, I added another commit which replaces Comments from Reviewable |
The leader maintains a pool of "proposal quota". Before proposing a Raft command, we acquire 1 unit of proposal quota. When all of the active followers have committed an entry, that unit of proposal quota is returned to the pool. The proposal quota pool size is hard coded to 1000 which allows fairly deep pipelining of Raft commands. We only consider "active" followers when determining if a unit of quota should be returned to the pool. An active follower is one we've received any type of message from in the past 2 seconds. See cockroachdb#8659
ddc6ef8 to
f16f2db
Compare
|
Cc @irfansharif |
Repurposing cockroachdb#13869. The leader maintains a pool of "proposal quota". Before proposing a Raft command, we acquire 1 unit of proposal quota. When all of the healthy followers have committed an entry, that unit of proposal quota is returned to the pool. The proposal quota pool size is hard coded to 1000 which allows fairly deep pipelining of Raft commands. We only consider followers that have "healthy" RPC connections when determining if a unit of quota should be returned to the pool.
Repurposing cockroachdb#13869. The leader maintains a pool of "proposal quota". Before proposing a Raft command, we acquire 1 unit of proposal quota. When all of the healthy followers have committed an entry, that unit of proposal quota is returned to the pool. The proposal quota pool size is hard coded to 1000 which allows fairly deep pipelining of Raft commands. We only consider followers that have "healthy" RPC connections when determining if a unit of quota should be returned to the pool.
Repurposing cockroachdb#13869. The leader maintains a pool of "proposal quota". Before proposing a Raft command, we acquire 1 unit of proposal quota. When all of the healthy followers have committed an entry, that unit of proposal quota is returned to the pool. The proposal quota pool size is hard coded to 1000 which allows fairly deep pipelining of Raft commands. We only consider followers that have "healthy" RPC connections when determining if a unit of quota should be returned to the pool.
Repurposing cockroachdb#13869. The leader maintains a pool of "proposal quota". Before proposing a Raft command, we acquire 1 unit of proposal quota. When all of the healthy followers have committed an entry, that unit of proposal quota is returned to the pool. The proposal quota pool size is hard coded to 1000 which allows fairly deep pipelining of Raft commands. We only consider followers that have "healthy" RPC connections when determining if a unit of quota should be returned to the pool.
Repurposing cockroachdb#13869. The leader maintains a pool of "proposal quota". Before proposing a Raft command, we acquire 1 unit of proposal quota. When all of the healthy followers have committed an entry, that unit of proposal quota is returned to the pool. The proposal quota pool size is hard coded to 1000 which allows fairly deep pipelining of Raft commands. We only consider followers that have "healthy" RPC connections when determining if a unit of quota should be returned to the pool.
Repurposing cockroachdb#13869. The leader maintains a pool of "proposal quota". Before proposing a Raft command, we acquire 1 unit of proposal quota. When all of the healthy followers have committed an entry, that unit of proposal quota is returned to the pool. The proposal quota pool size is hard coded to 1000 which allows fairly deep pipelining of Raft commands. We only consider followers that have "healthy" RPC connections when determining if a unit of quota should be returned to the pool.
Repurposing cockroachdb#13869. The leader maintains a pool of "proposal quota". Before proposing a Raft command, we acquire 1 unit of proposal quota. When all of the healthy followers have committed an entry, that unit of proposal quota is returned to the pool. The proposal quota pool size is hard coded to 1000 which allows fairly deep pipelining of Raft commands. We only consider followers that have "healthy" RPC connections when determining if a unit of quota should be returned to the pool.
Repurposing cockroachdb#13869. The leader maintains a pool of "proposal quota". Before proposing a Raft command, we acquire 1 unit of proposal quota. When all of the healthy followers have committed an entry, that unit of proposal quota is returned to the pool. The proposal quota pool size is hard coded to 1000 which allows fairly deep pipelining of Raft commands. We only consider followers that have "healthy" RPC connections when determining if a unit of quota should be returned to the pool.
This change is