Skip to content

Commit 21d5eb0

Browse files
craig[bot]aa-joshi
craig[bot]
andcommitted
Merge #147486
147486: aggmetric: acquire lock in label config evaluation in SQLMetric r=aa-joshi a=aa-joshi Previously, We are evaluating label config and then accordingly passing label values to `getOrAddChild` method in SQLMetric. `getOrAddChild` method acquires lock and then get/add child. This is inadequate because we are evaluating label config before invoking `getOrAddChild`. This resulted in below issue get/add child is happening inside the lock: P0,T0: initialise metrics with labelConfig as `LabelConfigApp`. P0,T1: increment SQL counter with 1 is invoked. P0,T2: `getChildByLabelConfig` method evaluates labelConfig as `LabelConfigApp`. P0,T3: invokes `getOrAddChild` method with just app as parameter. P1,T4: `ReinitialiseChildMetrics` acquires the lock, clears existing child metrics, updates labelConfig as `LabelConfigAppAndDB` and release lock. P0,T5: `getOrAddChild` acquires the lock, inserts the new child (c1) with app and release lock. P2,T6: scrape metrics invokes `Each` method inside the lock. It expects 2 label values for the child as latest labelConfig is LabelConfigAppAndDB and tries to fetch 2 label values app and db. However, child c1 has single value (app) which will throw an error. It is happening because methods on SQLMetric expects length of `labelValuesSlice` of `ChildMetric` should match according to `labelConfig` value. This contract is broken in `getOrAddChild` as we don't lock the metric object with its children map during modification of `labelConfig`. This is reflected in `Each`'s implementation, where the code assumes that every child's `labelValuesSlice` has a length consistent with the parent's `labelConfig`. To address this, this patch makes sure that we are evaluating LabelConfig and get/add child metric inside the same lock. Epic: None Fixes: #147475 Release note (bug fix): Concurrent invocation of child metric updates and metric reinitialisation will not result in error during scrape. Co-authored-by: Akshay Joshi <[email protected]>
2 parents 4f24997 + 4517659 commit 21d5eb0

File tree

5 files changed

+60
-27
lines changed

5 files changed

+60
-27
lines changed

pkg/util/metric/aggmetric/agg_metric.go

