Skip to content

Commit

Permalink
Merge #72989
Browse files Browse the repository at this point in the history
72989: kv: add telemetry for node liveness r=lidorcarmel a=lidorcarmel

Exporting the existing metrics 'HeartbeatFailures' and 'EpochIncrements'
as telementry counters.

These telemetry values can be seen in cockroach demo by decommissioning
a node and then querying crdb_internal.feature_usage, and also in the
'Diagnostics Reporting Data' page (/_status/diagnostics/local ->
"featureUsage").

Issue #71662

Release note: None

Co-authored-by: Lidor Carmel <[email protected]>
  • Loading branch information
craig[bot] and lidorcarmel committed Nov 24, 2021
2 parents 6d024a9 + 0b7ba56 commit 47ce2f0
Show file tree
Hide file tree
Showing 5 changed files with 37 additions and 6 deletions.
1 change: 1 addition & 0 deletions pkg/kv/kvserver/liveness/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ go_library(
"//pkg/kv",
"//pkg/kv/kvserver/liveness/livenesspb",
"//pkg/roachpb:with-mocks",
"//pkg/server/telemetry",
"//pkg/settings/cluster",
"//pkg/storage",
"//pkg/util/contextutil",
Expand Down
13 changes: 7 additions & 6 deletions pkg/kv/kvserver/liveness/liveness.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util/contextutil"
Expand Down Expand Up @@ -139,8 +140,8 @@ type Metrics struct {
LiveNodes *metric.Gauge
HeartbeatsInFlight *metric.Gauge
HeartbeatSuccesses *metric.Counter
HeartbeatFailures *metric.Counter
EpochIncrements *metric.Counter
HeartbeatFailures telemetry.CounterWithMetric
EpochIncrements telemetry.CounterWithMetric
HeartbeatLatency *metric.Histogram
}

Expand Down Expand Up @@ -295,8 +296,8 @@ func NewNodeLiveness(opts NodeLivenessOptions) *NodeLiveness {
LiveNodes: metric.NewFunctionalGauge(metaLiveNodes, nl.numLiveNodes),
HeartbeatsInFlight: metric.NewGauge(metaHeartbeatsInFlight),
HeartbeatSuccesses: metric.NewCounter(metaHeartbeatSuccesses),
HeartbeatFailures: metric.NewCounter(metaHeartbeatFailures),
EpochIncrements: metric.NewCounter(metaEpochIncrements),
HeartbeatFailures: telemetry.NewCounterWithMetric(metaHeartbeatFailures),
EpochIncrements: telemetry.NewCounterWithMetric(metaEpochIncrements),
HeartbeatLatency: metric.NewLatency(metaHeartbeatLatency, opts.HistogramWindowInterval),
}
nl.mu.nodes = make(map[roachpb.NodeID]Record)
Expand Down Expand Up @@ -978,7 +979,7 @@ func (nl *NodeLiveness) heartbeatInternal(
nl.metrics.HeartbeatSuccesses.Inc(1)
return nil
}
nl.metrics.HeartbeatFailures.Inc(1)
nl.metrics.HeartbeatFailures.Inc()
return err
}

Expand Down Expand Up @@ -1211,7 +1212,7 @@ func (nl *NodeLiveness) IncrementEpoch(ctx context.Context, liveness livenesspb.

log.Infof(ctx, "incremented n%d liveness epoch to %d", written.NodeID, written.Epoch)
nl.maybeUpdate(ctx, written)
nl.metrics.EpochIncrements.Inc(1)
nl.metrics.EpochIncrements.Inc()
return nil
}

Expand Down
1 change: 1 addition & 0 deletions pkg/server/telemetry/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ go_test(
srcs = ["features_test.go"],
deps = [
":telemetry",
"//pkg/util/metric",
"@com_github_stretchr_testify//require",
],
)
7 changes: 7 additions & 0 deletions pkg/server/telemetry/features.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,6 +128,13 @@ func (c CounterWithMetric) Inc() {
c.metric.Inc(1)
}

// Count returns the value of the metric, not the telemetry. Note that the
// telemetry value may reset to zero when, for example, GetFeatureCounts() is
// called with ResetCounts to generate a report.
func (c CounterWithMetric) Count() int64 {
return c.metric.Count()
}

// Forward the metric.Iterable interface to the metric counter. We
// don't just embed the counter because our Inc() interface is a bit
// different.
Expand Down
21 changes: 21 additions & 0 deletions pkg/server/telemetry/features_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"testing"

"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -91,3 +92,23 @@ func TestBucket(t *testing.T) {
}
}
}

// TestCounterWithMetric verifies that only the telemetry is reset to zero when,
// for example, a report is created.
func TestCounterWithMetric(t *testing.T) {
cm := telemetry.NewCounterWithMetric(metric.Metadata{Name: "test-metric"})
cm.Inc()

// Using GetFeatureCounts to read the telemetry value.
m1 := telemetry.GetFeatureCounts(telemetry.Raw, telemetry.ReadOnly)
require.Equal(t, int32(1), m1["test-metric"])
require.Equal(t, int64(1), cm.Count())

// Reset the telemetry.
telemetry.GetFeatureCounts(telemetry.Raw, telemetry.ResetCounts)

// Verify only the telemetry is back to 0.
m2 := telemetry.GetFeatureCounts(telemetry.Raw, telemetry.ReadOnly)
require.Equal(t, int32(0), m2["test-metric"])
require.Equal(t, int64(1), cm.Count())
}

0 comments on commit 47ce2f0

Please sign in to comment.