From faad69dd8ee12d95f9de6b06f58da2265ab8eea5 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Thu, 16 Jan 2025 02:20:47 -0500 Subject: [PATCH 1/6] sql: fix ordering of arguments for IndexBackfillPlanner The "now" argument was being passed as the "writeAsOf" parameter. Release note: None --- pkg/sql/index_backfiller.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/pkg/sql/index_backfiller.go b/pkg/sql/index_backfiller.go index 0a4492adc938..e40aeef300ac 100644 --- a/pkg/sql/index_backfiller.go +++ b/pkg/sql/index_backfiller.go @@ -71,7 +71,7 @@ func (ib *IndexBackfillPlanner) BackfillIndexes( tracker scexec.BackfillerProgressWriter, job *jobs.Job, descriptor catalog.TableDescriptor, -) (err error) { +) (retErr error) { // Potentially install a protected timestamp before the GC interval is hit, // which can help avoid transaction retry errors, with shorter GC intervals. protectedTimestampCleaner := ib.execCfg.ProtectedTimestampManager.TryToProtectBeforeGC(ctx, @@ -81,7 +81,7 @@ func (ib *IndexBackfillPlanner) BackfillIndexes( defer func() { cleanupError := protectedTimestampCleaner(ctx) if cleanupError != nil { - err = errors.CombineErrors(cleanupError, err) + retErr = errors.CombineErrors(retErr, cleanupError) } }() @@ -117,18 +117,18 @@ func (ib *IndexBackfillPlanner) BackfillIndexes( return nil } now := ib.execCfg.DB.Clock().Now() - run, err := ib.plan( + run, retErr := ib.plan( ctx, descriptor, - progress.MinimumWriteTimestamp, now, progress.MinimumWriteTimestamp, + progress.MinimumWriteTimestamp, spansToDo, progress.DestIndexIDs, updateFunc, ) - if err != nil { - return err + if retErr != nil { + return retErr } return run(ctx) } From 694e87b92879536587d629c5e2e7b6e17d75b4b6 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Thu, 16 Jan 2025 02:22:17 -0500 Subject: [PATCH 2/6] jobsprotectedts: fix calculation in TryToProtectBeforeGC This function was attempting to compute 80% of the time until the GC threshold is reached. However, it did not take into account that the read was happening with a historical timestamp. Now this is part of the calculation. Release note: None --- pkg/jobs/jobsprotectedts/BUILD.bazel | 1 + .../jobs_protected_ts_manager.go | 19 +++++++++++++------ 2 files changed, 14 insertions(+), 6 deletions(-) diff --git a/pkg/jobs/jobsprotectedts/BUILD.bazel b/pkg/jobs/jobsprotectedts/BUILD.bazel index 3da23bacb7fa..49689d645e6c 100644 --- a/pkg/jobs/jobsprotectedts/BUILD.bazel +++ b/pkg/jobs/jobsprotectedts/BUILD.bazel @@ -23,6 +23,7 @@ go_library( "//pkg/sql/isql", "//pkg/util/ctxgroup", "//pkg/util/hlc", + "//pkg/util/timeutil", "//pkg/util/uuid", "@com_github_cockroachdb_errors//:errors", ], diff --git a/pkg/jobs/jobsprotectedts/jobs_protected_ts_manager.go b/pkg/jobs/jobsprotectedts/jobs_protected_ts_manager.go index 7eaac363c30a..d90f1a1d80e9 100644 --- a/pkg/jobs/jobsprotectedts/jobs_protected_ts_manager.go +++ b/pkg/jobs/jobsprotectedts/jobs_protected_ts_manager.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" ) @@ -107,22 +108,28 @@ func (p *Manager) TryToProtectBeforeGC( protectedTSInstallCancel := make(chan struct{}) var unprotectCallback Cleaner waitGrp.GoCtx(func(ctx context.Context) error { - // If we are starting up the system config can be nil, we are okay letting - // the job restart, due to the GC interval and lack of protected timestamp. + // If we are starting up, the system config can be nil. We are okay letting + // the job restart due to the GC interval and lack of protected timestamp. systemConfig := p.systemConfig.GetSystemConfig() if systemConfig == nil { return nil } // Determine what the GC interval is on the table, which will help us - // figure out when to apply a protected timestamp, as a percentage of this - // time. + // figure out when to apply a protected timestamp as a percentage of the + // time until GC can occur. zoneCfg, err := systemConfig.GetZoneConfigForObject(p.codec, config.ObjectID(tableDesc.GetID())) if err != nil { return err } - waitBeforeProtectedTS := time.Duration((time.Duration(zoneCfg.GC.TTLSeconds) * time.Second).Seconds() * - timedProtectTimeStampGCPct) + waitBeforeProtectedTS := time.Duration(0) + now := timeutil.Now() + readAsOfTime := timeutil.Unix(0, readAsOf.WallTime) + gcTTL := time.Duration(zoneCfg.GC.TTLSeconds) * time.Second + if readAsOfTime.Add(gcTTL).After(now) { + timeUntilGC := readAsOfTime.Add(gcTTL).Sub(now) + waitBeforeProtectedTS = time.Duration(float64(timeUntilGC) * timedProtectTimeStampGCPct) + } select { case <-time.After(waitBeforeProtectedTS): From 4a487d8de230601192d9c2767657f222bbd74f54 Mon Sep 17 00:00:00 2001 From: Matt Spilchen Date: Thu, 16 Jan 2025 11:57:13 -0400 Subject: [PATCH 3/6] sql/schemachanger: Add support for storing roles in policies A previous commit introduced basic support for CREATE/DROP POLICY. This commit extends that functionality by storing roles in the policy descriptor as strings. The roles are modelled as separate DSC elements, making it easier to modify the roles in a policy, which will be supported with ALTER POLICY. Epic: CRDB-11724 Informs: #136696 Release note: None --- pkg/sql/catalog/descpb/structured.proto | 3 + pkg/sql/catalog/tabledesc/validate.go | 64 +++++++-- pkg/sql/catalog/tabledesc/validate_test.go | 78 ++++++++--- pkg/sql/catalog/validate.go | 15 ++ pkg/sql/drop_role.go | 14 ++ .../testdata/logic_test/row_level_security | 129 ++++++++++++++++++ .../internal/scbuildstmt/create_policy.go | 38 ++++++ .../internal/scbuildstmt/drop_policy.go | 2 +- .../scbuild/testdata/create_policy | 14 +- .../scbuild/testdata/drop_policy | 14 +- pkg/sql/schemachanger/scdecomp/decomp.go | 7 + .../scexec/scmutationexec/policy.go | 43 ++++++ .../schemachanger/scop/immediate_mutation.go | 12 ++ .../immediate_mutation_visitor_generated.go | 12 ++ pkg/sql/schemachanger/scpb/elements.proto | 8 ++ .../schemachanger/scpb/elements_generated.go | 41 ++++++ pkg/sql/schemachanger/scpb/uml/table.puml | 7 + .../scplan/internal/opgen/BUILD.bazel | 1 + .../internal/opgen/opgen_policy_role.go | 33 +++++ .../scplan/internal/rules/current/helpers.go | 2 +- .../internal/rules/current/testdata/deprules | 40 +++--- pkg/sql/schemachanger/screl/attr.go | 5 + pkg/sql/schemachanger/screl/scalars.go | 2 +- .../create_policy/create_policy.definition | 2 +- .../create_policy/create_policy.explain | 29 ++-- .../create_policy/create_policy.explain_shape | 4 +- .../create_policy/create_policy.side_effects | 11 +- .../drop_policy/drop_policy.definition | 4 +- .../drop_policy/drop_policy.explain | 30 ++-- .../drop_policy/drop_policy.explain_shape | 4 +- .../drop_policy/drop_policy.side_effects | 18 ++- 31 files changed, 596 insertions(+), 90 deletions(-) create mode 100644 pkg/sql/schemachanger/scplan/internal/opgen/opgen_policy_role.go diff --git a/pkg/sql/catalog/descpb/structured.proto b/pkg/sql/catalog/descpb/structured.proto index b27a1a2ef2ff..632a684bfb14 100644 --- a/pkg/sql/catalog/descpb/structured.proto +++ b/pkg/sql/catalog/descpb/structured.proto @@ -712,6 +712,9 @@ message PolicyDescriptor { // Command specifies the SQL commands to which the policy applies. optional cockroach.sql.catalog.catpb.PolicyCommand command = 4 [(gogoproto.nullable) = false]; + + // RoleNames is a list of roles that this policy applies to. + repeated string role_names = 5; } // A DescriptorMutation represents a column or an index that diff --git a/pkg/sql/catalog/tabledesc/validate.go b/pkg/sql/catalog/tabledesc/validate.go index 7a3cf93b8f8c..6688afb5042e 100644 --- a/pkg/sql/catalog/tabledesc/validate.go +++ b/pkg/sql/catalog/tabledesc/validate.go @@ -9,6 +9,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" @@ -2060,18 +2061,11 @@ func (desc *wrapper) validatePolicies() error { names := make(map[string]descpb.PolicyID, len(policies)) idToName := make(map[descpb.PolicyID]string, len(policies)) for _, p := range policies { - if p.ID == 0 { - return errors.AssertionFailedf( - "policy ID was missing for policy %q", - p.Name) - } else if p.ID >= desc.NextPolicyID { - return errors.AssertionFailedf( - "policy %q has ID %d, which is not less than the NextPolicyID value %d for the table", - p.Name, p.ID, desc.NextPolicyID) - } - if p.Name == "" { - return pgerror.Newf(pgcode.Syntax, "empty policy name") + if err := desc.validatePolicy(&p); err != nil { + return err } + + // Perform validation across all policies defined for the table. if otherID, found := names[p.Name]; found && p.ID != otherID { return pgerror.Newf(pgcode.DuplicateObject, "duplicate policy name: %q", p.Name) @@ -2083,13 +2077,53 @@ func (desc *wrapper) validatePolicies() error { p.ID, p.Name, other) } idToName[p.ID] = p.Name - if _, ok := catpb.PolicyType_name[int32(p.Type)]; !ok || p.Type == catpb.PolicyType_POLICYTYPE_UNUSED { + } + return nil +} + +// validatePolicy will validate a single policy in isolation from other policies in the table. +func (desc *wrapper) validatePolicy(p *descpb.PolicyDescriptor) error { + if p.ID == 0 { + return errors.AssertionFailedf( + "policy ID was missing for policy %q", + p.Name) + } else if p.ID >= desc.NextPolicyID { + return errors.AssertionFailedf( + "policy %q has ID %d, which is not less than the NextPolicyID value %d for the table", + p.Name, p.ID, desc.NextPolicyID) + } + if p.Name == "" { + return pgerror.Newf(pgcode.Syntax, "empty policy name") + } + if _, ok := catpb.PolicyType_name[int32(p.Type)]; !ok || p.Type == catpb.PolicyType_POLICYTYPE_UNUSED { + return errors.AssertionFailedf( + "policy %q has an unknown policy type %v", p.Name, p.Type) + } + if _, ok := catpb.PolicyCommand_name[int32(p.Command)]; !ok || p.Command == catpb.PolicyCommand_POLICYCOMMAND_UNUSED { + return errors.AssertionFailedf( + "policy %q has an unknown policy command %v", p.Name, p.Command) + } + return desc.validatePolicyRoles(p) +} + +// validatePolicyRoles will validate the roles that are in one policy. +func (desc *wrapper) validatePolicyRoles(p *descpb.PolicyDescriptor) error { + if len(p.RoleNames) == 0 { + return errors.AssertionFailedf( + "policy %q has no roles defined", p.Name) + } + rolesInUse := make(map[string]struct{}, len(p.RoleNames)) + for i, roleName := range p.RoleNames { + if _, found := rolesInUse[roleName]; found { return errors.AssertionFailedf( - "policy %q has an unknown policy type %v", p.Name, p.Type) + "policy %q contains duplicate role name %q", p.Name, roleName) } - if _, ok := catpb.PolicyCommand_name[int32(p.Command)]; !ok || p.Command == catpb.PolicyCommand_POLICYCOMMAND_UNUSED { + rolesInUse[roleName] = struct{}{} + // The public role, if included, must always be the first entry in the + // role names slice. + if roleName == username.PublicRole && i > 0 { return errors.AssertionFailedf( - "policy %q has an unknown policy command %v", p.Name, p.Command) + "the public role must be the first role defined in policy %q", p.Name) } } return nil diff --git a/pkg/sql/catalog/tabledesc/validate_test.go b/pkg/sql/catalog/tabledesc/validate_test.go index dda790034dbb..cd4df4658dce 100644 --- a/pkg/sql/catalog/tabledesc/validate_test.go +++ b/pkg/sql/catalog/tabledesc/validate_test.go @@ -3107,16 +3107,18 @@ func TestValidateTableDesc(t *testing.T) { desc.NextPolicyID = 3 desc.Policies = []descpb.PolicyDescriptor{ { - ID: 1, - Name: "pol", - Type: catpb.PolicyType_PERMISSIVE, - Command: catpb.PolicyCommand_ALL, + ID: 1, + Name: "pol", + Type: catpb.PolicyType_PERMISSIVE, + Command: catpb.PolicyCommand_ALL, + RoleNames: []string{"u1"}, }, { - ID: 2, - Name: "pol", - Type: catpb.PolicyType_RESTRICTIVE, - Command: catpb.PolicyCommand_INSERT, + ID: 2, + Name: "pol", + Type: catpb.PolicyType_RESTRICTIVE, + Command: catpb.PolicyCommand_INSERT, + RoleNames: []string{"u1"}, }, } }), @@ -3126,16 +3128,18 @@ func TestValidateTableDesc(t *testing.T) { desc.NextPolicyID = 11 desc.Policies = []descpb.PolicyDescriptor{ { - ID: 10, - Name: "pol_old", - Type: catpb.PolicyType_RESTRICTIVE, - Command: catpb.PolicyCommand_UPDATE, + ID: 10, + Name: "pol_old", + Type: catpb.PolicyType_RESTRICTIVE, + Command: catpb.PolicyCommand_UPDATE, + RoleNames: []string{"u1"}, }, { - ID: 10, - Name: "pol_new", - Type: catpb.PolicyType_PERMISSIVE, - Command: catpb.PolicyCommand_DELETE, + ID: 10, + Name: "pol_new", + Type: catpb.PolicyType_PERMISSIVE, + Command: catpb.PolicyCommand_DELETE, + RoleNames: []string{"u1"}, }, } }), @@ -3179,6 +3183,48 @@ func TestValidateTableDesc(t *testing.T) { } }), }, + {err: `policy "pol" has no roles defined`, + desc: ModifyDescriptor(func(desc *descpb.TableDescriptor) { + desc.NextPolicyID = 2 + desc.Policies = []descpb.PolicyDescriptor{ + { + ID: 1, + Name: "pol", + Type: catpb.PolicyType_PERMISSIVE, + Command: catpb.PolicyCommand_DELETE, + RoleNames: nil, + }, + } + }), + }, + {err: `policy "pol" contains duplicate role name "u1"`, + desc: ModifyDescriptor(func(desc *descpb.TableDescriptor) { + desc.NextPolicyID = 2 + desc.Policies = []descpb.PolicyDescriptor{ + { + ID: 1, + Name: "pol", + Type: catpb.PolicyType_RESTRICTIVE, + Command: catpb.PolicyCommand_ALL, + RoleNames: []string{"u1", "u2", "u11", "u1"}, + }, + } + }), + }, + {err: `the public role must be the first role defined in policy "pol"`, + desc: ModifyDescriptor(func(desc *descpb.TableDescriptor) { + desc.NextPolicyID = 2 + desc.Policies = []descpb.PolicyDescriptor{ + { + ID: 1, + Name: "pol", + Type: catpb.PolicyType_PERMISSIVE, + Command: catpb.PolicyCommand_INSERT, + RoleNames: []string{"u1", "public"}, + }, + } + }), + }, } for i, d := range testData { diff --git a/pkg/sql/catalog/validate.go b/pkg/sql/catalog/validate.go index 11eac403273f..b0b7b6837934 100644 --- a/pkg/sql/catalog/validate.go +++ b/pkg/sql/catalog/validate.go @@ -222,6 +222,21 @@ func ValidateRolesInDescriptor( priv.User()) } } + // If a table descriptor, validate the roles that are stored in the policies. + if tbDesc, isTable := descriptor.(TableDescriptor); isTable { + for _, p := range tbDesc.GetPolicies() { + for _, r := range p.RoleNames { + exists, err := RoleExists(username.MakeSQLUsernameFromPreNormalizedString(r)) + if err != nil { + return err + } + if !exists { + return errors.AssertionFailedf("policy %q on table %q has a role %q that doesn't exist", + p.Name, tbDesc.GetName(), r) + } + } + } + } return nil } diff --git a/pkg/sql/drop_role.go b/pkg/sql/drop_role.go index a446bee59685..fb80a2d06350 100644 --- a/pkg/sql/drop_role.go +++ b/pkg/sql/drop_role.go @@ -203,6 +203,20 @@ func (n *DropRoleNode) startExec(params runParams) error { break } } + // Check that any of the roles we are dropping aren't referenced in any of + // the row-level security policies defined on this table. + for _, p := range tableDescriptor.GetPolicies() { + for _, rn := range p.RoleNames { + roleName := username.MakeSQLUsernameFromPreNormalizedString(rn) + if _, found := userNames[roleName]; found { + return errors.WithDetailf( + pgerror.Newf(pgcode.DependentObjectsStillExist, + "role %q cannot be dropped because some objects depend on it", + roleName), + "target of policy %q on table %q", p.Name, tableDescriptor.GetName()) + } + } + } } for _, schemaDesc := range lCtx.schemaDescs { if !descriptorIsVisible(schemaDesc, true /* allowAdding */, false /* includeDropped */) { diff --git a/pkg/sql/logictest/testdata/logic_test/row_level_security b/pkg/sql/logictest/testdata/logic_test/row_level_security index 545e458a5f54..ac43420fa1b0 100644 --- a/pkg/sql/logictest/testdata/logic_test/row_level_security +++ b/pkg/sql/logictest/testdata/logic_test/row_level_security @@ -1,5 +1,16 @@ # LogicTest: !local-legacy-schema-changer !local-mixed-24.3 +subtest setup + +statement ok +CREATE DATABASE db1; + +statement ok +USE db1; + +statement ok +GRANT ADMIN to testuser; + statement ok SET enable_row_level_security = on; @@ -161,4 +172,122 @@ multi_pol_tab1 CREATE TABLE public.multi_pol_tab1 ( statement ok DROP TABLE multi_pol_tab1 +subtest drop_role_is_blocked + +statement ok +CREATE TABLE drop_role_chk(); + +statement ok +CREATE USER fred; + +statement ok +CREATE USER bob; + +statement ok +CREATE POLICY p1 on drop_role_chk to fred,bob; + +statement error pq: role "bob" cannot be dropped because some objects depend on it\nDETAIL: target of policy "p1" on table "drop_role_chk" +DROP ROLE bob; + +statement error pq: role "fred" cannot be dropped because some objects depend on it\nDETAIL: target of policy "p1" on table "drop_role_chk" +DROP ROLE fred; + +statement ok +DROP POLICY p1 on drop_role_chk; + +statement ok +DROP ROLE bob,fred; + +statement ok +DROP TABLE drop_role_chk; + +subtest role_doesnt_exist + +statement ok +CREATE TABLE role_exist_chk(); + +statement error pq: role/user "zeke" does not exist +CREATE POLICY p1 on role_exist_chk to zeke; + +statement ok +CREATE USER zeke; + +statement ok +CREATE POLICY p1 on role_exist_chk to zeke; + +statement ok +DROP TABLE role_exist_chk; + +statement ok +DROP ROLE zeke; + +subtest current_session_and_user + +statement ok +CREATE TABLE target(); + +statement ok +CREATE USER john; + +statement ok +GRANT ALL ON db1.* to testuser; + +statement ok +GRANT ALL ON db1.* to john; + +statement ok +GRANT SYSTEM MODIFYCLUSTERSETTING TO testuser; + +user testuser + +statement ok +SET enable_row_level_security = on; + +statement ok +USE db1; + +statement ok +SET ROLE john; + +query TT +SELECT current_user, session_user +---- +john testuser + +statement ok +CREATE POLICY pol on target TO current_user,session_user; + +query TT +SHOW CREATE TABLE target +---- +target CREATE TABLE public.target ( + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT target_pkey PRIMARY KEY (rowid ASC) + ) + +user root + +statement ok +SET enable_row_level_security = on; + +statement ok +USE db1; + +query TT +SELECT current_user, session_user +---- +root root + +statement error pq: role "john" cannot be dropped because some objects depend on it\nDETAIL: target of policy "pol" on table "target" +DROP ROLE john; + +statement error pq: role "testuser" cannot be dropped because some objects depend on it\nDETAIL: target of policy "pol" on table "target" +DROP ROLE testuser; + +statement ok +DROP TABLE target; + +statement ok +DROP ROLE john; + subtest end diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_policy.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_policy.go index 533f4e64d233..e5e15c35f7ce 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_policy.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_policy.go @@ -6,7 +6,10 @@ package scbuildstmt import ( + "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/decodeusername" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/privilege" @@ -50,6 +53,7 @@ func CreatePolicy(b BuildCtx, n *tree.CreatePolicy) { PolicyID: policyID, Name: string(n.PolicyName), }) + addRoleElements(b, n, tbl.TableID, policyID) } // convertPolicyType will convert from a tree.PolicyType to a catpb.PolicyType @@ -81,3 +85,37 @@ func convertPolicyCommand(in tree.PolicyCommand) catpb.PolicyCommand { panic(errors.AssertionFailedf("cannot convert tree.PolicyCommand: %v", in)) } } + +// addRoleElements will add an element for each role defined in the policy. +func addRoleElements( + b BuildCtx, n *tree.CreatePolicy, tableID descpb.ID, policyID descpb.PolicyID, +) { + // If there were no roles provided, then we will default to adding the public + // role so that it applies to everyone. + if len(n.Roles) == 0 { + addRoleElement(b, tableID, policyID, username.PublicRoleName()) + return + } + for _, role := range n.Roles { + authRole, err := decodeusername.FromRoleSpec(b.SessionData(), username.PurposeValidation, role) + if err != nil { + panic(err) + } + addRoleElement(b, tableID, policyID, authRole) + } +} + +// addRoleElement will add a single role element for the given role name +func addRoleElement( + b BuildCtx, tableID descpb.ID, policyID descpb.PolicyID, username username.SQLUsername, +) { + err := b.CheckRoleExists(b, username) + if err != nil { + panic(err) + } + b.Add(&scpb.PolicyRole{ + TableID: tableID, + PolicyID: policyID, + RoleName: username.Normalized(), + }) +} diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_policy.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_policy.go index a8432ae9ce58..95cbd1e931a7 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_policy.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_policy.go @@ -39,7 +39,7 @@ func DropPolicy(b BuildCtx, n *tree.DropPolicy) { } policyElems.ForEach(func(_ scpb.Status, _ scpb.TargetStatus, e scpb.Element) { switch e.(type) { - case *scpb.Policy, *scpb.PolicyName: + case *scpb.Policy, *scpb.PolicyName, *scpb.PolicyRole: b.Drop(e) } }) diff --git a/pkg/sql/schemachanger/scbuild/testdata/create_policy b/pkg/sql/schemachanger/scbuild/testdata/create_policy index de2c683ae269..4b5fb86a3548 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/create_policy +++ b/pkg/sql/schemachanger/scbuild/testdata/create_policy @@ -14,6 +14,8 @@ CREATE POLICY "first policy" on defaultdb.foo AS PERMISSIVE FOR SELECT TO fred U {command: 2, policyId: 1, tableId: 104, type: 1} - [[PolicyName:{DescID: 104, Name: first policy, PolicyID: 1}, PUBLIC], ABSENT] {name: first policy, policyId: 1, tableId: 104} +- [[PolicyRole:{DescID: 104, Name: fred, PolicyID: 1}, PUBLIC], ABSENT] + {policyId: 1, roleName: fred, tableId: 104} build CREATE POLICY "second policy" on defaultdb.foo AS RESTRICTIVE FOR INSERT USING (false); @@ -26,6 +28,8 @@ CREATE POLICY "second policy" on defaultdb.foo AS RESTRICTIVE FOR INSERT USING ( {command: 3, policyId: 1, tableId: 104, type: 2} - [[PolicyName:{DescID: 104, Name: second policy, PolicyID: 1}, PUBLIC], ABSENT] {name: second policy, policyId: 1, tableId: 104} +- [[PolicyRole:{DescID: 104, Name: public, PolicyID: 1}, PUBLIC], ABSENT] + {policyId: 1, roleName: public, tableId: 104} build CREATE POLICY "third policy" on defaultdb.foo FOR DELETE TO CURRENT_USER,fred WITH CHECK (i < 0); @@ -38,9 +42,13 @@ CREATE POLICY "third policy" on defaultdb.foo FOR DELETE TO CURRENT_USER,fred WI {command: 5, policyId: 1, tableId: 104, type: 1} - [[PolicyName:{DescID: 104, Name: third policy, PolicyID: 1}, PUBLIC], ABSENT] {name: third policy, policyId: 1, tableId: 104} +- [[PolicyRole:{DescID: 104, Name: root, PolicyID: 1}, PUBLIC], ABSENT] + {policyId: 1, roleName: root, tableId: 104} +- [[PolicyRole:{DescID: 104, Name: fred, PolicyID: 1}, PUBLIC], ABSENT] + {policyId: 1, roleName: fred, tableId: 104} build -CREATE POLICY "fourth policy" on defaultdb.foo AS PERMISSIVE TO PUBLIC,CURRENT_SESSION; +CREATE POLICY "fourth policy" on defaultdb.foo AS PERMISSIVE TO PUBLIC,SESSION_USER; ---- - [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] {indexId: 1, tableId: 104} @@ -50,3 +58,7 @@ CREATE POLICY "fourth policy" on defaultdb.foo AS PERMISSIVE TO PUBLIC,CURRENT_S {command: 1, policyId: 1, tableId: 104, type: 1} - [[PolicyName:{DescID: 104, Name: fourth policy, PolicyID: 1}, PUBLIC], ABSENT] {name: fourth policy, policyId: 1, tableId: 104} +- [[PolicyRole:{DescID: 104, Name: public, PolicyID: 1}, PUBLIC], ABSENT] + {policyId: 1, roleName: public, tableId: 104} +- [[PolicyRole:{DescID: 104, Name: root, PolicyID: 1}, PUBLIC], ABSENT] + {policyId: 1, roleName: root, tableId: 104} diff --git a/pkg/sql/schemachanger/scbuild/testdata/drop_policy b/pkg/sql/schemachanger/scbuild/testdata/drop_policy index 812dd312955a..daf650fae280 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/drop_policy +++ b/pkg/sql/schemachanger/scbuild/testdata/drop_policy @@ -5,7 +5,7 @@ CREATE USER fred; CREATE POLICY "first policy" on defaultdb.foo AS PERMISSIVE FOR SELECT TO fred USING (i > 0) WITH CHECK (i % 2 = 0); CREATE POLICY "second policy" on defaultdb.foo AS RESTRICTIVE FOR INSERT USING (false); CREATE POLICY "third policy" on defaultdb.foo FOR DELETE TO CURRENT_USER,fred WITH CHECK (i < 0); -CREATE POLICY "fourth policy" on defaultdb.foo AS PERMISSIVE TO PUBLIC,CURRENT_SESSION; +CREATE POLICY "fourth policy" on defaultdb.foo AS PERMISSIVE TO PUBLIC,SESSION_USER; ---- build @@ -17,6 +17,8 @@ DROP POLICY "first policy" on defaultdb.foo; {command: 2, policyId: 1, tableId: 104, type: 1} - [[PolicyName:{DescID: 104, Name: first policy, PolicyID: 1}, ABSENT], PUBLIC] {name: first policy, policyId: 1, tableId: 104} +- [[PolicyRole:{DescID: 104, Name: fred, PolicyID: 1}, ABSENT], PUBLIC] + {policyId: 1, roleName: fred, tableId: 104} - [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] {databaseId: 100, tableId: 104} @@ -29,6 +31,8 @@ DROP POLICY "second policy" on defaultdb.foo CASCADE; {command: 3, policyId: 2, tableId: 104, type: 2} - [[PolicyName:{DescID: 104, Name: second policy, PolicyID: 2}, ABSENT], PUBLIC] {name: second policy, policyId: 2, tableId: 104} +- [[PolicyRole:{DescID: 104, Name: public, PolicyID: 2}, ABSENT], PUBLIC] + {policyId: 2, roleName: public, tableId: 104} - [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] {databaseId: 100, tableId: 104} @@ -41,6 +45,10 @@ DROP POLICY "third policy" on defaultdb.foo RESTRICT; {command: 5, policyId: 3, tableId: 104, type: 1} - [[PolicyName:{DescID: 104, Name: third policy, PolicyID: 3}, ABSENT], PUBLIC] {name: third policy, policyId: 3, tableId: 104} +- [[PolicyRole:{DescID: 104, Name: root, PolicyID: 3}, ABSENT], PUBLIC] + {policyId: 3, roleName: root, tableId: 104} +- [[PolicyRole:{DescID: 104, Name: fred, PolicyID: 3}, ABSENT], PUBLIC] + {policyId: 3, roleName: fred, tableId: 104} - [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] {databaseId: 100, tableId: 104} @@ -53,5 +61,9 @@ DROP POLICY "fourth policy" on defaultdb.foo; {command: 1, policyId: 4, tableId: 104, type: 1} - [[PolicyName:{DescID: 104, Name: fourth policy, PolicyID: 4}, ABSENT], PUBLIC] {name: fourth policy, policyId: 4, tableId: 104} +- [[PolicyRole:{DescID: 104, Name: public, PolicyID: 4}, ABSENT], PUBLIC] + {policyId: 4, roleName: public, tableId: 104} +- [[PolicyRole:{DescID: 104, Name: root, PolicyID: 4}, ABSENT], PUBLIC] + {policyId: 4, roleName: root, tableId: 104} - [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] {databaseId: 100, tableId: 104} diff --git a/pkg/sql/schemachanger/scdecomp/decomp.go b/pkg/sql/schemachanger/scdecomp/decomp.go index a9f626e8a7d5..8272009abe4e 100644 --- a/pkg/sql/schemachanger/scdecomp/decomp.go +++ b/pkg/sql/schemachanger/scdecomp/decomp.go @@ -886,6 +886,13 @@ func (w *walkCtx) walkPolicy(tbl catalog.TableDescriptor, p *descpb.PolicyDescri PolicyID: p.ID, Name: p.Name, }) + for _, role := range p.RoleNames { + w.ev(scpb.Status_PUBLIC, &scpb.PolicyRole{ + TableID: tbl.GetID(), + PolicyID: p.ID, + RoleName: role, + }) + } } func (w *walkCtx) walkForeignKeyConstraint( diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/policy.go b/pkg/sql/schemachanger/scexec/scmutationexec/policy.go index 339a78292dc6..de6cfbe08695 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/policy.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/policy.go @@ -8,6 +8,7 @@ package scmutationexec import ( "context" + "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" "github.com/cockroachdb/errors" @@ -57,3 +58,45 @@ func (i *immediateVisitor) RemovePolicy(ctx context.Context, op scop.RemovePolic } return nil } + +func (i *immediateVisitor) AddPolicyRole(ctx context.Context, op scop.AddPolicyRole) error { + policy, err := i.checkOutPolicy(ctx, op.Role.TableID, op.Role.PolicyID) + if err != nil { + return err + } + // Verify that the role doesn't already exist in the policy. + for _, r := range policy.RoleNames { + if r == op.Role.RoleName { + return errors.AssertionFailedf( + "role %q already exists in policy %d on table %d", + op.Role.RoleName, op.Role.PolicyID, op.Role.TableID) + } + } + // For convenience, if the public role is part of the policy, it will always be + // positioned as the first entry in the slice. This simplifies query runtime checks + // by allowing a quick determination of whether the policy applies to all roles + // (i.e., the public role). If the role is not public, it is appended to the end + // of the slice. + if op.Role.RoleName == username.PublicRole { + policy.RoleNames = append([]string{op.Role.RoleName}, policy.RoleNames...) + } else { + policy.RoleNames = append(policy.RoleNames, op.Role.RoleName) + } + return nil +} + +func (i *immediateVisitor) RemovePolicyRole(ctx context.Context, op scop.RemovePolicyRole) error { + policy, err := i.checkOutPolicy(ctx, op.Role.TableID, op.Role.PolicyID) + if err != nil { + return err + } + for inx, r := range policy.RoleNames { + if r == op.Role.RoleName { + policy.RoleNames = append(policy.RoleNames[:inx], policy.RoleNames[inx+1:]...) + return nil + } + } + return errors.AssertionFailedf( + "role %q does not exist in policy %d on table %d", + op.Role.RoleName, op.Role.PolicyID, op.Role.TableID) +} diff --git a/pkg/sql/schemachanger/scop/immediate_mutation.go b/pkg/sql/schemachanger/scop/immediate_mutation.go index ba5312d1958e..ccdbb5535e49 100644 --- a/pkg/sql/schemachanger/scop/immediate_mutation.go +++ b/pkg/sql/schemachanger/scop/immediate_mutation.go @@ -608,6 +608,18 @@ type SetPolicyName struct { Name string } +// AddPolicyRole adds a new role to a policy. +type AddPolicyRole struct { + immediateMutationOp + Role scpb.PolicyRole +} + +// RemovePolicyRole removes an existing role from a policy. +type RemovePolicyRole struct { + immediateMutationOp + Role scpb.PolicyRole +} + // UpdateTableBackReferencesInTypes updates back references to a table // in the specified types. type UpdateTableBackReferencesInTypes struct { diff --git a/pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go b/pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go index 3e2dc02586a5..9cdf5cadd90a 100644 --- a/pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go +++ b/pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go @@ -94,6 +94,8 @@ type ImmediateMutationVisitor interface { AddPolicy(context.Context, AddPolicy) error RemovePolicy(context.Context, RemovePolicy) error SetPolicyName(context.Context, SetPolicyName) error + AddPolicyRole(context.Context, AddPolicyRole) error + RemovePolicyRole(context.Context, RemovePolicyRole) error UpdateTableBackReferencesInTypes(context.Context, UpdateTableBackReferencesInTypes) error UpdateTypeBackReferencesInTypes(context.Context, UpdateTypeBackReferencesInTypes) error RemoveBackReferenceInTypes(context.Context, RemoveBackReferenceInTypes) error @@ -544,6 +546,16 @@ func (op SetPolicyName) Visit(ctx context.Context, v ImmediateMutationVisitor) e return v.SetPolicyName(ctx, op) } +// Visit is part of the ImmediateMutationOp interface. +func (op AddPolicyRole) Visit(ctx context.Context, v ImmediateMutationVisitor) error { + return v.AddPolicyRole(ctx, op) +} + +// Visit is part of the ImmediateMutationOp interface. +func (op RemovePolicyRole) Visit(ctx context.Context, v ImmediateMutationVisitor) error { + return v.RemovePolicyRole(ctx, op) +} + // Visit is part of the ImmediateMutationOp interface. func (op UpdateTableBackReferencesInTypes) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.UpdateTableBackReferencesInTypes(ctx, op) diff --git a/pkg/sql/schemachanger/scpb/elements.proto b/pkg/sql/schemachanger/scpb/elements.proto index bc41eee2e607..5c41aef30d05 100644 --- a/pkg/sql/schemachanger/scpb/elements.proto +++ b/pkg/sql/schemachanger/scpb/elements.proto @@ -169,6 +169,7 @@ message ElementProto { // Policy elements. PolicyName policy_name = 240 [(gogoproto.moretags) = "parent:\"Policy\""]; + PolicyRole policy_role = 241 [(gogoproto.moretags) = "parent:\"Policy\""]; // Next element group start id: 250 } @@ -379,6 +380,13 @@ message PolicyName { string name = 3; } +// PolicyRole is the role that is attached to a specific policy. +message PolicyRole { + uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; + uint32 policy_id = 2 [(gogoproto.customname) = "PolicyID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.PolicyID"]; + string role_name = 3; +} + message Sequence { uint32 sequence_id = 1 [(gogoproto.customname) = "SequenceID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; int64 restart_with = 2; diff --git a/pkg/sql/schemachanger/scpb/elements_generated.go b/pkg/sql/schemachanger/scpb/elements_generated.go index 550abfc87b72..2dd08b73d0f1 100644 --- a/pkg/sql/schemachanger/scpb/elements_generated.go +++ b/pkg/sql/schemachanger/scpb/elements_generated.go @@ -1749,6 +1749,43 @@ func (c *ElementCollection[E]) FilterPolicyName() *ElementCollection[*PolicyName return (*ElementCollection[*PolicyName])(ret) } +func (e PolicyRole) element() {} + +// Element implements ElementGetter. +func (e * ElementProto_PolicyRole) Element() Element { + return e.PolicyRole +} + +// ForEachPolicyRole iterates over elements of type PolicyRole. +// Deprecated +func ForEachPolicyRole( + c *ElementCollection[Element], fn func(current Status, target TargetStatus, e *PolicyRole), +) { + c.FilterPolicyRole().ForEach(fn) +} + +// FindPolicyRole finds the first element of type PolicyRole. +// Deprecated +func FindPolicyRole( + c *ElementCollection[Element], +) (current Status, target TargetStatus, element *PolicyRole) { + if tc := c.FilterPolicyRole(); !tc.IsEmpty() { + var e Element + current, target, e = tc.Get(0) + element = e.(*PolicyRole) + } + return current, target, element +} + +// PolicyRoleElements filters elements of type PolicyRole. +func (c *ElementCollection[E]) FilterPolicyRole() *ElementCollection[*PolicyRole] { + ret := c.genericFilter(func(_ Status, _ TargetStatus, e Element) bool { + _, ok := e.(*PolicyRole) + return ok + }) + return (*ElementCollection[*PolicyRole])(ret) +} + func (e PrimaryIndex) element() {} // Element implements ElementGetter. @@ -3180,6 +3217,8 @@ func (e* ElementProto) SetElement(element Element) { e.ElementOneOf = &ElementProto_Policy{ Policy: t} case *PolicyName: e.ElementOneOf = &ElementProto_PolicyName{ PolicyName: t} + case *PolicyRole: + e.ElementOneOf = &ElementProto_PolicyRole{ PolicyRole: t} case *PrimaryIndex: e.ElementOneOf = &ElementProto_PrimaryIndex{ PrimaryIndex: t} case *RowLevelTTL: @@ -3305,6 +3344,7 @@ func GetElementOneOfProtos() []interface{} { ((*ElementProto_PartitionZoneConfig)(nil)), ((*ElementProto_Policy)(nil)), ((*ElementProto_PolicyName)(nil)), + ((*ElementProto_PolicyRole)(nil)), ((*ElementProto_PrimaryIndex)(nil)), ((*ElementProto_RowLevelTTL)(nil)), ((*ElementProto_Schema)(nil)), @@ -3395,6 +3435,7 @@ func GetElementTypes() []interface{} { ((*PartitionZoneConfig)(nil)), ((*Policy)(nil)), ((*PolicyName)(nil)), + ((*PolicyRole)(nil)), ((*PrimaryIndex)(nil)), ((*RowLevelTTL)(nil)), ((*Schema)(nil)), diff --git a/pkg/sql/schemachanger/scpb/uml/table.puml b/pkg/sql/schemachanger/scpb/uml/table.puml index 12ff2a075186..9b1f4eddaa56 100644 --- a/pkg/sql/schemachanger/scpb/uml/table.puml +++ b/pkg/sql/schemachanger/scpb/uml/table.puml @@ -311,6 +311,12 @@ PolicyName : TableID PolicyName : PolicyID PolicyName : Name +object PolicyRole + +PolicyRole : TableID +PolicyRole : PolicyID +PolicyRole : RoleName + object PrimaryIndex PrimaryIndex : Index @@ -591,6 +597,7 @@ EnumType <|-- Owner TablePartitioning <|-- PartitionZoneConfig Table <|-- Policy Policy <|-- PolicyName +Policy <|-- PolicyRole Table <|-- PrimaryIndex View <|-- PrimaryIndex Table <|-- RowLevelTTL diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel b/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel index 86747623a8a5..84655691a25c 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel +++ b/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel @@ -52,6 +52,7 @@ go_library( "opgen_partition_zone_config.go", "opgen_policy.go", "opgen_policy_name.go", + "opgen_policy_role.go", "opgen_primary_index.go", "opgen_row_level_ttl.go", "opgen_schema.go", diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_policy_role.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_policy_role.go new file mode 100644 index 000000000000..c75b7fe641b2 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_policy_role.go @@ -0,0 +1,33 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package opgen + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" +) + +func init() { + opRegistry.register((*scpb.PolicyRole)(nil), + toPublic( + scpb.Status_ABSENT, + to(scpb.Status_PUBLIC, + emit(func(this *scpb.PolicyRole) *scop.AddPolicyRole { + return &scop.AddPolicyRole{Role: *protoutil.Clone(this).(*scpb.PolicyRole)} + }), + ), + ), + toAbsent( + scpb.Status_PUBLIC, + to(scpb.Status_ABSENT, + emit(func(this *scpb.PolicyRole) *scop.RemovePolicyRole { + return &scop.RemovePolicyRole{Role: *protoutil.Clone(this).(*scpb.PolicyRole)} + }), + ), + ), + ) +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go index 5b14e4a285bb..6e689d2fddaf 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go @@ -318,7 +318,7 @@ func isTriggerDependent(e scpb.Element) bool { func isPolicyDependent(e scpb.Element) bool { switch e.(type) { - case *scpb.PolicyName: + case *scpb.PolicyName, *scpb.PolicyRole: return true } return false diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules index 59e4426c162d..2d103c9426fb 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules @@ -2998,7 +2998,7 @@ deprules to: dependent-Node query: - $policy[Type] = '*scpb.Policy' - - $dependent[Type] = '*scpb.PolicyName' + - $dependent[Type] IN ['*scpb.PolicyName', '*scpb.PolicyRole'] - joinOnPolicyID($policy, $dependent, $table-id, $policy-id) - ToPublicOrTransient($policy-Target, $dependent-Target) - $policy-Node[CurrentStatus] = PUBLIC @@ -3010,7 +3010,7 @@ deprules kind: Precedence to: relation-Node query: - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - $relation[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - joinOnDescID($dependent, $relation, $relation-id) - ToPublicOrTransient($dependent-Target, $relation-Target) @@ -3185,7 +3185,7 @@ deprules kind: SameStagePrecedence to: policy-Node query: - - $dependent[Type] = '*scpb.PolicyName' + - $dependent[Type] IN ['*scpb.PolicyName', '*scpb.PolicyRole'] - $policy[Type] = '*scpb.Policy' - joinOnPolicyID($dependent, $policy, $table-id, $policy-id) - toAbsent($dependent-Target, $policy-Target) @@ -3198,7 +3198,7 @@ deprules kind: SameStagePrecedence to: policy-Node query: - - $dependent[Type] = '*scpb.PolicyName' + - $dependent[Type] IN ['*scpb.PolicyName', '*scpb.PolicyRole'] - $policy[Type] = '*scpb.Policy' - joinOnPolicyID($dependent, $policy, $table-id, $policy-id) - transient($dependent-Target, $policy-Target) @@ -3211,7 +3211,7 @@ deprules kind: SameStagePrecedence to: policy-Node query: - - $dependent[Type] = '*scpb.PolicyName' + - $dependent[Type] IN ['*scpb.PolicyName', '*scpb.PolicyRole'] - $policy[Type] = '*scpb.Policy' - joinOnPolicyID($dependent, $policy, $table-id, $policy-id) - $dependent-Target[TargetStatus] = TRANSIENT_ABSENT @@ -3225,7 +3225,7 @@ deprules kind: SameStagePrecedence to: policy-Node query: - - $dependent[Type] = '*scpb.PolicyName' + - $dependent[Type] IN ['*scpb.PolicyName', '*scpb.PolicyRole'] - $policy[Type] = '*scpb.Policy' - joinOnPolicyID($dependent, $policy, $table-id, $policy-id) - $dependent-Target[TargetStatus] = ABSENT @@ -3308,7 +3308,7 @@ deprules to: referencing-via-attr-Node query: - $referenced-descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $referencing-via-attr[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.RowLevelTTL', '*scpb.SchemaComment', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $referencing-via-attr[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.RowLevelTTL', '*scpb.SchemaComment', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinReferencedDescID($referencing-via-attr, $referenced-descriptor, $desc-id) - toAbsent($referenced-descriptor-Target, $referencing-via-attr-Target) - $referenced-descriptor-Node[CurrentStatus] = DROPPED @@ -3364,7 +3364,7 @@ deprules to: dependent-Node query: - $descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $dependent[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinOnDescID($descriptor, $dependent, $desc-id) - toAbsent($descriptor-Target, $dependent-Target) - $descriptor-Node[CurrentStatus] = DROPPED @@ -3403,7 +3403,7 @@ deprules to: dependent-Node query: - $relation[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinOnDescID($relation, $dependent, $relation-id) - ToPublicOrTransient($relation-Target, $dependent-Target) - $relation-Node[CurrentStatus] = DESCRIPTOR_ADDED @@ -3884,7 +3884,7 @@ deprules kind: Precedence to: descriptor-Node query: - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - $descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - joinOnDescID($dependent, $descriptor, $desc-id) - toAbsent($dependent-Target, $descriptor-Target) @@ -7654,7 +7654,7 @@ deprules to: dependent-Node query: - $policy[Type] = '*scpb.Policy' - - $dependent[Type] = '*scpb.PolicyName' + - $dependent[Type] IN ['*scpb.PolicyName', '*scpb.PolicyRole'] - joinOnPolicyID($policy, $dependent, $table-id, $policy-id) - ToPublicOrTransient($policy-Target, $dependent-Target) - $policy-Node[CurrentStatus] = PUBLIC @@ -7666,7 +7666,7 @@ deprules kind: Precedence to: relation-Node query: - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - $relation[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - joinOnDescID($dependent, $relation, $relation-id) - ToPublicOrTransient($dependent-Target, $relation-Target) @@ -7841,7 +7841,7 @@ deprules kind: SameStagePrecedence to: policy-Node query: - - $dependent[Type] = '*scpb.PolicyName' + - $dependent[Type] IN ['*scpb.PolicyName', '*scpb.PolicyRole'] - $policy[Type] = '*scpb.Policy' - joinOnPolicyID($dependent, $policy, $table-id, $policy-id) - toAbsent($dependent-Target, $policy-Target) @@ -7854,7 +7854,7 @@ deprules kind: SameStagePrecedence to: policy-Node query: - - $dependent[Type] = '*scpb.PolicyName' + - $dependent[Type] IN ['*scpb.PolicyName', '*scpb.PolicyRole'] - $policy[Type] = '*scpb.Policy' - joinOnPolicyID($dependent, $policy, $table-id, $policy-id) - transient($dependent-Target, $policy-Target) @@ -7867,7 +7867,7 @@ deprules kind: SameStagePrecedence to: policy-Node query: - - $dependent[Type] = '*scpb.PolicyName' + - $dependent[Type] IN ['*scpb.PolicyName', '*scpb.PolicyRole'] - $policy[Type] = '*scpb.Policy' - joinOnPolicyID($dependent, $policy, $table-id, $policy-id) - $dependent-Target[TargetStatus] = TRANSIENT_ABSENT @@ -7881,7 +7881,7 @@ deprules kind: SameStagePrecedence to: policy-Node query: - - $dependent[Type] = '*scpb.PolicyName' + - $dependent[Type] IN ['*scpb.PolicyName', '*scpb.PolicyRole'] - $policy[Type] = '*scpb.Policy' - joinOnPolicyID($dependent, $policy, $table-id, $policy-id) - $dependent-Target[TargetStatus] = ABSENT @@ -7964,7 +7964,7 @@ deprules to: referencing-via-attr-Node query: - $referenced-descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $referencing-via-attr[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.RowLevelTTL', '*scpb.SchemaComment', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $referencing-via-attr[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.RowLevelTTL', '*scpb.SchemaComment', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinReferencedDescID($referencing-via-attr, $referenced-descriptor, $desc-id) - toAbsent($referenced-descriptor-Target, $referencing-via-attr-Target) - $referenced-descriptor-Node[CurrentStatus] = DROPPED @@ -8020,7 +8020,7 @@ deprules to: dependent-Node query: - $descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $dependent[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinOnDescID($descriptor, $dependent, $desc-id) - toAbsent($descriptor-Target, $dependent-Target) - $descriptor-Node[CurrentStatus] = DROPPED @@ -8059,7 +8059,7 @@ deprules to: dependent-Node query: - $relation[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinOnDescID($relation, $dependent, $relation-id) - ToPublicOrTransient($relation-Target, $dependent-Target) - $relation-Node[CurrentStatus] = DESCRIPTOR_ADDED @@ -8540,7 +8540,7 @@ deprules kind: Precedence to: descriptor-Node query: - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PolicyRole', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - $descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - joinOnDescID($dependent, $descriptor, $desc-id) - toAbsent($dependent-Target, $descriptor-Target) diff --git a/pkg/sql/schemachanger/screl/attr.go b/pkg/sql/schemachanger/screl/attr.go index d64894d43343..267569e25f6b 100644 --- a/pkg/sql/schemachanger/screl/attr.go +++ b/pkg/sql/schemachanger/screl/attr.go @@ -373,6 +373,11 @@ var elementSchemaOptions = []rel.SchemaOption{ rel.EntityAttr(PolicyID, "PolicyID"), rel.EntityAttr(Name, "Name"), ), + rel.EntityMapping(t((*scpb.PolicyRole)(nil)), + rel.EntityAttr(DescID, "TableID"), + rel.EntityAttr(PolicyID, "PolicyID"), + rel.EntityAttr(Name, "RoleName"), + ), // Common elements. rel.EntityMapping(t((*scpb.Namespace)(nil)), rel.EntityAttr(DescID, "DescriptorID"), diff --git a/pkg/sql/schemachanger/screl/scalars.go b/pkg/sql/schemachanger/screl/scalars.go index eb7686e0024e..77f9fa84da02 100644 --- a/pkg/sql/schemachanger/screl/scalars.go +++ b/pkg/sql/schemachanger/screl/scalars.go @@ -130,7 +130,7 @@ func VersionSupportsElementUse(el scpb.Element, version clusterversion.ClusterVe *scpb.TriggerWhen, *scpb.TriggerFunctionCall, *scpb.TriggerDeps: // These elements need v24.3 so they can be used without checking any version gates. return true - case *scpb.NamedRangeZoneConfig, *scpb.Policy, *scpb.PolicyName: + case *scpb.NamedRangeZoneConfig, *scpb.Policy, *scpb.PolicyName, *scpb.PolicyRole: return version.IsActive(clusterversion.V25_1) default: panic(errors.AssertionFailedf("unknown element %T", el)) diff --git a/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.definition b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.definition index fbf50f7b20df..bd9caaf3a116 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.definition +++ b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.definition @@ -3,5 +3,5 @@ CREATE TABLE t1 (tenant_id uuid, c1 int); ---- test -CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR SELECT TO PUBLIC USING (tenant_id = '01538898-f55c-44db-a306-89078e2c430e'); +CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR SELECT USING (tenant_id = '01538898-f55c-44db-a306-89078e2c430e'); ---- diff --git a/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain index 68851f9d4fe2..7a4f51d5a8ef 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain @@ -2,28 +2,33 @@ CREATE TABLE t1 (tenant_id uuid, c1 int); /* test */ -EXPLAIN (DDL) CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR SELECT TO PUBLIC USING (tenant_id = '01538898-f55c-44db-a306-89078e2c430e'); +EXPLAIN (DDL) CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR SELECT USING (tenant_id = '01538898-f55c-44db-a306-89078e2c430e'); ---- -Schema change plan for CREATE POLICY ‹"policy 1"› ON ‹t1› AS PERMISSIVE FOR SELECT TO public USING (‹tenant_id› = ‹'01538898-f55c-44db-a306-89078e2c430e'›); +Schema change plan for CREATE POLICY ‹"policy 1"› ON ‹t1› AS PERMISSIVE FOR SELECT USING (‹tenant_id› = ‹'01538898-f55c-44db-a306-89078e2c430e'›); ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase - │ ├── 2 elements transitioning toward PUBLIC + │ ├── 3 elements transitioning toward PUBLIC │ │ ├── ABSENT → PUBLIC Policy:{DescID: 104 (t1), PolicyID: 1} - │ │ └── ABSENT → PUBLIC PolicyName:{DescID: 104 (t1), Name: "policy 1", PolicyID: 1} - │ └── 2 Mutation operations + │ │ ├── ABSENT → PUBLIC PolicyName:{DescID: 104 (t1), Name: "policy 1", PolicyID: 1} + │ │ └── ABSENT → PUBLIC PolicyRole:{DescID: 104 (t1), Name: "public", PolicyID: 1} + │ └── 3 Mutation operations │ ├── AddPolicy {"Policy":{"Command":2,"PolicyID":1,"TableID":104,"Type":1}} - │ └── SetPolicyName {"Name":"policy 1","PolicyID":1,"TableID":104} + │ ├── SetPolicyName {"Name":"policy 1","PolicyID":1,"TableID":104} + │ └── AddPolicyRole {"Role":{"PolicyID":1,"RoleName":"public","TableID":104}} └── PreCommitPhase ├── Stage 1 of 2 in PreCommitPhase - │ ├── 2 elements transitioning toward PUBLIC + │ ├── 3 elements transitioning toward PUBLIC │ │ ├── PUBLIC → ABSENT Policy:{DescID: 104 (t1), PolicyID: 1} - │ │ └── PUBLIC → ABSENT PolicyName:{DescID: 104 (t1), Name: "policy 1", PolicyID: 1} + │ │ ├── PUBLIC → ABSENT PolicyName:{DescID: 104 (t1), Name: "policy 1", PolicyID: 1} + │ │ └── PUBLIC → ABSENT PolicyRole:{DescID: 104 (t1), Name: "public", PolicyID: 1} │ └── 1 Mutation operation │ └── UndoAllInTxnImmediateMutationOpSideEffects └── Stage 2 of 2 in PreCommitPhase - ├── 2 elements transitioning toward PUBLIC + ├── 3 elements transitioning toward PUBLIC │ ├── ABSENT → PUBLIC Policy:{DescID: 104 (t1), PolicyID: 1} - │ └── ABSENT → PUBLIC PolicyName:{DescID: 104 (t1), Name: "policy 1", PolicyID: 1} - └── 2 Mutation operations + │ ├── ABSENT → PUBLIC PolicyName:{DescID: 104 (t1), Name: "policy 1", PolicyID: 1} + │ └── ABSENT → PUBLIC PolicyRole:{DescID: 104 (t1), Name: "public", PolicyID: 1} + └── 3 Mutation operations ├── AddPolicy {"Policy":{"Command":2,"PolicyID":1,"TableID":104,"Type":1}} - └── SetPolicyName {"Name":"policy 1","PolicyID":1,"TableID":104} + ├── SetPolicyName {"Name":"policy 1","PolicyID":1,"TableID":104} + └── AddPolicyRole {"Role":{"PolicyID":1,"RoleName":"public","TableID":104}} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain_shape b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain_shape index 2a9e01cc36d8..02a79378df87 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain_shape +++ b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain_shape @@ -2,7 +2,7 @@ CREATE TABLE t1 (tenant_id uuid, c1 int); /* test */ -EXPLAIN (DDL, SHAPE) CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR SELECT TO PUBLIC USING (tenant_id = '01538898-f55c-44db-a306-89078e2c430e'); +EXPLAIN (DDL, SHAPE) CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR SELECT USING (tenant_id = '01538898-f55c-44db-a306-89078e2c430e'); ---- -Schema change plan for CREATE POLICY ‹"policy 1"› ON ‹t1› AS PERMISSIVE FOR SELECT TO public USING (‹tenant_id› = ‹'01538898-f55c-44db-a306-89078e2c430e'›); +Schema change plan for CREATE POLICY ‹"policy 1"› ON ‹t1› AS PERMISSIVE FOR SELECT USING (‹tenant_id› = ‹'01538898-f55c-44db-a306-89078e2c430e'›); └── execute 1 system table mutations transaction diff --git a/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.side_effects b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.side_effects index 38d60dedde21..bc8e35b47043 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.side_effects +++ b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.side_effects @@ -5,13 +5,14 @@ CREATE TABLE t1 (tenant_id uuid, c1 int); +object {100 101 t1} -> 104 /* test */ -CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR SELECT TO PUBLIC USING (tenant_id = '01538898-f55c-44db-a306-89078e2c430e'); +CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR SELECT USING (tenant_id = '01538898-f55c-44db-a306-89078e2c430e'); ---- begin transaction #1 # begin StatementPhase checking for feature: CREATE POLICY increment telemetry for sql.schema.create_policy -## StatementPhase stage 1 of 1 with 2 MutationType ops +checking role/user "public" exists +## StatementPhase stage 1 of 1 with 3 MutationType ops upsert descriptor #104 ... nextIndexId: 2 @@ -22,6 +23,8 @@ upsert descriptor #104 + - command: SELECT + id: 1 + name: policy 1 + + roleNames: + + - public + type: PERMISSIVE primaryIndex: constraintId: 1 @@ -35,7 +38,7 @@ upsert descriptor #104 ## PreCommitPhase stage 1 of 2 with 1 MutationType op undo all catalog changes within txn #1 persist all catalog changes to storage -## PreCommitPhase stage 2 of 2 with 2 MutationType ops +## PreCommitPhase stage 2 of 2 with 3 MutationType ops upsert descriptor #104 ... nextIndexId: 2 @@ -46,6 +49,8 @@ upsert descriptor #104 + - command: SELECT + id: 1 + name: policy 1 + + roleNames: + + - public + type: PERMISSIVE primaryIndex: constraintId: 1 diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.definition b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.definition index c4897f39491e..8c47c5b9ccbd 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.definition +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.definition @@ -1,8 +1,10 @@ setup CREATE TABLE t1 (tenant_id uuid, c1 int); +CREATE USER user1; +CREATE USER user2; SET enable_row_level_security = true; CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR ALL TO PUBLIC USING (true); -CREATE POLICY "policy 2" on t1 AS PERMISSIVE FOR INSERT TO PUBLIC WITH CHECK (true); +CREATE POLICY "policy 2" on t1 AS PERMISSIVE FOR INSERT TO user1,user2 WITH CHECK (true); ---- test diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain index 56886d363680..7a85f36019d5 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain @@ -1,8 +1,10 @@ /* setup */ CREATE TABLE t1 (tenant_id uuid, c1 int); +CREATE USER user1; +CREATE USER user2; SET enable_row_level_security = true; CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR ALL TO PUBLIC USING (true); -CREATE POLICY "policy 2" on t1 AS PERMISSIVE FOR INSERT TO PUBLIC WITH CHECK (true); +CREATE POLICY "policy 2" on t1 AS PERMISSIVE FOR INSERT TO user1,user2 WITH CHECK (true); /* test */ EXPLAIN (DDL) DROP POLICY "policy 2" on t1; @@ -10,23 +12,33 @@ EXPLAIN (DDL) DROP POLICY "policy 2" on t1; Schema change plan for DROP POLICY ‹"policy 2"› ON ‹t1›; ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase - │ ├── 2 elements transitioning toward ABSENT + │ ├── 4 elements transitioning toward ABSENT │ │ ├── PUBLIC → ABSENT Policy:{DescID: 104 (t1), PolicyID: 2} - │ │ └── PUBLIC → ABSENT PolicyName:{DescID: 104 (t1), Name: "policy 2", PolicyID: 2} - │ └── 2 Mutation operations + │ │ ├── PUBLIC → ABSENT PolicyName:{DescID: 104 (t1), Name: "policy 2", PolicyID: 2} + │ │ ├── PUBLIC → ABSENT PolicyRole:{DescID: 104 (t1), Name: "user1", PolicyID: 2} + │ │ └── PUBLIC → ABSENT PolicyRole:{DescID: 104 (t1), Name: "user2", PolicyID: 2} + │ └── 4 Mutation operations │ ├── SetPolicyName {"Name":"crdb_internal_po...","PolicyID":2,"TableID":104} + │ ├── RemovePolicyRole {"Role":{"PolicyID":2,"RoleName":"user1","TableID":104}} + │ ├── RemovePolicyRole {"Role":{"PolicyID":2,"RoleName":"user2","TableID":104}} │ └── RemovePolicy {"Policy":{"Command":3,"PolicyID":2,"TableID":104,"Type":1}} └── PreCommitPhase ├── Stage 1 of 2 in PreCommitPhase - │ ├── 2 elements transitioning toward ABSENT + │ ├── 4 elements transitioning toward ABSENT │ │ ├── ABSENT → PUBLIC Policy:{DescID: 104 (t1), PolicyID: 2} - │ │ └── ABSENT → PUBLIC PolicyName:{DescID: 104 (t1), Name: "policy 2", PolicyID: 2} + │ │ ├── ABSENT → PUBLIC PolicyName:{DescID: 104 (t1), Name: "policy 2", PolicyID: 2} + │ │ ├── ABSENT → PUBLIC PolicyRole:{DescID: 104 (t1), Name: "user1", PolicyID: 2} + │ │ └── ABSENT → PUBLIC PolicyRole:{DescID: 104 (t1), Name: "user2", PolicyID: 2} │ └── 1 Mutation operation │ └── UndoAllInTxnImmediateMutationOpSideEffects └── Stage 2 of 2 in PreCommitPhase - ├── 2 elements transitioning toward ABSENT + ├── 4 elements transitioning toward ABSENT │ ├── PUBLIC → ABSENT Policy:{DescID: 104 (t1), PolicyID: 2} - │ └── PUBLIC → ABSENT PolicyName:{DescID: 104 (t1), Name: "policy 2", PolicyID: 2} - └── 2 Mutation operations + │ ├── PUBLIC → ABSENT PolicyName:{DescID: 104 (t1), Name: "policy 2", PolicyID: 2} + │ ├── PUBLIC → ABSENT PolicyRole:{DescID: 104 (t1), Name: "user1", PolicyID: 2} + │ └── PUBLIC → ABSENT PolicyRole:{DescID: 104 (t1), Name: "user2", PolicyID: 2} + └── 4 Mutation operations ├── SetPolicyName {"Name":"crdb_internal_po...","PolicyID":2,"TableID":104} + ├── RemovePolicyRole {"Role":{"PolicyID":2,"RoleName":"user1","TableID":104}} + ├── RemovePolicyRole {"Role":{"PolicyID":2,"RoleName":"user2","TableID":104}} └── RemovePolicy {"Policy":{"Command":3,"PolicyID":2,"TableID":104,"Type":1}} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain_shape b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain_shape index 38b62dd01d6c..f1753005ed4c 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain_shape +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain_shape @@ -1,8 +1,10 @@ /* setup */ CREATE TABLE t1 (tenant_id uuid, c1 int); +CREATE USER user1; +CREATE USER user2; SET enable_row_level_security = true; CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR ALL TO PUBLIC USING (true); -CREATE POLICY "policy 2" on t1 AS PERMISSIVE FOR INSERT TO PUBLIC WITH CHECK (true); +CREATE POLICY "policy 2" on t1 AS PERMISSIVE FOR INSERT TO user1,user2 WITH CHECK (true); /* test */ EXPLAIN (DDL, SHAPE) DROP POLICY "policy 2" on t1; diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.side_effects b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.side_effects index a3d5c38e43ad..fb6ff5f61b6d 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.side_effects +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.side_effects @@ -1,8 +1,10 @@ /* setup */ CREATE TABLE t1 (tenant_id uuid, c1 int); +CREATE USER user1; +CREATE USER user2; SET enable_row_level_security = true; CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR ALL TO PUBLIC USING (true); -CREATE POLICY "policy 2" on t1 AS PERMISSIVE FOR INSERT TO PUBLIC WITH CHECK (true); +CREATE POLICY "policy 2" on t1 AS PERMISSIVE FOR INSERT TO user1,user2 WITH CHECK (true); ---- ... +object {100 101 t1} -> 104 @@ -14,14 +16,17 @@ begin transaction #1 # begin StatementPhase checking for feature: DROP POLICY increment telemetry for sql.schema.drop_policy -## StatementPhase stage 1 of 1 with 2 MutationType ops +## StatementPhase stage 1 of 1 with 4 MutationType ops upsert descriptor #104 ... - name: policy 1 + - public type: PERMISSIVE - - command: INSERT - id: 2 - name: policy 2 + - roleNames: + - - user1 + - - user2 - type: PERMISSIVE primaryIndex: constraintId: 1 @@ -35,14 +40,17 @@ upsert descriptor #104 ## PreCommitPhase stage 1 of 2 with 1 MutationType op undo all catalog changes within txn #1 persist all catalog changes to storage -## PreCommitPhase stage 2 of 2 with 2 MutationType ops +## PreCommitPhase stage 2 of 2 with 4 MutationType ops upsert descriptor #104 ... - name: policy 1 + - public type: PERMISSIVE - - command: INSERT - id: 2 - name: policy 2 + - roleNames: + - - user1 + - - user2 - type: PERMISSIVE primaryIndex: constraintId: 1 From f47addb7ce839a93155062c087167a475f8c906d Mon Sep 17 00:00:00 2001 From: Gofastasf Date: Wed, 15 Jan 2025 06:47:28 +0000 Subject: [PATCH 4/6] refactor: replace filepath.Walk with filepath.WalkDir filepath.WalkDir, introduced in Go 1.16, is more performant as it avoids creating unnecessary intermediate os.FileInfo objects. While filepath.Walk calls os.Lstat for every file or directory to retrieve os.FileInfo, filepath.WalkDir provides a fs.DirEntry, which includes file type information without requiring a stat call. This change reduces unnecessary system calls and aligns with modern Go practices for directory traversal. Epic: None Release note (performance improvement): Improved directory traversal performance by switching from filepath.Walk to filepath.WalkDir. fix: solve the lint error by crlfmt --- pkg/acceptance/util_docker.go | 3 ++- pkg/backup/full_cluster_backup_restore_test.go | 9 +++++---- pkg/blobs/local_storage.go | 5 +++-- .../changefeedccl/sink_cloudstorage_test.go | 13 +++++++------ pkg/ccl/changefeedccl/testfeed_test.go | 7 ++++--- pkg/cli/debug_merge_logs.go | 6 +++--- pkg/cli/gen_test.go | 5 +++-- pkg/cli/userfiletable_test.go | 7 ++++--- pkg/cmd/dev/io/os/os.go | 4 ++-- pkg/cmd/roachtest/cluster.go | 4 ++-- pkg/cmd/roachtest/tests/network.go | 3 ++- pkg/cmd/roachtest/zip_util.go | 5 +++-- pkg/cmd/roachtest/zip_util_test.go | 5 +++-- pkg/cmd/whoownsit/whoownsit.go | 5 +++-- pkg/internal/codeowners/lint.go | 18 +++++++++--------- pkg/storage/pebble_iterator_test.go | 4 ++-- 16 files changed, 57 insertions(+), 46 deletions(-) diff --git a/pkg/acceptance/util_docker.go b/pkg/acceptance/util_docker.go index c6d85b7ad9db..94775c836029 100644 --- a/pkg/acceptance/util_docker.go +++ b/pkg/acceptance/util_docker.go @@ -9,6 +9,7 @@ import ( "context" "crypto/rand" "fmt" + "io/fs" "math/big" "os" "path/filepath" @@ -170,7 +171,7 @@ func testDocker( // so the files can be used inside a docker container. The caller function is responsible for cleaning up. // This function doesn't copy the original file permissions and uses 755 for directories and files. func copyRunfiles(source, destination string) error { - return filepath.WalkDir(source, func(path string, dirEntry os.DirEntry, walkErr error) error { + return filepath.WalkDir(source, func(path string, dirEntry fs.DirEntry, walkErr error) error { if walkErr != nil { return walkErr } diff --git a/pkg/backup/full_cluster_backup_restore_test.go b/pkg/backup/full_cluster_backup_restore_test.go index 9ad2c2378220..cbbfec619da4 100644 --- a/pkg/backup/full_cluster_backup_restore_test.go +++ b/pkg/backup/full_cluster_backup_restore_test.go @@ -9,6 +9,7 @@ import ( "context" gosql "database/sql" "fmt" + "io/fs" "os" "path/filepath" "strconv" @@ -667,14 +668,14 @@ func TestClusterRestoreFailCleanup(t *testing.T) { // Bugger the backup by removing the SST files. (Note this messes up all of // the backups, but there is only one at this point.) - if err := filepath.Walk(tempDir, func(path string, info os.FileInfo, err error) error { + if err := filepath.WalkDir(tempDir, func(path string, d fs.DirEntry, err error) error { if err != nil { t.Fatal(err) } - if info.Name() == backupbase.BackupManifestName || + if d.Name() == backupbase.BackupManifestName || !strings.HasSuffix(path, ".sst") || - info.Name() == backupinfo.BackupMetadataDescriptorsListPath || - info.Name() == backupinfo.BackupMetadataFilesListPath { + d.Name() == backupinfo.BackupMetadataDescriptorsListPath || + d.Name() == backupinfo.BackupMetadataFilesListPath { return nil } return os.Remove(path) diff --git a/pkg/blobs/local_storage.go b/pkg/blobs/local_storage.go index 8cd926b86255..6e3573f69531 100644 --- a/pkg/blobs/local_storage.go +++ b/pkg/blobs/local_storage.go @@ -8,6 +8,7 @@ package blobs import ( "context" "io" + "io/fs" "os" "path/filepath" "strings" @@ -186,11 +187,11 @@ func (l *LocalStorage) List(pattern string) ([]string, error) { } } - if err := filepath.Walk(walkRoot, func(p string, f os.FileInfo, err error) error { + if err := filepath.WalkDir(walkRoot, func(p string, d fs.DirEntry, err error) error { if err != nil { return err } - if f.IsDir() { + if d.IsDir() { return nil } if listingParent && !strings.HasPrefix(p, fullPath) { diff --git a/pkg/ccl/changefeedccl/sink_cloudstorage_test.go b/pkg/ccl/changefeedccl/sink_cloudstorage_test.go index 5814b31fe060..60ed91e4f74c 100644 --- a/pkg/ccl/changefeedccl/sink_cloudstorage_test.go +++ b/pkg/ccl/changefeedccl/sink_cloudstorage_test.go @@ -10,6 +10,7 @@ import ( "context" "fmt" "io" + "io/fs" "math" "net/url" "os" @@ -111,21 +112,21 @@ func TestCloudStorageSink(t *testing.T) { return false } - walkFn := func(path string, info os.FileInfo, err error) error { + walkDirFn := func(path string, d fs.DirEntry, err error) error { if err != nil { return err } if path == absRoot { return nil } - if info.IsDir() && !hasChildDirs(path) { + if d.IsDir() && !hasChildDirs(path) { relPath, _ := filepath.Rel(absRoot, path) folders = append(folders, relPath) } return nil } - require.NoError(t, filepath.Walk(absRoot, walkFn)) + require.NoError(t, filepath.WalkDir(absRoot, walkDirFn)) return folders } @@ -133,11 +134,11 @@ func TestCloudStorageSink(t *testing.T) { // temp dir created above), sorted by the name of the file. slurpDir := func(t *testing.T) []string { var files []string - walkFn := func(path string, info os.FileInfo, err error) error { + walkDirFn := func(path string, d fs.DirEntry, err error) error { if err != nil { return err } - if info.IsDir() { + if d.IsDir() { return nil } file, err := os.ReadFile(path) @@ -152,7 +153,7 @@ func TestCloudStorageSink(t *testing.T) { } absRoot := filepath.Join(externalIODir, testDir(t)) require.NoError(t, os.MkdirAll(absRoot, 0755)) - require.NoError(t, filepath.Walk(absRoot, walkFn)) + require.NoError(t, filepath.WalkDir(absRoot, walkDirFn)) return files } diff --git a/pkg/ccl/changefeedccl/testfeed_test.go b/pkg/ccl/changefeedccl/testfeed_test.go index f573ef705923..8a62a741896f 100644 --- a/pkg/ccl/changefeedccl/testfeed_test.go +++ b/pkg/ccl/changefeedccl/testfeed_test.go @@ -13,6 +13,7 @@ import ( "encoding/base64" gojson "encoding/json" "fmt" + "io/fs" "math/rand" "net/url" "os" @@ -1493,13 +1494,13 @@ func (c *cloudFeed) Next() (*cdctest.TestFeedMessage, error) { return nil, err } - if err := filepath.Walk(c.dir, c.walkDir); err != nil { + if err := filepath.WalkDir(c.dir, c.walkDir); err != nil { return nil, err } } } -func (c *cloudFeed) walkDir(path string, info os.FileInfo, err error) error { +func (c *cloudFeed) walkDir(path string, d fs.DirEntry, err error) error { if strings.HasSuffix(path, `.tmp`) { // File in the process of being written by ExternalStorage. Ignore. return nil @@ -1520,7 +1521,7 @@ func (c *cloudFeed) walkDir(path string, info os.FileInfo, err error) error { return err } - if info.IsDir() { + if d.IsDir() { // Nothing to do for directories. return nil } diff --git a/pkg/cli/debug_merge_logs.go b/pkg/cli/debug_merge_logs.go index 722420e24816..ef4053cadeda 100644 --- a/pkg/cli/debug_merge_logs.go +++ b/pkg/cli/debug_merge_logs.go @@ -11,6 +11,7 @@ import ( "container/heap" "context" "io" + "io/fs" "os" "path/filepath" "regexp" @@ -349,12 +350,11 @@ func findLogFiles( } var files []fileInfo for _, p := range paths { - // NB: come go1.16, we should use WalkDir here as it is more efficient. - if err := filepath.Walk(p, func(p string, info os.FileInfo, err error) error { + if err := filepath.WalkDir(p, func(p string, d fs.DirEntry, err error) error { if err != nil { return err } - if info.IsDir() { + if d.IsDir() { // Don't act on the directory itself, Walk will visit it for us. return nil } diff --git a/pkg/cli/gen_test.go b/pkg/cli/gen_test.go index c64940d0c0c1..25b24add8f4a 100644 --- a/pkg/cli/gen_test.go +++ b/pkg/cli/gen_test.go @@ -6,6 +6,7 @@ package cli import ( + "io/fs" "os" "path/filepath" "strings" @@ -28,8 +29,8 @@ func TestGenMan(t *testing.T) { // Ensure we have a sane number of man pages. count := 0 - err := filepath.Walk(manpath, func(path string, info os.FileInfo, err error) error { - if strings.HasSuffix(path, ".1") && !info.IsDir() { + err := filepath.WalkDir(manpath, func(path string, d fs.DirEntry, err error) error { + if strings.HasSuffix(path, ".1") && !d.IsDir() { count++ } return nil diff --git a/pkg/cli/userfiletable_test.go b/pkg/cli/userfiletable_test.go index 35057b092b6d..1cb16f8fa12c 100644 --- a/pkg/cli/userfiletable_test.go +++ b/pkg/cli/userfiletable_test.go @@ -10,6 +10,7 @@ import ( "context" "fmt" "io" + "io/fs" "net/url" "os" "path/filepath" @@ -446,12 +447,12 @@ func TestUserFileUploadRecursive(t *testing.T) { dstDir = tc.destination + "/" + filepath.Base(testDir) } - err = filepath.Walk(testDir, - func(path string, info os.FileInfo, err error) error { + err = filepath.WalkDir(testDir, + func(path string, d fs.DirEntry, err error) error { if err != nil { return err } - if info.IsDir() { + if d.IsDir() { return nil } relPath := strings.TrimPrefix(path, testDir+"/") diff --git a/pkg/cmd/dev/io/os/os.go b/pkg/cmd/dev/io/os/os.go index 32d39df578f5..bd2b6d98ea3a 100644 --- a/pkg/cmd/dev/io/os/os.go +++ b/pkg/cmd/dev/io/os/os.go @@ -459,10 +459,10 @@ func (o *OS) ListFilesWithSuffix(root, suffix string) ([]string, error) { output, err := o.Next(command, func() (output string, err error) { var ret []string - if err := filepath.Walk(root, func(path string, info fs.FileInfo, err error) error { + if err := filepath.WalkDir(root, func(path string, d fs.DirEntry, err error) error { // If there's an error walking the tree, throw it away -- there's // nothing interesting we can do with it. - if err != nil || info.IsDir() { + if err != nil || d.IsDir() { //nolint:returnerrcheck return nil } diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index d9801a212c2c..ffc202d2ce6d 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -2333,11 +2333,11 @@ func (c *clusterImpl) RefetchCertsFromNode(ctx context.Context, node int) error return errors.Wrap(err, "cluster.StartE") } // Need to prevent world readable files or lib/pq will complain. - return filepath.Walk(c.localCertsDir, func(path string, info fs.FileInfo, err error) error { + return filepath.WalkDir(c.localCertsDir, func(path string, d fs.DirEntry, err error) error { if err != nil { return errors.Wrap(err, "walking localCertsDir failed") } - if info.IsDir() { + if d.IsDir() { return nil } return os.Chmod(path, 0600) diff --git a/pkg/cmd/roachtest/tests/network.go b/pkg/cmd/roachtest/tests/network.go index 1f36ccec6568..b8415eb1ad6a 100644 --- a/pkg/cmd/roachtest/tests/network.go +++ b/pkg/cmd/roachtest/tests/network.go @@ -8,6 +8,7 @@ package tests import ( "context" "fmt" + "io/fs" "os" "path/filepath" "strconv" @@ -87,7 +88,7 @@ func runNetworkAuthentication(ctx context.Context, t test.Test, c cluster.Cluste require.NoError(t, err) require.NoError(t, os.RemoveAll(localCertsDir)) require.NoError(t, c.Get(ctx, t.L(), certsDir, localCertsDir, c.Node(1))) - require.NoError(t, filepath.Walk(localCertsDir, func(path string, info os.FileInfo, err error) error { + require.NoError(t, filepath.WalkDir(localCertsDir, func(path string, d fs.DirEntry, err error) error { // Don't change permissions for the certs directory. if path == localCertsDir { return nil diff --git a/pkg/cmd/roachtest/zip_util.go b/pkg/cmd/roachtest/zip_util.go index 0f646efb796b..e0260a2d3b1e 100644 --- a/pkg/cmd/roachtest/zip_util.go +++ b/pkg/cmd/roachtest/zip_util.go @@ -8,6 +8,7 @@ package main import ( "archive/zip" "io" + "io/fs" "os" "path/filepath" "sort" @@ -26,11 +27,11 @@ func moveToZipArchive(archiveName string, rootPath string, relPaths ...string) e z := zip.NewWriter(f) for _, relPath := range relPaths { // Walk the given path. - if err := filepath.Walk(filepath.Join(rootPath, relPath), func(path string, info os.FileInfo, err error) error { + if err := filepath.WalkDir(filepath.Join(rootPath, relPath), func(path string, d fs.DirEntry, err error) error { if err != nil { return err } - if info.IsDir() { + if d.IsDir() { // Let Walk recurse inside. return nil } diff --git a/pkg/cmd/roachtest/zip_util_test.go b/pkg/cmd/roachtest/zip_util_test.go index e3bd2f3a5dc2..6a43c4120e8d 100644 --- a/pkg/cmd/roachtest/zip_util_test.go +++ b/pkg/cmd/roachtest/zip_util_test.go @@ -7,6 +7,7 @@ package main import ( "archive/zip" + "io/fs" "os" "path/filepath" "strings" @@ -31,9 +32,9 @@ func TestMoveToZipArchive(t *testing.T) { expectLs := func(expected ...string) { t.Helper() var actual []string - require.NoError(t, filepath.Walk(baseDir, func(path string, info os.FileInfo, err error) error { + require.NoError(t, filepath.WalkDir(baseDir, func(path string, d fs.DirEntry, err error) error { require.NoError(t, err) - if !info.IsDir() { + if !d.IsDir() { rel, err := filepath.Rel(baseDir, path) require.NoError(t, err) actual = append(actual, rel) diff --git a/pkg/cmd/whoownsit/whoownsit.go b/pkg/cmd/whoownsit/whoownsit.go index 4c1b2a904bd8..bdde16eb39df 100644 --- a/pkg/cmd/whoownsit/whoownsit.go +++ b/pkg/cmd/whoownsit/whoownsit.go @@ -12,6 +12,7 @@ package main import ( "flag" "fmt" + "io/fs" "log" "os" "path/filepath" @@ -41,11 +42,11 @@ func main() { os.Exit(1) } } - if err := filepath.Walk(path, func(path string, info os.FileInfo, err error) error { + if err := filepath.WalkDir(path, func(path string, d fs.DirEntry, err error) error { if err != nil { return err } - if !*dirsOnly || info.IsDir() { + if !*dirsOnly || d.IsDir() { matches := codeOwners.Match(path) var aliases []string for _, match := range matches { diff --git a/pkg/internal/codeowners/lint.go b/pkg/internal/codeowners/lint.go index d296601851de..68e5746d5f03 100644 --- a/pkg/internal/codeowners/lint.go +++ b/pkg/internal/codeowners/lint.go @@ -7,7 +7,7 @@ package codeowners import ( "fmt" - "os" + "io/fs" "path/filepath" "sort" "strings" @@ -72,14 +72,14 @@ func LintEverythingIsOwned( walkRoot := filepath.Join(repoRoot, walkDir) - unownedWalkFn := func(path string, info os.FileInfo) error { + unownedWalkFn := func(path string, d fs.DirEntry) error { teams := co.Match(path) if len(teams) > 0 { // The file has an owner, so nothing to report. debug("%s <- has team(s) %v", path, teams) return nil } - if !info.IsDir() { + if !d.IsDir() { // We're looking at a file that has no owner. // // Let's say `path = ./pkg/foo/bar/baz.go`. @@ -113,7 +113,7 @@ func LintEverythingIsOwned( for len(dirsToWalk) != 0 { // We first visit each directory's files, and then the subdirectories. // See TestLintEverythingIsOwned for details. - require.NoError(t, filepath.Walk(dirsToWalk[0], func(path string, info os.FileInfo, err error) error { + require.NoError(t, filepath.WalkDir(dirsToWalk[0], func(path string, d fs.DirEntry, err error) error { if err != nil { return err } @@ -132,7 +132,7 @@ func LintEverythingIsOwned( if _, ok := skip[relPath]; ok { debug("skipping %s", relPath) - if info.IsDir() { + if d.IsDir() { return filepath.SkipDir } return nil @@ -144,7 +144,7 @@ func LintEverythingIsOwned( } if ok { debug("skipping %s", relPath) - if info.IsDir() { + if d.IsDir() { return filepath.SkipDir } return nil @@ -153,21 +153,21 @@ func LintEverythingIsOwned( fname := filepath.Base(relPath) if _, ok := skip[fname]; ok { debug("skipping %s", relPath) - if info.IsDir() { + if d.IsDir() { return filepath.SkipDir } return nil } } - if info.IsDir() { + if d.IsDir() { if path == dirsToWalk[0] { return nil } dirsToWalk = append(dirsToWalk, path) return filepath.SkipDir } - return unownedWalkFn(filepath.Join(walkDir, relPath), info) + return unownedWalkFn(filepath.Join(walkDir, relPath), d) })) dirsToWalk = dirsToWalk[1:] } diff --git a/pkg/storage/pebble_iterator_test.go b/pkg/storage/pebble_iterator_test.go index 2856f2e4274b..fc0e9ede1492 100644 --- a/pkg/storage/pebble_iterator_test.go +++ b/pkg/storage/pebble_iterator_test.go @@ -49,11 +49,11 @@ func TestPebbleIterator_Corruption(t *testing.T) { require.NoError(t, p.Flush()) // Corrupt the SSTs in the DB. - err = filepath.Walk(dataDir, func(path string, info stdfs.FileInfo, err error) error { + err = filepath.WalkDir(dataDir, func(path string, d stdfs.DirEntry, err error) error { if err != nil { return err } - if !strings.HasSuffix(info.Name(), ".sst") { + if !strings.HasSuffix(d.Name(), ".sst") { return nil } file, err := os.OpenFile(path, os.O_WRONLY, 0600) From 4d7999db3e98e5675b182f30d030eaf4976e7f52 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Thu, 16 Jan 2025 02:23:44 -0500 Subject: [PATCH 5/6] scexec: treat BatchTimestampBeforeGCError as permanent This uses the same check and testing that was added in a7cce246994b51b560572db7354fcaf11882cd7e for the materialized view backfill. Release note (bug fix): Fixed a bug where the error "batch timestamp T must be after replica GC threshold" could occur during a schema change backfill operation, and cause the schema change job to retry infinitely. Now this error is treated as permanent, and will cause the job to enter the failed state. --- pkg/sql/backfill_protected_timestamp_test.go | 277 ++++++++++++------ pkg/sql/schemachanger/scexec/exec_backfill.go | 19 +- 2 files changed, 204 insertions(+), 92 deletions(-) diff --git a/pkg/sql/backfill_protected_timestamp_test.go b/pkg/sql/backfill_protected_timestamp_test.go index e1ae446f7b26..38cf4d656241 100644 --- a/pkg/sql/backfill_protected_timestamp_test.go +++ b/pkg/sql/backfill_protected_timestamp_test.go @@ -8,6 +8,7 @@ package sql_test import ( "context" gosql "database/sql" + "fmt" "regexp" "strings" "sync" @@ -23,6 +24,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigptsreader" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -220,10 +223,10 @@ func TestValidationWithProtectedTS(t *testing.T) { } } -// TestBackfillQueryWithProtectedTS backfills a query into a table and confirms -// that a protected timestamp is setup. It also confirms that if the protected -// timestamp is not ready in time we do not infinitely retry. -func TestBackfillQueryWithProtectedTS(t *testing.T) { +// TestBackfillWithProtectedTS runs operations that backfill into a table and +// confirms that a protected timestamp is setup. It also confirms that if the +// protected timestamp is not ready in time we do not infinitely retry. +func TestBackfillWithProtectedTS(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -244,26 +247,68 @@ func TestBackfillQueryWithProtectedTS(t *testing.T) { SQLEvalContext: &eval.TestingKnobs{ ForceProductionValues: true, }, - Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx2 context.Context, request *kvpb.BatchRequest) *kvpb.Error { - // Detect the first operation on the backfill operation, this is before - // the PTS is setup, so the query will fail. - if blockBackFillsForPTSFailure.Load() && - request.Txn != nil && - request.Txn.Name == "schemaChangerBackfill" { + SQLDeclarativeSchemaChanger: &scexec.TestingKnobs{ + RunBeforeBackfill: func() error { + // Cause the backfill to pause before adding the protected + // timestamp. This knob is for testing schema changes that + // are on the declarative schema changer. + if blockBackFillsForPTSFailure.Load() { + if !blockBackFillsForPTSFailure.Swap(false) { + return nil + } + backfillQueryWait <- struct{}{} + <-backfillQueryResume + } + return nil + }, + }, + DistSQL: &execinfra.TestingKnobs{ + RunBeforeBackfillChunk: func(sp roachpb.Span) error { + // Cause the backfill to pause after it already began running + // and has installed a protected timestamp. This knob is for + // testing schema changes that use the index backfiller. + if blockBackFillsForPTSCheck.Load() { + _, prefix, err := s.Codec().DecodeTablePrefix(sp.Key) + if err != nil || prefix != tableID { + //nolint:returnerrcheck + return nil + } + if !blockBackFillsForPTSCheck.Swap(false) { + return nil + } + backfillQueryWait <- struct{}{} + <-backfillQueryResume + } + return nil + }, + }, + SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ + RunBeforeQueryBackfill: func() error { + // Cause the backfill to pause before adding the protected + // timestamp. This knob is for testing CREATE MATERIALIZED VIEW. + if blockBackFillsForPTSFailure.Load() { if !blockBackFillsForPTSFailure.Swap(false) { return nil } backfillQueryWait <- struct{}{} <-backfillQueryResume } - // Detect the first scan on table from the backfill, this is after the - // PTS has been set-up. + return nil + }, + }, + Store: &kvserver.StoreTestingKnobs{ + TestingRequestFilter: func(ctx context.Context, request *kvpb.BatchRequest) *kvpb.Error { + // Detect the first scan on table from the backfill, which is + // after the PTS has been set up. This knob is for testing CREATE + // MATERIALIZED VIEW. if blockBackFillsForPTSCheck.Load() && + request.Txn != nil && + request.Txn.Name == "schemaChangerBackfill" && request.Requests[0].GetInner().Method() == kvpb.Scan { scan := request.Requests[0].GetScan() _, prefix, err := s.Codec().DecodeTablePrefix(scan.Key) if err != nil || prefix != tableID { + //nolint:returnerrcheck return nil } if !blockBackFillsForPTSCheck.Swap(false) { @@ -287,7 +332,7 @@ func TestBackfillQueryWithProtectedTS(t *testing.T) { rSys := sqlutils.MakeSQLRunner(systemSqlDb) // Refreshes the in-memory protected timestamp state to asOf. - refreshTo := func(t *testing.T, tableKey roachpb.Key, asOf hlc.Timestamp) error { + refreshTo := func(ctx context.Context, tableKey roachpb.Key, asOf hlc.Timestamp) error { store, err := s.GetStores().(*kvserver.Stores).GetStore(s.GetFirstStoreID()) if err != nil { return err @@ -310,7 +355,7 @@ func TestBackfillQueryWithProtectedTS(t *testing.T) { return repl.ReadProtectedTimestampsForTesting(ctx) } // Refresh forces the PTS cache to update to at least asOf. - refreshPTSCacheTo := func(t *testing.T, asOf hlc.Timestamp) error { + refreshPTSCacheTo := func(ctx context.Context, asOf hlc.Timestamp) error { ptp := ts.ExecutorConfig().(sql.ExecutorConfig).ProtectedTimestampProvider return ptp.Refresh(ctx, asOf) } @@ -322,87 +367,137 @@ func TestBackfillQueryWithProtectedTS(t *testing.T) { } { rSys.Exec(t, sql) } - for _, sql := range []string{ - "SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false", - "ALTER DATABASE defaultdb CONFIGURE ZONE USING gc.ttlseconds = 1", - "CREATE TABLE t(n int)", - "ALTER TABLE t CONFIGURE ZONE USING range_min_bytes = 0, range_max_bytes = 67108864, gc.ttlseconds = 1", - "INSERT INTO t(n) SELECT * FROM generate_series(1, 500000)", - } { - r.Exec(t, sql) - } - getTableID := func() (tableID uint32) { - r.QueryRow(t, `SELECT table_id FROM crdb_internal.tables`+ - ` WHERE name = 't' AND database_name = current_database()`).Scan(&tableID) - return tableID - } - tableID = getTableID() - tableKey := ts.Codec().TablePrefix(tableID) + const initialRowCount = 500000 + const rowsDeletedPerIteration = 200000 + const rowsAddedPerIteration = 1 - grp := ctxgroup.WithContext(ctx) - grp.Go(func() error { - // We are going to do this twice, first to cause a PTS related failure, - // and a second time for the successful case. The first time we will cause - // the GC to happen before the PTS is setup. The second time we will allow - // the PTS to be installed and then cause the GC. - for i := 0; i < 2; i++ { - <-backfillQueryWait - if _, err := db.ExecContext(ctx, "SET sql_safe_updates=off"); err != nil { - return err - } - if _, err := db.ExecContext(ctx, "BEGIN; DELETE FROM t LIMIT 250000; INSERT INTO t VALUES('9999999'); COMMIT"); err != nil { - return err - } - if err := refreshTo(t, tableKey, ts.Clock().Now()); err != nil { - return err + for _, tc := range []struct { + name string + backfillSchemaChange string + jobDescriptionPrefix string + postTestQuery string + expectedCount int + }{ + { + name: "create materialized view", + backfillSchemaChange: "CREATE MATERIALIZED VIEW test AS (SELECT n from t)", + jobDescriptionPrefix: "CREATE MATERIALIZED VIEW", + postTestQuery: "SELECT count(*) FROM test", + expectedCount: initialRowCount - rowsDeletedPerIteration + rowsAddedPerIteration, + }, + { + name: "create index", + backfillSchemaChange: "CREATE INDEX idx ON t(n)", + jobDescriptionPrefix: "CREATE INDEX idx", + postTestQuery: "SELECT count(*) FROM t@idx", + expectedCount: initialRowCount - 2*rowsDeletedPerIteration + 2*rowsAddedPerIteration, + }, + } { + t.Run(tc.name, func(t *testing.T) { + for _, sql := range []string{ + "SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false", + "ALTER DATABASE defaultdb CONFIGURE ZONE USING gc.ttlseconds = 1", + "DROP TABLE IF EXISTS t CASCADE", + "CREATE TABLE t(n int)", + "ALTER TABLE t CONFIGURE ZONE USING range_min_bytes = 0, range_max_bytes = 67108864, gc.ttlseconds = 1", + fmt.Sprintf("INSERT INTO t(n) SELECT * FROM generate_series(1, %d)", initialRowCount), + } { + r.Exec(t, sql) } - if err := refreshPTSCacheTo(t, ts.Clock().Now()); err != nil { - return err + + getTableID := func() (tableID uint32) { + r.QueryRow(t, `SELECT 't'::regclass::oid`).Scan(&tableID) + return tableID } - if _, err := db.ExecContext(ctx, `SELECT crdb_internal.kv_enqueue_replica(range_id, 'mvccGC', true) + tableID = getTableID() + tableKey := ts.Codec().TablePrefix(tableID) + + grp := ctxgroup.WithContext(ctx) + grp.GoCtx(func(ctx context.Context) error { + // We are going to do this twice, first to cause a PTS related failure, + // and a second time for the successful case. The first time we will cause + // the GC to happen before the PTS is setup. The second time we will allow + // the PTS to be installed and then cause the GC. + for i := 0; i < 2; i++ { + <-backfillQueryWait + if _, err := db.ExecContext(ctx, "SET sql_safe_updates=off"); err != nil { + return err + } + if _, err := db.ExecContext(ctx, fmt.Sprintf( + "BEGIN; DELETE FROM t LIMIT %d; INSERT INTO t VALUES('9999999'); COMMIT", + rowsDeletedPerIteration, + )); err != nil { + return err + } + if err := refreshTo(ctx, tableKey, ts.Clock().Now()); err != nil { + return err + } + if err := refreshPTSCacheTo(ctx, ts.Clock().Now()); err != nil { + return err + } + if _, err := db.ExecContext(ctx, ` +SELECT crdb_internal.kv_enqueue_replica(range_id, 'mvccGC', true) FROM (SELECT range_id FROM [SHOW RANGES FROM TABLE t] ORDER BY start_key);`); err != nil { - return err - } - row := db.QueryRow("SELECT count(*) FROM system.protected_ts_records WHERE meta_type='jobs'") - var count int - if err := row.Scan(&count); err != nil { - return err - } - // First iteration is before the PTS is setup, so it will be 0. Second - // iteration the PTS should be setup. - expectedCount := i - if count != expectedCount { - return errors.AssertionFailedf("no protected timestamp was set up by the schema change job (expected %d, got : %d)", expectedCount, count) - } - backfillQueryResume <- struct{}{} - } - return nil - }) - grp.Go(func() error { - // Backfill with the PTS being not setup early enough, which will - // lead to failure. - blockBackFillsForPTSFailure.Swap(true) - _, err := db.ExecContext(ctx, `CREATE MATERIALIZED VIEW test AS (SELECT n from t)`) - if err == nil || !testutils.IsError(err, "unable to retry backfill since fixed timestamp is before the GC timestamp") { - return errors.AssertionFailedf("expected error was not hit") - } - // Next backfill with the PTS being setup on time, which should always - // succeed. - blockBackFillsForPTSCheck.Swap(true) - _, err = db.ExecContext(ctx, `CREATE MATERIALIZED VIEW test AS (SELECT n from t)`) - return err - }) + return err + } + row := db.QueryRowContext(ctx, "SELECT count(*) FROM system.protected_ts_records WHERE meta_type='jobs'") + var count int + if err := row.Scan(&count); err != nil { + return err + } + // First iteration is before the PTS is setup, so it will be 0. Second + // iteration the PTS should be setup. + expectedCount := i + if count != expectedCount { + return errors.AssertionFailedf("no protected timestamp was set up by the schema change job (expected %d, got : %d)", expectedCount, count) + } + backfillQueryResume <- struct{}{} + } + return nil + }) + grp.GoCtx(func(ctx context.Context) error { + // Backfill with the PTS being not setup early enough, which will + // lead to failure. + blockBackFillsForPTSFailure.Swap(true) + _, err := db.ExecContext(ctx, tc.backfillSchemaChange) + if err == nil || !testutils.IsError(err, "unable to retry backfill since fixed timestamp is before the GC timestamp") { + return errors.AssertionFailedf("expected error was not hit") + } + testutils.SucceedsSoon(t, func() error { + // Wait until schema change is fully rolled back. + var status string + err = db.QueryRowContext(ctx, fmt.Sprintf( + "SELECT status FROM crdb_internal.jobs WHERE description LIKE '%s%%'", + tc.jobDescriptionPrefix, + )).Scan(&status) + if err != nil { + return err + } + if status != "failed" { + return errors.Newf("schema change not rolled back yet; status=%s", status) + } + return nil + }) + // Next backfill with the PTS being setup on time, which should always + // succeed. + blockBackFillsForPTSCheck.Swap(true) + _, err = db.ExecContext(ctx, tc.backfillSchemaChange) + if err != nil { + return err + } + return nil + }) - require.NoError(t, grp.Wait()) - var rowCount int - res := r.QueryRow(t, `SELECT count(*) FROM test`) - res.Scan(&rowCount) - // Half the row count plus the one row inserted above. - const expectedCount = 250000 + 1 - if rowCount != expectedCount { - t.Errorf("expected %d entries, got %d", expectedCount, rowCount) + require.NoError(t, grp.Wait()) + var rowCount int + res := r.QueryRow(t, tc.postTestQuery) + res.Scan(&rowCount) + if rowCount != tc.expectedCount { + t.Errorf("expected %d entries, got %d", tc.expectedCount, rowCount) + } + require.Falsef(t, blockBackFillsForPTSFailure.Load(), "no backfill txn was detected in testing knob.") + require.Falsef(t, blockBackFillsForPTSCheck.Load(), "no backfill txn was detected in testing knob.") + }) } - require.NoError(t, db.Close()) - require.Equalf(t, false, blockBackFillsForPTSFailure.Load(), "no backfill txn was dected in testing knob.") } diff --git a/pkg/sql/schemachanger/scexec/exec_backfill.go b/pkg/sql/schemachanger/scexec/exec_backfill.go index 0d24abde482e..c5d733822a85 100644 --- a/pkg/sql/schemachanger/scexec/exec_backfill.go +++ b/pkg/sql/schemachanger/scexec/exec_backfill.go @@ -41,9 +41,26 @@ func executeBackfillOps(ctx context.Context, deps Dependencies, execute []scop.O return err } if err := runBackfiller(ctx, deps, tracker, backfillProgresses, mergeProgresses, deps.TransactionalJobRegistry().CurrentJob(), tables); err != nil { - if errors.HasType(err, &kvpb.InsufficientSpaceError{}) { + if errors.HasType(err, (*kvpb.InsufficientSpaceError)(nil)) { return jobs.MarkPauseRequestError(errors.UnwrapAll(err)) } + if errors.HasType(err, (*kvpb.BatchTimestampBeforeGCError)(nil)) { + // We will not ever move the timestamp forward so this will fail forever. + // Mark as a permanent error. + if scerrors.HasSchemaChangerUserError(err) { + // We need to unwrap this so that the PermanentJobError is marked + // at the correct level. + err = scerrors.UnwrapSchemaChangerUserError(err) + } + return scerrors.SchemaChangerUserError( + jobs.MarkAsPermanentJobError( + errors.Wrap( + err, + "unable to retry backfill since fixed timestamp is before the GC timestamp", + ), + ), + ) + } return err } return nil From 3afe619cfda0859b0b0e9c120f0ed47a1063bdbe Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Thu, 16 Jan 2025 13:25:36 -0500 Subject: [PATCH 6/6] scerrors: make schemaChangerUserError implement SafeFormatError Release note: None --- pkg/sql/schemachanger/scerrors/errors.go | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/pkg/sql/schemachanger/scerrors/errors.go b/pkg/sql/schemachanger/scerrors/errors.go index 36738550cfa9..9ccef1352ae0 100644 --- a/pkg/sql/schemachanger/scerrors/errors.go +++ b/pkg/sql/schemachanger/scerrors/errors.go @@ -173,6 +173,8 @@ type schemaChangerUserError struct { err error } +var _ errors.SafeFormatter = &schemaChangerUserError{} + // SchemaChangerUserError wraps an error as user consumable, which will surface // it from the declarative schema changer without any wrapping. Normally errors // from the declarative schema changer get wrapped with plan details inside @@ -198,8 +200,14 @@ func UnwrapSchemaChangerUserError(err error) error { return nil } +// SafeFormatError is part of the errors.SafeFormatter interface. +func (e *schemaChangerUserError) SafeFormatError(p errors.Printer) (next error) { + p.Printf("schema change operation encountered an error") + return e.err +} + func (e *schemaChangerUserError) Error() string { - return fmt.Sprintf("schema change operation encountered an error: %s", e.err.Error()) + return fmt.Sprintf("schema change operation encountered an error: %v", e.err) } func (e *schemaChangerUserError) Unwrap() error {