Skip to content

Commit c860bad

Browse files
committed
kvserver: absorb tenant metrics ref
This should be simpler overall: there is a single struct that represents the metrics for a tenant on the store.
1 parent cf98810 commit c860bad

File tree

3 files changed

+49
-78
lines changed

3 files changed

+49
-78
lines changed

pkg/kv/kvserver/metrics.go

Lines changed: 41 additions & 67 deletions
Original file line numberDiff line numberDiff line change
@@ -3207,29 +3207,6 @@ type StoreMetrics struct {
32073207
DiskWriteMaxBytesPerSecond *metric.Gauge
32083208
}
32093209

3210-
type tenantMetricsRef struct {
3211-
// All fields are internal. Don't access them.
3212-
3213-
_tenantID roachpb.TenantID
3214-
_state int32 // atomic; 0=usable 1=poisoned
3215-
3216-
// _stack helps diagnose use-after-release when it occurs.
3217-
// This field is populated in releaseTenant and printed
3218-
// in assertions on failure.
3219-
_stack struct {
3220-
syncutil.Mutex
3221-
debugutil.SafeStack
3222-
}
3223-
}
3224-
3225-
func (ref *tenantMetricsRef) assert(ctx context.Context) {
3226-
if atomic.LoadInt32(&ref._state) != 0 {
3227-
ref._stack.Lock()
3228-
defer ref._stack.Unlock()
3229-
log.FatalfDepth(ctx, 1, "tenantMetricsRef already finalized in:\n%s", ref._stack.SafeStack)
3230-
}
3231-
}
3232-
32333210
// TenantsStorageMetrics are metrics which are aggregated over all tenants
32343211
// present on the server. The struct maintains child metrics used by each
32353212
// tenant to track their individual values. The struct expects that children
@@ -3304,7 +3281,7 @@ func (sm *TenantsStorageMetrics) MetricStruct() {}
33043281
// method are reference counted with decrements occurring in the corresponding
33053282
// releaseTenant call. This method must be called prior to adding or subtracting
33063283
// MVCC stats.
3307-
func (sm *TenantsStorageMetrics) acquireTenant(tenantID roachpb.TenantID) *tenantMetricsRef {
3284+
func (sm *TenantsStorageMetrics) acquireTenant(tenantID roachpb.TenantID) *tenantStorageMetrics {
33083285
// incRef increments the reference count if it is not already zero indicating
33093286
// that the struct has already been destroyed.
33103287
incRef := func(m *tenantStorageMetrics) (alreadyDestroyed bool) {
@@ -3319,17 +3296,13 @@ func (sm *TenantsStorageMetrics) acquireTenant(tenantID roachpb.TenantID) *tenan
33193296
for {
33203297
if m, ok := sm.tenants.Load(tenantID); ok {
33213298
if alreadyDestroyed := incRef(m); !alreadyDestroyed {
3322-
return &m.ref
3299+
return m
33233300
}
33243301
// Somebody else concurrently took the reference count to zero, go back
33253302
// around. Because of the locking in releaseTenant, we know that we'll
33263303
// find a different value or no value at all on the next iteration.
33273304
} else {
3328-
m := &tenantStorageMetrics{
3329-
ref: tenantMetricsRef{
3330-
_tenantID: tenantID,
3331-
},
3332-
}
3305+
m := &tenantStorageMetrics{tenantID: tenantID}
33333306
m.mu.Lock()
33343307
_, loaded := sm.tenants.LoadOrStore(tenantID, m)
33353308
if loaded {
@@ -3361,33 +3334,28 @@ func (sm *TenantsStorageMetrics) acquireTenant(tenantID roachpb.TenantID) *tenan
33613334
m.SysCount = sm.SysCount.AddChild(tenantIDStr)
33623335
m.AbortSpanBytes = sm.AbortSpanBytes.AddChild(tenantIDStr)
33633336
m.mu.Unlock()
3364-
return &m.ref
3337+
return m
33653338
}
33663339
}
33673340
}
33683341

33693342
// releaseTenant releases the reference to the metrics for this tenant which was
33703343
// acquired with acquireTenant. It will fatally log if no entry exists for this
33713344
// tenant.
3372-
func (sm *TenantsStorageMetrics) releaseTenant(ctx context.Context, ref *tenantMetricsRef) {
3373-
m := sm.getTenant(ctx, ref)
3374-
3375-
ref._stack.Lock()
3376-
ref._stack.SafeStack = debugutil.Stack()
3377-
ref._stack.Unlock()
3345+
func (sm *TenantsStorageMetrics) releaseTenant(ctx context.Context, m *tenantStorageMetrics) {
33783346
m.mu.Lock()
33793347
defer m.mu.Unlock()
3348+
if m.mu.released.Load() {
3349+
log.FatalfDepth(ctx, 1, "tenant metrics already released in:\n%s", m.mu.stack)
3350+
}
33803351
m.mu.refCount--
3381-
if m.mu.refCount < 0 {
3382-
log.Fatalf(ctx, "invalid refCount on metrics for tenant %v: %d", ref._tenantID, m.mu.refCount)
3383-
} else if m.mu.refCount > 0 {
3352+
if n := m.mu.refCount; n < 0 {
3353+
log.Fatalf(ctx, "invalid refCount on metrics for tenant %v: %d", m.tenantID, n)
3354+
} else if n > 0 {
33843355
return
33853356
}
33863357

3387-
if atomic.SwapInt32(&ref._state, 1) != 0 {
3388-
log.FatalfDepth(ctx, 1, "tenant metrics already released in:\n%s", ref._stack.SafeStack)
3389-
return // unreachable
3390-
}
3358+
m.mu.released.Store(true)
33913359

33923360
// The refCount is zero, delete this instance after destroying its metrics.
33933361
// Note that concurrent attempts to create an instance will detect the zero
@@ -3419,27 +3387,26 @@ func (sm *TenantsStorageMetrics) releaseTenant(ctx context.Context, ref *tenantM
34193387
(*gptr).Unlink()
34203388
*gptr = nil
34213389
}
3422-
sm.tenants.Delete(ref._tenantID)
3423-
}
3424-
3425-
// getTenant is a helper method used to retrieve the metrics for a tenant. The
3426-
// call will log fatally if no such tenant has been previously acquired.
3427-
func (sm *TenantsStorageMetrics) getTenant(
3428-
ctx context.Context, ref *tenantMetricsRef,
3429-
) *tenantStorageMetrics {
3430-
ref.assert(ctx)
3431-
m, ok := sm.tenants.Load(ref._tenantID)
3432-
if !ok {
3433-
log.Fatalf(ctx, "no metrics exist for tenant %v", ref._tenantID)
3434-
}
3435-
return m
3390+
sm.tenants.Delete(m.tenantID)
34363391
}
34373392

3393+
// tenantStorageMetrics is a struct that holds the metrics for all replicas (
3394+
// within a Store) of a given tenant.
3395+
//
3396+
// Whenever it is guaranteed that the replica is not destroyed, the metrics
3397+
// fields can be accessed directly (for example, when holding raftMu and having
3398+
// previously checked the replica's destroyStatus).
3399+
//
3400+
// Whenever this is *not* guaranteed, use `TenantsStorageMetrics.acquireTenant`
3401+
// ( followed by releaseTenant after completion of usage) instead to avoid
3402+
// racing with a potential concurrent attempt to release the metrics.
34383403
type tenantStorageMetrics struct {
3439-
ref tenantMetricsRef
3440-
mu struct {
3404+
tenantID roachpb.TenantID
3405+
mu struct {
34413406
syncutil.Mutex
34423407
refCount int
3408+
released atomic.Bool // allowed to read without holding mu
3409+
stack debugutil.SafeStack
34433410
}
34443411

34453412
LiveBytes *aggmetric.Gauge
@@ -3464,6 +3431,14 @@ type tenantStorageMetrics struct {
34643431
AbortSpanBytes *aggmetric.Gauge
34653432
}
34663433

3434+
func (tm *tenantStorageMetrics) assert(ctx context.Context) {
3435+
if tm.mu.released.Load() {
3436+
tm.mu.Lock()
3437+
defer tm.mu.Unlock()
3438+
log.Fatalf(ctx, "tenant metrics already released in:\n%s", tm.mu.stack)
3439+
}
3440+
}
3441+
34673442
func newTenantsStorageMetrics() *TenantsStorageMetrics {
34683443
b := aggmetric.MakeBuilder(multitenant.TenantIDLabel)
34693444
sm := &TenantsStorageMetrics{
@@ -4023,10 +3998,9 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics {
40233998
// single snapshot of these gauges in the registry might mix the values of two
40243999
// subsequent updates.
40254000
func (sm *TenantsStorageMetrics) incMVCCGauges(
4026-
ctx context.Context, ref *tenantMetricsRef, delta enginepb.MVCCStats,
4001+
ctx context.Context, tm *tenantStorageMetrics, delta enginepb.MVCCStats,
40274002
) {
4028-
ref.assert(ctx)
4029-
tm := sm.getTenant(ctx, ref)
4003+
tm.assert(ctx)
40304004
tm.LiveBytes.Inc(delta.LiveBytes)
40314005
tm.KeyBytes.Inc(delta.KeyBytes)
40324006
tm.ValBytes.Inc(delta.ValBytes)
@@ -4050,17 +4024,17 @@ func (sm *TenantsStorageMetrics) incMVCCGauges(
40504024
}
40514025

40524026
func (sm *TenantsStorageMetrics) addMVCCStats(
4053-
ctx context.Context, ref *tenantMetricsRef, delta enginepb.MVCCStats,
4027+
ctx context.Context, tm *tenantStorageMetrics, delta enginepb.MVCCStats,
40544028
) {
4055-
sm.incMVCCGauges(ctx, ref, delta)
4029+
sm.incMVCCGauges(ctx, tm, delta)
40564030
}
40574031

40584032
func (sm *TenantsStorageMetrics) subtractMVCCStats(
4059-
ctx context.Context, ref *tenantMetricsRef, delta enginepb.MVCCStats,
4033+
ctx context.Context, tm *tenantStorageMetrics, delta enginepb.MVCCStats,
40604034
) {
40614035
var neg enginepb.MVCCStats
40624036
neg.Subtract(delta)
4063-
sm.incMVCCGauges(ctx, ref, neg)
4037+
sm.incMVCCGauges(ctx, tm, neg)
40644038
}
40654039

40664040
func (sm *StoreMetrics) updateEngineMetrics(m storage.Metrics) {

pkg/kv/kvserver/metrics_test.go

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -27,15 +27,14 @@ import (
2727
func TestTenantsStorageMetricsRelease(t *testing.T) {
2828
defer leaktest.AfterTest(t)()
2929
m := newTenantsStorageMetrics()
30-
var refs []*tenantMetricsRef
30+
var refs []*tenantStorageMetrics
3131
const tenants = 7
3232
for i := 0; i < tenants; i++ {
3333
id := roachpb.MustMakeTenantID(roachpb.MinTenantID.InternalValue + uint64(i))
34-
ref := m.acquireTenant(id)
35-
tm := m.getTenant(context.Background(), ref)
34+
tm := m.acquireTenant(id)
3635
tm.SysBytes.Update(1023)
3736
tm.KeyCount.Inc(123)
38-
refs = append(refs, ref)
37+
refs = append(refs, tm)
3938
}
4039
for i, ref := range refs {
4140
require.Equal(t, int64(1023*(tenants-i)), m.SysBytes.Value(), i)

pkg/kv/kvserver/replica.go

Lines changed: 5 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -548,13 +548,11 @@ type Replica struct {
548548
// [^1]: TODO(pavelkalinnikov): we can but it'd be a larger refactor.
549549
tenantLimiter tenantrate.Limiter
550550

551-
// tenantMetricsRef is a metrics reference indicating the tenant under
552-
// which to track the range's contributions. This is determined by the
553-
// start key of the Replica, once initialized.
554-
// Its purpose is to help track down missing/extraneous release operations
555-
// that would not be apparent or easy to resolve when refcounting at the store
556-
// level only.
557-
tenantMetricsRef *tenantMetricsRef
551+
// tenantMetricsRef is a struct for per-tenant metrics contributed by this
552+
// replica. It is determined by the start key of the Replica, once
553+
// initialized. See tenantStorageMetrics for precautions that must be taken
554+
// when accessing this.
555+
tenantMetricsRef *tenantStorageMetrics
558556

559557
// sideTransportClosedTimestamp encapsulates state related to the closed
560558
// timestamp's information about the range. Note that the

0 commit comments

Comments
 (0)