From 991b134fd41c965b0da3d8cbd12a9f2ee05c6b6a Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Mon, 19 Sep 2022 22:15:14 -0400 Subject: [PATCH] kvserver: (partially) deflake transfer-leases/drain-other-node In #85629 we changed our lease transfer protocol to only ever transfer expiration-based leases, and have recipients later upgrade them to the more efficient epoch based ones. This was done to limit the effects of ill-advised lease transfers since the incoming leaseholder would need to recognize itself as such within a few seconds -- so we wanted this upgrade happen after having received the lease. In #83261 however we noticed that the upgrade was not immediate -- we were waiting until the current lease's expiration was within its renewal duration -- 4.5s. When the lease was eventually renewed the upgrade did happen, but it was not immediate. We fix this here and remove the manual clock advancing the supporting test had that masked this issue. It now demonstrates that we're no longer relying on upgrades happen as part of the (slow) renewal process. Release note: None --- pkg/kv/kvserver/client_lease_test.go | 41 +++++++++++++++++++++++----- pkg/kv/kvserver/client_merge_test.go | 6 ++++ pkg/kv/kvserver/replica_proposal.go | 7 ++++- pkg/kv/kvserver/testing_knobs.go | 4 +++ 4 files changed, 50 insertions(+), 8 deletions(-) diff --git a/pkg/kv/kvserver/client_lease_test.go b/pkg/kv/kvserver/client_lease_test.go index f63805890c77..bc435d368d59 100644 --- a/pkg/kv/kvserver/client_lease_test.go +++ b/pkg/kv/kvserver/client_lease_test.go @@ -41,6 +41,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) @@ -701,8 +702,9 @@ func TestLeaseholderRelocate(t *testing.T) { // We start with having the range under test on (1,2,3). tc.AddVotersOrFatal(t, rhsDesc.StartKey.AsRawKey(), tc.Targets(1, 2)...) - // Make sure the lease is on 3 + // Make sure the lease is on 3 and is fully upgraded. tc.TransferRangeLeaseOrFatal(t, rhsDesc, tc.Target(2)) + tc.WaitForLeaseUpgrade(ctx, t, rhsDesc) // Check that the lease moved to 3. leaseHolder, err := tc.FindRangeLeaseHolder(rhsDesc, nil) @@ -730,7 +732,7 @@ func TestLeaseholderRelocate(t *testing.T) { return nil }) - // Make sure lease moved to the preferred region, if . + // Make sure lease moved to the preferred region. leaseHolder, err = tc.FindRangeLeaseHolder(rhsDesc, nil) require.NoError(t, err) require.Equal(t, tc.Target(3), leaseHolder) @@ -739,10 +741,13 @@ func TestLeaseholderRelocate(t *testing.T) { repl := tc.GetFirstStoreFromServer(t, 3). LookupReplica(roachpb.RKey(rhsDesc.StartKey.AsRawKey())) history := repl.GetLeaseHistory() + require.Equal(t, leaseHolder.NodeID, history[len(history)-1].Replica.NodeID) + require.Equal(t, leaseHolder.NodeID, + history[len(history)-2].Replica.NodeID) // account for the lease upgrade require.Equal(t, tc.Target(2).NodeID, - history[len(history)-2].Replica.NodeID) + history[len(history)-3].Replica.NodeID) } func gossipLiveness(t *testing.T, tc *testcluster.TestCluster) { @@ -1303,10 +1308,17 @@ func TestAcquireLeaseTimeout(t *testing.T) { } } +// TestLeaseTransfersUseExpirationLeasesAndBumpToEpochBasedOnes does what it +// says on the tin. func TestLeaseTransfersUseExpirationLeasesAndBumpToEpochBasedOnes(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + mu := struct { + syncutil.Mutex + lease *roachpb.Lease + }{} + ctx := context.Background() manualClock := hlc.NewHybridManualClock() @@ -1315,10 +1327,21 @@ func TestLeaseTransfersUseExpirationLeasesAndBumpToEpochBasedOnes(t *testing.T) ServerArgs: base.TestServerArgs{ Knobs: base.TestingKnobs{ Server: &server.TestingKnobs{ + // Never ticked -- demonstrating that we're not relying on + // internal timers to upgrade leases. WallClock: manualClock, }, Store: &kvserver.StoreTestingKnobs{ - LeaseRenewalDurationOverride: 10 * time.Millisecond, // speed up the test + // Outlandishly high to disable proactive renewal of + // expiration based leases. Lease upgrades happen + // immediately after applying without needing active + // renewal. + LeaseRenewalDurationOverride: 100 * time.Hour, + LeaseUpgradeInterceptor: func(lease *roachpb.Lease) { + mu.Lock() + defer mu.Unlock() + mu.lease = lease + }, }, }, }, @@ -1333,8 +1356,7 @@ func TestLeaseTransfersUseExpirationLeasesAndBumpToEpochBasedOnes(t *testing.T) n2 := tc.Server(1) n2Target := tc.Target(1) - // Transfer the lease from n1 to n2. Expect it to be transferred as an - // expiration based lease. + // Transfer the lease from n1 to n2. tc.TransferRangeLeaseOrFatal(t, desc, n2Target) testutils.SucceedsSoon(t, func() error { li, _, err := tc.FindRangeLeaseEx(ctx, desc, nil) @@ -1346,6 +1368,11 @@ func TestLeaseTransfersUseExpirationLeasesAndBumpToEpochBasedOnes(t *testing.T) return nil }) - tc.IncrClockForLeaseUpgrade(t, manualClock) + // Expect it to be upgraded to an epoch based lease. tc.WaitForLeaseUpgrade(ctx, t, desc) + + // Expect it to have been upgraded from an expiration based lease. + mu.Lock() + defer mu.Unlock() + require.Equal(t, roachpb.LeaseExpiration, mu.lease.Type()) } diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index 0e42078a19de..0d2ee436a716 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -539,6 +539,9 @@ func mergeCheckingTimestampCaches( if !rhsRepl.OwnsValidLease(ctx, tc.Servers[1].Clock().NowAsClockTimestamp()) { return errors.New("rhs store does not own valid lease for rhs range") } + if rhsRepl.CurrentLeaseStatus(ctx).Lease.Type() != roachpb.LeaseEpoch { + return errors.Errorf("lease still an expiration based lease") + } return nil }) } @@ -1005,6 +1008,9 @@ func TestStoreRangeMergeTimestampCacheCausality(t *testing.T) { if !lhsRepl1.OwnsValidLease(ctx, tc.Servers[1].Clock().NowAsClockTimestamp()) { return errors.New("s2 does not own valid lease for lhs range") } + if lhsRepl1.CurrentLeaseStatus(ctx).Lease.Type() != roachpb.LeaseEpoch { + return errors.Errorf("lease still an expiration based lease") + } return nil }) diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index 308deef3c54b..92a161fd7daf 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -374,8 +374,13 @@ func (r *Replica) leasePostApplyLocked( if log.V(1) { log.VEventf(ctx, 1, "upgrading expiration lease %s to an epoch-based one", newLease) } + + if r.store.TestingKnobs().LeaseUpgradeInterceptor != nil { + r.store.TestingKnobs().LeaseUpgradeInterceptor(newLease) + } st := r.leaseStatusForRequestRLocked(ctx, now, hlc.Timestamp{}) - r.maybeExtendLeaseAsyncLocked(ctx, st) + // Ignore the returned handle as we won't block on it. + _ = r.requestLeaseLocked(ctx, st) } } diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index 2763515b0774..8eb85129b526 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -434,6 +434,10 @@ type StoreTestingKnobs struct { // - rangefeed.TestingKnobs.IgnoreOnDeleteRangeError // - kvserverbase.BatchEvalTestingKnobs.DisableInitPutFailOnTombstones GlobalMVCCRangeTombstone bool + + // LeaseUpgradeInterceptor intercepts leases that get upgraded to + // epoch-based ones. + LeaseUpgradeInterceptor func(*roachpb.Lease) } // ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface.