Skip to content

Commit 3389d85

Browse files
craig[bot]aa-joshi
andcommitted
Merge #153571
153571: util/metric: introduce HighCardinalityGauge in aggregated metrics r=aa-joshi a=aa-joshi This patch introduces `HighCardinalityGauge` metric which is similar to the `HighCardinalityCounter` introduced in #153568. It relies on unordered cache with LRU eviction as child storage. The parent values represents the aggregation of all child metric values. The child metrics values are only exported and aggregated values is persisted in CRDB. It relies on LabelSliceCache to efficiently store label values at registry. The child metric eviction policy is combination of max cache size of 5000 and minimum retention time of 20 seconds. This guarantees that we would see the child metric values at least in one scrape with default interval of 10 seconds before getting evicted due to cache size. The child eviction won't impact the parent value. Epic: CRDB-53398 Part of: CRDB-53832 Release note: None Co-authored-by: Akshay Joshi <[email protected]>
2 parents 24f76c2 + 92cd985 commit 3389d85

File tree

5 files changed

+342
-0
lines changed

5 files changed

+342
-0
lines changed

pkg/util/metric/aggmetric/gauge.go

Lines changed: 211 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -8,8 +8,10 @@ package aggmetric
88
import (
99
"math"
1010
"sync/atomic"
11+
"time"
1112

1213
"github.com/cockroachdb/cockroach/pkg/util/metric"
14+
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
1315
"github.com/cockroachdb/errors"
1416
"github.com/gogo/protobuf/proto"
1517
io_prometheus_client "github.com/prometheus/client_model/go"
@@ -521,3 +523,212 @@ func (scg *SQLChildGauge) Inc(i int64) {
521523
func (scg *SQLChildGauge) Dec(i int64) {
522524
scg.gauge.Dec(i)
523525
}
526+
527+
// HighCardinalityGauge is similar to AggGauge but uses cache storage instead of B-tree,
528+
// allowing for automatic eviction of less frequently used child metrics.
529+
// This is useful when dealing with high cardinality metrics that might exceed resource limits.
530+
type HighCardinalityGauge struct {
531+
g metric.Gauge
532+
childSet
533+
labelSliceCache *metric.LabelSliceCache
534+
}
535+
536+
var _ metric.Iterable = (*HighCardinalityGauge)(nil)
537+
var _ metric.PrometheusEvictable = (*HighCardinalityGauge)(nil)
538+
539+
// NewHighCardinalityGauge constructs a new HighCardinalityGauge that uses cache storage
540+
// with eviction for child metrics.
541+
func NewHighCardinalityGauge(
542+
metadata metric.Metadata, childLabels ...string,
543+
) *HighCardinalityGauge {
544+
g := &HighCardinalityGauge{g: *metric.NewGauge(metadata)}
545+
g.initWithCacheStorageType(childLabels, metadata.Name)
546+
return g
547+
}
548+
549+
// GetName is part of the metric.Iterable interface.
550+
func (g *HighCardinalityGauge) GetName(useStaticLabels bool) string {
551+
return g.g.GetName(useStaticLabels)
552+
}
553+
554+
// GetHelp is part of the metric.Iterable interface.
555+
func (g *HighCardinalityGauge) GetHelp() string { return g.g.GetHelp() }
556+
557+
// GetMeasurement is part of the metric.Iterable interface.
558+
func (g *HighCardinalityGauge) GetMeasurement() string { return g.g.GetMeasurement() }
559+
560+
// GetUnit is part of the metric.Iterable interface.
561+
func (g *HighCardinalityGauge) GetUnit() metric.Unit { return g.g.GetUnit() }
562+
563+
// GetMetadata is part of the metric.Iterable interface.
564+
func (g *HighCardinalityGauge) GetMetadata() metric.Metadata { return g.g.GetMetadata() }
565+
566+
// Inspect is part of the metric.Iterable interface.
567+
func (g *HighCardinalityGauge) Inspect(f func(interface{})) { f(g) }
568+
569+
// GetType is part of the metric.PrometheusExportable interface.
570+
func (g *HighCardinalityGauge) GetType() *io_prometheus_client.MetricType {
571+
return g.g.GetType()
572+
}
573+
574+
// GetLabels is part of the metric.PrometheusExportable interface.
575+
func (g *HighCardinalityGauge) GetLabels(useStaticLabels bool) []*io_prometheus_client.LabelPair {
576+
return g.g.GetLabels(useStaticLabels)
577+
}
578+
579+
// ToPrometheusMetric is part of the metric.PrometheusExportable interface.
580+
func (g *HighCardinalityGauge) ToPrometheusMetric() *io_prometheus_client.Metric {
581+
return g.g.ToPrometheusMetric()
582+
}
583+
584+
// Value returns the aggregate sum of all of its current children.
585+
func (g *HighCardinalityGauge) Value() int64 {
586+
return g.g.Value()
587+
}
588+
589+
// Inc increments the gauge value by i for the given label values. If a
590+
// gauge with the given label values doesn't exist yet, it creates a new
591+
// gauge and increments it. Inc increments parent metrics as well.
592+
func (g *HighCardinalityGauge) Inc(i int64, labelValues ...string) {
593+
g.g.Inc(i)
594+
595+
childMetric := g.GetOrAddChild(labelValues...)
596+
597+
if childMetric != nil {
598+
childMetric.Inc(i)
599+
}
600+
}
601+
602+
// Dec decrements the gauge value by i for the given label values. If a
603+
// gauge with the given label values doesn't exist yet, it creates a new
604+
// gauge and decrements it. Dec decrements parent metrics as well.
605+
func (g *HighCardinalityGauge) Dec(i int64, labelValues ...string) {
606+
g.g.Dec(i)
607+
608+
childMetric := g.GetOrAddChild(labelValues...)
609+
610+
if childMetric != nil {
611+
childMetric.Dec(i)
612+
}
613+
}
614+
615+
// Update sets the gauge value to val for the given label values. If a
616+
// gauge with the given label values doesn't exist yet, it creates a new
617+
// gauge and sets it. Update updates parent metrics as well.
618+
//
619+
// The parent metric value represents the sum of all children. The parent
620+
// metric is updated by the delta (new - old) to maintain the aggregate
621+
// sum of all children.
622+
// For example, if a child gauge changes from 10 to 25, the
623+
// parent is incremented by 15, preserving the total sum across all children.
624+
func (g *HighCardinalityGauge) Update(val int64, labelValues ...string) {
625+
childMetric := g.GetOrAddChild(labelValues...)
626+
627+
if childMetric != nil {
628+
old := childMetric.Value()
629+
// Increment the parent by the delta of the child metric.
630+
g.g.Inc(val - old)
631+
childMetric.Update(val)
632+
}
633+
}
634+
635+
// Each is part of the metric.PrometheusIterable interface.
636+
func (g *HighCardinalityGauge) Each(
637+
labels []*io_prometheus_client.LabelPair, f func(metric *io_prometheus_client.Metric),
638+
) {
639+
g.EachWithLabels(labels, f, g.labelSliceCache)
640+
}
641+
642+
// InitializeMetrics is part of the PrometheusEvictable interface.
643+
func (g *HighCardinalityGauge) InitializeMetrics(labelCache *metric.LabelSliceCache) {
644+
g.mu.Lock()
645+
defer g.mu.Unlock()
646+
647+
g.labelSliceCache = labelCache
648+
}
649+
650+
// GetOrAddChild returns the existing child gauge for the given label values,
651+
// or creates a new one if it doesn't exist. This is the preferred method for
652+
// cache-based storage to avoid panics on existing keys.
653+
func (g *HighCardinalityGauge) GetOrAddChild(labelVals ...string) *HighCardinalityChildGauge {
654+
655+
if len(labelVals) == 0 {
656+
return nil
657+
}
658+
659+
// Create a LabelSliceCacheKey from the tenantID.
660+
key := metric.LabelSliceCacheKey(metricKey(labelVals...))
661+
662+
child := g.getOrAddWithLabelSliceCache(g.GetMetadata().Name, g.createHighCardinalityChildGauge, g.labelSliceCache, labelVals...)
663+
664+
g.labelSliceCache.Upsert(key, &metric.LabelSliceCacheValue{
665+
LabelValues: labelVals,
666+
})
667+
668+
return child.(*HighCardinalityChildGauge)
669+
}
670+
671+
func (g *HighCardinalityGauge) createHighCardinalityChildGauge(
672+
key uint64, cache *metric.LabelSliceCache,
673+
) LabelSliceCachedChildMetric {
674+
return &HighCardinalityChildGauge{
675+
LabelSliceCacheKey: metric.LabelSliceCacheKey(key),
676+
LabelSliceCache: cache,
677+
createdAt: timeutil.Now(),
678+
}
679+
}
680+
681+
// HighCardinalityChildGauge is a child of a HighCardinalityGauge. When metrics are
682+
// collected by prometheus, each of the children will appear with a distinct label,
683+
// however, when cockroach internally collects metrics, only the parent is collected.
684+
type HighCardinalityChildGauge struct {
685+
metric.LabelSliceCacheKey
686+
value metric.Gauge
687+
*metric.LabelSliceCache
688+
createdAt time.Time
689+
}
690+
691+
func (g *HighCardinalityChildGauge) CreatedAt() time.Time {
692+
return g.createdAt
693+
}
694+
695+
func (g *HighCardinalityChildGauge) DecrementLabelSliceCacheReference() {
696+
g.LabelSliceCache.DecrementAndDeleteIfZero(g.LabelSliceCacheKey)
697+
}
698+
699+
// ToPrometheusMetric constructs a prometheus metric for this HighCardinalityChildGauge.
700+
func (g *HighCardinalityChildGauge) ToPrometheusMetric() *io_prometheus_client.Metric {
701+
return &io_prometheus_client.Metric{
702+
Gauge: &io_prometheus_client.Gauge{
703+
Value: proto.Float64(float64(g.Value())),
704+
},
705+
}
706+
}
707+
708+
func (g *HighCardinalityChildGauge) labelValues() []string {
709+
lv, ok := g.LabelSliceCache.Get(g.LabelSliceCacheKey)
710+
if !ok {
711+
return nil
712+
}
713+
return lv.LabelValues
714+
}
715+
716+
// Value returns the HighCardinalityChildGauge's current value.
717+
func (g *HighCardinalityChildGauge) Value() int64 {
718+
return g.value.Value()
719+
}
720+
721+
// Inc increments the HighCardinalityChildGauge's value.
722+
func (g *HighCardinalityChildGauge) Inc(i int64) {
723+
g.value.Inc(i)
724+
}
725+
726+
// Dec decrements the HighCardinalityChildGauge's value.
727+
func (g *HighCardinalityChildGauge) Dec(i int64) {
728+
g.value.Dec(i)
729+
}
730+
731+
// Update sets the HighCardinalityChildGauge's value.
732+
func (g *HighCardinalityChildGauge) Update(val int64) {
733+
g.value.Update(val)
734+
}

pkg/util/metric/aggmetric/gauge_test.go

Lines changed: 95 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -8,12 +8,14 @@ package aggmetric
88
import (
99
"strconv"
1010
"testing"
11+
"time"
1112

1213
"github.com/cockroachdb/cockroach/pkg/testutils/datapathutils"
1314
"github.com/cockroachdb/cockroach/pkg/testutils/echotest"
1415
"github.com/cockroachdb/cockroach/pkg/util/cache"
1516
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
1617
"github.com/cockroachdb/cockroach/pkg/util/metric"
18+
"github.com/stretchr/testify/require"
1719
)
1820

1921
func TestSQLGaugeEviction(t *testing.T) {
@@ -80,3 +82,96 @@ func TestSQLGaugeMethods(t *testing.T) {
8082
testFile := "SQLGauge.txt"
8183
echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, testFile))
8284
}
85+
86+
func TestHighCardinalityGauge(t *testing.T) {
87+
defer leaktest.AfterTest(t)()
88+
89+
const cacheSize = 10
90+
r := metric.NewRegistry()
91+
writePrometheusMetrics := WritePrometheusMetricsFunc(r)
92+
93+
g := NewHighCardinalityGauge(metric.Metadata{
94+
Name: "foo_gauge",
95+
}, "database", "application_name")
96+
g.mu.children = &UnorderedCacheWrapper{
97+
cache: initialiseCacheStorageForTesting(),
98+
}
99+
r.AddMetric(g)
100+
101+
// Initialize with a label slice cache to test eviction
102+
labelSliceCache := metric.NewLabelSliceCache()
103+
g.InitializeMetrics(labelSliceCache)
104+
105+
for i := 0; i < cacheSize+5; i++ {
106+
g.Update(int64(i+1), "1", strconv.Itoa(i))
107+
}
108+
109+
// wait more than cache eviction time to make sure that keys are not evicted based on only cache size.
110+
time.Sleep(6 * time.Second)
111+
112+
testFile := "HighCardinalityGauge_pre_eviction.txt"
113+
echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, testFile))
114+
115+
for i := 0; i < cacheSize+5; i++ {
116+
metricKey := metric.LabelSliceCacheKey(metricKey("1", strconv.Itoa(i)))
117+
labelSliceValue, ok := labelSliceCache.Get(metricKey)
118+
require.True(t, ok, "missing labelSliceValue in label slice cache")
119+
require.Equal(t, int64(1), labelSliceValue.Counter.Load(), "the reference count should be 1")
120+
require.Equal(t, []string{"1", strconv.Itoa(i)}, labelSliceValue.LabelValues, "label values are mismatching")
121+
}
122+
123+
for i := 0 + cacheSize; i < cacheSize+5; i++ {
124+
g.Inc(5, "2", strconv.Itoa(i))
125+
}
126+
127+
testFile = "HighCardinalityGauge_post_eviction.txt"
128+
echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, testFile))
129+
130+
for i := 0; i < 5; i++ {
131+
metricKey := metric.LabelSliceCacheKey(metricKey("1", strconv.Itoa(i)))
132+
_, ok := labelSliceCache.Get(metricKey)
133+
require.False(t, ok, "labelSliceValue should not be present.")
134+
}
135+
136+
for i := 10; i < 15; i++ {
137+
metricKey := metric.LabelSliceCacheKey(metricKey("1", strconv.Itoa(i)))
138+
labelSliceValue, ok := labelSliceCache.Get(metricKey)
139+
require.True(t, ok, "missing labelSliceValue in label slice cache")
140+
require.Equal(t, int64(1), labelSliceValue.Counter.Load(), "the reference count should be 1")
141+
require.Equal(t, []string{"1", strconv.Itoa(i)}, labelSliceValue.LabelValues, "label values are mismatching")
142+
}
143+
144+
for i := 10; i < 15; i++ {
145+
metricKey := metric.LabelSliceCacheKey(metricKey("2", strconv.Itoa(i)))
146+
labelSliceValue, ok := labelSliceCache.Get(metricKey)
147+
require.True(t, ok, "missing labelSliceValue in label slice cache")
148+
require.Equal(t, int64(1), labelSliceValue.Counter.Load(), "the reference count should be 1")
149+
require.Equal(t, []string{"2", strconv.Itoa(i)}, labelSliceValue.LabelValues, "label values are mismatching")
150+
}
151+
}
152+
153+
func TestHighCardinalityGaugeMethods(t *testing.T) {
154+
defer leaktest.AfterTest(t)()
155+
r := metric.NewRegistry()
156+
writePrometheusMetrics := WritePrometheusMetricsFunc(r)
157+
158+
g := NewHighCardinalityGauge(metric.Metadata{
159+
Name: "foo_gauge",
160+
}, "database", "application_name")
161+
g.mu.children = &UnorderedCacheWrapper{
162+
cache: initialiseCacheStorageForTesting(),
163+
}
164+
165+
r.AddMetric(g)
166+
167+
// Test Update operation
168+
g.Update(10, "1", "1")
169+
g.Update(20, "2", "2")
170+
171+
// Test Inc operation
172+
g.Inc(5, "1", "1")
173+
g.Dec(3, "2", "2")
174+
175+
testFile := "HighCardinalityGauge.txt"
176+
echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, testFile))
177+
}
Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,5 @@
1+
echo
2+
----
3+
foo_gauge 32
4+
foo_gauge{database="1",application_name="1"} 15
5+
foo_gauge{database="2",application_name="2"} 17
Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,13 @@
1+
echo
2+
----
3+
foo_gauge 145
4+
foo_gauge{database="1",application_name="10"} 11
5+
foo_gauge{database="1",application_name="11"} 12
6+
foo_gauge{database="1",application_name="12"} 13
7+
foo_gauge{database="1",application_name="13"} 14
8+
foo_gauge{database="1",application_name="14"} 15
9+
foo_gauge{database="2",application_name="10"} 5
10+
foo_gauge{database="2",application_name="11"} 5
11+
foo_gauge{database="2",application_name="12"} 5
12+
foo_gauge{database="2",application_name="13"} 5
13+
foo_gauge{database="2",application_name="14"} 5
Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,18 @@
1+
echo
2+
----
3+
foo_gauge 120
4+
foo_gauge{database="1",application_name="0"} 1
5+
foo_gauge{database="1",application_name="1"} 2
6+
foo_gauge{database="1",application_name="10"} 11
7+
foo_gauge{database="1",application_name="11"} 12
8+
foo_gauge{database="1",application_name="12"} 13
9+
foo_gauge{database="1",application_name="13"} 14
10+
foo_gauge{database="1",application_name="14"} 15
11+
foo_gauge{database="1",application_name="2"} 3
12+
foo_gauge{database="1",application_name="3"} 4
13+
foo_gauge{database="1",application_name="4"} 5
14+
foo_gauge{database="1",application_name="5"} 6
15+
foo_gauge{database="1",application_name="6"} 7
16+
foo_gauge{database="1",application_name="7"} 8
17+
foo_gauge{database="1",application_name="8"} 9
18+
foo_gauge{database="1",application_name="9"} 10

0 commit comments

Comments
 (0)