Skip to content

Commit

Permalink
sql: migration for privilege descriptors.
Browse files Browse the repository at this point in the history
This migration fixes corrupted privilege descriptors in the wild resulting
from the fallout of the ZONECONFIG/USAGE bug and the ALTER DATABASE CONVERT TO
SCHEMA bug. Additionally, all privilege descriptors will be updated to explicitly
have Owners. Privilege descriptors created before 20.2 do not have their owner field
set and instead owners are implicitly inferred.

Finally all privilege descriptors will be bumped up to Version21_2 after the migration
so we can strictly validate privileges.

Release note (bug fix): Summary: upon upgrading to 21.2,
users will have all privileges fixed. This means
any invalid privileges on all objects will be removed.

Details:
This migration will be run when users upgrade to
21.2 (Specifically 21.2.14).
This migration fixes any privilege descriptors that are corrupted from the
fallout of the ZONECONFIG/USAGE bug on tables and databases after upgrading
from 20.1 -> 20.2 (cockroachdb#65010) and those that are corrupted after converting a
database to a schema (cockroachdb#65697).
  • Loading branch information
RichardJCai committed Jun 22, 2021
1 parent e1697d1 commit 3fc4b38
Show file tree
Hide file tree
Showing 28 changed files with 503 additions and 103 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 @@ -111,4 +111,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 21.1-12 set the active cluster version in the format '<major>.<minor>'
version version 21.1-14 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 @@ -113,6 +113,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>21.1-12</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>21.1-14</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
</tbody>
</table>
6 changes: 6 additions & 0 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -300,6 +300,8 @@ const (
ExpressionBasedIndexes
// DeleteDeprecatedNamespaceTableDescriptorMigration deletes the descriptor at ID=2.
DeleteDeprecatedNamespaceTableDescriptorMigration
// FixedPrivilegeDescriptors is for the migration to fix all privilege descriptors.
FixedPrivilegeDescriptors

// Step (1): Add new versions here.
)
Expand Down Expand Up @@ -501,6 +503,10 @@ var versionsSingleton = keyedVersions([]keyedVersion{
Key: DeleteDeprecatedNamespaceTableDescriptorMigration,
Version: roachpb.Version{Major: 21, Minor: 1, Internal: 12},
},
{
Key: FixedPrivilegeDescriptors,
Version: roachpb.Version{Major: 21, Minor: 1, Internal: 14},
},

// 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.

6 changes: 5 additions & 1 deletion pkg/migration/migrations/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ go_library(
name = "migrations",
srcs = [
"delete_deprecated_namespace_tabledesc.go",
"fix_privileges_migration.go",
"foreign_key_representation_upgrade.go",
"join_tokens.go",
"migrations.go",
Expand All @@ -20,10 +21,13 @@ go_library(
"//pkg/migration",
"//pkg/roachpb",
"//pkg/server/serverpb",
"//pkg/sql/catalog",
"//pkg/sql/catalog/catalogkeys",
"//pkg/sql/catalog/catalogkv",
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/descs",
"//pkg/sql/catalog/systemschema",
"//pkg/sql/privilege",
"//pkg/sql/sem/tree",
"//pkg/sql/sqlutil",
"//pkg/sqlmigrations",
Expand All @@ -36,9 +40,9 @@ go_library(

go_test(
name = "migrations_test",
size = "medium",
srcs = [
"delete_deprecated_namespace_tabledesc_external_test.go",
"fix_privileges_migration_external_test.go",
"foreign_key_representation_upgrade_external_test.go",
"main_test.go",
"protected_ts_meta_migration_external_test.go",
Expand Down
122 changes: 122 additions & 0 deletions pkg/migration/migrations/fix_privileges_migration.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,122 @@
// 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"
"fmt"

"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"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/errors"
)

// getMutableDescFromDescriptorRow takes in an InternalRow from the query:
// SELECT id, descriptor, crdb_internal_mvcc_timestamp FROM system.descriptor ORDER BY ID ASC
// and parses the id, mvcc_timestamp and descriptor fields to create and return
// a MutableDescriptor.
func getMutableDescFromDescriptorRow(rows sqlutil.InternalRows) (catalog.MutableDescriptor, error) {
row := rows.Cur()
id := descpb.ID(*row[0].(*tree.DInt))
ts, err := tree.DecimalToHLC(&row[2].(*tree.DDecimal).Decimal)
if err != nil {
return nil, errors.Wrapf(err,
"failed to convert MVCC timestamp decimal to HLC for ID %d", id)
}
var desc descpb.Descriptor
if err := protoutil.Unmarshal(([]byte)(*row[1].(*tree.DBytes)), &desc); err != nil {
return nil, errors.Wrapf(err,
"failed to unmarshal descriptor with ID %d", id)
}

b := catalogkv.NewBuilderWithMVCCTimestamp(&desc, ts)
if b == nil {
return nil, errors.Newf("unable to find descriptor for id %d", id)
}

return b.BuildExistingMutable(), nil
}

func descriptorTypeToObjectType(b catalog.MutableDescriptor) privilege.ObjectType {
switch b.DescriptorType() {
case catalog.Database:
return privilege.Database
case catalog.Schema:
return privilege.Schema
case catalog.Table:
return privilege.Table
case catalog.Type:
return privilege.Type
default:
panic(fmt.Sprintf("unexpected descriptor type: %s", b.DescriptorType()))
}
}

func upgradePrivileges(
ctx context.Context, desc catalog.MutableDescriptor, d migration.TenantDeps,
) error {
objectType := descriptorTypeToObjectType(desc)
return descs.Txn(ctx, d.Settings, d.LeaseManager, d.InternalExecutor, d.DB, func(
ctx context.Context, txn *kv.Txn, descriptors *descs.Collection) error {

privs := desc.GetPrivileges()
descpb.MaybeFixPrivileges(
desc.GetID(), desc.GetParentID(),
&privs, objectType,
)

return descriptors.WriteDesc(ctx, false /* kvTrace */, desc, txn)
})
}

// fixPrivilegesMigration calls descpb.MaybeFixPrivileges on every PrivilegeDescriptor
// that is not yet on Version21_2. After the migration, all PrivilegeDescriptors
// should have valid privileges, have it's owner field populated and be on
// Version21_2.
func fixPrivilegesMigration(
ctx context.Context, _ clusterversion.ClusterVersion, d migration.TenantDeps,
) error {
rows, err := d.InternalExecutor.QueryIterator(ctx, "fix-privileges", nil, /* txn */
`
SELECT id, descriptor, crdb_internal_mvcc_timestamp FROM system.descriptor ORDER BY ID ASC`)
if err != nil {
return err
}
defer func() { _ = rows.Close() }()
ok, err := rows.Next(ctx)
for ; ok; ok, err = rows.Next(ctx) {
desc, err := getMutableDescFromDescriptorRow(rows)
if err != nil {
return err
}
if desc.GetPrivileges().Version >= descpb.Version21_2 {
log.Infof(ctx, "privilege descriptor has already been fixed, skipping %d", desc.GetID())
continue
}
if err := upgradePrivileges(ctx, desc, d); err != nil {
return err
}
}
if err != nil {
return err
}
return nil
}
192 changes: 192 additions & 0 deletions pkg/migration/migrations/fix_privileges_migration_external_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,192 @@
// 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"
"encoding/hex"
"errors"
"strings"
"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/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/stretchr/testify/require"
)

func TestFixPrivilegesMigration(t *testing.T) {
defer leaktest.AfterTest(t)()
// The SQL statements below are run on a 20.1.2 cluster, the ZONECONFIG bit
// becomes the USAGE bit in versions >20.2 causing the privilege descriptor
// to become corrupted since USAGE is not valid on tables and dbs.
/*
CREATE USER testuser;
CREATE DATABASE db;
GRANT ZONECONFIG ON DATABASE db TO testuser;
USE db;
CREATE TABLE tb();
GRANT ZONECONFIG ON tb TO testuser;
The descriptors were then extracted using:
SELECT encode(descriptor, 'hex') AS descriptor
FROM system.descriptor
WHERE id
IN (
SELECT id
FROM system.namespace
WHERE "parentID"
= (
SELECT id
FROM system.namespace
WHERE "parentID" = 0 AND name = 'db'
)
OR "parentID" = 0 AND name = 'db'
);
*/
const descriptorStringsToInjectZoneConfigFixTest = `
122c0a026462103b1a240a090a0561646d696e10020a080a04726f6f7410020a0d0a087465737475736572108004
0afe010a027462183c203b28023a00422b0a05726f77696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293001480252480a077072696d617279100118012205726f776964300140004a10080010001a00200028003000380040005a007a020800800100880100900100980100a20106080012001800a8010060026a240a090a0561646d696e10020a080a04726f6f7410020a0d0a087465737475736572108004800101880103980100b201160a077072696d61727910001a05726f77696420012800b80101c20100e80100f2010408001200f801008002009202009a020a08d0b7a9c4fcdd9fc416b20200b80200c0021d
`

// Test fixing a corrupted schema privilege descriptor after converting
// from a database to a schema. The SQL statements below are run on a 20.2.5
// cluster where ALTER DATABASE ... CONVERT TO SCHEMA did not correctly handle
// privileges. The "schema" is corrupted due to testuser having SELECT
// privilege on it.
/*
CREATE DATABASE parent;
CREATE DATABASE schema;
CREATE USER TESTUSER;
GRANT SELECT ON DATABASE SCHEMA TO TESTUSER;
ALTER DATABASE schema CONVERT TO SCHEMA WITH PARENT parent;
The descriptors were then extracted using:
SELECT encode(descriptor, 'hex') AS descriptor
FROM system.descriptor
WHERE id
IN (
SELECT id
FROM system.namespace
WHERE "parentID"
= (
SELECT id
FROM system.namespace
WHERE "parentID" = 0 AND name = 'parent'
)
OR "parentID" = 0 AND name = 'parent'
);
*/
const descriptorStringsToInjectSchemaTest = `
12410a06706172656e74103b1a1d0a090a0561646d696e10020a080a04726f6f7410021204726f6f741801220028023a0e0a06736368656d611204083d100040004a00
2241083b1206736368656d61183d222b0a090a0561646d696e10020a080a04726f6f7410020a0c0a08746573747573657210201204726f6f7418012a00300140004a00
`

// Ensure types can also be upgraded. Types in practice should not be corrupted
// and should always have an owner since they're created in 20.2 and onwards.
// We still want to upgrade the Version of the privilege descriptor.
// The following SQL statements were run in a 20.2.5 cluster.
/*
CREATE DATABASE db;
USE db;
CREATE TYPE t AS ENUM();
The descriptors were then extracted using:
SELECT encode(descriptor, 'hex') AS descriptor
FROM system.descriptor
WHERE id
IN (
SELECT id
FROM system.namespace
WHERE "parentID"
= (
SELECT id
FROM system.namespace
WHERE "parentID" = 0 AND name = 'db'
)
OR "parentID" = 0 AND name = 'db'
);
*/
const descriptorStringsToInjectUserDefinedTypeTest = `
122d0a026462103d1a1d0a090a0561646d696e10020a080a04726f6f7410021204726f6f7418012200280140004a00
1a41083d101d1a0174203e2800403f480152006800722a0a090a0561646d696e10020a0b0a067075626c69631080040a080a04726f6f7410021204726f6f7418017a00
1a6a083d101d1a025f74203f28013a26080f100018003000381850df8d065a14081810001800300050de8d0660007a0410df8d0660004000480152006800722a0a090a0561646d696e10020a0b0a067075626c69631080040a080a04726f6f7410021204726f6f7418017a00
`

testCases := []string{
descriptorStringsToInjectZoneConfigFixTest,
descriptorStringsToInjectSchemaTest,
descriptorStringsToInjectUserDefinedTypeTest,
}
for _, descriptorStringsToInject := range testCases {
var descriptorsToInject []*descpb.Descriptor
for _, s := range strings.Split(strings.TrimSpace(descriptorStringsToInject), "\n") {
encoded, err := hex.DecodeString(s)
require.NoError(t, err)
var desc descpb.Descriptor
require.NoError(t, protoutil.Unmarshal(encoded, &desc))
descriptorsToInject = append(descriptorsToInject, &desc)
}

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.FixedPrivilegeDescriptors - 1),
},
},
},
})

db := tc.ServerConn(0)
kvDB := tc.Server(0).DB()
require.NoError(t, sqlutils.InjectDescriptors(
ctx, db, descriptorsToInject, true, /* force */
))

_, err := tc.Conns[0].ExecContext(ctx, `SET CLUSTER SETTING version = $1`,
clusterversion.ByKey(clusterversion.FixedPrivilegeDescriptors).String())
require.NoError(t, err)

err = kvDB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
// GetAllDescriptors calls PrivilegeDescriptor.Validate, all the invalid
// descriptors should be updated.
descs, err := catalogkv.GetAllDescriptors(ctx, txn, keys.SystemSQLCodec)
if err != nil {
return err
}

for _, desc := range descs {
privilegeDesc := desc.GetPrivileges()
if privilegeDesc.Version < descpb.Version21_2 {
return errors.New("privilege descriptors must have at least Version21_2")
}
}
return nil
})

require.NoError(t, err)
tc.Stopper().Stop(ctx)
}
}
Loading

0 comments on commit 3fc4b38

Please sign in to comment.