Skip to content

Commit

Permalink
Merge #104008
Browse files Browse the repository at this point in the history
104008: kvserver: improve system lease observability r=erikgrinaker a=erikgrinaker

**kvserver: add `leases.liveness` metric**

This patch adds the metric `leases.liveness` tracking the number of liveness range leases per node (generally 1 or 0). This is useful to find out which node had the liveness lease at a particular time.

I ran a 10k range cluster to look at the CPU cost of the key comparisons, it didn't show up on CPU profiles.

Epic: none
Release note (ops change): added the metric `leases.liveness` showing the number of liveness range leases per node (generally 1 or 0), to track the liveness range leaseholder.
  
**kvserver: log system range lease acquisition**

This patch logs acquisition of meta/liveness range leases to the health log. These leases are critical to cluster health, and during debugging it's useful to know their location over time.

Resolves #99472.

Epic: none
Release note: none

Co-authored-by: Erik Grinaker <[email protected]>
  • Loading branch information
craig[bot] and erikgrinaker committed May 30, 2023
2 parents 2a7427c + b9148c7 commit 5ee25de
Show file tree
Hide file tree
Showing 4 changed files with 35 additions and 6 deletions.
8 changes: 8 additions & 0 deletions pkg/kv/kvserver/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -164,6 +164,12 @@ var (
Measurement: "Replicas",
Unit: metric.Unit_COUNT,
}
metaLeaseLivenessCount = metric.Metadata{
Name: "leases.liveness",
Help: "Number of replica leaseholders for the liveness range(s)",
Measurement: "Replicas",
Unit: metric.Unit_COUNT,
}

// Storage metrics.
metaLiveBytes = metric.Metadata{
Expand Down Expand Up @@ -2022,6 +2028,7 @@ type StoreMetrics struct {
LeaseTransferErrorCount *metric.Counter
LeaseExpirationCount *metric.Gauge
LeaseEpochCount *metric.Gauge
LeaseLivenessCount *metric.Gauge

// Storage metrics.
ResolveCommitCount *metric.Counter
Expand Down Expand Up @@ -2653,6 +2660,7 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics {
LeaseTransferErrorCount: metric.NewCounter(metaLeaseTransferErrorCount),
LeaseExpirationCount: metric.NewGauge(metaLeaseExpirationCount),
LeaseEpochCount: metric.NewGauge(metaLeaseEpochCount),
LeaseLivenessCount: metric.NewGauge(metaLeaseLivenessCount),

// Intent resolution metrics.
ResolveCommitCount: metric.NewCounter(metaResolveCommit),
Expand Down
17 changes: 11 additions & 6 deletions pkg/kv/kvserver/replica_metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"math"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/allocatorimpl"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
Expand All @@ -27,11 +28,12 @@ import (

// ReplicaMetrics contains details on the current status of the replica.
type ReplicaMetrics struct {
Leader bool
LeaseValid bool
Leaseholder bool
LeaseType roachpb.LeaseType
LeaseStatus kvserverpb.LeaseStatus
Leader bool
LeaseValid bool
Leaseholder bool
LeaseType roachpb.LeaseType
LeaseStatus kvserverpb.LeaseStatus
LivenessLease bool

// Quiescent indicates whether the replica believes itself to be quiesced.
Quiescent bool
Expand Down Expand Up @@ -125,12 +127,14 @@ type calcReplicaMetricsInput struct {
}

func calcReplicaMetrics(d calcReplicaMetricsInput) ReplicaMetrics {
var validLease, validLeaseOwner bool
var validLease, validLeaseOwner, livenessLease bool
var validLeaseType roachpb.LeaseType
if d.leaseStatus.IsValid() {
validLease = true
validLeaseOwner = d.leaseStatus.Lease.OwnedBy(d.storeID)
validLeaseType = d.leaseStatus.Lease.Type()
livenessLease = validLeaseOwner &&
keys.NodeLivenessSpan.Overlaps(d.desc.RSpan().AsRawSpanWithNoLocals())
}

rangeCounter, unavailable, underreplicated, overreplicated := calcRangeCounter(
Expand All @@ -152,6 +156,7 @@ func calcReplicaMetrics(d calcReplicaMetricsInput) ReplicaMetrics {
Leaseholder: validLeaseOwner,
LeaseType: validLeaseType,
LeaseStatus: d.leaseStatus,
LivenessLease: livenessLease,
Quiescent: d.quiescent,
Ticking: d.ticking,
RangeCounter: rangeCounter,
Expand Down
11 changes: 11 additions & 0 deletions pkg/kv/kvserver/replica_proposal.go
Original file line number Diff line number Diff line change
Expand Up @@ -366,6 +366,17 @@ func (r *Replica) leasePostApplyLocked(
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)
}
}

st := r.leaseStatusAtRLocked(ctx, now)
if leaseChangingHands && newLease.Type() == roachpb.LeaseExpiration &&
r.ownsValidLeaseRLocked(ctx, now) && !r.shouldUseExpirationLeaseRLocked() {
Expand Down
5 changes: 5 additions & 0 deletions pkg/kv/kvserver/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -2892,6 +2892,7 @@ func (s *Store) updateReplicationGauges(ctx context.Context) error {
leaseHolderCount int64
leaseExpirationCount int64
leaseEpochCount int64
leaseLivenessCount int64
raftLeaderNotLeaseHolderCount int64
raftLeaderInvalidLeaseCount int64
quiescentCount int64
Expand Down Expand Up @@ -2963,6 +2964,9 @@ func (s *Store) updateReplicationGauges(ctx context.Context) error {
case roachpb.LeaseEpoch:
leaseEpochCount++
}
if metrics.LivenessLease {
leaseLivenessCount++
}
}
if metrics.Quiescent {
quiescentCount++
Expand Down Expand Up @@ -3021,6 +3025,7 @@ func (s *Store) updateReplicationGauges(ctx context.Context) error {
s.metrics.LeaseHolderCount.Update(leaseHolderCount)
s.metrics.LeaseExpirationCount.Update(leaseExpirationCount)
s.metrics.LeaseEpochCount.Update(leaseEpochCount)
s.metrics.LeaseLivenessCount.Update(leaseLivenessCount)
s.metrics.QuiescentCount.Update(quiescentCount)
s.metrics.UninitializedCount.Update(uninitializedCount)
s.metrics.AverageQueriesPerSecond.Update(averageQueriesPerSecond)
Expand Down

0 comments on commit 5ee25de

Please sign in to comment.