Skip to content

Commit

Permalink
sql: create system.mvcc_statistics table and update job
Browse files Browse the repository at this point in the history
The system.mvcc_statistics table stores historical mvcc data
for a tenant's SQL objects. It's purpose it to serve mvcc data for a
SQL object quickly - The span stats API is too slow to use in a hot path.
Storing data over time unlocks new use cases like showing a table or
index's accumulated garbage over time.

Indexes have been added for common access use-cases.
- by database_id, table_id, index_id and time.
- by database_id and time.
- by table_id and time.
- by index_id and time.

The MVCCStatisticsUpdate Job is responsible for managing the contents of
the table, decoupled from the read-hotpath.

Both the table and job are baked when a cluster bootstraps itself, or upgrades
itself from a previous version.

Epic: CRDB-25491
Release note: None
  • Loading branch information
Zach Lite committed Sep 29, 2023
1 parent af05d81 commit ef6736b
Show file tree
Hide file tree
Showing 18 changed files with 278 additions and 4 deletions.
12 changes: 12 additions & 0 deletions docs/generated/metrics/metrics.html
Original file line number Diff line number Diff line change
Expand Up @@ -1077,6 +1077,18 @@
<tr><td>APPLICATION</td><td>jobs.migration.resume_completed</td><td>Number of migration jobs which successfully resumed to completion</td><td>jobs</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>jobs.migration.resume_failed</td><td>Number of migration jobs which failed with a non-retriable error</td><td>jobs</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>jobs.migration.resume_retry_error</td><td>Number of migration jobs which failed with a retriable error</td><td>jobs</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>jobs.mvcc_statistics_update.currently_idle</td><td>Number of mvcc_statistics_update jobs currently considered Idle and can be freely shut down</td><td>jobs</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>APPLICATION</td><td>jobs.mvcc_statistics_update.currently_paused</td><td>Number of mvcc_statistics_update jobs currently considered Paused</td><td>jobs</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>APPLICATION</td><td>jobs.mvcc_statistics_update.currently_running</td><td>Number of mvcc_statistics_update jobs currently running in Resume or OnFailOrCancel state</td><td>jobs</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>APPLICATION</td><td>jobs.mvcc_statistics_update.expired_pts_records</td><td>Number of expired protected timestamp records owned by mvcc_statistics_update jobs</td><td>records</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>jobs.mvcc_statistics_update.fail_or_cancel_completed</td><td>Number of mvcc_statistics_update jobs which successfully completed their failure or cancelation process</td><td>jobs</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>jobs.mvcc_statistics_update.fail_or_cancel_failed</td><td>Number of mvcc_statistics_update jobs which failed with a non-retriable error on their failure or cancelation process</td><td>jobs</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>jobs.mvcc_statistics_update.fail_or_cancel_retry_error</td><td>Number of mvcc_statistics_update jobs which failed with a retriable error on their failure or cancelation process</td><td>jobs</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>jobs.mvcc_statistics_update.protected_age_sec</td><td>The age of the oldest PTS record protected by mvcc_statistics_update jobs</td><td>seconds</td><td>GAUGE</td><td>SECONDS</td><td>AVG</td><td>NONE</td></tr>
<tr><td>APPLICATION</td><td>jobs.mvcc_statistics_update.protected_record_count</td><td>Number of protected timestamp records held by mvcc_statistics_update jobs</td><td>records</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>APPLICATION</td><td>jobs.mvcc_statistics_update.resume_completed</td><td>Number of mvcc_statistics_update jobs which successfully resumed to completion</td><td>jobs</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>jobs.mvcc_statistics_update.resume_failed</td><td>Number of mvcc_statistics_update jobs which failed with a non-retriable error</td><td>jobs</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>jobs.mvcc_statistics_update.resume_retry_error</td><td>Number of mvcc_statistics_update jobs which failed with a retriable error</td><td>jobs</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>APPLICATION</td><td>jobs.new_schema_change.currently_idle</td><td>Number of new_schema_change jobs currently considered Idle and can be freely shut down</td><td>jobs</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>APPLICATION</td><td>jobs.new_schema_change.currently_paused</td><td>Number of new_schema_change jobs currently considered Paused</td><td>jobs</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>APPLICATION</td><td>jobs.new_schema_change.currently_running</td><td>Number of new_schema_change jobs currently running in Resume or OnFailOrCancel state</td><td>jobs</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
Expand Down
2 changes: 1 addition & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -320,4 +320,4 @@ trace.snapshot.rate duration 0s if non-zero, interval at which background trace
trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https://<ui>/#/debug/tracez tenant-rw
trace.zipkin.collector string the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used. tenant-rw
ui.display_timezone enumeration etc/utc the timezone used to format timestamps in the ui [etc/utc = 0, america/new_york = 1] tenant-rw
version version 1000023.1-26 set the active cluster version in the format '<major>.<minor>' tenant-rw
version version 1000023.1-28 set the active cluster version in the format '<major>.<minor>' tenant-rw
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -268,6 +268,6 @@
<tr><td><div id="setting-trace-span-registry-enabled" class="anchored"><code>trace.span_registry.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>if set, ongoing traces can be seen at https://&lt;ui&gt;/#/debug/tracez</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-trace-zipkin-collector" class="anchored"><code>trace.zipkin.collector</code></div></td><td>string</td><td><code></code></td><td>the address of a Zipkin instance to receive traces, as &lt;host&gt;:&lt;port&gt;. If no port is specified, 9411 will be used.</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-ui-display-timezone" class="anchored"><code>ui.display_timezone</code></div></td><td>enumeration</td><td><code>etc/utc</code></td><td>the timezone used to format timestamps in the ui [etc/utc = 0, america/new_york = 1]</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-version" class="anchored"><code>version</code></div></td><td>version</td><td><code>1000023.1-26</code></td><td>set the active cluster version in the format &#39;&lt;major&gt;.&lt;minor&gt;&#39;</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-version" class="anchored"><code>version</code></div></td><td>version</td><td><code>1000023.1-28</code></td><td>set the active cluster version in the format &#39;&lt;major&gt;.&lt;minor&gt;&#39;</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
</tbody>
</table>
3 changes: 3 additions & 0 deletions pkg/ccl/backupccl/system_schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -818,6 +818,9 @@ var systemTableBackupConfiguration = map[string]systemBackupConfiguration{
systemschema.RegionLivenessTable.GetName(): {
shouldIncludeInClusterBackup: optOutOfClusterBackup,
},
systemschema.SystemMVCCStatisticsTable.GetName(): {
shouldIncludeInClusterBackup: optOutOfClusterBackup,
},
}