Lines changed: 27 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -11,7 +11,6 @@ package aggmetric
1111
import (
1212
"hash/fnv"
1313
"strings"
14-
"sync/atomic"
1514

1615
"github.com/cockroachdb/cockroach/pkg/util/cache"
1716
"github.com/cockroachdb/cockroach/pkg/util/metric"
@@ -175,16 +174,16 @@ func (cs *childSet) clear() {
175174
}
176175

177176
type SQLMetric struct {
178-
labelConfig atomic.Uint64
179-
mu struct {
177+
mu struct {
178+
labelConfig metric.LabelConfig
180179
syncutil.Mutex
181180
children ChildrenStorage
182181
}
183182
}
184183

185184
func NewSQLMetric(labelConfig metric.LabelConfig) *SQLMetric {
186185
sm := &SQLMetric{}
187-
sm.labelConfig.Store(uint64(labelConfig))
186+
sm.mu.labelConfig = labelConfig
188187
sm.mu.children = &UnorderedCacheWrapper{
189188
cache: getCacheStorage(),
190189
}
@@ -205,18 +204,18 @@ func (sm *SQLMetric) Each(
205204
lvs := cm.labelValues()
206205
dbLabel := dbLabel
207206
appLabel := appLabel
208-
switch sm.labelConfig.Load() {
209-
case uint64(metric.LabelConfigDB):
207+
switch sm.mu.labelConfig {
208+
case metric.LabelConfigDB:
210209
childLabels = append(childLabels, &io_prometheus_client.LabelPair{
211210
Name: &dbLabel,
212211
Value: &lvs[0],
213212
})
214-
case uint64(metric.LabelConfigApp):
213+
case metric.LabelConfigApp:
215214
childLabels = append(childLabels, &io_prometheus_client.LabelPair{
216215
Name: &appLabel,
217216
Value: &lvs[0],
218217
})
219-
case uint64(metric.LabelConfigAppAndDB):
218+
case metric.LabelConfigAppAndDB:
220219
childLabels = append(childLabels, &io_prometheus_client.LabelPair{
221220
Name: &dbLabel,
222221
Value: &lvs[0],
@@ -242,12 +241,12 @@ func (sm *SQLMetric) add(metric ChildMetric) {
242241

243242
type createChildMetricFunc func(labelValues labelValuesSlice) ChildMetric
244243

245-
// getOrAddChild returns the child metric for the given label values. If the child
244+
// getOrAddChildLocked returns the child metric for the given label values. If the child
246245
// doesn't exist, it creates a new one and adds it to the collection.
247-
func (sm *SQLMetric) getOrAddChild(f createChildMetricFunc, labelValues ...string) ChildMetric {
248-
sm.mu.Lock()
249-
defer sm.mu.Unlock()
250-
246+
// REQUIRES: sm.mu is locked.
247+
func (sm *SQLMetric) getOrAddChildLocked(
248+
f createChildMetricFunc, labelValues ...string,
249+
) ChildMetric {
251250
// If the child already exists, return it.
252251
if child, ok := sm.get(labelValues...); ok {
253252
return child
@@ -266,18 +265,24 @@ func (sm *SQLMetric) getOrAddChild(f createChildMetricFunc, labelValues ...strin
266265
func (sm *SQLMetric) getChildByLabelConfig(
267266
f createChildMetricFunc, db string, app string,
268267
) (ChildMetric, bool) {
268+
// We should acquire the lock before evaluating the label configuration
269+
// and accessing the children storage in a thread-safe manner. We have moved
270+
// the lock acquisition from getOrAddChildLocked to here to fix bug #147475.
271+
sm.mu.Lock()
272+
defer sm.mu.Unlock()
273+
269274
var childMetric ChildMetric
270-
switch sm.labelConfig.Load() {
271-
case uint64(metric.LabelConfigDisabled):
275+
switch sm.mu.labelConfig {
276+
case metric.LabelConfigDisabled:
272277
return nil, false
273-
case uint64(metric.LabelConfigDB):
274-
childMetric = sm.getOrAddChild(f, db)
278+
case metric.LabelConfigDB:
279+
childMetric = sm.getOrAddChildLocked(f, db)
275280
return childMetric, true
276-
case uint64(metric.LabelConfigApp):
277-
childMetric = sm.getOrAddChild(f, app)
281+
case metric.LabelConfigApp:
282+
childMetric = sm.getOrAddChildLocked(f, app)
278283
return childMetric, true
279-
case uint64(metric.LabelConfigAppAndDB):
280-
childMetric = sm.getOrAddChild(f, db, app)
284+
case metric.LabelConfigAppAndDB:
285+
childMetric = sm.getOrAddChildLocked(f, db, app)
281286
return childMetric, true
282287
default:
283288
return nil, false
@@ -290,7 +295,7 @@ func (sm *SQLMetric) ReinitialiseChildMetrics(labelConfig metric.LabelConfig) {
290295
sm.mu.Lock()
291296
defer sm.mu.Unlock()
292297
sm.mu.children.Clear()
293-
sm.labelConfig.Store(uint64(labelConfig))
298+
sm.mu.labelConfig = labelConfig
294299
}
295300

296301
type MetricItem interface {

pkg/util/metric/aggmetric/agg_metric_test.go

Lines changed: 29 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -9,7 +9,9 @@ import (
99
"bufio"
1010
"bytes"
1111
"sort"
12+
"strconv"
1213
"strings"
14+
"sync"
1315
"testing"
1416
"time"
1517

@@ -299,7 +301,7 @@ func TestAggMetricClear(t *testing.T) {
299301
Name: "bar_counter",
300302
})
301303
r.AddMetric(d)
302-
d.labelConfig.Store(uint64(metric.LabelConfigAppAndDB))
304+
d.mu.labelConfig = metric.LabelConfigAppAndDB
303305
tenant2 := roachpb.MustMakeTenantID(2)
304306
c1 := c.AddChild(tenant2.String())
305307

@@ -420,3 +422,29 @@ func TestSQLMetricsReinitialise(t *testing.T) {
420422
})
421423

422424
}
425+
426+
// TestConcurrentUpdatesAndReinitialiseMetric tests that concurrent updates to a metric
427+
// do not cause a panic when the metric is reinitialised and scraped. The test case
428+
// validates the fix for the bug #147475.
429+
func TestConcurrentUpdatesAndReinitialiseMetric(t *testing.T) {
430+
defer leaktest.AfterTest(t)()
431+
r := metric.NewRegistry()
432+
433+
c := NewSQLCounter(metric.Metadata{Name: "test.counter"})
434+
c.ReinitialiseChildMetrics(metric.LabelConfigApp)
435+
r.AddMetric(c)
436+
var wg sync.WaitGroup
437+
for i := 0; i < 100; i++ {
438+
wg.Add(1)
439+
go func() {
440+
c.Inc(1, "test_db"+"_"+strconv.Itoa(i), "test_app"+"_"+strconv.Itoa(i))
441+
wg.Done()
442+
}()
443+
}
444+
c.ReinitialiseChildMetrics(metric.LabelConfigAppAndDB)
445+
wg.Wait()
446+
pe := metric.MakePrometheusExporter()
447+
require.NotPanics(t, func() {
448+
pe.ScrapeRegistry(r, metric.WithIncludeChildMetrics(true), metric.WithIncludeAggregateMetrics(true))
449+
})
450+
}

pkg/util/metric/aggmetric/counter_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -46,7 +46,7 @@ func TestAggCounter(t *testing.T) {
4646
c := NewSQLCounter(metric.Metadata{
4747
Name: "foo_counter",
4848
})
49-
c.labelConfig.Store(uint64(metric.LabelConfigAppAndDB))
49+
c.mu.labelConfig = metric.LabelConfigAppAndDB
5050
r.AddMetric(c)
5151
cacheStorage := cache.NewUnorderedCache(cache.Config{
5252
Policy: cache.CacheLRU,

pkg/util/metric/aggmetric/gauge_test.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ func TestSQLGaugeEviction(t *testing.T) {
3535
g.mu.children = &UnorderedCacheWrapper{
3636
cache: cacheStorage,
3737
}
38-
g.labelConfig.Store(uint64(metric.LabelConfigAppAndDB))
38+
g.mu.labelConfig = metric.LabelConfigAppAndDB
3939

4040
for i := 0; i < cacheSize; i++ {
4141
g.Update(1, "1", strconv.Itoa(i))
@@ -69,7 +69,7 @@ func TestSQLGaugeMethods(t *testing.T) {
6969
cache: cacheStorage,
7070
}
7171
r.AddMetric(g)
72-
g.labelConfig.Store(uint64(metric.LabelConfigAppAndDB))
72+
g.mu.labelConfig = metric.LabelConfigAppAndDB
7373

7474
g.Update(10, "1", "1")
7575
g.Update(10, "2", "2")

pkg/util/metric/aggmetric/histogram_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -63,7 +63,7 @@ func TestSQLHistogram(t *testing.T) {
6363
h.mu.children = &UnorderedCacheWrapper{
6464
cache: cacheStorage,
6565
}
66-
h.labelConfig.Store(uint64(metric.LabelConfigAppAndDB))
66+
h.mu.labelConfig = metric.LabelConfigAppAndDB
6767

6868
for i := 0; i < cacheSize; i++ {
6969
h.RecordValue(1, "1", strconv.Itoa(i))

0 commit comments

Comments
 (0)