Skip to content

Commit

Permalink
kvserver: allow voter demoting to get a lease, in case there's an inc…
Browse files Browse the repository at this point in the history
…oming voter

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.

TODO: test and backport this

Release note: None
  • Loading branch information
shralex committed Jul 6, 2022
1 parent 82923fc commit ce195e7
Show file tree
Hide file tree
Showing 11 changed files with 226 additions and 59 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); 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
6 changes: 5 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,12 @@ 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
55 changes: 43 additions & 12 deletions pkg/kv/kvserver/batcheval/cmd_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -265,17 +265,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 +305,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
118 changes: 110 additions & 8 deletions pkg/kv/kvserver/client_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -164,19 +164,20 @@ func TestGossipNodeLivenessOnLeaseChange(t *testing.T) {
}

// TestCannotTransferLeaseToVoterOutgoing ensures that the evaluation of lease
// requests for nodes which are already in the VOTER_OUTGOING state will fail.
func TestCannotTransferLeaseToVoterOutgoing(t *testing.T) {
// requests for nodes which are already in the VOTER_DEMOTING_LEARNER state will fail
// (in this test, there is no VOTER_INCOMING node).
func TestCannotTransferLeaseToVoterDemoting(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()

knobs, ltk := makeReplicationTestKnobs()
// Add a testing knob to allow us to block the change replicas command
// while it is being proposed. When we detect that the change replicas
// command to move n3 to VOTER_OUTGOING has been evaluated, we'll send
// the request to transfer the lease to n3. The hope is that it will
// get past the sanity above latch acquisition prior to change replicas
// command committing.
// command to move n3 to VOTER_DEMOTING_LEARNER has been evaluated, we'll
// send the request to transfer the lease to n3. The hope is that it will
// get past the sanity check above latch acquisition prior to the change
// replicas command committing.
var scratchRangeID atomic.Value
scratchRangeID.Store(roachpb.RangeID(0))
changeReplicasChan := make(chan chan struct{}, 1)
Expand Down Expand Up @@ -214,7 +215,7 @@ func TestCannotTransferLeaseToVoterOutgoing(t *testing.T) {
// The test proceeds as follows:
//
// - Send an AdminChangeReplicasRequest to remove n3 and add n4
// - Block the step that moves n3 to VOTER_OUTGOING on changeReplicasChan
// - 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.
Expand All @@ -229,7 +230,6 @@ func TestCannotTransferLeaseToVoterOutgoing(t *testing.T) {
_, err = tc.Server(0).DB().AdminChangeReplicas(ctx,
scratchStartKey, desc, []roachpb.ReplicationChange{
{ChangeType: roachpb.REMOVE_VOTER, Target: tc.Target(2)},
{ChangeType: roachpb.ADD_VOTER, Target: tc.Target(3)},
})
require.NoError(t, err)
}()
Expand Down Expand Up @@ -262,7 +262,109 @@ func TestCannotTransferLeaseToVoterOutgoing(t *testing.T) {
close(ch)
wg.Wait()
})
}

// 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 TestTransferLeaseToVoterDemotinggWithIncoming(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()

knobs, ltk := makeReplicationTestKnobs()
// Add a testing knob to allow us to block the change replicas command
// while it is being proposed. When we detect that the change replicas
// command to move n3 to VOTER_DEMOTING_LEARNER has been evaluated, we'll
// send the request to transfer the lease to n3. The hope is that it will
// get past the sanity check above latch acquisition prior to the change
// replicas command committing.
var scratchRangeID atomic.Value
scratchRangeID.Store(roachpb.RangeID(0))
changeReplicasChan := make(chan chan struct{}, 1)
shouldBlock := func(args kvserverbase.ProposalFilterArgs) bool {
// Block if a ChangeReplicas command is removing a node from our range.
return args.Req.RangeID == scratchRangeID.Load().(roachpb.RangeID) &&
args.Cmd.ReplicatedEvalResult.ChangeReplicas != nil &&
len(args.Cmd.ReplicatedEvalResult.ChangeReplicas.Removed()) > 0
}
blockIfShould := func(args kvserverbase.ProposalFilterArgs) {
if shouldBlock(args) {
ch := make(chan struct{})
changeReplicasChan <- ch
<-ch
}
}
knobs.Store.(*kvserver.StoreTestingKnobs).TestingProposalFilter = func(args kvserverbase.ProposalFilterArgs) *roachpb.Error {
blockIfShould(args)
return nil
}
tc := testcluster.StartTestCluster(t, 4, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{Knobs: knobs},
ReplicationMode: base.ReplicationManual,
})
defer tc.Stopper().Stop(ctx)

