Skip to content

Commit a1da8ae

Browse files
craig[bot]aa-joshi
andcommitted
Merge #144377
144377: metrics: add SQLCounter to support high cardinality metrics r=aa-joshi a=aa-joshi This patch introduces `SQLCounter` which is an aggregation counter of `SQLChildCounter` metrics. SQLCounter supports combination of `database` and `application_name` labels. The SQLChildCounter stores the value of a counter for a given combination of database and application name. SQLCounter internally uses cache.UnorderedCache to store child metrics with default size of 5000. SQLCounter will report to crdb-internal time series only the aggregate sum of all its children, while its children are additionally exported to prometheus. SQLCounter differs from AggCounter in that a SQLCounter creates child metrics dynamically while AggCounter needs child creation up front. We have extracted out dynamic child creation from AggCounter in SQLCounter. Epic: CRDB-43153 Part of: CRDB-48489 Release note: None Co-authored-by: Akshay Joshi <[email protected]>
2 parents ae06933 + 174e6be commit a1da8ae

File tree

9 files changed

+237
-70
lines changed

9 files changed

+237
-70
lines changed

pkg/util/metric/aggmetric/agg_metric.go

Lines changed: 14 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,18 @@ import (
2222

2323
var delimiter = []byte{'_'}
2424

25+
const (
26+
dbLabel = "database"
27+
appLabel = "application_name"
28+
)
29+
30+
const (
31+
LabelConfigDisabled = iota
32+
LabelConfigApp
33+
LabelConfigDB
34+
LabelConfigAppAndDB
35+
)
36+
2537
// Builder is used to ease constructing metrics with the same labels.
2638
type Builder struct {
2739
labels []string
@@ -78,22 +90,16 @@ func (cs *childSet) initWithBTreeStorageType(labels []string) {
7890
}
7991
}
8092

81-
func (cs *childSet) initWithCacheStorageType(labels []string) {
82-
// cacheSize is the default number of children that can be stored in the cache.
83-
// If the cache exceeds this size, the oldest children are evicted. This is
84-
// specific to cache storage for children
93+
func getCacheStorage() *cache.UnorderedCache {
8594
const cacheSize = 5000
86-
cs.labels = labels
8795
cacheStorage := cache.NewUnorderedCache(cache.Config{
8896
Policy: cache.CacheLRU,
8997
//TODO (aa-joshi) : make cacheSize configurable in the future
9098
ShouldEvict: func(size int, key, value interface{}) bool {
9199
return size > cacheSize
92100
},
93101
})
94-
cs.mu.children = &UnorderedCacheWrapper{
95-
cache: cacheStorage,
96-
}
102+
return cacheStorage
97103
}
98104

99105
func (cs *childSet) Each(

pkg/util/metric/aggmetric/agg_metric_test.go

Lines changed: 5 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -293,24 +293,23 @@ func TestAggMetricClear(t *testing.T) {
293293
}, "tenant_id")
294294
r.AddMetric(c)
295295

296-
d := NewCounter(metric.Metadata{
296+
d := NewSQLCounter(metric.Metadata{
297297
Name: "bar_counter",
298-
}, "tenant_id")
299-
d.initWithCacheStorageType([]string{"tenant_id"})
298+
})
300299
r.AddMetric(d)
301-
300+
d.labelConfig.Store(LabelConfigAppAndDB)
302301
tenant2 := roachpb.MustMakeTenantID(2)
303302
c1 := c.AddChild(tenant2.String())
304303

305304
t.Run("before clear", func(t *testing.T) {
306305
c1.Inc(2)
307-
d.Inc(2, "3")
306+
d.Inc(2, "test-db", "test-app")
308307
testFile := "aggMetric_pre_clear.txt"
309308
echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, testFile))
310309
})
311310

312311
c.clear()
313-
d.clear()
312+
d.mu.children.Clear()
314313

