From e0c82d9a6f00c96dab1f132ee07049351e52a168 Mon Sep 17 00:00:00 2001 From: Jack Wu Date: Tue, 9 Nov 2021 16:24:48 -0500 Subject: [PATCH 1/2] 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" From 5b77b33001e85540d7a2f7c544f8ce2336dfed04 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Fri, 21 Jan 2022 12:44:58 -0500 Subject: [PATCH 2/2] roachtest,sql: enhance grant option migration test and fix bugs This changes the migration so that it runs on all the privilege descriptors that don't have grant options set. Additionally, the GRANT/REVOKE logic is changed so that it also applies the grant option if the grantee currently has the GRANT privilege. Release note: None --- .../roachtest/tests/validate_grant_option.go | 60 ++++++-- .../grant_option_migration_external_test.go | 133 +++++++++--------- pkg/sql/alter_default_privileges.go | 31 ++-- .../catalog/catprivilege/default_privilege.go | 2 +- pkg/sql/catalog/catprivilege/fix.go | 14 +- pkg/sql/catalog/descpb/privilege.go | 9 +- .../descpb/privilegedescversion_string.go | 5 +- pkg/sql/catalog/systemschema/system.go | 2 +- pkg/sql/descriptor.go | 3 +- pkg/sql/grant_revoke.go | 55 +++++--- pkg/sql/logictest/testdata/logic_test/bytes | 2 +- .../logic_test/grant_option_mixed_version | 16 +++ .../logic_test/grant_revoke_with_grant_option | 13 +- .../testdata/show_trace_nonmetamorphic | 16 +-- pkg/sql/rename_table.go | 3 +- pkg/sql/schemachanger/testdata/drop | 26 ++-- pkg/sql/serial.go | 3 +- 17 files changed, 230 insertions(+), 163 deletions(-) create mode 100644 pkg/sql/logictest/testdata/logic_test/grant_option_mixed_version diff --git a/pkg/cmd/roachtest/tests/validate_grant_option.go b/pkg/cmd/roachtest/tests/validate_grant_option.go index b99975790b9b..8d1ba174ec26 100644 --- a/pkg/cmd/roachtest/tests/validate_grant_option.go +++ b/pkg/cmd/roachtest/tests/validate_grant_option.go @@ -22,7 +22,7 @@ import ( func registerValidateGrantOption(r registry.Registry) { r.Add(registry.TestSpec{ - Name: "sql-experience/validate-grant-option", + Name: "validate-grant-option", Owner: registry.OwnerSQLExperience, Cluster: r.MakeClusterSpec(3), Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { @@ -36,6 +36,7 @@ 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) + t.L().PrintfCtx(ctx, "user root on node %d executing: %s", node, sqlStatement) _, err = conn.Exec(sqlStatement) if len(expectedErrText) == 0 { require.NoError(t, err) @@ -50,6 +51,7 @@ func execSQLAsUser(sqlStatement string, user string, expectedErrText string, nod return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { conn, err := u.c.ConnEAsUser(ctx, t.L(), node, user) require.NoError(t, err) + t.L().PrintfCtx(ctx, "user %s on node %d executing: %s", user, node, sqlStatement) _, err = conn.Exec(sqlStatement) if len(expectedErrText) == 0 { require.NoError(t, err) @@ -62,6 +64,11 @@ func execSQLAsUser(sqlStatement string, user string, expectedErrText string, nod func runRegisterValidateGrantOption( ctx context.Context, t test.Test, c cluster.Cluster, buildVersion version.Version, ) { + // This is meant to test a migration from 21.2 to 22.1. + if buildVersion.Major() != 22 || buildVersion.Minor() != 1 { + t.L().PrintfCtx(ctx, "skipping test because build version is %s", buildVersion) + return + } const mainVersion = "" predecessorVersion, err := PredecessorVersion(buildVersion) if err != nil { @@ -81,6 +88,7 @@ func runRegisterValidateGrantOption( execSQL("CREATE USER foo2;", "", 2), execSQL("CREATE USER foo3;", "", 2), execSQL("CREATE USER foo4;", "", 2), + execSQL("CREATE USER foo5;", "", 2), execSQL("CREATE USER target;", "", 1), execSQL("CREATE DATABASE d;", "", 2), @@ -99,17 +107,39 @@ func runRegisterValidateGrantOption( 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). + // Node 1 is upgraded, and nodes 2 and 3 are on the previous version. binaryUpgradeStep(c.Node(1), mainVersion), allowAutoUpgradeStep(1), execSQLAsUser("GRANT CREATE ON DATABASE d TO target;", "foo", "", 1), + execSQLAsUser("GRANT CREATE ON DATABASE defaultdb TO foo;", "root", "", 1), + execSQLAsUser("CREATE TABLE t2();", "foo", "", 1), // t2 created on a new node. + execSQLAsUser("CREATE TABLE t3();", "foo", "", 2), // t3 created on an old node. + execSQLAsUser("GRANT CREATE ON TABLE t2 TO foo2;", "foo", "", 1), // foo2 does not get GRANT or grant option on t2. + execSQLAsUser("GRANT GRANT, CREATE ON TABLE t3 TO foo2;", "foo", "", 2), // foo2 gets GRANT and grant option on t3 + execSQLAsUser("GRANT CREATE ON TABLE t2 TO foo3;", "foo", "", 1), // foo3 does not get GRANT or grant option on t2. + execSQLAsUser("GRANT GRANT, CREATE ON TABLE t3 TO foo3;", "foo", "", 2), // foo3 gets GRANT, but not grant option on t3 + + execSQLAsUser("GRANT CREATE ON TABLE t2 TO foo4 WITH GRANT OPTION;", "foo", "pq: version 21.2-22 must be finalized to use grant options", 1), + execSQLAsUser("GRANT CREATE ON TABLE t3 TO foo4 WITH GRANT OPTION;", "foo", "pq: at or near \"with\": syntax error", 2), + + execSQLAsUser("GRANT CREATE ON TABLE t2 TO foo3;", "foo2", + "pq: user foo2 does not have GRANT privilege on relation t2", 1), + execSQLAsUser("GRANT CREATE ON TABLE t2 TO foo3;", "foo2", + "pq: user foo2 does not have GRANT privilege on relation t2", 2), // Same error from node 2. + execSQLAsUser("GRANT CREATE ON TABLE t3 TO foo3;", "foo2", "", 1), + execSQLAsUser("GRANT CREATE ON TABLE t3 TO foo3;", "foo2", "", 2), + execSQLAsUser("GRANT CREATE ON TABLE t2 TO foo4;", "foo3", + "pq: user foo3 does not have GRANT privilege on relation t2", 1), + execSQLAsUser("GRANT CREATE ON TABLE t2 TO foo4;", "foo3", + "pq: user foo3 does not have GRANT privilege on relation t2", 2), // Same error from node 2. + execSQLAsUser("GRANT CREATE ON TABLE t3 TO foo4;", "foo3", "", 1), + execSQLAsUser("GRANT CREATE ON TABLE t3 TO foo4;", "foo3", "", 2), + 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). + // Node 2 is upgraded. binaryUpgradeStep(c.Node(2), mainVersion), allowAutoUpgradeStep(2), execSQLAsUser("GRANT ALL PRIVILEGES ON DATABASE d TO target;", "foo", "", 3), @@ -135,18 +165,20 @@ func runRegisterValidateGrantOption( 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 USAGE ON SCHEMA s TO foo;", "foo3", "", 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), + execSQLAsUser("GRANT DELETE ON TABLE t1 TO target;", "foo2", "pq: user foo2 missing WITH GRANT OPTION privilege on DELETE", 1), + execSQLAsUser("GRANT DELETE ON TABLE t1 TO target;", "foo4", "", 1), + execSQLAsUser("GRANT SELECT ON TABLE t1 TO target;", "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), + execSQLAsUser("GRANT SELECT ON TABLE t2 TO foo4 WITH GRANT OPTION;", "foo", "", 1), + execSQLAsUser("GRANT SELECT ON TABLE t3 TO foo4 WITH GRANT OPTION;", "foo", "", 2), + + execSQLAsUser("GRANT CREATE ON TABLE t2 TO foo3;", "foo2", "pq: user foo2 missing WITH GRANT OPTION privilege on CREATE", 1), + execSQLAsUser("GRANT CREATE ON TABLE t3 TO target;", "foo2", "", 2), + execSQLAsUser("GRANT CREATE ON TABLE t2 TO foo4;", "foo3", "pq: user foo3 missing WITH GRANT OPTION privilege on CREATE", 3), + execSQLAsUser("GRANT CREATE ON TABLE t3 TO target;", "foo3", "", 3), ) u.run(ctx, t) } diff --git a/pkg/migration/migrations/grant_option_migration_external_test.go b/pkg/migration/migrations/grant_option_migration_external_test.go index 185e40d16765..e12d01c013a5 100644 --- a/pkg/migration/migrations/grant_option_migration_external_test.go +++ b/pkg/migration/migrations/grant_option_migration_external_test.go @@ -65,9 +65,9 @@ func TestGrantOptionMigration(t *testing.T) { */ const objectTest = ` -124e0a02646210341a310a0b0a0561646d696e100218020a0a0a04726f6f74100218020a0e0a087465737475736572100418001204726f6f741802220028033a090a017312040835100040004a005a00 -22420834120173183522310a0b0a0561646d696e100218020a0a0a04726f6f74100218020a0e0a087465737475736572100418001204726f6f7418022a00300240004a00 -0aab020a0274311836203428023a00423a0a05726f77696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293001680070007800800100880100980100480252500a0774315f706b6579100118012205726f776964300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba010060026a310a0b0a0561646d696e100218020a0a0a04726f6f74100218020a0e0a087465737475736572106418001204726f6f741802800101880103980100b201160a077072696d61727910001a05726f77696420012800b80101c20100e80100f2010408001200f801008002009202009a020a08d8bfcfc594a985dc16b20200b80200c0021dc80200e00200f00200 +12480a02646210341a2b0a090a0561646d696e10020a080a04726f6f7410020a0c0a08746573747573657210041204726f6f741802220028033a090a017312040835100040004a005a00 +223c08341201731835222b0a090a0561646d696e10020a080a04726f6f7410020a0c0a08746573747573657210041204726f6f7418022a00300240004a00 +0aa5020a0274311836203428023a00423a0a05726f77696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293001680070007800800100880100980100480252500a077072696d617279100118012205726f776964300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba010060026a2b0a090a0561646d696e10020a080a04726f6f7410020a0c0a08746573747573657210641204726f6f741802800101880103980100b201160a077072696d61727910001a05726f77696420012800b80101c20100e80100f2010408001200f801008002009202009a020a08e0bea485c6bbede616b20200b80200c0021dc80200e00200f00200 ` // Check that descriptors for multiple users are pulled and updated when @@ -104,10 +104,10 @@ func TestGrantOptionMigration(t *testing.T) { */ const multipleUsersTest = ` -124e0a02646210341a310a0b0a0561646d696e100218020a0a0a04726f6f74100218020a0e0a087465737475736572100218021204726f6f741802220028033a090a017312040835100040004a005a00 -22530834120173183522420a0b0a0561646d696e100218020a0a0a04726f6f74100218020a0e0a087465737475736572100218020a0f0a09746573747573657232101418001204726f6f7418022a00300240004a00 -0abd020a0274311836203428023a00423a0a05726f77696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293001680070007800800100880100980100480252500a0774315f706b6579100118012205726f776964300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba010060026a430a0b0a0561646d696e100218020a0a0a04726f6f74100218020a0e0a087465737475736572100218020a100a0974657374757365723210a00118001204726f6f741802800101880103980100b201160a077072696d61727910001a05726f77696420012800b80101c20100e80100f2010408001200f801008002009202009a020a08b8b091afa2d484dc16b20200b80200c00235c80200e00200f00200 -0abc020a0274321837203428023a00423a0a05726f77696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293001680070007800800100880100980100480252500a0774325f706b6579100118012205726f776964300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba010060026a420a0b0a0561646d696e100218020a0a0a04726f6f74100218020a0e0a087465737475736572100218020a0f0a09746573747573657232100218001204726f6f741802800101880103980100b201160a077072696d61727910001a05726f77696420012800b80101c20100e80100f2010408001200f801008002009202009a020a08d8efb894b8d484dc16b20200b80200c00235c80200e00200f00200 +12480a02646210371a2b0a090a0561646d696e10020a080a04726f6f7410020a0c0a08746573747573657210021204726f6f741802220028033a090a017312040838100040004a005a00 +224b08371201731838223a0a090a0561646d696e10020a080a04726f6f7410020a0c0a08746573747573657210020a0d0a0974657374757365723210141204726f6f7418022a00300240004a00 +0ab5020a0274311839203728023a00423a0a05726f77696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293001680070007800800100880100980100480252500a077072696d617279100118012205726f776964300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba010060026a3b0a090a0561646d696e10020a080a04726f6f7410020a0c0a08746573747573657210020a0e0a0974657374757365723210a0011204726f6f741802800101880103980100b201160a077072696d61727910001a05726f77696420012800b80101c20100e80100f2010408001200f801008002009202009a020a08c8c9daf5dabdede616b20200b80200c00238c80200e00200f00200 +0ab4020a027432183a203728023a00423a0a05726f77696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293001680070007800800100880100980100480252500a077072696d617279100118012205726f776964300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba010060026a3a0a090a0561646d696e10020a080a04726f6f7410020a0c0a08746573747573657210020a0d0a0974657374757365723210021204726f6f741802800101880103980100b201160a077072696d61727910001a05726f77696420012800b80101c20100e80100f2010408001200f801008002009202009a020a08c8c7f884dcbdede616b20200b80200c00238c80200e00200f00200 ` // Test the migration for types @@ -140,75 +140,78 @@ func TestGrantOptionMigration(t *testing.T) { */ const typesTest = ` -123e0a02646210341a210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f741802220028023a090a017312040835100040004a005a00 -22320834120173183522210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f7418022a00300140004a00 -1a590834101d1a02747920362800403748025200680072410a0b0a0561646d696e100218020a0d0a067075626c696310800418000a0a0a04726f6f74100218020a0f0a08746573747573657210800418001204726f6f7418027a00 -1a710834101d1a035f7479203728013a26080f100018003000381850d78d065a14081810001800300050d68d0660007a0410d78d066000400048015200680072300a0b0a0561646d696e100218020a0d0a067075626c696310800418000a0a0a04726f6f74100218021204726f6f7418027a00 -1a59083410351a0374793220382800403948025200680072400a0b0a0561646d696e100218020a0d0a067075626c696310800418000a0a0a04726f6f74100218020a0e0a087465737475736572100218001204726f6f7418027a00 -1a72083410351a045f747932203928013a26080f100018003000381850d98d065a14081810001800300050d88d0660007a0410d98d066000400048015200680072300a0b0a0561646d696e100218020a0d0a067075626c696310800418000a0a0a04726f6f74100218021204726f6f7418027a00 +123a0a026462103b1a1d0a090a0561646d696e10020a080a04726f6f7410021204726f6f741802220028023a090a01731204083c100040004a005a00 +222e083b120173183c221d0a090a0561646d696e10020a080a04726f6f7410021204726f6f7418022a00300140004a00 +1a51083b101d1a027479203d2800403e48025200680072390a090a0561646d696e10020a0b0a067075626c69631080040a080a04726f6f7410020a0d0a0874657374757365721080041204726f6f7418027a00 +1a6b083b101d1a035f7479203e28013a26080f100018003000381850de8d065a14081810001800300050dd8d0660007a0410de8d0660004000480152006800722a0a090a0561646d696e10020a0b0a067075626c69631080040a080a04726f6f7410021204726f6f7418027a00 +1a51083b103c1a03747932203f2800404048025200680072380a090a0561646d696e10020a0b0a067075626c69631080040a080a04726f6f7410020a0c0a08746573747573657210021204726f6f7418027a00 +1a6c083b103c1a045f747932204028013a26080f100018003000381850e08d065a14081810001800300050df8d0660007a0410e08d0660004000480152006800722a0a090a0561646d696e10020a0b0a067075626c69631080040a080a04726f6f7410021204726f6f7418027a00 ` - testCases := []string{ - objectTest, - multipleUsersTest, - typesTest, + testCases := map[string]string{ + "object_test": objectTest, + "multipleUsersTest": multipleUsersTest, + "typesTest": 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 + for name, descriptorStringsToInject := range testCases { + t.Run(name, func(t *testing.T) { + 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 */ - )) + 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 - } + _, err := tc.Conns[0].ExecContext(ctx, `SET CLUSTER SETTING version = $1`, + clusterversion.ByKey(clusterversion.ValidateGrantOption).String()) + require.NoError(t, err) - 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) + err = sql.TestingDescsTxn(ctx, tc.Server(0), func(ctx context.Context, txn *kv.Txn, col *descs.Collection) error { + // Avoid running validation on the descriptors. + cat, err := col.GetCatalogUnvalidated(ctx, txn) + if err != nil { + return err + } + for _, desc := range cat.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 %s, Privileges: %d, Grant Option: %d", + u.User(), u.Privileges, u.WithGrantOption) + } } } - } - } - return nil + } + return nil + }) + require.NoError(t, err) + tc.Stopper().Stop(ctx) }) - - require.NoError(t, err) - tc.Stopper().Stop(ctx) } } diff --git a/pkg/sql/alter_default_privileges.go b/pkg/sql/alter_default_privileges.go index 982eeb68cff3..e703d875fa48 100644 --- a/pkg/sql/alter_default_privileges.go +++ b/pkg/sql/alter_default_privileges.go @@ -90,6 +90,12 @@ func (p *planner) alterDefaultPrivileges( } func (n *alterDefaultPrivilegesNode) startExec(params runParams) error { + if (n.n.Grant.WithGrantOption || n.n.Revoke.GrantOptionFor) && + !params.p.ExecCfg().Settings.Version.IsActive(params.ctx, clusterversion.ValidateGrantOption) { + return pgerror.Newf(pgcode.FeatureNotSupported, + "version %v must be finalized to use grant options", + clusterversion.ByKey(clusterversion.ValidateGrantOption)) + } targetRoles, err := n.n.Roles.ToSQLUsernames(params.SessionData(), security.UsernameValidation) if err != nil { return err @@ -194,12 +200,11 @@ func (n *alterDefaultPrivilegesNode) alterDefaultPrivilegesForSchemas( } grantPresent, allPresent := false, false + for _, priv := range privileges { + grantPresent = grantPresent || priv == privilege.GRANT + allPresent = allPresent || priv == privilege.ALL + } if params.ExecCfg().Settings.Version.IsActive(params.ctx, clusterversion.ValidateGrantOption) { - for _, priv := range privileges { - grantPresent = grantPresent || priv == privilege.GRANT - allPresent = allPresent || priv == privilege.ALL - } - noticeMessage := "" // we only output the message for ALL privilege if it is being granted without the WITH GRANT OPTION flag // if GRANT privilege is involved, we must always output the message @@ -208,9 +213,9 @@ func (n *alterDefaultPrivilegesNode) alterDefaultPrivilegesForSchemas( } else if grantPresent { noticeMessage = "the GRANT privilege is deprecated" } - if len(noticeMessage) > 0 { - params.p.noticeSender.BufferNotice( + params.p.BufferClientNotice( + params.ctx, errors.WithHint( pgnotice.Newf("%s", noticeMessage), "please use WITH GRANT OPTION", @@ -297,12 +302,11 @@ func (n *alterDefaultPrivilegesNode) alterDefaultPrivilegesForDatabase( } grantPresent, allPresent := false, false + for _, priv := range privileges { + grantPresent = grantPresent || priv == privilege.GRANT + allPresent = allPresent || priv == privilege.ALL + } if params.ExecCfg().Settings.Version.IsActive(params.ctx, clusterversion.ValidateGrantOption) { - for _, priv := range privileges { - grantPresent = grantPresent || priv == privilege.GRANT - allPresent = allPresent || priv == privilege.ALL - } - noticeMessage := "" // we only output the message for ALL privilege if it is being granted without the WITH GRANT OPTION flag // if GRANT privilege is involved, we must always output the message @@ -313,7 +317,8 @@ func (n *alterDefaultPrivilegesNode) alterDefaultPrivilegesForDatabase( } if len(noticeMessage) > 0 { - params.p.noticeSender.BufferNotice( + params.p.BufferClientNotice( + params.ctx, errors.WithHint( pgnotice.Newf("%s", noticeMessage), "please use WITH GRANT OPTION", diff --git a/pkg/sql/catalog/catprivilege/default_privilege.go b/pkg/sql/catalog/catprivilege/default_privilege.go index 3066317b0c64..140562d00b6c 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.GrantOptionVersion + newPrivs.Version = descpb.Version21_2 // 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 cb88a0321ee7..fd9737e54e1b 100644 --- a/pkg/sql/catalog/catprivilege/fix.go +++ b/pkg/sql/catalog/catprivilege/fix.go @@ -143,12 +143,15 @@ func MaybeFixPrivileges( // 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 + // If admin has grant option bits set, then we know the descriptor was + // created by a new binary, so all the other grant options are already + // correct. Note that admin always has SELECT on *every* table including + // system tables. + if p.CheckGrantOptions(security.AdminRoleName(), privilege.List{privilege.SELECT}) { + return false } + changed := false for i := range p.Users { u := &p.Users[i] if privilege.ALL.IsSetIn(u.Privileges) { @@ -166,8 +169,5 @@ func MaybeUpdateGrantOptions(p *descpb.PrivilegeDescriptor) bool { } } - p.SetVersion(descpb.GrantOptionVersion) - changed = true - return changed } diff --git a/pkg/sql/catalog/descpb/privilege.go b/pkg/sql/catalog/descpb/privilege.go index ad0047818b76..7fdef981ad37 100644 --- a/pkg/sql/catalog/descpb/privilege.go +++ b/pkg/sql/catalog/descpb/privilege.go @@ -39,11 +39,6 @@ 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. @@ -126,7 +121,7 @@ func NewCustomSuperuserPrivilegeDescriptor( WithGrantOption: priv.ToBitField(), }, }, - Version: GrantOptionVersion, + Version: Version21_2, } } @@ -156,7 +151,7 @@ func NewPrivilegeDescriptor( WithGrantOption: grantOption.ToBitField(), }, }, - Version: GrantOptionVersion, + Version: Version21_2, } } diff --git a/pkg/sql/catalog/descpb/privilegedescversion_string.go b/pkg/sql/catalog/descpb/privilegedescversion_string.go index f4f0858bf3f1..295607e8c4c3 100644 --- a/pkg/sql/catalog/descpb/privilegedescversion_string.go +++ b/pkg/sql/catalog/descpb/privilegedescversion_string.go @@ -11,12 +11,11 @@ func _() { _ = x[InitialVersion-0] _ = x[OwnerVersion-1] _ = x[Version21_2-2] - _ = x[GrantOptionVersion-3] } -const _PrivilegeDescVersion_name = "InitialVersionOwnerVersionVersion21_2GrantOptionVersion" +const _PrivilegeDescVersion_name = "InitialVersionOwnerVersionVersion21_2" -var _PrivilegeDescVersion_index = [...]uint8{0, 14, 26, 37, 55} +var _PrivilegeDescVersion_index = [...]uint8{0, 14, 26, 37} func (i PrivilegeDescVersion) String() string { if i >= PrivilegeDescVersion(len(_PrivilegeDescVersion_index)-1) { diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go index 69b405340a48..045022419e13 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.GrantOptionVersion + tbl.Privileges.Version = descpb.Version21_2 tbl.Privileges.Users = append(tbl.Privileges.Users, descpb.UserPrivileges{ UserProto: security.PublicRoleName().EncodeProto(), Privileges: privilege.List{privilege.SELECT}.ToBitField(), diff --git a/pkg/sql/descriptor.go b/pkg/sql/descriptor.go index 1f66546ee6da..8d6c1f6f3e6e 100644 --- a/pkg/sql/descriptor.go +++ b/pkg/sql/descriptor.go @@ -387,7 +387,8 @@ func (p *planner) maybeInitializeMultiRegionMetadata( } primaryRegion = tree.Name(defaultPrimaryRegion) // TODO(#67156): send notice immediately, so it pops up even on error. - p.noticeSender.BufferNotice( + p.BufferClientNotice( + ctx, pgnotice.Newf("setting %s as the PRIMARY REGION as no PRIMARY REGION was specified", primaryRegion), ) } diff --git a/pkg/sql/grant_revoke.go b/pkg/sql/grant_revoke.go index 2a3601a76395..c54926a8283a 100644 --- a/pkg/sql/grant_revoke.go +++ b/pkg/sql/grant_revoke.go @@ -120,6 +120,12 @@ func (n *changePrivilegesNode) startExec(params runParams) error { ctx := params.ctx p := params.p + if n.withGrantOption && !p.ExecCfg().Settings.Version.IsActive(ctx, clusterversion.ValidateGrantOption) { + return pgerror.Newf(pgcode.FeatureNotSupported, + "version %v must be finalized to use grant options", + clusterversion.ByKey(clusterversion.ValidateGrantOption)) + } + if err := p.validateRoles(ctx, n.grantees, true /* isPublicValid */); err != nil { return err } @@ -188,11 +194,10 @@ func (n *changePrivilegesNode) startExec(params runParams) error { } if len(n.desiredprivs) > 0 { - // Only allow granting/revoking privileges that the requesting - // user themselves have on the descriptor. - grantPresent, allPresent := false, false for _, priv := range n.desiredprivs { + // Only allow granting/revoking privileges that the requesting + // user themselves have on the descriptor. if err := p.CheckPrivilege(ctx, descriptor, priv); err != nil { return err } @@ -201,45 +206,51 @@ func (n *changePrivilegesNode) startExec(params runParams) error { } privileges := descriptor.GetPrivileges() + noticeMessage := "" if p.ExecCfg().Settings.Version.IsActive(ctx, clusterversion.ValidateGrantOption) { err := p.CheckGrantOptionsForUser(ctx, descriptor, n.desiredprivs, n.isGrant) if err != nil { return err } - noticeMessage := "" - // we only output the message for ALL privilege if it is being granted without the WITH GRANT OPTION flag - // if GRANT privilege is involved, we must always output the message + // We only output the message for ALL privilege if it is being granted + // without the WITH GRANT OPTION flag if GRANT privilege is involved, we + // must always output the message if allPresent && n.isGrant && !n.withGrantOption { noticeMessage = "grant options were automatically applied but this behavior is deprecated" } else if grantPresent { noticeMessage = "the GRANT privilege is deprecated" } - - if len(noticeMessage) > 0 { - params.p.noticeSender.BufferNotice( - errors.WithHint( - pgnotice.Newf("%s", noticeMessage), - "please use WITH GRANT OPTION", - ), - ) - } } for _, grantee := range n.grantees { n.changePrivilege(privileges, n.desiredprivs, grantee) - if p.ExecCfg().Settings.Version.IsActive(ctx, clusterversion.ValidateGrantOption) { - if grantPresent || allPresent { - if n.isGrant { - privileges.GrantPrivilegeToGrantOptions(grantee, true /*isGrant*/) - } else if !n.isGrant && !n.withGrantOption { - privileges.GrantPrivilegeToGrantOptions(grantee, false /*isGrant*/) - } + // TODO (sql-exp): remove the rest of this loop in 22.2. + granteeHasGrantPriv := privileges.CheckPrivilege(grantee, privilege.GRANT) + + if granteeHasGrantPriv && n.isGrant && !n.withGrantOption && len(noticeMessage) == 0 { + noticeMessage = "grant options were automatically applied but this behavior is deprecated" + } + if grantPresent || allPresent || (granteeHasGrantPriv && n.isGrant) { + if n.isGrant { + privileges.GrantPrivilegeToGrantOptions(grantee, true /*isGrant*/) + } else if !n.isGrant && !n.withGrantOption { + privileges.GrantPrivilegeToGrantOptions(grantee, false /*isGrant*/) } } } + if len(noticeMessage) > 0 { + params.p.BufferClientNotice( + ctx, + errors.WithHint( + pgnotice.Newf("%s", noticeMessage), + "please use WITH GRANT OPTION", + ), + ) + } + // Ensure superusers have exactly the allowed privilege set. // Postgres does not actually enforce this, instead of checking that // superusers have all the privileges, Postgres allows superusers to diff --git a/pkg/sql/logictest/testdata/logic_test/bytes b/pkg/sql/logictest/testdata/logic_test/bytes index e913f9e04d65..b136506a1ae0 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\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 +\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\002"\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/logictest/testdata/logic_test/grant_option_mixed_version b/pkg/sql/logictest/testdata/logic_test/grant_option_mixed_version new file mode 100644 index 000000000000..a6017dec052d --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/grant_option_mixed_version @@ -0,0 +1,16 @@ +# LogicTest: local-mixed-21.2-22.1 + +statement ok +CREATE TABLE t(a INT) + +statement error version 21.2-22 must be finalized to use grant options +GRANT SELECT ON t TO testuser WITH GRANT OPTION + +statement error version 21.2-22 must be finalized to use grant options +ALTER DEFAULT PRIVILEGES GRANT SELECT, INSERT ON TABLES TO testuser WITH GRANT OPTION + +statement error version 21.2-22 must be finalized to use grant options +REVOKE GRANT OPTION FOR SELECT ON t FROM testuser + +statement error version 21.2-22 must be finalized to use grant options +ALTER DEFAULT PRIVILEGES REVOKE GRANT OPTION FOR INSERT, DELETE ON TABLES FROM testuser diff --git a/pkg/sql/logictest/testdata/logic_test/grant_revoke_with_grant_option b/pkg/sql/logictest/testdata/logic_test/grant_revoke_with_grant_option index 12553c7d8226..bbd0c77a3aa0 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_revoke_with_grant_option +++ b/pkg/sql/logictest/testdata/logic_test/grant_revoke_with_grant_option @@ -92,13 +92,13 @@ test public t testuser2 SELECT user testuser2 -# in version 22.1, granting GRANT to a user means they now have grant options on all their privileges. -# This is to promote backwards compatibility as we deprecate GRANT +# In version 22.1, granting GRANT to a user means they now have grant options on +# all their privileges. This is to promote backwards compatibility as we deprecate GRANT. statement ok GRANT INSERT, SELECT ON TABLE t TO target -# however, future privileges do not automatically get grant options just because the user currently -# holds GRANT - you would need to either specify grant options or grant GRANT again +# Also, privileges will automatically get grant options if the user currently +# holds GRANT. user root statement ok @@ -106,8 +106,11 @@ GRANT DELETE ON TABLE t TO testuser2 user testuser2 -statement error user testuser2 missing WITH GRANT OPTION privilege on DELETE +query T noticetrace GRANT DELETE ON TABLE t TO target +---- +NOTICE: grant options were automatically applied but this behavior is deprecated +HINT: please use WITH GRANT OPTION user testuser diff --git a/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic b/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic index bd0bffa140c3..f2328dcefaa2 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:3 > > +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 /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:3 > 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:2 > 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: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 > +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 > 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: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 > +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 > 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: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 > +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 > 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: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 > +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 > 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: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 > +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 > 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: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 > +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 > exec stmt rows affected: 0 # Check that session tracing does not inhibit the fast path for inserts & diff --git a/pkg/sql/rename_table.go b/pkg/sql/rename_table.go index 56b5abe7d4fb..6f695af0569d 100644 --- a/pkg/sql/rename_table.go +++ b/pkg/sql/rename_table.go @@ -135,7 +135,8 @@ func (n *renameTableNode) startExec(params runParams) error { // process of deprecating qualified rename targets, so issue a notice. // TODO (rohany): Convert this to take in an unqualified name after 20.2 // is released (#51445). - params.p.noticeSender.BufferNotice( + params.p.BufferClientNotice( + ctx, errors.WithHintf( pgnotice.Newf("renaming tables with a qualification is deprecated"), "use ALTER TABLE %s RENAME TO %s instead", diff --git a/pkg/sql/schemachanger/testdata/drop b/pkg/sql/schemachanger/testdata/drop index 772d199089e9..de4648f486f8 100644 --- a/pkg/sql/schemachanger/testdata/drop +++ b/pkg/sql/schemachanger/testdata/drop @@ -30,7 +30,7 @@ upsert descriptor #54 upsert descriptor #56 ... withGrantOption: 2 - version: 3 + version: 2 - version: "1" + state: DROP + version: "2" @@ -139,21 +139,21 @@ upsert descriptor #54 upsert descriptor #57 ... withGrantOption: 2 - version: 3 + version: 2 - version: "1" + state: DROP + version: "2" upsert descriptor #59 ... withGrantOption: 2 - version: 3 + version: 2 - version: "1" + state: DROP + version: "2" upsert descriptor #60 ... withGrantOption: 2 - version: 3 + version: 2 - version: "1" + state: DROP + version: "2" @@ -188,7 +188,7 @@ create job #1: "schema change job" upsert descriptor #54 ... withGrantOption: 2 - version: 3 + version: 2 - schemas: - public: - id: 55 @@ -199,7 +199,7 @@ upsert descriptor #54 upsert descriptor #55 ... withGrantOption: 2 - version: 3 + version: 2 - 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: 3 + version: 2 state: DROP - version: "6" + version: "7" @@ -302,7 +302,7 @@ delete object namespace entry {61 63 v5} -> 74 upsert descriptor #61 ... withGrantOption: 2 - version: 3 + version: 2 - schemas: - public: - id: 62 @@ -315,14 +315,14 @@ upsert descriptor #61 upsert descriptor #62 ... withGrantOption: 2 - version: 3 + version: 2 - version: "1" + state: DROP + version: "2" upsert descriptor #63 ... withGrantOption: 2 - version: 3 + version: 2 - version: "1" + state: DROP + version: "2" @@ -551,7 +551,7 @@ upsert descriptor #71 upsert descriptor #72 ... withGrantOption: 2 - version: 3 + version: 2 - referencingDescriptorIds: - - 74 - version: "2" @@ -561,7 +561,7 @@ upsert descriptor #72 upsert descriptor #73 ... withGrantOption: 2 - version: 3 + version: 2 - 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: 3 + version: 2 state: DROP - version: "3" + version: "4" diff --git a/pkg/sql/serial.go b/pkg/sql/serial.go index cdef293ef6fc..c7ab56d8c6b2 100644 --- a/pkg/sql/serial.go +++ b/pkg/sql/serial.go @@ -155,7 +155,8 @@ func (p *planner) generateSerialInColumnDef( // switch this behavior around. upgradeType := types.Int if defType.Width() < upgradeType.Width() { - p.noticeSender.BufferNotice( + p.BufferClientNotice( + ctx, errors.WithHintf( pgnotice.Newf( "upgrading the column %s to %s to utilize the session serial_normalization setting",