From e0c82d9a6f00c96dab1f132ee07049351e52a168 Mon Sep 17 00:00:00 2001 From: Jack Wu Date: Tue, 9 Nov 2021 16:24:48 -0500 Subject: [PATCH] sql: migration and mixed cluster version roachtest for grant option The roachtest ensures that the validation check for grant options that is performed on a grant/revoke only runs when all nodes on a cluster are upgraded to 22.1. The migration scans through existing users and sets their grant option bits equal to their privilege bits if they have GRANT or ALL privileges (otherwise, it would have been impossible for them to grant anyways, so we do not do anything). Release note: None --- pkg/cmd/roachtest/cluster.go | 22 ++ .../roachtest/cluster/cluster_interface.go | 2 +- pkg/cmd/roachtest/tests/BUILD.bazel | 1 + pkg/cmd/roachtest/tests/registry.go | 1 + .../roachtest/tests/validate_grant_option.go | 152 +++++++++++++ pkg/migration/migrations/BUILD.bazel | 7 + .../migrations/grant_option_migration.go | 159 +++++++++++++ .../grant_option_migration_external_test.go | 214 ++++++++++++++++++ pkg/migration/migrations/migrations.go | 6 + .../catalog/catprivilege/default_privilege.go | 2 +- pkg/sql/catalog/catprivilege/fix.go | 33 +++ .../catalog/dbdesc/database_desc_builder.go | 5 +- pkg/sql/catalog/descpb/privilege.go | 19 +- .../descpb/privilegedescversion_string.go | 5 +- .../catalog/schemadesc/schema_desc_builder.go | 6 +- pkg/sql/catalog/systemschema/system.go | 2 +- .../catalog/tabledesc/table_desc_builder.go | 6 +- pkg/sql/catalog/typedesc/type_desc_builder.go | 6 +- pkg/sql/logictest/testdata/logic_test/bytes | 2 +- .../testdata/show_trace_nonmetamorphic | 16 +- pkg/sql/schemachanger/testdata/drop | 48 ++-- 21 files changed, 662 insertions(+), 52 deletions(-) create mode 100644 pkg/cmd/roachtest/tests/validate_grant_option.go create mode 100644 pkg/migration/migrations/grant_option_migration.go create mode 100644 pkg/migration/migrations/grant_option_migration_external_test.go diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index acb1466c227b..4a0cd832248a 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -2388,6 +2388,28 @@ func (c *clusterImpl) ConnE(ctx context.Context, l *logger.Logger, node int) (*g return db, nil } +// ConnEAsUser returns a SQL connection to the specified node as a specific user +func (c *clusterImpl) ConnEAsUser( + ctx context.Context, l *logger.Logger, node int, user string, +) (*gosql.DB, error) { + urls, err := c.ExternalPGUrl(ctx, l, c.Node(node)) + if err != nil { + return nil, err + } + + u, err := url.Parse(urls[0]) + if err != nil { + return nil, err + } + u.User = url.User(user) + dataSourceName := u.String() + db, err := gosql.Open("postgres", dataSourceName) + if err != nil { + return nil, err + } + return db, nil +} + func (c *clusterImpl) MakeNodes(opts ...option.Option) string { var r option.NodeListOption for _, o := range opts { diff --git a/pkg/cmd/roachtest/cluster/cluster_interface.go b/pkg/cmd/roachtest/cluster/cluster_interface.go index b2f04f93c198..74211c648a9a 100644 --- a/pkg/cmd/roachtest/cluster/cluster_interface.go +++ b/pkg/cmd/roachtest/cluster/cluster_interface.go @@ -69,9 +69,9 @@ type Cluster interface { ExternalPGUrl(ctx context.Context, l *logger.Logger, node option.NodeListOption) ([]string, error) // SQL clients to nodes. - Conn(ctx context.Context, l *logger.Logger, node int) *gosql.DB ConnE(ctx context.Context, l *logger.Logger, node int) (*gosql.DB, error) + ConnEAsUser(ctx context.Context, l *logger.Logger, node int, user string) (*gosql.DB, error) // URLs for the Admin UI. diff --git a/pkg/cmd/roachtest/tests/BUILD.bazel b/pkg/cmd/roachtest/tests/BUILD.bazel index 7e21af2a6e37..d8bc4ba29bd0 100644 --- a/pkg/cmd/roachtest/tests/BUILD.bazel +++ b/pkg/cmd/roachtest/tests/BUILD.bazel @@ -142,6 +142,7 @@ go_library( "util_disk_usage.go", "util_if_local.go", "util_load_group.go", + "validate_grant_option.go", "version.go", "version_upgrade_public_schema.go", "versionupgrade.go", diff --git a/pkg/cmd/roachtest/tests/registry.go b/pkg/cmd/roachtest/tests/registry.go index 2fc5c75cd5db..4241ff3af7a7 100644 --- a/pkg/cmd/roachtest/tests/registry.go +++ b/pkg/cmd/roachtest/tests/registry.go @@ -117,6 +117,7 @@ func RegisterTests(r registry.Registry) { registerKVBench(r) registerTypeORM(r) registerLoadSplits(r) + registerValidateGrantOption(r) registerVersion(r) registerYCSB(r) registerTPCHBench(r) diff --git a/pkg/cmd/roachtest/tests/validate_grant_option.go b/pkg/cmd/roachtest/tests/validate_grant_option.go new file mode 100644 index 000000000000..b99975790b9b --- /dev/null +++ b/pkg/cmd/roachtest/tests/validate_grant_option.go @@ -0,0 +1,152 @@ +// 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 tests + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" + "github.com/cockroachdb/cockroach/pkg/util/version" + "github.com/stretchr/testify/require" +) + +func registerValidateGrantOption(r registry.Registry) { + r.Add(registry.TestSpec{ + Name: "sql-experience/validate-grant-option", + Owner: registry.OwnerSQLExperience, + Cluster: r.MakeClusterSpec(3), + Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { + runRegisterValidateGrantOption(ctx, t, c, *t.BuildVersion()) + }, + }) +} + +// execSQL executes the SQL statement as "root". +func execSQL(sqlStatement string, expectedErrText string, node int) versionStep { + return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { + conn, err := u.c.ConnE(ctx, t.L(), node) + require.NoError(t, err) + _, err = conn.Exec(sqlStatement) + if len(expectedErrText) == 0 { + require.NoError(t, err) + } else { + require.EqualError(t, err, expectedErrText) + } + } +} + +// execSQLAsUser executes the SQL statement as the specified user. +func execSQLAsUser(sqlStatement string, user string, expectedErrText string, node int) versionStep { + return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { + conn, err := u.c.ConnEAsUser(ctx, t.L(), node, user) + require.NoError(t, err) + _, err = conn.Exec(sqlStatement) + if len(expectedErrText) == 0 { + require.NoError(t, err) + } else { + require.EqualError(t, err, expectedErrText) + } + } +} + +func runRegisterValidateGrantOption( + ctx context.Context, t test.Test, c cluster.Cluster, buildVersion version.Version, +) { + const mainVersion = "" + predecessorVersion, err := PredecessorVersion(buildVersion) + if err != nil { + t.Fatal(err) + } + + u := newVersionUpgradeTest(c, + uploadAndStart(c.All(), predecessorVersion), + waitForUpgradeStep(c.All()), + preventAutoUpgradeStep(1), + preventAutoUpgradeStep(2), + preventAutoUpgradeStep(3), + + // No nodes have been upgraded; any user can grant or revoke a privilege that they hold (grant options + // are ignored). + execSQL("CREATE USER foo;", "", 1), + execSQL("CREATE USER foo2;", "", 2), + execSQL("CREATE USER foo3;", "", 2), + execSQL("CREATE USER foo4;", "", 2), + execSQL("CREATE USER target;", "", 1), + + execSQL("CREATE DATABASE d;", "", 2), + execSQL("CREATE SCHEMA s;", "", 3), + execSQL("CREATE TABLE t1();", "", 2), + execSQL("CREATE TYPE ty AS ENUM();", "", 3), + + execSQL("GRANT ALL PRIVILEGES ON DATABASE d TO foo;", "", 1), + execSQL("GRANT ALL PRIVILEGES ON SCHEMA s TO foo2;", "", 3), + execSQL("GRANT ALL PRIVILEGES ON TABLE t1 TO foo3;", "", 1), + execSQL("GRANT ALL PRIVILEGES ON TYPE ty TO foo4;", "", 1), + + execSQLAsUser("SELECT * FROM t1;", "foo3", "", 2), + execSQLAsUser("GRANT CREATE ON DATABASE d TO target;", "foo", "", 1), + execSQLAsUser("GRANT USAGE ON SCHEMA s TO target;", "foo2", "", 3), + execSQLAsUser("GRANT SELECT ON TABLE t1 TO target;", "foo3", "", 1), + execSQLAsUser("GRANT USAGE ON TYPE ty TO target;", "foo4", "", 1), + + // Node 1 is upgraded; because nodes 2 and 3 are on the previous version, any user can still grant + // or revoke a privilege that they hold (grant options are ignored). + binaryUpgradeStep(c.Node(1), mainVersion), + allowAutoUpgradeStep(1), + execSQLAsUser("GRANT CREATE ON DATABASE d TO target;", "foo", "", 1), + execSQLAsUser("GRANT USAGE ON SCHEMA s TO target;", "foo2", "", 3), + execSQLAsUser("GRANT INSERT ON TABLE t1 TO target;", "foo3", "", 2), + execSQLAsUser("GRANT GRANT ON TYPE ty TO target;", "foo4", "", 2), + + // Node 2 is upgraded; because node 3 is on the previous version, any user can still grant or revoke + // a privilege that they hold (grant options are ignored). + binaryUpgradeStep(c.Node(2), mainVersion), + allowAutoUpgradeStep(2), + execSQLAsUser("GRANT ALL PRIVILEGES ON DATABASE d TO target;", "foo", "", 3), + execSQLAsUser("GRANT ALL PRIVILEGES ON SCHEMA s TO target;", "foo2", "", 2), + execSQLAsUser("GRANT ALL PRIVILEGES ON TABLE t1 TO target;", "foo3", "", 1), + execSQLAsUser("GRANT ALL PRIVILEGES ON TYPE ty TO target;", "foo4", "", 1), + execSQLAsUser("GRANT GRANT ON DATABASE d TO foo2;", "foo", "", 1), + execSQLAsUser("GRANT GRANT ON SCHEMA s TO foo3;", "foo2", "", 1), + execSQLAsUser("GRANT GRANT, SELECT ON TABLE t1 TO foo4;", "foo3", "", 2), + execSQLAsUser("GRANT DELETE ON TABLE t1 TO foo4;", "foo3", "", 2), + + // Node 3 is upgraded; because all the nodes have been upgraded, the migration will begin running after + // allowAutoUpgradeStep(3). The roachtest will ensure that the migration will be complete before moving + // beyond waitForUpgradeStep(c.All()). + binaryUpgradeStep(c.Node(3), mainVersion), + allowAutoUpgradeStep(3), + waitForUpgradeStep(c.All()), + + // All the nodes have been upgraded and the migration has finished; existing users will have had their grant + // option bits set equal to their privilege bits if they hold "GRANT" or "ALL" privileges. Grant options will + // now be enforced (an error will be returned if a user tries to grant a privilege they do not hold grant + // options for, which corresponds to a nonempty string in the errorExpected field). + execSQLAsUser("GRANT DELETE ON TABLE t1 TO foo2;", "foo3", "", 3), + execSQLAsUser("GRANT ALL PRIVILEGES ON SCHEMA s TO foo3;", "target", "", 2), + execSQLAsUser("GRANT CREATE ON DATABASE d TO foo3;", "foo2", "pq: user foo2 does not have CREATE privilege on database d", 2), + execSQLAsUser("GRANT USAGE ON SCHEMA s TO foo;", "foo3", "pq: missing WITH GRANT OPTION privilege type USAGE", 2), + execSQLAsUser("GRANT INSERT ON TABLE t1 TO foo;", "foo4", "pq: user foo4 does not have INSERT privilege on relation t1", 1), + execSQLAsUser("GRANT SELECT ON TABLE t1 TO foo;", "foo4", "", 1), + execSQLAsUser("GRANT DELETE ON TABLE t1 TO foo;", "foo4", "", 1), + + execSQL("CREATE USER foo5;", "", 2), + execSQL("CREATE USER foo6;", "", 2), + execSQL("CREATE TABLE t2();", "", 3), + execSQL("GRANT ALL PRIVILEGES ON TABLE t2 TO foo5;", "", 1), + execSQLAsUser("GRANT DELETE ON TABLE t2 TO foo2;", "foo5", "pq: missing WITH GRANT OPTION privilege type DELETE", 2), + execSQL("GRANT ALL PRIVILEGES ON TABLE t2 TO foo6 WITH GRANT OPTION;", "", 1), + execSQLAsUser("GRANT DELETE ON TABLE t2 TO foo2;", "foo6", "", 3), + ) + u.run(ctx, t) +} diff --git a/pkg/migration/migrations/BUILD.bazel b/pkg/migration/migrations/BUILD.bazel index 8d592a2e10fd..77562d799790 100644 --- a/pkg/migration/migrations/BUILD.bazel +++ b/pkg/migration/migrations/BUILD.bazel @@ -7,6 +7,7 @@ go_library( "alter_table_protected_timestamp_records.go", "alter_table_statistics_avg_size.go", "ensure_no_draining_names.go", + "grant_option_migration.go", "insert_missing_public_schema_namespace_entry.go", "migrate_span_configs.go", "migrations.go", @@ -30,6 +31,7 @@ go_library( "//pkg/sql/catalog", "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/dbdesc", + "//pkg/sql/catalog/descbuilder", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", "//pkg/sql/catalog/schemadesc", @@ -39,6 +41,8 @@ go_library( "//pkg/sql/privilege", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", + "//pkg/sql/sqlutil", + "//pkg/util/hlc", "//pkg/util/log", "//pkg/util/protoutil", "//pkg/util/retry", @@ -56,6 +60,7 @@ go_test( "alter_table_statistics_avg_size_test.go", "builtins_test.go", "ensure_no_draining_names_external_test.go", + "grant_option_migration_external_test.go", "helpers_test.go", "main_test.go", "migrate_span_configs_test.go", @@ -94,6 +99,8 @@ go_test( "//pkg/testutils/testcluster", "//pkg/util/leaktest", "//pkg/util/log", + "//pkg/util/protoutil", + "@com_github_cockroachdb_errors//:errors", "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", ], diff --git a/pkg/migration/migrations/grant_option_migration.go b/pkg/migration/migrations/grant_option_migration.go new file mode 100644 index 000000000000..c4b146119ac1 --- /dev/null +++ b/pkg/migration/migrations/grant_option_migration.go @@ -0,0 +1,159 @@ +// 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/kv" + "github.com/cockroachdb/cockroach/pkg/migration" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descbuilder" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" +) + +// grantOptionMigration iterates through every descriptor and sets a user's grant option bits +// equal to its privilege bits if it holds the "GRANT" privilege. +func grantOptionMigration( + ctx context.Context, _ clusterversion.ClusterVersion, d migration.TenantDeps, _ *jobs.Job, +) error { + query := `SELECT id, descriptor, crdb_internal_mvcc_timestamp FROM system.descriptor ORDER BY ID ASC` + rows, err := d.InternalExecutor.QueryIterator( + ctx, "retrieve-grant-options", nil /* txn */, query, + ) + if err != nil { + return err + } + + addGrantOptionFunc := func(ids []descpb.ID, descs []descpb.Descriptor, timestamps []hlc.Timestamp) error { + var modifiedDescs []catalog.MutableDescriptor + for i, id := range ids { + b := descbuilder.NewBuilderWithMVCCTimestamp(&descs[i], timestamps[i]) + if b == nil { + return errors.Newf("unable to find descriptor for id %d", id) + } + + b.RunPostDeserializationChanges() + mutableDesc := b.BuildExistingMutable() + + modifiedDescs = append(modifiedDescs, mutableDesc) + } + if err := writeModifiedDescriptors(ctx, d, modifiedDescs); err != nil { + return err + } + return nil + } + + return addGrantOptionMigration(ctx, rows, addGrantOptionFunc, 1<<19 /* 512 KiB batch size */) +} + +// addGrantOptionFunction is used in addGrantOptionMigration to maybe add grant options +// of descriptors specified by the id. +type addGrantOptionFunction func(ids []descpb.ID, descs []descpb.Descriptor, timestamps []hlc.Timestamp) error + +// addGrantOptionMigration is an abstraction for adding grant options. +// The rows provided should be the result of a select ID, descriptor, crdb_internal_mvcc_timestamp +// from system.descriptor table. +// The datums returned from the query are parsed to grab the descpb.Descriptor +// and addGrantOptionFunction is called on the desc. +// If minBatchSizeInBytes is specified, fixDescriptors will only be called once the +// size of the descriptors in the id array surpasses minBatchSizeInBytes. +func addGrantOptionMigration( + ctx context.Context, + rows sqlutil.InternalRows, + grantOptionFunc addGrantOptionFunction, + minBatchSizeInBytes int, +) error { + defer func() { _ = rows.Close() }() + ok, err := rows.Next(ctx) + if err != nil { + return err + } + currSize := 0 // in bytes. + var ids []descpb.ID + var descs []descpb.Descriptor + var timestamps []hlc.Timestamp + for ; ok; ok, err = rows.Next(ctx) { + if err != nil { + return err + } + datums := rows.Cur() + id, desc, ts, err := unmarshalDescFromDescriptorRow(datums) + if err != nil { + return err + } + ids = append(ids, id) + descs = append(descs, desc) + timestamps = append(timestamps, ts) + currSize += desc.Size() + if currSize > minBatchSizeInBytes || minBatchSizeInBytes == 0 { + err = grantOptionFunc(ids, descs, timestamps) + if err != nil { + return err + } + // Reset size and id array after the batch is fixed. + currSize = 0 + ids = nil + descs = nil + timestamps = nil + } + } + // Fix remaining descriptors. + return grantOptionFunc(ids, descs, timestamps) +} + +// unmarshalDescFromDescriptorRow takes in an InternalRow from a query that gets: +// ID, descriptor, crdb_internal_mvcc_timestamp from the system.descriptor table. +// ie: SELECT id, descriptor, crdb_internal_mvcc_timestamp FROM system.descriptor ORDER BY ID ASC +// and parses the id, descriptor and mvcc_timestamp fields. +func unmarshalDescFromDescriptorRow( + datums tree.Datums, +) (descpb.ID, descpb.Descriptor, hlc.Timestamp, error) { + id := descpb.ID(*datums[0].(*tree.DInt)) + ts, err := tree.DecimalToHLC(&datums[2].(*tree.DDecimal).Decimal) + if err != nil { + return id, descpb.Descriptor{}, ts, errors.Wrapf(err, + "failed to convert MVCC timestamp decimal to HLC for id %d", id) + } + var desc descpb.Descriptor + if err := protoutil.Unmarshal(([]byte)(*datums[1].(*tree.DBytes)), &desc); err != nil { + return id, descpb.Descriptor{}, ts, errors.Wrapf(err, + "failed to unmarshal descriptor with ID %d", id) + } + return id, desc, ts, nil +} + +// writeModifiedDescriptors writes the descriptors that we have given grant option privileges +// to back to batch +func writeModifiedDescriptors( + ctx context.Context, d migration.TenantDeps, modifiedDescs []catalog.MutableDescriptor, +) error { + return d.CollectionFactory.Txn(ctx, d.InternalExecutor, d.DB, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + ) error { + batch := txn.NewBatch() + for _, desc := range modifiedDescs { + err := descriptors.WriteDescToBatch(ctx, false, desc, batch) + if err != nil { + return err + } + } + return txn.Run(ctx, batch) + }) +} diff --git a/pkg/migration/migrations/grant_option_migration_external_test.go b/pkg/migration/migrations/grant_option_migration_external_test.go new file mode 100644 index 000000000000..185e40d16765 --- /dev/null +++ b/pkg/migration/migrations/grant_option_migration_external_test.go @@ -0,0 +1,214 @@ +// 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" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/sql" + "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/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/cockroachdb/errors" + "github.com/stretchr/testify/require" +) + +func TestGrantOptionMigration(t *testing.T) { + defer leaktest.AfterTest(t)() + + // Test when a user holds privileges on a database, schema, and table + /* + CREATE USER testuser; + CREATE DATABASE db; + GRANT CREATE ON DATABASE db TO testuser; + USE db; + CREATE SCHEMA s; + CREATE TABLE t1(); + GRANT CREATE ON SCHEMA s TO testuser; + GRANT INSERT, SELECT ON TABLE t1 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 objectTest = ` +124e0a02646210341a310a0b0a0561646d696e100218020a0a0a04726f6f74100218020a0e0a087465737475736572100418001204726f6f741802220028033a090a017312040835100040004a005a00 +22420834120173183522310a0b0a0561646d696e100218020a0a0a04726f6f74100218020a0e0a087465737475736572100418001204726f6f7418022a00300240004a00 +0aab020a0274311836203428023a00423a0a05726f77696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293001680070007800800100880100980100480252500a0774315f706b6579100118012205726f776964300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba010060026a310a0b0a0561646d696e100218020a0a0a04726f6f74100218020a0e0a087465737475736572106418001204726f6f741802800101880103980100b201160a077072696d61727910001a05726f77696420012800b80101c20100e80100f2010408001200f801008002009202009a020a08d8bfcfc594a985dc16b20200b80200c0021dc80200e00200f00200 +` + + // Check that descriptors for multiple users are pulled and updated when + // creating schemas and tables within those schemas + /* + CREATE USER testuser; + CREATE USER testuser2; + CREATE DATABASE db; + USE db; + GRANT ALL PRIVILEGES ON DATABASE db TO testuser; + CREATE schema s; + CREATE table s.t1(); + CREATE table s.t2(); + GRANT GRANT, CREATE ON SCHEMA s TO testuser2; + GRANT SELECT, DELETE ON TABLE s.t1 TO testuser2; + GRANT ALL PRIVILEGES ON TABLE s.t2 TO testuser2; + + 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 multipleUsersTest = ` +124e0a02646210341a310a0b0a0561646d696e100218020a0a0a04726f6f74100218020a0e0a087465737475736572100218021204726f6f741802220028033a090a017312040835100040004a005a00 +22530834120173183522420a0b0a0561646d696e100218020a0a0a04726f6f74100218020a0e0a087465737475736572100218020a0f0a09746573747573657232101418001204726f6f7418022a00300240004a00 +0abd020a0274311836203428023a00423a0a05726f77696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293001680070007800800100880100980100480252500a0774315f706b6579100118012205726f776964300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba010060026a430a0b0a0561646d696e100218020a0a0a04726f6f74100218020a0e0a087465737475736572100218020a100a0974657374757365723210a00118001204726f6f741802800101880103980100b201160a077072696d61727910001a05726f77696420012800b80101c20100e80100f2010408001200f801008002009202009a020a08b8b091afa2d484dc16b20200b80200c00235c80200e00200f00200 +0abc020a0274321837203428023a00423a0a05726f77696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293001680070007800800100880100980100480252500a0774325f706b6579100118012205726f776964300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba010060026a420a0b0a0561646d696e100218020a0a0a04726f6f74100218020a0e0a087465737475736572100218020a0f0a09746573747573657232100218001204726f6f741802800101880103980100b201160a077072696d61727910001a05726f77696420012800b80101c20100e80100f2010408001200f801008002009202009a020a08d8efb894b8d484dc16b20200b80200c00235c80200e00200f00200 +` + + // Test the migration for types + /* + CREATE USER testuser; + CREATE DATABASE db; + USE db; + CREATE schema s; + CREATE TYPE ty AS ENUM(); + CREATE TYPE s.ty2 AS ENUM(); + GRANT USAGE ON TYPE ty TO testuser; + GRANT ALL PRIVILEGES ON TYPE s.ty2 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 typesTest = ` +123e0a02646210341a210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f741802220028023a090a017312040835100040004a005a00 +22320834120173183522210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f7418022a00300140004a00 +1a590834101d1a02747920362800403748025200680072410a0b0a0561646d696e100218020a0d0a067075626c696310800418000a0a0a04726f6f74100218020a0f0a08746573747573657210800418001204726f6f7418027a00 +1a710834101d1a035f7479203728013a26080f100018003000381850d78d065a14081810001800300050d68d0660007a0410d78d066000400048015200680072300a0b0a0561646d696e100218020a0d0a067075626c696310800418000a0a0a04726f6f74100218021204726f6f7418027a00 +1a59083410351a0374793220382800403948025200680072400a0b0a0561646d696e100218020a0d0a067075626c696310800418000a0a0a04726f6f74100218020a0e0a087465737475736572100218001204726f6f7418027a00 +1a72083410351a045f747932203928013a26080f100018003000381850d98d065a14081810001800300050d88d0660007a0410d98d066000400048015200680072300a0b0a0561646d696e100218020a0d0a067075626c696310800418000a0a0a04726f6f74100218021204726f6f7418027a00 +` + + testCases := []string{ + objectTest, + multipleUsersTest, + typesTest, + } + 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.ValidateGrantOption - 1), // changed cluster version + }, + }, + }, + }) + + db := tc.ServerConn(0) + require.NoError(t, sqlutils.InjectDescriptors( + ctx, db, descriptorsToInject, true, /* force */ + )) + + _, err := tc.Conns[0].ExecContext(ctx, `SET CLUSTER SETTING version = $1`, + clusterversion.ByKey(clusterversion.ValidateGrantOption).String()) + require.NoError(t, err) + + sql.TestingDescsTxn(ctx, tc.Server(0), func(ctx context.Context, txn *kv.Txn, col *descs.Collection) error { + // Avoid running validation on the descriptors. + descs, err := col.GetCatalogUnvalidated(ctx, txn) + if err != nil { + return err + } + for _, desc := range descs.OrderedDescriptors() { + privilegeDesc := desc.GetPrivileges() + for _, u := range privilegeDesc.Users { + if privilege.GRANT.IsSetIn(u.Privileges) || privilege.ALL.IsSetIn(u.Privileges) { + if u.UserProto.Decode().IsAdminRole() || u.UserProto.Decode().IsRootUser() || u.UserProto.Decode().IsNodeUser() { + continue + } + + if u.Privileges != u.WithGrantOption { + return errors.Newf("grant options not updated properly for %d, Privileges: %d, Grant Option: %d", u.User(), u.Privileges, u.WithGrantOption) + } + } + } + + } + return nil + }) + + require.NoError(t, err) + tc.Stopper().Stop(ctx) + } +} diff --git a/pkg/migration/migrations/migrations.go b/pkg/migration/migrations/migrations.go index 34657761ef7a..832180f7dd9e 100644 --- a/pkg/migration/migrations/migrations.go +++ b/pkg/migration/migrations/migrations.go @@ -91,6 +91,12 @@ var migrations = []migration.Migration{ toCV(clusterversion.EnsureSpanConfigSubscription), ensureSpanConfigSubscription, ), + migration.NewTenantMigration( + "track grant options on users and enable granting/revoking with them", + toCV(clusterversion.ValidateGrantOption), + NoPrecondition, + grantOptionMigration, + ), } func init() { diff --git a/pkg/sql/catalog/catprivilege/default_privilege.go b/pkg/sql/catalog/catprivilege/default_privilege.go index 140562d00b6c..3066317b0c64 100644 --- a/pkg/sql/catalog/catprivilege/default_privilege.go +++ b/pkg/sql/catalog/catprivilege/default_privilege.go @@ -234,7 +234,7 @@ func CreatePrivilegesFromDefaultPrivileges( } } - newPrivs.Version = descpb.Version21_2 + newPrivs.Version = descpb.GrantOptionVersion // TODO(richardjcai): Remove this depending on how we handle the migration. // For backwards compatibility, also "inherit" privileges from the dbDesc. diff --git a/pkg/sql/catalog/catprivilege/fix.go b/pkg/sql/catalog/catprivilege/fix.go index 221db6852245..cb88a0321ee7 100644 --- a/pkg/sql/catalog/catprivilege/fix.go +++ b/pkg/sql/catalog/catprivilege/fix.go @@ -138,3 +138,36 @@ func MaybeFixPrivileges( } return changed } + +// MaybeUpdateGrantOptions iterates over the users of the descriptor and checks +// if they have the GRANT privilege - if so, then set the user's grant option +// bits equal to the privilege bits. +func MaybeUpdateGrantOptions(p *descpb.PrivilegeDescriptor) bool { + changed := false + if p.Version >= descpb.GrantOptionVersion { + // do not apply changes because the grant bits will have already been set. + return changed + } + + for i := range p.Users { + u := &p.Users[i] + if privilege.ALL.IsSetIn(u.Privileges) { + if !privilege.ALL.IsSetIn(u.WithGrantOption) { + changed = true + } + u.WithGrantOption = privilege.ALL.Mask() + continue + } + if privilege.GRANT.IsSetIn(u.Privileges) { + if u.Privileges != u.WithGrantOption { + changed = true + } + u.WithGrantOption |= u.Privileges + } + } + + p.SetVersion(descpb.GrantOptionVersion) + changed = true + + return changed +} diff --git a/pkg/sql/catalog/dbdesc/database_desc_builder.go b/pkg/sql/catalog/dbdesc/database_desc_builder.go index 418cfa4a31ee..3b5f714b5db2 100644 --- a/pkg/sql/catalog/dbdesc/database_desc_builder.go +++ b/pkg/sql/catalog/dbdesc/database_desc_builder.go @@ -65,10 +65,11 @@ func (ddb *databaseDescriptorBuilder) RunPostDeserializationChanges() { privilege.Database, ddb.maybeModified.GetName()) removedSelfEntryInSchemas := maybeRemoveDroppedSelfEntryFromSchemas(ddb.maybeModified) - ddb.changed = privsChanged || removedSelfEntryInSchemas + addedGrantOptions := catprivilege.MaybeUpdateGrantOptions(ddb.maybeModified.Privileges) + ddb.changed = privsChanged || removedSelfEntryInSchemas || addedGrantOptions } -// RunPostRestoreChanges implements the catalog.DescriptorBuilder interface. +// RunRestoreChanges implements the catalog.DescriptorBuilder interface. func (ddb *databaseDescriptorBuilder) RunRestoreChanges( _ func(id descpb.ID) catalog.Descriptor, ) error { diff --git a/pkg/sql/catalog/descpb/privilege.go b/pkg/sql/catalog/descpb/privilege.go index c2e4779b70d3..ad0047818b76 100644 --- a/pkg/sql/catalog/descpb/privilege.go +++ b/pkg/sql/catalog/descpb/privilege.go @@ -39,6 +39,11 @@ const ( // These descriptors should have all the correct privileges and the owner field // explicitly set. These descriptors should be strictly validated. Version21_2 + + // GrantOptionVersion corresponds to descriptors created in 22.1 and onwards. + // These descriptors should have grant options for all their privileges + // if they have the GRANT or ALL were created prior to 22.1. + GrantOptionVersion ) // Owner accesses the owner field. @@ -111,15 +116,17 @@ func NewCustomSuperuserPrivilegeDescriptor( OwnerProto: owner.EncodeProto(), Users: []UserPrivileges{ { - UserProto: security.AdminRoleName().EncodeProto(), - Privileges: priv.ToBitField(), + UserProto: security.AdminRoleName().EncodeProto(), + Privileges: priv.ToBitField(), + WithGrantOption: priv.ToBitField(), }, { - UserProto: security.RootUserName().EncodeProto(), - Privileges: priv.ToBitField(), + UserProto: security.RootUserName().EncodeProto(), + Privileges: priv.ToBitField(), + WithGrantOption: priv.ToBitField(), }, }, - Version: Version21_2, + Version: GrantOptionVersion, } } @@ -149,7 +156,7 @@ func NewPrivilegeDescriptor( WithGrantOption: grantOption.ToBitField(), }, }, - Version: Version21_2, + Version: GrantOptionVersion, } } diff --git a/pkg/sql/catalog/descpb/privilegedescversion_string.go b/pkg/sql/catalog/descpb/privilegedescversion_string.go index 295607e8c4c3..f4f0858bf3f1 100644 --- a/pkg/sql/catalog/descpb/privilegedescversion_string.go +++ b/pkg/sql/catalog/descpb/privilegedescversion_string.go @@ -11,11 +11,12 @@ func _() { _ = x[InitialVersion-0] _ = x[OwnerVersion-1] _ = x[Version21_2-2] + _ = x[GrantOptionVersion-3] } -const _PrivilegeDescVersion_name = "InitialVersionOwnerVersionVersion21_2" +const _PrivilegeDescVersion_name = "InitialVersionOwnerVersionVersion21_2GrantOptionVersion" -var _PrivilegeDescVersion_index = [...]uint8{0, 14, 26, 37} +var _PrivilegeDescVersion_index = [...]uint8{0, 14, 26, 37, 55} func (i PrivilegeDescVersion) String() string { if i >= PrivilegeDescVersion(len(_PrivilegeDescVersion_index)-1) { diff --git a/pkg/sql/catalog/schemadesc/schema_desc_builder.go b/pkg/sql/catalog/schemadesc/schema_desc_builder.go index 85dedc5c80dc..2e44185d2184 100644 --- a/pkg/sql/catalog/schemadesc/schema_desc_builder.go +++ b/pkg/sql/catalog/schemadesc/schema_desc_builder.go @@ -52,16 +52,18 @@ func (sdb *schemaDescriptorBuilder) DescriptorType() catalog.DescriptorType { // interface. func (sdb *schemaDescriptorBuilder) RunPostDeserializationChanges() { sdb.maybeModified = protoutil.Clone(sdb.original).(*descpb.SchemaDescriptor) - sdb.changed = catprivilege.MaybeFixPrivileges( + privsChanged := catprivilege.MaybeFixPrivileges( &sdb.maybeModified.Privileges, sdb.maybeModified.GetParentID(), descpb.InvalidID, privilege.Schema, sdb.maybeModified.GetName(), ) + addedGrantOptions := catprivilege.MaybeUpdateGrantOptions(sdb.maybeModified.Privileges) + sdb.changed = privsChanged || addedGrantOptions } -// RunPostRestoreChanges implements the catalog.DescriptorBuilder interface. +// RunRestoreChanges implements the catalog.DescriptorBuilder interface. func (sdb *schemaDescriptorBuilder) RunRestoreChanges( _ func(id descpb.ID) catalog.Descriptor, ) error { diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go index 045022419e13..69b405340a48 100644 --- a/pkg/sql/catalog/systemschema/system.go +++ b/pkg/sql/catalog/systemschema/system.go @@ -1421,7 +1421,7 @@ var ( }, ), func(tbl *descpb.TableDescriptor) { - tbl.Privileges.Version = descpb.Version21_2 + tbl.Privileges.Version = descpb.GrantOptionVersion tbl.Privileges.Users = append(tbl.Privileges.Users, descpb.UserPrivileges{ UserProto: security.PublicRoleName().EncodeProto(), Privileges: privilege.List{privilege.SELECT}.ToBitField(), diff --git a/pkg/sql/catalog/tabledesc/table_desc_builder.go b/pkg/sql/catalog/tabledesc/table_desc_builder.go index c8dc7a0671ad..b439dda17d35 100644 --- a/pkg/sql/catalog/tabledesc/table_desc_builder.go +++ b/pkg/sql/catalog/tabledesc/table_desc_builder.go @@ -101,7 +101,7 @@ func (tdb *tableDescriptorBuilder) RunPostDeserializationChanges() { tdb.changes = maybeFillInDescriptor(tdb.maybeModified) } -// RunPostRestoreChanges implements the catalog.DescriptorBuilder interface. +// RunRestoreChanges implements the catalog.DescriptorBuilder interface. func (tdb *tableDescriptorBuilder) RunRestoreChanges( descLookupFn func(id descpb.ID) catalog.Descriptor, ) (err error) { @@ -214,13 +214,15 @@ func maybeFillInDescriptor( if parentSchemaID == descpb.InvalidID { parentSchemaID = keys.PublicSchemaID } - changes.UpgradedPrivileges = catprivilege.MaybeFixPrivileges( + fixedPrivileges := catprivilege.MaybeFixPrivileges( &desc.Privileges, desc.GetParentID(), parentSchemaID, privilege.Table, desc.GetName(), ) + addedGrantOptions := catprivilege.MaybeUpdateGrantOptions(desc.Privileges) + changes.UpgradedPrivileges = fixedPrivileges || addedGrantOptions return changes } diff --git a/pkg/sql/catalog/typedesc/type_desc_builder.go b/pkg/sql/catalog/typedesc/type_desc_builder.go index f76a06b5febf..769af5244d8d 100644 --- a/pkg/sql/catalog/typedesc/type_desc_builder.go +++ b/pkg/sql/catalog/typedesc/type_desc_builder.go @@ -53,16 +53,18 @@ func (tdb *typeDescriptorBuilder) DescriptorType() catalog.DescriptorType { // interface. func (tdb *typeDescriptorBuilder) RunPostDeserializationChanges() { tdb.maybeModified = protoutil.Clone(tdb.original).(*descpb.TypeDescriptor) - tdb.changed = catprivilege.MaybeFixPrivileges( + fixedPrivileges := catprivilege.MaybeFixPrivileges( &tdb.maybeModified.Privileges, tdb.maybeModified.GetParentID(), tdb.maybeModified.GetParentSchemaID(), privilege.Type, tdb.maybeModified.GetName(), ) + addedGrantOptions := catprivilege.MaybeUpdateGrantOptions(tdb.maybeModified.Privileges) + tdb.changed = fixedPrivileges || addedGrantOptions } -// RunPostRestoreChanges implements the catalog.DescriptorBuilder interface. +// RunRestoreChanges implements the catalog.DescriptorBuilder interface. func (tdb *typeDescriptorBuilder) RunRestoreChanges(_ func(id descpb.ID) catalog.Descriptor) error { return nil } diff --git a/pkg/sql/logictest/testdata/logic_test/bytes b/pkg/sql/logictest/testdata/logic_test/bytes index 29c37623fdc6..e913f9e04d65 100644 --- a/pkg/sql/logictest/testdata/logic_test/bytes +++ b/pkg/sql/logictest/testdata/logic_test/bytes @@ -153,7 +153,7 @@ PREPARE r1(bytes) AS SELECT descriptor::STRING FROM system.descriptor WHERE desc query T EXECUTE r1('abc') ---- -\022[\012\011defaultdb\0202\0320\012\013\012\005admin\020\002\030\000\012\015\012\006public\020\200\020\030\000\012\012\012\004root\020\002\030\000\022\004root\030\002"\000(\001:\016\012\006public\022\004\0103\020\000@\000J\000Z\002\020\000 +\022[\012\011defaultdb\0202\0320\012\013\012\005admin\020\002\030\002\012\015\012\006public\020\200\020\030\000\012\012\012\004root\020\002\030\002\022\004root\030\003"\000(\001:\016\012\006public\022\004\0103\020\000@\000J\000Z\002\020\000 statement ok create table regression_71444 (col bytes[]); diff --git a/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic b/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic index da0d5688be4d..bd0bffa140c3 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic +++ b/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic @@ -18,9 +18,9 @@ WHERE message NOT LIKE '%Z/%' AND operation != 'dist sender send' ---- batch flow coordinator CPut /NamespaceTable/30/1/56/0/"public"/4/1 -> 57 -batch flow coordinator CPut /Table/3/1/57/2/1 -> schema: version:1 parent_id:56 privileges: users: users: owner_proto:"admin" version:2 > > +batch flow coordinator CPut /Table/3/1/57/2/1 -> schema: version:1 parent_id:56 privileges: users: users: owner_proto:"admin" version:3 > > batch flow coordinator CPut /NamespaceTable/30/1/0/0/"t"/4/1 -> 56 -batch flow coordinator CPut /Table/3/1/56/2/1 -> database: version:1 privileges: users: users: owner_proto:"root" version:2 > schemas: > state:PUBLIC offline_reason:"" default_privileges: > +batch flow coordinator CPut /Table/3/1/56/2/1 -> database: version:1 privileges: users: users: owner_proto:"root" version:3 > schemas: > state:PUBLIC offline_reason:"" default_privileges: > exec stmt rows affected: 0 @@ -40,7 +40,7 @@ WHERE message NOT LIKE '%Z/%' AND operation != 'dist sender send' ---- batch flow coordinator CPut /NamespaceTable/30/1/56/57/"kv"/4/1 -> 58 -batch flow coordinator CPut /Table/3/1/58/2/1 -> table: parent_id:56 unexposed_parent_schema_id:57 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false > next_index_id:2 privileges: users: users: owner_proto:"root" version:2 > next_mutation_id:1 format_version:3 state:PUBLIC offline_reason:"" view_query:"" is_materialized_view:false new_schema_change_job_id:0 drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<> temporary:false partition_all_by:false > +batch flow coordinator CPut /Table/3/1/58/2/1 -> table: parent_id:56 unexposed_parent_schema_id:57 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false > next_index_id:2 privileges: users: users: owner_proto:"root" version:3 > next_mutation_id:1 format_version:3 state:PUBLIC offline_reason:"" view_query:"" is_materialized_view:false new_schema_change_job_id:0 drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<> temporary:false partition_all_by:false > exec stmt rows affected: 0 # We avoid using the full trace output, because that would make the @@ -66,7 +66,7 @@ WHERE message NOT LIKE '%Z/%' AND message NOT LIKE 'querying next range at%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -batch flow coordinator Put /Table/3/1/58/2/1 -> table: parent_id:56 unexposed_parent_schema_id:57 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false > next_index_id:3 privileges: users: users: owner_proto:"root" version:2 > mutations: interleave:<> partitioning: type:FORWARD created_explicitly:true encoding_type:0 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false > state:DELETE_ONLY direction:ADD mutation_id:1 rollback:false > next_mutation_id:2 format_version:3 state:PUBLIC offline_reason:"" view_query:"" is_materialized_view:false mutationJobs:<...> new_schema_change_job_id:0 drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false > +batch flow coordinator Put /Table/3/1/58/2/1 -> table: parent_id:56 unexposed_parent_schema_id:57 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false > next_index_id:3 privileges: users: users: owner_proto:"root" version:3 > mutations: interleave:<> partitioning: type:FORWARD created_explicitly:true encoding_type:0 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false > state:DELETE_ONLY direction:ADD mutation_id:1 rollback:false > next_mutation_id:2 format_version:3 state:PUBLIC offline_reason:"" view_query:"" is_materialized_view:false mutationJobs:<...> new_schema_change_job_id:0 drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false > exec stmt rows affected: 0 statement ok @@ -121,7 +121,7 @@ WHERE message NOT LIKE '%Z/%' AND operation != 'dist sender send' ---- batch flow coordinator CPut /NamespaceTable/30/1/56/57/"kv2"/4/1 -> 59 -batch flow coordinator CPut /Table/3/1/59/2/1 -> table: parent_id:56 unexposed_parent_schema_id:57 columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:false default_expr:"unique_rowid()" hidden:true inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false > next_index_id:2 privileges: users: users: owner_proto:"root" version:2 > next_mutation_id:1 format_version:3 state:ADD offline_reason:"" view_query:"" is_materialized_view:false new_schema_change_job_id:0 drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"TABLE t.public.kv" create_as_of_time:<> temporary:false partition_all_by:false > +batch flow coordinator CPut /Table/3/1/59/2/1 -> table: parent_id:56 unexposed_parent_schema_id:57 columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:false default_expr:"unique_rowid()" hidden:true inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false > next_index_id:2 privileges: users: users: owner_proto:"root" version:3 > next_mutation_id:1 format_version:3 state:ADD offline_reason:"" view_query:"" is_materialized_view:false new_schema_change_job_id:0 drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"TABLE t.public.kv" create_as_of_time:<> temporary:false partition_all_by:false > exec stmt rows affected: 0 statement ok @@ -170,7 +170,7 @@ WHERE message NOT LIKE '%Z/%' AND message NOT LIKE 'querying next range at%' AND operation != 'dist sender send' ---- batch flow coordinator Del /NamespaceTable/30/1/56/57/"kv2"/4/1 -batch flow coordinator Put /Table/3/1/59/2/1 -> table: parent_id:56 unexposed_parent_schema_id:57 columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:false default_expr:"unique_rowid()" hidden:true inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false > next_index_id:2 privileges: users: users: owner_proto:"root" version:2 > next_mutation_id:1 format_version:3 state:DROP offline_reason:"" view_query:"" is_materialized_view:false new_schema_change_job_id:0 drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"TABLE t.public.kv" create_as_of_time:<...> temporary:false partition_all_by:false > +batch flow coordinator Put /Table/3/1/59/2/1 -> table: parent_id:56 unexposed_parent_schema_id:57 columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:false default_expr:"unique_rowid()" hidden:true inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false > next_index_id:2 privileges: users: users: owner_proto:"root" version:3 > next_mutation_id:1 format_version:3 state:DROP offline_reason:"" view_query:"" is_materialized_view:false new_schema_change_job_id:0 drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"TABLE t.public.kv" create_as_of_time:<...> temporary:false partition_all_by:false > exec stmt rows affected: 0 statement ok @@ -204,7 +204,7 @@ WHERE message NOT LIKE '%Z/%' AND message NOT LIKE 'querying next range at%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -batch flow coordinator Put /Table/3/1/58/2/1 -> table: parent_id:56 unexposed_parent_schema_id:57 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false > next_index_id:3 privileges: users: users: owner_proto:"root" version:2 > mutations: interleave:<> partitioning: type:FORWARD created_explicitly:true encoding_type:0 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false > state:DELETE_AND_WRITE_ONLY direction:DROP mutation_id:2 rollback:false > next_mutation_id:3 format_version:3 state:PUBLIC offline_reason:"" view_query:"" is_materialized_view:false mutationJobs:<...> new_schema_change_job_id:0 drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false > +batch flow coordinator Put /Table/3/1/58/2/1 -> table: parent_id:56 unexposed_parent_schema_id:57 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false > next_index_id:3 privileges: users: users: owner_proto:"root" version:3 > mutations: interleave:<> partitioning: type:FORWARD created_explicitly:true encoding_type:0 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false > state:DELETE_AND_WRITE_ONLY direction:DROP mutation_id:2 rollback:false > next_mutation_id:3 format_version:3 state:PUBLIC offline_reason:"" view_query:"" is_materialized_view:false mutationJobs:<...> new_schema_change_job_id:0 drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false > exec stmt rows affected: 0 statement ok @@ -222,7 +222,7 @@ WHERE message NOT LIKE '%Z/%' AND message NOT LIKE 'querying next range at%' AND operation != 'dist sender send' ---- batch flow coordinator Del /NamespaceTable/30/1/56/57/"kv"/4/1 -batch flow coordinator Put /Table/3/1/58/2/1 -> table: parent_id:56 unexposed_parent_schema_id:57 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false > next_index_id:3 privileges: users: users: owner_proto:"root" version:2 > next_mutation_id:3 format_version:3 state:DROP offline_reason:"" view_query:"" is_materialized_view:false new_schema_change_job_id:0 drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false > +batch flow coordinator Put /Table/3/1/58/2/1 -> table: parent_id:56 unexposed_parent_schema_id:57 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false > next_index_id:3 privileges: users: users: owner_proto:"root" version:3 > next_mutation_id:3 format_version:3 state:DROP offline_reason:"" view_query:"" is_materialized_view:false new_schema_change_job_id:0 drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false > exec stmt rows affected: 0 # Check that session tracing does not inhibit the fast path for inserts & diff --git a/pkg/sql/schemachanger/testdata/drop b/pkg/sql/schemachanger/testdata/drop index ede150c2e3ff..772d199089e9 100644 --- a/pkg/sql/schemachanger/testdata/drop +++ b/pkg/sql/schemachanger/testdata/drop @@ -29,8 +29,8 @@ upsert descriptor #54 + version: "3" upsert descriptor #56 ... - userProto: root - version: 2 + withGrantOption: 2 + version: 3 - version: "1" + state: DROP + version: "2" @@ -138,22 +138,22 @@ upsert descriptor #54 + version: "5" upsert descriptor #57 ... - userProto: root - version: 2 + withGrantOption: 2 + version: 3 - version: "1" + state: DROP + version: "2" upsert descriptor #59 ... - userProto: root - version: 2 + withGrantOption: 2 + version: 3 - version: "1" + state: DROP + version: "2" upsert descriptor #60 ... - userProto: root - version: 2 + withGrantOption: 2 + version: 3 - version: "1" + state: DROP + version: "2" @@ -187,8 +187,8 @@ create job #1: "schema change job" descriptor IDs: [54 55] upsert descriptor #54 ... - userProto: root - version: 2 + withGrantOption: 2 + version: 3 - schemas: - public: - id: 55 @@ -198,8 +198,8 @@ upsert descriptor #54 + version: "6" upsert descriptor #55 ... - userProto: root - version: 2 + withGrantOption: 2 + version: 3 - version: "1" + state: DROP + version: "2" @@ -217,7 +217,7 @@ delete database namespace entry {0 0 db} -> 54 delete schema namespace entry {54 0 public} -> 55 upsert descriptor #54 ... - version: 2 + version: 3 state: DROP - version: "6" + version: "7" @@ -301,8 +301,8 @@ delete object namespace entry {61 63 _typ} -> 73 delete object namespace entry {61 63 v5} -> 74 upsert descriptor #61 ... - userProto: root - version: 2 + withGrantOption: 2 + version: 3 - schemas: - public: - id: 62 @@ -314,15 +314,15 @@ upsert descriptor #61 + version: "3" upsert descriptor #62 ... - userProto: root - version: 2 + withGrantOption: 2 + version: 3 - version: "1" + state: DROP + version: "2" upsert descriptor #63 ... - userProto: root - version: 2 + withGrantOption: 2 + version: 3 - version: "1" + state: DROP + version: "2" @@ -550,8 +550,8 @@ upsert descriptor #71 viewQuery: (SELECT n2, n1 FROM db1.sc1.v2) upsert descriptor #72 ... - userProto: root - version: 2 + withGrantOption: 2 + version: 3 - referencingDescriptorIds: - - 74 - version: "2" @@ -560,8 +560,8 @@ upsert descriptor #72 + version: "3" upsert descriptor #73 ... - userProto: root - version: 2 + withGrantOption: 2 + version: 3 - referencingDescriptorIds: - - 74 - version: "2" @@ -605,7 +605,7 @@ delete schema namespace entry {61 0 public} -> 62 delete schema namespace entry {61 0 sc1} -> 63 upsert descriptor #61 ... - version: 2 + version: 3 state: DROP - version: "3" + version: "4"