Skip to content

Commit

Permalink
Merge #101899
Browse files Browse the repository at this point in the history
101899: Revert "kvserver: allow expired leases to quiesce" r=erikgrinaker a=erikgrinaker

This reverts commit 76afb00.

This change was flawed, because it assumed an expired lease would have to be reacquired which would wake up the range, but that's not necessarily true. In cases where the Raft leader is not colocated with the leaseholder, the Raft leader may have a stale liveness record, or the leaseholder may temporarily fail to heartbeat liveness without anyone bumping its epoch. If this happens, it's possible for an expired lease to revert to a valid lease from the leader's point of view, but at that point it's too late: the range is already quiesced.

This can prevent e.g. lease transfers, because we don't allow leaseholders to transfer their lease if they're not the Raft leader (they need to make sure the target is not behind on the Raft log). We won't attempt to colocate the Raft leader with the leaseholder for a quiesced range, since we don't tick it.

Touches #101885.
Touches #97289.

Epic: none
Release note: None

Co-authored-by: Erik Grinaker <[email protected]>
  • Loading branch information
craig[bot] and erikgrinaker committed Apr 20, 2023
2 parents 17d130b + d73a05d commit d2e36ef
Show file tree
Hide file tree
Showing 2 changed files with 23 additions and 102 deletions.
35 changes: 6 additions & 29 deletions pkg/kv/kvserver/replica_raft_quiesce.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"context"
"sort"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftlog"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand Down Expand Up @@ -200,9 +199,8 @@ type quiescer interface {
hasRaftReadyRLocked() bool
hasPendingProposalsRLocked() bool
hasPendingProposalQuotaRLocked() bool
leaseStatusAtRLocked(ctx context.Context, now hlc.ClockTimestamp) kvserverpb.LeaseStatus
StoreID() roachpb.StoreID
getLeaseRLocked() (roachpb.Lease, roachpb.Lease)
ownsValidLeaseRLocked(ctx context.Context, now hlc.ClockTimestamp) bool
mergeInProgressRLocked() bool
isDestroyedRLocked() (DestroyReason, error)
}
Expand Down Expand Up @@ -357,36 +355,15 @@ func shouldReplicaQuiesce(
return nil, nil, false
}

