diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index 8e1bfcf78f5c..06b593b361c4 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -694,9 +694,6 @@ func setupLeaseTransferTest(t *testing.T) *leaseTransferTest { TestingEvalFilter: testingEvalFilter, }, LeaseTransferBlockedOnExtensionEvent: leaseTransferBlockedOnExtensionEvent, - // TODO(andrei): remove this knob once #59179 is fixed. It should - // only be needed by TestLeaseExpirationBelowFutureTimeRequest. - AllowLeaseRequestProposalsWhenNotLeader: true, }, Server: &server.TestingKnobs{ ClockSource: l.manualClock.UnixNano, @@ -1072,10 +1069,15 @@ func TestLeaseExpirationBelowFutureTimeRequest(t *testing.T) { l := setupLeaseTransferTest(t) defer l.tc.Stopper().Stop(ctx) - // Ensure that replica1 has the lease. + // 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)) l.checkHasLease(t, 1) preLease, _ := l.replica1.GetLease() + require.Eventually(t, func() bool { + lease, _ := l.replica0.GetLease() + return lease.Replica.StoreID == l.replica1.StoreID() + }, 5*time.Second, 100*time.Millisecond, "timed out waiting for replica 0 to pick up new lease") // Pause the cluster's clocks. l.manualClock.Pause() diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index 2d4cb62846fb..85deb72a6e99 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -223,6 +223,7 @@ type proposer interface { withGroupLocked(func(proposerRaft) error) error registerProposalLocked(*ProposalData) leaderStatusRLocked(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 // acquisitions when another replica is the leader; the intended consequence @@ -562,18 +563,26 @@ func (b *propBuf) FlushLockedWithRaftGroup( // // A special case is when the leader is known, but is ineligible to get the // lease. In that case, we have no choice but to continue with the proposal. + // + // Lease extensions for a currently held lease always go through, to + // keep the lease alive until the normal lease transfer mechanism can + // colocate it with the leader. if !leaderInfo.iAmTheLeader && p.Request.IsLeaseRequest() { leaderKnownAndEligible := leaderInfo.leaderKnown && leaderInfo.leaderEligibleForLease - if leaderKnownAndEligible && !b.testing.allowLeaseProposalWhenNotLeader { + ownsCurrentLease := b.p.ownsValidLeaseRLocked(ctx, b.clock.NowAsClockTimestamp()) + if leaderKnownAndEligible && !ownsCurrentLease && !b.testing.allowLeaseProposalWhenNotLeader { log.VEventf(ctx, 2, "not proposing lease acquisition because we're not the leader; replica %d is", leaderInfo.leader) b.p.rejectProposalWithRedirectLocked(ctx, p, leaderInfo.leader) p.tok.doneIfNotMovedLocked(ctx) continue } - // If the leader is not known, or if it is known but it's ineligible for - // the lease, continue with the proposal as explained above. - if !leaderInfo.leaderKnown { + // If the leader is not known, or if it is known but it's ineligible + // for the lease, continue with the proposal as explained above. We + // also send lease extensions for an existing leaseholder. + if ownsCurrentLease { + log.VEventf(ctx, 2, "proposing lease extension even though we're not the leader; we hold the current lease") + } else if !leaderInfo.leaderKnown { log.VEventf(ctx, 2, "proposing lease acquisition even though we're not the leader; the leader is unknown") } else { log.VEventf(ctx, 2, "proposing lease acquisition even though we're not the leader; the leader is ineligible") @@ -1066,6 +1075,10 @@ func (rp *replicaProposer) leaderStatusRLocked(raftGroup proposerRaft) rangeLead } } +func (rp *replicaProposer) ownsValidLeaseRLocked(ctx context.Context, now hlc.ClockTimestamp) bool { + return (*Replica)(rp).ownsValidLeaseRLocked(ctx, now) +} + // rejectProposalWithRedirectLocked is part of the proposer interface. func (rp *replicaProposer) rejectProposalWithRedirectLocked( ctx context.Context, prop *ProposalData, redirectTo roachpb.ReplicaID, diff --git a/pkg/kv/kvserver/replica_proposal_buf_test.go b/pkg/kv/kvserver/replica_proposal_buf_test.go index 83460917ff6a..f7dd181a69d9 100644 --- a/pkg/kv/kvserver/replica_proposal_buf_test.go +++ b/pkg/kv/kvserver/replica_proposal_buf_test.go @@ -51,6 +51,8 @@ type testProposer struct { // If not nil, this is called by RejectProposalWithRedirectLocked(). If nil, // RejectProposalWithRedirectLocked() panics. onRejectProposalWithRedirectLocked func(prop *ProposalData, redirectTo roachpb.ReplicaID) + // ownsValidLease is returned by ownsValidLeaseRLocked() + ownsValidLease bool // leaderReplicaInDescriptor is set if the leader (as indicated by raftGroup) // is known, and that leader is part of the range's descriptor (as seen by the @@ -148,6 +150,10 @@ func (t *testProposer) registerProposalLocked(p *ProposalData) { t.registered++ } +func (t *testProposer) ownsValidLeaseRLocked(ctx context.Context, now hlc.ClockTimestamp) bool { + return t.ownsValidLease +} + func (t *testProposer) leaderStatusRLocked(raftGroup proposerRaft) rangeLeaderInfo { leaderKnown := raftGroup.BasicStatus().Lead != raft.None var leaderRep roachpb.ReplicaID @@ -560,6 +566,8 @@ func TestProposalBufferRejectLeaseAcqOnFollower(t *testing.T) { // Set to simulate situations where the local replica is so behind that the // leader is not even part of the range descriptor. leaderNotInRngDesc bool + // If true, the follower has a valid lease. + ownsValidLease bool expRejection bool }{ @@ -578,6 +586,15 @@ func TestProposalBufferRejectLeaseAcqOnFollower(t *testing.T) { // Rejection - a follower can't request a lease. expRejection: true, }, + { + name: "follower, lease extension despite known eligible leader", + state: raft.StateFollower, + // Someone else is leader, but we're the leaseholder. + leader: self + 1, + ownsValidLease: true, + // No rejection of lease extensions. + expRejection: false, + }, { name: "follower, known ineligible leader", state: raft.StateFollower, @@ -643,6 +660,7 @@ func TestProposalBufferRejectLeaseAcqOnFollower(t *testing.T) { p.raftGroup = r p.leaderReplicaInDescriptor = !tc.leaderNotInRngDesc p.leaderReplicaType = tc.leaderRepType + p.ownsValidLease = tc.ownsValidLease var b propBuf clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond)