Skip to content

Commit

Permalink
kvserver: ignore lease validity when checking lease preferences
Browse files Browse the repository at this point in the history
In cockroachdb#107507, we began eagerly enqueuing into the replicate queue, when
acquiring a replica acquired a new lease which violated lease
preferences. Lease preferences were only considered violated when the
lease itself was valid. In cockroachdb#107507, we saw that it is uncommon, but
possible for an invalid lease to be acquired, violate lease preferences
and not be enqueued as a result. The end result was a leaseholder
violating the applied lease preferences which would not be resolved
until the next scanner cycle.

Update the eager enqueue check to only check that the replica is the
incoming leaseholder when applying the lease, and that the replica
violates the applied lease preferences. The check now applies on any
lease acquisition, where previously it only occurred on the leaseholder
changing.

Note the purgatory error introduced in cockroachdb#107507, still checks that the
lease is valid and owned by the store before proceeding. It is a
condition that the lease must be valid+owned by the store to have a
change planned, so whilst it is possible the lease becomes invalid
somewhere in-between planning, when the replica applies a valid lease,
it will still be enqueued, so purgatory is unnecessary.

Fixes: cockroachdb#107862
Release note: None
  • Loading branch information
kvoli committed Aug 1, 2023
1 parent e9add29 commit 1bb0d44
Show file tree
Hide file tree
Showing 4 changed files with 44 additions and 41 deletions.
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/allocator/plan/replicate.go
Original file line number Diff line number Diff line change
Expand Up @@ -963,7 +963,8 @@ func (rp ReplicaPlanner) shedLeaseTarget(
liveVoters, _ := rp.storePool.LiveAndDeadReplicas(
existingVoters, false /* includeSuspectAndDrainingStores */)
preferred := rp.allocator.PreferredLeaseholders(rp.storePool, conf, liveVoters)
if len(preferred) > 0 && repl.LeaseViolatesPreferences(ctx) {
if len(preferred) > 0 &&
repl.LeaseViolatesPreferences(ctx) {
return nil, CantTransferLeaseViolatingPreferencesError{RangeID: desc.RangeID}
}
return nil, nil
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/replica_metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -149,8 +149,8 @@ func calcReplicaMetrics(d calcReplicaMetricsInput) ReplicaMetrics {
validLeaseType = d.leaseStatus.Lease.Type()
if validLeaseOwner {
livenessLease = keys.NodeLivenessSpan.Overlaps(d.desc.RSpan().AsRawSpanWithNoLocals())
switch makeLeasePreferenceStatus(
d.leaseStatus, d.storeID, d.storeAttrs, d.nodeAttrs,
switch checkStoreAgainstLeasePreferences(
d.storeID, d.storeAttrs, d.nodeAttrs,
d.nodeLocality, d.conf.LeasePreferences) {
case leasePreferencesViolating:
violatingLeasePreferences = true
Expand Down
40 changes: 21 additions & 19 deletions pkg/kv/kvserver/replica_proposal.go
Original file line number Diff line number Diff line change
Expand Up @@ -517,25 +517,27 @@ func (r *Replica) leasePostApplyLocked(
})
}

// If we acquired a new lease, and it violates the lease preferences, enqueue
// it in the replicate queue.
if leaseChangingHands && iAmTheLeaseHolder {
if LeaseCheckPreferencesOnAcquisitionEnabled.Get(&r.store.cfg.Settings.SV) {
preferenceStatus := makeLeasePreferenceStatus(st, r.store.StoreID(), r.store.Attrs(),
r.store.nodeDesc.Attrs, r.store.nodeDesc.Locality, r.mu.conf.LeasePreferences)
switch preferenceStatus {
case leasePreferencesOK, leasePreferencesLessPreferred, leasePreferencesUnknown:
// We could also enqueue the lease when we are a less preferred
// leaseholder, however the replicate queue will eventually get to it and
// we already satisfy _some_ preference.
case leasePreferencesViolating:
log.VEventf(ctx, 2,
"acquired lease violates lease preferences, enqueueing for transfer [lease=%v preferences=%v]",
newLease, r.mu.conf.LeasePreferences)
r.store.replicateQueue.AddAsync(ctx, r, replicateQueueLeasePreferencePriority)
default:
log.Fatalf(ctx, "unknown lease preferences status: %v", preferenceStatus)
}
// If we acquired a lease, and it violates the lease preferences, enqueue it
// in the replicate queue. NOTE: We don't check whether the lease is valid,
// it is possible that the lease being applied is invalid due to replication
// lag, or previously needing a snapshot. The replicate queue will ensure the
// lease is valid and owned by the replica before processing.
if iAmTheLeaseHolder &&
LeaseCheckPreferencesOnAcquisitionEnabled.Get(&r.store.cfg.Settings.SV) {
preferenceStatus := checkStoreAgainstLeasePreferences(r.store.StoreID(), r.store.Attrs(),
r.store.nodeDesc.Attrs, r.store.nodeDesc.Locality, r.mu.conf.LeasePreferences)
switch preferenceStatus {
case leasePreferencesOK, leasePreferencesLessPreferred:
// We could also enqueue the lease when we are a less preferred
// leaseholder, however the replicate queue will eventually get to it and
// we already satisfy _some_ preference.
case leasePreferencesViolating:
log.VEventf(ctx, 2,
"acquired lease violates lease preferences, enqueuing for transfer [lease=%v preferences=%v]",
newLease, r.mu.conf.LeasePreferences)
r.store.replicateQueue.AddAsync(ctx, r, replicateQueueLeasePreferencePriority)
default:
log.Fatalf(ctx, "unknown lease preferences status: %v", preferenceStatus)
}
}

Expand Down
38 changes: 19 additions & 19 deletions pkg/kv/kvserver/replica_range_lease.go
Original file line number Diff line number Diff line change
Expand Up @@ -1556,16 +1556,14 @@ func (r *Replica) hasCorrectLeaseTypeRLocked(lease roachpb.Lease) bool {
type leasePreferencesStatus int

const (
// leasePreferencesUnknown indicates the preferences status cannot be
// determined.
leasePreferencesUnknown leasePreferencesStatus = iota
// leasePreferencesViolating indicates the leaseholder does not
// satisfy any lease preference applied.
_ leasePreferencesStatus = iota
// leasePreferencesViolating indicates the checked store does not satisfy any
// lease preference applied.
leasePreferencesViolating
// leasePreferencesLessPreferred indicates the leaseholder satisfies _some_
// leasePreferencesLessPreferred indicates the checked store satisfies _some_
// preference, however not the most preferred.
leasePreferencesLessPreferred
// leasePreferencesOK indicates the lease satisfies the first
// leasePreferencesOK indicates the checked store satisfies the first
// preference, or no lease preferences are applied.
leasePreferencesOK
)
Expand All @@ -1577,32 +1575,34 @@ func (r *Replica) LeaseViolatesPreferences(ctx context.Context) bool {
storeID := r.store.StoreID()
now := r.Clock().NowAsClockTimestamp()
r.mu.RLock()
leaseStatus := r.leaseStatusAtRLocked(ctx, now)
preferences := r.mu.conf.LeasePreferences
leaseStatus := r.leaseStatusAtRLocked(ctx, now)
r.mu.RUnlock()

if !leaseStatus.IsValid() || !leaseStatus.Lease.OwnedBy(storeID) {
// We can't determine if the lease preferences are being conformed to or
// not, as the store either doesn't own the lease, or doesn't own a valid
// lease.
return true
}

storeAttrs := r.store.Attrs()
nodeAttrs := r.store.nodeDesc.Attrs
nodeLocality := r.store.nodeDesc.Locality
preferenceStatus := makeLeasePreferenceStatus(
leaseStatus, storeID, storeAttrs, nodeAttrs, nodeLocality, preferences)

preferenceStatus := checkStoreAgainstLeasePreferences(
storeID, storeAttrs, nodeAttrs, nodeLocality, preferences)
return preferenceStatus == leasePreferencesViolating
}

func makeLeasePreferenceStatus(
leaseStatus kvserverpb.LeaseStatus,
// checkStoreAgainstLeasePreferences returns whether the given store would
// violate, be less preferred or ok, leaseholder, according the the lease
// preferences.
func checkStoreAgainstLeasePreferences(
storeID roachpb.StoreID,
storeAttrs, nodeAttrs roachpb.Attributes,
nodeLocality roachpb.Locality,
preferences []roachpb.LeasePreference,
) leasePreferencesStatus {
if !leaseStatus.IsValid() || !leaseStatus.Lease.OwnedBy(storeID) {
// We can't determine if the lease preferences are being conformed to or
// not, as the store either doesn't own the lease, or doesn't own a valid
// lease.
return leasePreferencesUnknown
}
if len(preferences) == 0 {
return leasePreferencesOK
}
Expand Down

0 comments on commit 1bb0d44

Please sign in to comment.