Skip to content

Commit 7260245

Browse files
committed
pkg/sql: export sql.aggregated_livebytes metric for out-of-process tenants
Previously, in order to obtain livebytes metrics for tenants, one would need to query such values via the KV servers, and this can be problematic if we only have access to just the SQL servers. For example, in CockroachDB Cloud, only metrics from the SQL servers are exported to end-users, and is done so directly from the cockroachdb process. It is not trivial to export an additional subset of metrics from the KV servers filtered by tenant ID. To address that, this commit exposes livebytes for tenants directly via an aggregated metric on the SQL nodes. The aggregated metric will be updated every 60 seconds by default, and will be exported via the existing MVCC statistics update job. Unlike other job metrics where metrics are registered at initialization time and stays forever, this aggregated metric is tied to the lifespan of the job (i.e. it is only exported if the job is running, and unexported otherwise). This feature is scoped to standalone SQL servers only, which at this point of writing, is only supported in CockroachDB Cloud. If we wanted to backport this into 23.2, it should be straightforward as well since the permanent upgrade to insert the job is already in release-23.2. Fixes: #119139 Epic: none Release note (sql change): Out-of-process SQL servers will start exporting a new sql.aggregated_livebytes metric. This metric gets updated once every 60 seconds by default, and its update interval can be configured via the `tenant_global_metrics_exporter_interval` cluster setting.
1 parent 25f20a9 commit 7260245

File tree

7 files changed

+425
-13
lines changed

7 files changed

+425
-13
lines changed

pkg/jobs/registry.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -321,6 +321,8 @@ const (
321321
// SqlActivityUpdaterJobID A static job ID is used for the SQL activity tables.
322322
SqlActivityUpdaterJobID = jobspb.JobID(103)
323323

324+
// MVCCStatisticsJobID A static job ID used for the MVCC statistics update
325+
// job.
324326
MVCCStatisticsJobID = jobspb.JobID(104)
325327
)
326328

