Skip to content

Commit

Permalink
kvserver: don't allow VOTER_DEMOTING to acquire lease after transfer
Browse files Browse the repository at this point in the history
This PR restricts the case when a VOTER_DEMOTING_LEARNER can
aquire the lease in a joint configuration to only the case where
it was the last leaseholder. Since it is being removed, we only
want it to get the lease if no other replica can aquire it,
the scenario described in #83687

This fix solves a potential starvation scenario where a VOTER_DEMOTING_LEARNER keeps
transferring the lease to the VOTER_INCOMING, succeeding, but then re-acquiring
because the VOTER_INCOMING is dead and the lease expires. In this case, we would
want another replica to pick up the lease, which would allow us to exit the joint configuration.

This PR also removes the leaseHolderRemovalAllowed parameter of CheckCanReceiveLease,
since it is always true since 22.2.

Release note (bug fix): narrows down the conditions under which a VOTER_DEMOTING_LEARNER
can acquire the lease in a joint configuration to a) there has to be an VOTER_INCOMING
in the configuration and b) the VOTER_DEMOTING_LEARNER was the last leaseholder. This
prevents it from acquiring the lease unless it is the only one that can acquire it,
because transferring the lease away is necessary before exiting the joint config (without
the fix the system can be stuck in a joint configuration in some rare situations).

Fixes: #88667
See also #89340
  • Loading branch information
