From ef6736b3a3fd164825e6b0ba3ab14e3c07258f12 Mon Sep 17 00:00:00 2001 From: Zach Lite Date: Wed, 27 Sep 2023 14:31:57 -0400 Subject: [PATCH] sql: create system.mvcc_statistics table and update job 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 --- docs/generated/metrics/metrics.html | 12 +++ .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/ccl/backupccl/system_schema.go | 3 + pkg/clusterversion/cockroach_versions.go | 10 ++- pkg/jobs/jobspb/jobs.proto | 11 +++ pkg/jobs/jobspb/wrap.go | 16 +++- pkg/jobs/registry.go | 2 + pkg/sql/BUILD.bazel | 1 + pkg/sql/catalog/bootstrap/metadata.go | 1 + pkg/sql/catalog/catprivilege/system.go | 1 + pkg/sql/catalog/systemschema/system.go | 78 +++++++++++++++++++ pkg/sql/mvcc_statistics_update_job.go | 68 ++++++++++++++++ pkg/sql/sem/catconstants/constants.go | 1 + pkg/upgrade/upgradebase/testing_knobs.go | 2 + pkg/upgrade/upgrades/BUILD.bazel | 2 + .../upgrades/mvcc_statistics_migration.go | 64 +++++++++++++++ pkg/upgrade/upgrades/upgrades.go | 6 ++ 18 files changed, 278 insertions(+), 4 deletions(-) create mode 100644 pkg/sql/mvcc_statistics_update_job.go create mode 100644 pkg/upgrade/upgrades/mvcc_statistics_migration.go diff --git a/docs/generated/metrics/metrics.html b/docs/generated/metrics/metrics.html index d78973d637fc..6b5992882220 100644 --- a/docs/generated/metrics/metrics.html +++ b/docs/generated/metrics/metrics.html @@ -1077,6 +1077,18 @@ APPLICATIONjobs.migration.resume_completedNumber of migration jobs which successfully resumed to completionjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONjobs.migration.resume_failedNumber of migration jobs which failed with a non-retriable errorjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONjobs.migration.resume_retry_errorNumber of migration jobs which failed with a retriable errorjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +APPLICATIONjobs.mvcc_statistics_update.currently_idleNumber of mvcc_statistics_update jobs currently considered Idle and can be freely shut downjobsGAUGECOUNTAVGNONE +APPLICATIONjobs.mvcc_statistics_update.currently_pausedNumber of mvcc_statistics_update jobs currently considered PausedjobsGAUGECOUNTAVGNONE +APPLICATIONjobs.mvcc_statistics_update.currently_runningNumber of mvcc_statistics_update jobs currently running in Resume or OnFailOrCancel statejobsGAUGECOUNTAVGNONE +APPLICATIONjobs.mvcc_statistics_update.expired_pts_recordsNumber of expired protected timestamp records owned by mvcc_statistics_update jobsrecordsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +APPLICATIONjobs.mvcc_statistics_update.fail_or_cancel_completedNumber of mvcc_statistics_update jobs which successfully completed their failure or cancelation processjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +APPLICATIONjobs.mvcc_statistics_update.fail_or_cancel_failedNumber of mvcc_statistics_update jobs which failed with a non-retriable error on their failure or cancelation processjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +APPLICATIONjobs.mvcc_statistics_update.fail_or_cancel_retry_errorNumber of mvcc_statistics_update jobs which failed with a retriable error on their failure or cancelation processjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +APPLICATIONjobs.mvcc_statistics_update.protected_age_secThe age of the oldest PTS record protected by mvcc_statistics_update jobssecondsGAUGESECONDSAVGNONE +APPLICATIONjobs.mvcc_statistics_update.protected_record_countNumber of protected timestamp records held by mvcc_statistics_update jobsrecordsGAUGECOUNTAVGNONE +APPLICATIONjobs.mvcc_statistics_update.resume_completedNumber of mvcc_statistics_update jobs which successfully resumed to completionjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +APPLICATIONjobs.mvcc_statistics_update.resume_failedNumber of mvcc_statistics_update jobs which failed with a non-retriable errorjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +APPLICATIONjobs.mvcc_statistics_update.resume_retry_errorNumber of mvcc_statistics_update jobs which failed with a retriable errorjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONjobs.new_schema_change.currently_idleNumber of new_schema_change jobs currently considered Idle and can be freely shut downjobsGAUGECOUNTAVGNONE APPLICATIONjobs.new_schema_change.currently_pausedNumber of new_schema_change jobs currently considered PausedjobsGAUGECOUNTAVGNONE APPLICATIONjobs.new_schema_change.currently_runningNumber of new_schema_change jobs currently running in Resume or OnFailOrCancel statejobsGAUGECOUNTAVGNONE diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 0da64253c148..e54ba53ed672 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -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:///#/debug/tracez tenant-rw trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. 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 '.' tenant-rw +version version 1000023.1-28 set the active cluster version in the format '.' tenant-rw diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index b8a21c736fa0..a1546bd410db 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -268,6 +268,6 @@
trace.span_registry.enabled
booleantrueif set, ongoing traces can be seen at https://<ui>/#/debug/tracezServerless/Dedicated/Self-Hosted
trace.zipkin.collector
stringthe address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.Serverless/Dedicated/Self-Hosted
ui.display_timezone
enumerationetc/utcthe timezone used to format timestamps in the ui [etc/utc = 0, america/new_york = 1]Serverless/Dedicated/Self-Hosted -
version
version1000023.1-26set the active cluster version in the format '<major>.<minor>'Serverless/Dedicated/Self-Hosted +
version
version1000023.1-28set the active cluster version in the format '<major>.<minor>'Serverless/Dedicated/Self-Hosted diff --git a/pkg/ccl/backupccl/system_schema.go b/pkg/ccl/backupccl/system_schema.go index 264f091fce54..87641b2527c8 100644 --- a/pkg/ccl/backupccl/system_schema.go +++ b/pkg/ccl/backupccl/system_schema.go @@ -818,6 +818,9 @@ var systemTableBackupConfiguration = map[string]systemBackupConfiguration{ systemschema.RegionLivenessTable.GetName(): { shouldIncludeInClusterBackup: optOutOfClusterBackup, }, + systemschema.SystemMVCCStatisticsTable.GetName(): { + shouldIncludeInClusterBackup: optOutOfClusterBackup, + }, } func rekeySystemTable( diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index d5b79122ddf1..045919c90021 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -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. @@ -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. diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index a65b996f0b69..791e87e1b9c9 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -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. @@ -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 @@ -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"]; @@ -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 { diff --git a/pkg/jobs/jobspb/wrap.go b/pkg/jobs/jobspb/wrap.go index 1f75b182d989..db603f543fe4 100644 --- a/pkg/jobs/jobspb/wrap.go +++ b/pkg/jobs/jobspb/wrap.go @@ -52,6 +52,7 @@ var ( _ Details = AutoConfigEnvRunnerDetails{} _ Details = AutoConfigTaskDetails{} _ Details = AutoUpdateSQLActivityDetails{} + _ Details = MVCCStatisticsJobDetails{} ) // ProgressDetails is a marker interface for job progress details proto structs. @@ -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. @@ -159,6 +161,7 @@ var AutomaticJobTypes = [...]Type{ TypeAutoConfigTask, TypeKeyVisualizer, TypeAutoUpdateSQLActivity, + TypeMVCCStatisticsUpdate, } // DetailsType returns the type for a payload detail. @@ -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) } @@ -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 @@ -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)) } @@ -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 } @@ -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 } @@ -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)) } @@ -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 diff --git a/pkg/jobs/registry.go b/pkg/jobs/registry.go index ef26c0ca5a54..d4965109231e 100644 --- a/pkg/jobs/registry.go +++ b/pkg/jobs/registry.go @@ -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. diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index b5194b30431a..d42636d4cd8e 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -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", diff --git a/pkg/sql/catalog/bootstrap/metadata.go b/pkg/sql/catalog/bootstrap/metadata.go index 5aca65ef1b97..9098c88a9af3 100644 --- a/pkg/sql/catalog/bootstrap/metadata.go +++ b/pkg/sql/catalog/bootstrap/metadata.go @@ -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. diff --git a/pkg/sql/catalog/catprivilege/system.go b/pkg/sql/catalog/catprivilege/system.go index f092b5cf5a05..a34a512efa71 100644 --- a/pkg/sql/catalog/catprivilege/system.go +++ b/pkg/sql/catalog/catprivilege/system.go @@ -75,6 +75,7 @@ var ( catconstants.SpanStatsSamples, catconstants.SpanStatsTenantBoundaries, catconstants.RegionalLiveness, + catconstants.MVCCStatistics, } readWriteSystemSequences = []catconstants.SystemTableName{ diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go index 8ad91759305a..f8e19d3c2685 100644 --- a/pkg/sql/catalog/systemschema/system.go +++ b/pkg/sql/catalog/systemschema/system.go @@ -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 { @@ -1237,6 +1252,7 @@ func MakeSystemTables() []SystemTable { StatementActivityTable, TransactionActivityTable, RegionLivenessTable, + SystemMVCCStatisticsTable, } } @@ -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. diff --git a/pkg/sql/mvcc_statistics_update_job.go b/pkg/sql/mvcc_statistics_update_job.go new file mode 100644 index 000000000000..198a2bff09d7 --- /dev/null +++ b/pkg/sql/mvcc_statistics_update_job.go @@ -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, + ) +} diff --git a/pkg/sql/sem/catconstants/constants.go b/pkg/sql/sem/catconstants/constants.go index dd753cd4ab6d..83d4380684ee 100644 --- a/pkg/sql/sem/catconstants/constants.go +++ b/pkg/sql/sem/catconstants/constants.go @@ -98,6 +98,7 @@ const ( SpanStatsSamples SystemTableName = "span_stats_samples" SpanStatsTenantBoundaries SystemTableName = "span_stats_tenant_boundaries" RegionalLiveness SystemTableName = "region_liveness" + MVCCStatistics SystemTableName = "mvcc_statistics" ) // Oid for virtual database and table. diff --git a/pkg/upgrade/upgradebase/testing_knobs.go b/pkg/upgrade/upgradebase/testing_knobs.go index d52147390bb5..908e8ade8082 100644 --- a/pkg/upgrade/upgradebase/testing_knobs.go +++ b/pkg/upgrade/upgradebase/testing_knobs.go @@ -76,6 +76,8 @@ type TestingKnobs struct { // clusterversion.V23_1AddSystemActivityTables upgrade, which prevents a // job from being created. SkipUpdateSQLActivityJobBootstrap bool + + SkipMVCCStatisticsJobBootstrap bool } // ModuleTestingKnobs makes TestingKnobs a base.ModuleTestingKnobs. diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index b02fa27135fe..38e644547cb5 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -22,6 +22,7 @@ go_library( "first_upgrade.go", "grant_execute_to_public.go", "key_visualizer_migration.go", + "mvcc_statistics_migration.go", "permanent_upgrades.go", "plan_gist_stmt_diagnostics_requests.go", "role_members_ids_migration.go", @@ -57,6 +58,7 @@ go_library( "//pkg/security/username", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/sql", "//pkg/sql/catalog", "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/catenumpb", diff --git a/pkg/upgrade/upgrades/mvcc_statistics_migration.go b/pkg/upgrade/upgrades/mvcc_statistics_migration.go new file mode 100644 index 000000000000..5cce2c8f5501 --- /dev/null +++ b/pkg/upgrade/upgrades/mvcc_statistics_migration.go @@ -0,0 +1,64 @@ +// 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 upgrades + +import ( + "context" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/security/username" + // Import for the side effect of registering the MVCC statistics update job. + _ "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/sql/isql" + "github.com/cockroachdb/cockroach/pkg/upgrade" +) + +func createMVCCStatisticsTableAndJobMigration( + ctx context.Context, _ clusterversion.ClusterVersion, d upgrade.TenantDeps, +) error { + + // Create the table. + err := createSystemTable( + ctx, + d.DB.KV(), + d.Settings, + keys.SystemSQLCodec, + systemschema.SystemMVCCStatisticsTable, + ) + if err != nil { + return err + } + + // Bake the job. + if d.TestingKnobs != nil && d.TestingKnobs.SkipMVCCStatisticsJobBootstrap { + return nil + } + + record := jobs.Record{ + JobID: jobs.MVCCStatisticsJobID, + Description: "mvcc statistics update job", + Username: username.NodeUserName(), + Details: jobspb.MVCCStatisticsJobDetails{}, + Progress: jobspb.MVCCStatisticsJobProgress{}, + NonCancelable: true, // The job can't be canceled, but it can be paused. + } + return d.DB.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { + return d.JobRegistry.CreateIfNotExistAdoptableJobWithTxn(ctx, record, txn) + }) +} diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go index 11b01e4e683b..7a66237fc31b 100644 --- a/pkg/upgrade/upgrades/upgrades.go +++ b/pkg/upgrade/upgrades/upgrades.go @@ -330,6 +330,12 @@ var upgrades = []upgradebase.Upgrade{ upgrade.NoPrecondition, grantExecuteToPublicOnAllFunctions, ), + upgrade.NewPermanentTenantUpgrade( + "create system.mvcc_statistics table and job", + toCV(clusterversion.V23_2_MVCCStatisticsTable), + createMVCCStatisticsTableAndJobMigration, + "create system.mvcc_statistics table and job", + ), } var (