Skip to content

Commit

Permalink
metrics: improve ux around _status/vars output
Browse files Browse the repository at this point in the history
Previously, the addition of the `tenant` metric label was applied
uniformly and could result in confusion for customers who never enable
multi-tenancy or c2c. The `tenant="system"` label carries little
meaning when there's no tenancy in use.

This change modifies the system tenant label application to only
happen when a non- sytem in-process tenant is created.

Additionally, an environment variable:
`COCKROACH_DISABLE_NODE_AND_TENANT_METRIC_LABELS` can be set to
`false` to disable the new `tenant` and `node_id` labels. This can be
used on single-process tenants to disable the `tenant` label.

When the `tenantNameContainer` is nil, or the `nodeID` is set to
0, the labels will not be applied during recorder configuration on
startup. This is currently the case when running a separate process
tenant using `mt start-sql`. Those tenants *will not* have `tenant` or
`nodeID` labels available.

Resolves: #94668

Epic: CRDB-18798

Release note (ops change): The
`COCKROACH_DISABLE_NODE_AND_TENANT_METRIC_LABELS` env var can be used
to disable the newly introduced metric labels in the `_status/vars`
output if they conflict with a customer's scrape configuration.
  • Loading branch information
dhartunian committed Mar 27, 2023
1 parent a5b41ca commit a6a1a4c
Show file tree
Hide file tree
Showing 4 changed files with 50 additions and 24 deletions.
1 change: 1 addition & 0 deletions pkg/server/status/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@ go_library(
"//pkg/server/status/statuspb",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/sql/sem/catconstants",
"//pkg/ts/tspb",
"//pkg/ts/tsutil",
"//pkg/util/cgroups",
Expand Down
37 changes: 31 additions & 6 deletions pkg/server/status/recorder.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"os"
"runtime"
"strconv"
"sync"
"sync/atomic"
"time"

Expand All @@ -33,6 +34,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/server/status/statuspb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
"github.com/cockroachdb/cockroach/pkg/ts/tspb"
"github.com/cockroachdb/cockroach/pkg/ts/tsutil"
"github.com/cockroachdb/cockroach/pkg/util/cgroups"
Expand Down Expand Up @@ -60,8 +62,14 @@ const (
advertiseAddrLabelKey = "advertise-addr"
httpAddrLabelKey = "http-addr"
sqlAddrLabelKey = "sql-addr"

disableNodeAndTenantLabelsEnvVar = "COCKROACH_DISABLE_NODE_AND_TENANT_METRIC_LABELS"
)

// This option is provided as an escape hatch for customers who have
// custom scrape logic that adds relevant labels already.
var disableNodeAndTenantLabels = envutil.EnvOrDefaultBool(disableNodeAndTenantLabelsEnvVar, false)

type quantile struct {
suffix string
quantile float64
Expand Down Expand Up @@ -130,6 +138,7 @@ type MetricsRecorder struct {
// RLock on it.
mu struct {
syncutil.RWMutex
sync.Once
// nodeRegistry contains, as subregistries, the multiple component-specific
// registries which are recorded as "node level" metrics.
nodeRegistry *metric.Registry
Expand Down Expand Up @@ -187,6 +196,14 @@ func (mr *MetricsRecorder) AddTenantRegistry(tenantID roachpb.TenantID, rec *met
mr.mu.Lock()
defer mr.mu.Unlock()

if !disableNodeAndTenantLabels {
// If there are no in-process tenants running, we don't set the
// tenant label on the system tenant metrics until a seconary
// tenant is initialized.
mr.mu.Do(func() {
mr.mu.nodeRegistry.AddLabel("tenant", catconstants.SystemTenantName)
})
}
mr.mu.tenantRegistries[tenantID] = rec
}

Expand Down Expand Up @@ -226,12 +243,20 @@ func (mr *MetricsRecorder) AddNode(
nodeIDGauge := metric.NewGauge(metadata)
nodeIDGauge.Update(int64(desc.NodeID))
reg.AddMetric(nodeIDGauge)
reg.AddLabel("tenant", mr.tenantNameContainer)
reg.AddLabel("node_id", strconv.Itoa(int(desc.NodeID)))
// We assume that all stores have been added to the registry
// prior to calling `AddNode`.
for _, s := range mr.mu.storeRegistries {
s.AddLabel("node_id", strconv.Itoa(int(desc.NodeID)))

if !disableNodeAndTenantLabels {
nodeIDInt := int(desc.NodeID)
if nodeIDInt != 0 {
reg.AddLabel("node_id", strconv.Itoa(int(desc.NodeID)))
// We assume that all stores have been added to the registry
// prior to calling `AddNode`.
for _, s := range mr.mu.storeRegistries {
s.AddLabel("node_id", strconv.Itoa(int(desc.NodeID)))
}
}
if mr.tenantNameContainer != nil && mr.tenantNameContainer.String() != catconstants.SystemTenantName {
reg.AddLabel("tenant", mr.tenantNameContainer)
}
}
}

Expand Down
16 changes: 8 additions & 8 deletions pkg/server/status/recorder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -154,15 +154,15 @@ func TestMetricsRecorderLabels(t *testing.T) {
err = recorder.PrintAsText(buf)
require.NoError(t, err)

require.Contains(t, buf.String(), `some_metric{tenant="system",node_id="7"} 123`)
require.Contains(t, buf.String(), `some_metric{tenant="application",node_id="7"} 456`)
require.Contains(t, buf.String(), `some_metric{node_id="7",tenant="system"} 123`)
require.Contains(t, buf.String(), `some_metric{node_id="7",tenant="application"} 456`)

bufTenant := bytes.NewBuffer([]byte{})
err = recorderTenant.PrintAsText(bufTenant)
require.NoError(t, err)

require.NotContains(t, bufTenant.String(), `some_metric{tenant="system",node_id="7"} 123`)
require.Contains(t, bufTenant.String(), `some_metric{tenant="application",node_id="7"} 456`)
require.NotContains(t, bufTenant.String(), `some_metric{node_id="7",tenant="system"} 123`)
require.Contains(t, bufTenant.String(), `some_metric{node_id="7",tenant="application"} 456`)

// Update app name in container and ensure
// output changes accordingly.
Expand All @@ -172,15 +172,15 @@ func TestMetricsRecorderLabels(t *testing.T) {
err = recorder.PrintAsText(buf)
require.NoError(t, err)

require.Contains(t, buf.String(), `some_metric{tenant="system",node_id="7"} 123`)
require.Contains(t, buf.String(), `some_metric{tenant="application2",node_id="7"} 456`)
require.Contains(t, buf.String(), `some_metric{node_id="7",tenant="system"} 123`)
require.Contains(t, buf.String(), `some_metric{node_id="7",tenant="application2"} 456`)

bufTenant = bytes.NewBuffer([]byte{})
err = recorderTenant.PrintAsText(bufTenant)
require.NoError(t, err)

require.NotContains(t, bufTenant.String(), `some_metric{tenant="system",node_id="7"} 123`)
require.Contains(t, bufTenant.String(), `some_metric{tenant="application2",node_id="7"} 456`)
require.NotContains(t, bufTenant.String(), `some_metric{node_id="7",tenant="system"} 123`)
require.Contains(t, bufTenant.String(), `some_metric{node_id="7",tenant="application2"} 456`)

// ========================================
// Verify that the recorder processes tenant time series registries
Expand Down
20 changes: 10 additions & 10 deletions pkg/server/status_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1339,20 +1339,20 @@ func TestStatusVarsTxnMetrics(t *testing.T) {
if err != nil {
t.Fatal(err)
}
if !bytes.Contains(body, []byte("sql_txn_begin_count{tenant=\"system\",node_id=\"1\"} 1")) {
t.Errorf("expected `sql_txn_begin_count{tenant=\"system\",node_id=\"1\"} 1`, got: %s", body)
if !bytes.Contains(body, []byte("sql_txn_begin_count{node_id=\"1\"} 1")) {
t.Errorf("expected `sql_txn_begin_count{node_id=\"1\"} 1`, got: %s", body)
}
if !bytes.Contains(body, []byte("sql_restart_savepoint_count{tenant=\"system\",node_id=\"1\"} 1")) {
t.Errorf("expected `sql_restart_savepoint_count{tenant=\"system\",node_id=\"1\"} 1`, got: %s", body)
if !bytes.Contains(body, []byte("sql_restart_savepoint_count{node_id=\"1\"} 1")) {
t.Errorf("expected `sql_restart_savepoint_count{node_id=\"1\"} 1`, got: %s", body)
}
if !bytes.Contains(body, []byte("sql_restart_savepoint_release_count{tenant=\"system\",node_id=\"1\"} 1")) {
t.Errorf("expected `sql_restart_savepoint_release_count{tenant=\"system\",node_id=\"1\"} 1`, got: %s", body)
if !bytes.Contains(body, []byte("sql_restart_savepoint_release_count{node_id=\"1\"} 1")) {
t.Errorf("expected `sql_restart_savepoint_release_count{node_id=\"1\"} 1`, got: %s", body)
}
if !bytes.Contains(body, []byte("sql_txn_commit_count{tenant=\"system\",node_id=\"1\"} 1")) {
t.Errorf("expected `sql_txn_commit_count{tenant=\"system\",node_id=\"1\"} 1`, got: %s", body)
if !bytes.Contains(body, []byte("sql_txn_commit_count{node_id=\"1\"} 1")) {
t.Errorf("expected `sql_txn_commit_count{node_id=\"1\"} 1`, got: %s", body)
}
if !bytes.Contains(body, []byte("sql_txn_rollback_count{tenant=\"system\",node_id=\"1\"} 0")) {
t.Errorf("expected `sql_txn_rollback_count{tenant=\"system\",node_id=\"1\"} 0`, got: %s", body)
if !bytes.Contains(body, []byte("sql_txn_rollback_count{node_id=\"1\"} 0")) {
t.Errorf("expected `sql_txn_rollback_count{node_id=\"1\"} 0`, got: %s", body)
}
}

Expand Down

0 comments on commit a6a1a4c

Please sign in to comment.