Skip to content

Commit

Permalink
kvserver: (partially) deflake transfer-leases/drain-other-node
Browse files Browse the repository at this point in the history
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
  • Loading branch information
irfansharif committed Sep 20, 2022
1 parent 55c6568 commit 991b134
Show file tree
Hide file tree
Showing 4 changed files with 50 additions and 8 deletions.
41 changes: 34 additions & 7 deletions pkg/kv/kvserver/client_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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)
Expand All @@ -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) {
Expand Down Expand Up @@ -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()
Expand All @@ -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
},
},
},
},
Expand All @@ -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)
Expand All @@ -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())
}
6 changes: 6 additions & 0 deletions pkg/kv/kvserver/client_merge_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
})
}
Expand Down Expand Up @@ -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
})

Expand Down
7 changes: 6 additions & 1 deletion pkg/kv/kvserver/replica_proposal.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
}

Expand Down
4 changes: 4 additions & 0 deletions pkg/kv/kvserver/testing_knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down

0 comments on commit 991b134

Please sign in to comment.