From 0b3d8371ff2050c25b78735f5d4442e8e804792b Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Mon, 15 Mar 2021 12:06:26 -0400 Subject: [PATCH] migration: add protected_ts_meta privilege migration In 20.2 and prior versions, the protected_ts_meta system table had incorrect superuser privileges. This was fixed in a recent commit in the 21.2 release branch. However, there was no associated migration job so the descriptors in storage remainted untouched. This remained unnoticed until now because most of the time, system table descriptors are retrieved from an in-memory cache. However, tools like doctor will read the descriptors from storage and will report validation failures. This commit adds the missing migration job which fixes the protected_ts_meta descriptor in the descriptors table by setting the proper privileges. Fixes #61950. Release note: None --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/clusterversion/cockroach_versions.go | 7 ++ pkg/clusterversion/key_string.go | 5 +- pkg/migration/migrations/BUILD.bazel | 4 + pkg/migration/migrations/migrations.go | 5 + .../migrations/protected_ts_meta_migration.go | 47 ++++++++ ...otected_ts_meta_migration_external_test.go | 107 ++++++++++++++++++ 8 files changed, 175 insertions(+), 4 deletions(-) create mode 100644 pkg/migration/migrations/protected_ts_meta_migration.go create mode 100644 pkg/migration/migrations/protected_ts_meta_migration_external_test.go diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 763102319aaf..419ba49215d2 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -102,4 +102,4 @@ trace.datadog.project string CockroachDB the project under which traces will be trace.debug.enable boolean false if set, traces for recent requests can be seen at https:///debug/requests trace.lightstep.token string if set, traces go to Lightstep using this token trace.zipkin.collector string if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'). Only one tracer can be configured at a time. -version version 20.2-46 set the active cluster version in the format '.' +version version 20.2-48 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index df9488b94c03..9f0370eb2607 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -104,6 +104,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'). Only one tracer can be configured at a time. -versionversion20.2-46set the active cluster version in the format '.' +versionversion20.2-48set the active cluster version in the format '.' diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index c4139a0bbb2a..651af26ac892 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -272,6 +272,9 @@ const ( PriorReadSummaries // NonVotingReplicas enables the creation of non-voting replicas. NonVotingReplicas + // ProtectedTsMetaPrivilegesMigration is for the migration which fixes the + // privileges of the protected_ts_meta system table. + ProtectedTsMetaPrivilegesMigration // Step (1): Add new versions here. ) @@ -470,6 +473,10 @@ var versionsSingleton = keyedVersions([]keyedVersion{ Key: NonVotingReplicas, Version: roachpb.Version{Major: 20, Minor: 2, Internal: 46}, }, + { + Key: ProtectedTsMetaPrivilegesMigration, + Version: roachpb.Version{Major: 20, Minor: 2, Internal: 48}, + }, // Step (2): Add new versions here. }) diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 0ddfe2369caa..3b78d363f6aa 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -51,11 +51,12 @@ func _() { _ = x[ForeignKeyRepresentationMigration-40] _ = x[PriorReadSummaries-41] _ = x[NonVotingReplicas-42] + _ = x[ProtectedTsMetaPrivilegesMigration-43] } -const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesNodeMembershipStatusMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesUniqueWithoutIndexConstraintsVirtualComputedColumnsCPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationNewSchemaChangerLongRunningMigrationsTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationSeparatedIntentsTracingVerbosityIndependentSemanticsSequencesRegclassImplicitColumnPartitioningMultiRegionFeaturesClosedTimestampsRaftTransportChangefeedsSupportPrimaryIndexChangesNamespaceTableWithSchemasMigrationForeignKeyRepresentationMigrationPriorReadSummariesNonVotingReplicas" +const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesNodeMembershipStatusMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesUniqueWithoutIndexConstraintsVirtualComputedColumnsCPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationNewSchemaChangerLongRunningMigrationsTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationSeparatedIntentsTracingVerbosityIndependentSemanticsSequencesRegclassImplicitColumnPartitioningMultiRegionFeaturesClosedTimestampsRaftTransportChangefeedsSupportPrimaryIndexChangesNamespaceTableWithSchemasMigrationForeignKeyRepresentationMigrationPriorReadSummariesNonVotingReplicasProtectedTsMetaPrivilegesMigration" -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, 725, 761, 778, 804, 823, 852, 889, 923, 956, 974, 991} +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, 725, 761, 778, 804, 823, 852, 889, 923, 956, 974, 991, 1025} 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 ee2b7d6b50ed..dc2d58e07a5a 100644 --- a/pkg/migration/migrations/BUILD.bazel +++ b/pkg/migration/migrations/BUILD.bazel @@ -7,6 +7,7 @@ go_library( "migrations.go", "migrations_table.go", "namespace_migration.go", + "protected_ts_meta_migration.go", "truncated_state.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/migration/migrations", @@ -40,6 +41,7 @@ go_test( "foreign_key_representation_upgrade_external_test.go", "main_test.go", "namespace_migration_external_test.go", + "protected_ts_meta_migration_external_test.go", "truncated_state_external_test.go", ], deps = [ @@ -53,7 +55,9 @@ go_test( "//pkg/security/securitytest", "//pkg/server", "//pkg/sql/catalog/catalogkeys", + "//pkg/sql/catalog/catalogkv", "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/systemschema", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", diff --git a/pkg/migration/migrations/migrations.go b/pkg/migration/migrations/migrations.go index dedf1ca72f0f..66655dfc5495 100644 --- a/pkg/migration/migrations/migrations.go +++ b/pkg/migration/migrations/migrations.go @@ -57,6 +57,11 @@ var migrations = []migration.Migration{ toCV(clusterversion.ForeignKeyRepresentationMigration), foreignKeyRepresentationUpgrade, ), + migration.NewSQLMigration( + "fix system.protected_ts_meta privileges", + toCV(clusterversion.ProtectedTsMetaPrivilegesMigration), + protectedTsMetaPrivilegesMigration, + ), } func init() { diff --git a/pkg/migration/migrations/protected_ts_meta_migration.go b/pkg/migration/migrations/protected_ts_meta_migration.go new file mode 100644 index 000000000000..2948b2c7cb30 --- /dev/null +++ b/pkg/migration/migrations/protected_ts_meta_migration.go @@ -0,0 +1,47 @@ +// 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/kv" + "github.com/cockroachdb/cockroach/pkg/migration" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +func protectedTsMetaPrivilegesMigration( + ctx context.Context, _ clusterversion.ClusterVersion, d migration.SQLDeps, +) error { + id := systemschema.ProtectedTimestampsMetaTable.GetID() + return descs.Txn(ctx, d.Settings, d.LeaseManager, d.InternalExecutor, d.DB, + func(ctx context.Context, txn *kv.Txn, descriptors *descs.Collection) error { + log.Infof(ctx, "%s", "updating privileges in system.protected_ts_meta descriptor") + mut, err := descriptors.GetMutableTableByID(ctx, txn, id, tree.ObjectLookupFlagsWithRequired()) + if err != nil { + return err + } + if mut.GetVersion() > 1 { + // Descriptor has already been upgraded, skip. + return nil + } + // Privileges have already been fixed at this point by the descriptor + // unwrapping logic in catalogkv which runs post-deserialization changes, + // but we still need to bump the version number. + mut.Version = 2 + return descriptors.WriteDesc(ctx, false /* kvTrace */, mut, txn) + }, + ) +} diff --git a/pkg/migration/migrations/protected_ts_meta_migration_external_test.go b/pkg/migration/migrations/protected_ts_meta_migration_external_test.go new file mode 100644 index 000000000000..65fa624df379 --- /dev/null +++ b/pkg/migration/migrations/protected_ts_meta_migration_external_test.go @@ -0,0 +1,107 @@ +// 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_test + +import ( + "context" + "fmt" + "strconv" + "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/security" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "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 TestProtectedTimestampMetaMigration(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: 1, + BinaryVersionOverride: clusterversion.ByKey(clusterversion.ProtectedTsMetaPrivilegesMigration - 1), + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + // Replicate bad descriptor privilege bug. + err := tc.Servers[0].DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + id := systemschema.ProtectedTimestampsMetaTable.GetID() + mut, err := catalogkv.MustGetMutableTableDescByID(ctx, txn, keys.SystemSQLCodec, id) + if err != nil { + return err + } + mut.Version = 1 + mut.Privileges = descpb.NewCustomSuperuserPrivilegeDescriptor( + descpb.SystemAllowedPrivileges[keys.ReplicationStatsTableID], security.NodeUserName()) + b := txn.NewBatch() + b.Put(catalogkeys.MakeDescMetadataKey(keys.SystemSQLCodec, id), mut.DescriptorProto()) + return txn.Run(ctx, b) + }) + require.NoError(t, err) + + tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + checkPrivileges := func(expectedPrivileges int, expectedVersion int) { + expectedStr := fmt.Sprintf( + `{"ownerProto": "node", "users": [`+ + `{"privileges": %d, "userProto": "admin"}, `+ + `{"privileges": %d, "userProto": "root"}`+ + `], "version": 1}`, + expectedPrivileges, + expectedPrivileges, + ) + var actualStr string + tdb.QueryRow(t, ` + SELECT + crdb_internal.pb_to_json( + 'cockroach.sql.sqlbase.Descriptor', + descriptor, + false + )->'table'->>'privileges' + FROM system.descriptor WHERE id = 31 + `).Scan(&actualStr) + require.EqualValues(t, expectedStr, actualStr) + var actualVersionStr string + tdb.QueryRow(t, ` + SELECT + crdb_internal.pb_to_json( + 'cockroach.sql.sqlbase.Descriptor', + descriptor, + false + )->'table'->>'version' + FROM system.descriptor WHERE id = 31 + `).Scan(&actualVersionStr) + require.EqualValues(t, strconv.Itoa(expectedVersion), actualVersionStr) + } + + checkPrivileges(496, 1) + + tdb.Exec(t, `SET CLUSTER SETTING version = $1`, + clusterversion.ByKey(clusterversion.ProtectedTsMetaPrivilegesMigration).String()) + + checkPrivileges(48, 2) +}