scratchStartKey := tc.ScratchRange(t)
desc := tc.AddVotersOrFatal(t, scratchStartKey, tc.Targets(1, 2)...)
scratchRangeID.Store(desc.RangeID)
// Make sure n1 has the lease to start with.
err := tc.Server(0).DB().AdminTransferLease(context.Background(),
scratchStartKey, tc.Target(0).StoreID)
require.NoError(t, err)

// The test proceeds as follows:
//
// - 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.
// - Unblock the ChangeReplicas.
// - Make sure the lease transfer succeeds.

ltk.withStopAfterJointConfig(func() {
var wg sync.WaitGroup
wg.Add(1)
go func() {
defer wg.Done()
_, err = tc.Server(0).DB().AdminChangeReplicas(ctx,
scratchStartKey, desc, []roachpb.ReplicationChange{
{ChangeType: roachpb.REMOVE_VOTER, Target: tc.Target(2)},
{ChangeType: roachpb.ADD_VOTER, Target: tc.Target(3)},
})
require.NoError(t, err)
}()
ch := <-changeReplicasChan
wg.Add(1)
go func() {
defer wg.Done()
// Make sure the lease is currently on n1.
desc, err := tc.LookupRange(scratchStartKey)
require.NoError(t, err)
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."))
// Move the lease to n3, the VOTER_DEMOTING_LEARNER.
err = tc.Server(0).DB().AdminTransferLease(context.Background(),
scratchStartKey, tc.Target(2).StoreID)
require.NoError(t, err)
// Make sure the lease moved to n3.
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."))
}()
// Try really hard to make sure that our request makes it past the
// sanity check error to the evaluation error.
for i := 0; i < 100; i++ {
runtime.Gosched()
time.Sleep(time.Microsecond)
}
close(ch)
wg.Wait()
})
}