// Don't quiesce if there is a current leaseholder elsewhere. Otherwise, the
// leaseholder may have pending commands which it's waiting on this leader to
// propose.
//
// We allow quiescing with an expired lease, since leases are not always
// eagerly renewed (but see expiration-based lease check above). This replica
// thinks it's the leader, and it checks that there are no unapplied entries,
// so there can't be a new leaseholder if that's still the case. If someone
// else recently acquired leadership then this replica would not be able to
// quiesce those followers, only itself and any stale followers, and it would
// unquiesce once it hears from the new leader.
st := q.leaseStatusAtRLocked(ctx, now)
switch st.State {
// Allow quiescing if the current lease is ours, even if we can't use it.
case kvserverpb.LeaseState_VALID, kvserverpb.LeaseState_UNUSABLE, kvserverpb.LeaseState_PROSCRIBED:
if !st.OwnedBy(q.StoreID()) {
if log.V(4) {
log.Infof(ctx, "not quiescing: not leaseholder")
}
return nil, nil, false
}
// Allow expired leases to quiesce.
case kvserverpb.LeaseState_EXPIRED:
default:
// Only quiesce if this replica is the leaseholder as well;
// otherwise the replica which is the valid leaseholder may have
// pending commands which it's waiting on this leader to propose.
if !q.ownsValidLeaseRLocked(ctx, now) {
if log.V(4) {
log.Infof(ctx, "not quiescing: lease in state %s", st)
log.Infof(ctx, "not quiescing: not leaseholder")
}
return nil, nil, false
}

// We need all of Applied, Commit, LastIndex and Progress.Match indexes to be
// equal in order to quiesce.
if status.Applied != status.Commit {
Expand Down
90 changes: 17 additions & 73 deletions pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -9908,8 +9908,7 @@ type testQuiescer struct {
status *raftSparseStatus
lastIndex uint64
raftReady bool
leaseStatus kvserverpb.LeaseStatus
storeID roachpb.StoreID
lease roachpb.Lease
mergeInProgress bool
isDestroyed bool

Expand Down Expand Up @@ -9954,18 +9953,12 @@ func (q *testQuiescer) hasPendingProposalQuotaRLocked() bool {
return q.pendingQuota
}

func (q *testQuiescer) leaseStatusAtRLocked(
ctx context.Context, now hlc.ClockTimestamp,
) kvserverpb.LeaseStatus {
return q.leaseStatus
}

func (q *testQuiescer) StoreID() roachpb.StoreID {
return q.storeID
func (q *testQuiescer) ownsValidLeaseRLocked(ctx context.Context, now hlc.ClockTimestamp) bool {
return q.lease.Replica.ReplicaID == 1
}

func (q *testQuiescer) getLeaseRLocked() (roachpb.Lease, roachpb.Lease) {
return q.leaseStatus.Lease, q.leaseStatus.Lease
return q.lease, q.lease
}

func (q *testQuiescer) mergeInProgressRLocked() bool {
Expand All @@ -9991,8 +9984,7 @@ func TestShouldReplicaQuiesce(t *testing.T) {
// true. The transform function is intended to perform one mutation to
// this quiescer so that shouldReplicaQuiesce will return false.
q := &testQuiescer{
st: cluster.MakeTestingClusterSettings(),
storeID: 1,
st: cluster.MakeTestingClusterSettings(),
desc: roachpb.RangeDescriptor{
InternalReplicas: []roachpb.ReplicaDescriptor{
{NodeID: 1, ReplicaID: 1},
Expand Down Expand Up @@ -10020,16 +10012,13 @@ func TestShouldReplicaQuiesce(t *testing.T) {
},
lastIndex: logIndex,
raftReady: false,
leaseStatus: kvserverpb.LeaseStatus{
State: kvserverpb.LeaseState_VALID,
Lease: roachpb.Lease{
Sequence: 1,
Epoch: 1,
Replica: roachpb.ReplicaDescriptor{
NodeID: 1,
StoreID: 1,
ReplicaID: 1,
},
lease: roachpb.Lease{
Sequence: 1,
Epoch: 1,
Replica: roachpb.ReplicaDescriptor{
NodeID: 1,
StoreID: 1,
ReplicaID: 1,
},
},
livenessMap: livenesspb.IsLiveMap{
Expand Down Expand Up @@ -10113,52 +10102,7 @@ func TestShouldReplicaQuiesce(t *testing.T) {
return q
})
test(false, func(q *testQuiescer) *testQuiescer {
q.leaseStatus.State = kvserverpb.LeaseState_ERROR
return q
})
test(true, func(q *testQuiescer) *testQuiescer {
q.leaseStatus.State = kvserverpb.LeaseState_VALID
return q
})
test(true, func(q *testQuiescer) *testQuiescer {
q.leaseStatus.State = kvserverpb.LeaseState_UNUSABLE
return q
})
test(true, func(q *testQuiescer) *testQuiescer {
q.leaseStatus.State = kvserverpb.LeaseState_EXPIRED
return q
})
test(true, func(q *testQuiescer) *testQuiescer {
q.leaseStatus.State = kvserverpb.LeaseState_PROSCRIBED
return q
})
test(false, func(q *testQuiescer) *testQuiescer {
q.leaseStatus.State = -99
return q
})
test(false, func(q *testQuiescer) *testQuiescer {
q.leaseStatus.Lease.Replica.StoreID = 9
q.leaseStatus.State = kvserverpb.LeaseState_ERROR
return q
})
test(false, func(q *testQuiescer) *testQuiescer {
q.leaseStatus.Lease.Replica.StoreID = 9
q.leaseStatus.State = kvserverpb.LeaseState_VALID
return q
})
test(false, func(q *testQuiescer) *testQuiescer {
q.leaseStatus.Lease.Replica.StoreID = 9
q.leaseStatus.State = kvserverpb.LeaseState_UNUSABLE
return q
})
test(true, func(q *testQuiescer) *testQuiescer {
q.leaseStatus.Lease.Replica.StoreID = 9
q.leaseStatus.State = kvserverpb.LeaseState_EXPIRED
return q
})
test(false, func(q *testQuiescer) *testQuiescer {
q.leaseStatus.Lease.Replica.StoreID = 9
q.leaseStatus.State = kvserverpb.LeaseState_PROSCRIBED
q.lease.Replica.ReplicaID = 9
return q
})
test(false, func(q *testQuiescer) *testQuiescer {
Expand Down Expand Up @@ -10220,16 +10164,16 @@ func TestShouldReplicaQuiesce(t *testing.T) {
// kv.expiration_leases_only.enabled is true.
test(false, func(q *testQuiescer) *testQuiescer {
ExpirationLeasesOnly.Override(context.Background(), &q.st.SV, true)
q.leaseStatus.Lease.Epoch = 0
q.leaseStatus.Lease.Expiration = &hlc.Timestamp{
q.lease.Epoch = 0
q.lease.Expiration = &hlc.Timestamp{
WallTime: timeutil.Now().Add(time.Minute).Unix(),
}
return q
})
test(true, func(q *testQuiescer) *testQuiescer {
ExpirationLeasesOnly.Override(context.Background(), &q.st.SV, false)
q.leaseStatus.Lease.Epoch = 0
q.leaseStatus.Lease.Expiration = &hlc.Timestamp{
q.lease.Epoch = 0
q.lease.Expiration = &hlc.Timestamp{
WallTime: timeutil.Now().Add(time.Minute).Unix(),
}
return q
Expand Down

0 comments on commit d2e36ef

Please sign in to comment.