Skip to content

Commit

Permalink
Merge #83686
Browse files Browse the repository at this point in the history
83686: kvserver: allow voter demoting to get a lease, in case there's an incoming voter r=shralex a=shralex

Fixes #83687.

It is possible that we've entered a joint config where the leaseholder is being
replaced with an incoming voter. We try to transfer the lease away, but this fails.
In this case, we need the demoted voter to re-aquire the lease, as it might be an epoch
based lease, that doesn't expire. Otherwise we might be stuck without anyone else getting
the lease (the original leaseholder will be in the PROSCRIBED state, repeatedly trying to 
re-aquire the lease, but will fail since its a VOTER_DEMOTING_LEARNER).

Release note (bug fix): Fixes a critical bug (#83687) introduced in 22.1.0 where
failure to transfer a lease in the joint config may result in range unavailability. The fix
allows the original leaseholder to re-aquire the lease so that lease transfer can be retried.

Co-authored-by: shralex <[email protected]>
  • Loading branch information
craig[bot] and shralex committed Jul 8, 2022
2 parents 2dd8e76 + 139dc42 commit 365b9c4
Show file tree
Hide file tree
Showing 11 changed files with 331 additions and 67 deletions.
2 changes: 1 addition & 1 deletion pkg/kv/kvclient/kvcoord/replica_slice.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ func NewReplicaSlice(
}
}
canReceiveLease := func(rDesc roachpb.ReplicaDescriptor) bool {
if err := roachpb.CheckCanReceiveLease(rDesc, desc.Replicas()); err != nil {
if err := roachpb.CheckCanReceiveLease(rDesc, desc.Replicas(), true /* leaseHolderRemovalAllowed */); err != nil {
return false
}
return true
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/allocator/allocatorimpl/allocator.go
Original file line number Diff line number Diff line change
Expand Up @@ -1479,8 +1479,9 @@ func (a *Allocator) ValidLeaseTargets(
) []roachpb.ReplicaDescriptor {
candidates := make([]roachpb.ReplicaDescriptor, 0, len(existing))
replDescs := roachpb.MakeReplicaSet(existing)
lhRemovalAllowed := a.StorePool.St.Version.IsActive(ctx, clusterversion.EnableLeaseHolderRemoval)
for i := range existing {
if err := roachpb.CheckCanReceiveLease(existing[i], replDescs); err != nil {
if err := roachpb.CheckCanReceiveLease(existing[i], replDescs, lhRemovalAllowed); err != nil {
continue
}
// If we're not allowed to include the current replica, remove it from
Expand Down
7 changes: 6 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_lease_request.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb"
Expand Down Expand Up @@ -67,9 +68,13 @@ func RequestLease(
Requested: args.Lease,
}

lhRemovalAllowed :=
cArgs.EvalCtx.ClusterSettings().Version.IsActive(ctx, clusterversion.EnableLeaseHolderRemoval)
// 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()); err != nil {
if err := roachpb.CheckCanReceiveLease(args.Lease.Replica, cArgs.EvalCtx.Desc().Replicas(),
lhRemovalAllowed,
); err != nil {
rErr.Message = err.Error()
return newFailedLeaseTrigger(false /* isTransfer */), rErr
}
Expand Down
62 changes: 47 additions & 15 deletions pkg/kv/kvserver/batcheval/cmd_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -130,9 +130,10 @@ func TestLeaseCommandLearnerReplica(t *testing.T) {
clock := hlc.NewClock(timeutil.NewManualTime(timeutil.Unix(0, 123)), time.Nanosecond /* maxOffset */)
cArgs := CommandArgs{
EvalCtx: (&MockEvalCtx{
StoreID: voterStoreID,
Desc: &desc,
Clock: clock,
ClusterSettings: cluster.MakeTestingClusterSettings(),
StoreID: voterStoreID,
Desc: &desc,
Clock: clock,
}).EvalContext(),
Args: &roachpb.TransferLeaseRequest{
Lease: roachpb.Lease{
Expand Down Expand Up @@ -265,17 +266,37 @@ func TestCheckCanReceiveLease(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

const none = roachpb.ReplicaType(-1)

for _, tc := range []struct {
leaseholderType roachpb.ReplicaType
eligible bool
leaseholderType roachpb.ReplicaType
anotherReplicaType roachpb.ReplicaType
eligibleLhRemovalEnabled bool
eligibleLhRemovalDisabled bool
}{
{leaseholderType: roachpb.VOTER_FULL, eligible: true},
{leaseholderType: roachpb.VOTER_INCOMING, eligible: true},
{leaseholderType: roachpb.VOTER_OUTGOING, eligible: false},
{leaseholderType: roachpb.VOTER_DEMOTING_LEARNER, eligible: false},
{leaseholderType: roachpb.VOTER_DEMOTING_NON_VOTER, eligible: false},
{leaseholderType: roachpb.LEARNER, eligible: false},
{leaseholderType: roachpb.NON_VOTER, eligible: false},
{leaseholderType: roachpb.VOTER_FULL, anotherReplicaType: none, eligibleLhRemovalEnabled: true, eligibleLhRemovalDisabled: true},
{leaseholderType: roachpb.VOTER_INCOMING, anotherReplicaType: none, eligibleLhRemovalEnabled: true, eligibleLhRemovalDisabled: 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_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_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.LEARNER, anotherReplicaType: none, eligibleLhRemovalEnabled: false, eligibleLhRemovalDisabled: false},
{leaseholderType: roachpb.NON_VOTER, anotherReplicaType: none, eligibleLhRemovalEnabled: false, eligibleLhRemovalDisabled: false},
} {
t.Run(tc.leaseholderType.String(), func(t *testing.T) {
repDesc := roachpb.ReplicaDescriptor{
Expand All @@ -285,14 +306,25 @@ func TestCheckCanReceiveLease(t *testing.T) {
rngDesc := roachpb.RangeDescriptor{
InternalReplicas: []roachpb.ReplicaDescriptor{repDesc},
}
err := roachpb.CheckCanReceiveLease(rngDesc.InternalReplicas[0], rngDesc.Replicas())
require.Equal(t, tc.eligible, err == nil, "err: %v", err)
if tc.anotherReplicaType != none {
anotherDesc := roachpb.ReplicaDescriptor{
ReplicaID: 2,
Type: &tc.anotherReplicaType,
}
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)

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

t.Run("replica not in range desc", func(t *testing.T) {
repDesc := roachpb.ReplicaDescriptor{ReplicaID: 1}
rngDesc := roachpb.RangeDescriptor{}
require.Regexp(t, "replica.*not found", roachpb.CheckCanReceiveLease(repDesc, rngDesc.Replicas()))
require.Regexp(t, "replica.*not found", roachpb.CheckCanReceiveLease(repDesc,
rngDesc.Replicas(), true))
})
}
7 changes: 6 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_lease_transfer.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
Expand Down Expand Up @@ -76,9 +77,13 @@ func TransferLease(
newLease := args.Lease
args.Lease = roachpb.Lease{} // prevent accidental use below

lhRemovalAllowed := cArgs.EvalCtx.ClusterSettings().Version.IsActive(ctx,
clusterversion.EnableLeaseHolderRemoval)
// 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()); err != nil {
if err := roachpb.CheckCanReceiveLease(
newLease.Replica, cArgs.EvalCtx.Desc().Replicas(), lhRemovalAllowed,
); err != nil {
return newFailedLeaseTrigger(true /* isTransfer */), err
}

Expand Down
Loading

0 comments on commit 365b9c4

Please sign in to comment.