From b0109075d16ea4cc7f37f1f5e1704e6298bcce58 Mon Sep 17 00:00:00 2001 From: Xin Hao Zhang Date: Wed, 17 Jul 2024 14:47:57 -0400 Subject: [PATCH] sql: scaffolding for db metadata update job This commit creates a new forever running BG job called UpdateCachedTableMetadata. This is just some scaffolding - the job doesn't do anything right now but it will be used to populate the system table storing cached table metadata used by obs surfaces. Epic: CRDB-37558 Fixes: #127891 Release note: None --- docs/generated/metrics/metrics.html | 12 ++++ pkg/cli/testdata/doctor/test_examine_cluster | 2 +- .../doctor/test_examine_cluster_dropped | 2 +- .../testdata/doctor/test_examine_cluster_jobs | 2 +- pkg/jobs/jobs_test.go | 9 +-- pkg/jobs/jobspb/jobs.proto | 6 ++ pkg/jobs/jobspb/wrap.go | 16 ++++- pkg/jobs/registry.go | 2 + pkg/jobs/registry_test.go | 25 +++---- pkg/sql/BUILD.bazel | 1 + pkg/sql/update_table_metadata_cache_job.go | 65 +++++++++++++++++++ pkg/upgrade/upgradebase/testing_knobs.go | 2 + pkg/upgrade/upgrades/BUILD.bazel | 1 + ..._create_update_table_metadata_cache_job.go | 44 +++++++++++++ pkg/upgrade/upgrades/permanent_upgrades.go | 1 + pkg/upgrade/upgrades/upgrades.go | 4 +- .../v24_3_table_metadata_system_table.go | 12 +++- 17 files changed, 182 insertions(+), 24 deletions(-) create mode 100644 pkg/sql/update_table_metadata_cache_job.go create mode 100644 pkg/upgrade/upgrades/permanent_create_update_table_metadata_cache_job.go diff --git a/docs/generated/metrics/metrics.html b/docs/generated/metrics/metrics.html index 7602c06f9a92..152e7ebd80b5 100644 --- a/docs/generated/metrics/metrics.html +++ b/docs/generated/metrics/metrics.html @@ -1362,6 +1362,18 @@ APPLICATIONjobs.typedesc_schema_change.resume_completedNumber of typedesc_schema_change jobs which successfully resumed to completionjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONjobs.typedesc_schema_change.resume_failedNumber of typedesc_schema_change jobs which failed with a non-retriable errorjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONjobs.typedesc_schema_change.resume_retry_errorNumber of typedesc_schema_change jobs which failed with a retriable errorjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +APPLICATIONjobs.update_table_metadata_cache.currently_idleNumber of update_table_metadata_cache jobs currently considered Idle and can be freely shut downjobsGAUGECOUNTAVGNONE +APPLICATIONjobs.update_table_metadata_cache.currently_pausedNumber of update_table_metadata_cache jobs currently considered PausedjobsGAUGECOUNTAVGNONE +APPLICATIONjobs.update_table_metadata_cache.currently_runningNumber of update_table_metadata_cache jobs currently running in Resume or OnFailOrCancel statejobsGAUGECOUNTAVGNONE +APPLICATIONjobs.update_table_metadata_cache.expired_pts_recordsNumber of expired protected timestamp records owned by update_table_metadata_cache jobsrecordsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +APPLICATIONjobs.update_table_metadata_cache.fail_or_cancel_completedNumber of update_table_metadata_cache jobs which successfully completed their failure or cancelation processjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +APPLICATIONjobs.update_table_metadata_cache.fail_or_cancel_failedNumber of update_table_metadata_cache jobs which failed with a non-retriable error on their failure or cancelation processjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +APPLICATIONjobs.update_table_metadata_cache.fail_or_cancel_retry_errorNumber of update_table_metadata_cache jobs which failed with a retriable error on their failure or cancelation processjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +APPLICATIONjobs.update_table_metadata_cache.protected_age_secThe age of the oldest PTS record protected by update_table_metadata_cache jobssecondsGAUGESECONDSAVGNONE +APPLICATIONjobs.update_table_metadata_cache.protected_record_countNumber of protected timestamp records held by update_table_metadata_cache jobsrecordsGAUGECOUNTAVGNONE +APPLICATIONjobs.update_table_metadata_cache.resume_completedNumber of update_table_metadata_cache jobs which successfully resumed to completionjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +APPLICATIONjobs.update_table_metadata_cache.resume_failedNumber of update_table_metadata_cache jobs which failed with a non-retriable errorjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE +APPLICATIONjobs.update_table_metadata_cache.resume_retry_errorNumber of update_table_metadata_cache jobs which failed with a retriable errorjobsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONkv.protectedts.reconciliation.errorsnumber of errors encountered during reconciliation runs on this nodeCountCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONkv.protectedts.reconciliation.num_runsnumber of successful reconciliation runs on this nodeCountCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONkv.protectedts.reconciliation.records_processednumber of records processed without error during reconciliation on this nodeCountCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE diff --git a/pkg/cli/testdata/doctor/test_examine_cluster b/pkg/cli/testdata/doctor/test_examine_cluster index a05a7f1e495d..5dc28a7efa8a 100644 --- a/pkg/cli/testdata/doctor/test_examine_cluster +++ b/pkg/cli/testdata/doctor/test_examine_cluster @@ -3,5 +3,5 @@ debug doctor examine cluster debug doctor examine cluster Examining 64 descriptors and 63 namespace entries... ParentID 100, ParentSchemaID 101: relation "foo" (105): expected matching namespace entry, found none -Examining 10 jobs... +Examining 11 jobs... ERROR: validation failed diff --git a/pkg/cli/testdata/doctor/test_examine_cluster_dropped b/pkg/cli/testdata/doctor/test_examine_cluster_dropped index 34dd837c7581..75feb6cf5dde 100644 --- a/pkg/cli/testdata/doctor/test_examine_cluster_dropped +++ b/pkg/cli/testdata/doctor/test_examine_cluster_dropped @@ -2,5 +2,5 @@ debug doctor examine cluster ---- debug doctor examine cluster Examining 63 descriptors and 63 namespace entries... -Examining 8 jobs... +Examining 9 jobs... No problems found! diff --git a/pkg/cli/testdata/doctor/test_examine_cluster_jobs b/pkg/cli/testdata/doctor/test_examine_cluster_jobs index 071b037ed902..373f80aaeb3b 100644 --- a/pkg/cli/testdata/doctor/test_examine_cluster_jobs +++ b/pkg/cli/testdata/doctor/test_examine_cluster_jobs @@ -7,5 +7,5 @@ Examining 63 descriptors and 64 namespace entries... ParentID 183, ParentSchemaID 381: relation "foo" (104): expected matching namespace entry, found none ParentID 183, ParentSchemaID 381: relation "foo" (104): mutation job 962952277419655169: job 962952277419655169 not found ParentID 100, ParentSchemaID 101: namespace entry "foo" (104): mismatched name "foo" in relation descriptor -Examining 8 jobs... +Examining 9 jobs... ERROR: validation failed diff --git a/pkg/jobs/jobs_test.go b/pkg/jobs/jobs_test.go index 1de827c42cc1..40383eafdf0e 100644 --- a/pkg/jobs/jobs_test.go +++ b/pkg/jobs/jobs_test.go @@ -216,10 +216,11 @@ func (rts *registryTestSuite) setUp(t *testing.T) func() { ManagerDisableJobCreation: true, } args.Knobs.UpgradeManager = &upgradebase.TestingKnobs{ - DontUseJobs: true, - SkipJobMetricsPollingJobBootstrap: true, - SkipUpdateSQLActivityJobBootstrap: true, - SkipMVCCStatisticsJobBootstrap: true, + DontUseJobs: true, + SkipJobMetricsPollingJobBootstrap: true, + SkipUpdateSQLActivityJobBootstrap: true, + SkipMVCCStatisticsJobBootstrap: true, + SkipUpdateTableMetadataCacheBootstrap: true, } args.Knobs.KeyVisualizer = &keyvisualizer.TestingKnobs{SkipJobBootstrap: true} diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index b5abace6953a..07516fb13df0 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -1334,6 +1334,9 @@ message MVCCStatisticsJobProgress { } +message UpdateTableMetadataCacheDetails {} +message UpdateTableMetadataCacheProgress {} + message ImportRollbackDetails { // TableID is the descriptor ID of table that should be rolled back. // @@ -1413,6 +1416,7 @@ message Payload { ImportRollbackDetails import_rollback_details = 46; HistoryRetentionDetails history_retention_details = 47; LogicalReplicationDetails logical_replication_details = 48; + UpdateTableMetadataCacheDetails update_table_metadata_cache_details = 49; } reserved 26; // PauseReason is used to describe the reason that the job is currently paused @@ -1490,6 +1494,7 @@ message Progress { ImportRollbackProgress import_rollback_progress = 34; HistoryRetentionProgress HistoryRetentionProgress = 35; LogicalReplicationProgress LogicalReplication = 36; + UpdateTableMetadataCacheProgress table_metadata_cache = 37; } uint64 trace_id = 21 [(gogoproto.nullable) = false, (gogoproto.customname) = "TraceID", (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb.TraceID"]; @@ -1530,6 +1535,7 @@ enum Type { HISTORY_RETENTION = 26 [(gogoproto.enumvalue_customname) = "TypeHistoryRetention"]; LOGICAL_REPLICATION = 27 [(gogoproto.enumvalue_customname) = "TypeLogicalReplication"]; AUTO_CREATE_PARTIAL_STATS = 28 [(gogoproto.enumvalue_customname) = "TypeAutoCreatePartialStats"]; + UPDATE_TABLE_METADATA_CACHE = 29 [(gogoproto.enumvalue_customname) = "TypeUpdateTableMetadataCache"]; } message Job { diff --git a/pkg/jobs/jobspb/wrap.go b/pkg/jobs/jobspb/wrap.go index 1980bce4aba7..af0f8a8daa84 100644 --- a/pkg/jobs/jobspb/wrap.go +++ b/pkg/jobs/jobspb/wrap.go @@ -49,6 +49,7 @@ var ( _ Details = ImportRollbackDetails{} _ Details = HistoryRetentionDetails{} _ Details = LogicalReplicationDetails{} + _ Details = UpdateTableMetadataCacheDetails{} ) // ProgressDetails is a marker interface for job progress details proto structs. @@ -77,6 +78,7 @@ var ( _ ProgressDetails = ImportRollbackProgress{} _ ProgressDetails = HistoryRetentionProgress{} _ ProgressDetails = LogicalReplicationProgress{} + _ ProgressDetails = UpdateTableMetadataCacheProgress{} ) // Type returns the payload's job type and panics if the type is invalid. @@ -169,6 +171,7 @@ var AutomaticJobTypes = [...]Type{ TypeKeyVisualizer, TypeAutoUpdateSQLActivity, TypeMVCCStatisticsUpdate, + TypeUpdateTableMetadataCache, } // DetailsType returns the type for a payload detail. @@ -232,6 +235,8 @@ func DetailsType(d isPayload_Details) (Type, error) { return TypeHistoryRetention, nil case *Payload_LogicalReplicationDetails: return TypeLogicalReplication, nil + case *Payload_UpdateTableMetadataCacheDetails: + return TypeUpdateTableMetadataCache, nil default: return TypeUnspecified, errors.Newf("Payload.Type called on a payload with an unknown details type: %T", d) } @@ -283,6 +288,7 @@ var JobDetailsForEveryJobType = map[Type]Details{ TypeImportRollback: ImportRollbackDetails{}, TypeHistoryRetention: HistoryRetentionDetails{}, TypeLogicalReplication: LogicalReplicationDetails{}, + TypeUpdateTableMetadataCache: UpdateTableMetadataCacheDetails{}, } // WrapProgressDetails wraps a ProgressDetails object in the protobuf wrapper @@ -346,6 +352,8 @@ func WrapProgressDetails(details ProgressDetails) interface { return &Progress_HistoryRetentionProgress{HistoryRetentionProgress: &d} case LogicalReplicationProgress: return &Progress_LogicalReplication{LogicalReplication: &d} + case UpdateTableMetadataCacheProgress: + return &Progress_TableMetadataCache{TableMetadataCache: &d} default: panic(errors.AssertionFailedf("WrapProgressDetails: unknown progress type %T", d)) } @@ -407,6 +415,8 @@ func (p *Payload) UnwrapDetails() Details { return *d.HistoryRetentionDetails case *Payload_LogicalReplicationDetails: return *d.LogicalReplicationDetails + case *Payload_UpdateTableMetadataCacheDetails: + return *d.UpdateTableMetadataCacheDetails default: return nil } @@ -468,6 +478,8 @@ func (p *Progress) UnwrapDetails() ProgressDetails { return *d.HistoryRetentionProgress case *Progress_LogicalReplication: return *d.LogicalReplication + case *Progress_TableMetadataCache: + return *d.TableMetadataCache default: return nil } @@ -553,6 +565,8 @@ func WrapPayloadDetails(details Details) interface { return &Payload_HistoryRetentionDetails{HistoryRetentionDetails: &d} case LogicalReplicationDetails: return &Payload_LogicalReplicationDetails{LogicalReplicationDetails: &d} + case UpdateTableMetadataCacheDetails: + return &Payload_UpdateTableMetadataCacheDetails{UpdateTableMetadataCacheDetails: &d} default: panic(errors.AssertionFailedf("jobs.WrapPayloadDetails: unknown details type %T", d)) } @@ -588,7 +602,7 @@ const ( func (Type) SafeValue() {} // NumJobTypes is the number of jobs types. -const NumJobTypes = 29 +const NumJobTypes = 30 // 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 55b94d02a15b..c7aed6a7ff84 100644 --- a/pkg/jobs/registry.go +++ b/pkg/jobs/registry.go @@ -318,6 +318,8 @@ const ( // MVCCStatisticsJobID A static job ID used for the MVCC statistics update // job. MVCCStatisticsJobID = jobspb.JobID(104) + + UpdateTableMetadataCacheJobID = jobspb.JobID(105) ) // MakeJobID generates a new job ID. diff --git a/pkg/jobs/registry_test.go b/pkg/jobs/registry_test.go index dc9308c3e951..ba96f85e07a6 100644 --- a/pkg/jobs/registry_test.go +++ b/pkg/jobs/registry_test.go @@ -126,9 +126,10 @@ func TestRegistryGC(t *testing.T) { }, UpgradeManager: &upgradebase.TestingKnobs{ // This test wants to look at job records. - DontUseJobs: true, - SkipJobMetricsPollingJobBootstrap: true, - SkipMVCCStatisticsJobBootstrap: true, + DontUseJobs: true, + SkipJobMetricsPollingJobBootstrap: true, + SkipMVCCStatisticsJobBootstrap: true, + SkipUpdateTableMetadataCacheBootstrap: true, }, KeyVisualizer: &keyvisualizer.TestingKnobs{ SkipJobBootstrap: true, @@ -284,10 +285,11 @@ func TestRegistryGCPagination(t *testing.T) { }, UpgradeManager: &upgradebase.TestingKnobs{ // This test wants to count job records. - DontUseJobs: true, - SkipJobMetricsPollingJobBootstrap: true, - SkipUpdateSQLActivityJobBootstrap: true, - SkipMVCCStatisticsJobBootstrap: true, + DontUseJobs: true, + SkipJobMetricsPollingJobBootstrap: true, + SkipUpdateSQLActivityJobBootstrap: true, + SkipMVCCStatisticsJobBootstrap: true, + SkipUpdateTableMetadataCacheBootstrap: true, }, KeyVisualizer: &keyvisualizer.TestingKnobs{ SkipJobBootstrap: true, @@ -758,10 +760,11 @@ func TestRetriesWithExponentialBackoff(t *testing.T) { ManagerDisableJobCreation: true, }, UpgradeManager: &upgradebase.TestingKnobs{ - DontUseJobs: true, - SkipJobMetricsPollingJobBootstrap: true, - SkipUpdateSQLActivityJobBootstrap: true, - SkipMVCCStatisticsJobBootstrap: true, + DontUseJobs: true, + SkipJobMetricsPollingJobBootstrap: true, + SkipUpdateSQLActivityJobBootstrap: true, + SkipMVCCStatisticsJobBootstrap: true, + SkipUpdateTableMetadataCacheBootstrap: true, }, KeyVisualizer: &keyvisualizer.TestingKnobs{ SkipJobBootstrap: true, diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index ac7a1c30a8f3..60c0b7cbf6c1 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -277,6 +277,7 @@ go_library( "unsplit.go", "unsupported_vars.go", "update.go", + "update_table_metadata_cache_job.go", "upsert.go", "user.go", "values.go", diff --git a/pkg/sql/update_table_metadata_cache_job.go b/pkg/sql/update_table_metadata_cache_job.go new file mode 100644 index 000000000000..de746a173e42 --- /dev/null +++ b/pkg/sql/update_table_metadata_cache_job.go @@ -0,0 +1,65 @@ +// Copyright 2024 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 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 tableMetadataUpdateJobResumer struct { + job *jobs.Job +} + +var _ jobs.Resumer = (*tableMetadataUpdateJobResumer)(nil) + +// Resume is part of the jobs.Resumer interface. +func (j *tableMetadataUpdateJobResumer) Resume(ctx context.Context, execCtxI interface{}) error { + log.Infof(ctx, "starting table metadata update job") + j.job.MarkIdle(true) + + <-ctx.Done() + return nil +} + +// OnFailOrCancel implements jobs.Resumer. +func (j *tableMetadataUpdateJobResumer) OnFailOrCancel( + ctx context.Context, execCtx 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 +} + +// CollectProfile implements jobs.Resumer. +func (j *tableMetadataUpdateJobResumer) CollectProfile( + ctx context.Context, execCtx interface{}, +) error { + return nil +} + +func init() { + jobs.RegisterConstructor( + jobspb.TypeUpdateTableMetadataCache, + func(job *jobs.Job, settings *cluster.Settings) jobs.Resumer { + return &tableMetadataUpdateJobResumer{job: job} + }, jobs.DisablesTenantCostControl, + ) +} diff --git a/pkg/upgrade/upgradebase/testing_knobs.go b/pkg/upgrade/upgradebase/testing_knobs.go index 628ef4d5f5f5..5a7333be5892 100644 --- a/pkg/upgrade/upgradebase/testing_knobs.go +++ b/pkg/upgrade/upgradebase/testing_knobs.go @@ -73,6 +73,8 @@ type TestingKnobs struct { SkipUpdateSQLActivityJobBootstrap bool SkipMVCCStatisticsJobBootstrap bool + + SkipUpdateTableMetadataCacheBootstrap bool } // ModuleTestingKnobs makes TestingKnobs a base.ModuleTestingKnobs. diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index 6f46d411856c..5c443a198a29 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -6,6 +6,7 @@ go_library( "descriptor_utils.go", "first_upgrade.go", "permanent_create_jobs_metrics_polling_job.go", + "permanent_create_update_table_metadata_cache_job.go", "permanent_ensure_sql_schema_telemetry_schedule.go", "permanent_key_visualizer_migration.go", "permanent_mvcc_statistics_migration.go", diff --git a/pkg/upgrade/upgrades/permanent_create_update_table_metadata_cache_job.go b/pkg/upgrade/upgrades/permanent_create_update_table_metadata_cache_job.go new file mode 100644 index 000000000000..23587869ed4e --- /dev/null +++ b/pkg/upgrade/upgrades/permanent_create_update_table_metadata_cache_job.go @@ -0,0 +1,44 @@ +// Copyright 2024 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 + +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/jobs/metricspoller" // Ensure job implementation is linked. + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql/isql" + "github.com/cockroachdb/cockroach/pkg/upgrade" +) + +func createUpdateTableMetadataCacheJob( + ctx context.Context, _ clusterversion.ClusterVersion, d upgrade.TenantDeps, +) error { + if d.TestingKnobs != nil && d.TestingKnobs.SkipUpdateTableMetadataCacheBootstrap { + return nil + } + + return d.DB.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { + jr := jobs.Record{ + JobID: jobs.UpdateTableMetadataCacheJobID, + Description: jobspb.TypeUpdateTableMetadataCache.String(), + Details: jobspb.UpdateTableMetadataCacheDetails{}, + Progress: jobspb.UpdateTableMetadataCacheProgress{}, + CreatedBy: &jobs.CreatedByInfo{Name: username.NodeUser, ID: username.NodeUserID}, + Username: username.NodeUserName(), + NonCancelable: true, + } + return d.JobRegistry.CreateIfNotExistAdoptableJobWithTxn(ctx, jr, txn) + }) +} diff --git a/pkg/upgrade/upgrades/permanent_upgrades.go b/pkg/upgrade/upgrades/permanent_upgrades.go index 61c874cb9f06..82aedb163733 100644 --- a/pkg/upgrade/upgrades/permanent_upgrades.go +++ b/pkg/upgrade/upgrades/permanent_upgrades.go @@ -76,6 +76,7 @@ func bootstrapCluster( {"create jobs metrics polling job", createJobsMetricsPollingJob}, {"create sql activity updater job", createActivityUpdateJobMigration}, {"create mvcc stats job", createMVCCStatisticsJob}, + {"create update cached table metadata job", createUpdateTableMetadataCacheJob}, } { log.Infof(ctx, "executing bootstrap step %q", u.name) if err := u.fn(ctx, cv, deps); err != nil { diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go index d72f6afd25dc..854b0800fca7 100644 --- a/pkg/upgrade/upgrades/upgrades.go +++ b/pkg/upgrade/upgrades/upgrades.go @@ -108,10 +108,10 @@ var upgrades = []upgradebase.Upgrade{ ), upgrade.NewTenantUpgrade( - "add new table_metadata table to the system tenant", + "add new table_metadata table and job to the system tenant", clusterversion.V24_3_TableMetadata.Version(), upgrade.NoPrecondition, - addTableMetadataTable, + addTableMetadataTableAndJob, upgrade.RestoreActionNotRequired("cluster restore does not restore this table"), ), diff --git a/pkg/upgrade/upgrades/v24_3_table_metadata_system_table.go b/pkg/upgrade/upgrades/v24_3_table_metadata_system_table.go index 008a289287bf..b4786fafa4d8 100644 --- a/pkg/upgrade/upgrades/v24_3_table_metadata_system_table.go +++ b/pkg/upgrade/upgrades/v24_3_table_metadata_system_table.go @@ -19,12 +19,18 @@ import ( "github.com/cockroachdb/cockroach/pkg/upgrade" ) -// addTableMetadataTable creates the system.table_metadata table if it does not exist. -func addTableMetadataTable( - ctx context.Context, _ clusterversion.ClusterVersion, d upgrade.TenantDeps, +// addTableMetadataTableAndJob creates the system.table_metadata table if it does not exist. +func addTableMetadataTableAndJob( + ctx context.Context, version clusterversion.ClusterVersion, d upgrade.TenantDeps, ) error { if err := createSystemTable(ctx, d.DB, d.Settings, d.Codec, systemschema.TableMetadata, tree.LocalityLevelTable); err != nil { return err } + + // Add job to backfill the table metadata table. + if err := createUpdateTableMetadataCacheJob(ctx, version, d); err != nil { + return err + } + return nil }