kvserver: skip non-live nodes when considering candidates for transfers#55808
kvserver: skip non-live nodes when considering candidates for transfers#55808craig[bot] merged 1 commit intocockroachdb:masterfrom
Conversation
tbg
left a comment
There was a problem hiding this comment.
if there weren't any unresolved comments on the diff in the original PR.
Reviewed 7 of 7 files at r1.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @andreimatei and @knz)
pkg/kv/kvserver/store_rebalancer.go, line 675 at r1 (raw file):
} // If the target store is on a separate node, we will also care
This probably does fix replicate/wide, though not the general problem underlying it. I think the roachtest always ends up wanting to move replicas to a dead node, which will be prevented by your new check. Though it could still end up wanting to move data to another live node in theory. I shall validate.
pkg/kv/kvserver/allocator_test.go
Outdated
| // TestAllocateCandidatesExcludeDrainingNodes checks that draining nodes, | ||
| // as per a store pool's isNodeValidForRoutineReplicaTransfer(), | ||
| // are excluded from the list of candidates for an allocation. | ||
| func TestAllocateCandidatesExcludeDrainingNodes(t *testing.T) { |
There was a problem hiding this comment.
nit: there's nothing particular about draining here. You're testing the isNodeReadyFor.... method. Maybe TestAllocateCandidatesExcludeNonReadyNodes.
|
What has me confused is that I think you made this PR primarily for draining nodes, but are you really changing anything for draining nodes? I think a draining node will show up as LIVE, won't it? cockroach/pkg/kv/kvserver/store_pool.go Lines 142 to 161 in 785aea7 The concept of draining seems wholly unconnected to the NodeLivenessStatus. |
knz
left a comment
There was a problem hiding this comment.
are you really changing anything for draining nodes? I think a draining node will show up as LIVE, won't it?
no it shows as unavailable: see, if l.Draining comes before l.IsLive.
Is this ready to merge?
Reviewable status:
complete! 1 of 0 LGTMs obtained (waiting on @andreimatei and @tbg)
pkg/kv/kvserver/allocator_test.go, line 2460 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
nit: there's nothing particular about draining here. You're testing the
isNodeReadyFor....method. MaybeTestAllocateCandidatesExcludeNonReadyNodes.
Renamed.
pkg/kv/kvserver/store_rebalancer.go, line 675 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
This probably does fix replicate/wide, though not the general problem underlying it. I think the roachtest always ends up wanting to move replicas to a dead node, which will be prevented by your new check. Though it could still end up wanting to move data to another live node in theory. I shall validate.
Yes agreed.
ff76faf to
b398d4d
Compare
tbg
left a comment
There was a problem hiding this comment.
no it shows as unavailable: see, if l.Draining comes before l.IsLive.
Oh, missed that, thanks.
LGTM, though can you update the comment on NODE_STATUS_UNAVAILABLE to indicate that a draining node falls into this category?
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @andreimatei and @tbg)
Prior to this patch, 3 components could attempt to transfer a replica to a node currently being drained: - the store rebalancer, which rebalances replicas based on disk usage and QPS. - the allocator, to place new replicas. - the allocator, to rebalance replicas depending on load. This commit introduces a consideration for node liveness when building the list of candidates, to detect whether a target node is acceptable. Any node that is not LIVE according to its liveness status is not considered for a transfer. Release note (bug fix): In some cases CockroachDB would attempt to transfer ranges to nodes in the process of being decommissioned or being shut down; this could cause disruption the moment the node did actually terminate. This bug has been fixed. It had been introduced some time before v2.0.
b398d4d to
ffa0ce3
Compare
knz
left a comment
There was a problem hiding this comment.
LGTM, though can you update the comment on NODE_STATUS_UNAVAILABLE to indicate that a draining node falls into this category?
Done.
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @andreimatei and @tbg)
|
bors r=tbg |
|
This PR was included in a batch that was canceled, it will be automatically retried |
|
Build succeeded: |
(This PR is forked off #55460 to simplify the discussion. I believe there's no discussion left here? Maybe I can merge it directly?)
Fixes #55440.
Prior to this patch, 3 components could attempt to transfer a replica
to a node currently being drained:
usage and QPS.
This commit introduces a consideration for node liveness when building
the list of candidates, to detect whether a target node is
acceptable. Any node that is not LIVE according to its liveness status
is not considered for a transfer.
Release note (bug fix): In some cases CockroachDB would attempt to
transfer ranges to nodes in the process of being decommissioned or
being shut down; this could cause disruption the moment the node
did actually terminate. This bug has been fixed. It had been
introduced some time before v2.0.