pkg/server/server_sql.go

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2087,3 +2087,8 @@ func (s *SQLServer) ExecutorConfig() *sql.ExecutorConfig {
20872087
func (s *SQLServer) InternalExecutor() isql.Executor {
20882088
return s.internalExecutor
20892089
}
2090+
2091+
// MetricsRegistry returns the application-level metrics registry.
2092+
func (s *SQLServer) MetricsRegistry() *metric.Registry {
2093+
return s.metricsRegistry
2094+
}

pkg/server/status/recorder.go

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -223,6 +223,13 @@ func (mr *MetricsRecorder) RemoveTenantRegistry(tenantID roachpb.TenantID) {
223223
delete(mr.mu.tenantRegistries, tenantID)
224224
}
225225

226+
// AppRegistry returns the metric registry for application-level metrics.
227+
func (mr *MetricsRecorder) AppRegistry() *metric.Registry {
228+
mr.mu.Lock()
229+
defer mr.mu.Unlock()
230+
return mr.mu.appRegistry
231+
}
232+
226233
// AddNode adds various metric registries an initialized server, along
227234
// with its descriptor and start time.
228235
// The registries are:

pkg/sql/BUILD.bazel

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -673,6 +673,7 @@ go_test(
673673
"multitenant_admin_function_test.go",
674674
"mutation_test.go",
675675
"mvcc_backfiller_test.go",
676+
"mvcc_statistics_update_job_test.go",
676677
"normalization_test.go",
677678
"pg_metadata_test.go",
678679
"pg_oid_test.go",
@@ -788,6 +789,7 @@ go_test(
788789
"//pkg/server/serverpb",
789790
"//pkg/server/settingswatcher",
790791
"//pkg/server/srvtestutils",
792+
"//pkg/server/status",
791793
"//pkg/server/status/statuspb",
792794
"//pkg/server/telemetry",
793795
"//pkg/settings",
@@ -884,6 +886,7 @@ go_test(
884886
"//pkg/testutils/skip",
885887
"//pkg/testutils/sqlutils",
886888
"//pkg/testutils/testcluster",
889+
"//pkg/ts",
887890
"//pkg/upgrade/upgradebase",
888891
"//pkg/util",
889892
"//pkg/util/admission",
@@ -936,6 +939,7 @@ go_test(
936939
"@com_github_lib_pq//oid",
937940
"@com_github_petermattis_goid//:goid",
938941
"@com_github_pmezard_go_difflib//difflib",
942+
"@com_github_prometheus_common//expfmt",
939943
"@com_github_stretchr_testify//assert",
940944
"@com_github_stretchr_testify//require",
941945
"@in_gopkg_yaml_v2//:yaml_v2",

pkg/sql/exec_util.go

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1177,10 +1177,11 @@ type NodeInfo struct {
11771177
PGURL func(*url.Userinfo) (*pgurl.URL, error)
11781178
}
11791179

1180-
// nodeStatusGenerator is a limited portion of the status.MetricsRecorder
1180+
// limitedMetricsRecorder is a limited portion of the status.MetricsRecorder
11811181
// struct, to avoid having to import all of status in sql.
1182-
type nodeStatusGenerator interface {
1182+
type limitedMetricsRecorder interface {
11831183
GenerateNodeStatus(ctx context.Context) *statuspb.NodeStatus
1184+
AppRegistry() *metric.Registry
11841185
}
11851186

11861187
// SystemTenantOnly wraps an object in the ExecutorConfig that is only
@@ -1249,7 +1250,7 @@ type ExecutorConfig struct {
12491250
// available when not running as a system tenant.
12501251
SQLStatusServer serverpb.SQLStatusServer
12511252
TenantStatusServer serverpb.TenantStatusServer
1252-
MetricsRecorder nodeStatusGenerator
1253+
MetricsRecorder limitedMetricsRecorder
12531254
SessionRegistry *SessionRegistry
12541255
ClosedSessionCache *ClosedSessionCache
12551256
SQLLiveness sqlliveness.Provider

pkg/sql/mvcc_statistics_update_job.go

Lines changed: 120 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -17,40 +17,141 @@ package sql
1717

1818
import (
1919
"context"
20+
"time"
2021

2122
"github.com/cockroachdb/cockroach/pkg/jobs"
2223
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
24+
"github.com/cockroachdb/cockroach/pkg/roachpb"
25+
"github.com/cockroachdb/cockroach/pkg/settings"
2326
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
2427
"github.com/cockroachdb/cockroach/pkg/util/log"
28+
"github.com/cockroachdb/cockroach/pkg/util/metric"
29+
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
2530
"github.com/cockroachdb/errors"
2631
)
2732

33+
// TenantGlobalMetricsExporterInterval is the interval at which an external
34+
// tenant's process in the cluster will update the global metrics. This is
35+
// exported for testing purposes.
36+
var TenantGlobalMetricsExporterInterval = settings.RegisterDurationSetting(
37+
settings.ApplicationLevel,
38+
"tenant_global_metrics_exporter_interval",
39+
"the interval at which a node in the cluster will update the exported global metrics",
40+
60*time.Second,
41+
settings.PositiveDuration,
42+
)
43+
44+
// mvccStatisticsUpdateJob is a singleton job that is meant to update MVCC
45+
// statistics. Historically, this was added to update system.mvcc_statistics,
46+
// but the project was deprioritized. Currently, this is used by external
47+
// process tenants to export global metrics periodically. For such metrics,
48+
// they will only be present on a SQL node if the job is running. Once the job
49+
// stops, the metrics will be removed from the metric registry.
2850
type mvccStatisticsUpdateJob struct {
2951
job *jobs.Job
52+
st *cluster.Settings
53+
54+
// dynamicMetrics keep track of metrics which are added/removed dynamically
55+
// as the job runs. Unlike regular job metrics (i.e. WithJobMetrics), which
56+
// are registered when the job starts the first time, and never removed from
57+
// the metric registry, metrics in this list should be removed when the job
58+
// is not running.
59+
dynamicMetrics struct {
60+
livebytes *metric.Gauge
61+
}
3062
}
3163

3264
var _ jobs.Resumer = (*mvccStatisticsUpdateJob)(nil)
3365

34-
func (j *mvccStatisticsUpdateJob) Resume(ctx context.Context, execCtxI interface{}) (jobErr error) {
66+
// Resume implements the jobs.Resumer interface.
67+
func (j *mvccStatisticsUpdateJob) Resume(ctx context.Context, execCtxI interface{}) error {
68+
log.Infof(ctx, "starting mvcc statistics update job")
69+
70+
// This job is a forever running background job, and it is always safe to
71+
// terminate the SQL pod whenever the job is running, so mark it as idle.
72+
j.job.MarkIdle(true)
73+
74+
execCtx := execCtxI.(JobExecContext)
75+
76+
// Export global metrics for tenants if this is an out-of-process SQL node.
77+
// All external mode tenant servers have no node IDs.
78+
if _, hasNodeID := execCtx.ExecCfg().NodeInfo.NodeID.OptionalNodeID(); !hasNodeID {
79+
return j.runTenantGlobalMetricsExporter(ctx, execCtx)
80+
}
81+
3582
// TODO(zachlite):
3683
// Delete samples older than configurable setting...
3784
// Collect span stats for tenant descriptors...
3885
// Write new samples...
39-
execCtx := execCtxI.(JobExecContext)
40-
stopper := execCtx.ExecCfg().DistSQLSrv.Stopper
41-
j.job.MarkIdle(true)
86+
87+
// Block until context is cancelled since there's nothing that needs to be
88+
// done here. We should not return nil, or else the job will be marked as
89+
// succeeded.
90+
<-ctx.Done()
91+
return ctx.Err()
92+
}
93+
94+
// runTenantGlobalMetricsExporter executes the logic to export global metrics
95+
// for tenants.
96+
func (j *mvccStatisticsUpdateJob) runTenantGlobalMetricsExporter(
97+
ctx context.Context, execCtx JobExecContext,
98+
) error {
99+
metricsRegistry := execCtx.ExecCfg().MetricsRecorder.AppRegistry()
100+
101+
initialRun := true
102+
defer func() {
103+
metricsRegistry.RemoveMetric(j.dynamicMetrics.livebytes)
104+
}()
105+
106+
runTask := func() error {
107+
resp, err := execCtx.ExecCfg().TenantStatusServer.SpanStats(
108+
ctx,
109+
&roachpb.SpanStatsRequest{
110+
// Fan out to all nodes. SpanStats takes care of only contacting
111+
// the relevant nodes with the tenant's span.
112+
NodeID: "0",
113+
Spans: []roachpb.Span{execCtx.ExecCfg().Codec.TenantSpan()},
114+
},
115+
)
116+
if err != nil {
117+
return err
118+
}
119+
var total int64
120+
for _, stats := range resp.SpanToStats {
121+
total += stats.ApproximateTotalStats.LiveBytes
122+
}
123+
j.dynamicMetrics.livebytes.Update(total)
124+
125+
// Only register metrics once we get our initial values. This avoids
126+
// metrics from fluctuating whenever the job restarts.
127+
if initialRun {
128+
metricsRegistry.AddMetric(j.dynamicMetrics.livebytes)
129+
initialRun = false
130+
}
131+
return nil
132+
}
133+
134+
timer := timeutil.NewTimer()
135+
defer timer.Stop()
136+
137+
// Fire the timer immediately to start the initial update.
138+
timer.Reset(0)
42139

43140
for {
44141
select {
45142
case <-ctx.Done():
46-
return nil
47-
48-
case <-stopper.ShouldQuiesce():
49-
return nil
143+
return ctx.Err()
144+
case <-timer.C:
145+
timer.Read = true
146+
timer.Reset(TenantGlobalMetricsExporterInterval.Get(&execCtx.ExecCfg().Settings.SV))
147+
if err := runTask(); err != nil {
148+
log.Errorf(ctx, "mvcc statistics update job error: %v", err)
149+
}
50150
}
51151
}
52152
}
53153

154+
// OnFailOrCancel implements the jobs.Resumer interface.
54155
func (j *mvccStatisticsUpdateJob) OnFailOrCancel(
55156
ctx context.Context, _ interface{}, jobErr error,
56157
) error {
@@ -63,14 +164,23 @@ func (j *mvccStatisticsUpdateJob) OnFailOrCancel(
63164
return nil
64165
}
65166

167+
// CollectProfile implements the jobs.Resumer interface.
66168
func (j *mvccStatisticsUpdateJob) CollectProfile(_ context.Context, _ interface{}) error {
67169
return nil
68170
}
69171

70172
func init() {
71-
jobs.RegisterConstructor(jobspb.TypeMVCCStatisticsUpdate,
173+
jobs.RegisterConstructor(
174+
jobspb.TypeMVCCStatisticsUpdate,
72175
func(job *jobs.Job, settings *cluster.Settings) jobs.Resumer {
73-
return &mvccStatisticsUpdateJob{job: job}
176+
exporter := &mvccStatisticsUpdateJob{job: job, st: settings}
177+
exporter.dynamicMetrics.livebytes = metric.NewGauge(metric.Metadata{
178+
Name: "sql.aggregated_livebytes",
179+
Help: "Aggregated number of bytes of live data (keys plus values)",
180+
Measurement: "Storage",
181+
Unit: metric.Unit_BYTES,
182+
})
183+
return exporter
74184
},
75185
jobs.DisablesTenantCostControl,
76186
)

0 commit comments

Comments
 (0)