// TestStoreLeaseTransferTimestampCacheRead verifies that the timestamp cache on
Expand Down
13 changes: 9 additions & 4 deletions pkg/kv/kvserver/replica_proposal_buf.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"sync"
"sync/atomic"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/tracker"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftutil"
Expand Down Expand Up @@ -150,7 +151,7 @@ type proposer interface {
// The following require the proposer to hold an exclusive lock.
withGroupLocked(func(proposerRaft) error) error
registerProposalLocked(*ProposalData)
leaderStatusRLocked(raftGroup proposerRaft) rangeLeaderInfo
leaderStatusRLocked(ctx context.Context, raftGroup proposerRaft) rangeLeaderInfo
ownsValidLeaseRLocked(ctx context.Context, now hlc.ClockTimestamp) bool
// rejectProposalWithRedirectLocked rejects a proposal and redirects the
// proposer to try it on another node. This is used to sometimes reject lease
Expand Down Expand Up @@ -713,7 +714,7 @@ func (b *propBuf) maybeRejectUnsafeProposalLocked(
// leaderStatusRLocked returns the rangeLeaderInfo for the provided raft group,
// or an empty rangeLeaderInfo if the raftGroup is nil.
func (b *propBuf) leaderStatusRLocked(ctx context.Context, raftGroup proposerRaft) rangeLeaderInfo {
leaderInfo := b.p.leaderStatusRLocked(raftGroup)
leaderInfo := b.p.leaderStatusRLocked(ctx, raftGroup)
// Sanity check.
if leaderInfo.leaderKnown && leaderInfo.leader == b.p.getReplicaID() &&
!leaderInfo.iAmTheLeader {
Expand Down Expand Up @@ -1185,7 +1186,9 @@ func (rp *replicaProposer) registerProposalLocked(p *ProposalData) {
rp.mu.proposals[p.idKey] = p
}

func (rp *replicaProposer) leaderStatusRLocked(raftGroup proposerRaft) rangeLeaderInfo {
func (rp *replicaProposer) leaderStatusRLocked(
ctx context.Context, raftGroup proposerRaft,
) rangeLeaderInfo {
r := (*Replica)(rp)

status := raftGroup.Status()
Expand All @@ -1205,7 +1208,9 @@ func (rp *replicaProposer) leaderStatusRLocked(raftGroup proposerRaft) rangeLead
// lease again, and by then hopefully we will have caught up.
leaderEligibleForLease = true
} else {
err := roachpb.CheckCanReceiveLease(leaderRep, rangeDesc.Replicas())
lhRemovalAllowed := r.store.cfg.Settings.Version.IsActive(
ctx, clusterversion.EnableLeaseHolderRemoval)
err := roachpb.CheckCanReceiveLease(leaderRep, rangeDesc.Replicas(), lhRemovalAllowed)
leaderEligibleForLease = err == nil
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_proposal_buf_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -196,7 +196,7 @@ func (t *testProposer) leaderStatusRLocked(raftGroup proposerRaft) rangeLeaderIn
rngDesc := roachpb.RangeDescriptor{
InternalReplicas: []roachpb.ReplicaDescriptor{repDesc},
}
err := roachpb.CheckCanReceiveLease(repDesc, rngDesc.Replicas())
err := roachpb.CheckCanReceiveLease(repDesc, rngDesc.Replicas(), true)
leaderEligibleForLease = err == nil
} else {
// This matches replicaProposed.leaderStatusRLocked().
Expand Down
39 changes: 21 additions & 18 deletions pkg/kv/kvserver/replica_raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -364,26 +364,29 @@ func (r *Replica) propose(
// we move to a joint config where v1 (VOTER_DEMOTING_LEARNER) transfer the
// lease to v4 (VOTER_INCOMING) directly.
// See also https://github.com/cockroachdb/cockroach/issues/67740.
replID := r.ReplicaID()
rDesc, ok := p.command.ReplicatedEvalResult.State.Desc.GetReplicaDescriptorByID(replID)
hasVoterIncoming := p.command.ReplicatedEvalResult.State.Desc.ContainsVoterIncoming()
lhRemovalAllowed := hasVoterIncoming && r.store.cfg.Settings.Version.IsActive(ctx,
clusterversion.EnableLeaseHolderRemoval)
// Previously, we were not allowed to enter a joint config where the
// leaseholder is being removed (i.e., not a full voter). In the new version
// we're allowed to enter such a joint config (if it has a VOTER_INCOMING),
// 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).
if !ok ||
(lhRemovalAllowed && !rDesc.IsAnyVoter()) ||
(!lhRemovalAllowed && !rDesc.IsVoterNewConfig()) {
err := errors.Mark(errors.Newf("received invalid ChangeReplicasTrigger %s to remove "+
"self (leaseholder); hasVoterIncoming: %v, lhRemovalAllowed: %v; proposed descriptor: %v",
crt, hasVoterIncoming, lhRemovalAllowed, p.command.ReplicatedEvalResult.State.Desc),
errMarkInvalidReplicationChange)
log.Errorf(p.ctx, "%v", err)
lhRemovalAllowed := r.store.cfg.Settings.Version.IsActive(
ctx, clusterversion.EnableLeaseHolderRemoval)
lhDescriptor, err := r.GetReplicaDescriptor()
if err != nil {
return roachpb.NewError(err)
}
proposedDesc := p.command.ReplicatedEvalResult.State.Desc
// This is a reconfiguration command, we make sure the proposed
// config is legal w.r.t. the current leaseholder: we now allow the
// leaseholder to be a VOTER_DEMOTING as long as there is a VOTER_INCOMING.
// Otherwise, the leaseholder must be a full voter in the target config.
// When the leaseholder is being removed, this check won't allow exiting
// the joint config before the lease is transferred away since the previous
// leaseholder is a LEARNER in the target config.
if err := roachpb.CheckCanReceiveLease(
lhDescriptor, proposedDesc.Replicas(), lhRemovalAllowed,
); err != nil {
e := errors.Mark(errors.Wrapf(err, "received invalid ChangeReplicasTrigger %s to "+
"remove self (leaseholder); lhRemovalAllowed: %v; proposed descriptor: %v", crt,
lhRemovalAllowed, proposedDesc), errMarkInvalidReplicationChange)
log.Errorf(p.ctx, "%v", e)
return roachpb.NewError(e)
}
} else if p.command.ReplicatedEvalResult.AddSSTable != nil {
log.VEvent(p.ctx, 4, "sideloadable proposal detected")
version = kvserverbase.RaftVersionSideloaded
Expand Down
Loading

0 comments on commit ce195e7

Please sign in to comment.