Skip to content

Commit d851cc3

Browse files
committed
metric: add useStaticLabels to GetName()
Remove the `GetLabeledName()` API and change all call-sites. Release note: None
1 parent 2733799 commit d851cc3

File tree

17 files changed

+63
-88
lines changed

17 files changed

+63
-88
lines changed

pkg/ccl/multitenantccl/tenantcostclient/tenant_side_test.go

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -636,19 +636,19 @@ func (ts *testState) metrics(*testing.T, *datadriven.TestData, cmdArgs) string {
636636
typ.Inspect(func(v interface{}) {
637637
switch it := v.(type) {
638638
case *metric.Gauge:
639-
state[typ.GetName()] = it.Value()
639+
state[typ.GetName(false /* useStaticLabels */)] = it.Value()
640640
case *metric.Counter:
641-
state[typ.GetName()] = it.Count()
641+
state[typ.GetName(false /* useStaticLabels */)] = it.Count()
642642
case *metric.CounterFloat64:
643-
state[typ.GetName()] = fmt.Sprintf("%.2f", it.Count())
643+
state[typ.GetName(false /* useStaticLabels */)] = fmt.Sprintf("%.2f", it.Count())
644644
case *aggmetric.AggCounter:
645-
state[typ.GetName()] = it.Count()
645+
state[typ.GetName(false /* useStaticLabels */)] = it.Count()
646646
promIter, ok := v.(metric.PrometheusIterable)
647647
if !ok {
648648
return
649649
}
650650
promIter.Each(it.GetLabels(false /* useStaticLabels */), func(m *prometheusgo.Metric) {
651-
childMetrics += fmt.Sprintf("%s{", typ.GetName())
651+
childMetrics += fmt.Sprintf("%s{", typ.GetName(false /* useStaticLabels */))
652652
for i, l := range m.Label {
653653
if i > 0 {
654654
childMetrics += ","

pkg/gossip/client_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -244,7 +244,7 @@ func TestClientGossipMetrics(t *testing.T) {
244244
s.nodeMetrics.BytesReceived,
245245
} {
246246
if count := counter.Count(); count <= 0 {
247-
return errors.Errorf("%d: expected metrics counter %q > 0; = %d", i, counter.GetName(), count)
247+
return errors.Errorf("%d: expected metrics counter %q > 0; = %d", i, counter.GetName(false /* useStaticLabels */), count)
248248
}
249249
}
250250
}

pkg/kv/kvserver/client_metrics_test.go

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -36,14 +36,14 @@ import (
3636
)
3737

3838
type gaugeValuer interface {
39-
GetName() string
39+
GetName(useStaticLabels bool) string
4040
Value() int64
4141
}
4242

4343
func checkGauge(t *testing.T, id string, g gaugeValuer, e int64) {
4444
t.Helper()
4545
if a := g.Value(); a != e {
46-
t.Error(errors.Errorf("%s for store %s: gauge %d != computed %d", g.GetName(), id, a, e))
46+
t.Error(errors.Errorf("%s for store %s: gauge %d != computed %d", g.GetName(false /* useStaticLabels */), id, a, e))
4747
}
4848
}
4949

@@ -368,7 +368,7 @@ func TestStoreMetrics(t *testing.T) {
368368
{m.RdbTableReadersMemEstimate, 50},
369369
} {
370370
if a := tc.gauge.Value(); a < tc.min {
371-
t.Errorf("gauge %s = %d < min %d", tc.gauge.GetName(), a, tc.min)
371+
t.Errorf("gauge %s = %d < min %d", tc.gauge.GetName(false /* useStaticLabels */), a, tc.min)
372372
}
373373
}
374374
for _, tc := range []struct {
@@ -382,7 +382,7 @@ func TestStoreMetrics(t *testing.T) {
382382
{m.RdbCompactions, 0},
383383
} {
384384
if a := tc.counter.Count(); a < tc.min {
385-
t.Errorf("counter %s = %d < min %d", tc.counter.GetName(), a, tc.min)
385+
t.Errorf("counter %s = %d < min %d", tc.counter.GetName(false /* useStaticLabels */), a, tc.min)
386386
}
387387
}
388388
}

pkg/kv/kvserver/kvflowcontrol/rac2/range_controller_test.go

Lines changed: 13 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -1522,20 +1522,20 @@ func TestRangeController(t *testing.T) {
15221522
admissionpb.RegularWorkClass,
15231523
admissionpb.ElasticWorkClass,
15241524
} {
1525-
fmt.Fprintf(&buf, "%-50v: %v\n", evalMetrics.Waiting[wc].GetName(), evalMetrics.Waiting[wc].Value())
1526-
fmt.Fprintf(&buf, "%-50v: %v\n", evalMetrics.Admitted[wc].GetName(), evalMetrics.Admitted[wc].Count())
1527-
fmt.Fprintf(&buf, "%-50v: %v\n", evalMetrics.Errored[wc].GetName(), evalMetrics.Errored[wc].Count())
1528-
fmt.Fprintf(&buf, "%-50v: %v\n", evalMetrics.Bypassed[wc].GetName(), evalMetrics.Bypassed[wc].Count())
1525+
fmt.Fprintf(&buf, "%-50v: %v\n", evalMetrics.Waiting[wc].GetName(false /* useStaticLabels */), evalMetrics.Waiting[wc].Value())
1526+
fmt.Fprintf(&buf, "%-50v: %v\n", evalMetrics.Admitted[wc].GetName(false /* useStaticLabels */), evalMetrics.Admitted[wc].Count())
1527+
fmt.Fprintf(&buf, "%-50v: %v\n", evalMetrics.Errored[wc].GetName(false /* useStaticLabels */), evalMetrics.Errored[wc].Count())
1528+
fmt.Fprintf(&buf, "%-50v: %v\n", evalMetrics.Bypassed[wc].GetName(false /* useStaticLabels */), evalMetrics.Bypassed[wc].Count())
15291529
// We only print the number of recorded durations, instead of any
15301530
// percentiles or cumulative wait times as these are
15311531
// non-deterministic in the test.
15321532
fmt.Fprintf(&buf, "%-50v: %v\n",
1533-
fmt.Sprintf("%v.count", evalMetrics.Duration[wc].GetName()),
1533+
fmt.Sprintf("%v.count", evalMetrics.Duration[wc].GetName(false /* useStaticLabels */)),
15341534
testingFirst(evalMetrics.Duration[wc].CumulativeSnapshot().Total()))
15351535
}
15361536
case "range_controller":
15371537
rcMetrics := state.rcMetrics
1538-
fmt.Fprintf(&buf, "%v: %v\n", rcMetrics.Count.GetName(), rcMetrics.Count.Value())
1538+
fmt.Fprintf(&buf, "%v: %v\n", rcMetrics.Count.GetName(false /* useStaticLabels */), rcMetrics.Count.Value())
15391539
case "send_queue":
15401540
sendQueueMetrics := state.rcMetrics.SendQueue
15411541
sendQueueTokenMetrics := state.ssTokenCounter.tokenMetrics.CounterMetrics[SendToken].SendQueue[0]
@@ -1556,17 +1556,17 @@ func TestRangeController(t *testing.T) {
15561556
}
15571557
sendQueueMetrics.SizeBytes.Update(sizeBytes)
15581558
sendQueueMetrics.SizeCount.Update(sizeCount)
1559-
fmt.Fprintf(&buf, "%-66v: %v\n", sendQueueMetrics.SizeCount.GetName(), sendQueueMetrics.SizeCount.Value())
1560-
fmt.Fprintf(&buf, "%-66v: %v\n", sendQueueMetrics.SizeBytes.GetName(), sendQueueMetrics.SizeBytes.Value())
1561-
fmt.Fprintf(&buf, "%-66v: %v\n", sendQueueMetrics.ForceFlushedScheduledCount.GetName(), sendQueueMetrics.ForceFlushedScheduledCount.Value())
1562-
fmt.Fprintf(&buf, "%-66v: %v\n", sendQueueMetrics.DeductedForSchedulerBytes.GetName(), sendQueueMetrics.DeductedForSchedulerBytes.Value())
1563-
fmt.Fprintf(&buf, "%-66v: %v\n", sendQueueMetrics.PreventionCount.GetName(), sendQueueMetrics.PreventionCount.Count())
1564-
fmt.Fprintf(&buf, "%-66v: %v\n", sendQueueTokenMetrics.ForceFlushDeducted.GetName(), sendQueueTokenMetrics.ForceFlushDeducted.Count())
1559+
fmt.Fprintf(&buf, "%-66v: %v\n", sendQueueMetrics.SizeCount.GetName(false /* useStaticLabels */), sendQueueMetrics.SizeCount.Value())
1560+
fmt.Fprintf(&buf, "%-66v: %v\n", sendQueueMetrics.SizeBytes.GetName(false /* useStaticLabels */), sendQueueMetrics.SizeBytes.Value())
1561+
fmt.Fprintf(&buf, "%-66v: %v\n", sendQueueMetrics.ForceFlushedScheduledCount.GetName(false /* useStaticLabels */), sendQueueMetrics.ForceFlushedScheduledCount.Value())
1562+
fmt.Fprintf(&buf, "%-66v: %v\n", sendQueueMetrics.DeductedForSchedulerBytes.GetName(false /* useStaticLabels */), sendQueueMetrics.DeductedForSchedulerBytes.Value())
1563+
fmt.Fprintf(&buf, "%-66v: %v\n", sendQueueMetrics.PreventionCount.GetName(false /* useStaticLabels */), sendQueueMetrics.PreventionCount.Count())
1564+
fmt.Fprintf(&buf, "%-66v: %v\n", sendQueueTokenMetrics.ForceFlushDeducted.GetName(false /* useStaticLabels */), sendQueueTokenMetrics.ForceFlushDeducted.Count())
15651565
for _, wc := range []admissionpb.WorkClass{
15661566
admissionpb.RegularWorkClass,
15671567
admissionpb.ElasticWorkClass,
15681568
} {
1569-
fmt.Fprintf(&buf, "%-66v: %v\n", sendQueueTokenMetrics.PreventionDeducted[wc].GetName(), sendQueueTokenMetrics.PreventionDeducted[wc].Count())
1569+
fmt.Fprintf(&buf, "%-66v: %v\n", sendQueueTokenMetrics.PreventionDeducted[wc].GetName(false /* useStaticLabels */), sendQueueTokenMetrics.PreventionDeducted[wc].Count())
15701570
}
15711571

15721572
default:

pkg/kv/kvserver/kvflowcontrol/rac2/token_counter_test.go

Lines changed: 9 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -66,23 +66,23 @@ func TestTokenAdjustment(t *testing.T) {
6666
admissionpb.ElasticWorkClass,
6767
} {
6868
fmt.Fprintf(buf, " %-7v\n", wc)
69-
fmt.Fprintf(buf, " %-66v: %v\n", streamMetrics.Count[wc].GetName(), streamMetrics.Count[wc].Value())
70-
fmt.Fprintf(buf, " %-66v: %v\n", streamMetrics.BlockedCount[wc].GetName(), streamMetrics.BlockedCount[wc].Value())
71-
fmt.Fprintf(buf, " %-66v: %v\n", streamMetrics.TokensAvailable[wc].GetName(), ft(streamMetrics.TokensAvailable[wc].Value()))
72-
fmt.Fprintf(buf, " %-66v: %v\n", counterMetrics.Deducted[wc].GetName(), ft(counterMetrics.Deducted[wc].Count()))
73-
fmt.Fprintf(buf, " %-66v: %v\n", counterMetrics.Disconnected[wc].GetName(), ft(counterMetrics.Disconnected[wc].Count()))
74-
fmt.Fprintf(buf, " %-66v: %v\n", counterMetrics.Returned[wc].GetName(), ft(counterMetrics.Returned[wc].Count()))
75-
fmt.Fprintf(buf, " %-66v: %v\n", counterMetrics.Unaccounted[wc].GetName(), ft(counterMetrics.Unaccounted[wc].Count()))
69+
fmt.Fprintf(buf, " %-66v: %v\n", streamMetrics.Count[wc].GetName(false /* useStaticLabels */), streamMetrics.Count[wc].Value())
70+
fmt.Fprintf(buf, " %-66v: %v\n", streamMetrics.BlockedCount[wc].GetName(false /* useStaticLabels */), streamMetrics.BlockedCount[wc].Value())
71+
fmt.Fprintf(buf, " %-66v: %v\n", streamMetrics.TokensAvailable[wc].GetName(false /* useStaticLabels */), ft(streamMetrics.TokensAvailable[wc].Value()))
72+
fmt.Fprintf(buf, " %-66v: %v\n", counterMetrics.Deducted[wc].GetName(false /* useStaticLabels */), ft(counterMetrics.Deducted[wc].Count()))
73+
fmt.Fprintf(buf, " %-66v: %v\n", counterMetrics.Disconnected[wc].GetName(false /* useStaticLabels */), ft(counterMetrics.Disconnected[wc].Count()))
74+
fmt.Fprintf(buf, " %-66v: %v\n", counterMetrics.Returned[wc].GetName(false /* useStaticLabels */), ft(counterMetrics.Returned[wc].Count()))
75+
fmt.Fprintf(buf, " %-66v: %v\n", counterMetrics.Unaccounted[wc].GetName(false /* useStaticLabels */), ft(counterMetrics.Unaccounted[wc].Count()))
7676
}
7777
if t == SendToken {
7878
sendQueueMetrics := counterMetrics.SendQueue[0]
7979
fmt.Fprintf(buf, " send queue token metrics\n")
80-
fmt.Fprintf(buf, " %-66v: %v\n", sendQueueMetrics.ForceFlushDeducted.GetName(), ft(sendQueueMetrics.ForceFlushDeducted.Count()))
80+
fmt.Fprintf(buf, " %-66v: %v\n", sendQueueMetrics.ForceFlushDeducted.GetName(false /* useStaticLabels */), ft(sendQueueMetrics.ForceFlushDeducted.Count()))
8181
for _, wc := range []admissionpb.WorkClass{
8282
admissionpb.RegularWorkClass,
8383
admissionpb.ElasticWorkClass,
8484
} {
85-
fmt.Fprintf(buf, " %-66v: %v\n", sendQueueMetrics.PreventionDeducted[wc].GetName(), ft(sendQueueMetrics.PreventionDeducted[wc].Count()))
85+
fmt.Fprintf(buf, " %-66v: %v\n", sendQueueMetrics.PreventionDeducted[wc].GetName(false /* useStaticLabels */), ft(sendQueueMetrics.PreventionDeducted[wc].Count()))
8686
}
8787
}
8888
}

pkg/server/status/recorder.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -831,7 +831,7 @@ func (rr registryRecorder) recordChild(
831831
return
832832
}
833833
*dest = append(*dest, tspb.TimeSeriesData{
834-
Name: fmt.Sprintf(rr.format, prom.GetName()),
834+
Name: fmt.Sprintf(rr.format, prom.GetName(false /* useStaticLabels */)),
835835
Source: rr.source,
836836
Datapoints: []tspb.TimeSeriesDatapoint{
837837
{

pkg/server/status/recorder_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -757,7 +757,7 @@ func BenchmarkExtractValueAllocs(b *testing.B) {
757757

758758
// Run a benchmark and report allocations.
759759
for n := 0; n < b.N; n++ {
760-
if err := extractValue(h.GetName(), h, func(string, float64) {}); err != nil {
760+
if err := extractValue(h.GetName(false /* useStaticLabels */), h, func(string, float64) {}); err != nil {
761761
b.Error(err)
762762
}
763763
}

pkg/server/telemetry/features.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -135,8 +135,8 @@ func (c CounterWithMetric) Count() int64 {
135135
// different.
136136

137137
// GetName implements metric.Iterable
138-
func (c CounterWithMetric) GetName() string {
139-
return c.metric.GetName()
138+
func (c CounterWithMetric) GetName(useStaticLabels bool) string {
139+
return c.metric.GetName(useStaticLabels)
140140
}
141141

142142
// GetHelp implements metric.Iterable

pkg/util/asciitsdb/asciitsdb.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -225,7 +225,7 @@ func (t *TSDB) registerIterable(metric metric.Iterable) {
225225
if t.mu.scraped {
226226
t.t.Fatalf("register all metrics upfront before Scrape()")
227227
}
228-
t.mu.points[metric.GetName()] = []float64{}
228+
t.mu.points[metric.GetName(false /* useStaticLabels */)] = []float64{}
229229
}
230230

231231
// Option represents a configuration setting.

pkg/util/metric/aggmetric/counter.go

Lines changed: 2 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -35,10 +35,7 @@ func NewCounter(metadata metric.Metadata, childLabels ...string) *AggCounter {
3535
}
3636

3737
// GetName is part of the metric.Iterable interface.
38-
func (c *AggCounter) GetName() string { return c.g.GetName() }
39-
40-
// GetLabeledName is part of the metric.Iterable interface.
41-
func (c *AggCounter) GetLabeledName() string { return c.g.GetLabeledName() }
38+
func (c *AggCounter) GetName(useStaticLabels bool) string { return c.g.GetName(useStaticLabels) }
4239

4340
// GetHelp is part of the metric.Iterable interface.
4441
func (c *AggCounter) GetHelp() string { return c.g.GetHelp() }
@@ -177,10 +174,7 @@ func NewCounterFloat64(metadata metric.Metadata, childLabels ...string) *AggCoun
177174
}
178175

179176
// GetName is part of the metric.Iterable interface.
180-
func (c *AggCounterFloat64) GetName() string { return c.g.GetName() }
181-
182-
// GetLabeledName is part of the metric.Iterable interface.
183-
func (c *AggCounterFloat64) GetLabeledName() string { return c.g.GetLabeledName() }
177+
func (c *AggCounterFloat64) GetName(useStaticLabels bool) string { return c.g.GetName(useStaticLabels) }
184178

185179
// GetHelp is part of the metric.Iterable interface.
186180
func (c *AggCounterFloat64) GetHelp() string { return c.g.GetHelp() }

0 commit comments

Comments
 (0)