Skip to content

Commit

Permalink
Merge pull request #135204 from cockroachdb/blathers/backport-release…
Browse files Browse the repository at this point in the history
…-24.3.0-rc-135115

release-24.3.0-rc: kvserver: log most lease applications
  • Loading branch information
arulajmani authored Nov 14, 2024
2 parents 12a2b4a + a17c07f commit c22e145
Showing 1 changed file with 89 additions and 62 deletions.
151 changes: 89 additions & 62 deletions pkg/kv/kvserver/replica_proposal.go
Original file line number Diff line number Diff line change
Expand Up @@ -312,6 +312,8 @@ const (
allowLeaseJump = true
)

var leaseAcquisitionLoggerEvery = log.Every(1 * time.Second)

// leasePostApplyLocked updates the Replica's internal state to reflect the
// application of a new Range lease. The method is idempotent, so it can be
// called repeatedly for the same lease safely. However, the method will panic
Expand Down Expand Up @@ -386,17 +388,6 @@ func (r *Replica) leasePostApplyLocked(
// timestamp cache.
leaseChangingHands := prevLease.Replica.StoreID != newLease.Replica.StoreID || prevLease.Sequence != newLease.Sequence

if iAmTheLeaseHolder {
// Log lease acquisitions loudly when verbose logging is enabled or when the
// new leaseholder is draining, in which case it should be shedding leases.
// Otherwise, log a trace event.
if log.V(1) || (leaseChangingHands && r.store.IsDraining()) {
log.Infof(ctx, "new range lease %s following %s", newLease, prevLease)
} else {
log.Eventf(ctx, "new range lease %s following %s", newLease, prevLease)
}
}

if leaseChangingHands && iAmTheLeaseHolder {
// When taking over the lease, we need to check whether a merge is in
// progress, as only the old leaseholder would have been explicitly notified
Expand Down Expand Up @@ -483,9 +474,9 @@ func (r *Replica) leasePostApplyLocked(
r.gossipFirstRangeLocked(ctx)
}

// Log the lease acquisition, if appropriate.
if leaseChangingHands && iAmTheLeaseHolder {
r.maybeLogLeaseAcquisition(ctx, now, prevLease, newLease)
// Log the lease, if appropriate.
if iAmTheLeaseHolder {
r.maybeLogLease(ctx, now, prevLease, newLease)
}

st := r.leaseStatusAtRLocked(ctx, now)
Expand Down Expand Up @@ -598,58 +589,94 @@ func (r *Replica) leasePostApplyLocked(
}
}

// maybeLogLeaseAcquisition is called on the new leaseholder when the lease
// changes hands, to log the lease acquisition if appropriate.
func (r *Replica) maybeLogLeaseAcquisition(
// maybeLogLease is called on the new leaseholder to log the lease
// if appropriate.
func (r *Replica) maybeLogLease(
ctx context.Context, now hlc.ClockTimestamp, prevLease, newLease *roachpb.Lease,
) {
// Log acquisition of meta and liveness range leases. These are critical to
// cluster health, so it's useful to know their location over time.
if r.descRLocked().StartKey.Less(roachpb.RKey(keys.NodeLivenessKeyMax)) {
if r.ownsValidLeaseRLocked(ctx, now) {
log.Health.Infof(ctx, "acquired system range lease: %s [acquisition-type=%s]",
newLease, newLease.AcquisitionType)
} else {
log.Health.Warningf(ctx, "applied system range lease after it expired: %s [acquisition-type=%s]",
newLease, newLease.AcquisitionType)
}
leaseChangingHands := prevLease.Replica.StoreID != newLease.Replica.StoreID ||
prevLease.Sequence != newLease.Sequence

// TODO(arul): consider pulling out all these leasing related logging into
// a separate log channel.

extension := newLease.Type() == roachpb.LeaseExpiration && !leaseChangingHands
promotion := prevLease.Type() == roachpb.LeaseExpiration &&
newLease.Type() != roachpb.LeaseExpiration && !leaseChangingHands
if r.store.IsDraining() && leaseChangingHands {
// If the new leaseholder is on a draining node, in which case it should be
// shedding leases, indicate this in the log line.
log.Health.Infof(ctx, "new range lease %s on draining node following %s", newLease, prevLease)
} else if log.V(1) {
// Log every lease acquisition if verbose logging is enabled.
log.Health.Infof(ctx, "new range lease %s following %s", newLease, prevLease)
} else if !extension && leaseAcquisitionLoggerEvery.ShouldLog() {
// We log lease applications once every leaseAcquisitionLoggerEvery
// duration. to prevent logs from getting too spammy. Moreover, to make
// these logs useful, we don't log extensions for expiration based leases,
// as those are fairly frequent.
log.Health.Infof(ctx, "new range lease %s following %s", newLease, prevLease)
} else if promotion {
// Lease is being promoted. It likely won't be caught by the
// leaseAcquisitionLoggerEvery above, as we attempt to promote the moment
// the lease transfer is applied. Log it here. Note that we log every
// lease transfer anyway, so this shouldn't be too much more chatty.
log.Health.Infof(ctx, "new range lease %s promoted from %s", newLease, prevLease)
} else {
// If none of the above is true, just log as a trace event.
log.Eventf(ctx, "new range lease %s following %s", newLease, prevLease)
}

const slowLeaseApplyWarnThreshold = time.Second
newLeaseAppDelay := time.Duration(now.WallTime - newLease.ProposedTS.WallTime)
if newLeaseAppDelay > slowLeaseApplyWarnThreshold {
// If we hold the lease now and the lease was proposed "earlier", there
// must have been replication lag, and possibly reads and/or writes were
// delayed.
//
// We see this most commonly with lease transfers targeting a behind replica,
// or, in the worst case, a snapshot. We are constantly improving our
// heuristics for avoiding that[^1] but if it does happen it's good to know
// from the logs.
//
// In the case of a lease transfer, the two timestamps compared below are from
// different clocks, so there could be skew. We just pretend this is not the
// case, which is good enough here.
//
// [^1]: https://github.com/cockroachdb/cockroach/pull/82758
log.Health.Warningf(ctx,
"applied lease after ~%.2fs replication lag, client traffic may have "+
"been delayed [lease=%v prev=%v acquisition-type=%s]",
newLeaseAppDelay.Seconds(), newLease, prevLease, newLease.AcquisitionType)
} else if prevLease.Type() == roachpb.LeaseExpiration &&
newLease.Type() != roachpb.LeaseExpiration &&
prevLease.Expiration != nil && // nil when there is no previous lease
prevLease.Expiration.LessEq(newLease.Start.ToTimestamp()) {
// If the previous lease is expiration-based, but the new lease is not and
// starts at or after its expiration, it is likely that a lease transfer
// (which is expiration-based) went to a follower that then couldn't upgrade
// it to an epoch lease (for example, didn't apply it in time for it to
// actually serve any traffic). The result was likely an outage which
// resolves right now, so log to point this out.
log.Health.Warningf(ctx,
"lease expired before epoch/leader lease upgrade, client traffic may "+
"have been delayed [lease=%v prev=%v acquisition-type=%s]",
newLease, prevLease, newLease.AcquisitionType)
if leaseChangingHands {
// Log acquisition of meta and liveness range leases. These are critical to
// cluster health, so it's useful to know their location over time.
if r.descRLocked().StartKey.Less(roachpb.RKey(keys.NodeLivenessKeyMax)) {
if r.ownsValidLeaseRLocked(ctx, now) {
log.Health.Infof(ctx, "acquired system range lease: %s [acquisition-type=%s]",
newLease, newLease.AcquisitionType)
} else {
log.Health.Warningf(ctx, "applied system range lease after it expired: %s [acquisition-type=%s]",
newLease, newLease.AcquisitionType)
}
}

// Log slow lease applications in the Health log.
const slowLeaseApplyWarnThreshold = time.Second
newLeaseAppDelay := time.Duration(now.WallTime - newLease.ProposedTS.WallTime)
if newLeaseAppDelay > slowLeaseApplyWarnThreshold {
// If we hold the lease now and the lease was proposed "earlier", there
// must have been replication lag, and possibly reads and/or writes were
// delayed.
//
// We see this most commonly with lease transfers targeting a behind replica,
// or, in the worst case, a snapshot. We are constantly improving our
// heuristics for avoiding that[^1] but if it does happen it's good to know
// from the logs.
//
// In the case of a lease transfer, the two timestamps compared below are from
// different clocks, so there could be skew. We just pretend this is not the
// case, which is good enough here.
//
// [^1]: https://github.com/cockroachdb/cockroach/pull/82758
log.Health.Warningf(ctx,
"applied lease after ~%.2fs replication lag, client traffic may have "+
"been delayed [lease=%v prev=%v acquisition-type=%s]",
newLeaseAppDelay.Seconds(), newLease, prevLease, newLease.AcquisitionType)
} else if prevLease.Type() == roachpb.LeaseExpiration &&
newLease.Type() != roachpb.LeaseExpiration &&
prevLease.Expiration != nil && // nil when there is no previous lease
prevLease.Expiration.LessEq(newLease.Start.ToTimestamp()) {
// If the previous lease is expiration-based, but the new lease is not and
// starts at or after its expiration, it is likely that a lease transfer
// (which is expiration-based) went to a follower that then couldn't upgrade
// it to an epoch lease (for example, didn't apply it in time for it to
// actually serve any traffic). The result was likely an outage which
// resolves right now, so log to point this out.
log.Health.Warningf(ctx,
"lease expired before epoch/leader lease upgrade, client traffic may "+
"have been delayed [lease=%v prev=%v acquisition-type=%s]",
newLease, prevLease, newLease.AcquisitionType)
}
}
}

Expand Down

0 comments on commit c22e145

Please sign in to comment.