Skip to content

Commit

Permalink
kvserver: fix and re-enable slow lease application logging
Browse files Browse the repository at this point in the history
Epic: none
Release note: None
  • Loading branch information
erikgrinaker committed Jun 25, 2023
1 parent bcbc49f commit 0de16c1
Showing 1 changed file with 33 additions and 35 deletions.
68 changes: 33 additions & 35 deletions pkg/kv/kvserver/replica_proposal.go
Original file line number Diff line number Diff line change
Expand Up @@ -357,29 +357,16 @@ func (r *Replica) leasePostApplyLocked(

now := r.store.Clock().NowAsClockTimestamp()

// NB: ProposedTS is non-nil in practice, but we never fully migrated it
// in so we need to assume that it can be nil.
const slowLeaseWarningEnabled = false // see https://github.com/cockroachdb/cockroach/issues/97209
if slowLeaseWarningEnabled && iAmTheLeaseHolder && leaseChangingHands && newLease.ProposedTS != nil {
maybeLogSlowLeaseApplyWarning(ctx, time.Duration(now.WallTime-newLease.ProposedTS.WallTime), prevLease, newLease)
}

// Gossip the first range whenever its lease is acquired. We check to make
// sure the lease is active so that a trailing replica won't process an old
// lease request and attempt to gossip the first range.
if leaseChangingHands && iAmTheLeaseHolder && r.IsFirstRange() && r.ownsValidLeaseRLocked(ctx, now) {
r.gossipFirstRangeLocked(ctx)
}

// 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 leaseChangingHands && iAmTheLeaseHolder &&
r.descRLocked().StartKey.Less(roachpb.RKey(keys.NodeLivenessKeyMax)) {
if r.ownsValidLeaseRLocked(ctx, now) {
log.Health.Infof(ctx, "acquired system range lease: %s", newLease)
} else {
log.Health.Warningf(ctx, "applied system range lease after it expired: %s", newLease)
}
// Log the lease acquisition, if appropriate.
if leaseChangingHands && iAmTheLeaseHolder {
r.maybeLogLeaseAcquisition(ctx, now, prevLease, newLease)
}

isExpirationLease := newLease.Type() == roachpb.LeaseExpiration
Expand Down Expand Up @@ -473,14 +460,26 @@ func (r *Replica) leasePostApplyLocked(
}
}

// maybeLogSlowLeaseApplyWarning is called when the lease changes hands on the
// new leaseholder. It logs if either the new lease was proposed well before it
// became visible on the leaseholder (indicating replication lag) or if the
// previous lease looks like we transferred a lease to a behind/offline replica.
func maybeLogSlowLeaseApplyWarning(
ctx context.Context, newLeaseAppDelay time.Duration, prevLease, newLease *roachpb.Lease,
// maybeLogLeaseAcquisition is called on the new leaseholder when the lease
// changes hands, to log the lease acquisition if appropriate.
func (r *Replica) maybeLogLeaseAcquisition(
ctx context.Context, now hlc.ClockTimestamp, prevLease, newLease *roachpb.Lease,
) {
const slowLeaseApplyWarnThreshold = 500 * time.Millisecond
// 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", newLease)
} else {
log.Health.Warningf(ctx, "applied system range lease after it expired: %s", newLease)
}
}

const slowLeaseApplyWarnThreshold = time.Second
var newLeaseAppDelay time.Duration
if newLease.ProposedTS != nil { // non-nil in practice, but never migrated
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
Expand All @@ -496,23 +495,22 @@ func maybeLogSlowLeaseApplyWarning(
// case, which is good enough here.
//
// [^1]: https://github.com/cockroachdb/cockroach/pull/82758
log.Warningf(ctx,
"lease %v active after replication lag of ~%.2fs; foreground traffic may have been impacted [prev=%v]",
newLease, newLeaseAppDelay.Seconds(), prevLease,
)
log.Health.Warningf(ctx,
"applied lease after ~%.2fs replication lag, client traffic may have been delayed [lease=%v prev=%v]",
newLeaseAppDelay.Seconds(), newLease, prevLease)
} else if prevLease.Type() == roachpb.LeaseExpiration &&
newLease.Type() == roachpb.LeaseEpoch &&
newLease.AcquisitionType == roachpb.LeaseAcquisitionType_Request {
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
// the acquisition was non-cooperative, it is likely that a lease transfer
// (which is expiration-based) went to a follower that then couldn't hold
// the lease alive (for example, didn't apply it in time for it to
// 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.Warningf(ctx,
"lease %v expired before being followed by lease %s; foreground traffic may have been impacted",
prevLease, newLease,
)
log.Health.Warningf(ctx,
"lease expired before epoch lease upgrade, client traffic may have been delayed [lease=%v prev=%v]",
newLease, prevLease)
}
}

Expand Down

0 comments on commit 0de16c1

Please sign in to comment.