diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 81c75d1ecda7..237467a83db6 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -100,6 +100,6 @@ trace.debug.enablebooleanfalseif set, traces for recent requests can be seen at https:///debug/requests trace.lightstep.tokenstringif set, traces go to Lightstep using this token trace.zipkin.collectorstringif set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set -versionversion20.2-18set the active cluster version in the format '.' +versionversion20.2-24set the active cluster version in the format '.' diff --git a/pkg/base/testing_knobs.go b/pkg/base/testing_knobs.go index 1d96537b8d43..04a62ea6ae01 100644 --- a/pkg/base/testing_knobs.go +++ b/pkg/base/testing_knobs.go @@ -36,4 +36,5 @@ type TestingKnobs struct { TenantTestingKnobs ModuleTestingKnobs JobsTestingKnobs ModuleTestingKnobs BackupRestore ModuleTestingKnobs + MigrationManager ModuleTestingKnobs } diff --git a/pkg/ccl/backupccl/system_schema.go b/pkg/ccl/backupccl/system_schema.go index f90f1ea22bd3..b3279929d121 100644 --- a/pkg/ccl/backupccl/system_schema.go +++ b/pkg/ccl/backupccl/system_schema.go @@ -227,6 +227,9 @@ var systemTableBackupConfiguration = map[string]systemBackupConfiguration{ systemschema.WebSessionsTable.GetName(): { includeInClusterBackup: optOutOfClusterBackup, }, + systemschema.LongRunningMigrationsTable.GetName(): { + includeInClusterBackup: optOutOfClusterBackup, + }, } // getSystemTablesToBackup returns a set of system table names that should be diff --git a/pkg/cli/testdata/doctor/testcluster b/pkg/cli/testdata/doctor/testcluster index 6d175ad4183f..2953ab3a8ff3 100644 --- a/pkg/cli/testdata/doctor/testcluster +++ b/pkg/cli/testdata/doctor/testcluster @@ -1,7 +1,7 @@ doctor cluster ---- debug doctor cluster -Examining 34 descriptors and 35 namespace entries... +Examining 35 descriptors and 36 namespace entries... Table 53: ParentID 50, ParentSchemaID 29, Name 'foo': not being dropped but no namespace entry found Examining 1 running jobs... ERROR: validation failed diff --git a/pkg/cli/testdata/zip/partial1 b/pkg/cli/testdata/zip/partial1 index c1d256f48693..71f18043445f 100644 --- a/pkg/cli/testdata/zip/partial1 +++ b/pkg/cli/testdata/zip/partial1 @@ -58,7 +58,7 @@ requesting goroutine files for node 1... writing: debug/nodes/1/goroutines.err.t ^- resulted in ... requesting log file ... requesting log file ... -requesting ranges... 35 found +requesting ranges... 36 found writing: debug/nodes/1/ranges/1.json writing: debug/nodes/1/ranges/2.json writing: debug/nodes/1/ranges/3.json @@ -94,6 +94,7 @@ writing: debug/nodes/1/ranges/32.json writing: debug/nodes/1/ranges/33.json writing: debug/nodes/1/ranges/34.json writing: debug/nodes/1/ranges/35.json +writing: debug/nodes/1/ranges/36.json writing: debug/nodes/2/status.json using SQL connection URL for node 2: postgresql://... retrieving SQL data for crdb_internal.feature_usage... writing: debug/nodes/2/crdb_internal.feature_usage.txt @@ -194,7 +195,7 @@ requesting goroutine files for node 3... writing: debug/nodes/3/goroutines.err.t ^- resulted in ... requesting log file ... requesting log file ... -requesting ranges... 35 found +requesting ranges... 36 found writing: debug/nodes/3/ranges/1.json writing: debug/nodes/3/ranges/2.json writing: debug/nodes/3/ranges/3.json @@ -230,6 +231,7 @@ writing: debug/nodes/3/ranges/32.json writing: debug/nodes/3/ranges/33.json writing: debug/nodes/3/ranges/34.json writing: debug/nodes/3/ranges/35.json +writing: debug/nodes/3/ranges/36.json doctor examining cluster...No problems found! writing: debug/reports/doctor.txt requesting list of SQL databases... 3 found @@ -238,7 +240,7 @@ requesting database details for defaultdb... writing: debug/schema/defaultdb@det requesting database details for postgres... writing: debug/schema/postgres@details.json 0 tables found requesting database details for system... writing: debug/schema/system@details.json -29 tables found +30 tables found requesting table details for system.public.namespace... writing: debug/schema/system/public_namespace.json requesting table details for system.public.descriptor... writing: debug/schema/system/public_descriptor.json requesting table details for system.public.users... writing: debug/schema/system/public_users.json @@ -268,5 +270,6 @@ requesting table details for system.public.statement_diagnostics_requests... wri requesting table details for system.public.statement_diagnostics... writing: debug/schema/system/public_statement_diagnostics.json requesting table details for system.public.scheduled_jobs... writing: debug/schema/system/public_scheduled_jobs.json requesting table details for system.public.sqlliveness... writing: debug/schema/system/public_sqlliveness.json +requesting table details for system.public.long_running_migrations... writing: debug/schema/system/public_long_running_migrations.json writing: debug/pprof-summary.sh writing: debug/hot-ranges.sh diff --git a/pkg/cli/testdata/zip/partial1_excluded b/pkg/cli/testdata/zip/partial1_excluded index f3ed1e74423c..271d54727dc8 100644 --- a/pkg/cli/testdata/zip/partial1_excluded +++ b/pkg/cli/testdata/zip/partial1_excluded @@ -58,7 +58,7 @@ requesting goroutine files for node 1... writing: debug/nodes/1/goroutines.err.t ^- resulted in ... requesting log file ... requesting log file ... -requesting ranges... 35 found +requesting ranges... 36 found writing: debug/nodes/1/ranges/1.json writing: debug/nodes/1/ranges/2.json writing: debug/nodes/1/ranges/3.json @@ -94,6 +94,7 @@ writing: debug/nodes/1/ranges/32.json writing: debug/nodes/1/ranges/33.json writing: debug/nodes/1/ranges/34.json writing: debug/nodes/1/ranges/35.json +writing: debug/nodes/1/ranges/36.json writing: debug/nodes/2.skipped writing: debug/nodes/3/status.json using SQL connection URL for node 3: postgresql://... @@ -125,7 +126,7 @@ requesting goroutine files for node 3... writing: debug/nodes/3/goroutines.err.t ^- resulted in ... requesting log file ... requesting log file ... -requesting ranges... 35 found +requesting ranges... 36 found writing: debug/nodes/3/ranges/1.json writing: debug/nodes/3/ranges/2.json writing: debug/nodes/3/ranges/3.json @@ -161,6 +162,7 @@ writing: debug/nodes/3/ranges/32.json writing: debug/nodes/3/ranges/33.json writing: debug/nodes/3/ranges/34.json writing: debug/nodes/3/ranges/35.json +writing: debug/nodes/3/ranges/36.json doctor examining cluster...No problems found! writing: debug/reports/doctor.txt requesting list of SQL databases... 3 found @@ -169,7 +171,7 @@ requesting database details for defaultdb... writing: debug/schema/defaultdb@det requesting database details for postgres... writing: debug/schema/postgres@details.json 0 tables found requesting database details for system... writing: debug/schema/system@details.json -29 tables found +30 tables found requesting table details for system.public.namespace... writing: debug/schema/system/public_namespace.json requesting table details for system.public.descriptor... writing: debug/schema/system/public_descriptor.json requesting table details for system.public.users... writing: debug/schema/system/public_users.json @@ -199,5 +201,6 @@ requesting table details for system.public.statement_diagnostics_requests... wri requesting table details for system.public.statement_diagnostics... writing: debug/schema/system/public_statement_diagnostics.json requesting table details for system.public.scheduled_jobs... writing: debug/schema/system/public_scheduled_jobs.json requesting table details for system.public.sqlliveness... writing: debug/schema/system/public_sqlliveness.json +requesting table details for system.public.long_running_migrations... writing: debug/schema/system/public_long_running_migrations.json writing: debug/pprof-summary.sh writing: debug/hot-ranges.sh diff --git a/pkg/cli/testdata/zip/partial2 b/pkg/cli/testdata/zip/partial2 index 728b1054bee9..ce31c4cd914a 100644 --- a/pkg/cli/testdata/zip/partial2 +++ b/pkg/cli/testdata/zip/partial2 @@ -58,7 +58,7 @@ requesting goroutine files for node 1... writing: debug/nodes/1/goroutines.err.t ^- resulted in ... requesting log file ... requesting log file ... -requesting ranges... 35 found +requesting ranges... 36 found writing: debug/nodes/1/ranges/1.json writing: debug/nodes/1/ranges/2.json writing: debug/nodes/1/ranges/3.json @@ -94,6 +94,7 @@ writing: debug/nodes/1/ranges/32.json writing: debug/nodes/1/ranges/33.json writing: debug/nodes/1/ranges/34.json writing: debug/nodes/1/ranges/35.json +writing: debug/nodes/1/ranges/36.json writing: debug/nodes/3/status.json using SQL connection URL for node 3: postgresql://... retrieving SQL data for crdb_internal.feature_usage... writing: debug/nodes/3/crdb_internal.feature_usage.txt @@ -124,7 +125,7 @@ requesting goroutine files for node 3... writing: debug/nodes/3/goroutines.err.t ^- resulted in ... requesting log file ... requesting log file ... -requesting ranges... 35 found +requesting ranges... 36 found writing: debug/nodes/3/ranges/1.json writing: debug/nodes/3/ranges/2.json writing: debug/nodes/3/ranges/3.json @@ -160,6 +161,7 @@ writing: debug/nodes/3/ranges/32.json writing: debug/nodes/3/ranges/33.json writing: debug/nodes/3/ranges/34.json writing: debug/nodes/3/ranges/35.json +writing: debug/nodes/3/ranges/36.json doctor examining cluster...No problems found! writing: debug/reports/doctor.txt requesting list of SQL databases... 3 found @@ -168,7 +170,7 @@ requesting database details for defaultdb... writing: debug/schema/defaultdb@det requesting database details for postgres... writing: debug/schema/postgres@details.json 0 tables found requesting database details for system... writing: debug/schema/system@details.json -29 tables found +30 tables found requesting table details for system.public.namespace... writing: debug/schema/system/public_namespace.json requesting table details for system.public.descriptor... writing: debug/schema/system/public_descriptor.json requesting table details for system.public.users... writing: debug/schema/system/public_users.json @@ -198,5 +200,6 @@ requesting table details for system.public.statement_diagnostics_requests... wri requesting table details for system.public.statement_diagnostics... writing: debug/schema/system/public_statement_diagnostics.json requesting table details for system.public.scheduled_jobs... writing: debug/schema/system/public_scheduled_jobs.json requesting table details for system.public.sqlliveness... writing: debug/schema/system/public_sqlliveness.json +requesting table details for system.public.long_running_migrations... writing: debug/schema/system/public_long_running_migrations.json writing: debug/pprof-summary.sh writing: debug/hot-ranges.sh diff --git a/pkg/cli/testdata/zip/specialnames b/pkg/cli/testdata/zip/specialnames index 98fbbba190da..32fe1f36c9f9 100644 --- a/pkg/cli/testdata/zip/specialnames +++ b/pkg/cli/testdata/zip/specialnames @@ -22,7 +22,7 @@ requesting table details for defaultdb.public."../system"... writing: debug/sche requesting database details for postgres... writing: debug/schema/postgres@details.json 0 tables found requesting database details for system... writing: debug/schema/system-1@details.json -29 tables found +30 tables found requesting table details for system.public.namespace... writing: debug/schema/system-1/public_namespace.json requesting table details for system.public.descriptor... writing: debug/schema/system-1/public_descriptor.json requesting table details for system.public.users... writing: debug/schema/system-1/public_users.json @@ -52,3 +52,4 @@ requesting table details for system.public.statement_diagnostics_requests... wri requesting table details for system.public.statement_diagnostics... writing: debug/schema/system-1/public_statement_diagnostics.json requesting table details for system.public.scheduled_jobs... writing: debug/schema/system-1/public_scheduled_jobs.json requesting table details for system.public.sqlliveness... writing: debug/schema/system-1/public_sqlliveness.json +requesting table details for system.public.long_running_migrations... writing: debug/schema/system-1/public_long_running_migrations.json diff --git a/pkg/cli/testdata/zip/testzip b/pkg/cli/testdata/zip/testzip index 410c47f283fc..9f39a09aa05f 100644 --- a/pkg/cli/testdata/zip/testzip +++ b/pkg/cli/testdata/zip/testzip @@ -57,7 +57,7 @@ requesting heap profile for node 1... writing: debug/nodes/1/heap.pprof requesting heap files for node 1... ? found requesting goroutine files for node 1... 0 found requesting log file ... -requesting ranges... 35 found +requesting ranges... 36 found writing: debug/nodes/1/ranges/1.json writing: debug/nodes/1/ranges/2.json writing: debug/nodes/1/ranges/3.json @@ -93,6 +93,7 @@ writing: debug/nodes/1/ranges/32.json writing: debug/nodes/1/ranges/33.json writing: debug/nodes/1/ranges/34.json writing: debug/nodes/1/ranges/35.json +writing: debug/nodes/1/ranges/36.json doctor examining cluster...No problems found! writing: debug/reports/doctor.txt requesting list of SQL databases... 3 found @@ -101,7 +102,7 @@ requesting database details for defaultdb... writing: debug/schema/defaultdb@det requesting database details for postgres... writing: debug/schema/postgres@details.json 0 tables found requesting database details for system... writing: debug/schema/system@details.json -29 tables found +30 tables found requesting table details for system.public.namespace... writing: debug/schema/system/public_namespace.json requesting table details for system.public.descriptor... writing: debug/schema/system/public_descriptor.json requesting table details for system.public.users... writing: debug/schema/system/public_users.json @@ -131,5 +132,6 @@ requesting table details for system.public.statement_diagnostics_requests... wri requesting table details for system.public.statement_diagnostics... writing: debug/schema/system/public_statement_diagnostics.json requesting table details for system.public.scheduled_jobs... writing: debug/schema/system/public_scheduled_jobs.json requesting table details for system.public.sqlliveness... writing: debug/schema/system/public_sqlliveness.json +requesting table details for system.public.long_running_migrations... writing: debug/schema/system/public_long_running_migrations.json writing: debug/pprof-summary.sh writing: debug/hot-ranges.sh diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index e15974015e4d..4be5ed066e8a 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -203,6 +203,24 @@ const ( CPutInline // ReplicaVersions enables the versioning of Replica state. ReplicaVersions + // replacedTruncatedAndRangeAppliedStateMigration stands in for + // TruncatedAndRangeAppliedStateMigration which was re-introduced after the + // migration job was introduced. This is necessary because the jobs + // infrastructure used to run this migration in v21.1 and its later alphas + // was introduced after this version was first introduced. Later code in the + // release relies on the job to run the migration but the job relies on + // its startup migrations having been run. Versions associated with long + // running migrations must follow LongRunningMigrations. + replacedTruncatedAndRangeAppliedStateMigration + // replacedPostTruncatedAndRangeAppliedStateMigration is like the above + // version. See its comment. + replacedPostTruncatedAndRangeAppliedStateMigration + // NewSchemaChanger enables the new schema changer. + NewSchemaChanger + // LongRunningMigrations introduces the LongRunningMigrations table and jobs. + // All versions which have a registered long-running migration must have a + // version higher than this version. + LongRunningMigrations // TruncatedAndRangeAppliedStateMigration is part of the migration to stop // using the legacy truncated state within KV. After the migration, we'll be // using the unreplicated truncated state and the RangeAppliedState on all @@ -217,8 +235,6 @@ const ( // using the replicated legacy TruncatedState. It's also used in asserting // that no replicated truncated state representation is found. PostTruncatedAndRangeAppliedStateMigration - // NewSchemaChanger enables the new schema changer. - NewSchemaChanger // Step (1): Add new versions here. ) @@ -350,17 +366,29 @@ var versionsSingleton = keyedVersions([]keyedVersion{ Version: roachpb.Version{Major: 20, Minor: 2, Internal: 12}, }, { - Key: TruncatedAndRangeAppliedStateMigration, + Key: replacedTruncatedAndRangeAppliedStateMigration, Version: roachpb.Version{Major: 20, Minor: 2, Internal: 14}, }, { - Key: PostTruncatedAndRangeAppliedStateMigration, + Key: replacedPostTruncatedAndRangeAppliedStateMigration, Version: roachpb.Version{Major: 20, Minor: 2, Internal: 16}, }, { Key: NewSchemaChanger, Version: roachpb.Version{Major: 20, Minor: 2, Internal: 18}, }, + { + Key: LongRunningMigrations, + Version: roachpb.Version{Major: 20, Minor: 2, Internal: 20}, + }, + { + Key: TruncatedAndRangeAppliedStateMigration, + Version: roachpb.Version{Major: 20, Minor: 2, Internal: 22}, + }, + { + Key: PostTruncatedAndRangeAppliedStateMigration, + Version: roachpb.Version{Major: 20, Minor: 2, Internal: 24}, + }, // Step (2): Add new versions here. }) diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 140c4bc46ddd..4c84f52af97f 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -34,14 +34,17 @@ func _() { _ = x[VirtualComputedColumns-23] _ = x[CPutInline-24] _ = x[ReplicaVersions-25] - _ = x[TruncatedAndRangeAppliedStateMigration-26] - _ = x[PostTruncatedAndRangeAppliedStateMigration-27] + _ = x[replacedTruncatedAndRangeAppliedStateMigration-26] + _ = x[replacedPostTruncatedAndRangeAppliedStateMigration-27] _ = x[NewSchemaChanger-28] + _ = x[LongRunningMigrations-29] + _ = x[TruncatedAndRangeAppliedStateMigration-30] + _ = x[PostTruncatedAndRangeAppliedStateMigration-31] } -const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesNodeMembershipStatusMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesUniqueWithoutIndexConstraintsVirtualComputedColumnsCPutInlineReplicaVersionsTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationNewSchemaChanger" +const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesNodeMembershipStatusMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesUniqueWithoutIndexConstraintsVirtualComputedColumnsCPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationNewSchemaChangerLongRunningMigrationsTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigration" -var _Key_index = [...]uint16{0, 25, 34, 48, 53, 68, 90, 124, 145, 163, 180, 200, 217, 231, 295, 312, 321, 346, 366, 378, 383, 392, 420, 449, 471, 481, 496, 534, 576, 592} +var _Key_index = [...]uint16{0, 25, 34, 48, 53, 68, 90, 124, 145, 163, 180, 200, 217, 231, 295, 312, 321, 346, 366, 378, 383, 392, 420, 449, 471, 481, 496, 542, 592, 608, 629, 667, 709} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go index 150372342a4e..9fe58a24553f 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -399,6 +399,7 @@ const ( ScheduledJobsTableID = 37 TenantsRangesID = 38 // pseudo SqllivenessID = 39 + LongRunningMigrationsID = 40 // CommentType is type for system.comments DatabaseCommentType = 0 diff --git a/pkg/migration/BUILD.bazel b/pkg/migration/BUILD.bazel index 8a69c4c52a12..2176539d7db6 100644 --- a/pkg/migration/BUILD.bazel +++ b/pkg/migration/BUILD.bazel @@ -10,10 +10,12 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/clusterversion", + "//pkg/keys", "//pkg/kv", "//pkg/roachpb", "//pkg/security", "//pkg/server/serverpb", + "//pkg/settings/cluster", "//pkg/util/log", "@com_github_cockroachdb_logtags//:logtags", ], diff --git a/pkg/migration/migration.go b/pkg/migration/migration.go index fbef1ed7d3a3..04273868520e 100644 --- a/pkg/migration/migration.go +++ b/pkg/migration/migration.go @@ -26,19 +26,33 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/logtags" ) // Manager coordinates long-running migrations. type Manager interface { + RunDependencies Migrate(ctx context.Context, user security.SQLUsername, from, to clusterversion.ClusterVersion) error } -// Cluster abstracts a physical KV cluster and can be utilized by a long-runnng +// Registry provides access to migrations. +type Registry interface { + GetMigration(key clusterversion.ClusterVersion) (Migration, bool) +} + +// RunDependencies are used by the job to run migrations. +type RunDependencies interface { + Registry + Cluster() Cluster +} + +// Cluster abstracts a physical KV cluster and can be utilized by a long-running // migration. type Cluster interface { @@ -182,6 +196,10 @@ type KVMigration struct { fn KVMigrationFn } +// KVMigrationFn is used to perform kv-level migrations. It should only be +// run from the system tenant. +type KVMigrationFn func(context.Context, clusterversion.ClusterVersion, Cluster) error + // NewKVMigration constructs a KVMigration. func NewKVMigration( description string, cv clusterversion.ClusterVersion, fn KVMigrationFn, @@ -195,21 +213,47 @@ func NewKVMigration( } } -// KVMigrationFn contains the logic of a KVMigration. -type KVMigrationFn func(context.Context, clusterversion.ClusterVersion, Cluster) error +// SQLDeps are the dependencies of migrations which perform actions at the +// SQL layer. +type SQLDeps struct { + DB *kv.DB + Codec keys.SQLCodec + Settings *cluster.Settings +} -// Run kickstarts the actual migration process. It's responsible for recording -// the ongoing status of the migration into a system table. -// -// TODO(irfansharif): Introduce a `system.migrations` table, and populate it here. -func (m *KVMigration) Run( - ctx context.Context, cv clusterversion.ClusterVersion, h Cluster, -) (err error) { - ctx = logtags.AddTag(ctx, fmt.Sprintf("migration=%s", cv), nil) +// SQLMigration is an implementation of Migration for SQL-level migrations. +type SQLMigration struct { + migration + fn SQLMigrationFn +} - if err := m.fn(ctx, cv, h); err != nil { - return err +// NewSQLMigration constructs a SQLMigration. +func NewSQLMigration( + description string, cv clusterversion.ClusterVersion, fn SQLMigrationFn, +) *SQLMigration { + return &SQLMigration{ + migration: migration{ + description: description, + cv: cv, + }, + fn: fn, } +} - return nil +// SQLMigrationFn is used to perform sql-level migrations. It may be run from +// any tenant. +type SQLMigrationFn func(context.Context, clusterversion.ClusterVersion, SQLDeps) error + +// Run kickstarts the actual migration process for KV-level migrations. +func (m *KVMigration) Run(ctx context.Context, cv clusterversion.ClusterVersion, h Cluster) error { + ctx = logtags.AddTag(ctx, fmt.Sprintf("migration=%s", cv), nil) + return m.fn(ctx, cv, h) +} + +// Run kickstarts the actual migration process for SQL-level migrations. +func (m *SQLMigration) Run( + ctx context.Context, cv clusterversion.ClusterVersion, d SQLDeps, +) (err error) { + ctx = logtags.AddTag(ctx, fmt.Sprintf("migration=%s", cv), nil) + return m.fn(ctx, cv, d) } diff --git a/pkg/migration/migrationjob/BUILD.bazel b/pkg/migration/migrationjob/BUILD.bazel index f05f934286c0..8b7717c32bae 100644 --- a/pkg/migration/migrationjob/BUILD.bazel +++ b/pkg/migration/migrationjob/BUILD.bazel @@ -6,11 +6,18 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/migration/migrationjob", visibility = ["//visibility:public"], deps = [ + "//pkg/clusterversion", "//pkg/jobs", "//pkg/jobs/jobspb", + "//pkg/kv", "//pkg/migration", - "//pkg/migration/migrations", + "//pkg/security", "//pkg/settings/cluster", "//pkg/sql", + "//pkg/sql/sem/tree", + "//pkg/sql/sessiondata", + "//pkg/sql/sqlutil", + "//pkg/util/timeutil", + "@com_github_cockroachdb_errors//:errors", ], ) diff --git a/pkg/migration/migrationjob/migration_job.go b/pkg/migration/migrationjob/migration_job.go index ab0702a5117e..4e3926e725cf 100644 --- a/pkg/migration/migrationjob/migration_job.go +++ b/pkg/migration/migrationjob/migration_job.go @@ -15,12 +15,19 @@ package migrationjob 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/kv" "github.com/cockroachdb/cockroach/pkg/migration" - "github.com/cockroachdb/cockroach/pkg/migration/migrations" + "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" ) func init() { @@ -29,6 +36,19 @@ func init() { }) } +// NewRecord constructs a new jobs.Record for this migration. +func NewRecord(version clusterversion.ClusterVersion, user security.SQLUsername) jobs.Record { + return jobs.Record{ + Description: "Long-running migration", + Details: jobspb.LongRunningMigrationDetails{ + ClusterVersion: &version, + }, + Username: user, + Progress: jobspb.LongRunningMigrationProgress{}, + NonCancelable: true, + } +} + type resumer struct { j *jobs.Job } @@ -36,15 +56,101 @@ type resumer struct { var _ jobs.Resumer = (*resumer)(nil) func (r resumer) Resume(ctx context.Context, execCtxI interface{}) error { - // TODO(ajwerner): add some check to see if we're done. + execCtx := execCtxI.(sql.JobExecContext) pl := r.j.Payload() cv := *pl.GetLongRunningMigration().ClusterVersion - m, ok := migrations.GetMigration(cv) + ie := execCtx.ExecCfg().InternalExecutor + + alreadyCompleted, err := CheckIfMigrationCompleted(ctx, nil /* txn */, ie, cv) + if alreadyCompleted || err != nil { + return errors.Wrapf(err, "checking migration completion for %v", cv) + } + mc := execCtx.MigrationRunDependencies() + m, ok := mc.GetMigration(cv) if !ok { + // TODO(ajwerner): Consider treating this as an assertion failure. Jobs + // should only be created for a cluster version if there is an associated + // migration. It seems possible that a migration job could be launched by + // a node running a older version where a migration then runs on a job + // with a newer version where the migration has been re-ordered to be later. + // This should only happen between alphas but is theoretically not illegal. return nil } - return m.(*migration.KVMigration).Run(ctx, cv, execCtx.MigrationCluster()) + switch m := m.(type) { + case *migration.KVMigration: + err = m.Run(ctx, cv, mc.Cluster()) + case *migration.SQLMigration: + err = m.Run(ctx, cv, migration.SQLDeps{ + DB: execCtx.ExecCfg().DB, + Codec: execCtx.ExecCfg().Codec, + Settings: execCtx.ExecCfg().Settings, + }) + default: + return errors.AssertionFailedf("unknown migration type %T", m) + } + if err != nil { + return errors.Wrapf(err, "running migration for %v", cv) + } + + // Mark the migration as having been completed so that subsequent iterations + // no-op and new jobs are not created. + return errors.Wrapf(markMigrationCompleted(ctx, ie, cv), + "marking migration complete for %v", cv) +} + +// CheckIfMigrationCompleted queries the system.long_running_migrations table +// to determine if the migration associated with this version has already been +// completed. The txn may be nil, in which case the check will be run in its +// own transaction. +func CheckIfMigrationCompleted( + ctx context.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + version clusterversion.ClusterVersion, +) (alreadyCompleted bool, _ error) { + row, err := ie.QueryRow(ctx, "migration-manager-find-already-completed", txn, ` +SELECT EXISTS( + SELECT * + FROM system.long_running_migrations + WHERE major = $1 + AND minor = $2 + AND patch = $3 + AND internal = $4 + ); +`, + version.Major, version.Minor, version.Patch, version.Internal) + if err != nil { + return false, err + } + return bool(*row[0].(*tree.DBool)), nil +} + +func markMigrationCompleted( + ctx context.Context, ie sqlutil.InternalExecutor, cv clusterversion.ClusterVersion, +) error { + _, err := ie.ExecEx( + ctx, + "migration-job-mark-job-succeeded", + nil, /* txn */ + sessiondata.NodeUserSessionDataOverride, + ` +INSERT + INTO system.long_running_migrations + ( + major, + minor, + patch, + internal, + completed_at + ) +VALUES ($1, $2, $3, $4, $5)`, + cv.Major, + cv.Minor, + cv.Patch, + cv.Internal, + timeutil.Now()) + return err } // The long-running migration resumer has no reverting logic. diff --git a/pkg/migration/migrationmanager/BUILD.bazel b/pkg/migration/migrationmanager/BUILD.bazel index 95a418de34b0..14848caa9dfc 100644 --- a/pkg/migration/migrationmanager/BUILD.bazel +++ b/pkg/migration/migrationmanager/BUILD.bazel @@ -2,19 +2,24 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "migrationmanager", - srcs = ["manager.go"], + srcs = [ + "manager.go", + "testing_knobs.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/migration/migrationmanager", visibility = ["//visibility:public"], deps = [ + "//pkg/base", "//pkg/clusterversion", "//pkg/jobs", - "//pkg/jobs/jobspb", + "//pkg/keys", "//pkg/kv", "//pkg/migration", "//pkg/migration/migrationjob", "//pkg/migration/migrations", "//pkg/security", "//pkg/server/serverpb", + "//pkg/settings/cluster", "//pkg/sql/protoreflect", "//pkg/sql/sem/tree", "//pkg/sql/sqlutil", @@ -32,13 +37,13 @@ go_test( "manager_external_test.go", ], deps = [ + ":migrationmanager", "//pkg/base", "//pkg/clusterversion", "//pkg/jobs", "//pkg/kv/kvserver/batcheval", "//pkg/kv/kvserver/liveness", "//pkg/migration", - "//pkg/migration/migrations", "//pkg/roachpb", "//pkg/security", "//pkg/security/securitytest", @@ -48,10 +53,12 @@ go_test( "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/testcluster", + "//pkg/util", "//pkg/util/leaktest", "//pkg/util/log", "//pkg/util/tracing", "@com_github_cockroachdb_errors//:errors", "@com_github_stretchr_testify//require", + "@org_golang_x_sync//errgroup", ], ) diff --git a/pkg/migration/migrationmanager/manager.go b/pkg/migration/migrationmanager/manager.go index 127d1aaaa7e0..279be5b858a0 100644 --- a/pkg/migration/migrationmanager/manager.go +++ b/pkg/migration/migrationmanager/manager.go @@ -18,13 +18,14 @@ import ( "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/kv" "github.com/cockroachdb/cockroach/pkg/migration" - _ "github.com/cockroachdb/cockroach/pkg/migration/migrationjob" + "github.com/cockroachdb/cockroach/pkg/migration/migrationjob" "github.com/cockroachdb/cockroach/pkg/migration/migrations" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/protoreflect" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" @@ -37,20 +38,56 @@ import ( // Manager is the instance responsible for executing migrations across the // cluster. type Manager struct { - c migration.Cluster - ie sqlutil.InternalExecutor - jr *jobs.Registry + c migration.Cluster + ie sqlutil.InternalExecutor + jr *jobs.Registry + codec keys.SQLCodec + settings *cluster.Settings + knobs TestingKnobs } -// NewManager constructs a new Manager. -func NewManager(c migration.Cluster, ie sqlutil.InternalExecutor, jr *jobs.Registry) *Manager { +// GetMigration returns the migration associated with this key. +func (m *Manager) GetMigration(key clusterversion.ClusterVersion) (migration.Migration, bool) { + if m.knobs.RegistryOverride != nil { + if m, ok := m.knobs.RegistryOverride(key); ok { + return m, ok + } + } + return migrations.GetMigration(key) +} + +// Cluster returns the cluster associated with this manager. It may be nil +// in a secondary tenant. +func (m *Manager) Cluster() migration.Cluster { + return m.c +} + +// NewManager constructs a new Manager. The Cluster parameter may be nil in +// secondary tenants. The testingKnobs parameter may be nil. +func NewManager( + c migration.Cluster, + ie sqlutil.InternalExecutor, + jr *jobs.Registry, + codec keys.SQLCodec, + settings *cluster.Settings, + testingKnobs *TestingKnobs, +) *Manager { + var knobs TestingKnobs + if testingKnobs != nil { + knobs = *testingKnobs + } return &Manager{ - c: c, - ie: ie, - jr: jr, + c: c, + ie: ie, + jr: jr, + codec: codec, + settings: settings, + knobs: knobs, } } +var _ migration.Manager = (*Manager)(nil) + // Migrate runs the set of migrations required to upgrade the cluster version // from the current version to the target one. func (m *Manager) Migrate( @@ -65,24 +102,7 @@ func (m *Manager) Migrate( return nil } - // TODO(irfansharif): We'll need to acquire a lease here and refresh it - // throughout during the migration to ensure mutual exclusion. - - // TODO(irfansharif): We'll need to create a system table to store - // in-progress state of long running migrations, for introspection. - - clusterVersions := clusterversion.ListBetween(from, to) - if len(clusterVersions) == 0 { - // We're attempt to migrate to something that's not defined in cluster - // versions. This only happens in tests, when we're exercising version - // upgrades over non-existent versions (like in the cluster_version - // logictest). These tests explicitly override the - // binary{,MinSupportedVersion} in order to work. End-user attempts to - // do something similar would be caught at the sql layer (also tested in - // the same logictest). We'll just explicitly append the target version - // here instead, so that we're able to actually migrate into it. - clusterVersions = append(clusterVersions, to) - } + clusterVersions := m.listBetween(from, to) log.Infof(ctx, "migrating cluster from %s to %s (stepping through %s)", from, to, clusterVersions) for _, clusterVersion := range clusterVersions { @@ -218,11 +238,25 @@ func (m *Manager) Migrate( func (m *Manager) runMigration( ctx context.Context, user security.SQLUsername, version clusterversion.ClusterVersion, ) error { - if _, exists := migrations.GetMigration(version); !exists { + mig, exists := m.GetMigration(version) + if !exists { return nil } - id, err := m.getOrCreateMigrationJob(ctx, user, version) - if err != nil { + // The migration which introduces the infrastructure for running other long + // running migrations in jobs. It needs to be special-cased and run without + // a job or leasing for bootstrapping purposes. Fortunately it has been + // designed to be idempotent and cheap. + // + // TODO(ajwerner): Remove in 21.2. + if version.Version == clusterversion.ByKey(clusterversion.LongRunningMigrations) { + return mig.(*migration.SQLMigration).Run(ctx, version, migration.SQLDeps{ + DB: m.c.DB(), + Codec: m.codec, + Settings: m.settings, + }) + } + alreadyCompleted, id, err := m.getOrCreateMigrationJob(ctx, user, version) + if alreadyCompleted || err != nil { return err } return m.jr.Run(ctx, m.ie, []int64{id}) @@ -230,9 +264,12 @@ func (m *Manager) runMigration( func (m *Manager) getOrCreateMigrationJob( ctx context.Context, user security.SQLUsername, version clusterversion.ClusterVersion, -) (jobID int64, _ error) { - +) (alreadyCompleted bool, jobID int64, _ error) { if err := m.c.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + alreadyCompleted, err = migrationjob.CheckIfMigrationCompleted(ctx, txn, m.ie, version) + if alreadyCompleted || err != nil { + return err + } var found bool found, jobID, err = m.getRunningMigrationJob(ctx, txn, version) if err != nil { @@ -242,24 +279,16 @@ func (m *Manager) getOrCreateMigrationJob( return nil } var j *jobs.Job - j, err = m.jr.CreateJobWithTxn(ctx, jobs.Record{ - Description: "Long running migration", - Details: jobspb.LongRunningMigrationDetails{ - ClusterVersion: &version, - }, - Username: user, - Progress: jobspb.LongRunningMigrationProgress{}, - NonCancelable: true, - }, txn) + j, err = m.jr.CreateJobWithTxn(ctx, migrationjob.NewRecord(version, user), txn) if err != nil { return err } jobID = *j.ID() return nil }); err != nil { - return 0, err + return false, 0, err } - return jobID, nil + return alreadyCompleted, jobID, nil } func (m *Manager) getRunningMigrationJob( @@ -272,14 +301,14 @@ SELECT id, status status, crdb_internal.pb_to_json( 'cockroach.sql.jobs.jobspb.Payload', - payload + payload, + false -- emit_defaults ) AS pl FROM system.jobs WHERE status IN ` + jobs.NonTerminalStatusTupleString + ` ) WHERE pl->'longRunningMigration'->'clusterVersion' = $1::JSON;` - // TODO(ajwerner): Flip the emitDefaults flag once this is rebased on master. - jsonMsg, err := protoreflect.MessageToJSON(&version, true /* emitDefaults */) + jsonMsg, err := protoreflect.MessageToJSON(&version, false /* emitDefaults */) if err != nil { return false, 0, errors.Wrap(err, "failed to marshal version to JSON") } @@ -312,3 +341,24 @@ SELECT id, status return false, 0, errors.AssertionFailedf("%s", buf) } } + +func (m *Manager) listBetween( + from clusterversion.ClusterVersion, to clusterversion.ClusterVersion, +) []clusterversion.ClusterVersion { + if m.knobs.ListBetweenOverride != nil { + return m.knobs.ListBetweenOverride(from, to) + } + versions := clusterversion.ListBetween(from, to) + if len(versions) == 0 { + // We're attempt to migrate to something that's not defined in cluster + // versions. This only happens in tests, when we're exercising version + // upgrades over non-existent versions (like in the cluster_version + // logictest). These tests explicitly override the + // binary{,MinSupportedVersion} in order to work. End-user attempts to + // do something similar would be caught at the sql layer (also tested in + // the same logictest). We'll just explicitly append the target version + // here instead, so that we're able to actually migrate into it. + versions = []clusterversion.ClusterVersion{to} + } + return versions +} diff --git a/pkg/migration/migrationmanager/manager_external_test.go b/pkg/migration/migrationmanager/manager_external_test.go index 56ab855d4ce0..516f94ffa05d 100644 --- a/pkg/migration/migrationmanager/manager_external_test.go +++ b/pkg/migration/migrationmanager/manager_external_test.go @@ -12,7 +12,10 @@ package migrationmanager_test import ( "context" + gosql "database/sql" + "sync/atomic" "testing" + "time" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/clusterversion" @@ -20,18 +23,20 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/migration" - "github.com/cockroachdb/cockroach/pkg/migration/migrations" + "github.com/cockroachdb/cockroach/pkg/migration/migrationmanager" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" + "golang.org/x/sync/errgroup" ) // TestAlreadyRunningJobsAreHandledProperly is a relatively low-level test to @@ -49,6 +54,8 @@ func TestAlreadyRunningJobsAreHandledProperly(t *testing.T) { startCV := clusterversion.ClusterVersion{Version: roachpb.Version{Major: 41}} endCV := clusterversion.ClusterVersion{Version: roachpb.Version{Major: 42}} + ch := make(chan chan error) + ctx := context.Background() tc := testcluster.StartTestCluster(t, 2, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, @@ -59,19 +66,28 @@ func TestAlreadyRunningJobsAreHandledProperly(t *testing.T) { BinaryVersionOverride: startCV.Version, DisableAutomaticVersionUpgrade: 1, }, + MigrationManager: &migrationmanager.TestingKnobs{ + ListBetweenOverride: func(from, to clusterversion.ClusterVersion) []clusterversion.ClusterVersion { + return []clusterversion.ClusterVersion{to} + }, + RegistryOverride: func(cv clusterversion.ClusterVersion) (migration.Migration, bool) { + if cv != endCV { + return nil, false + } + return migration.NewSQLMigration("test", cv, func( + ctx context.Context, version clusterversion.ClusterVersion, deps migration.SQLDeps, + ) error { + canResume := make(chan error) + ch <- canResume + return <-canResume + }), true + }, + }, }, }, }) defer tc.Stopper().Stop(ctx) - ch := make(chan chan error) - defer migrations.TestingRegisterMigrationInterceptor(endCV, func( - ctx context.Context, cv clusterversion.ClusterVersion, h migration.Cluster, - ) error { - canResume := make(chan error) - ch <- canResume - return <-canResume - })() upgrade1Err := make(chan error, 1) go func() { _, err := tc.ServerConn(0).ExecContext(ctx, `SET CLUSTER SETTING version = $1`, endCV.String()) @@ -158,6 +174,7 @@ func TestMigrateUpdatesReplicaVersion(t *testing.T) { startCV := clusterversion.ClusterVersion{Version: roachpb.Version{Major: 41}} endCV := clusterversion.ClusterVersion{Version: roachpb.Version{Major: 42}} + var desc roachpb.RangeDescriptor ctx := context.Background() tc := testcluster.StartTestCluster(t, 2, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, @@ -168,10 +185,28 @@ func TestMigrateUpdatesReplicaVersion(t *testing.T) { BinaryVersionOverride: startCV.Version, DisableAutomaticVersionUpgrade: 1, }, + MigrationManager: &migrationmanager.TestingKnobs{ + ListBetweenOverride: func(from, to clusterversion.ClusterVersion) []clusterversion.ClusterVersion { + return []clusterversion.ClusterVersion{from, to} + }, + RegistryOverride: func(cv clusterversion.ClusterVersion) (migration.Migration, bool) { + if cv != endCV { + return nil, false + } + return migration.NewKVMigration("test", cv, func( + ctx context.Context, version clusterversion.ClusterVersion, c migration.Cluster, + ) error { + return c.DB().Migrate(ctx, desc.StartKey, desc.EndKey, cv.Version) + }), true + }, + }, }, }, }) defer tc.Stopper().Stop(ctx) + // RegisterKVMigration the below raft migration. + unregisterKVMigration := batcheval.TestingRegisterMigrationInterceptor(endCV.Version, func() {}) + defer unregisterKVMigration() // We'll take a specific range, still running at startCV, generate an // outgoing snapshot and then suspend it temporarily. We'll then bump the @@ -183,7 +218,8 @@ func TestMigrateUpdatesReplicaVersion(t *testing.T) { key := tc.ScratchRange(t) require.NoError(t, tc.WaitForSplitAndInitialization(key)) - desc, err := tc.LookupRange(key) + var err error + desc, err = tc.LookupRange(key) require.NoError(t, err) rangeID := desc.RangeID @@ -198,18 +234,6 @@ func TestMigrateUpdatesReplicaVersion(t *testing.T) { t.Fatalf("got replica version %s, expected %s", got, startCV.Version) } - // Register the below raft migration. - unregisterKVMigration := batcheval.TestingRegisterMigrationInterceptor(endCV.Version, func() {}) - defer unregisterKVMigration() - - // Register the top-level migration. - unregister := migrations.TestingRegisterMigrationInterceptor(endCV, func( - ctx context.Context, cv clusterversion.ClusterVersion, c migration.Cluster, - ) error { - return c.DB().Migrate(ctx, desc.StartKey, desc.EndKey, cv.Version) - }) - defer unregister() - // Wait until all nodes have are considered live. nl := tc.Server(0).NodeLiveness().(*liveness.NodeLiveness) testutils.SucceedsSoon(t, func() error { @@ -234,3 +258,102 @@ func TestMigrateUpdatesReplicaVersion(t *testing.T) { t.Fatalf("got replica version %s, expected %s", got, endCV.Version) } } + +// TestConcurrentMigrationAttempts ensures that concurrent attempts to run +// migrations over a number of versions exhibits reasonable behavior. Namely, +// that each migration gets run one time and that migrations do not get run +// again. +func TestConcurrentMigrationAttempts(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + // We're going to be migrating from startKey to endKey. We end up needing + // to use real versions because the ListBetween uses the keys compiled into + // the clusterversion package. + const ( + startMajor = 42 + endMajor = 48 + ) + migrationRunCounts := make(map[clusterversion.ClusterVersion]int) + + // RegisterKVMigration the migrations to update the map with run counts. + // There should definitely not be any concurrency of execution, so the race + // detector should not fire. + var versions []clusterversion.ClusterVersion + + for major := int32(startMajor); major <= endMajor; major++ { + versions = append(versions, clusterversion.ClusterVersion{ + Version: roachpb.Version{ + Major: major, + }, + }) + } + ctx := context.Background() + var active int32 // used to detect races + tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: cluster.MakeTestingClusterSettingsWithVersions( + versions[len(versions)-1].Version, + versions[0].Version, + false, + ), + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + BinaryVersionOverride: versions[0].Version, + DisableAutomaticVersionUpgrade: 1, + }, + MigrationManager: &migrationmanager.TestingKnobs{ + ListBetweenOverride: func(from, to clusterversion.ClusterVersion) []clusterversion.ClusterVersion { + return versions + }, + RegistryOverride: func(cv clusterversion.ClusterVersion) (migration.Migration, bool) { + return migration.NewKVMigration("test", cv, func( + ctx context.Context, version clusterversion.ClusterVersion, c migration.Cluster, + ) error { + if atomic.AddInt32(&active, 1) != 1 { + t.Error("unexpected concurrency") + } + time.Sleep(time.Millisecond) + atomic.AddInt32(&active, -1) + migrationRunCounts[version]++ + return nil + }), true + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + // Run N instances of the migration concurrently on different connections. + // They should all eventually succeed; some may internally experience a + // serializable restart but cockroach will handle that transparently. + // Afterwards we'll ensure that no migration was run more than once. + N := 25 + if util.RaceEnabled { + N = 5 + } + db := tc.ServerConn(0) + db.SetMaxOpenConns(N) + conns := make([]*gosql.Conn, N) + for i := range conns { + var err error + conns[i], err = db.Conn(ctx) + require.NoError(t, err) + } + var g errgroup.Group + for i := 0; i < N; i++ { + conn := conns[i] + g.Go(func() error { + _, err := conn.ExecContext(ctx, `SET CLUSTER SETTING version = $1`, + versions[len(versions)-1].String()) + return err + }) + } + require.Nil(t, g.Wait()) + for k, c := range migrationRunCounts { + require.Equalf(t, 1, c, "version: %v", k) + } + require.Len(t, migrationRunCounts, len(versions)) +} diff --git a/pkg/migration/migrationmanager/testing_knobs.go b/pkg/migration/migrationmanager/testing_knobs.go new file mode 100644 index 000000000000..669aec7b1123 --- /dev/null +++ b/pkg/migration/migrationmanager/testing_knobs.go @@ -0,0 +1,35 @@ +// Copyright 2021 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 migrationmanager + +import ( + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/migration" +) + +// TestingKnobs are knobs to inject behavior into the migration manager which +// are useful for testing. +type TestingKnobs struct { + + // ListBetweenOverride injects an override for `clusterversion.ListBetween() + // in order to run migrations corresponding to versions which do not + // actually exist. + ListBetweenOverride func(from, to clusterversion.ClusterVersion) []clusterversion.ClusterVersion + + // RegistryOverride is used to inject migrations for specific cluster versions. + RegistryOverride func(cv clusterversion.ClusterVersion) (migration.Migration, bool) +} + +// ModuleTestingKnobs makes TestingKnobs a base.ModuleTestingKnobs. +func (t *TestingKnobs) ModuleTestingKnobs() {} + +var _ base.ModuleTestingKnobs = (*TestingKnobs)(nil) diff --git a/pkg/migration/migrations/BUILD.bazel b/pkg/migration/migrations/BUILD.bazel index 13aac289d90e..8f4252457e02 100644 --- a/pkg/migration/migrations/BUILD.bazel +++ b/pkg/migration/migrations/BUILD.bazel @@ -3,6 +3,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "migrations", srcs = [ + "long_running_migrations.go", "migrations.go", "truncated_state.go", ], @@ -14,6 +15,8 @@ go_library( "//pkg/migration", "//pkg/roachpb", "//pkg/server/serverpb", + "//pkg/sql/catalog/systemschema", + "//pkg/sqlmigrations", "//pkg/util/log", ], ) diff --git a/pkg/migration/migrations/long_running_migrations.go b/pkg/migration/migrations/long_running_migrations.go new file mode 100644 index 000000000000..e2601f133a44 --- /dev/null +++ b/pkg/migration/migrations/long_running_migrations.go @@ -0,0 +1,32 @@ +// Copyright 2021 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 migrations + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/migration" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/sqlmigrations" +) + +func init() { + +} + +func longRunningMigrationsTableMigration( + ctx context.Context, _ clusterversion.ClusterVersion, d migration.SQLDeps, +) error { + return sqlmigrations.CreateSystemTable( + ctx, d.DB, d.Codec, d.Settings, systemschema.LongRunningMigrationsTable, + ) +} diff --git a/pkg/migration/migrations/migrations.go b/pkg/migration/migrations/migrations.go index 353b6e5726dd..6a4154bb2427 100644 --- a/pkg/migration/migrations/migrations.go +++ b/pkg/migration/migrations/migrations.go @@ -15,28 +15,10 @@ package migrations import ( - "context" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/migration" - "github.com/cockroachdb/cockroach/pkg/util/log" ) -// registry defines the global mapping between a cluster version and the -// associated migration. The migration is only executed after a cluster-wide -// bump of the corresponding version gate. -var registry = make(map[clusterversion.ClusterVersion]migration.Migration) - -// register is a short hand to register a given migration within the global -// registry. -func register(key clusterversion.Key, fn migration.KVMigrationFn, desc string) { - cv := clusterversion.ClusterVersion{Version: clusterversion.ByKey(key)} - if _, ok := registry[cv]; ok { - log.Fatalf(context.Background(), "doubly registering migration for %s", cv) - } - registry[cv] = migration.NewKVMigration(desc, cv, fn) -} - // GetMigration returns the migration corresponding to this version if // one exists. func GetMigration(key clusterversion.ClusterVersion) (migration.Migration, bool) { @@ -44,38 +26,37 @@ func GetMigration(key clusterversion.ClusterVersion) (migration.Migration, bool) return m, ok } -// TestingRegisterMigrationInterceptor is used in tests to register an -// interceptor for a version migration. -// -// TODO(irfansharif): This is a gross anti-pattern, we're letting tests mutate -// global state. This should instead be a testing knob that the migration -// manager checks when search for attached migrations. -func TestingRegisterMigrationInterceptor( - cv clusterversion.ClusterVersion, fn migration.KVMigrationFn, -) (unregister func()) { - registry[cv] = migration.NewKVMigration("", cv, fn) - return func() { delete(registry, cv) } -} +// registry defines the global mapping between a cluster version and the +// associated migration. The migration is only executed after a cluster-wide +// bump of the corresponding version gate. +var registry = make(map[clusterversion.ClusterVersion]migration.Migration) -var kvMigrations = []struct { - cv clusterversion.Key - fn migration.KVMigrationFn - description string -}{ - { - clusterversion.TruncatedAndRangeAppliedStateMigration, - truncatedStateMigration, +var migrations = []migration.Migration{ + migration.NewSQLMigration( + "add the system.long_running_migrations table", + toCV(clusterversion.LongRunningMigrations), + longRunningMigrationsTableMigration, + ), + migration.NewKVMigration( "use unreplicated TruncatedState and RangeAppliedState for all ranges", - }, - { - clusterversion.PostTruncatedAndRangeAppliedStateMigration, - postTruncatedStateMigration, + toCV(clusterversion.TruncatedAndRangeAppliedStateMigration), + truncatedStateMigration, + ), + migration.NewKVMigration( "purge all replicas using the replicated TruncatedState", - }, + toCV(clusterversion.PostTruncatedAndRangeAppliedStateMigration), + postTruncatedStateMigration, + ), } func init() { - for _, m := range kvMigrations { - register(m.cv, m.fn, m.description) + for _, m := range migrations { + registry[m.ClusterVersion()] = m + } +} + +func toCV(key clusterversion.Key) clusterversion.ClusterVersion { + return clusterversion.ClusterVersion{ + Version: clusterversion.ByKey(key), } } diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 47e0db41aac7..01ef51b688d2 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -76,7 +76,6 @@ go_library( "//pkg/kv/kvserver/reports", "//pkg/migration/migrationcluster", "//pkg/migration/migrationmanager", - "//pkg/migration/migrations", "//pkg/roachpb", "//pkg/rpc", "//pkg/rpc/nodedialer", diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index ec98447d327c..366e33c43c66 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -34,7 +34,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/migration/migrationcluster" "github.com/cockroachdb/cockroach/pkg/migration/migrationmanager" - _ "github.com/cockroachdb/cockroach/pkg/migration/migrations" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" @@ -627,13 +626,20 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { // We only need to attach a version upgrade hook if we're the system // tenant. Regular tenants are disallowed from changing cluster // versions. + // + // TODO(ajwerner): Allow tenants to set their cluster version and to + // perform sql migrations through the migration infrastructure. + // See #48436. c := migrationcluster.New(migrationcluster.ClusterConfig{ NodeLiveness: nodeLiveness, Dialer: cfg.nodeDialer, DB: cfg.db, }) - migrationMgr := migrationmanager.NewManager(c, cfg.circularInternalExecutor, jobRegistry) - execCfg.MigrationCluster = c + knobs, _ := cfg.TestingKnobs.MigrationManager.(*migrationmanager.TestingKnobs) + migrationMgr := migrationmanager.NewManager( + c, cfg.circularInternalExecutor, jobRegistry, codec, cfg.Settings, knobs, + ) + execCfg.MigrationRunDependencies = migrationMgr execCfg.VersionUpgradeHook = migrationMgr.Migrate } diff --git a/pkg/sql/catalog/bootstrap/metadata.go b/pkg/sql/catalog/bootstrap/metadata.go index 7818715e16f8..4c8bbedd5743 100644 --- a/pkg/sql/catalog/bootstrap/metadata.go +++ b/pkg/sql/catalog/bootstrap/metadata.go @@ -342,6 +342,7 @@ func addSystemDescriptorsToSchema(target *MetadataSchema) { target.AddDescriptor(keys.SystemDatabaseID, systemschema.ScheduledJobsTable) target.AddDescriptor(keys.SystemDatabaseID, systemschema.SqllivenessTable) + target.AddDescriptor(keys.SystemDatabaseID, systemschema.LongRunningMigrationsTable) } // addSplitIDs adds a split point for each of the PseudoTableIDs to the supplied diff --git a/pkg/sql/catalog/descpb/privilege.go b/pkg/sql/catalog/descpb/privilege.go index f59406574fca..a925f3823119 100644 --- a/pkg/sql/catalog/descpb/privilege.go +++ b/pkg/sql/catalog/descpb/privilege.go @@ -435,6 +435,7 @@ var SystemAllowedPrivileges = map[ID]privilege.List{ keys.StatementDiagnosticsTableID: privilege.ReadWriteData, keys.ScheduledJobsTableID: privilege.ReadWriteData, keys.SqllivenessID: privilege.ReadWriteData, + keys.LongRunningMigrationsID: privilege.ReadWriteData, } // SetOwner sets the owner of the privilege descriptor to the provided string. diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go index 5adbb79e2b60..745e5165c7ec 100644 --- a/pkg/sql/catalog/systemschema/system.go +++ b/pkg/sql/catalog/systemschema/system.go @@ -345,6 +345,17 @@ CREATE TABLE system.sqlliveness ( expiration DECIMAL NOT NULL, FAMILY fam0_session_id_expiration (session_id, expiration) )` + + LongRunningMigrationsTableSchema = ` +CREATE TABLE system.long_running_migrations ( + major INT8 NOT NULL, + minor INT8 NOT NULL, + patch INT8 NOT NULL, + internal INT8 NOT NULL, + completed_at TIMESTAMPTZ NOT NULL, + FAMILY "primary" (major, minor, patch, internal, completed_at), + PRIMARY KEY (major, minor, patch, internal) +)` ) func pk(name string) descpb.IndexDescriptor { @@ -1690,6 +1701,55 @@ var ( FormatVersion: descpb.InterleavedFormatVersion, NextMutationID: 1, }) + + // LongRunningMigrationsTable is the descriptor for the + // long_running_migrations table. It stores facts about the completion state + // of long-running migrations. It is used to prevent migrations from running + // again after they have been completed. + LongRunningMigrationsTable = tabledesc.NewImmutable(descpb.TableDescriptor{ + Name: "long_running_migrations", + ID: keys.LongRunningMigrationsID, + ParentID: keys.SystemDatabaseID, + UnexposedParentSchemaID: keys.PublicSchemaID, + Version: 1, + Columns: []descpb.ColumnDescriptor{ + {Name: "major", ID: 1, Type: types.Int, Nullable: false}, + {Name: "minor", ID: 2, Type: types.Int, Nullable: false}, + {Name: "patch", ID: 3, Type: types.Int, Nullable: false}, + {Name: "internal", ID: 4, Type: types.Int, Nullable: false}, + {Name: "completed_at", ID: 5, Type: types.TimestampTZ, Nullable: false}, + }, + NextColumnID: 6, + Families: []descpb.ColumnFamilyDescriptor{ + { + Name: "primary", + ID: 0, + ColumnNames: []string{"major", "minor", "patch", "internal", "completed_at"}, + ColumnIDs: []descpb.ColumnID{1, 2, 3, 4, 5}, + DefaultColumnID: 5, + }, + }, + NextFamilyID: 1, + PrimaryIndex: descpb.IndexDescriptor{ + Name: tabledesc.PrimaryKeyIndexName, + ID: 1, + Unique: true, + ColumnNames: []string{"major", "minor", "patch", "internal"}, + ColumnDirections: []descpb.IndexDescriptor_Direction{ + descpb.IndexDescriptor_ASC, + descpb.IndexDescriptor_ASC, + descpb.IndexDescriptor_ASC, + descpb.IndexDescriptor_ASC, + }, + ColumnIDs: []descpb.ColumnID{1, 2, 3, 4}, + Version: descpb.EmptyArraysInInvertedIndexesVersion, + }, + NextIndexID: 2, + Privileges: descpb.NewCustomSuperuserPrivilegeDescriptor( + descpb.SystemAllowedPrivileges[keys.JobsTableID], security.NodeUserName()), + FormatVersion: descpb.InterleavedFormatVersion, + NextMutationID: 1, + }) ) // newCommentPrivilegeDescriptor returns a privilege descriptor for comment table diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 39448a162bff..f02c621d411a 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -802,9 +802,10 @@ type ExecutorConfig struct { // TODO(tbg,irfansharif,ajwerner): Hook up for secondary tenants. VersionUpgradeHook func(ctx context.Context, user security.SQLUsername, from, to clusterversion.ClusterVersion) error - // MigrationCluster is used to drive KV-level, long-running migrations. - // It will only be populated on the system tenant. - MigrationCluster migration.Cluster + // MigrationRunDependencies is used to drive migrations. + // + // TODO(tbg,irfansharif,ajwerner): Hook up for secondary tenants. + MigrationRunDependencies migration.RunDependencies // IndexBackfiller is used to backfill indexes. It is another rather circular // object which mostly just holds on to an ExecConfig. diff --git a/pkg/sql/job_exec_context.go b/pkg/sql/job_exec_context.go index 024a9af47e95..f59a7b8e26cb 100644 --- a/pkg/sql/job_exec_context.go +++ b/pkg/sql/job_exec_context.go @@ -42,11 +42,13 @@ func (e *plannerJobExecContext) ExtendedEvalContext() *extendedEvalContext { func (e *plannerJobExecContext) SessionData() *sessiondata.SessionData { return e.p.SessionData() } -func (e *plannerJobExecContext) ExecCfg() *ExecutorConfig { return e.p.ExecCfg() } -func (e *plannerJobExecContext) DistSQLPlanner() *DistSQLPlanner { return e.p.DistSQLPlanner() } -func (e *plannerJobExecContext) LeaseMgr() *lease.Manager { return e.p.LeaseMgr() } -func (e *plannerJobExecContext) User() security.SQLUsername { return e.p.User() } -func (e *plannerJobExecContext) MigrationCluster() migration.Cluster { return e.p.MigrationCluster() } +func (e *plannerJobExecContext) ExecCfg() *ExecutorConfig { return e.p.ExecCfg() } +func (e *plannerJobExecContext) DistSQLPlanner() *DistSQLPlanner { return e.p.DistSQLPlanner() } +func (e *plannerJobExecContext) LeaseMgr() *lease.Manager { return e.p.LeaseMgr() } +func (e *plannerJobExecContext) User() security.SQLUsername { return e.p.User() } +func (e *plannerJobExecContext) MigrationRunDependencies() migration.RunDependencies { + return e.p.MigrationRunDependencies() +} // JobExecContext provides the execution environment for a job. It is what is // passed to the Resume/OnFailOrCancel/OnPauseRequested methods of a jobs's @@ -65,5 +67,5 @@ type JobExecContext interface { DistSQLPlanner() *DistSQLPlanner LeaseMgr() *lease.Manager User() security.SQLUsername - MigrationCluster() migration.Cluster + MigrationRunDependencies() migration.RunDependencies } diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index 60d5ecd953cb..87cd09f9cd12 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -481,302 +481,312 @@ SELECT * FROM [SHOW GRANTS] WHERE schema_name NOT IN ('crdb_internal', 'pg_catalog', 'information_schema') ---- database_name schema_name relation_name grantee privilege_type -a pg_extension NULL readwrite ALL -a pg_extension NULL admin ALL -a pg_extension NULL root ALL -a pg_extension geography_columns public SELECT -a pg_extension geometry_columns public SELECT -a pg_extension spatial_ref_sys public SELECT -a public NULL readwrite ALL -a public NULL admin ALL -a public NULL root ALL -defaultdb pg_extension NULL admin ALL -defaultdb pg_extension NULL root ALL +system pg_extension geography_columns public SELECT +system pg_extension geometry_columns public SELECT +system pg_extension spatial_ref_sys public SELECT defaultdb pg_extension geography_columns public SELECT defaultdb pg_extension geometry_columns public SELECT defaultdb pg_extension spatial_ref_sys public SELECT -defaultdb public NULL admin ALL -defaultdb public NULL root ALL -postgres pg_extension NULL root ALL -postgres pg_extension NULL admin ALL postgres pg_extension geography_columns public SELECT postgres pg_extension geometry_columns public SELECT postgres pg_extension spatial_ref_sys public SELECT -postgres public NULL root ALL -postgres public NULL admin ALL -system pg_extension NULL admin GRANT -system pg_extension NULL root USAGE -system pg_extension NULL root GRANT -system pg_extension NULL admin USAGE -system pg_extension geography_columns public SELECT -system pg_extension geometry_columns public SELECT -system pg_extension spatial_ref_sys public SELECT -system public NULL admin GRANT -system public NULL admin USAGE -system public NULL root USAGE -system public NULL root GRANT -system public comments root UPDATE -system public comments admin GRANT -system public comments root SELECT -system public comments root DELETE -system public comments admin UPDATE -system public comments admin INSERT -system public comments admin DELETE -system public comments root INSERT -system public comments admin SELECT -system public comments public SELECT -system public comments root GRANT +test pg_extension geography_columns public SELECT +test pg_extension geometry_columns public SELECT +test pg_extension spatial_ref_sys public SELECT +a pg_extension geography_columns public SELECT +a pg_extension geometry_columns public SELECT +a pg_extension spatial_ref_sys public SELECT +system public namespace admin GRANT +system public namespace admin SELECT +system public namespace root GRANT +system public namespace root SELECT system public descriptor admin GRANT -system public descriptor root SELECT -system public descriptor root GRANT system public descriptor admin SELECT -system public eventlog admin GRANT -system public eventlog root UPDATE -system public eventlog root GRANT +system public descriptor root GRANT +system public descriptor root SELECT +system public users admin DELETE +system public users admin GRANT +system public users admin INSERT +system public users admin SELECT +system public users admin UPDATE +system public users root DELETE +system public users root GRANT +system public users root INSERT +system public users root SELECT +system public users root UPDATE +system public zones admin DELETE +system public zones admin GRANT +system public zones admin INSERT +system public zones admin SELECT +system public zones admin UPDATE +system public zones root DELETE +system public zones root GRANT +system public zones root INSERT +system public zones root SELECT +system public zones root UPDATE +system public settings admin DELETE +system public settings admin GRANT +system public settings admin INSERT +system public settings admin SELECT +system public settings admin UPDATE +system public settings root DELETE +system public settings root GRANT +system public settings root INSERT +system public settings root SELECT +system public settings root UPDATE +system public tenants admin GRANT +system public tenants admin SELECT +system public tenants root GRANT +system public tenants root SELECT +system public lease admin DELETE +system public lease admin GRANT +system public lease admin INSERT +system public lease admin SELECT +system public lease admin UPDATE +system public lease root DELETE +system public lease root GRANT +system public lease root INSERT +system public lease root SELECT +system public lease root UPDATE system public eventlog admin DELETE +system public eventlog admin GRANT +system public eventlog admin INSERT +system public eventlog admin SELECT system public eventlog admin UPDATE system public eventlog root DELETE -system public eventlog admin SELECT -system public eventlog admin INSERT -system public eventlog root SELECT +system public eventlog root GRANT system public eventlog root INSERT -system public jobs admin INSERT -system public jobs admin UPDATE -system public jobs root SELECT -system public jobs root INSERT -system public jobs root GRANT -system public jobs admin SELECT -system public jobs root DELETE -system public jobs admin DELETE -system public jobs admin GRANT -system public jobs root UPDATE -system public lease admin INSERT -system public lease admin GRANT -system public lease root UPDATE -system public lease root SELECT -system public lease root GRANT -system public lease root DELETE -system public lease admin UPDATE -system public lease admin SELECT -system public lease admin DELETE -system public lease root INSERT -system public locations root UPDATE -system public locations root SELECT -system public locations root INSERT -system public locations root GRANT -system public locations root DELETE -system public locations admin UPDATE -system public locations admin SELECT -system public locations admin INSERT -system public locations admin GRANT -system public locations admin DELETE -system public namespace admin SELECT -system public namespace root GRANT -system public namespace admin GRANT -system public namespace root SELECT -system public namespace2 root SELECT -system public namespace2 root GRANT -system public namespace2 admin GRANT -system public namespace2 admin SELECT -system public protected_ts_meta admin GRANT -system public protected_ts_meta admin SELECT -system public protected_ts_meta root SELECT -system public protected_ts_meta root GRANT -system public protected_ts_records root GRANT -system public protected_ts_records admin GRANT -system public protected_ts_records admin SELECT -system public protected_ts_records root SELECT -system public rangelog admin UPDATE -system public rangelog admin SELECT +system public eventlog root SELECT +system public eventlog root UPDATE +system public rangelog admin DELETE system public rangelog admin GRANT +system public rangelog admin INSERT +system public rangelog admin SELECT +system public rangelog admin UPDATE system public rangelog root DELETE system public rangelog root GRANT system public rangelog root INSERT system public rangelog root SELECT -system public rangelog admin DELETE -system public rangelog admin INSERT system public rangelog root UPDATE -system public replication_constraint_stats root SELECT -system public replication_constraint_stats root INSERT -system public replication_constraint_stats root GRANT -system public replication_constraint_stats root DELETE -system public replication_constraint_stats admin UPDATE -system public replication_constraint_stats admin SELECT -system public replication_constraint_stats admin INSERT -system public replication_constraint_stats root UPDATE +system public ui admin DELETE +system public ui admin GRANT +system public ui admin INSERT +system public ui admin SELECT +system public ui admin UPDATE +system public ui root DELETE +system public ui root GRANT +system public ui root INSERT +system public ui root SELECT +system public ui root UPDATE +system public jobs admin DELETE +system public jobs admin GRANT +system public jobs admin INSERT +system public jobs admin SELECT +system public jobs admin UPDATE +system public jobs root DELETE +system public jobs root GRANT +system public jobs root INSERT +system public jobs root SELECT +system public jobs root UPDATE +system public web_sessions admin DELETE +system public web_sessions admin GRANT +system public web_sessions admin INSERT +system public web_sessions admin SELECT +system public web_sessions admin UPDATE +system public web_sessions root DELETE +system public web_sessions root GRANT +system public web_sessions root INSERT +system public web_sessions root SELECT +system public web_sessions root UPDATE +system public table_statistics admin DELETE +system public table_statistics admin GRANT +system public table_statistics admin INSERT +system public table_statistics admin SELECT +system public table_statistics admin UPDATE +system public table_statistics root DELETE +system public table_statistics root GRANT +system public table_statistics root INSERT +system public table_statistics root SELECT +system public table_statistics root UPDATE +system public locations admin DELETE +system public locations admin GRANT +system public locations admin INSERT +system public locations admin SELECT +system public locations admin UPDATE +system public locations root DELETE +system public locations root GRANT +system public locations root INSERT +system public locations root SELECT +system public locations root UPDATE +system public role_members admin DELETE +system public role_members admin GRANT +system public role_members admin INSERT +system public role_members admin SELECT +system public role_members admin UPDATE +system public role_members root DELETE +system public role_members root GRANT +system public role_members root INSERT +system public role_members root SELECT +system public role_members root UPDATE +system public comments admin DELETE +system public comments admin GRANT +system public comments admin INSERT +system public comments admin SELECT +system public comments admin UPDATE +system public comments public SELECT +system public comments root DELETE +system public comments root GRANT +system public comments root INSERT +system public comments root SELECT +system public comments root UPDATE system public replication_constraint_stats admin DELETE system public replication_constraint_stats admin GRANT -system public replication_critical_localities root GRANT +system public replication_constraint_stats admin INSERT +system public replication_constraint_stats admin SELECT +system public replication_constraint_stats admin UPDATE +system public replication_constraint_stats root DELETE +system public replication_constraint_stats root GRANT +system public replication_constraint_stats root INSERT +system public replication_constraint_stats root SELECT +system public replication_constraint_stats root UPDATE system public replication_critical_localities admin DELETE -system public replication_critical_localities root INSERT -system public replication_critical_localities root SELECT system public replication_critical_localities admin GRANT -system public replication_critical_localities root UPDATE system public replication_critical_localities admin INSERT system public replication_critical_localities admin SELECT system public replication_critical_localities admin UPDATE system public replication_critical_localities root DELETE -system public replication_stats admin SELECT -system public replication_stats root SELECT +system public replication_critical_localities root GRANT +system public replication_critical_localities root INSERT +system public replication_critical_localities root SELECT +system public replication_critical_localities root UPDATE system public replication_stats admin DELETE system public replication_stats admin GRANT -system public replication_stats root DELETE +system public replication_stats admin INSERT +system public replication_stats admin SELECT system public replication_stats admin UPDATE +system public replication_stats root DELETE system public replication_stats root GRANT -system public replication_stats root UPDATE -system public replication_stats admin INSERT system public replication_stats root INSERT +system public replication_stats root SELECT +system public replication_stats root UPDATE system public reports_meta admin DELETE -system public reports_meta root INSERT -system public reports_meta root SELECT -system public reports_meta admin UPDATE -system public reports_meta admin SELECT -system public reports_meta root UPDATE -system public reports_meta admin INSERT system public reports_meta admin GRANT +system public reports_meta admin INSERT +system public reports_meta admin SELECT +system public reports_meta admin UPDATE system public reports_meta root DELETE system public reports_meta root GRANT -system public role_members admin DELETE -system public role_members admin GRANT -system public role_members admin INSERT -system public role_members admin SELECT -system public role_members root DELETE -system public role_members root GRANT -system public role_members root INSERT -system public role_members root UPDATE -system public role_members root SELECT -system public role_members admin UPDATE -system public role_options admin UPDATE -system public role_options root GRANT -system public role_options root DELETE +system public reports_meta root INSERT +system public reports_meta root SELECT +system public reports_meta root UPDATE +system public namespace2 admin GRANT +system public namespace2 admin SELECT +system public namespace2 root GRANT +system public namespace2 root SELECT +system public protected_ts_meta admin GRANT +system public protected_ts_meta admin SELECT +system public protected_ts_meta root GRANT +system public protected_ts_meta root SELECT +system public protected_ts_records admin GRANT +system public protected_ts_records admin SELECT +system public protected_ts_records root GRANT +system public protected_ts_records root SELECT system public role_options admin DELETE system public role_options admin GRANT system public role_options admin INSERT +system public role_options admin SELECT +system public role_options admin UPDATE +system public role_options root DELETE +system public role_options root GRANT system public role_options root INSERT system public role_options root SELECT system public role_options root UPDATE -system public role_options admin SELECT -system public scheduled_jobs admin INSERT -system public scheduled_jobs root UPDATE +system public statement_bundle_chunks admin DELETE +system public statement_bundle_chunks admin GRANT +system public statement_bundle_chunks admin INSERT +system public statement_bundle_chunks admin SELECT +system public statement_bundle_chunks admin UPDATE +system public statement_bundle_chunks root DELETE +system public statement_bundle_chunks root GRANT +system public statement_bundle_chunks root INSERT +system public statement_bundle_chunks root SELECT +system public statement_bundle_chunks root UPDATE +system public statement_diagnostics_requests admin DELETE +system public statement_diagnostics_requests admin GRANT +system public statement_diagnostics_requests admin INSERT +system public statement_diagnostics_requests admin SELECT +system public statement_diagnostics_requests admin UPDATE +system public statement_diagnostics_requests root DELETE +system public statement_diagnostics_requests root GRANT +system public statement_diagnostics_requests root INSERT +system public statement_diagnostics_requests root SELECT +system public statement_diagnostics_requests root UPDATE +system public statement_diagnostics admin DELETE +system public statement_diagnostics admin GRANT +system public statement_diagnostics admin INSERT +system public statement_diagnostics admin SELECT +system public statement_diagnostics admin UPDATE +system public statement_diagnostics root DELETE +system public statement_diagnostics root GRANT +system public statement_diagnostics root INSERT +system public statement_diagnostics root SELECT +system public statement_diagnostics root UPDATE system public scheduled_jobs admin DELETE -system public scheduled_jobs root SELECT system public scheduled_jobs admin GRANT -system public scheduled_jobs root INSERT -system public scheduled_jobs root GRANT -system public scheduled_jobs root DELETE +system public scheduled_jobs admin INSERT system public scheduled_jobs admin SELECT system public scheduled_jobs admin UPDATE -system public settings root DELETE -system public settings root INSERT -system public settings root SELECT -system public settings admin DELETE -system public settings admin GRANT -system public settings admin INSERT -system public settings admin SELECT -system public settings root UPDATE -system public settings admin UPDATE -system public settings root GRANT +system public scheduled_jobs root DELETE +system public scheduled_jobs root GRANT +system public scheduled_jobs root INSERT +system public scheduled_jobs root SELECT +system public scheduled_jobs root UPDATE +system public sqlliveness admin DELETE +system public sqlliveness admin GRANT +system public sqlliveness admin INSERT system public sqlliveness admin SELECT system public sqlliveness admin UPDATE -system public sqlliveness admin GRANT system public sqlliveness root DELETE system public sqlliveness root GRANT -system public sqlliveness admin DELETE +system public sqlliveness root INSERT system public sqlliveness root SELECT system public sqlliveness root UPDATE -system public sqlliveness root INSERT -system public sqlliveness admin INSERT -system public statement_bundle_chunks root SELECT -system public statement_bundle_chunks root INSERT -system public statement_bundle_chunks root DELETE -system public statement_bundle_chunks admin UPDATE -system public statement_bundle_chunks admin SELECT -system public statement_bundle_chunks root GRANT -system public statement_bundle_chunks admin INSERT -system public statement_bundle_chunks root UPDATE -system public statement_bundle_chunks admin DELETE -system public statement_bundle_chunks admin GRANT -system public statement_diagnostics root SELECT -system public statement_diagnostics root GRANT -system public statement_diagnostics root DELETE -system public statement_diagnostics admin UPDATE -system public statement_diagnostics admin SELECT -system public statement_diagnostics admin INSERT -system public statement_diagnostics admin GRANT -system public statement_diagnostics admin DELETE -system public statement_diagnostics root UPDATE -system public statement_diagnostics root INSERT -system public statement_diagnostics_requests admin SELECT -system public statement_diagnostics_requests root UPDATE -system public statement_diagnostics_requests root SELECT -system public statement_diagnostics_requests root INSERT -system public statement_diagnostics_requests root GRANT -system public statement_diagnostics_requests admin DELETE -system public statement_diagnostics_requests admin UPDATE -system public statement_diagnostics_requests admin INSERT -system public statement_diagnostics_requests root DELETE -system public statement_diagnostics_requests admin GRANT -system public table_statistics root SELECT -system public table_statistics admin UPDATE -system public table_statistics admin DELETE -system public table_statistics admin INSERT -system public table_statistics admin SELECT -system public table_statistics root INSERT -system public table_statistics root GRANT -system public table_statistics root UPDATE -system public table_statistics admin GRANT -system public table_statistics root DELETE -system public tenants root SELECT -system public tenants root GRANT -system public tenants admin SELECT -system public tenants admin GRANT -system public ui admin GRANT -system public ui root SELECT -system public ui root UPDATE -system public ui admin DELETE -system public ui admin UPDATE -system public ui admin SELECT -system public ui root INSERT -system public ui root DELETE -system public ui root GRANT -system public ui admin INSERT -system public users admin SELECT -system public users admin GRANT -system public users admin DELETE -system public users admin UPDATE -system public users root DELETE -system public users root GRANT -system public users root INSERT -system public users admin INSERT -system public users root UPDATE -system public users root SELECT -system public web_sessions admin SELECT -system public web_sessions root UPDATE -system public web_sessions root SELECT -system public web_sessions admin INSERT -system public web_sessions root GRANT -system public web_sessions root DELETE -system public web_sessions admin GRANT -system public web_sessions admin DELETE -system public web_sessions root INSERT -system public web_sessions admin UPDATE -system public zones root UPDATE -system public zones admin UPDATE -system public zones admin INSERT -system public zones admin SELECT -system public zones root DELETE -system public zones root GRANT -system public zones admin GRANT -system public zones root SELECT -system public zones admin DELETE -system public zones root INSERT +system public long_running_migrations admin DELETE +system public long_running_migrations admin GRANT +system public long_running_migrations admin INSERT +system public long_running_migrations admin SELECT +system public long_running_migrations admin UPDATE +system public long_running_migrations root DELETE +system public long_running_migrations root GRANT +system public long_running_migrations root INSERT +system public long_running_migrations root SELECT +system public long_running_migrations root UPDATE +a pg_extension NULL admin ALL +a pg_extension NULL readwrite ALL +a pg_extension NULL root ALL +a public NULL admin ALL +a public NULL readwrite ALL +a public NULL root ALL +defaultdb pg_extension NULL admin ALL +defaultdb pg_extension NULL root ALL +defaultdb public NULL admin ALL +defaultdb public NULL root ALL +postgres pg_extension NULL admin ALL +postgres pg_extension NULL root ALL +postgres public NULL admin ALL +postgres public NULL root ALL +system pg_extension NULL admin GRANT +system pg_extension NULL admin USAGE +system pg_extension NULL root GRANT +system pg_extension NULL root USAGE +system public NULL admin GRANT +system public NULL admin USAGE +system public NULL root GRANT +system public NULL root USAGE test pg_extension NULL admin ALL test pg_extension NULL root ALL -test pg_extension geography_columns public SELECT -test pg_extension geometry_columns public SELECT -test pg_extension spatial_ref_sys public SELECT -test public NULL root ALL test public NULL admin ALL +test public NULL root ALL query TTTTT colnames SHOW GRANTS FOR root @@ -1139,6 +1149,11 @@ system public locations root GRA system public locations root INSERT system public locations root SELECT system public locations root UPDATE +system public long_running_migrations root DELETE +system public long_running_migrations root GRANT +system public long_running_migrations root INSERT +system public long_running_migrations root SELECT +system public long_running_migrations root UPDATE system public namespace root GRANT system public namespace root SELECT system public namespace2 root GRANT diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 24c8e1bdb9ea..bc7cc81deac7 100755 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -774,6 +774,7 @@ system public statement_diagnostics_requests BASE T system public statement_diagnostics BASE TABLE YES 1 system public scheduled_jobs BASE TABLE YES 1 system public sqlliveness BASE TABLE YES 1 +system public long_running_migrations BASE TABLE YES 1 statement ok ALTER TABLE other_db.xyz ADD COLUMN j INT @@ -859,6 +860,12 @@ system public 630200280_21_2_not_null system system public 630200280_21_3_not_null system public locations CHECK NO NO system public 630200280_21_4_not_null system public locations CHECK NO NO system public primary system public locations PRIMARY KEY NO NO +system public 630200280_40_1_not_null system public long_running_migrations CHECK NO NO +system public 630200280_40_2_not_null system public long_running_migrations CHECK NO NO +system public 630200280_40_3_not_null system public long_running_migrations CHECK NO NO +system public 630200280_40_4_not_null system public long_running_migrations CHECK NO NO +system public 630200280_40_5_not_null system public long_running_migrations CHECK NO NO +system public primary system public long_running_migrations PRIMARY KEY NO NO system public 630200280_2_1_not_null system public namespace CHECK NO NO system public 630200280_2_2_not_null system public namespace CHECK NO NO system public primary system public namespace PRIMARY KEY NO NO @@ -1089,6 +1096,11 @@ system public 630200280_37_9_not_null executor_type I system public 630200280_39_1_not_null session_id IS NOT NULL system public 630200280_39_2_not_null expiration IS NOT NULL system public 630200280_3_1_not_null id IS NOT NULL +system public 630200280_40_1_not_null major IS NOT NULL +system public 630200280_40_2_not_null minor IS NOT NULL +system public 630200280_40_3_not_null patch IS NOT NULL +system public 630200280_40_4_not_null internal IS NOT NULL +system public 630200280_40_5_not_null completed_at IS NOT NULL system public 630200280_4_1_not_null username IS NOT NULL system public 630200280_4_3_not_null isRole IS NOT NULL system public 630200280_5_1_not_null id IS NOT NULL @@ -1118,6 +1130,10 @@ system public lease nodeID sy system public lease version system public primary system public locations localityKey system public primary system public locations localityValue system public primary +system public long_running_migrations internal system public primary +system public long_running_migrations major system public primary +system public long_running_migrations minor system public primary +system public long_running_migrations patch system public primary system public namespace name system public primary system public namespace parentID system public primary system public namespace2 name system public primary @@ -1282,6 +1298,11 @@ system public locations latitude system public locations localityKey 1 system public locations localityValue 2 system public locations longitude 4 +system public long_running_migrations completed_at 5 +system public long_running_migrations internal 4 +system public long_running_migrations major 1 +system public long_running_migrations minor 2 +system public long_running_migrations patch 3 system public namespace id 3 system public namespace name 2 system public namespace parentID 1 @@ -2041,6 +2062,16 @@ NULL root system public locations NULL root system public locations INSERT NULL NO NULL root system public locations SELECT NULL YES NULL root system public locations UPDATE NULL NO +NULL admin system public long_running_migrations DELETE NULL NO +NULL admin system public long_running_migrations GRANT NULL NO +NULL admin system public long_running_migrations INSERT NULL NO +NULL admin system public long_running_migrations SELECT NULL YES +NULL admin system public long_running_migrations UPDATE NULL NO +NULL root system public long_running_migrations DELETE NULL NO +NULL root system public long_running_migrations GRANT NULL NO +NULL root system public long_running_migrations INSERT NULL NO +NULL root system public long_running_migrations SELECT NULL YES +NULL root system public long_running_migrations UPDATE NULL NO NULL admin system public namespace GRANT NULL NO NULL admin system public namespace SELECT NULL YES NULL root system public namespace GRANT NULL NO @@ -2627,6 +2658,16 @@ NULL root system public sqlliveness NULL root system public sqlliveness INSERT NULL NO NULL root system public sqlliveness SELECT NULL YES NULL root system public sqlliveness UPDATE NULL NO +NULL admin system public long_running_migrations DELETE NULL NO +NULL admin system public long_running_migrations GRANT NULL NO +NULL admin system public long_running_migrations INSERT NULL NO +NULL admin system public long_running_migrations SELECT NULL YES +NULL admin system public long_running_migrations UPDATE NULL NO +NULL root system public long_running_migrations DELETE NULL NO +NULL root system public long_running_migrations GRANT NULL NO +NULL root system public long_running_migrations INSERT NULL NO +NULL root system public long_running_migrations SELECT NULL YES +NULL root system public long_running_migrations UPDATE NULL NO statement ok CREATE TABLE other_db.xyz (i INT) diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 7c082b4fa5c9..589e8b673bcc 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -796,6 +796,7 @@ indexrelid indrelid indnatts indisunique indisprimary indisexclusion indim 2008917578 37 1 false false false false false true false false true false 5 0 0 2 NULL NULL 2101708905 5 1 true true false true false true false false true false 1 0 0 2 NULL NULL 2148104569 21 2 true true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL +2268653844 40 4 true true false true false true false false true false 1 2 3 4 0 0 0 0 0 0 0 0 2 2 2 2 NULL NULL 2361445172 8 1 true true false true false true false false true false 1 0 0 2 NULL NULL 2407840836 24 3 true true false true false true false false true false 1 2 3 0 0 0 0 0 0 2 2 2 NULL NULL 2621181440 15 2 false false false false false true false false true false 2 3 3403232968 0 0 0 2 2 NULL NULL @@ -852,6 +853,10 @@ indexrelid operator_argument_type_oid operator_argument_position 2101708905 0 1 2148104569 0 1 2148104569 0 2 +2268653844 0 1 +2268653844 0 2 +2268653844 0 3 +2268653844 0 4 2361445172 0 1 2407840836 0 1 2407840836 0 2 diff --git a/pkg/sql/logictest/testdata/logic_test/ranges b/pkg/sql/logictest/testdata/logic_test/ranges index 039d165ce29e..e164b7ad10a3 100644 --- a/pkg/sql/logictest/testdata/logic_test/ranges +++ b/pkg/sql/logictest/testdata/logic_test/ranges @@ -309,7 +309,8 @@ start_key start_pretty end_key [172] /Table/36 [173] /Table/37 system statement_diagnostics · {1} 1 [173] /Table/37 [174] /Table/38 system scheduled_jobs · {1} 1 [174] /Table/38 [175] /Table/39 · · · {1} 1 -[175] /Table/39 [189 137] /Table/53/1 system sqlliveness · {1} 1 +[175] /Table/39 [176] /Table/40 system sqlliveness · {1} 1 +[176] /Table/40 [189 137] /Table/53/1 system long_running_migrations · {1} 1 [189 137] /Table/53/1 [189 137 137] /Table/53/1/1 test t · {1} 1 [189 137 137] /Table/53/1/1 [189 137 141 137] /Table/53/1/5/1 test t · {3,4} 3 [189 137 141 137] /Table/53/1/5/1 [189 137 141 138] /Table/53/1/5/2 test t · {1,2,3} 1 @@ -367,7 +368,8 @@ start_key start_pretty end_key [172] /Table/36 [173] /Table/37 system statement_diagnostics · {1} 1 [173] /Table/37 [174] /Table/38 system scheduled_jobs · {1} 1 [174] /Table/38 [175] /Table/39 · · · {1} 1 -[175] /Table/39 [189 137] /Table/53/1 system sqlliveness · {1} 1 +[175] /Table/39 [176] /Table/40 system sqlliveness · {1} 1 +[176] /Table/40 [189 137] /Table/53/1 system long_running_migrations · {1} 1 [189 137] /Table/53/1 [189 137 137] /Table/53/1/1 test t · {1} 1 [189 137 137] /Table/53/1/1 [189 137 141 137] /Table/53/1/5/1 test t · {3,4} 3 [189 137 141 137] /Table/53/1/5/1 [189 137 141 138] /Table/53/1/5/2 test t · {1,2,3} 1 diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source index 7bc7be1535f1..f17fd7ef9231 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_source +++ b/pkg/sql/logictest/testdata/logic_test/show_source @@ -196,68 +196,70 @@ SELECT * FROM [SHOW TABLES FROM system] ---- schema_name table_name type owner estimated_row_count locality public namespace table NULL NULL NULL -public descriptor table NULL NULL NULL -public users table NULL NULL NULL -public zones table NULL NULL NULL -public settings table NULL NULL NULL -public tenants table NULL NULL NULL -public lease table NULL NULL NULL -public eventlog table NULL NULL NULL -public rangelog table NULL NULL NULL -public ui table NULL NULL NULL -public jobs table NULL NULL NULL -public web_sessions table NULL NULL NULL -public table_statistics table NULL NULL NULL -public locations table NULL NULL NULL -public role_members table NULL NULL NULL -public comments table NULL NULL NULL -public replication_constraint_stats table NULL NULL NULL -public replication_critical_localities table NULL NULL NULL -public replication_stats table NULL NULL NULL -public reports_meta table NULL NULL NULL -public namespace2 table NULL NULL NULL -public protected_ts_meta table NULL NULL NULL -public protected_ts_records table NULL NULL NULL -public role_options table NULL NULL NULL -public statement_bundle_chunks table NULL NULL NULL -public statement_diagnostics_requests table NULL NULL NULL -public statement_diagnostics table NULL NULL NULL -public scheduled_jobs table NULL NULL NULL +public long_running_migrations table NULL NULL NULL public sqlliveness table NULL NULL NULL +public scheduled_jobs table NULL NULL NULL +public statement_diagnostics table NULL NULL NULL +public statement_diagnostics_requests table NULL NULL NULL +public statement_bundle_chunks table NULL NULL NULL +public role_options table NULL NULL NULL +public protected_ts_records table NULL NULL NULL +public protected_ts_meta table NULL NULL NULL +public namespace2 table NULL NULL NULL +public reports_meta table NULL NULL NULL +public replication_stats table NULL NULL NULL +public replication_critical_localities table NULL NULL NULL +public replication_constraint_stats table NULL NULL NULL +public comments table NULL NULL NULL +public role_members table NULL NULL NULL +public locations table NULL NULL NULL +public table_statistics table NULL NULL NULL +public web_sessions table NULL NULL NULL +public jobs table NULL NULL NULL +public ui table NULL NULL NULL +public rangelog table NULL NULL NULL +public eventlog table NULL NULL NULL +public lease table NULL NULL NULL +public tenants table NULL NULL NULL +public settings table NULL NULL NULL +public zones table NULL NULL NULL +public users table NULL NULL NULL +public descriptor table NULL NULL NULL query TTTTTTT colnames,rowsort SELECT * FROM [SHOW TABLES FROM system WITH COMMENT] ---- schema_name table_name type owner estimated_row_count locality comment public namespace table NULL NULL NULL · -public descriptor table NULL NULL NULL · -public users table NULL NULL NULL · -public zones table NULL NULL NULL · -public settings table NULL NULL NULL · -public tenants table NULL NULL NULL · -public lease table NULL NULL NULL · -public eventlog table NULL NULL NULL · -public rangelog table NULL NULL NULL · -public ui table NULL NULL NULL · -public jobs table NULL NULL NULL · -public web_sessions table NULL NULL NULL · -public table_statistics table NULL NULL NULL · -public locations table NULL NULL NULL · -public role_members table NULL NULL NULL · -public comments table NULL NULL NULL · -public replication_constraint_stats table NULL NULL NULL · -public replication_critical_localities table NULL NULL NULL · -public replication_stats table NULL NULL NULL · -public reports_meta table NULL NULL NULL · -public namespace2 table NULL NULL NULL · -public protected_ts_meta table NULL NULL NULL · -public protected_ts_records table NULL NULL NULL · -public role_options table NULL NULL NULL · -public statement_bundle_chunks table NULL NULL NULL · -public statement_diagnostics_requests table NULL NULL NULL · -public statement_diagnostics table NULL NULL NULL · -public scheduled_jobs table NULL NULL NULL · +public long_running_migrations table NULL NULL NULL · public sqlliveness table NULL NULL NULL · +public scheduled_jobs table NULL NULL NULL · +public statement_diagnostics table NULL NULL NULL · +public statement_diagnostics_requests table NULL NULL NULL · +public statement_bundle_chunks table NULL NULL NULL · +public role_options table NULL NULL NULL · +public protected_ts_records table NULL NULL NULL · +public protected_ts_meta table NULL NULL NULL · +public namespace2 table NULL NULL NULL · +public reports_meta table NULL NULL NULL · +public replication_stats table NULL NULL NULL · +public replication_critical_localities table NULL NULL NULL · +public replication_constraint_stats table NULL NULL NULL · +public comments table NULL NULL NULL · +public role_members table NULL NULL NULL · +public locations table NULL NULL NULL · +public table_statistics table NULL NULL NULL · +public web_sessions table NULL NULL NULL · +public jobs table NULL NULL NULL · +public ui table NULL NULL NULL · +public rangelog table NULL NULL NULL · +public eventlog table NULL NULL NULL · +public lease table NULL NULL NULL · +public tenants table NULL NULL NULL · +public settings table NULL NULL NULL · +public zones table NULL NULL NULL · +public users table NULL NULL NULL · +public descriptor table NULL NULL NULL · query ITTT colnames SELECT node_id, user_name, application_name, active_queries diff --git a/pkg/sql/logictest/testdata/logic_test/system b/pkg/sql/logictest/testdata/logic_test/system index 036fae47b2d1..0da6ad0c45b2 100644 --- a/pkg/sql/logictest/testdata/logic_test/system +++ b/pkg/sql/logictest/testdata/logic_test/system @@ -16,6 +16,7 @@ public eventlog table NULL NULL NULL public jobs table NULL NULL NULL public lease table NULL NULL NULL public locations table NULL NULL NULL +public long_running_migrations table NULL NULL NULL public namespace table NULL NULL NULL public namespace2 table NULL NULL NULL public protected_ts_meta table NULL NULL NULL @@ -73,6 +74,7 @@ SELECT id FROM system.descriptor 36 37 39 +40 50 51 52 @@ -246,6 +248,16 @@ system public locations root GRANT system public locations root INSERT system public locations root SELECT system public locations root UPDATE +system public long_running_migrations admin DELETE +system public long_running_migrations admin GRANT +system public long_running_migrations admin INSERT +system public long_running_migrations admin SELECT +system public long_running_migrations admin UPDATE +system public long_running_migrations root DELETE +system public long_running_migrations root GRANT +system public long_running_migrations root INSERT +system public long_running_migrations root SELECT +system public long_running_migrations root UPDATE system public namespace admin GRANT system public namespace admin SELECT system public namespace root GRANT diff --git a/pkg/sql/logictest/testdata/logic_test/system_namespace b/pkg/sql/logictest/testdata/logic_test/system_namespace index ce32ec8c320b..eb65ba4aef90 100644 --- a/pkg/sql/logictest/testdata/logic_test/system_namespace +++ b/pkg/sql/logictest/testdata/logic_test/system_namespace @@ -18,6 +18,7 @@ SELECT * FROM system.namespace 1 29 jobs 15 1 29 lease 11 1 29 locations 21 +1 29 long_running_migrations 40 1 29 namespace 2 1 29 namespace2 30 1 29 protected_ts_meta 31 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic b/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic index f52fa55cec6c..7191149f250a 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic +++ b/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic @@ -41,7 +41,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 # Multi-row insert should auto-commit. query B @@ -62,7 +62,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -86,7 +86,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 2 CPut to (n1,s1):1 +dist sender send r36: sending batch 2 CPut to (n1,s1):1 statement ok ROLLBACK @@ -110,7 +110,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -132,8 +132,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 2 CPut to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 CPut to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 # Insert with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -156,8 +156,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 2 CPut to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 CPut to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 # Another way to test the scenario above: generate an error and ensure that the # mutation was not committed. @@ -192,7 +192,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 1 Put, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Put, 1 EndTxn to (n1,s1):1 # Multi-row upsert should auto-commit. query B @@ -213,7 +213,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -237,7 +237,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 2 Put to (n1,s1):1 +dist sender send r36: sending batch 2 Put to (n1,s1):1 statement ok ROLLBACK @@ -261,7 +261,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -283,8 +283,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 2 Put to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 Put to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 # Upsert with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -307,8 +307,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 2 Put to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 Put to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 # Another way to test the scenario above: generate an error and ensure that the # mutation was not committed. @@ -343,8 +343,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -368,8 +368,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 2 Put to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 2 Put to (n1,s1):1 statement ok ROLLBACK @@ -393,8 +393,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -416,9 +416,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 2 Put to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 2 Put to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 # Update with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -441,9 +441,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 2 Put to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 2 Put to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 # Another way to test the scenario above: generate an error and ensure that the # mutation was not committed. @@ -478,7 +478,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 # Multi-row delete should auto-commit. query B @@ -499,7 +499,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -523,7 +523,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 1 DelRng to (n1,s1):1 +dist sender send r36: sending batch 1 DelRng to (n1,s1):1 statement ok ROLLBACK @@ -547,8 +547,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 2 Del, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 2 Del, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -570,9 +570,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 2 Del to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 2 Del to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 # Insert with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -595,9 +595,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 2 Del to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 2 Del to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 statement ok INSERT INTO ab VALUES (12, 0); @@ -644,9 +644,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 2 CPut to (n1,s1):1 -dist sender send r35: sending batch 2 Scan to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 CPut to (n1,s1):1 +dist sender send r36: sending batch 2 Scan to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -667,10 +667,10 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 1 Put to (n1,s1):1 -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 1 Put to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -692,10 +692,10 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 1 Del to (n1,s1):1 -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 1 Del to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 # Test with a single cascade, which should use autocommit. statement ok @@ -719,9 +719,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 1 DelRng to (n1,s1):1 -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 1 Del, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 DelRng to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 1 Del, 1 EndTxn to (n1,s1):1 # ----------------------- # Multiple mutation tests @@ -749,9 +749,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 2 CPut to (n1,s1):1 -dist sender send r35: sending batch 2 CPut to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 CPut to (n1,s1):1 +dist sender send r36: sending batch 2 CPut to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -774,6 +774,6 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 2 CPut to (n1,s1):1 -dist sender send r35: sending batch 2 CPut to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 CPut to (n1,s1):1 +dist sender send r36: sending batch 2 CPut to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/delete b/pkg/sql/opt/exec/execbuilder/testdata/delete index 37fef69ef74f..752dd4fb9228 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/delete +++ b/pkg/sql/opt/exec/execbuilder/testdata/delete @@ -231,9 +231,9 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE '%DelRange%' OR message LIKE '%DelRng%' ---- flow DelRange /Table/57/1 - /Table/57/2 -dist sender send r35: sending batch 1 DelRng to (n1,s1):1 +dist sender send r36: sending batch 1 DelRng to (n1,s1):1 flow DelRange /Table/57/1/601/0 - /Table/57/2 -dist sender send r35: sending batch 1 DelRng to (n1,s1):1 +dist sender send r36: sending batch 1 DelRng to (n1,s1):1 # Ensure that DelRange requests are autocommitted when DELETE FROM happens on a # chunk of fewer than 600 keys. @@ -249,7 +249,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE '%DelRange%' OR message LIKE '%sending batch%' ---- flow DelRange /Table/57/1/5 - /Table/57/1/5/# -dist sender send r35: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 # Test use of fast path when there are interleaved tables. diff --git a/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic b/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic index 2dab4bee6428..bef8099194a4 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic +++ b/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic @@ -77,7 +77,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- flow CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 flow InitPut /Table/54/2/2/0 -> /BYTES/0x89 -kv.DistSender: sending partial batch r35: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r36: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 flow fast path completed exec stmt rows affected: 1 @@ -92,7 +92,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- flow CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 flow InitPut /Table/54/2/2/0 -> /BYTES/0x89 -kv.DistSender: sending partial batch r35: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r36: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 exec stmt execution failed after 0 rows: duplicate key value violates unique constraint "primary" statement error duplicate key value @@ -105,7 +105,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- flow CPut /Table/54/1/2/0 -> /TUPLE/2:2:Int/2 flow InitPut /Table/54/2/2/0 -> /BYTES/0x8a -kv.DistSender: sending partial batch r35: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r36: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 exec stmt execution failed after 0 rows: duplicate key value violates unique constraint "woo" statement ok @@ -185,7 +185,7 @@ materializer Scan /Table/54/{1-2} materializer fetched: /kv/primary/1/v -> /2 flow Del /Table/54/2/2/0 flow Del /Table/54/1/1/0 -kv.DistSender: sending partial batch r35: sending batch 1 Del to (n1,s1):1 +kv.DistSender: sending partial batch r36: sending batch 1 Del to (n1,s1):1 flow fast path completed exec stmt rows affected: 1 @@ -242,7 +242,7 @@ SET tracing = on; INSERT INTO t.kv3 (k, v) VALUES (1,1); SET tracing = off query T SELECT message FROM [SHOW TRACE FOR SESSION] WHERE message LIKE e'%1 CPut, 1 EndTxn%' AND message NOT LIKE e'%proposing command%' ---- -r36: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +r37: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 node received request: 1 CPut, 1 EndTxn # Temporarily disabled flaky test (#58202). diff --git a/pkg/sql/opt/exec/execbuilder/testdata/upsert_nonmetamorphic b/pkg/sql/opt/exec/execbuilder/testdata/upsert_nonmetamorphic index 049fdc1041d9..f41ae14e334f 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/upsert_nonmetamorphic +++ b/pkg/sql/opt/exec/execbuilder/testdata/upsert_nonmetamorphic @@ -41,7 +41,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] materializer Scan /Table/55/1/2{-/#} flow CPut /Table/55/1/2/0 -> /TUPLE/2:2:Int/3 flow InitPut /Table/55/2/3/0 -> /BYTES/0x8a -kv.DistSender: sending partial batch r35: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r36: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 flow fast path completed exec stmt rows affected: 1 @@ -55,7 +55,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] materializer Scan /Table/55/1/1{-/#} flow CPut /Table/55/1/1/0 -> /TUPLE/2:2:Int/2 flow InitPut /Table/55/2/2/0 -> /BYTES/0x89 -kv.DistSender: sending partial batch r35: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r36: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 flow fast path completed exec stmt rows affected: 1 @@ -72,5 +72,5 @@ materializer fetched: /kv/primary/2/v -> /3 flow Put /Table/55/1/2/0 -> /TUPLE/2:2:Int/2 flow Del /Table/55/2/3/0 flow CPut /Table/55/2/2/0 -> /BYTES/0x8a (expecting does not exist) -kv.DistSender: sending partial batch r35: sending batch 1 Put, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r36: sending batch 1 Put, 1 EndTxn to (n1,s1):1 exec stmt execution failed after 0 rows: duplicate key value violates unique constraint "woo" diff --git a/pkg/sql/pgwire/pgwire_test.go b/pkg/sql/pgwire/pgwire_test.go index 704e08914fea..447b39baef90 100644 --- a/pkg/sql/pgwire/pgwire_test.go +++ b/pkg/sql/pgwire/pgwire_test.go @@ -556,7 +556,7 @@ func TestPGPreparedQuery(t *testing.T) { baseTest.Results("users", "primary", false, 1, "username", "ASC", false, false), }}, {"SHOW TABLES FROM system", []preparedQueryTest{ - baseTest.Results("public", "comments", "table", gosql.NullString{}, gosql.NullString{}, gosql.NullString{}).Others(28), + baseTest.Results("public", "comments", "table", gosql.NullString{}, gosql.NullString{}, gosql.NullString{}).Others(29), }}, {"SHOW SCHEMAS FROM system", []preparedQueryTest{ baseTest.Results("crdb_internal", gosql.NullString{}).Others(4), diff --git a/pkg/sql/planhook.go b/pkg/sql/planhook.go index cf1747471105..51a6c6e15467 100644 --- a/pkg/sql/planhook.go +++ b/pkg/sql/planhook.go @@ -98,7 +98,7 @@ type PlanHookState interface { ) (string, error) CreateSchemaNamespaceEntry(ctx context.Context, schemaNameKey roachpb.Key, schemaID descpb.ID) error - MigrationCluster() migration.Cluster + MigrationRunDependencies() migration.RunDependencies } // AddPlanHook adds a hook used to short-circuit creating a planNode from a diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index beb0ba52d128..8705f8d972b8 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -474,9 +474,9 @@ func (p *planner) DistSQLPlanner() *DistSQLPlanner { return p.extendedEvalCtx.DistSQLPlanner } -// MigrationCluster returns the migration.Cluster if there is one. -func (p *planner) MigrationCluster() migration.Cluster { - return p.execCfg.MigrationCluster +// MigrationRunDependencies returns the migration.Cluster if there is one. +func (p *planner) MigrationRunDependencies() migration.RunDependencies { + return p.execCfg.MigrationRunDependencies } // GetTypeFromValidSQLSyntax implements the tree.EvalPlanner interface. diff --git a/pkg/sql/tests/system_table_test.go b/pkg/sql/tests/system_table_test.go index fe110ada1484..e369b2db7335 100644 --- a/pkg/sql/tests/system_table_test.go +++ b/pkg/sql/tests/system_table_test.go @@ -191,6 +191,7 @@ func TestSystemTableLiterals(t *testing.T) { {keys.StatementDiagnosticsTableID, systemschema.StatementDiagnosticsTableSchema, systemschema.StatementDiagnosticsTable}, {keys.ScheduledJobsTableID, systemschema.ScheduledJobsTableSchema, systemschema.ScheduledJobsTable}, {keys.SqllivenessID, systemschema.SqllivenessTableSchema, systemschema.SqllivenessTable}, + {keys.LongRunningMigrationsID, systemschema.LongRunningMigrationsTableSchema, systemschema.LongRunningMigrationsTable}, } { privs := *test.pkg.GetPrivileges() gen, err := sql.CreateTestTableDescriptor( diff --git a/pkg/sql/tests/testdata/initial_keys b/pkg/sql/tests/testdata/initial_keys index 0a3eeea059ec..faee24d160ab 100644 --- a/pkg/sql/tests/testdata/initial_keys +++ b/pkg/sql/tests/testdata/initial_keys @@ -1,6 +1,6 @@ initial-keys tenant=system ---- -69 keys: +71 keys: /System/"desc-idgen" /Table/3/1/1/2/1 /Table/3/1/2/2/1 @@ -32,6 +32,7 @@ initial-keys tenant=system /Table/3/1/36/2/1 /Table/3/1/37/2/1 /Table/3/1/39/2/1 + /Table/3/1/40/2/1 /Table/5/1/0/2/1 /Table/5/1/1/2/1 /Table/5/1/16/2/1 @@ -47,6 +48,7 @@ initial-keys tenant=system /NamespaceTable/30/1/1/29/"jobs"/4/1 /NamespaceTable/30/1/1/29/"lease"/4/1 /NamespaceTable/30/1/1/29/"locations"/4/1 + /NamespaceTable/30/1/1/29/"long_running_migrations"/4/1 /NamespaceTable/30/1/1/29/"namespace"/4/1 /NamespaceTable/30/1/1/29/"namespace2"/4/1 /NamespaceTable/30/1/1/29/"protected_ts_meta"/4/1 @@ -70,7 +72,7 @@ initial-keys tenant=system /NamespaceTable/30/1/1/29/"users"/4/1 /NamespaceTable/30/1/1/29/"web_sessions"/4/1 /NamespaceTable/30/1/1/29/"zones"/4/1 -29 splits: +30 splits: /Table/11 /Table/12 /Table/13 @@ -100,10 +102,11 @@ initial-keys tenant=system /Table/37 /Table/38 /Table/39 + /Table/40 initial-keys tenant=5 ---- -60 keys: +62 keys: /Tenant/5/Table/3/1/1/2/1 /Tenant/5/Table/3/1/2/2/1 /Tenant/5/Table/3/1/3/2/1 @@ -133,6 +136,7 @@ initial-keys tenant=5 /Tenant/5/Table/3/1/36/2/1 /Tenant/5/Table/3/1/37/2/1 /Tenant/5/Table/3/1/39/2/1 + /Tenant/5/Table/3/1/40/2/1 /Tenant/5/Table/7/1/0/0 /Tenant/5/NamespaceTable/30/1/0/0/"system"/4/1 /Tenant/5/NamespaceTable/30/1/1/0/"public"/4/1 @@ -143,6 +147,7 @@ initial-keys tenant=5 /Tenant/5/NamespaceTable/30/1/1/29/"jobs"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"lease"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"locations"/4/1 + /Tenant/5/NamespaceTable/30/1/1/29/"long_running_migrations"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"namespace"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"namespace2"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"protected_ts_meta"/4/1 @@ -169,7 +174,7 @@ initial-keys tenant=5 initial-keys tenant=999 ---- -60 keys: +62 keys: /Tenant/999/Table/3/1/1/2/1 /Tenant/999/Table/3/1/2/2/1 /Tenant/999/Table/3/1/3/2/1 @@ -199,6 +204,7 @@ initial-keys tenant=999 /Tenant/999/Table/3/1/36/2/1 /Tenant/999/Table/3/1/37/2/1 /Tenant/999/Table/3/1/39/2/1 + /Tenant/999/Table/3/1/40/2/1 /Tenant/999/Table/7/1/0/0 /Tenant/999/NamespaceTable/30/1/0/0/"system"/4/1 /Tenant/999/NamespaceTable/30/1/1/0/"public"/4/1 @@ -209,6 +215,7 @@ initial-keys tenant=999 /Tenant/999/NamespaceTable/30/1/1/29/"jobs"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"lease"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"locations"/4/1 + /Tenant/999/NamespaceTable/30/1/1/29/"long_running_migrations"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"namespace"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"namespace2"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"protected_ts_meta"/4/1 diff --git a/pkg/sqlmigrations/migrations.go b/pkg/sqlmigrations/migrations.go index 8868a0032439..7571b70fc00d 100644 --- a/pkg/sqlmigrations/migrations.go +++ b/pkg/sqlmigrations/migrations.go @@ -334,14 +334,14 @@ var backwardCompatibleMigrations = []migrationDescriptor{ func staticIDs( ids ...descpb.ID, -) func(ctx context.Context, db db, codec keys.SQLCodec) ([]descpb.ID, error) { - return func(ctx context.Context, db db, codec keys.SQLCodec) ([]descpb.ID, error) { return ids, nil } +) func(ctx context.Context, db DB, codec keys.SQLCodec) ([]descpb.ID, error) { + return func(ctx context.Context, db DB, codec keys.SQLCodec) ([]descpb.ID, error) { return ids, nil } } func databaseIDs( names ...string, -) func(ctx context.Context, db db, codec keys.SQLCodec) ([]descpb.ID, error) { - return func(ctx context.Context, db db, codec keys.SQLCodec) ([]descpb.ID, error) { +) func(ctx context.Context, db DB, codec keys.SQLCodec) ([]descpb.ID, error) { + return func(ctx context.Context, db DB, codec keys.SQLCodec) ([]descpb.ID, error) { var ids []descpb.ID for _, name := range names { // This runs as part of an older migration (introduced in 2.1). We use @@ -392,7 +392,7 @@ type migrationDescriptor struct { // descriptors that were added by this migration. This is needed to automate // certain tests, which check the number of ranges/descriptors present on // server bootup. - newDescriptorIDs func(ctx context.Context, db db, codec keys.SQLCodec) ([]descpb.ID, error) + newDescriptorIDs func(ctx context.Context, db DB, codec keys.SQLCodec) ([]descpb.ID, error) } func init() { @@ -408,7 +408,7 @@ func init() { } type runner struct { - db db + db DB codec keys.SQLCodec sqlExecutor *sql.InternalExecutor settings *cluster.Settings @@ -449,9 +449,9 @@ type leaseManager interface { TimeRemaining(l *leasemanager.Lease) time.Duration } -// db is defined just to allow us to use a fake client.DB when testing this +// DB is defined just to allow us to use a fake client.DB when testing this // package. -type db interface { +type DB interface { Scan(ctx context.Context, begin, end interface{}, maxRows int64) ([]kv.KeyValue, error) Get(ctx context.Context, key interface{}) (kv.KeyValue, error) Put(ctx context.Context, key, value interface{}) error @@ -463,7 +463,7 @@ type db interface { type Manager struct { stopper *stop.Stopper leaseManager leaseManager - db db + db DB codec keys.SQLCodec sqlExecutor *sql.InternalExecutor testingKnobs MigrationManagerTestingKnobs @@ -508,7 +508,7 @@ func NewManager( // lifecycle is tightly controlled. func ExpectedDescriptorIDs( ctx context.Context, - db db, + db DB, codec keys.SQLCodec, defaultZoneConfig *zonepb.ZoneConfig, defaultSystemZoneConfig *zonepb.ZoneConfig, @@ -891,7 +891,7 @@ func (m *Manager) migrateSystemNamespace( } func getCompletedMigrations( - ctx context.Context, db db, codec keys.SQLCodec, + ctx context.Context, db DB, codec keys.SQLCodec, ) (map[string]struct{}, error) { if log.V(1) { log.Info(ctx, "trying to get the list of completed migrations") @@ -913,14 +913,26 @@ func migrationKey(codec keys.SQLCodec, migration migrationDescriptor) roachpb.Ke } func createSystemTable(ctx context.Context, r runner, desc catalog.TableDescriptor) error { + return CreateSystemTable(ctx, r.db, r.codec, r.settings, desc) +} + +// CreateSystemTable is a function to inject a new system table. If the table +// already exists, ths function is a no-op. +func CreateSystemTable( + ctx context.Context, + db DB, + codec keys.SQLCodec, + settings *cluster.Settings, + desc catalog.TableDescriptor, +) error { // We install the table at the KV layer so that we can choose a known ID in // the reserved ID space. (The SQL layer doesn't allow this.) - err := r.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { b := txn.NewBatch() - tKey := catalogkv.MakePublicTableNameKey(ctx, r.settings, desc.GetParentID(), desc.GetName()) - b.CPut(tKey.Key(r.codec), desc.GetID(), nil) - b.CPut(catalogkeys.MakeDescMetadataKey(r.codec, desc.GetID()), desc.DescriptorProto(), nil) - if err := txn.SetSystemConfigTrigger(r.codec.ForSystemTenant()); err != nil { + tKey := catalogkv.MakePublicTableNameKey(ctx, settings, desc.GetParentID(), desc.GetName()) + b.CPut(tKey.Key(codec), desc.GetID(), nil) + b.CPut(catalogkeys.MakeDescMetadataKey(codec, desc.GetID()), desc.DescriptorProto(), nil) + if err := txn.SetSystemConfigTrigger(codec.ForSystemTenant()); err != nil { return err } return txn.Run(ctx, b) @@ -1137,7 +1149,7 @@ func createDefaultDbs(ctx context.Context, r runner) error { for retry := retry.Start(retry.Options{MaxRetries: 5}); retry.Next(); { for _, dbName := range []string{catalogkeys.DefaultDatabaseName, catalogkeys.PgDatabaseName} { stmt := fmt.Sprintf(createDbStmt, dbName) - err = r.execAsRoot(ctx, "create-default-db", stmt) + err = r.execAsRoot(ctx, "create-default-DB", stmt) if err != nil { log.Warningf(ctx, "failed attempt to add database %q: %s", dbName, err) break