From 434178afb7e86530c6334d74f7a0c3ce372d4170 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Fri, 12 Jan 2024 16:43:49 -0500 Subject: [PATCH 1/3] roachpb: improve TestLeaseEquivalence Test changes to just the leaseholder or just the start time, instead of both at the same time. This ensures that either lead to a sequence number change. Epic: None Release note: None --- pkg/roachpb/data_test.go | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index fa6ca7ff95f8..188e08883bf6 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -1032,11 +1032,15 @@ func TestLeaseEquivalence(t *testing.T) { ts3 := makeClockTS(3, 1) epoch1 := Lease{Replica: r1, Start: ts1, Epoch: 1} + epoch1R2 := Lease{Replica: r2, Start: ts1, Epoch: 1} + epoch1TS2 := Lease{Replica: r1, Start: ts2, Epoch: 1} epoch2 := Lease{Replica: r1, Start: ts1, Epoch: 2} + epoch2R2TS2 := Lease{Replica: r2, Start: ts2, Epoch: 2} expire1 := Lease{Replica: r1, Start: ts1, Expiration: ts2.ToTimestamp().Clone()} + expire1R2 := Lease{Replica: r2, Start: ts1, Expiration: ts2.ToTimestamp().Clone()} + expire1TS2 := Lease{Replica: r1, Start: ts2, Expiration: ts2.ToTimestamp().Clone()} expire2 := Lease{Replica: r1, Start: ts1, Expiration: ts3.ToTimestamp().Clone()} - epoch2TS2 := Lease{Replica: r2, Start: ts2, Epoch: 2} - expire2TS2 := Lease{Replica: r2, Start: ts2, Expiration: ts3.ToTimestamp().Clone()} + expire2R2TS2 := Lease{Replica: r2, Start: ts2, Expiration: ts3.ToTimestamp().Clone()} proposed1 := Lease{Replica: r1, Start: ts1, Epoch: 1, ProposedTS: &ts1} proposed2 := Lease{Replica: r1, Start: ts1, Epoch: 2, ProposedTS: &ts1} @@ -1057,9 +1061,13 @@ func TestLeaseEquivalence(t *testing.T) { }{ {epoch1, epoch1, true}, // same epoch lease {expire1, expire1, true}, // same expiration lease + {epoch1, epoch1R2, false}, // different epoch leases + {epoch1, epoch1TS2, false}, // different epoch leases {epoch1, epoch2, false}, // different epoch leases - {epoch1, epoch2TS2, false}, // different epoch leases - {expire1, expire2TS2, false}, // different expiration leases + {epoch1, epoch2R2TS2, false}, // different epoch leases + {expire1, expire1R2, false}, // different expiration leases + {expire1, expire1TS2, false}, // different expiration leases + {expire1, expire2R2TS2, false}, // different expiration leases {expire1, expire2, true}, // same expiration lease, extended {expire2, expire1, false}, // same expiration lease, extended but backwards {epoch1, expire1, false}, // epoch and expiration leases From f46f8e9e3d19e679542d92b4e32ad16a7b2d3298 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Fri, 12 Jan 2024 18:57:10 -0500 Subject: [PATCH 2/3] batcheval: consolidate read summary construction in evalNewLease This commit is a small refactor so that we can eliminate the second call to Lease.Equivalent in RequestLease. There is already one in evalNewLease which is shared between RequestLease and TransferLease. Epic: None Release note: None --- pkg/kv/kvserver/batcheval/cmd_lease.go | 26 ++++++++++++++++++- .../kvserver/batcheval/cmd_lease_request.go | 18 +------------ .../kvserver/batcheval/cmd_lease_transfer.go | 8 +----- 3 files changed, 27 insertions(+), 25 deletions(-) diff --git a/pkg/kv/kvserver/batcheval/cmd_lease.go b/pkg/kv/kvserver/batcheval/cmd_lease.go index 78967376b2a7..df29feaca138 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease.go @@ -56,7 +56,6 @@ func evalNewLease( ms *enginepb.MVCCStats, lease roachpb.Lease, prevLease roachpb.Lease, - priorReadSum *rspb.ReadSummary, isExtension bool, isTransfer bool, ) (result.Result, error) { @@ -102,6 +101,7 @@ func evalNewLease( Message: "sequence number should not be set", } } + var priorReadSum *rspb.ReadSummary if prevLease.Equivalent(lease) { // If the proposed lease is equivalent to the previous lease, it is // given the same sequence number. This is subtle, but is important @@ -121,6 +121,30 @@ func evalNewLease( // retry with a different sequence number. This is actually exactly what // the sequence number is used to enforce! lease.Sequence = prevLease.Sequence + 1 + + // If the new lease is not equivalent to the old lease, construct a read + // summary to instruct the new leaseholder on how to update its timestamp + // cache to respect prior reads served on the range. + if isTransfer { + // Collect a read summary from the outgoing leaseholder to ship to the + // incoming leaseholder. This is used to instruct the new leaseholder on + // how to update its timestamp cache to ensure that no future writes are + // allowed to invalidate prior reads. + localReadSum := rec.GetCurrentReadSummary(ctx) + priorReadSum = &localReadSum + } else { + // If the new lease is not equivalent to the old lease (i.e. either the + // lease is changing hands or the leaseholder restarted), construct a + // read summary to instruct the new leaseholder on how to update its + // timestamp cache. Since we are not the leaseholder ourselves, we must + // pessimistically assume that prior leaseholders served reads all the + // way up to the start of the new lease. + // + // NB: this is equivalent to the leaseChangingHands condition in + // leasePostApplyLocked. + worstCaseSum := rspb.FromTimestamp(lease.Start.ToTimestamp()) + priorReadSum = &worstCaseSum + } } // Record information about the type of event that resulted in this new lease. diff --git a/pkg/kv/kvserver/batcheval/cmd_lease_request.go b/pkg/kv/kvserver/batcheval/cmd_lease_request.go index 13040c882ddd..8b13288553c7 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease_request.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease_request.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/lockspanset" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -146,22 +145,7 @@ func RequestLease( } newLease.Start = effectiveStart - var priorReadSum *rspb.ReadSummary - if !prevLease.Equivalent(newLease) { - // If the new lease is not equivalent to the old lease (i.e. either the - // lease is changing hands or the leaseholder restarted), construct a - // read summary to instruct the new leaseholder on how to update its - // timestamp cache. Since we are not the leaseholder ourselves, we must - // pessimistically assume that prior leaseholders served reads all the - // way up to the start of the new lease. - // - // NB: this is equivalent to the leaseChangingHands condition in - // leasePostApplyLocked. - worstCaseSum := rspb.FromTimestamp(newLease.Start.ToTimestamp()) - priorReadSum = &worstCaseSum - } - log.VEventf(ctx, 2, "lease request: prev lease: %+v, new lease: %+v", prevLease, newLease) return evalNewLease(ctx, cArgs.EvalCtx, readWriter, cArgs.Stats, - newLease, prevLease, priorReadSum, isExtension, false /* isTransfer */) + newLease, prevLease, isExtension, false /* isTransfer */) } diff --git a/pkg/kv/kvserver/batcheval/cmd_lease_transfer.go b/pkg/kv/kvserver/batcheval/cmd_lease_transfer.go index 02e55d78d519..edea28dcd0cc 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease_transfer.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease_transfer.go @@ -115,13 +115,7 @@ func TransferLease( // previous lease was revoked). newLease.Start.Forward(cArgs.EvalCtx.Clock().NowAsClockTimestamp()) - // Collect a read summary from the outgoing leaseholder to ship to the - // incoming leaseholder. This is used to instruct the new leaseholder on how - // to update its timestamp cache to ensure that no future writes are allowed - // to invalidate prior reads. - priorReadSum := cArgs.EvalCtx.GetCurrentReadSummary(ctx) - log.VEventf(ctx, 2, "lease transfer: prev lease: %+v, new lease: %+v", prevLease, newLease) return evalNewLease(ctx, cArgs.EvalCtx, readWriter, cArgs.Stats, - newLease, prevLease, &priorReadSum, false /* isExtension */, true /* isTransfer */) + newLease, prevLease, false /* isExtension */, true /* isTransfer */) } From 54ebbaca27afea8263d7f84e6487a65eefb900ad Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Tue, 16 Jan 2024 18:01:31 -0500 Subject: [PATCH 3/3] kv: promote expiration-based lease to epoch without sequence number change Fixes #117630. Fixes #90656. Fixes #98553. Informs #61986. Informs #115191. This commit updates the post-lease transfer promotion of expiration-based leases to epoch-based leases to not change the sequence number of the lease. This avoids invalidating all requests proposed under the original expiration-based lease, which can lead to `RETRY_ASYNC_WRITE_FAILURE` errors. The change accomplishes this by updating the `Lease.Equivalent` method to consider an expiration-based lease to be equivalent to an epoch-based lease that is held by the same replica and has the same start time. Doing so requires some care, because lease equivalency is checked below Raft and needs to remain deterministic across binary versions. This change requires a cluster version check, so it cannot be backported. Release note (bug fix): Improved an interaction during range lease transfers which could previously cause `RETRY_ASYNC_WRITE_FAILURE` errors to be returned to clients. --- pkg/kv/kvserver/batcheval/cmd_lease.go | 4 +- pkg/kv/kvserver/client_lease_test.go | 11 +++-- pkg/kv/kvserver/client_replica_test.go | 2 +- pkg/kv/kvserver/kvserverbase/forced_error.go | 3 +- pkg/kv/kvserver/kvserverpb/proposer_kv.proto | 22 +++++++++ pkg/kv/kvserver/replica_application_result.go | 1 + pkg/kv/kvserver/replica_proposal.go | 18 +++++++- pkg/roachpb/data.go | 45 ++++++++++++++----- pkg/roachpb/data_test.go | 19 +++++--- pkg/testutils/testcluster/testcluster.go | 9 ++-- 10 files changed, 109 insertions(+), 25 deletions(-) diff --git a/pkg/kv/kvserver/batcheval/cmd_lease.go b/pkg/kv/kvserver/batcheval/cmd_lease.go index df29feaca138..a8da5c025167 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease.go @@ -14,6 +14,7 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -101,8 +102,9 @@ func evalNewLease( Message: "sequence number should not be set", } } + isV24_1 := rec.ClusterSettings().Version.IsActive(ctx, clusterversion.V24_1Start) var priorReadSum *rspb.ReadSummary - if prevLease.Equivalent(lease) { + if prevLease.Equivalent(lease, isV24_1 /* expToEpochEquiv */) { // If the proposed lease is equivalent to the previous lease, it is // given the same sequence number. This is subtle, but is important // to ensure that leases which are meant to be considered the same diff --git a/pkg/kv/kvserver/client_lease_test.go b/pkg/kv/kvserver/client_lease_test.go index e9107b9ce616..4aa87980dc61 100644 --- a/pkg/kv/kvserver/client_lease_test.go +++ b/pkg/kv/kvserver/client_lease_test.go @@ -1557,12 +1557,17 @@ func TestLeaseTransfersUseExpirationLeasesAndBumpToEpochBasedOnes(t *testing.T) }) // Expect it to be upgraded to an epoch based lease. - tc.WaitForLeaseUpgrade(ctx, t, desc) + epochL := tc.WaitForLeaseUpgrade(ctx, t, desc) + require.Equal(t, roachpb.LeaseEpoch, epochL.Type()) // Expect it to have been upgraded from an expiration based lease. mu.Lock() - defer mu.Unlock() - require.Equal(t, roachpb.LeaseExpiration, mu.lease.Type()) + expirationL := mu.lease + mu.Unlock() + require.Equal(t, roachpb.LeaseExpiration, expirationL.Type()) + + // Expect the two leases to have the same sequence number. + require.Equal(t, expirationL.Sequence, epochL.Sequence) } // TestLeaseRequestBumpsEpoch tests that a non-cooperative lease acquisition of diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index 3c13b37e8932..174fa0f244d9 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -1639,7 +1639,7 @@ func TestLeaseExpirationBasedRangeTransfer(t *testing.T) { t.Fatal(err) } newLease, _ := l.replica0.GetLease() - if !origLease.Equivalent(newLease) { + if !origLease.Equivalent(newLease, true /* expToEpochEquiv */) { t.Fatalf("original lease %v and new lease %v not equivalent", origLease, newLease) } } diff --git a/pkg/kv/kvserver/kvserverbase/forced_error.go b/pkg/kv/kvserver/kvserverbase/forced_error.go index 6b6aa3a16d71..ec6a25d0dc67 100644 --- a/pkg/kv/kvserver/kvserverbase/forced_error.go +++ b/pkg/kv/kvserver/kvserverbase/forced_error.go @@ -133,7 +133,8 @@ func CheckForcedErr( if replicaState.Lease.Sequence == requestedLease.Sequence { // It is only possible for this to fail when expiration-based // lease extensions are proposed concurrently. - leaseMismatch = !replicaState.Lease.Equivalent(requestedLease) + expToEpochEquiv := raftCmd.ReplicatedEvalResult.IsLeaseRequestWithExpirationToEpochEquivalent + leaseMismatch = !replicaState.Lease.Equivalent(requestedLease, expToEpochEquiv) } // This is a check to see if the lease we proposed this lease request diff --git a/pkg/kv/kvserver/kvserverpb/proposer_kv.proto b/pkg/kv/kvserver/kvserverpb/proposer_kv.proto index 5cd70a66dfc2..2ea294eb8e21 100644 --- a/pkg/kv/kvserver/kvserverpb/proposer_kv.proto +++ b/pkg/kv/kvserver/kvserverpb/proposer_kv.proto @@ -123,6 +123,28 @@ message ReplicatedEvalResult { Merge merge = 4; ComputeChecksum compute_checksum = 21; bool is_lease_request = 6; + // Set to true for lease requests in v24.1 clusters, where new behavior in + // Lease.Equivalent is supported which allows for expiration-based leases to + // be promoted to epoch-based leases under the same lease sequence. + // + // The field is used to avoid divergence between replicas of a range that + // apply lease requests while running different versions of the binary. + // + // Deprecation notes: + // - in v24.1, we introduced this field and set it to true for lease requests + // when the cluster version is detected to be v24.1 or higher. This ensures + // that all replicas who may apply the lease request will correctly handle + // the field. + // - in v24.2, we can set this field to true unconditionally for lease request + // proposals. It must still be consulted during lease application, because + // the raft proposal may have been performed by an older node. + // - in v24.2 or later, we run a Barrier migration to flush out old raft + // entries from all replica's raft logs. This ensures that no replica will + // ever apply a lease request with this field set to false. + // - in v25.1, we stop consulting this field below raft but keep setting it + // to true above raft for mixed-version compatibility with v24.2 nodes. + // - in v25.2, we delete the field. + bool is_lease_request_with_expiration_to_epoch_equivalent = 26; bool is_probe = 23; // The timestamp at which this command is writing. Used to verify the validity // of the command against the GC threshold and to update the followers' diff --git a/pkg/kv/kvserver/replica_application_result.go b/pkg/kv/kvserver/replica_application_result.go index 0ed13dcd9477..e378ecbfa1bd 100644 --- a/pkg/kv/kvserver/replica_application_result.go +++ b/pkg/kv/kvserver/replica_application_result.go @@ -49,6 +49,7 @@ func clearTrivialReplicatedEvalResultFields(r *kvserverpb.ReplicatedEvalResult) // they don't trigger an assertion at the end of the application process // (which checks that all fields were handled). r.IsLeaseRequest = false + r.IsLeaseRequestWithExpirationToEpochEquivalent = false r.WriteTimestamp = hlc.Timestamp{} r.PrevLeaseProposal = nil // The state fields cleared here were already applied to the in-memory view of diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index 0b5a8974fe3b..fdc7d8a35249 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -16,6 +16,7 @@ import ( "path/filepath" "time" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" @@ -348,7 +349,13 @@ func (r *Replica) leasePostApplyLocked( // the same lease. This can happen when callers are using // leasePostApply for some of its side effects, like with // splitPostApply. It can also happen during lease extensions. - if !prevLease.Equivalent(*newLease) { + // + // NOTE: we pass true for expToEpochEquiv because we may be in a cluster + // where some node has detected the version upgrade and is considering + // this lease type promotion to be valid, even if our local node has not + // yet detected the upgrade. Passing true broadens the definition of + // equivalence and weakens the assertion. + if !prevLease.Equivalent(*newLease, true /* expToEpochEquiv */) { log.Fatalf(ctx, "sequence identical for different leases, prevLease=%s, newLease=%s", redact.Safe(prevLease), redact.Safe(newLease)) } @@ -977,6 +984,15 @@ func (r *Replica) evaluateProposal( // Set the proposal's replicated result, which contains metadata and // side-effects that are to be replicated to all replicas. res.Replicated.IsLeaseRequest = ba.IsSingleRequestLeaseRequest() + if res.Replicated.IsLeaseRequest { + // NOTE: this cluster version check may return true even after the + // corresponding check in evalNewLease has returned false. That's ok, as + // this check is used to inform raft about whether an expiration-based + // lease **can** be promoted to an epoch-based lease without a sequence + // change, not that it **is** being promoted without a sequence change. + isV24_1 := r.ClusterSettings().Version.IsActive(ctx, clusterversion.V24_1Start) + res.Replicated.IsLeaseRequestWithExpirationToEpochEquivalent = isV24_1 + } if ba.AppliesTimestampCache() { res.Replicated.WriteTimestamp = ba.WriteTimestamp() } diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index da10e52652ef..d3b95c67412c 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -1875,10 +1875,15 @@ func (l Lease) Speculative() bool { // based leases, the start time of the lease is sufficient to // avoid using an older lease with same epoch. // +// expToEpochEquiv indicates whether an expiration-based lease +// can be considered equivalent to an epoch-based lease during +// a promotion from expiration-based to epoch-based. It is used +// for mixed-version compatibility. +// // NB: Lease.Equivalent is NOT symmetric. For expiration-based // leases, a lease is equivalent to another with an equal or // later expiration, but not an earlier expiration. -func (l Lease) Equivalent(newL Lease) bool { +func (l Lease) Equivalent(newL Lease, expToEpochEquiv bool) bool { // Ignore proposed timestamp & deprecated start stasis. l.ProposedTS, newL.ProposedTS = nil, nil l.DeprecatedStartStasis, newL.DeprecatedStartStasis = nil, nil @@ -1907,15 +1912,35 @@ func (l Lease) Equivalent(newL Lease) bool { l.Epoch, newL.Epoch = 0, 0 } case LeaseExpiration: - // See the comment above, though this field's nullability wasn't - // changed. We nil it out for completeness only. - l.Epoch, newL.Epoch = 0, 0 - - // For expiration-based leases, extensions are considered equivalent. - // This is the one case where Equivalent is not commutative and, as - // such, requires special handling beneath Raft (see checkForcedErr). - if l.GetExpiration().LessEq(newL.GetExpiration()) { - l.Expiration, newL.Expiration = nil, nil + switch newL.Type() { + case LeaseEpoch: + // An expiration-based lease being promoted to an epoch-based lease. This + // transition occurs after a successful lease transfer if the setting + // kv.transfer_expiration_leases_first.enabled is enabled. + // + // Expiration-based leases carry a local expiration timestamp. Epoch-based + // leases store their expiration indirectly in NodeLiveness. We assume that + // this promotion is only proposed if the liveness expiration is later than + // previous expiration carried by the expiration-based lease. + // + // Ignore epoch and expiration. The remaining fields which are compared + // are Replica and Start. + if expToEpochEquiv { + l.Epoch, newL.Epoch = 0, 0 + l.Expiration, newL.Expiration = nil, nil + } + + case LeaseExpiration: + // See the comment above, though this field's nullability wasn't + // changed. We nil it out for completeness only. + l.Epoch, newL.Epoch = 0, 0 + + // For expiration-based leases, extensions are considered equivalent. + // This is one case where Equivalent is not commutative and, as such, + // requires special handling beneath Raft (see checkForcedErr). + if l.GetExpiration().LessEq(newL.GetExpiration()) { + l.Expiration, newL.Expiration = nil, nil + } } } return l == newL diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index 188e08883bf6..c6ad82e26898 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -1070,8 +1070,12 @@ func TestLeaseEquivalence(t *testing.T) { {expire1, expire2R2TS2, false}, // different expiration leases {expire1, expire2, true}, // same expiration lease, extended {expire2, expire1, false}, // same expiration lease, extended but backwards - {epoch1, expire1, false}, // epoch and expiration leases - {expire1, epoch1, false}, // expiration and epoch leases + {epoch1, expire1, false}, // epoch and expiration leases, same replica and start time + {epoch1, expire1R2, false}, // epoch and expiration leases, different replica + {epoch1, expire1TS2, false}, // epoch and expiration leases, different start time + {expire1, epoch1, true}, // expiration and epoch leases, same replica and start time + {expire1, epoch1R2, false}, // expiration and epoch leases, different replica + {expire1, epoch1TS2, false}, // expiration and epoch leases, different start time {proposed1, proposed1, true}, // exact leases with identical timestamps {proposed1, proposed2, false}, // same proposed timestamps, but diff epochs {proposed1, proposed3, true}, // different proposed timestamps, same lease @@ -1082,8 +1086,13 @@ func TestLeaseEquivalence(t *testing.T) { } for i, tc := range testCases { - if ok := tc.l.Equivalent(tc.ol); tc.expSuccess != ok { - t.Errorf("%d: expected success? %t; got %t", i, tc.expSuccess, ok) + // Test expToEpochEquiv = true. + require.Equal(t, tc.expSuccess, tc.l.Equivalent(tc.ol, true /* expToEpochEquiv */), "%d", i) + if tc.l == expire1 && tc.ol == epoch1 { + // The one case where expToEpochEquiv = false makes a difference. + require.Equal(t, !tc.expSuccess, tc.l.Equivalent(tc.ol, false /* expToEpochEquiv */), "%d", i) + } else { + require.Equal(t, tc.expSuccess, tc.l.Equivalent(tc.ol, false /* expToEpochEquiv */), "%d", i) } } @@ -1105,7 +1114,7 @@ func TestLeaseEquivalence(t *testing.T) { postPRLease.DeprecatedStartStasis = nil postPRLease.Expiration = nil - if !postPRLease.Equivalent(prePRLease) || !prePRLease.Equivalent(postPRLease) { + if !postPRLease.Equivalent(prePRLease, true) || !prePRLease.Equivalent(postPRLease, true) { t.Fatalf("leases not equivalent but should be despite diff(pre,post) = %s", pretty.Diff(prePRLease, postPRLease)) } } diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index cbb5ea06e0cc..ca89de64fc27 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -1140,18 +1140,21 @@ func (tc *TestCluster) MaybeWaitForLeaseUpgrade( // is upgraded to an epoch-based one. func (tc *TestCluster) WaitForLeaseUpgrade( ctx context.Context, t serverutils.TestFataler, desc roachpb.RangeDescriptor, -) { +) roachpb.Lease { require.False(t, kvserver.ExpirationLeasesOnly.Get(&tc.Server(0).ClusterSettings().SV), "cluster configured to only use expiration leases") + var l roachpb.Lease testutils.SucceedsSoon(t, func() error { li, _, err := tc.FindRangeLeaseEx(ctx, desc, nil) require.NoError(t, err) - if li.Current().Type() != roachpb.LeaseEpoch { + l = li.Current() + if l.Type() != roachpb.LeaseEpoch { return errors.Errorf("lease still an expiration based lease") } - require.Equal(t, int64(1), li.Current().Epoch) + require.Equal(t, int64(1), l.Epoch) return nil }) + return l } // RemoveLeaseHolderOrFatal is a convenience version of TransferRangeLease and RemoveVoter