Skip to content

Commit

Permalink
kv: bypass lease transfer safety checks during joint consensus
Browse files Browse the repository at this point in the history
Fixes #88667.

This commit adds logic to bypass lease transfer safety checks (added in
034611b) when in a joint configuration and transferring the lease from a
VOTER_DEMOTING to a VOTER_INCOMING. We do so because we could get stuck
without a path to exit the joint configuration if we rejected this lease
transfer while waiting to confirm that the target is up-to-date on its
log. That confirmation may never arrive if the target is dead or
partitioned away, and while we'd rather not transfer the lease to a dead
node, at least we have a mechanism to recovery from that state. We also
just sent the VOTER_INCOMING a snapshot (as a LEARNER, before
promotion), so it is unlikely that the replica is actually dead or
behind on its log.

A better alternative here would be to introduce a mechanism to choose an
alternate lease transfer target after some amount of time, if the lease
transfer to the VOTER_INCOMING cannot be confirmed to be safe. We may do
this in the future, but given the proximity to the release and given that
this matches the behavior in v22.1, we choose this approach for now.

Release note: None

Release justification: Needed to resolve release blocker.
  • Loading branch information
nvanbenschoten committed Oct 9, 2022
1 parent 9cf8b37 commit 0036160
Show file tree
Hide file tree
Showing 16 changed files with 97 additions and 47 deletions.
7 changes: 5 additions & 2 deletions pkg/kv/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -734,7 +734,9 @@ func (b *Batch) adminUnsplit(splitKeyIn interface{}) {

// adminTransferLease is only exported on DB. It is here for symmetry with the
// other operations.
func (b *Batch) adminTransferLease(key interface{}, target roachpb.StoreID) {
func (b *Batch) adminTransferLease(
key interface{}, target roachpb.StoreID, bypassSafetyChecks bool,
) {
k, err := marshalKey(key)
if err != nil {
b.initResult(0, 0, notRaw, err)
Expand All @@ -744,7 +746,8 @@ func (b *Batch) adminTransferLease(key interface{}, target roachpb.StoreID) {
RequestHeader: roachpb.RequestHeader{
Key: k,
},
Target: target,
Target: target,
BypassSafetyChecks: bypassSafetyChecks,
}
b.appendReqs(req)
b.initResult(1, 0, notRaw, nil)
Expand Down
13 changes: 12 additions & 1 deletion pkg/kv/db.go
Original file line number Diff line number Diff line change
Expand Up @@ -649,7 +649,18 @@ func (db *DB) AdminTransferLease(
ctx context.Context, key interface{}, target roachpb.StoreID,
) error {
b := &Batch{}
b.adminTransferLease(key, target)
b.adminTransferLease(key, target, false /* bypassSafetyChecks */)
return getOneErr(db.Run(ctx, b), b)
}

// AdminTransferLeaseBypassingSafetyChecks is like AdminTransferLease, but
// configures the lease transfer to bypass safety checks. See the comment on
// AdminTransferLeaseRequest.BypassSafetyChecks for details.
func (db *DB) AdminTransferLeaseBypassingSafetyChecks(
ctx context.Context, key interface{}, target roachpb.StoreID,
) error {
b := &Batch{}
b.adminTransferLease(key, target, true /* bypassSafetyChecks */)
return getOneErr(db.Run(ctx, b), b)
}

Expand Down
4 changes: 3 additions & 1 deletion pkg/kv/kvserver/asim/storerebalancer/candidate_replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,9 @@ func (sr *simulatorReplica) Stats() *replicastats.RatedSummary {
}

// AdminTransferLease transfers the LeaderLease to another replica.
func (sr *simulatorReplica) AdminTransferLease(ctx context.Context, target roachpb.StoreID) error {
func (sr *simulatorReplica) AdminTransferLease(
ctx context.Context, target roachpb.StoreID, bypassSafetyChecks bool,
) error {
if !sr.state.ValidTransfer(sr.repl.Range(), state.StoreID(target)) {
return errors.Errorf(
"unable to transfer lease for r%d to store %d, invalid transfer.",
Expand Down
9 changes: 6 additions & 3 deletions pkg/kv/kvserver/client_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -398,13 +398,16 @@ func internalTransferLeaseFailureDuringJointConfig(t *testing.T, isManual bool)
// range when they are being proposed.
var scratchRangeID int64
shouldFailProposal := func(args kvserverbase.ProposalFilterArgs) bool {
// Block if a ChangeReplicas command is removing a node from our range.
return args.Req.RangeID == roachpb.RangeID(atomic.LoadInt64(&scratchRangeID)) &&
args.Req.IsSingleTransferLeaseRequest()
}
const failureMsg = "injected lease transfer"
knobs.Store.(*kvserver.StoreTestingKnobs).TestingProposalFilter = func(args kvserverbase.ProposalFilterArgs) *roachpb.Error {
if shouldFailProposal(args) {
return roachpb.NewErrorf("Injecting lease transfer failure")
// The lease transfer should be configured to bypass safety checks.
// See maybeTransferLeaseDuringLeaveJoint for an explanation.
require.True(t, args.Req.Requests[0].GetTransferLease().BypassSafetyChecks)
return roachpb.NewErrorf(failureMsg)
}
return nil
}
Expand All @@ -430,7 +433,7 @@ func internalTransferLeaseFailureDuringJointConfig(t *testing.T, isManual bool)
{ChangeType: roachpb.ADD_VOTER, Target: tc.Target(3)},
})
require.Error(t, err)
require.Regexp(t, "Injecting lease transfer failure", err)
require.Regexp(t, failureMsg, err)

// We're now in a joint configuration, n1 already revoked its lease but all
// other replicas think n1 is the leaseholder. As long as n1 is alive, it is
Expand Down
23 changes: 12 additions & 11 deletions pkg/kv/kvserver/client_replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1589,7 +1589,7 @@ func TestLeaseExpirationBasedRangeTransfer(t *testing.T) {
origLease, _ := l.replica0.GetLease()
{
// Transferring the lease to ourself should be a no-op.
if err := l.replica0.AdminTransferLease(ctx, l.replica0Desc.StoreID); err != nil {
if err := l.replica0.AdminTransferLease(ctx, l.replica0Desc.StoreID, false /* bypassSafetyChecks */); err != nil {
t.Fatal(err)
}
newLease, _ := l.replica0.GetLease()
Expand All @@ -1601,12 +1601,12 @@ func TestLeaseExpirationBasedRangeTransfer(t *testing.T) {
{
// An invalid target should result in an error.
const expected = "unable to find store .* in range"
if err := l.replica0.AdminTransferLease(ctx, 1000); !testutils.IsError(err, expected) {
if err := l.replica0.AdminTransferLease(ctx, 1000, false /* bypassSafetyChecks */); !testutils.IsError(err, expected) {
t.Fatalf("expected %s, but found %v", expected, err)
}
}

if err := l.replica0.AdminTransferLease(ctx, l.replica1Desc.StoreID); err != nil {
if err := l.replica0.AdminTransferLease(ctx, l.replica1Desc.StoreID, false /* bypassSafetyChecks */); err != nil {
t.Fatal(err)
}

Expand Down Expand Up @@ -1651,7 +1651,7 @@ func TestLeaseExpirationBasedRangeTransferWithExtension(t *testing.T) {
l := setupLeaseTransferTest(t)
defer l.tc.Stopper().Stop(ctx)
// Ensure that replica1 has the lease.
if err := l.replica0.AdminTransferLease(ctx, l.replica1Desc.StoreID); err != nil {
if err := l.replica0.AdminTransferLease(ctx, l.replica1Desc.StoreID, false /* bypassSafetyChecks */); err != nil {
t.Fatal(err)
}
l.checkHasLease(t, 1)
Expand All @@ -1673,7 +1673,7 @@ func TestLeaseExpirationBasedRangeTransferWithExtension(t *testing.T) {
transferErrCh := make(chan error)
go func() {
// Transfer back from replica1 to replica0.
err := l.replica1.AdminTransferLease(context.Background(), l.replica0Desc.StoreID)
err := l.replica1.AdminTransferLease(context.Background(), l.replica0Desc.StoreID, false /* bypassSafetyChecks */)
// Ignore not leaseholder errors which can arise due to re-proposals.
if errors.HasType(err, (*roachpb.NotLeaseHolderError)(nil)) {
err = nil
Expand Down Expand Up @@ -1738,7 +1738,7 @@ func TestLeaseExpirationBasedDrainTransferWithExtension(t *testing.T) {
l := setupLeaseTransferTest(t)
defer l.tc.Stopper().Stop(ctx)
// Ensure that replica1 has the lease.
if err := l.replica0.AdminTransferLease(ctx, l.replica1Desc.StoreID); err != nil {
if err := l.replica0.AdminTransferLease(ctx, l.replica1Desc.StoreID, false /* bypassSafetyChecks */); err != nil {
t.Fatal(err)
}
l.checkHasLease(t, 1)
Expand Down Expand Up @@ -1791,7 +1791,7 @@ func TestLeaseExpirationBelowFutureTimeRequest(t *testing.T) {

// Ensure that replica1 has the lease, and that replica0 has also picked up
// on the lease transfer.
require.NoError(t, l.replica0.AdminTransferLease(ctx, l.replica1Desc.StoreID))
require.NoError(t, l.replica0.AdminTransferLease(ctx, l.replica1Desc.StoreID, false /* bypassSafetyChecks */))
l.checkHasLease(t, 1)
preLease, _ := l.replica1.GetLease()
require.Eventually(t, func() bool {
Expand Down Expand Up @@ -2780,9 +2780,6 @@ func TestLeaseTransferInSnapshotUpdatesTimestampCache(t *testing.T) {
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
Store: &kvserver.StoreTestingKnobs{
AllowLeaseTransfersWhenTargetMayNeedSnapshot: true,
},
Server: &server.TestingKnobs{
WallClock: manualClock,
},
Expand Down Expand Up @@ -2872,7 +2869,11 @@ func TestLeaseTransferInSnapshotUpdatesTimestampCache(t *testing.T) {
// Finally, transfer the lease to node 2 while it is still unavailable and
// behind. We try to avoid this case when picking new leaseholders in practice,
// but we're never 100% successful.
tc.TransferRangeLeaseOrFatal(t, *repl0.Desc(), tc.Target(2))
// NOTE: we bypass safety checks because the target node is behind on its log,
// so the lease transfer would be rejected otherwise.
err = tc.Servers[0].DB().AdminTransferLeaseBypassingSafetyChecks(ctx,
repl0.Desc().StartKey.AsRawKey(), tc.Target(2).StoreID)
require.Nil(t, err)

// Remove the partition. A snapshot to node 2 should follow. This snapshot
// will inform node 2 that it is the new leaseholder for the range. Node 2
Expand Down
16 changes: 14 additions & 2 deletions pkg/kv/kvserver/replica_command.go
Original file line number Diff line number Diff line change
Expand Up @@ -1230,7 +1230,19 @@ func (r *Replica) maybeTransferLeaseDuringLeaveJoint(
}
log.VEventf(ctx, 2, "leaseholder %v is being removed through an atomic "+
"replication change, transferring lease to %v", voterDemotingTarget, voterIncomingTarget)
err = r.store.DB().AdminTransferLease(ctx, r.startKey, voterIncomingTarget.StoreID)
// We bypass safety checks when transferring the lease to the VOTER_INCOMING.
// We do so because we could get stuck without a path to exit the joint
// configuration if we rejected this lease transfer while waiting to confirm
// that the target is up-to-date on its log. That confirmation may never
// arrive if the target is dead or partitioned away, and while we'd rather not
// transfer the lease to a dead node, at least we have a mechanism to recovery
// from that state. We also just sent the VOTER_INCOMING a snapshot (as a
// LEARNER, before promotion), so it is unlikely that the replica is actually
// dead or behind on its log.
// TODO(nvanbenschoten): this isn't great. Instead of bypassing safety checks,
// we should build a mechanism to choose an alternate lease transfer target
// after some amount of time.
err = r.store.DB().AdminTransferLeaseBypassingSafetyChecks(ctx, r.startKey, voterIncomingTarget.StoreID)
if err != nil {
return err
}
Expand Down Expand Up @@ -3693,7 +3705,7 @@ func (r *Replica) adminScatter(
targetStoreID := potentialLeaseTargets[newLeaseholderIdx].StoreID
if targetStoreID != r.store.StoreID() {
log.VEventf(ctx, 2, "randomly transferring lease to s%d", targetStoreID)
if err := r.AdminTransferLease(ctx, targetStoreID); err != nil {
if err := r.AdminTransferLease(ctx, targetStoreID, false /* bypassSafetyChecks */); err != nil {
log.Warningf(ctx, "failed to scatter lease to s%d: %+v", targetStoreID, err)
}
}
Expand Down
1 change: 0 additions & 1 deletion pkg/kv/kvserver/replica_init.go
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,6 @@ func newUnloadedReplica(
r.mu.checksums = map[uuid.UUID]*replicaChecksum{}
r.mu.proposalBuf.Init((*replicaProposer)(r), tracker.NewLockfreeTracker(), r.Clock(), r.ClusterSettings())
r.mu.proposalBuf.testing.allowLeaseProposalWhenNotLeader = store.cfg.TestingKnobs.AllowLeaseRequestProposalsWhenNotLeader
r.mu.proposalBuf.testing.allowLeaseTransfersWhenTargetMayNeedSnapshot = store.cfg.TestingKnobs.AllowLeaseTransfersWhenTargetMayNeedSnapshot
r.mu.proposalBuf.testing.dontCloseTimestamps = store.cfg.TestingKnobs.DontCloseTimestamps
r.mu.proposalBuf.testing.submitProposalFilter = store.cfg.TestingKnobs.TestingProposalSubmitFilter

Expand Down
6 changes: 1 addition & 5 deletions pkg/kv/kvserver/replica_proposal_buf.go
Original file line number Diff line number Diff line change
Expand Up @@ -111,10 +111,6 @@ type propBuf struct {
// heartbeats and then expect other replicas to take the lease without
// worrying about Raft).
allowLeaseProposalWhenNotLeader bool
// allowLeaseTransfersWhenTargetMayNeedSnapshot, if set, makes the proposal
// buffer allow lease request proposals even when the proposer cannot prove
// that the lease transfer target does not need a Raft snapshot.
allowLeaseTransfersWhenTargetMayNeedSnapshot bool
// dontCloseTimestamps inhibits the closing of timestamps.
dontCloseTimestamps bool
}
Expand Down Expand Up @@ -699,7 +695,7 @@ func (b *propBuf) maybeRejectUnsafeProposalLocked(
newLease := p.command.ReplicatedEvalResult.State.Lease
newLeaseTarget := newLease.Replica.ReplicaID
snapStatus := raftutil.ReplicaMayNeedSnapshot(&status, firstIndex, newLeaseTarget)
if snapStatus != raftutil.NoSnapshotNeeded && !b.testing.allowLeaseTransfersWhenTargetMayNeedSnapshot {
if snapStatus != raftutil.NoSnapshotNeeded && !p.Request.Requests[0].GetTransferLease().BypassSafetyChecks {
b.p.rejectProposalWithLeaseTransferRejectedLocked(ctx, p, newLease, snapStatus)
return true
}
Expand Down
26 changes: 19 additions & 7 deletions pkg/kv/kvserver/replica_range_lease.go
Original file line number Diff line number Diff line change
Expand Up @@ -197,6 +197,7 @@ func (p *pendingLeaseRequest) InitOrJoinRequest(
status kvserverpb.LeaseStatus,
startKey roachpb.Key,
transfer bool,
bypassSafetyChecks bool,
) *leaseRequestHandle {
if nextLease, ok := p.RequestPending(); ok {
if nextLease.Replica.ReplicaID == nextLeaseHolder.ReplicaID {
Expand Down Expand Up @@ -291,11 +292,19 @@ func (p *pendingLeaseRequest) InitOrJoinRequest(
var leaseReq roachpb.Request
if transfer {
leaseReq = &roachpb.TransferLeaseRequest{
RequestHeader: reqHeader,
Lease: reqLease,
PrevLease: status.Lease,
RequestHeader: reqHeader,
Lease: reqLease,
PrevLease: status.Lease,
BypassSafetyChecks: bypassSafetyChecks,
}
} else {
if bypassSafetyChecks {
// TODO(nvanbenschoten): we could support a similar bypassSafetyChecks
// flag for RequestLeaseRequest, which would disable the protection in
// propBuf.maybeRejectUnsafeProposalLocked. For now, we use a testing
// knob.
log.Fatal(ctx, "bypassSafetyChecks not supported for RequestLeaseRequest")
}
minProposedTS := p.repl.mu.minLeaseProposedTS
leaseReq = &roachpb.RequestLeaseRequest{
RequestHeader: reqHeader,
Expand Down Expand Up @@ -817,7 +826,8 @@ func (r *Replica) requestLeaseLocked(
return r.mu.pendingLeaseRequest.newResolvedHandle(roachpb.NewError(err))
}
return r.mu.pendingLeaseRequest.InitOrJoinRequest(
ctx, repDesc, status, r.mu.state.Desc.StartKey.AsRawKey(), false /* transfer */)
ctx, repDesc, status, r.mu.state.Desc.StartKey.AsRawKey(),
false /* transfer */, false /* bypassSafetyChecks */)
}

// AdminTransferLease transfers the LeaderLease to another replica. Only the
Expand All @@ -842,7 +852,9 @@ func (r *Replica) requestLeaseLocked(
// replica. Otherwise, a NotLeaseHolderError is returned.
//
// AdminTransferLease implements the ReplicaLeaseMover interface.
func (r *Replica) AdminTransferLease(ctx context.Context, target roachpb.StoreID) error {
func (r *Replica) AdminTransferLease(
ctx context.Context, target roachpb.StoreID, bypassSafetyChecks bool,
) error {
// initTransferHelper inits a transfer if no extension is in progress.
// It returns a channel for waiting for the result of a pending
// extension (if any is in progress) and a channel for waiting for the
Expand Down Expand Up @@ -896,7 +908,7 @@ func (r *Replica) AdminTransferLease(ctx context.Context, target roachpb.StoreID
raftStatus := r.raftStatusRLocked()
raftFirstIndex := r.raftFirstIndexRLocked()
snapStatus := raftutil.ReplicaMayNeedSnapshot(raftStatus, raftFirstIndex, nextLeaseHolder.ReplicaID)
if snapStatus != raftutil.NoSnapshotNeeded && !r.store.TestingKnobs().AllowLeaseTransfersWhenTargetMayNeedSnapshot {
if snapStatus != raftutil.NoSnapshotNeeded && !bypassSafetyChecks {
r.store.metrics.LeaseTransferErrorCount.Inc(1)
log.VEventf(ctx, 2, "not initiating lease transfer because the target %s may "+
"need a snapshot: %s", nextLeaseHolder, snapStatus)
Expand All @@ -905,7 +917,7 @@ func (r *Replica) AdminTransferLease(ctx context.Context, target roachpb.StoreID
}

transfer = r.mu.pendingLeaseRequest.InitOrJoinRequest(
ctx, nextLeaseHolder, status, desc.StartKey.AsRawKey(), true, /* transfer */
ctx, nextLeaseHolder, status, desc.StartKey.AsRawKey(), true /* transfer */, bypassSafetyChecks,
)
return nil, transfer, nil
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_rangefeed_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -830,7 +830,7 @@ func TestReplicaRangefeedRetryErrors(t *testing.T) {
if raftStatus != nil && raftStatus.RaftState == raft.StateFollower {
return nil
}
err = repl.AdminTransferLease(ctx, roachpb.StoreID(1))
err = repl.AdminTransferLease(ctx, roachpb.StoreID(1), false /* bypassSafetyChecks */)
// NB: errors.Wrapf(nil, ...) returns nil.
// nolint:errwrap
return errors.Errorf("not raft follower: %+v, transferred lease: %v", raftStatus, err)
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/replica_rankings.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,8 +57,8 @@ type CandidateReplica interface {
// Stats returns a snapshot of the QPS replica load stats
Stats() *replicastats.RatedSummary
// AdminTransferLease transfers the LeaderLease to another replica.
AdminTransferLease(ctx context.Context, target roachpb.StoreID) error
// Replica returns the underlying replica for this CandidateReplica. It is
AdminTransferLease(ctx context.Context, target roachpb.StoreID, bypassSafetyChecks bool) error
// Repl returns the underlying replica for this CandidateReplica. It is
// only used for determining timeouts in production code and not the
// simulator.
Repl() *Replica
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_send.go
Original file line number Diff line number Diff line change
Expand Up @@ -934,7 +934,7 @@ func (r *Replica) executeAdminBatch(
resp = &reply

case *roachpb.AdminTransferLeaseRequest:
pErr = roachpb.NewError(r.AdminTransferLease(ctx, tArgs.Target))
pErr = roachpb.NewError(r.AdminTransferLease(ctx, tArgs.Target, tArgs.BypassSafetyChecks))
resp = &roachpb.AdminTransferLeaseResponse{}

case *roachpb.AdminChangeReplicasRequest:
Expand Down
7 changes: 3 additions & 4 deletions pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -616,9 +616,6 @@ func TestBehaviorDuringLeaseTransfer(t *testing.T) {
tsc := TestStoreConfig(clock)
var leaseAcquisitionTrap atomic.Value
tsc.TestingKnobs.DisableAutomaticLeaseRenewal = true
// We're transferring the lease to a bogus replica, so disable protection
// which would otherwise notice this and reject the lease transfer.
tsc.TestingKnobs.AllowLeaseTransfersWhenTargetMayNeedSnapshot = true
tsc.TestingKnobs.LeaseRequestEvent = func(ts hlc.Timestamp, _ roachpb.StoreID, _ roachpb.RangeID) *roachpb.Error {
val := leaseAcquisitionTrap.Load()
if val == nil {
Expand Down Expand Up @@ -669,7 +666,9 @@ func TestBehaviorDuringLeaseTransfer(t *testing.T) {
// Initiate a transfer (async) and wait for it to be blocked.
transferResChan := make(chan error)
go func() {
err := tc.repl.AdminTransferLease(ctx, secondReplica.StoreID)
// We're transferring the lease to a bogus replica, so disable protection
// which would otherwise notice this and reject the lease transfer.
err := tc.repl.AdminTransferLease(ctx, secondReplica.StoreID, true /* bypassSafetyChecks */)
if !testutils.IsError(err, "injected") {
transferResChan <- err
} else {
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/replicate_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -1827,7 +1827,7 @@ func (rq *replicateQueue) shedLease(
// ReplicaLeaseMover handles lease transfers for a single range.
type ReplicaLeaseMover interface {
// AdminTransferLease moves the lease to the requested store.
AdminTransferLease(ctx context.Context, target roachpb.StoreID) error
AdminTransferLease(ctx context.Context, target roachpb.StoreID, bypassSafetyChecks bool) error

// String returns info about the replica.
String() string
Expand Down Expand Up @@ -1860,7 +1860,7 @@ func (rq *replicateQueue) TransferLease(
) error {
rq.metrics.TransferLeaseCount.Inc(1)
log.KvDistribution.Infof(ctx, "transferring lease to s%d", target)
if err := rlm.AdminTransferLease(ctx, target); err != nil {
if err := rlm.AdminTransferLease(ctx, target, false /* bypassSafetyChecks */); err != nil {
return errors.Wrapf(err, "%s: unable to transfer lease to s%d", rlm, target)
}
rq.lastLeaseTransfer.Store(timeutil.Now())
Expand Down
4 changes: 0 additions & 4 deletions pkg/kv/kvserver/testing_knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -340,10 +340,6 @@ type StoreTestingKnobs struct {
// heartbeats and then expect other replicas to take the lease without
// worrying about Raft).
AllowLeaseRequestProposalsWhenNotLeader bool
// AllowLeaseTransfersWhenTargetMayNeedSnapshot, if set, makes the Replica
// and proposal buffer allow lease request proposals even when the proposer
// cannot prove that the lease transfer target does not need a Raft snapshot.
AllowLeaseTransfersWhenTargetMayNeedSnapshot bool
// LeaseTransferRejectedRetryLoopCount, if set, configures the maximum number
// of retries for the retry loop used during lease transfers. This retry loop
// retries after transfer attempts are rejected because the transfer is deemed
Expand Down
Loading

0 comments on commit 0036160

Please sign in to comment.