Skip to content

Commit

Permalink
Merge #62017
Browse files Browse the repository at this point in the history
62017: migration: add protected_ts_meta privilege migration r=ajwerner a=postamar

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

Co-authored-by: Marius Posta <[email protected]>
  • Loading branch information
craig[bot] and Marius Posta committed Mar 15, 2021
2 parents ae3a358 + 0b3d837 commit 77a21a4
Show file tree
Hide file tree
Showing 8 changed files with 175 additions and 4 deletions.
2 changes: 1 addition & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -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://<ui>/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 '<major>.<minor>'
version version 20.2-48 set the active cluster version in the format '<major>.<minor>'
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,6 @@
<tr><td><code>trace.debug.enable</code></td><td>boolean</td><td><code>false</code></td><td>if set, traces for recent requests can be seen at https://<ui>/debug/requests</td></tr>
<tr><td><code>trace.lightstep.token</code></td><td>string</td><td><code></code></td><td>if set, traces go to Lightstep using this token</td></tr>
<tr><td><code>trace.zipkin.collector</code></td><td>string</td><td><code></code></td><td>if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'). Only one tracer can be configured at a time.</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>20.2-46</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>20.2-48</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
</tbody>
</table>
7 changes: 7 additions & 0 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
)
Expand Down Expand Up @@ -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.
})

Expand Down
5 changes: 3 additions & 2 deletions pkg/clusterversion/key_string.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

4 changes: 4 additions & 0 deletions pkg/migration/migrations/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down Expand Up @@ -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 = [
Expand All @@ -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",
Expand Down
5 changes: 5 additions & 0 deletions pkg/migration/migrations/migrations.go
Original file line number Diff line number Diff line change
Expand Up @@ -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() {
Expand Down
47 changes: 47 additions & 0 deletions pkg/migration/migrations/protected_ts_meta_migration.go
Original file line number Diff line number Diff line change
@@ -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)
},
)
}
107 changes: 107 additions & 0 deletions pkg/migration/migrations/protected_ts_meta_migration_external_test.go
Original file line number Diff line number Diff line change
@@ -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)
}

0 comments on commit 77a21a4

Please sign in to comment.