shralex committed Oct 7, 2022
1 parent d71ff86 commit c406fca
Show file tree
Hide file tree
Showing 9 changed files with 118 additions and 59 deletions.
9 changes: 8 additions & 1 deletion pkg/kv/kvclient/kvcoord/replica_slice.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,14 @@ func NewReplicaSlice(
}
}
canReceiveLease := func(rDesc roachpb.ReplicaDescriptor) bool {
if err := roachpb.CheckCanReceiveLease(rDesc, desc.Replicas(), true /* leaseHolderRemovalAllowed */); err != nil {
// NOTE: This logic is client-side and it’s trying to determine the set of
// all replicas that could potentially be leaseholders. We pass
// wasLastLeaseholder = true because we don't know who the
// leaseholder is, so it's possible that a VOTER_DEMOTING still holds on to
// the lease.
if err := roachpb.CheckCanReceiveLease(
rDesc, desc.Replicas(), true, /* wasLastLeaseholder */
); err != nil {
return false
}
return true
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/allocator/allocatorimpl/allocator.go
Original file line number Diff line number Diff line change
Expand Up @@ -1546,7 +1546,7 @@ func (a *Allocator) ValidLeaseTargets(
candidates := make([]roachpb.ReplicaDescriptor, 0, len(existing))
replDescs := roachpb.MakeReplicaSet(existing)
for i := range existing {
if err := roachpb.CheckCanReceiveLease(existing[i], replDescs, true /* lhRemovalAllowed */); err != nil {
if err := roachpb.CheckCanReceiveLease(existing[i], replDescs, false /* wasLastLeaseholder */); err != nil {
continue
}
// If we're not allowed to include the current replica, remove it from
Expand Down
19 changes: 10 additions & 9 deletions pkg/kv/kvserver/batcheval/cmd_lease_request.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,15 +67,6 @@ func RequestLease(
Requested: args.Lease,
}

// If this check is removed at some point, the filtering of learners on the
// sending side would have to be removed as well.
if err := roachpb.CheckCanReceiveLease(args.Lease.Replica, cArgs.EvalCtx.Desc().Replicas(),
true, /* lhRemovalAllowed */
); err != nil {
rErr.Message = err.Error()
return newFailedLeaseTrigger(false /* isTransfer */), rErr
}

// MIGRATION(tschottdorf): needed to apply Raft commands which got proposed
// before the StartStasis field was introduced.
newLease := args.Lease
Expand All @@ -86,6 +77,16 @@ func RequestLease(
isExtension := prevLease.Replica.StoreID == newLease.Replica.StoreID
effectiveStart := newLease.Start

// If this check is removed at some point, the filtering of learners on the
// sending side would have to be removed as well.
wasLastLeaseholder := isExtension
if err := roachpb.CheckCanReceiveLease(
args.Lease.Replica, cArgs.EvalCtx.Desc().Replicas(), wasLastLeaseholder,
); err != nil {
rErr.Message = err.Error()
return newFailedLeaseTrigger(false /* isTransfer */), rErr
}

// Wind the start timestamp back as far towards the previous lease as we
// can. That'll make sure that when multiple leases are requested out of
// order at the same replica (after all, they use the request timestamp,
Expand Down
48 changes: 24 additions & 24 deletions pkg/kv/kvserver/batcheval/cmd_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -274,34 +274,34 @@ func TestCheckCanReceiveLease(t *testing.T) {
const none = roachpb.ReplicaType(-1)

for _, tc := range []struct {
leaseholderType roachpb.ReplicaType
anotherReplicaType roachpb.ReplicaType
eligibleLhRemovalEnabled bool
eligibleLhRemovalDisabled bool
leaseholderType roachpb.ReplicaType
anotherReplicaType roachpb.ReplicaType
expIfWasLastLeaseholderTrue bool
expIfWasLastLeaseholderFalse bool
}{
{leaseholderType: roachpb.VOTER_FULL, anotherReplicaType: none, eligibleLhRemovalEnabled: true, eligibleLhRemovalDisabled: true},
{leaseholderType: roachpb.VOTER_INCOMING, anotherReplicaType: none, eligibleLhRemovalEnabled: true, eligibleLhRemovalDisabled: true},
{leaseholderType: roachpb.VOTER_FULL, anotherReplicaType: none, expIfWasLastLeaseholderTrue: true, expIfWasLastLeaseholderFalse: true},
{leaseholderType: roachpb.VOTER_INCOMING, anotherReplicaType: none, expIfWasLastLeaseholderTrue: true, expIfWasLastLeaseholderFalse: true},

// A VOTER_OUTGOING should only be able to get the lease if there's a VOTER_INCOMING.
{leaseholderType: roachpb.VOTER_OUTGOING, anotherReplicaType: none, eligibleLhRemovalEnabled: false, eligibleLhRemovalDisabled: false},
{leaseholderType: roachpb.VOTER_OUTGOING, anotherReplicaType: roachpb.VOTER_INCOMING, eligibleLhRemovalEnabled: true, eligibleLhRemovalDisabled: false},
{leaseholderType: roachpb.VOTER_OUTGOING, anotherReplicaType: roachpb.VOTER_OUTGOING, eligibleLhRemovalEnabled: false, eligibleLhRemovalDisabled: false},
{leaseholderType: roachpb.VOTER_OUTGOING, anotherReplicaType: roachpb.VOTER_FULL, eligibleLhRemovalEnabled: false, eligibleLhRemovalDisabled: false},
// A VOTER_OUTGOING should only be able to get the lease if there's a VOTER_INCOMING and wasLastLeaseholderTrue.
{leaseholderType: roachpb.VOTER_OUTGOING, anotherReplicaType: none, expIfWasLastLeaseholderTrue: false, expIfWasLastLeaseholderFalse: false},
{leaseholderType: roachpb.VOTER_OUTGOING, anotherReplicaType: roachpb.VOTER_INCOMING, expIfWasLastLeaseholderTrue: true, expIfWasLastLeaseholderFalse: false},
{leaseholderType: roachpb.VOTER_OUTGOING, anotherReplicaType: roachpb.VOTER_OUTGOING, expIfWasLastLeaseholderTrue: false, expIfWasLastLeaseholderFalse: false},
{leaseholderType: roachpb.VOTER_OUTGOING, anotherReplicaType: roachpb.VOTER_FULL, expIfWasLastLeaseholderTrue: false, expIfWasLastLeaseholderFalse: false},

// A VOTER_DEMOTING_LEARNER should only be able to get the lease if there's a VOTER_INCOMING.
{leaseholderType: roachpb.VOTER_DEMOTING_LEARNER, anotherReplicaType: none, eligibleLhRemovalEnabled: false, eligibleLhRemovalDisabled: false},
{leaseholderType: roachpb.VOTER_DEMOTING_LEARNER, anotherReplicaType: roachpb.VOTER_INCOMING, eligibleLhRemovalEnabled: true, eligibleLhRemovalDisabled: false},
{leaseholderType: roachpb.VOTER_DEMOTING_LEARNER, anotherReplicaType: roachpb.VOTER_FULL, eligibleLhRemovalEnabled: false, eligibleLhRemovalDisabled: false},
{leaseholderType: roachpb.VOTER_DEMOTING_LEARNER, anotherReplicaType: roachpb.VOTER_OUTGOING, eligibleLhRemovalEnabled: false, eligibleLhRemovalDisabled: false},
// A VOTER_DEMOTING_LEARNER should only be able to get the lease if there's a VOTER_INCOMING and wasLastLeaseholderTrue.
{leaseholderType: roachpb.VOTER_DEMOTING_LEARNER, anotherReplicaType: none, expIfWasLastLeaseholderTrue: false, expIfWasLastLeaseholderFalse: false},
{leaseholderType: roachpb.VOTER_DEMOTING_LEARNER, anotherReplicaType: roachpb.VOTER_INCOMING, expIfWasLastLeaseholderTrue: true, expIfWasLastLeaseholderFalse: false},
{leaseholderType: roachpb.VOTER_DEMOTING_LEARNER, anotherReplicaType: roachpb.VOTER_FULL, expIfWasLastLeaseholderTrue: false, expIfWasLastLeaseholderFalse: false},
{leaseholderType: roachpb.VOTER_DEMOTING_LEARNER, anotherReplicaType: roachpb.VOTER_OUTGOING, expIfWasLastLeaseholderTrue: false, expIfWasLastLeaseholderFalse: false},

// A VOTER_DEMOTING_NON_VOTER should only be able to get the lease if there's a VOTER_INCOMING.
{leaseholderType: roachpb.VOTER_DEMOTING_NON_VOTER, anotherReplicaType: none, eligibleLhRemovalEnabled: false, eligibleLhRemovalDisabled: false},
{leaseholderType: roachpb.VOTER_DEMOTING_NON_VOTER, anotherReplicaType: roachpb.VOTER_INCOMING, eligibleLhRemovalEnabled: true, eligibleLhRemovalDisabled: false},
{leaseholderType: roachpb.VOTER_DEMOTING_NON_VOTER, anotherReplicaType: roachpb.VOTER_FULL, eligibleLhRemovalEnabled: false, eligibleLhRemovalDisabled: false},
{leaseholderType: roachpb.VOTER_DEMOTING_NON_VOTER, anotherReplicaType: roachpb.VOTER_OUTGOING, eligibleLhRemovalEnabled: false, eligibleLhRemovalDisabled: false},
{leaseholderType: roachpb.VOTER_DEMOTING_NON_VOTER, anotherReplicaType: none, expIfWasLastLeaseholderTrue: false, expIfWasLastLeaseholderFalse: false},
{leaseholderType: roachpb.VOTER_DEMOTING_NON_VOTER, anotherReplicaType: roachpb.VOTER_INCOMING, expIfWasLastLeaseholderTrue: true, expIfWasLastLeaseholderFalse: false},
{leaseholderType: roachpb.VOTER_DEMOTING_NON_VOTER, anotherReplicaType: roachpb.VOTER_FULL, expIfWasLastLeaseholderTrue: false, expIfWasLastLeaseholderFalse: false},
{leaseholderType: roachpb.VOTER_DEMOTING_NON_VOTER, anotherReplicaType: roachpb.VOTER_OUTGOING, expIfWasLastLeaseholderTrue: false, expIfWasLastLeaseholderFalse: false},

{leaseholderType: roachpb.LEARNER, anotherReplicaType: none, eligibleLhRemovalEnabled: false, eligibleLhRemovalDisabled: false},
{leaseholderType: roachpb.NON_VOTER, anotherReplicaType: none, eligibleLhRemovalEnabled: false, eligibleLhRemovalDisabled: false},
{leaseholderType: roachpb.LEARNER, anotherReplicaType: none, expIfWasLastLeaseholderTrue: false, expIfWasLastLeaseholderFalse: false},
{leaseholderType: roachpb.NON_VOTER, anotherReplicaType: none, expIfWasLastLeaseholderTrue: false, expIfWasLastLeaseholderFalse: false},
} {
t.Run(tc.leaseholderType.String(), func(t *testing.T) {
repDesc := roachpb.ReplicaDescriptor{
Expand All @@ -319,10 +319,10 @@ func TestCheckCanReceiveLease(t *testing.T) {
rngDesc.InternalReplicas = append(rngDesc.InternalReplicas, anotherDesc)
}
err := roachpb.CheckCanReceiveLease(rngDesc.InternalReplicas[0], rngDesc.Replicas(), true)
require.Equal(t, tc.eligibleLhRemovalEnabled, err == nil, "err: %v", err)
require.Equal(t, tc.expIfWasLastLeaseholderTrue, err == nil, "err: %v", err)

err = roachpb.CheckCanReceiveLease(rngDesc.InternalReplicas[0], rngDesc.Replicas(), false)
require.Equal(t, tc.eligibleLhRemovalDisabled, err == nil, "err: %v", err)
require.Equal(t, tc.expIfWasLastLeaseholderFalse, err == nil, "err: %v", err)
})
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_lease_transfer.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ func TransferLease(
// If this check is removed at some point, the filtering of learners on the
// sending side would have to be removed as well.
if err := roachpb.CheckCanReceiveLease(
newLease.Replica, cArgs.EvalCtx.Desc().Replicas(), true, /* lhRemovalAllowed */
newLease.Replica, cArgs.EvalCtx.Desc().Replicas(), false, /* wasLastLeaseholder */
); err != nil {
return newFailedLeaseTrigger(true /* isTransfer */), err
}
Expand Down
66 changes: 50 additions & 16 deletions pkg/kv/kvserver/client_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -267,10 +267,10 @@ func TestCannotTransferLeaseToVoterDemoting(t *testing.T) {
})
}

// TestTransferLeaseToVoterOutgoingWithIncoming ensures that the evaluation of lease
// requests for nodes which are already in the VOTER_DEMOTING_LEARNER state succeeds
// when there is a VOTER_INCOMING node.
func TestTransferLeaseToVoterDemotingWithIncoming(t *testing.T) {
// TestTransferLeaseToVoterDemotingFails ensures that the evaluation of lease
// requests for nodes which are already in the VOTER_DEMOTING_LEARNER state fails
// if they weren't previously holding the lease, even if there is a VOTER_INCOMING..
func TestTransferLeaseToVoterDemotingFails(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()
Expand Down Expand Up @@ -320,8 +320,8 @@ func TestTransferLeaseToVoterDemotingWithIncoming(t *testing.T) {
// - Send an AdminChangeReplicasRequest to remove n3 and add n4
// - Block the step that moves n3 to VOTER_DEMOTING_LEARNER on changeReplicasChan
// - Send an AdminLeaseTransfer to make n3 the leaseholder
// - Try really hard to make sure that the lease transfer at least gets to
// latch acquisition before unblocking the ChangeReplicas.
// - Make sure that this request fails (since n3 is VOTER_DEMOTING_LEARNER and
// wasn't previously leaseholder)
// - Unblock the ChangeReplicas.
// - Make sure the lease transfer succeeds.

Expand All @@ -348,15 +348,17 @@ func TestTransferLeaseToVoterDemotingWithIncoming(t *testing.T) {
require.NoError(t, err)
require.Equal(t, tc.Target(0), leaseHolder,
errors.Errorf("Leaseholder supposed to be on n1."))
// Move the lease to n3, the VOTER_DEMOTING_LEARNER.
// Try to move the lease to n3, the VOTER_DEMOTING_LEARNER.
// This should fail since the last leaseholder wasn't n3
// (wasLastLeaseholder = false in CheckCanReceiveLease).
err = tc.Server(0).DB().AdminTransferLease(context.Background(),
scratchStartKey, tc.Target(2).StoreID)
require.NoError(t, err)
// Make sure the lease moved to n3.
require.EqualError(t, err, `replica cannot hold lease`)
// Make sure the lease is still on n1.
leaseHolder, err = tc.FindRangeLeaseHolder(desc, nil)
require.NoError(t, err)
require.Equal(t, tc.Target(2), leaseHolder,
errors.Errorf("Leaseholder supposed to be on n3."))
require.Equal(t, tc.Target(0), leaseHolder,
errors.Errorf("Leaseholder supposed to be on n1."))
}()
// Try really hard to make sure that our request makes it past the
// sanity check error to the evaluation error.
Expand All @@ -369,7 +371,7 @@ func TestTransferLeaseToVoterDemotingWithIncoming(t *testing.T) {
})
}

// TestTransferLeaseFailureDuringJointConfig reproduces
// internalTransferLeaseFailureDuringJointConfig reproduces
// https://github.com/cockroachdb/cockroach/issues/83687
// and makes sure that if lease transfer fails during a joint configuration
// the previous leaseholder will successfully re-aquire the lease.
Expand All @@ -385,10 +387,9 @@ func TestTransferLeaseToVoterDemotingWithIncoming(t *testing.T) {
// - We re-enable lease transfers on this range.
// - n1 is able to re-aquire the lease, due to the fix in #83686 which enables a
// VOTER_DEMOTING_LEARNER (n1) replica to get the lease if there's also a VOTER_INCOMING
// which is the case here (n4).
// which is the case here (n4) and since n1 was the last leaseholder.
// - n1 transfers the lease away and the range leaves the joint configuration.
func TestTransferLeaseFailureDuringJointConfig(t *testing.T) {
defer leaktest.AfterTest(t)()
func internalTransferLeaseFailureDuringJointConfig(t *testing.T, isManual bool) {
defer log.Scope(t).Close(t)
ctx := context.Background()

Expand Down Expand Up @@ -439,7 +440,22 @@ func TestTransferLeaseFailureDuringJointConfig(t *testing.T) {
require.NoError(t, err)
require.True(t, desc.Replicas().InAtomicReplicationChange())

// Further lease transfers should succeed, allowing the atomic replication change to complete.
// Allow further lease transfers to succeed.
atomic.StoreInt64(&scratchRangeID, 0)

if isManual {
// Manually transfer the lease to n1 (VOTER_DEMOTING_LEARNER).
err = tc.Server(0).DB().AdminTransferLease(context.Background(),
scratchStartKey, tc.Target(0).StoreID)
require.NoError(t, err)
// Make sure n1 has the lease
leaseHolder, err := tc.FindRangeLeaseHolder(desc, nil)
require.NoError(t, err)
require.Equal(t, tc.Target(0), leaseHolder,
errors.Errorf("Leaseholder supposed to be on n1."))
}

// Complete the replication change.
atomic.StoreInt64(&scratchRangeID, 0)
store := tc.GetFirstStoreFromServer(t, 0)
repl := store.LookupReplica(roachpb.RKey(scratchStartKey))
Expand All @@ -452,6 +468,24 @@ func TestTransferLeaseFailureDuringJointConfig(t *testing.T) {
require.False(t, desc.Replicas().InAtomicReplicationChange())
}

// TestTransferLeaseFailureDuringJointConfig is using
// internalTransferLeaseFailureDuringJointConfig and
// completes the lease transfer to n1 “automatically”
// by relying on replicate queue.
func TestTransferLeaseFailureDuringJointConfigAuto(t *testing.T) {
defer leaktest.AfterTest(t)()
internalTransferLeaseFailureDuringJointConfig(t, false)
}

// TestTransferLeaseFailureDuringJointConfigManual is using
// internalTransferLeaseFailureDuringJointConfig and
// completes the lease transfer to n1 “manually” using
// AdminTransferLease.
func TestTransferLeaseFailureDuringJointConfigManual(t *testing.T) {
defer leaktest.AfterTest(t)()
internalTransferLeaseFailureDuringJointConfig(t, true)
}

// TestStoreLeaseTransferTimestampCacheRead verifies that the timestamp cache on
// the new leaseholder is properly updated after a lease transfer to prevent new
// writes from invalidating previously served reads.
Expand Down
11 changes: 10 additions & 1 deletion pkg/kv/kvserver/replica_proposal_buf.go
Original file line number Diff line number Diff line change
Expand Up @@ -1207,7 +1207,16 @@ func (rp *replicaProposer) leaderStatusRLocked(
// lease again, and by then hopefully we will have caught up.
leaderEligibleForLease = true
} else {
err := roachpb.CheckCanReceiveLease(leaderRep, rangeDesc.Replicas(), true /* lhRemovalAllowed */)
// If the current leader is a VOTER_DEMOTING and it was the last one to
// hold the lease (according to our possibly stale applied lease state),
// CheckCanReceiveLease considers it eligible to continue holding the
// lease, so we don't allow our proposal through. Otherwise, if it was not
// the last one to hold the lease, it will never be allowed to acquire it
// again, so we don't consider it eligible.
lastLease, _ := r.getLeaseRLocked()
wasLastLeaseholder := leaderRep.ReplicaID == lastLease.Replica.ReplicaID
err := roachpb.CheckCanReceiveLease(
leaderRep, rangeDesc.Replicas(), wasLastLeaseholder)
leaderEligibleForLease = err == nil
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -397,7 +397,7 @@ func (r *Replica) propose(
// transferred away. The previous leaseholder is a LEARNER in the target config,
// and therefore shouldn't continue holding the lease.
if err := roachpb.CheckCanReceiveLease(
lhDesc, proposedDesc.Replicas(), true, /* lhRemovalAllowed */
lhDesc, proposedDesc.Replicas(), true, /* wasLastLeaseholder */
); err != nil {
e := errors.Mark(errors.Wrapf(err, "%v received invalid ChangeReplicasTrigger %s to "+
"remove self (leaseholder); lhRemovalAllowed: %v; current desc: %v; proposed desc: %v",
Expand Down
18 changes: 13 additions & 5 deletions pkg/roachpb/metadata_replicas.go
Original file line number Diff line number Diff line change
Expand Up @@ -503,22 +503,30 @@ var errReplicaCannotHoldLease = errors.Errorf("replica cannot hold lease")
// but not to exit it in this state, i.e., the leaseholder must be some
// kind of voter in the next new config (potentially VOTER_DEMOTING).
//
// It is possible (and sometimes needed) that while in the joint configuration,
// the replica being removed will receive lease. This is allowed only if
// a) there is a VOTER_INCOMING replica to which the lease will be trasferred
// when transitioning out of the joint config, and b) the replica being removed
// was the last leaseholder (as indictated by wasLastLeaseholder). The
// information we use for (b) is potentially stale, but if it incorrect
// the removed node either does not need to get the lease or will not be able
// to get it. In particular, when we think we are the last leaseholder but we
// aren't, the CAS call for extending the lease will fail (see
// wasLastLeaseholder := isExtension in cmd_lease_request.go).
//
// An error is also returned is the replica is not part of `replDescs`.
// leaseHolderRemovalAllowed is intended to check if the cluster version is
// EnableLeaseHolderRemoval or higher.
// TODO(shralex): remove this flag in 23.1
// NB: This logic should be in sync with constraint_stats_report as report
// will check voter constraint violations. When changing this method, you need
// to update replica filter in report to keep it correct.
func CheckCanReceiveLease(
wouldbeLeaseholder ReplicaDescriptor, replDescs ReplicaSet, leaseHolderRemovalAllowed bool,
wouldbeLeaseholder ReplicaDescriptor, replDescs ReplicaSet, wasLastLeaseholder bool,
) error {
repDesc, ok := replDescs.GetReplicaDescriptorByID(wouldbeLeaseholder.ReplicaID)
if !ok {
return errReplicaNotFound
}
if !(repDesc.IsVoterNewConfig() ||
(repDesc.IsVoterOldConfig() && replDescs.containsVoterIncoming() && leaseHolderRemovalAllowed)) {
(repDesc.IsVoterOldConfig() && replDescs.containsVoterIncoming() && wasLastLeaseholder)) {
// We allow a demoting / incoming voter to receive the lease if there's an incoming voter.
// In this case, when exiting the joint config, we will transfer the lease to the incoming
// voter.
Expand Down

0 comments on commit c406fca

Please sign in to comment.