diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 1406ceebe0cb..e1e74805b049 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -169,4 +169,4 @@ trace.debug.enable boolean false if set, traces for recent requests can be seen trace.jaeger.agent string the address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as :. If no port is specified, 6381 will be used. trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 21.2-32 set the active cluster version in the format '.' +version version 21.2-34 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 62291352de1a..0265b2428046 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -174,6 +174,6 @@ trace.jaeger.agentstringthe address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as :. If no port is specified, 6381 will be used. trace.opentelemetry.collectorstringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.zipkin.collectorstringthe address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -versionversion21.2-32set the active cluster version in the format '.' +versionversion21.2-34set the active cluster version in the format '.' diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index 6dfc23b8fea8..46e12cb435b5 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -141,6 +141,7 @@ go_test( "full_cluster_backup_restore_test.go", "helpers_test.go", "import_spans_test.go", + "insert_missing_public_schema_namespace_entry_restore_test.go", "key_rewriter_test.go", "main_test.go", "partitioned_backup_test.go", @@ -193,6 +194,7 @@ go_test( "//pkg/settings/cluster", "//pkg/sql", "//pkg/sql/catalog", + "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/catalogkv", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", diff --git a/pkg/ccl/backupccl/insert_missing_public_schema_namespace_entry_restore_test.go b/pkg/ccl/backupccl/insert_missing_public_schema_namespace_entry_restore_test.go new file mode 100644 index 000000000000..7206c1e5815d --- /dev/null +++ b/pkg/ccl/backupccl/insert_missing_public_schema_namespace_entry_restore_test.go @@ -0,0 +1,93 @@ +// Copyright 2021 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package backupccl_test + +import ( + "context" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +func TestInsertMissingPublicSchemaNamespaceEntry(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + dir, cleanup := testutils.TempDir(t) + defer cleanup() + tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + ExternalIODir: dir, + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: 1, + BinaryVersionOverride: clusterversion.ByKey(clusterversion.InsertPublicSchemaNamespaceEntryOnRestore - 1), + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + db := tc.ServerConn(0) + defer db.Close() + sqlDB := sqlutils.MakeSQLRunner(tc.Conns[0]) + + // Mimic a restore where the public schema system.namespace entries are + // missing. + sqlDB.Exec(t, `CREATE DATABASE db1`) + sqlDB.Exec(t, `CREATE TABLE db1.t()`) + sqlDB.Exec(t, `CREATE SCHEMA db1.s`) + sqlDB.Exec(t, `CREATE DATABASE db2`) + sqlDB.Exec(t, `CREATE TABLE db2.t(x INT)`) + sqlDB.Exec(t, `INSERT INTO db2.t VALUES (1), (2)`) + sqlDB.Exec(t, `CREATE SCHEMA db2.s`) + sqlDB.Exec(t, `CREATE TABLE db2.s.t(x INT)`) + sqlDB.Exec(t, `INSERT INTO db2.s.t VALUES (1), (2)`) + + var db1ID, db2ID descpb.ID + row := sqlDB.QueryRow(t, `SELECT id FROM system.namespace WHERE name = 'db1'`) + row.Scan(&db1ID) + row = sqlDB.QueryRow(t, `SELECT id FROM system.namespace WHERE name = 'db2'`) + row.Scan(&db2ID) + + // Remove system.namespace entries for the public schema for the two + // databases. + err := tc.Servers[0].DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + codec := keys.SystemSQLCodec + b := txn.NewBatch() + b.Del(catalogkeys.MakeSchemaNameKey(codec, db1ID, `public`)) + b.Del(catalogkeys.MakeSchemaNameKey(codec, db2ID, `public`)) + return txn.Run(ctx, b) + }) + require.NoError(t, err) + + // Verify that there are no system.namespace entries for the public schema for + // the two databases. + sqlDB.CheckQueryResults(t, fmt.Sprintf(`SELECT id FROM system.namespace WHERE name = 'public' AND "parentID"=%d`, db1ID), [][]string{}) + sqlDB.CheckQueryResults(t, fmt.Sprintf(`SELECT id FROM system.namespace WHERE name = 'public' AND "parentID"=%d`, db2ID), [][]string{}) + + // Kick off migration by upgrading to the new version. + _ = sqlDB.Exec(t, `SET CLUSTER SETTING version = $1`, + clusterversion.ByKey(clusterversion.InsertPublicSchemaNamespaceEntryOnRestore).String()) + + sqlDB.CheckQueryResults(t, fmt.Sprintf(`SELECT id FROM system.namespace WHERE name = 'public' AND "parentID"=%d`, db1ID), [][]string{{"29"}}) + sqlDB.CheckQueryResults(t, fmt.Sprintf(`SELECT id FROM system.namespace WHERE name = 'public' AND "parentID"=%d`, db2ID), [][]string{{"29"}}) + +} diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index eccb36333f63..4a97fc190423 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -291,8 +291,9 @@ const ( // MVCCAddSSTable supports MVCC-compliant AddSSTable requests via the new // WriteAtRequestTimestamp and DisallowConflicts parameters. MVCCAddSSTable - // Public schema is backed by a descriptor. - PublicSchemasWithDescriptors + // InsertPublicSchemaNamespaceEntryOnRestore ensures all public schemas + // have an entry in system.namespace upon being restored. + InsertPublicSchemaNamespaceEntryOnRestore // UnsplitRangesInAsyncGCJobs moves ranges unsplitting from transaction of // "drop table"/"truncate table" to async gc jobs UnsplitRangesInAsyncGCJobs @@ -325,6 +326,8 @@ const ( // This version comes with a migration to populate the same seed data // for existing tenants. SeedTenantSpanConfigs + // Public schema is backed by a descriptor. + PublicSchemasWithDescriptors // ************************************************* // Step (1): Add new versions here. @@ -539,7 +542,7 @@ var versionsSingleton = keyedVersions{ Version: roachpb.Version{Major: 21, Minor: 2, Internal: 16}, }, { - Key: PublicSchemasWithDescriptors, + Key: InsertPublicSchemaNamespaceEntryOnRestore, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 18}, }, { @@ -570,6 +573,10 @@ var versionsSingleton = keyedVersions{ Key: SeedTenantSpanConfigs, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 32}, }, + { + Key: PublicSchemasWithDescriptors, + Version: roachpb.Version{Major: 21, Minor: 2, Internal: 34}, + }, // ************************************************* // Step (2): Add new versions here. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 7da3b3fa26ab..4a0a5e7adf65 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -51,7 +51,7 @@ func _() { _ = x[AlterSystemTableStatisticsAddAvgSizeCol-40] _ = x[AlterSystemStmtDiagReqs-41] _ = x[MVCCAddSSTable-42] - _ = x[PublicSchemasWithDescriptors-43] + _ = x[InsertPublicSchemaNamespaceEntryOnRestore-43] _ = x[UnsplitRangesInAsyncGCJobs-44] _ = x[ValidateGrantOption-45] _ = x[PebbleFormatBlockPropertyCollector-46] @@ -59,11 +59,12 @@ func _() { _ = x[SelectRPCsTakeTracingInfoInband-48] _ = x[PreSeedTenantSpanConfigs-49] _ = x[SeedTenantSpanConfigs-50] + _ = x[PublicSchemasWithDescriptors-51] } -const _Key_name = "V21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoffRecordsBasedRegistryAutoSpanConfigReconciliationJobDefaultPrivilegesZonesTableForSecondaryTenantsUseKeyEncodeForHashShardedIndexesDatabasePlacementPolicyGeneratedAsIdentityOnUpdateExpressionsSpanConfigurationsTableBoundedStalenessDateAndIntervalStylePebbleFormatVersionedMarkerDataKeysRegistryPebbleSetWithDeleteTenantUsageSingleConsumptionColumnSQLStatsTablesSQLStatsCompactionScheduledJobV21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTablePublicSchemasWithDescriptorsUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigs" +const _Key_name = "V21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoffRecordsBasedRegistryAutoSpanConfigReconciliationJobDefaultPrivilegesZonesTableForSecondaryTenantsUseKeyEncodeForHashShardedIndexesDatabasePlacementPolicyGeneratedAsIdentityOnUpdateExpressionsSpanConfigurationsTableBoundedStalenessDateAndIntervalStylePebbleFormatVersionedMarkerDataKeysRegistryPebbleSetWithDeleteTenantUsageSingleConsumptionColumnSQLStatsTablesSQLStatsCompactionScheduledJobV21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptors" -var _Key_index = [...]uint16{0, 5, 18, 27, 42, 71, 88, 105, 154, 168, 188, 204, 221, 248, 283, 308, 337, 368, 388, 419, 436, 465, 498, 521, 540, 559, 582, 598, 618, 639, 661, 680, 714, 728, 758, 763, 772, 794, 812, 834, 871, 910, 933, 947, 975, 1001, 1020, 1054, 1066, 1097, 1121, 1142} +var _Key_index = [...]uint16{0, 5, 18, 27, 42, 71, 88, 105, 154, 168, 188, 204, 221, 248, 283, 308, 337, 368, 388, 419, 436, 465, 498, 521, 540, 559, 582, 598, 618, 639, 661, 680, 714, 728, 758, 763, 772, 794, 812, 834, 871, 910, 933, 947, 988, 1014, 1033, 1067, 1079, 1110, 1134, 1155, 1183} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/migration/migrations/BUILD.bazel b/pkg/migration/migrations/BUILD.bazel index 144f8ce1c229..d349629d33f4 100644 --- a/pkg/migration/migrations/BUILD.bazel +++ b/pkg/migration/migrations/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "delete_deprecated_namespace_tabledesc.go", "ensure_no_draining_names.go", "fix_descriptor_migration.go", + "insert_missing_public_schema_namespace_entry.go", "join_tokens.go", "migrations.go", "records_based_registry.go", diff --git a/pkg/migration/migrations/insert_missing_public_schema_namespace_entry.go b/pkg/migration/migrations/insert_missing_public_schema_namespace_entry.go new file mode 100644 index 000000000000..28bf9b8ed635 --- /dev/null +++ b/pkg/migration/migrations/insert_missing_public_schema_namespace_entry.go @@ -0,0 +1,82 @@ +// 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/jobs" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/migration" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) + +// insertMissingPublicSchemaNamespaceEntry creates a system.namespace entries +// for public schemas that are missing a system.namespace entry. +// This arises from restore where we mistakenly did not create system.namespace +// entries for public schemas when restoring databases. +func insertMissingPublicSchemaNamespaceEntry( + ctx context.Context, _ clusterversion.ClusterVersion, d migration.TenantDeps, _ *jobs.Job, +) error { + // Get the ID of all databases where we're missing a public schema namespace + // entry for. + query := ` + SELECT id + FROM system.namespace + WHERE id + NOT IN ( + SELECT ns_db.id + FROM system.namespace AS ns_db + INNER JOIN system.namespace + AS ns_sc ON ( + ns_db.id + = ns_sc."parentID" + ) + WHERE ns_db."parentSchemaID" = 0 + AND ns_db."parentID" = 0 + AND ns_sc."parentSchemaID" = 0 + AND ns_sc.name = 'public' + AND ns_sc.id = 29 + ) + AND "parentID" = 0 +ORDER BY id ASC; +` + rows, err := d.InternalExecutor.QueryIterator( + ctx, "get_databases_without_public_schema_namespace_entry", nil /* txn */, query, + ) + if err != nil { + return err + } + var databaseIDs []descpb.ID + for ok, err := rows.Next(ctx); ok; ok, err = rows.Next(ctx) { + if err != nil { + return err + } + id := descpb.ID(tree.MustBeDInt(rows.Cur()[0])) + databaseIDs = append(databaseIDs, id) + } + + return d.CollectionFactory.Txn(ctx, d.InternalExecutor, d.DB, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + ) error { + b := txn.NewBatch() + for _, dbID := range databaseIDs { + publicSchemaKey := catalogkeys.MakeSchemaNameKey(d.Codec, dbID, tree.PublicSchema) + b.Put(publicSchemaKey, keys.PublicSchemaID) + } + return txn.Run(ctx, b) + }) +} diff --git a/pkg/migration/migrations/migrations.go b/pkg/migration/migrations/migrations.go index e9dc99b64743..c294903ee2b2 100644 --- a/pkg/migration/migrations/migrations.go +++ b/pkg/migration/migrations/migrations.go @@ -148,6 +148,11 @@ var migrations = []migration.Migration{ NoPrecondition, seedTenantSpanConfigsMigration, ), + migration.NewTenantMigration("insert missing system.namespace entries for public schemas", + toCV(clusterversion.InsertPublicSchemaNamespaceEntryOnRestore), + NoPrecondition, + insertMissingPublicSchemaNamespaceEntry, + ), } func init() {