func rekeySystemTable(
Expand Down
10 changes: 9 additions & 1 deletion pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -572,6 +572,11 @@ const (
// role for all existing functions.
V23_2_GrantExecuteToPublic

// V23_2_MVCCStatisticsTable adds the system.mvcc_statistics
// table and update job. The table is used to serve fast reads of historical
// mvcc data from observability surfaces.
V23_2_MVCCStatisticsTable

// *************************************************
// Step (1) Add new versions here.
// Do not add new versions to a patch release.
Expand Down Expand Up @@ -1001,7 +1006,10 @@ var rawVersionsSingleton = keyedVersions{
Key: V23_2_GrantExecuteToPublic,
Version: roachpb.Version{Major: 23, Minor: 1, Internal: 26},
},

{
Key: V23_2_MVCCStatisticsTable,
Version: roachpb.Version{Major: 23, Minor: 1, Internal: 28},
},
// *************************************************
// Step (2): Add new versions here.
// Do not add new versions to a patch release.
Expand Down
11 changes: 11 additions & 0 deletions pkg/jobs/jobspb/jobs.proto
Original file line number Diff line number Diff line change
Expand Up @@ -1237,6 +1237,14 @@ message AutoUpdateSQLActivityDetails {
message AutoUpdateSQLActivityProgress {
}

message MVCCStatisticsJobDetails {

}

message MVCCStatisticsJobProgress {

}

message Payload {
string description = 1;
// If empty, the description is assumed to be the statement.
Expand Down Expand Up @@ -1299,6 +1307,7 @@ message Payload {
AutoConfigEnvRunnerDetails auto_config_env_runner = 42;
AutoConfigTaskDetails auto_config_task = 43;
AutoUpdateSQLActivityDetails auto_update_sql_activities = 44;
MVCCStatisticsJobDetails mvcc_statistics_details = 45;
}
reserved 26;
// PauseReason is used to describe the reason that the job is currently paused
Expand Down Expand Up @@ -1372,6 +1381,7 @@ message Progress {
AutoConfigEnvRunnerProgress auto_config_env_runner = 30;
AutoConfigTaskProgress auto_config_task = 31;
AutoUpdateSQLActivityProgress update_sql_activity = 32;
MVCCStatisticsJobProgress mvcc_statistics_progress = 33;
}

uint64 trace_id = 21 [(gogoproto.nullable) = false, (gogoproto.customname) = "TraceID", (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb.TraceID"];
Expand Down Expand Up @@ -1407,6 +1417,7 @@ enum Type {
AUTO_CONFIG_ENV_RUNNER = 21 [(gogoproto.enumvalue_customname) = "TypeAutoConfigEnvRunner"];
AUTO_CONFIG_TASK = 22 [(gogoproto.enumvalue_customname) = "TypeAutoConfigTask"];
AUTO_UPDATE_SQL_ACTIVITY = 23 [(gogoproto.enumvalue_customname) = "TypeAutoUpdateSQLActivity"];
MVCC_STATISTICS_UPDATE = 24 [(gogoproto.enumvalue_customname) = "TypeMVCCStatisticsUpdate"];
}

message Job {
Expand Down
16 changes: 15 additions & 1 deletion pkg/jobs/jobspb/wrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ var (
_ Details = AutoConfigEnvRunnerDetails{}
_ Details = AutoConfigTaskDetails{}
_ Details = AutoUpdateSQLActivityDetails{}
_ Details = MVCCStatisticsJobDetails{}
)

// ProgressDetails is a marker interface for job progress details proto structs.
Expand All @@ -76,6 +77,7 @@ var (
_ ProgressDetails = AutoConfigEnvRunnerProgress{}
_ ProgressDetails = AutoConfigTaskProgress{}
_ ProgressDetails = AutoUpdateSQLActivityProgress{}
_ ProgressDetails = MVCCStatisticsJobProgress{}
)

// Type returns the payload's job type and panics if the type is invalid.
Expand Down Expand Up @@ -159,6 +161,7 @@ var AutomaticJobTypes = [...]Type{
TypeAutoConfigTask,
TypeKeyVisualizer,
TypeAutoUpdateSQLActivity,
TypeMVCCStatisticsUpdate,
}

// DetailsType returns the type for a payload detail.
Expand Down Expand Up @@ -212,6 +215,8 @@ func DetailsType(d isPayload_Details) (Type, error) {
return TypeAutoConfigTask, nil
case *Payload_AutoUpdateSqlActivities:
return TypeAutoUpdateSQLActivity, nil
case *Payload_MvccStatisticsDetails:
return TypeMVCCStatisticsUpdate, nil
default:
return TypeUnspecified, errors.Newf("Payload.Type called on a payload with an unknown details type: %T", d)
}
Expand Down Expand Up @@ -256,6 +261,7 @@ var JobDetailsForEveryJobType = map[Type]Details{
TypeAutoConfigEnvRunner: AutoConfigEnvRunnerDetails{},
TypeAutoConfigTask: AutoConfigTaskDetails{},
TypeAutoUpdateSQLActivity: AutoUpdateSQLActivityDetails{},
TypeMVCCStatisticsUpdate: MVCCStatisticsJobDetails{},
}

// WrapProgressDetails wraps a ProgressDetails object in the protobuf wrapper
Expand Down Expand Up @@ -311,6 +317,8 @@ func WrapProgressDetails(details ProgressDetails) interface {
return &Progress_AutoConfigTask{AutoConfigTask: &d}
case AutoUpdateSQLActivityProgress:
return &Progress_UpdateSqlActivity{UpdateSqlActivity: &d}
case MVCCStatisticsJobProgress:
return &Progress_MvccStatisticsProgress{MvccStatisticsProgress: &d}
default:
panic(errors.AssertionFailedf("WrapProgressDetails: unknown progress type %T", d))
}
Expand Down Expand Up @@ -364,6 +372,8 @@ func (p *Payload) UnwrapDetails() Details {
return *d.AutoConfigTask
case *Payload_AutoUpdateSqlActivities:
return *d.AutoUpdateSqlActivities
case *Payload_MvccStatisticsDetails:
return *d.MvccStatisticsDetails
default:
return nil
}
Expand Down Expand Up @@ -417,6 +427,8 @@ func (p *Progress) UnwrapDetails() ProgressDetails {
return *d.AutoConfigTask
case *Progress_UpdateSqlActivity:
return *d.UpdateSqlActivity
case *Progress_MvccStatisticsProgress:
return *d.MvccStatisticsProgress
default:
return nil
}
Expand Down Expand Up @@ -494,6 +506,8 @@ func WrapPayloadDetails(details Details) interface {
return &Payload_AutoConfigTask{AutoConfigTask: &d}
case AutoUpdateSQLActivityDetails:
return &Payload_AutoUpdateSqlActivities{AutoUpdateSqlActivities: &d}
case MVCCStatisticsJobDetails:
return &Payload_MvccStatisticsDetails{MvccStatisticsDetails: &d}
default:
panic(errors.AssertionFailedf("jobs.WrapPayloadDetails: unknown details type %T", d))
}
Expand Down Expand Up @@ -529,7 +543,7 @@ const (
func (Type) SafeValue() {}

// NumJobTypes is the number of jobs types.
const NumJobTypes = 24
const NumJobTypes = 25

// ChangefeedDetailsMarshaler allows for dependency injection of
// cloud.SanitizeExternalStorageURI to avoid the dependency from this
Expand Down
2 changes: 2 additions & 0 deletions pkg/jobs/registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -322,6 +322,8 @@ const (

// SqlActivityUpdaterJobID A static job ID is used for the SQL activity tables.
SqlActivityUpdaterJobID = jobspb.JobID(103)

MVCCStatisticsJobID = jobspb.JobID(104)
)

// MakeJobID generates a new job ID.
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -151,6 +151,7 @@ go_library(
"max_one_row.go",
"mem_metrics.go",
"mvcc_backfiller.go",
"mvcc_statistics_update_job.go",
"name_util.go",
"notice.go",
"opaque.go",
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/bootstrap/metadata.go
Original file line number Diff line number Diff line change
Expand Up @@ -475,6 +475,7 @@ func addSystemDescriptorsToSchema(target *MetadataSchema) {

// Tables introduced in 23.2.
target.AddDescriptor(systemschema.RegionLivenessTable)
target.AddDescriptor(systemschema.SystemMVCCStatisticsTable)

// Adding a new system table? It should be added here to the metadata schema,
// and also created as a migration for older clusters.
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/catprivilege/system.go
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,7 @@ var (
catconstants.SpanStatsSamples,
catconstants.SpanStatsTenantBoundaries,
catconstants.RegionalLiveness,
catconstants.MVCCStatistics,
}

readWriteSystemSequences = []catconstants.SystemTableName{
Expand Down
78 changes: 78 additions & 0 deletions pkg/sql/catalog/systemschema/system.go
Original file line number Diff line number Diff line change
Expand Up @@ -1007,6 +1007,21 @@ CREATE TABLE system.span_stats_tenant_boundaries (
FAMILY "primary" (crdb_region, unavailable_at)
) ;
`

SystemMVCCStatisticsSchema = `
CREATE TABLE system.mvcc_statistics (
id INT8 NOT NULL DEFAULT unique_rowid(),
database_id INT8 NOT NULL,
table_id INT8 NOT NULL,
index_id INT8 NOT NULL,
created_at TIMESTAMPTZ NOT NULL DEFAULT now():::TIMESTAMP,
statistics JSONB NOT NULL,
CONSTRAINT mvcc_statistics_pkey PRIMARY KEY (id ASC),
INDEX mvcc_statistics_idx_db_table_idx_created_at (database_id ASC, table_id ASC, index_id ASC, created_at ASC),
INDEX mvcc_statistics_idx_database_id_created_at (database_id ASC, created_at ASC),
INDEX mvcc_statistics_idx_table_id_created_at (table_id ASC, created_at ASC),
INDEX mvcc_statistics_idx_index_id_id_created_at (index_id ASC, created_at ASC)
);`
)

func pk(name string) descpb.IndexDescriptor {
Expand Down Expand Up @@ -1237,6 +1252,7 @@ func MakeSystemTables() []SystemTable {
StatementActivityTable,
TransactionActivityTable,
RegionLivenessTable,
SystemMVCCStatisticsTable,
}
}

Expand Down Expand Up @@ -4103,6 +4119,68 @@ var (
},
),
)

SystemMVCCStatisticsTable = makeSystemTable(
SystemMVCCStatisticsSchema,
systemTable(
catconstants.MVCCStatistics,
descpb.InvalidID, // dynamically assigned table ID
[]descpb.ColumnDescriptor{
{Name: "id", ID: 1, Type: types.Int,
DefaultExpr: &uniqueRowIDString},
{Name: "database_id", ID: 2, Type: types.Int},
{Name: "table_id", ID: 3, Type: types.Int},
{Name: "index_id", ID: 4, Type: types.Int},
{Name: "created_at", ID: 5, Type: types.TimestampTZ,
DefaultExpr: &nowTZString},
{Name: "statistics", ID: 6, Type: types.Jsonb},
},
[]descpb.ColumnFamilyDescriptor{
{
Name: "primary",
ID: 0,
ColumnNames: []string{
"id",
"database_id",
"table_id",
"index_id",
"created_at",
"statistics",
},
ColumnIDs: []descpb.ColumnID{1, 2, 3, 4, 5, 6},
},
},
descpb.IndexDescriptor{
Name: "mvcc_statistics_pkey",
ID: 1,
Unique: true,
KeyColumnNames: []string{"id"},
KeyColumnDirections: []catenumpb.IndexColumn_Direction{
catenumpb.IndexColumn_ASC,
},
KeyColumnIDs: []descpb.ColumnID{1},
},
descpb.IndexDescriptor{
Name: "mvcc_statistics_idx_db_table_idx_created_at",
ID: 2,
Unique: false,
KeyColumnNames: []string{
"database_id",
"table_id",
"index_id",
"created_at",
},
KeyColumnDirections: []catenumpb.IndexColumn_Direction{
catenumpb.IndexColumn_ASC,
catenumpb.IndexColumn_ASC,
catenumpb.IndexColumn_ASC,
catenumpb.IndexColumn_ASC,
},
KeyColumnIDs: []descpb.ColumnID{2, 3, 4, 5},
KeySuffixColumnIDs: []descpb.ColumnID{1},
},
),
)
)

// SpanConfigurationsTableName represents system.span_configurations.
Expand Down
68 changes: 68 additions & 0 deletions pkg/sql/mvcc_statistics_update_job.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
// Copyright 2023 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

// Package upgrades contains the implementation of upgrades. It is imported
// by the server library.
//
// This package registers the upgrades with the upgrade package.

package sql

import (
"context"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
)

type mvccStatisticsUpdateJob struct {
job *jobs.Job
}

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

func (j *mvccStatisticsUpdateJob) Resume(
ctx context.Context, execCtxI interface{},
) (jobErr error) {

// TODO(zachlite):
// Delete samples older than configurable setting...
// Collect span stats for tenant descriptors...
// Write new samples...

return nil
}

func (j *mvccStatisticsUpdateJob) OnFailOrCancel(
ctx context.Context, _ interface{}, jobErr error,
) error {
if jobs.HasErrJobCanceled(jobErr) {
err := errors.NewAssertionErrorWithWrappedErrf(
jobErr, "mvcc statistics update job is not cancelable",
)
log.Errorf(ctx, "%v", err)
}
return nil
}

func (j *mvccStatisticsUpdateJob) CollectProfile(_ context.Context, _ interface{}) error {
return nil
}

func init() {
jobs.RegisterConstructor(jobspb.TypeMVCCStatisticsUpdate,
func(job *jobs.Job, settings *cluster.Settings) jobs.Resumer {
return &mvccStatisticsUpdateJob{job: job}
},
jobs.DisablesTenantCostControl,
)
}
Loading

0 comments on commit ef6736b

Please sign in to comment.