Skip to content

Commit

Permalink
Merge #108841
Browse files Browse the repository at this point in the history
108841: rpc: fix missed gauge Unlink r=aliher1911 a=pavelkalinnikov

The `peerMetrics.ConnectionInactive` gauge is not unlinked from the aggregated metric. Correspondingly, when a connection with the same `peerKey` is re-created, the metric with this key is already registered, which causes a panic.

This PR introduces the missed metric `Unlink`. It also add a regression test that would have caught the bug.

Fixes #108499
Epic: CRDB-21710
Release note: none

Co-authored-by: Pavel Kalinnikov <[email protected]>
  • Loading branch information
craig[bot] and pav-kv committed Aug 16, 2023
2 parents 126274a + 6f737d9 commit e160427
Show file tree
Hide file tree
Showing 3 changed files with 71 additions and 0 deletions.
2 changes: 2 additions & 0 deletions pkg/rpc/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -112,6 +112,7 @@ go_test(
"heartbeat_test.go",
"helpers_test.go",
"main_test.go",
"metrics_test.go",
"peer_test.go",
"snappy_test.go",
"tls_test.go",
Expand Down Expand Up @@ -159,6 +160,7 @@ go_test(
"@com_github_cockroachdb_redact//:redact",
"@com_github_gogo_status//:status",
"@com_github_golang_mock//gomock",
"@com_github_prometheus_client_model//go",
"@com_github_stretchr_testify//assert",
"@com_github_stretchr_testify//require",
"@io_etcd_go_raft_v3//raftpb",
Expand Down
3 changes: 3 additions & 0 deletions pkg/rpc/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -147,6 +147,8 @@ type peerMetrics struct {
// must be reset before removing, and there must not be any chance that
// they're set again because even if they're unlinked from the parent, they
// will continue to add to the parent!
//
// See TestMetricsRelease.

// 1 on first heartbeat success (via reportHealthy), reset after
// runHeartbeatUntilFailure returns.
Expand Down Expand Up @@ -216,6 +218,7 @@ func (pm *peerMetrics) release() {

pm.ConnectionHealthy.Unlink()
pm.ConnectionUnhealthy.Unlink()
pm.ConnectionInactive.Unlink()
pm.ConnectionHealthyFor.Unlink()
pm.ConnectionUnhealthyFor.Unlink()
pm.ConnectionHeartbeats.Unlink()
Expand Down
66 changes: 66 additions & 0 deletions pkg/rpc/metrics_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,66 @@
// Copyright 2023 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package rpc

import (
"reflect"
"testing"

"github.com/cockroachdb/cockroach/pkg/util/leaktest"
io_prometheus_client "github.com/prometheus/client_model/go"
"github.com/stretchr/testify/require"
)

// TestMetricsRelease verifies that peerMetrics.release() removes tracking for
// *all* the metrics from their parent aggregate metric.
func TestMetricsRelease(t *testing.T) {
defer leaktest.AfterTest(t)()

// All metrics in aggmetric package satisfy this interface. The `Each` method
// can be used to scan all child metrics of the aggregated metric. We use it
// for counting children.
type eacher interface {
Each([]*io_prometheus_client.LabelPair, func(metric *io_prometheus_client.Metric))
}
countChildren := func(metric eacher) (count int) {
metric.Each(nil /*labels*/, func(*io_prometheus_client.Metric) {
count++
})
return count
}

verifyAllFields := func(m Metrics, wantChildren int) (metricFields int) {
r := reflect.ValueOf(m)
for i, n := 0, r.NumField(); i < n; i++ {
field := r.Field(i).Interface()
metric, ok := field.(eacher)
if !ok { // skip all non-metric fields
continue
}
metricFields++
require.Equal(t, wantChildren, countChildren(metric), r.Type().Field(i).Name)
}
return metricFields
}

m := makeMetrics()
// Verify that each metric doesn't have any children at first. Verify the
// number of metric fields, as a sanity check (to be modified if fields are
// added/deleted).
require.Equal(t, 8, verifyAllFields(m, 0))
// Verify that a new peer's metrics all get registered.
k := peerKey{NodeID: 5, TargetAddr: "192.168.0.1:1234", Class: DefaultClass}
pm := m.acquire(k)
require.Equal(t, 8, verifyAllFields(m, 1))
// Verify that all metrics are unlinked when the peer is released.
pm.release()
require.Equal(t, 8, verifyAllFields(m, 0))
}

0 comments on commit e160427

Please sign in to comment.