315314
t.Run("post clear", func(t *testing.T) {
316315
testFile := "aggMetric_post_clear.txt"

pkg/util/metric/aggmetric/counter.go

Lines changed: 186 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -9,7 +9,7 @@ import (
99
"sync/atomic"
1010

1111
"github.com/cockroachdb/cockroach/pkg/util/metric"
12-
"github.com/cockroachdb/errors"
12+
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
1313
"github.com/gogo/protobuf/proto"
1414
io_prometheus_client "github.com/prometheus/client_model/go"
1515
)
@@ -83,29 +83,6 @@ func (c *AggCounter) AddChild(labelVals ...string) *Counter {
8383
return child
8484
}
8585

86-
// Inc increments the counter value by i for the given label values. If a
87-
// counter with the given label values doesn't exist yet, it creates a new
88-
// counter and increments it. Panics if the number of label values doesn't
89-
// match the number of labels defined for this counter and if the storage type
90-
// is not StorageTypeCache.
91-
func (c *AggCounter) Inc(i int64, labelVals ...string) {
92-
if len(c.labels) != len(labelVals) {
93-
panic(errors.AssertionFailedf(
94-
"cannot increment child with %d label values %v to a metric with %d labels %v",
95-
len(labelVals), labelVals, len(c.labels), c.labels))
96-
}
97-
98-
// If the child already exists, increment it.
99-
if child, ok := c.get(labelVals...); ok {
100-
child.(*Counter).Inc(i)
101-
return
102-
}
103-
104-
// Otherwise, create a new child and increment it.
105-
child := c.AddChild(labelVals...)
106-
child.Inc(i)
107-
}
108-
10986
// RemoveChild removes a Gauge from this AggGauge. This method panics if a Gauge
11087
// does not exist for this set of labelVals.
11188
func (g *AggCounter) RemoveChild(labelVals ...string) {
@@ -267,3 +244,188 @@ func (g *CounterFloat64) Inc(i float64) {
267244
func (g *CounterFloat64) UpdateIfHigher(i float64) {
268245
g.value.UpdateIfHigher(i)
269246
}
247+
248+
// SQLCounter maintains a value as the sum of its children. The counter will
249+
// report to crdb-internal time series only the aggregate sum of all of its
250+
// children, while its children are additionally exported to prometheus via the
251+
// PrometheusIterable interface. SQLCounter differs from AggCounter in that
252+
// a SQLCounter creates child metrics dynamically while AggCounter needs the
253+
// child creation up front.
254+
type SQLCounter struct {
255+
g metric.Counter
256+
labelConfig atomic.Uint64
257+
mu struct {
258+
syncutil.Mutex
259+
children ChildrenStorage
260+
}
261+
}
262+
263+
func (c *SQLCounter) Each(
264+
labels []*io_prometheus_client.LabelPair, f func(metric *io_prometheus_client.Metric),
265+
) {
266+
c.mu.Lock()
267+
defer c.mu.Unlock()
268+
269+
c.mu.children.Do(func(e interface{}) {
270+
cm := c.mu.children.GetChildMetric(e)
271+
pm := cm.ToPrometheusMetric()
272+
273+
childLabels := make([]*io_prometheus_client.LabelPair, 0, len(labels)+2)
274+
childLabels = append(childLabels, labels...)
275+
lvs := cm.labelValues()
276+
dbLabel := dbLabel
277+
appLabel := appLabel
278+
switch c.labelConfig.Load() {
279+
case LabelConfigDB:
280+
childLabels = append(childLabels, &io_prometheus_client.LabelPair{
281+
Name: &dbLabel,
282+
Value: &lvs[0],
283+
})
284+
case LabelConfigApp:
285+
childLabels = append(childLabels, &io_prometheus_client.LabelPair{
286+
Name: &appLabel,
287+
Value: &lvs[0],
288+
})
289+
case LabelConfigAppAndDB:
290+
childLabels = append(childLabels, &io_prometheus_client.LabelPair{
291+
Name: &dbLabel,
292+
Value: &lvs[0],
293+
})
294+
childLabels = append(childLabels, &io_prometheus_client.LabelPair{
295+
Name: &appLabel,
296+
Value: &lvs[1],
297+
})
298+
default:
299+
}
300+
pm.Label = childLabels
301+
f(pm)
302+
})
303+
}
304+
305+
var _ metric.Iterable = (*SQLCounter)(nil)
306+
var _ metric.PrometheusIterable = (*SQLCounter)(nil)
307+
var _ metric.PrometheusExportable = (*SQLCounter)(nil)
308+
309+
// NewSQLCounter constructs a new SQLCounter.
310+
func NewSQLCounter(metadata metric.Metadata) *SQLCounter {
311+
c := &SQLCounter{
312+
g: *metric.NewCounter(metadata),
313+
}
314+
c.mu.children = &UnorderedCacheWrapper{
315+
cache: getCacheStorage(),
316+
}
317+
c.labelConfig.Store(LabelConfigDisabled)
318+
return c
319+
}
320+
321+
// getOrAddChild returns the child metric for the given label values. If the child
322+
// doesn't exist, it creates a new one and adds it to the collection.
323+
func (c *SQLCounter) getOrAddChild(labelValues ...string) ChildMetric {
324+
c.mu.Lock()
325+
defer c.mu.Unlock()
326+
327+
// If the child already exists, return it.
328+
if child, ok := c.mu.children.Get(labelValues...); ok {
329+
return child
330+
}
331+
332+
// Otherwise, create a new child and return it.
333+
child := &SQLChildCounter{
334+
labelValuesSlice: labelValuesSlice(labelValues),
335+
}
336+
c.mu.children.Add(child)
337+
return child
338+
}
339+
340+
// GetType is part of the metric.PrometheusExportable interface.
341+
func (c *SQLCounter) GetType() *io_prometheus_client.MetricType {
342+
return c.g.GetType()
343+
}
344+
345+
// GetLabels is part of the metric.PrometheusExportable interface.
346+
func (c *SQLCounter) GetLabels(useStaticLabels bool) []*io_prometheus_client.LabelPair {
347+
return c.g.GetLabels(useStaticLabels)
348+
}
349+
350+
// ToPrometheusMetric is part of the metric.PrometheusExportable interface.
351+
func (c *SQLCounter) ToPrometheusMetric() *io_prometheus_client.Metric {
352+
return c.g.ToPrometheusMetric()
353+
}
354+
355+
// GetName is part of the metric.Iterable interface.
356+
func (c *SQLCounter) GetName(useStaticLabels bool) string {
357+
return c.g.GetName(useStaticLabels)
358+
}
359+
360+
// GetHelp is part of the metric.Iterable interface.
361+
func (c *SQLCounter) GetHelp() string {
362+
return c.g.GetHelp()
363+
}
364+
365+
// GetMeasurement is part of the metric.Iterable interface.
366+
func (c *SQLCounter) GetMeasurement() string {
367+
return c.g.GetMeasurement()
368+
}
369+
370+
// GetUnit is part of the metric.Iterable interface.
371+
func (c *SQLCounter) GetUnit() metric.Unit {
372+
return c.g.GetUnit()
373+
}
374+
375+
// GetMetadata is part of the metric.Iterable interface.
376+
func (c *SQLCounter) GetMetadata() metric.Metadata {
377+
return c.g.GetMetadata()
378+
}
379+
380+
// Inspect is part of the metric.Iterable interface.
381+
func (c *SQLCounter) Inspect(f func(interface{})) {
382+
f(c)
383+
}
384+
385+
// Inc increments the counter value by i for the given label values. If a
386+
// counter with the given label values doesn't exist yet, it creates a new
387+
// counter based on labelConfig and increments it. Inc increments parent metrics
388+
// irrespective of labelConfig.
389+
func (c *SQLCounter) Inc(i int64, db, app string) {
390+
c.g.Inc(i)
391+
392+
var childMetric ChildMetric
393+
switch c.labelConfig.Load() {
394+
case LabelConfigDB:
395+
childMetric = c.getOrAddChild(db)
396+
case LabelConfigApp:
397+
childMetric = c.getOrAddChild(app)
398+
case LabelConfigAppAndDB:
399+
childMetric = c.getOrAddChild(db, app)
400+
default:
401+
return
402+
}
403+
childMetric.(*SQLChildCounter).Inc(i)
404+
}
405+
406+
// SQLChildCounter is a child of a SQLCounter. When metrics are collected by prometheus,
407+
// each of the children will appear with a distinct label, however, when cockroach
408+
// internally collects metrics, only the parent is collected.
409+
type SQLChildCounter struct {
410+
labelValuesSlice
411+
value metric.Counter
412+
}
413+
414+
// ToPrometheusMetric constructs a prometheus metric for this Counter.
415+
func (s *SQLChildCounter) ToPrometheusMetric() *io_prometheus_client.Metric {
416+
return &io_prometheus_client.Metric{
417+
Counter: &io_prometheus_client.Counter{
418+
Value: proto.Float64(float64(s.Value())),
419+
},
420+
}
421+
}
422+
423+
// Value returns the SQLChildCounter's current value.
424+
func (s *SQLChildCounter) Value() int64 {
425+
return s.value.Count()
426+
}
427+
428+
// Inc increments the SQLChildCounter's value.
429+
func (s *SQLChildCounter) Inc(i int64) {
430+
s.value.Inc(i)
431+
}

pkg/util/metric/aggmetric/counter_test.go

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -43,11 +43,11 @@ func TestAggCounter(t *testing.T) {
4343
return strings.Join(lines, "\n")
4444
}
4545

46-
c := NewCounter(metric.Metadata{
46+
c := NewSQLCounter(metric.Metadata{
4747
Name: "foo_counter",
48-
}, "tenant_id", "counter_label")
48+
})
49+
c.labelConfig.Store(LabelConfigAppAndDB)
4950
r.AddMetric(c)
50-
c.initWithCacheStorageType([]string{"tenant_id", "counter_label"})
5151
cacheStorage := cache.NewUnorderedCache(cache.Config{
5252
Policy: cache.CacheLRU,
5353
ShouldEvict: func(size int, key, value interface{}) bool {
@@ -62,13 +62,13 @@ func TestAggCounter(t *testing.T) {
6262
c.Inc(1, "1", strconv.Itoa(i))
6363
}
6464

65-
testFile := "aggCounter_pre_eviction.txt"
65+
testFile := "SQLCounter_pre_eviction.txt"
6666
echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, testFile))
6767

6868
for i := 0 + cacheSize; i < cacheSize+5; i++ {
6969
c.Inc(1, "2", strconv.Itoa(i))
7070
}
7171

72-
testFile = "aggCounter_post_eviction.txt"
72+
testFile = "SQLCounter_post_eviction.txt"
7373
echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, testFile))
7474
}
Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,13 @@
1+
echo
2+
----
3+
foo_counter 15
4+
foo_counter{database="1",application_name="5"} 1
5+
foo_counter{database="1",application_name="6"} 1
6+
foo_counter{database="1",application_name="7"} 1
7+
foo_counter{database="1",application_name="8"} 1
8+
foo_counter{database="1",application_name="9"} 1
9+
foo_counter{database="2",application_name="10"} 1
10+
foo_counter{database="2",application_name="11"} 1
11+
foo_counter{database="2",application_name="12"} 1
12+
foo_counter{database="2",application_name="13"} 1
13+
foo_counter{database="2",application_name="14"} 1
Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,13 @@
1+
echo
2+
----
3+
foo_counter 10
4+
foo_counter{database="1",application_name="0"} 1
5+
foo_counter{database="1",application_name="1"} 1
6+
foo_counter{database="1",application_name="2"} 1
7+
foo_counter{database="1",application_name="3"} 1
8+
foo_counter{database="1",application_name="4"} 1
9+
foo_counter{database="1",application_name="5"} 1
10+
foo_counter{database="1",application_name="6"} 1
11+
foo_counter{database="1",application_name="7"} 1
12+
foo_counter{database="1",application_name="8"} 1
13+
foo_counter{database="1",application_name="9"} 1

pkg/util/metric/aggmetric/testdata/aggCounter_post_eviction.txt

Lines changed: 0 additions & 13 deletions
This file was deleted.

pkg/util/metric/aggmetric/testdata/aggCounter_pre_eviction.txt

Lines changed: 0 additions & 13 deletions
This file was deleted.
Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,6 @@
11
echo
22
----
33
bar_counter 2
4-
bar_counter{tenant_id="3"} 2
4+
bar_counter{database="test-db",application_name="test-app"} 2
55
foo_counter 2
66
foo_counter{tenant_id="2"} 2

0 commit comments

Comments
 (0)