From c93d108cffdd9ee1cb852bbcd1ef8d87cd8d90b1 Mon Sep 17 00:00:00 2001 From: richardjcai Date: Wed, 23 Jun 2021 15:46:39 -0400 Subject: [PATCH] sql: support default privileges at the database level Release note (sql change): Added support for ALTER DEFAULT PRIVILEGES and default privileges stored on databases. All objects created in a database will have the privilege set defined by the default privileges for that type of object on the database. The types of objects are TABLES, SEQUENCES, SCHEMAS, TYPES. Example: ALTER DEFAULT PRIVILEGES GRANT SELECT ON TABLES TO foo makes it such that all tables created by the user that executed the ALTER DEFAULT PRIVILEGES command will have SELECT privilege on the table for user foo. Additionally, one can specify a role. Example: ALTER DEFAULT PRIVILEGES FOR ROLE bar GRANT SELECT ON TABLES TO foo. All tables created by bar will have SELECT privilege for foo. If a role is not specified, it uses the current user. See: https://www.postgresql.org/docs/current/sql-alterdefaultprivileges.html Currently, default privileges are not supported on the schema. Specifying a schema like ALTER DEFAULT PRIVILEGES IN SCHEMA s will error. WITH GRANT OPTION is ignored. GRANT OPTION FOR is also ignored. --- .../sql/bnf/alter_default_privileges_stmt.bnf | 4 +- docs/generated/sql/bnf/stmt_block.bnf | 12 +- pkg/ccl/backupccl/restore_job.go | 5 +- pkg/sql/BUILD.bazel | 1 + pkg/sql/alter_default_privileges.go | 140 +++- pkg/sql/catalog/dbdesc/database_desc.go | 14 + pkg/sql/catalog/descpb/BUILD.bazel | 2 + pkg/sql/catalog/descpb/default_privilege.go | 190 ++++++ pkg/sql/catalog/descpb/privilege.go | 22 +- pkg/sql/catalog/descpb/privilege.pb.go | 615 +++++++++++++++++- pkg/sql/catalog/descpb/privilege.proto | 29 + pkg/sql/catalog/descpb/structured.pb.go | 482 ++++++++------ pkg/sql/catalog/descpb/structured.proto | 3 + pkg/sql/catalog/descriptor.go | 1 + pkg/sql/catalog/tabledesc/validate_test.go | 21 +- pkg/sql/create_schema.go | 10 +- pkg/sql/create_sequence.go | 5 +- pkg/sql/create_table.go | 33 +- pkg/sql/create_type.go | 11 +- pkg/sql/create_view.go | 5 +- .../alter_default_privileges_for_schema | 96 +++ .../alter_default_privileges_for_sequence | 118 ++++ .../alter_default_privileges_for_table | 307 +++++++++ .../alter_default_privileges_for_type | 99 +++ pkg/sql/parser/sql.y | 43 +- .../parser/testdata/alter_default_privileges | 24 + pkg/sql/sem/tree/BUILD.bazel | 1 + pkg/sql/sem/tree/alter_default_privileges.go | 26 +- pkg/sql/walk.go | 1 + 29 files changed, 1990 insertions(+), 330 deletions(-) create mode 100644 pkg/sql/catalog/descpb/default_privilege.go create mode 100644 pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_schema create mode 100644 pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_sequence create mode 100644 pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_table create mode 100644 pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_type diff --git a/docs/generated/sql/bnf/alter_default_privileges_stmt.bnf b/docs/generated/sql/bnf/alter_default_privileges_stmt.bnf index e06511cacbc4..f7aec28124ee 100644 --- a/docs/generated/sql/bnf/alter_default_privileges_stmt.bnf +++ b/docs/generated/sql/bnf/alter_default_privileges_stmt.bnf @@ -1,3 +1,3 @@ alter_default_privileges_stmt ::= - 'ALTER' 'DEFAULT' 'PRIVILEGES' opt_for_role opt_in_schema abbreviated_grant_stmt - | 'ALTER' 'DEFAULT' 'PRIVILEGES' opt_for_role opt_in_schema abbreviated_revoke_stmt + 'ALTER' 'DEFAULT' 'PRIVILEGES' opt_for_roles opt_in_schemas abbreviated_grant_stmt + | 'ALTER' 'DEFAULT' 'PRIVILEGES' opt_for_roles opt_in_schemas abbreviated_revoke_stmt diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index fbaa4488df29..3e2b343f6dad 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -1311,8 +1311,8 @@ alter_type_stmt ::= | 'ALTER' 'TYPE' type_name 'OWNER' 'TO' role_spec alter_default_privileges_stmt ::= - 'ALTER' 'DEFAULT' 'PRIVILEGES' opt_for_role opt_in_schema abbreviated_grant_stmt - | 'ALTER' 'DEFAULT' 'PRIVILEGES' opt_for_role opt_in_schema abbreviated_revoke_stmt + 'ALTER' 'DEFAULT' 'PRIVILEGES' opt_for_roles opt_in_schemas abbreviated_grant_stmt + | 'ALTER' 'DEFAULT' 'PRIVILEGES' opt_for_roles opt_in_schemas abbreviated_revoke_stmt role_or_group_or_user ::= 'ROLE' @@ -1785,12 +1785,12 @@ opt_add_val_placement ::= | 'AFTER' 'SCONST' | -opt_for_role ::= - 'FOR' role_or_group_or_user role_spec +opt_for_roles ::= + 'FOR' role_or_group_or_user role_spec_list | -opt_in_schema ::= - 'IN' 'SCHEMA' qualifiable_schema_name +opt_in_schemas ::= + 'IN' 'SCHEMA' schema_name_list | abbreviated_grant_stmt ::= diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 4ce54a03cc32..6a22e1f5a8a8 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -2398,7 +2398,10 @@ func getRestoringPrivileges( // SCHEMA}. But also like CREATE {TABLE,SCHEMA}, we set the owner to the // user creating the table (the one running the restore). // TODO(dt): Make this more configurable. - updatedPrivileges = sql.CreateInheritedPrivilegesFromDBDesc(parentDB, user) + updatedPrivileges = descpb.CreatePrivilegesFromDefaultPrivileges( + parentDB.GetID(), parentDB.GetDefaultPrivilegeDescriptor(), user, tree.Tables, + parentDB.GetPrivileges(), + ) } case catalog.TypeDescriptor, catalog.DatabaseDescriptor: if descCoverage == tree.RequestedDescriptors { diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 9ef4d9546a49..c87259678bf4 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -8,6 +8,7 @@ go_library( "add_column.go", "alter_column_type.go", "alter_database.go", + "alter_default_privileges.go", "alter_index.go", "alter_primary_key.go", "alter_role.go", diff --git a/pkg/sql/alter_default_privileges.go b/pkg/sql/alter_default_privileges.go index e13e15f6a0b3..48f07c6b6e0c 100644 --- a/pkg/sql/alter_default_privileges.go +++ b/pkg/sql/alter_default_privileges.go @@ -12,13 +12,149 @@ package sql import ( "context" + "fmt" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" ) +var targetObjectToPrivilegeObject = map[tree.AlterDefaultPrivilegesTargetObject]privilege.ObjectType{ + tree.Tables: privilege.Table, + tree.Sequences: privilege.Table, + tree.Types: privilege.Type, + tree.Schemas: privilege.Schema, +} + +type alterDefaultPrivilegesNode struct { + n *tree.AlterDefaultPrivileges + + // Only one of dbDesc or schemaDesc should be populated. + dbDesc *dbdesc.Mutable + schemaDesc *schemadesc.Mutable +} + +func (n *alterDefaultPrivilegesNode) Next(runParams) (bool, error) { return false, nil } +func (n *alterDefaultPrivilegesNode) Values() tree.Datums { return tree.Datums{} } +func (n *alterDefaultPrivilegesNode) Close(context.Context) {} + func (p *planner) alterDefaultPrivileges( ctx context.Context, n *tree.AlterDefaultPrivileges, ) (planNode, error) { - // TODO: implement this. - return nil, nil + // ALTER DEFAULT PRIVILEGES without specifying a schema alters the privileges + // for the current database. + database := p.CurrentDatabase() + dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, database, + tree.DatabaseLookupFlags{Required: true}) + if err != nil { + return nil, err + } + + if len(n.Schemas) > 0 { + return nil, unimplemented.NewWithIssue( + 67376, "ALTER DEFAULT PRIVILEGES IN SCHEMA not implemented", + ) + } + + return &alterDefaultPrivilegesNode{ + n: n, + dbDesc: dbDesc, + }, err +} + +func (n *alterDefaultPrivilegesNode) startExec(params runParams) error { + var targetRoles []security.SQLUsername + if len(n.n.Roles) == 0 { + targetRoles = append(targetRoles, params.p.User()) + } else { + targetRoles = n.n.Roles + } + + users, err := params.p.GetAllRoles(params.ctx) + if err != nil { + return err + } + + for _, targetRole := range targetRoles { + if _, found := users[targetRole]; !found { + return pgerror.Newf(pgcode.UndefinedObject, + "role %s does not exist", targetRole.Normalized()) + } + } + + privileges := n.n.Grant.Privileges + grantees := n.n.Grant.Grantees + objectType := n.n.Grant.Target + if !n.n.IsGrant { + privileges = n.n.Revoke.Privileges + grantees = n.n.Revoke.Grantees + objectType = n.n.Revoke.Target + } + + granteesSQLUsername := make([]security.SQLUsername, len(grantees)) + for i, grantee := range grantees { + granteesSQLUsername[i] = security.MakeSQLUsernameFromPreNormalizedString(string(grantee)) + } + + for _, grantee := range granteesSQLUsername { + if _, found := users[grantee]; !found { + return pgerror.Newf(pgcode.UndefinedObject, + "role %s does not exist", grantee.Normalized()) + } + } + + // You can change default privileges only for objects that will be created + // by yourself or by roles that you are a member of. + for _, targetRole := range targetRoles { + if targetRole != params.p.User() { + memberOf, err := params.p.MemberOfWithAdminOption(params.ctx, params.p.User()) + if err != nil { + return err + } + + if _, found := memberOf[targetRole]; !found { + return pgerror.Newf(pgcode.InsufficientPrivilege, + "must be a member of %s", targetRole.Normalized()) + } + } + } + + if err := privilege.ValidatePrivileges( + privileges, + targetObjectToPrivilegeObject[objectType], + ); err != nil { + return err + } + + defaultPrivs := n.dbDesc.GetDefaultPrivilegeDescriptor() + + for _, targetRole := range targetRoles { + if n.n.IsGrant { + defaultPrivs.GrantDefaultPrivileges( + targetRole, privileges, grantees, objectType, + ) + } else { + defaultPrivs.RevokeDefaultPrivileges( + targetRole, privileges, grantees, objectType, + ) + } + } + + err = params.p.writeNonDropDatabaseChange( + params.ctx, n.dbDesc, tree.AsStringWithFQNames(n.n, params.Ann()), + ) + if err != nil { + return err + } + + if err := params.p.createNonDropDatabaseChangeJob(params.ctx, n.dbDesc.ID, + fmt.Sprintf("updating privileges for database %d", n.dbDesc.ID)); err != nil { + return err + } + return nil } diff --git a/pkg/sql/catalog/dbdesc/database_desc.go b/pkg/sql/catalog/dbdesc/database_desc.go index f3063578312f..1b9ed85923cc 100644 --- a/pkg/sql/catalog/dbdesc/database_desc.go +++ b/pkg/sql/catalog/dbdesc/database_desc.go @@ -214,6 +214,9 @@ func (desc *immutable) ValidateSelf(vea catalog.ValidationErrorAccumulator) { // Validate the privilege descriptor. vea.Report(desc.Privileges.Validate(desc.GetID(), privilege.Database)) + // Validate the default privilege descriptor. + vea.Report(desc.GetDefaultPrivilegeDescriptor().Validate()) + if desc.IsMultiRegion() { desc.validateMultiRegion(vea) } @@ -335,6 +338,17 @@ func (desc *Mutable) ImmutableCopy() catalog.Descriptor { return imm } +// GetDefaultPrivilegeDescriptor gets or creates the DefaultPrivilegeDescriptor +// for the database. +func (desc *immutable) GetDefaultPrivilegeDescriptor() *descpb.DefaultPrivilegeDescriptor { + if desc.DefaultPrivileges == nil { + defaultPrivileges := descpb.DefaultPrivilegeDescriptor{} + desc.DefaultPrivileges = &defaultPrivileges + } + + return desc.DefaultPrivileges +} + // IsNew implements the MutableDescriptor interface. func (desc *Mutable) IsNew() bool { return desc.ClusterVersion == nil diff --git a/pkg/sql/catalog/descpb/BUILD.bazel b/pkg/sql/catalog/descpb/BUILD.bazel index ee0b637385a9..7bb9d4921918 100644 --- a/pkg/sql/catalog/descpb/BUILD.bazel +++ b/pkg/sql/catalog/descpb/BUILD.bazel @@ -8,6 +8,7 @@ go_library( srcs = [ "column.go", "constraint.go", + "default_privilege.go", "descriptor.go", "index.go", "join_type.go", @@ -36,6 +37,7 @@ go_library( "//pkg/util/errorutil/unimplemented", "//pkg/util/hlc", "//pkg/util/log", + "//pkg/util/protoutil", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", ], diff --git a/pkg/sql/catalog/descpb/default_privilege.go b/pkg/sql/catalog/descpb/default_privilege.go new file mode 100644 index 000000000000..2c92c5dd63db --- /dev/null +++ b/pkg/sql/catalog/descpb/default_privilege.go @@ -0,0 +1,190 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package descpb + +import ( + "sort" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" +) + +var targetObjectToPrivilegeObject = map[tree.AlterDefaultPrivilegesTargetObject]privilege.ObjectType{ + tree.Tables: privilege.Table, + tree.Sequences: privilege.Table, + tree.Schemas: privilege.Schema, + tree.Types: privilege.Type, +} + +// GrantDefaultPrivileges grants privileges for the specified users. +func (p *DefaultPrivilegeDescriptor) GrantDefaultPrivileges( + role security.SQLUsername, + privileges privilege.List, + grantees tree.NameList, + targetObject tree.AlterDefaultPrivilegesTargetObject, +) { + defaultPrivileges := p.findOrCreateUser(role) + for _, grantee := range grantees { + defaultPrivilegesPerObject := defaultPrivileges.DefaultPrivilegesPerObject[targetObject] + defaultPrivilegesPerObject.Grant( + security.MakeSQLUsernameFromPreNormalizedString(string(grantee)), + privileges, + ) + + defaultPrivileges.DefaultPrivilegesPerObject[targetObject] = defaultPrivilegesPerObject + } +} + +// RevokeDefaultPrivileges revokes privileges for the specified users. +func (p *DefaultPrivilegeDescriptor) RevokeDefaultPrivileges( + role security.SQLUsername, + privileges privilege.List, + grantees tree.NameList, + targetObject tree.AlterDefaultPrivilegesTargetObject, +) { + defaultPrivileges := p.findOrCreateUser(role) + for _, grantee := range grantees { + defaultPrivilegesPerObject := defaultPrivileges.DefaultPrivilegesPerObject[targetObject] + defaultPrivilegesPerObject.Revoke( + security.MakeSQLUsernameFromPreNormalizedString(string(grantee)), + privileges, + targetObjectToPrivilegeObject[targetObject], + ) + + defaultPrivileges.DefaultPrivilegesPerObject[targetObject] = defaultPrivilegesPerObject + } +} + +// CreatePrivilegesFromDefaultPrivileges creates privileges for a +// the specified object with the corresponding default privileges and +// the appropriate owner (node for system, the restoring user otherwise.) +func CreatePrivilegesFromDefaultPrivileges( + dbID ID, + defaultPrivileges *DefaultPrivilegeDescriptor, + user security.SQLUsername, + targetObject tree.AlterDefaultPrivilegesTargetObject, + databasePrivileges *PrivilegeDescriptor, +) *PrivilegeDescriptor { + // If a new system table is being created (which should only be doable by + // an internal user account), make sure it gets the correct privileges. + if dbID == keys.SystemDatabaseID { + return NewDefaultPrivilegeDescriptor(security.NodeUserName()) + } + + defaultPrivilegesForRole, found := defaultPrivileges.GetDefaultPrivilegesForRole(user) + var newPrivs *PrivilegeDescriptor + if !found { + newPrivs = NewDefaultPrivilegeDescriptor(user) + } else { + defaultPrivs := defaultPrivilegesForRole.DefaultPrivilegesPerObject[targetObject] + newPrivs = protoutil.Clone(&defaultPrivs).(*PrivilegeDescriptor) + newPrivs.Grant(security.AdminRoleName(), DefaultSuperuserPrivileges) + newPrivs.Grant(security.RootUserName(), DefaultSuperuserPrivileges) + newPrivs.SetOwner(user) + newPrivs.Version = Version21_2 + } + // TODO (richardjcai): We also have to handle the "public role" for types. + // "Public" has USAGE by default but it can be revoked. + // We need to use a Version field to determine what a missing Public role + // entry in the DefaultPrivileges map means. + // Issue #67377. + + // TODO(richardjcai): Remove this depending on how we handle the migration. + // For backwards compatibility, also "inherit" privileges from the dbDesc. + // Issue #67378. + if targetObject == tree.Tables || targetObject == tree.Sequences { + for _, u := range databasePrivileges.Users { + newPrivs.Grant(u.UserProto.Decode(), privilege.ListFromBitField(u.Privileges, privilege.Table)) + } + } else if targetObject == tree.Schemas { + for _, u := range databasePrivileges.Users { + newPrivs.Grant(u.UserProto.Decode(), privilege.ListFromBitField(u.Privileges, privilege.Schema)) + } + } + return newPrivs +} + +// User accesses the role field. +func (u *DefaultPrivilegesForRole) User() security.SQLUsername { + return u.UserProto.Decode() +} + +// findUser looks for a specific user in the list. +// Returns (nil, false) if not found, or (obj, true) if found. +func (p *DefaultPrivilegeDescriptor) GetDefaultPrivilegesForRole( + user security.SQLUsername, +) (*DefaultPrivilegesForRole, bool) { + idx := p.findUserIndex(user) + if idx == -1 { + return nil, false + } + return &p.DefaultPrivileges[idx], true +} + +// findUserIndex looks for a given user and returns its +// index in the User array if found. Returns -1 otherwise. +func (p *DefaultPrivilegeDescriptor) findUserIndex(user security.SQLUsername) int { + idx := sort.Search(len(p.DefaultPrivileges), func(i int) bool { + return !p.DefaultPrivileges[i].User().LessThan(user) + }) + if idx < len(p.DefaultPrivileges) && p.DefaultPrivileges[idx].User() == user { + return idx + } + return -1 +} + +// findOrCreateUser looks for a specific user in the list, creating it if needed. +func (p *DefaultPrivilegeDescriptor) findOrCreateUser( + user security.SQLUsername, +) *DefaultPrivilegesForRole { + idx := sort.Search(len(p.DefaultPrivileges), func(i int) bool { + return !p.DefaultPrivileges[i].User().LessThan(user) + }) + if idx == len(p.DefaultPrivileges) { + // Not found but should be inserted at the end. + p.DefaultPrivileges = append(p.DefaultPrivileges, + DefaultPrivilegesForRole{ + UserProto: user.EncodeProto(), + DefaultPrivilegesPerObject: map[tree.AlterDefaultPrivilegesTargetObject]PrivilegeDescriptor{}, + }, + ) + } else if p.DefaultPrivileges[idx].User() == user { + // Found. + } else { + // New element to be inserted at idx. + p.DefaultPrivileges = append(p.DefaultPrivileges, DefaultPrivilegesForRole{}) + copy(p.DefaultPrivileges[idx+1:], p.DefaultPrivileges[idx:]) + p.DefaultPrivileges[idx] = DefaultPrivilegesForRole{ + UserProto: user.EncodeProto(), + DefaultPrivilegesPerObject: map[tree.AlterDefaultPrivilegesTargetObject]PrivilegeDescriptor{}, + } + } + return &p.DefaultPrivileges[idx] +} + +// Validate returns an assertion error if the default privilege descriptor is invalid. +func (p DefaultPrivilegeDescriptor) Validate() error { + for _, defaultPrivilegesForRole := range p.DefaultPrivileges { + for objectType, defaultPrivileges := range defaultPrivilegesForRole.DefaultPrivilegesPerObject { + privilegeObjectType := targetObjectToPrivilegeObject[objectType] + valid, u, remaining := defaultPrivileges.IsValidPrivilegesForObjectType(privilegeObjectType) + if !valid { + return errors.AssertionFailedf("user %s must not have %s privileges on %s", + u.User(), privilege.ListFromBitField(remaining, privilege.Any), objectType) + } + } + } + return nil +} diff --git a/pkg/sql/catalog/descpb/privilege.go b/pkg/sql/catalog/descpb/privilege.go index f4e0d0402d22..d89ac13cf3b4 100644 --- a/pkg/sql/catalog/descpb/privilege.go +++ b/pkg/sql/catalog/descpb/privilege.go @@ -386,6 +386,23 @@ func (p PrivilegeDescriptor) Validate(id ID, objectType privilege.ObjectType) er } } + valid, u, remaining := p.IsValidPrivilegesForObjectType(objectType) + if !valid { + return errors.AssertionFailedf("user %s must not have %s privileges on %s%s with ID=%d", + u.User(), privilege.ListFromBitField(remaining, privilege.Any), maybeSystem, objectType, id) + } + + return nil +} + +// IsValidPrivilegesForObjectType checks if the privileges on the descriptor +// is valid for the given object type. +// If the privileges are invalid, it returns false along with the first user +// found to have invalid privileges and the bits representing the invalid +// privileges. +func (p PrivilegeDescriptor) IsValidPrivilegesForObjectType( + objectType privilege.ObjectType, +) (bool, UserPrivileges, uint32) { allowedPrivilegesBits := privilege.GetValidPrivilegesForObject(objectType).ToBitField() // Validate can be called during the fix_privileges_migration introduced in @@ -415,12 +432,11 @@ func (p PrivilegeDescriptor) Validate(id ID, objectType privilege.ObjectType) er } if remaining := u.Privileges &^ allowedPrivilegesBits; remaining != 0 { - return errors.AssertionFailedf("user %s must not have %s privileges on %s%s with ID=%d", - u.User(), privilege.ListFromBitField(remaining, privilege.Any), maybeSystem, objectType, id) + return false, u, remaining } } - return nil + return true, UserPrivileges{}, 0 } func (p PrivilegeDescriptor) validateRequiredSuperuser( diff --git a/pkg/sql/catalog/descpb/privilege.pb.go b/pkg/sql/catalog/descpb/privilege.pb.go index 133dac4dd12f..7fe53880261d 100644 --- a/pkg/sql/catalog/descpb/privilege.pb.go +++ b/pkg/sql/catalog/descpb/privilege.pb.go @@ -6,8 +6,10 @@ package descpb import ( fmt "fmt" github_com_cockroachdb_cockroach_pkg_security "github.com/cockroachdb/cockroach/pkg/security" + github_com_cockroachdb_cockroach_pkg_sql_sem_tree "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" _ "github.com/gogo/protobuf/gogoproto" proto "github.com/gogo/protobuf/proto" + github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" io "io" math "math" math_bits "math/bits" @@ -97,9 +99,94 @@ func (m *PrivilegeDescriptor) XXX_DiscardUnknown() { var xxx_messageInfo_PrivilegeDescriptor proto.InternalMessageInfo +// DefaultPrivilegesForRole contains the default privileges for a role. +// DefaultPrivileges are the set of privileges that an object created by a user +// should have at creation time. +// DefaultPrivilegesForRole is further broken down depending on the object type. +// The object types (AlterDefaultPrivilegesTargetObject) are: +// 1: Tables +// 2: Sequences +// 3: Types +// 4: Schemas +// default_privileges_per_object are keyed on AlterDefaultPrivilegesTargetObject +// and it's value is a PrivilegeDescriptor that is only used for +// the list of UserPrivileges for that object. +type DefaultPrivilegesForRole struct { + UserProto github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto `protobuf:"bytes,1,opt,name=user_proto,json=userProto,casttype=github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto" json:"user_proto"` + DefaultPrivilegesPerObject map[github_com_cockroachdb_cockroach_pkg_sql_sem_tree.AlterDefaultPrivilegesTargetObject]PrivilegeDescriptor `protobuf:"bytes,2,rep,name=default_privileges_per_object,json=defaultPrivilegesPerObject,castkey=github.com/cockroachdb/cockroach/pkg/sql/sem/tree.AlterDefaultPrivilegesTargetObject" json:"default_privileges_per_object" protobuf_key:"varint,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` +} + +func (m *DefaultPrivilegesForRole) Reset() { *m = DefaultPrivilegesForRole{} } +func (m *DefaultPrivilegesForRole) String() string { return proto.CompactTextString(m) } +func (*DefaultPrivilegesForRole) ProtoMessage() {} +func (*DefaultPrivilegesForRole) Descriptor() ([]byte, []int) { + return fileDescriptor_9343d951995d5a76, []int{2} +} +func (m *DefaultPrivilegesForRole) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DefaultPrivilegesForRole) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *DefaultPrivilegesForRole) XXX_Merge(src proto.Message) { + xxx_messageInfo_DefaultPrivilegesForRole.Merge(m, src) +} +func (m *DefaultPrivilegesForRole) XXX_Size() int { + return m.Size() +} +func (m *DefaultPrivilegesForRole) XXX_DiscardUnknown() { + xxx_messageInfo_DefaultPrivilegesForRole.DiscardUnknown(m) +} + +var xxx_messageInfo_DefaultPrivilegesForRole proto.InternalMessageInfo + +// DefaultPrivilegeDescriptor describes the set of default privileges for a +// given role + object type. +// The default_privileges list must be sorted for fast access and user lookups. +type DefaultPrivilegeDescriptor struct { + DefaultPrivileges []DefaultPrivilegesForRole `protobuf:"bytes,1,rep,name=default_privileges,json=defaultPrivileges" json:"default_privileges"` +} + +func (m *DefaultPrivilegeDescriptor) Reset() { *m = DefaultPrivilegeDescriptor{} } +func (m *DefaultPrivilegeDescriptor) String() string { return proto.CompactTextString(m) } +func (*DefaultPrivilegeDescriptor) ProtoMessage() {} +func (*DefaultPrivilegeDescriptor) Descriptor() ([]byte, []int) { + return fileDescriptor_9343d951995d5a76, []int{3} +} +func (m *DefaultPrivilegeDescriptor) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DefaultPrivilegeDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *DefaultPrivilegeDescriptor) XXX_Merge(src proto.Message) { + xxx_messageInfo_DefaultPrivilegeDescriptor.Merge(m, src) +} +func (m *DefaultPrivilegeDescriptor) XXX_Size() int { + return m.Size() +} +func (m *DefaultPrivilegeDescriptor) XXX_DiscardUnknown() { + xxx_messageInfo_DefaultPrivilegeDescriptor.DiscardUnknown(m) +} + +var xxx_messageInfo_DefaultPrivilegeDescriptor proto.InternalMessageInfo + func init() { proto.RegisterType((*UserPrivileges)(nil), "cockroach.sql.sqlbase.UserPrivileges") proto.RegisterType((*PrivilegeDescriptor)(nil), "cockroach.sql.sqlbase.PrivilegeDescriptor") + proto.RegisterType((*DefaultPrivilegesForRole)(nil), "cockroach.sql.sqlbase.DefaultPrivilegesForRole") + proto.RegisterMapType((map[github_com_cockroachdb_cockroach_pkg_sql_sem_tree.AlterDefaultPrivilegesTargetObject]PrivilegeDescriptor)(nil), "cockroach.sql.sqlbase.DefaultPrivilegesForRole.DefaultPrivilegesPerObjectEntry") + proto.RegisterType((*DefaultPrivilegeDescriptor)(nil), "cockroach.sql.sqlbase.DefaultPrivilegeDescriptor") } func init() { @@ -107,29 +194,40 @@ func init() { } var fileDescriptor_9343d951995d5a76 = []byte{ - // 342 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x52, 0x2a, 0x2e, 0xcc, 0xd1, - 0x4f, 0x4e, 0x2c, 0x49, 0xcc, 0xc9, 0x4f, 0xd7, 0x4f, 0x49, 0x2d, 0x4e, 0x2e, 0x48, 0xd2, 0x2f, - 0x28, 0xca, 0x2c, 0xcb, 0xcc, 0x49, 0x4d, 0x4f, 0xd5, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x12, - 0x4d, 0xce, 0x4f, 0xce, 0x2e, 0xca, 0x4f, 0x4c, 0xce, 0xd0, 0x2b, 0x2e, 0xcc, 0x01, 0xe1, 0xa4, - 0xc4, 0xe2, 0x54, 0x29, 0x91, 0xf4, 0xfc, 0xf4, 0x7c, 0xb0, 0x0a, 0x7d, 0x10, 0x0b, 0xa2, 0x58, - 0x69, 0x31, 0x23, 0x17, 0x5f, 0x68, 0x71, 0x6a, 0x51, 0x00, 0xcc, 0x90, 0x62, 0xa1, 0x54, 0x2e, - 0xae, 0xd2, 0xe2, 0xd4, 0xa2, 0x78, 0xb0, 0x02, 0x09, 0x46, 0x05, 0x46, 0x0d, 0x4e, 0x27, 0xb7, - 0x13, 0xf7, 0xe4, 0x19, 0x7e, 0xdd, 0x93, 0xb7, 0x4b, 0xcf, 0x2c, 0xc9, 0x28, 0x4d, 0xd2, 0x4b, - 0xce, 0xcf, 0xd5, 0x87, 0x5b, 0x93, 0x92, 0x84, 0x60, 0xeb, 0x17, 0x64, 0xa7, 0xeb, 0x17, 0xa7, - 0x26, 0x97, 0x16, 0x65, 0x96, 0x54, 0xea, 0x05, 0x07, 0xfa, 0x80, 0x0c, 0xcf, 0x4b, 0xcc, 0x4d, - 0x0d, 0x00, 0x99, 0x16, 0xc4, 0x59, 0x0a, 0xb6, 0x0b, 0xe4, 0x4c, 0x15, 0x2e, 0x2e, 0xb8, 0xcb, - 0x8b, 0x25, 0x98, 0x14, 0x18, 0x35, 0x78, 0x9d, 0x58, 0x40, 0xd6, 0x04, 0x21, 0x89, 0x5b, 0xb1, - 0xbc, 0x58, 0x20, 0xcf, 0xa8, 0xf4, 0x9f, 0x91, 0x4b, 0x18, 0xee, 0x42, 0x97, 0xd4, 0xe2, 0xe4, - 0xa2, 0xcc, 0x82, 0x92, 0xfc, 0x22, 0x21, 0x47, 0x2e, 0x56, 0x90, 0x81, 0xc5, 0x12, 0x8c, 0x0a, - 0xcc, 0x1a, 0xdc, 0x46, 0xaa, 0x7a, 0x58, 0xbd, 0xae, 0x87, 0xea, 0x41, 0xa8, 0x2d, 0x10, 0x9d, - 0x42, 0xe9, 0x5c, 0xdc, 0xf9, 0xe5, 0x79, 0x70, 0xef, 0x32, 0x51, 0xd5, 0xbb, 0x5c, 0x60, 0xa3, - 0x21, 0xfe, 0x35, 0xe3, 0x62, 0x2f, 0x4b, 0x2d, 0x2a, 0xce, 0xcc, 0xcf, 0x93, 0x60, 0x06, 0x7b, - 0x56, 0x06, 0x6a, 0x89, 0x08, 0x8a, 0xcf, 0xc2, 0x20, 0x6a, 0x82, 0x60, 0x8a, 0x21, 0x21, 0xe0, - 0xa4, 0x71, 0xe2, 0xa1, 0x1c, 0xc3, 0x89, 0x47, 0x72, 0x8c, 0x17, 0x1e, 0xc9, 0x31, 0xde, 0x78, - 0x24, 0xc7, 0xf8, 0xe0, 0x91, 0x1c, 0xe3, 0x84, 0xc7, 0x72, 0x0c, 0x17, 0x1e, 0xcb, 0x31, 0xdc, - 0x78, 0x2c, 0xc7, 0x10, 0xc5, 0x06, 0x49, 0x0e, 0x80, 0x00, 0x00, 0x00, 0xff, 0xff, 0xcc, 0x67, - 0x63, 0xa3, 0x23, 0x02, 0x00, 0x00, + // 519 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x54, 0x4f, 0x6b, 0x13, 0x41, + 0x14, 0xcf, 0x24, 0xad, 0xd2, 0x17, 0x2a, 0xba, 0x56, 0x58, 0x82, 0xee, 0x86, 0xa0, 0x10, 0x3c, + 0xec, 0x40, 0x0f, 0x22, 0x3d, 0x88, 0x0d, 0xb5, 0x27, 0xa1, 0x71, 0xad, 0x1e, 0x44, 0x08, 0xfb, + 0xe7, 0xb9, 0x5d, 0xb3, 0xc9, 0x6c, 0x66, 0x66, 0x23, 0xb9, 0x7a, 0xf2, 0xe8, 0x47, 0x10, 0xfc, + 0x32, 0x39, 0xf6, 0xd8, 0x53, 0xd5, 0xe4, 0xe2, 0xd1, 0x8b, 0x17, 0x2f, 0xca, 0xce, 0x6c, 0x93, + 0xd6, 0x34, 0xa8, 0x20, 0x3d, 0x2c, 0x3c, 0x76, 0xdf, 0xfc, 0xfe, 0xbc, 0xdf, 0xbc, 0x85, 0x86, + 0x18, 0x24, 0x34, 0xf0, 0xa4, 0x97, 0xb0, 0x88, 0x86, 0x28, 0x82, 0xd4, 0xa7, 0x29, 0x8f, 0x87, + 0x71, 0x82, 0x11, 0x3a, 0x29, 0x67, 0x92, 0x19, 0x37, 0x02, 0x16, 0x74, 0x39, 0xf3, 0x82, 0x03, + 0x47, 0x0c, 0x92, 0xfc, 0xf1, 0x3d, 0x81, 0xb5, 0x8d, 0x88, 0x45, 0x4c, 0x75, 0xd0, 0xbc, 0xd2, + 0xcd, 0x8d, 0x8f, 0x04, 0xae, 0x3c, 0x13, 0xc8, 0xdb, 0x27, 0x20, 0xc2, 0x40, 0x80, 0x4c, 0x20, + 0xef, 0xa8, 0x06, 0x93, 0xd4, 0x49, 0x73, 0xad, 0xb5, 0x3b, 0x3e, 0xb6, 0x4b, 0x3f, 0x8e, 0xed, + 0x07, 0x51, 0x2c, 0x0f, 0x32, 0xdf, 0x09, 0x58, 0x8f, 0xce, 0x68, 0x42, 0x7f, 0x5e, 0xd3, 0xb4, + 0x1b, 0x51, 0x81, 0x41, 0xc6, 0x63, 0x39, 0x72, 0x9e, 0x3e, 0x79, 0x9c, 0x83, 0xf7, 0xbd, 0x1e, + 0xb6, 0x73, 0x34, 0x77, 0x2d, 0x53, 0x5c, 0xb9, 0xcc, 0xdb, 0x00, 0x33, 0xe5, 0xc2, 0x2c, 0xd7, + 0x49, 0x73, 0xbd, 0xb5, 0x92, 0xd3, 0xb8, 0xa7, 0xde, 0x6f, 0xad, 0x7c, 0xfd, 0x60, 0x93, 0xc6, + 0x4f, 0x02, 0xd7, 0x67, 0x0a, 0x77, 0x50, 0x04, 0x3c, 0x4e, 0x25, 0xe3, 0xc6, 0x36, 0xac, 0xe6, + 0x80, 0xc2, 0x24, 0xf5, 0x4a, 0xb3, 0xba, 0x79, 0xc7, 0x39, 0xd7, 0xba, 0x73, 0xd6, 0x60, 0xc1, + 0xa2, 0x4f, 0x1a, 0x11, 0x54, 0xd9, 0x9b, 0xfe, 0xcc, 0x6e, 0xf9, 0xbf, 0xda, 0x05, 0x05, 0xad, + 0xfd, 0xde, 0x83, 0xcb, 0x43, 0xe4, 0x22, 0x66, 0x7d, 0xb3, 0xa2, 0xcc, 0xde, 0x2c, 0x48, 0x36, + 0xce, 0x38, 0x7b, 0xae, 0x7b, 0xdc, 0x93, 0xe6, 0x62, 0x02, 0xdf, 0x2b, 0x60, 0xee, 0xe0, 0x2b, + 0x2f, 0x4b, 0xe4, 0xdc, 0xc9, 0x2e, 0xe3, 0x2e, 0x4b, 0xf0, 0xa2, 0x12, 0xfb, 0x46, 0xe0, 0x56, + 0xa8, 0x35, 0x74, 0xe6, 0x11, 0x75, 0x52, 0xe4, 0x1d, 0xe6, 0xbf, 0xc6, 0x40, 0x9a, 0x65, 0x15, + 0xc3, 0xde, 0x92, 0x18, 0x96, 0xe9, 0x5f, 0xfc, 0xd0, 0x46, 0xbe, 0xa7, 0x10, 0x1f, 0xf5, 0x25, + 0x1f, 0xb5, 0x5e, 0xe6, 0x5e, 0xde, 0x7e, 0xb2, 0xf7, 0xff, 0xce, 0xcb, 0x20, 0xa1, 0x02, 0x7b, + 0x54, 0x72, 0x44, 0x67, 0x3b, 0x91, 0xc8, 0x17, 0xf0, 0xf7, 0x3d, 0x1e, 0xa1, 0xd4, 0x14, 0x6e, + 0x2d, 0x5c, 0x4a, 0x5f, 0x1b, 0x81, 0xfd, 0x07, 0x71, 0xc6, 0x55, 0xa8, 0x74, 0x71, 0xa4, 0xa6, + 0xbe, 0xee, 0xe6, 0xa5, 0xf1, 0x10, 0x56, 0x87, 0x5e, 0x92, 0xa1, 0xba, 0x4c, 0xd5, 0xcd, 0xbb, + 0x4b, 0xc6, 0x71, 0xce, 0x85, 0x76, 0xf5, 0xc1, 0xad, 0xf2, 0x7d, 0x52, 0xe4, 0xfe, 0x8e, 0x40, + 0xed, 0x77, 0x05, 0xa7, 0x16, 0x20, 0x04, 0x63, 0x31, 0x91, 0x62, 0x1b, 0xe8, 0x3f, 0xc6, 0x50, + 0xec, 0xc5, 0xb5, 0x85, 0x71, 0x68, 0x29, 0xad, 0xe6, 0xf8, 0x8b, 0x55, 0x1a, 0x4f, 0x2c, 0x72, + 0x38, 0xb1, 0xc8, 0xd1, 0xc4, 0x22, 0x9f, 0x27, 0x16, 0x79, 0x3f, 0xb5, 0x4a, 0x87, 0x53, 0xab, + 0x74, 0x34, 0xb5, 0x4a, 0x2f, 0x2e, 0xe9, 0x3f, 0xd2, 0xaf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x7b, + 0x09, 0x9c, 0x56, 0xa6, 0x04, 0x00, 0x00, } func (this *UserPrivileges) Equal(that interface{}) bool { @@ -194,6 +292,69 @@ func (this *PrivilegeDescriptor) Equal(that interface{}) bool { } return true } +func (this *DefaultPrivilegesForRole) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*DefaultPrivilegesForRole) + if !ok { + that2, ok := that.(DefaultPrivilegesForRole) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.UserProto != that1.UserProto { + return false + } + if len(this.DefaultPrivilegesPerObject) != len(that1.DefaultPrivilegesPerObject) { + return false + } + for i := range this.DefaultPrivilegesPerObject { + a := this.DefaultPrivilegesPerObject[i] + b := that1.DefaultPrivilegesPerObject[i] + if !(&a).Equal(&b) { + return false + } + } + return true +} +func (this *DefaultPrivilegeDescriptor) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*DefaultPrivilegeDescriptor) + if !ok { + that2, ok := that.(DefaultPrivilegeDescriptor) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.DefaultPrivileges) != len(that1.DefaultPrivileges) { + return false + } + for i := range this.DefaultPrivileges { + if !this.DefaultPrivileges[i].Equal(&that1.DefaultPrivileges[i]) { + return false + } + } + return true +} func (m *UserPrivileges) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -270,6 +431,98 @@ func (m *PrivilegeDescriptor) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *DefaultPrivilegesForRole) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *DefaultPrivilegesForRole) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *DefaultPrivilegesForRole) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.DefaultPrivilegesPerObject) > 0 { + keysForDefaultPrivilegesPerObject := make([]uint32, 0, len(m.DefaultPrivilegesPerObject)) + for k := range m.DefaultPrivilegesPerObject { + keysForDefaultPrivilegesPerObject = append(keysForDefaultPrivilegesPerObject, uint32(k)) + } + github_com_gogo_protobuf_sortkeys.Uint32s(keysForDefaultPrivilegesPerObject) + for iNdEx := len(keysForDefaultPrivilegesPerObject) - 1; iNdEx >= 0; iNdEx-- { + v := m.DefaultPrivilegesPerObject[github_com_cockroachdb_cockroach_pkg_sql_sem_tree.AlterDefaultPrivilegesTargetObject(keysForDefaultPrivilegesPerObject[iNdEx])] + baseI := i + { + size, err := (&v).MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintPrivilege(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + i = encodeVarintPrivilege(dAtA, i, uint64(keysForDefaultPrivilegesPerObject[iNdEx])) + i-- + dAtA[i] = 0x8 + i = encodeVarintPrivilege(dAtA, i, uint64(baseI-i)) + i-- + dAtA[i] = 0x12 + } + } + i -= len(m.UserProto) + copy(dAtA[i:], m.UserProto) + i = encodeVarintPrivilege(dAtA, i, uint64(len(m.UserProto))) + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} + +func (m *DefaultPrivilegeDescriptor) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *DefaultPrivilegeDescriptor) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *DefaultPrivilegeDescriptor) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.DefaultPrivileges) > 0 { + for iNdEx := len(m.DefaultPrivileges) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.DefaultPrivileges[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintPrivilege(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + func encodeVarintPrivilege(dAtA []byte, offset int, v uint64) int { offset -= sovPrivilege(v) base := offset @@ -311,6 +564,41 @@ func (m *PrivilegeDescriptor) Size() (n int) { return n } +func (m *DefaultPrivilegesForRole) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.UserProto) + n += 1 + l + sovPrivilege(uint64(l)) + if len(m.DefaultPrivilegesPerObject) > 0 { + for k, v := range m.DefaultPrivilegesPerObject { + _ = k + _ = v + l = v.Size() + mapEntrySize := 1 + sovPrivilege(uint64(k)) + 1 + l + sovPrivilege(uint64(l)) + n += mapEntrySize + 1 + sovPrivilege(uint64(mapEntrySize)) + } + } + return n +} + +func (m *DefaultPrivilegeDescriptor) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.DefaultPrivileges) > 0 { + for _, e := range m.DefaultPrivileges { + l = e.Size() + n += 1 + l + sovPrivilege(uint64(l)) + } + } + return n +} + func sovPrivilege(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 } @@ -553,6 +841,287 @@ func (m *PrivilegeDescriptor) Unmarshal(dAtA []byte) error { } return nil } +func (m *DefaultPrivilegesForRole) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPrivilege + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DefaultPrivilegesForRole: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DefaultPrivilegesForRole: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field UserProto", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPrivilege + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthPrivilege + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthPrivilege + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.UserProto = github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DefaultPrivilegesPerObject", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPrivilege + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPrivilege + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPrivilege + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.DefaultPrivilegesPerObject == nil { + m.DefaultPrivilegesPerObject = make(map[github_com_cockroachdb_cockroach_pkg_sql_sem_tree.AlterDefaultPrivilegesTargetObject]PrivilegeDescriptor) + } + var mapkey uint32 + mapvalue := &PrivilegeDescriptor{} + for iNdEx < postIndex { + entryPreIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPrivilege + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + if fieldNum == 1 { + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPrivilege + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapkey |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPrivilege + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if mapmsglen < 0 { + return ErrInvalidLengthPrivilege + } + postmsgIndex := iNdEx + mapmsglen + if postmsgIndex < 0 { + return ErrInvalidLengthPrivilege + } + if postmsgIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = &PrivilegeDescriptor{} + if err := mapvalue.Unmarshal(dAtA[iNdEx:postmsgIndex]); err != nil { + return err + } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skipPrivilege(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthPrivilege + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.DefaultPrivilegesPerObject[github_com_cockroachdb_cockroach_pkg_sql_sem_tree.AlterDefaultPrivilegesTargetObject(mapkey)] = *mapvalue + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipPrivilege(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthPrivilege + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *DefaultPrivilegeDescriptor) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPrivilege + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DefaultPrivilegeDescriptor: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DefaultPrivilegeDescriptor: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DefaultPrivileges", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPrivilege + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPrivilege + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPrivilege + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DefaultPrivileges = append(m.DefaultPrivileges, DefaultPrivilegesForRole{}) + if err := m.DefaultPrivileges[len(m.DefaultPrivileges)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipPrivilege(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthPrivilege + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipPrivilege(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 diff --git a/pkg/sql/catalog/descpb/privilege.proto b/pkg/sql/catalog/descpb/privilege.proto index bd1a1c02951f..60b7346cadce 100644 --- a/pkg/sql/catalog/descpb/privilege.proto +++ b/pkg/sql/catalog/descpb/privilege.proto @@ -33,3 +33,32 @@ message PrivilegeDescriptor { optional uint32 version = 3 [(gogoproto.nullable) = false, (gogoproto.casttype) = "PrivilegeDescVersion"]; } + +// DefaultPrivilegesForRole contains the default privileges for a role. +// DefaultPrivileges are the set of privileges that an object created by a user +// should have at creation time. +// DefaultPrivilegesForRole is further broken down depending on the object type. +// The object types (AlterDefaultPrivilegesTargetObject) are: +// 1: Tables +// 2: Sequences +// 3: Types +// 4: Schemas +// default_privileges_per_object are keyed on AlterDefaultPrivilegesTargetObject +// and it's value is a PrivilegeDescriptor that is only used for +// the list of UserPrivileges for that object. +message DefaultPrivilegesForRole { + option (gogoproto.equal) = true; + optional string user_proto = 1 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto"]; + map default_privileges_per_object = 2 [(gogoproto.nullable) = false, + (gogoproto.castkey) = "github.com/cockroachdb/cockroach/pkg/sql/sem/tree.AlterDefaultPrivilegesTargetObject"]; + +} + +// DefaultPrivilegeDescriptor describes the set of default privileges for a +// given role + object type. +// The default_privileges list must be sorted for fast access and user lookups. +message DefaultPrivilegeDescriptor { + option (gogoproto.equal) = true; + repeated DefaultPrivilegesForRole default_privileges = 1 [(gogoproto.nullable) = false]; +} diff --git a/pkg/sql/catalog/descpb/structured.pb.go b/pkg/sql/catalog/descpb/structured.pb.go index a1c188832c25..500032f15746 100644 --- a/pkg/sql/catalog/descpb/structured.pb.go +++ b/pkg/sql/catalog/descpb/structured.pb.go @@ -3043,6 +3043,8 @@ type DatabaseDescriptor struct { OfflineReason string `protobuf:"bytes,9,opt,name=offline_reason,json=offlineReason" json:"offline_reason"` // RegionConfig is only set if multi-region controls are set on the database. RegionConfig *DatabaseDescriptor_RegionConfig `protobuf:"bytes,10,opt,name=region_config,json=regionConfig" json:"region_config,omitempty"` + // DefaultPrivileges contains the default privileges for the database. + DefaultPrivileges *DefaultPrivilegeDescriptor `protobuf:"bytes,11,opt,name=default_privileges,json=defaultPrivileges" json:"default_privileges,omitempty"` } func (m *DatabaseDescriptor) Reset() { *m = DatabaseDescriptor{} } @@ -3144,6 +3146,13 @@ func (m *DatabaseDescriptor) GetRegionConfig() *DatabaseDescriptor_RegionConfig return nil } +func (m *DatabaseDescriptor) GetDefaultPrivileges() *DefaultPrivilegeDescriptor { + if m != nil { + return m.DefaultPrivileges + } + return nil +} + // SchemaInfo represents the state of a child user defined schema. type DatabaseDescriptor_SchemaInfo struct { // ID is the ID of the schema. @@ -3754,148 +3763,148 @@ func init() { } var fileDescriptor_12dcc21c3bcc9571 = []byte{ - // 5421 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x7c, 0xdd, 0x6f, 0x23, 0xd7, - 0x75, 0xb8, 0xf8, 0x4d, 0x1e, 0x7e, 0x8d, 0xee, 0x6a, 0x77, 0x69, 0x65, 0x2d, 0x69, 0xb9, 0x5e, - 0x5b, 0x5e, 0xc7, 0xda, 0xb5, 0xec, 0x24, 0x6b, 0x3b, 0xc9, 0xcf, 0xa4, 0x48, 0xad, 0xb8, 0x2b, - 0x91, 0xeb, 0x11, 0xb5, 0xeb, 0x24, 0xbf, 0x66, 0x32, 0xe2, 0x5c, 0x52, 0x63, 0x0d, 0x67, 0xe8, - 0x99, 0xe1, 0x6a, 0x19, 0xf4, 0xa1, 0xc8, 0x53, 0x9f, 0xda, 0x3e, 0xf4, 0xad, 0x08, 0x1a, 0x14, - 0x01, 0x9a, 0xb7, 0x20, 0x28, 0xd0, 0x02, 0x2d, 0xd0, 0xd7, 0xe6, 0x31, 0x45, 0x80, 0x20, 0x4f, - 0x42, 0xab, 0x3c, 0xb4, 0x7f, 0x40, 0x9f, 0xfc, 0x54, 0xdc, 0xaf, 0xf9, 0xe0, 0x87, 0x4c, 0x49, - 0xdb, 0x3c, 0xd8, 0xd0, 0x9c, 0x7b, 0xce, 0xb9, 0xe7, 0xde, 0x7b, 0xbe, 0xef, 0xe5, 0xc2, 0x1d, - 0xe7, 0x0b, 0xe3, 0x7e, 0x47, 0x75, 0x55, 0xc3, 0xea, 0xdd, 0xd7, 0xb0, 0xd3, 0x19, 0x1c, 0xde, - 0x77, 0x5c, 0x7b, 0xd8, 0x71, 0x87, 0x36, 0xd6, 0x36, 0x06, 0xb6, 0xe5, 0x5a, 0xe8, 0x7a, 0xc7, - 0xea, 0x1c, 0xdb, 0x96, 0xda, 0x39, 0xda, 0x70, 0xbe, 0x30, 0xc8, 0x7f, 0x87, 0xaa, 0x83, 0x97, - 0x4b, 0x43, 0x57, 0x37, 0xee, 0x1f, 0x19, 0x9d, 0xfb, 0xae, 0xde, 0xc7, 0x8e, 0xab, 0xf6, 0x07, - 0x8c, 0x60, 0xb9, 0x3c, 0x85, 0xeb, 0xc0, 0xd6, 0x5f, 0xe8, 0x06, 0xee, 0x61, 0x8e, 0x73, 0x9d, - 0xe0, 0xb8, 0xa3, 0x01, 0x76, 0xd8, 0xff, 0x39, 0xf8, 0xb5, 0x1e, 0xb6, 0xee, 0xf7, 0xb0, 0xa5, - 0x9b, 0x1a, 0x7e, 0x79, 0xbf, 0x63, 0x99, 0x5d, 0xbd, 0xc7, 0x87, 0x96, 0x7a, 0x56, 0xcf, 0xa2, - 0x7f, 0xde, 0x27, 0x7f, 0x31, 0x68, 0xf9, 0x27, 0x09, 0xb8, 0xb6, 0x6d, 0xd9, 0x58, 0xef, 0x99, - 0x4f, 0xf0, 0x48, 0xc6, 0x5d, 0x6c, 0x63, 0xb3, 0x83, 0xd1, 0x1a, 0x24, 0x5c, 0xf5, 0xd0, 0xc0, - 0xa5, 0xc8, 0x5a, 0x64, 0x3d, 0x5f, 0x85, 0x5f, 0x9f, 0xae, 0x2e, 0x7c, 0x79, 0xba, 0x1a, 0x6d, - 0xd4, 0x64, 0x36, 0x80, 0xee, 0x42, 0x82, 0xce, 0x52, 0x8a, 0x52, 0x8c, 0x22, 0xc7, 0x48, 0x35, - 0x08, 0x90, 0xa0, 0xd1, 0x51, 0x54, 0x82, 0xb8, 0xa9, 0xf6, 0x71, 0x29, 0xb6, 0x16, 0x59, 0xcf, - 0x54, 0xe3, 0x04, 0x4b, 0xa6, 0x10, 0xf4, 0x04, 0xd2, 0x2f, 0x54, 0x43, 0xd7, 0x74, 0x77, 0x54, - 0x8a, 0xaf, 0x45, 0xd6, 0x0b, 0x9b, 0x6f, 0x6f, 0x4c, 0xdd, 0xaa, 0x8d, 0x2d, 0xcb, 0x74, 0x5c, - 0x5b, 0xd5, 0x4d, 0xf7, 0x19, 0x27, 0xe0, 0x8c, 0x3c, 0x06, 0xe8, 0x01, 0x2c, 0x3a, 0x47, 0xaa, - 0x8d, 0x35, 0x65, 0x60, 0xe3, 0xae, 0xfe, 0x52, 0x31, 0xb0, 0x59, 0x4a, 0xac, 0x45, 0xd6, 0x13, - 0x1c, 0xb5, 0xc8, 0x86, 0x9f, 0xd2, 0xd1, 0x5d, 0x6c, 0xa2, 0x36, 0x64, 0x2c, 0x53, 0xd1, 0xb0, - 0x81, 0x5d, 0x5c, 0x4a, 0xd2, 0xf9, 0xdf, 0x9b, 0x31, 0xff, 0x94, 0x0d, 0xda, 0xa8, 0x74, 0x5c, - 0xdd, 0x32, 0x85, 0x1c, 0x96, 0x59, 0xa3, 0x8c, 0x38, 0xd7, 0xe1, 0x40, 0x53, 0x5d, 0x5c, 0x4a, - 0x5d, 0x99, 0xeb, 0x01, 0x65, 0x84, 0x76, 0x21, 0xd1, 0x57, 0xdd, 0xce, 0x51, 0x29, 0x4d, 0x39, - 0x3e, 0xb8, 0x00, 0xc7, 0x3d, 0x42, 0xc7, 0x19, 0x32, 0x26, 0xe5, 0xe7, 0x90, 0x64, 0xf3, 0xa0, - 0x3c, 0x64, 0x9a, 0x2d, 0xa5, 0xb2, 0xd5, 0x6e, 0xb4, 0x9a, 0xd2, 0x02, 0xca, 0x41, 0x5a, 0xae, - 0xef, 0xb7, 0xe5, 0xc6, 0x56, 0x5b, 0x8a, 0x90, 0xaf, 0xfd, 0x7a, 0x5b, 0x69, 0x1e, 0xec, 0xee, - 0x4a, 0x51, 0x54, 0x84, 0x2c, 0xf9, 0xaa, 0xd5, 0xb7, 0x2b, 0x07, 0xbb, 0x6d, 0x29, 0x86, 0xb2, - 0x90, 0xda, 0xaa, 0xec, 0x6f, 0x55, 0x6a, 0x75, 0x29, 0xbe, 0x1c, 0xff, 0xc5, 0xcf, 0x57, 0x16, - 0xca, 0x0f, 0x20, 0x41, 0xa7, 0x43, 0x00, 0xc9, 0xfd, 0xc6, 0xde, 0xd3, 0xdd, 0xba, 0xb4, 0x80, - 0xd2, 0x10, 0xdf, 0x26, 0x2c, 0x22, 0x84, 0xe2, 0x69, 0x45, 0x6e, 0x37, 0x2a, 0xbb, 0x52, 0x94, - 0x51, 0x7c, 0x14, 0xff, 0xef, 0x9f, 0xad, 0x46, 0xca, 0xff, 0x9e, 0x80, 0x25, 0x5f, 0x76, 0xff, - 0xb4, 0xd1, 0x16, 0x14, 0x2d, 0x5b, 0xef, 0xe9, 0xa6, 0x42, 0x75, 0x4e, 0xd1, 0x35, 0xae, 0x8f, - 0x5f, 0x23, 0xeb, 0x39, 0x3b, 0x5d, 0xcd, 0xb7, 0xe8, 0x70, 0x9b, 0x8c, 0x36, 0x6a, 0x5c, 0x41, - 0xf3, 0x56, 0x00, 0xa8, 0xa1, 0x27, 0xb0, 0xc8, 0x99, 0x74, 0x2c, 0x63, 0xd8, 0x37, 0x15, 0x5d, - 0x73, 0x4a, 0xd1, 0xb5, 0xd8, 0x7a, 0xbe, 0xba, 0x7a, 0x76, 0xba, 0x5a, 0x64, 0x2c, 0xb6, 0xe8, - 0x58, 0xa3, 0xe6, 0x7c, 0x79, 0xba, 0x9a, 0x16, 0x1f, 0x32, 0x9f, 0x9e, 0x7f, 0x6b, 0x0e, 0x7a, - 0x0e, 0xd7, 0x6d, 0xb1, 0xb7, 0x5a, 0x90, 0x61, 0x8c, 0x32, 0xbc, 0x73, 0x76, 0xba, 0x7a, 0xcd, - 0xdb, 0x7c, 0x6d, 0x3a, 0xd3, 0x6b, 0xf6, 0x38, 0x82, 0xe6, 0xa0, 0x16, 0x04, 0xc0, 0xfe, 0x72, - 0xe3, 0x74, 0xb9, 0xab, 0x7c, 0xb9, 0x8b, 0x3e, 0xeb, 0xf0, 0x92, 0x17, 0xed, 0xb1, 0x01, 0xcd, - 0x33, 0xbc, 0xc4, 0xb9, 0x86, 0x97, 0xbc, 0xaa, 0xe1, 0x85, 0xcc, 0x28, 0xf5, 0x7f, 0x62, 0x46, - 0xe9, 0x57, 0x6e, 0x46, 0x99, 0x57, 0x60, 0x46, 0x4c, 0x77, 0x1f, 0xc7, 0xd3, 0x20, 0x65, 0x1f, - 0xc7, 0xd3, 0x59, 0x29, 0xf7, 0x38, 0x9e, 0xce, 0x49, 0xf9, 0xc7, 0xf1, 0x74, 0x5e, 0x2a, 0x94, - 0xff, 0x26, 0x0a, 0xb7, 0x0e, 0x4c, 0xfd, 0x8b, 0x21, 0x7e, 0xae, 0xbb, 0x47, 0xd6, 0xd0, 0xa5, - 0x7e, 0x31, 0xa0, 0xdb, 0x0f, 0x20, 0x3d, 0xa6, 0xd4, 0xd7, 0xf9, 0x29, 0xa7, 0xc2, 0x67, 0x9b, - 0x72, 0xf9, 0x89, 0x3e, 0x04, 0x98, 0xd0, 0xe0, 0xd7, 0xce, 0x4e, 0x57, 0x33, 0xd3, 0xd5, 0x2c, - 0xd3, 0xf1, 0x94, 0xeb, 0x8f, 0xe4, 0x84, 0xcb, 0x90, 0x19, 0xd8, 0x58, 0xd3, 0x3b, 0xe4, 0xd4, - 0x82, 0x7a, 0xe7, 0x83, 0xb9, 0xc5, 0xff, 0x4b, 0x02, 0x24, 0x26, 0x68, 0x0d, 0x3b, 0x1d, 0x5b, - 0x1f, 0xb8, 0x96, 0xed, 0x49, 0x19, 0x99, 0x90, 0xf2, 0x4d, 0x88, 0xea, 0x1a, 0x0f, 0x34, 0x37, - 0xf8, 0x2e, 0x45, 0xe9, 0x06, 0xf9, 0xcb, 0x8d, 0xea, 0x1a, 0xda, 0x80, 0x38, 0x89, 0x86, 0x74, - 0x9d, 0xd9, 0xcd, 0xe5, 0xf1, 0x95, 0xe0, 0xfe, 0x06, 0x0b, 0x96, 0x6d, 0x99, 0xe2, 0xa1, 0x35, - 0x48, 0x9b, 0x43, 0xc3, 0xa0, 0x81, 0x8e, 0xac, 0x3e, 0x2d, 0x96, 0x24, 0xa0, 0xe8, 0x36, 0xe4, - 0x34, 0xdc, 0x55, 0x87, 0x86, 0xab, 0xe0, 0x97, 0x03, 0x9b, 0xad, 0x4a, 0xce, 0x72, 0x58, 0xfd, - 0xe5, 0xc0, 0x46, 0xb7, 0x20, 0x79, 0xa4, 0x6b, 0x1a, 0x36, 0xa9, 0x31, 0x09, 0x16, 0x1c, 0x86, - 0xd6, 0x21, 0xa7, 0x9b, 0x6a, 0xa7, 0x83, 0x1d, 0x47, 0x27, 0xd3, 0x2c, 0x06, 0x70, 0x42, 0x23, - 0x68, 0x13, 0x16, 0x87, 0x0e, 0x76, 0x14, 0x07, 0x7f, 0x31, 0x24, 0x3a, 0x47, 0x4f, 0x19, 0xe8, - 0x29, 0x27, 0xb9, 0x2a, 0x14, 0x09, 0xc2, 0x3e, 0x1f, 0x27, 0x07, 0xbb, 0x09, 0x8b, 0xd6, 0x89, - 0x39, 0x46, 0x93, 0x0b, 0xd3, 0x10, 0x84, 0x20, 0xcd, 0x6d, 0xc8, 0x75, 0xac, 0xfe, 0x60, 0xe8, - 0x62, 0xb6, 0xa4, 0x2c, 0x5b, 0x12, 0x87, 0xd1, 0x25, 0xad, 0x40, 0xea, 0x85, 0x6e, 0xbb, 0x43, - 0xd5, 0x28, 0x49, 0x01, 0x79, 0x05, 0x10, 0x7d, 0x02, 0xd2, 0xa0, 0xa7, 0xa8, 0xae, 0x6b, 0xeb, - 0x87, 0x84, 0x8f, 0x39, 0xec, 0x97, 0xf2, 0xa1, 0xd3, 0x29, 0x3c, 0x7d, 0x54, 0x11, 0xc3, 0xcd, - 0x61, 0x5f, 0x2e, 0x0c, 0x7a, 0xc1, 0x6f, 0xb4, 0x0d, 0xaf, 0xab, 0x86, 0x8b, 0x6d, 0xe1, 0x42, - 0xc9, 0x71, 0x28, 0xba, 0xa9, 0x0c, 0x6c, 0xab, 0x67, 0x63, 0xc7, 0x29, 0x15, 0x02, 0xf3, 0xbe, - 0x46, 0x51, 0xd9, 0x49, 0xb7, 0x47, 0x03, 0xdc, 0x30, 0x9f, 0x72, 0x34, 0xf4, 0x03, 0x40, 0xce, - 0xc8, 0x71, 0x71, 0x5f, 0x30, 0x3a, 0xd6, 0x4d, 0xad, 0x54, 0xa4, 0x9a, 0xfc, 0xd6, 0x0c, 0x4d, - 0xde, 0xa7, 0x04, 0x8c, 0xdd, 0x13, 0xdd, 0xd4, 0xf8, 0x2c, 0x92, 0x33, 0x06, 0xf7, 0x2c, 0x3c, - 0x2d, 0x65, 0x1e, 0xc7, 0xd3, 0x19, 0x09, 0x1e, 0xc7, 0xd3, 0x29, 0x29, 0x5d, 0xfe, 0x8b, 0x28, - 0xdc, 0x60, 0x68, 0xdb, 0x6a, 0x5f, 0x37, 0x46, 0x57, 0xd5, 0x61, 0xc6, 0x85, 0xeb, 0x30, 0x3d, - 0x1e, 0xba, 0x14, 0x42, 0xc6, 0x02, 0x0b, 0x3d, 0x1e, 0x02, 0x6b, 0x12, 0xd0, 0x98, 0x23, 0x88, - 0x5f, 0xc0, 0x11, 0xb4, 0x60, 0x51, 0xa8, 0xb3, 0xc7, 0x81, 0xea, 0x74, 0xbe, 0x7a, 0x87, 0xcb, - 0x54, 0xac, 0x31, 0x04, 0x41, 0x1e, 0x8e, 0x87, 0x5a, 0x68, 0x90, 0x6f, 0x51, 0xf9, 0x9f, 0xa2, - 0xb0, 0xd4, 0x30, 0x5d, 0x6c, 0x1b, 0x58, 0x7d, 0x81, 0x03, 0xdb, 0xf1, 0x19, 0x64, 0x54, 0xb3, - 0x83, 0x1d, 0xd7, 0xb2, 0x9d, 0x52, 0x64, 0x2d, 0xb6, 0x9e, 0xdd, 0xfc, 0x60, 0xc6, 0xa9, 0x4c, - 0xa3, 0xdf, 0xa8, 0x70, 0x62, 0xe1, 0x47, 0x3c, 0x66, 0xcb, 0xff, 0x1a, 0x81, 0xb4, 0x18, 0xbd, - 0x84, 0x2f, 0xfd, 0x06, 0xa4, 0x69, 0x7e, 0xaa, 0x78, 0x67, 0xb2, 0x2c, 0x28, 0x78, 0x02, 0x1b, - 0xcc, 0x65, 0x53, 0x14, 0xb7, 0xa1, 0xa1, 0xad, 0x69, 0x69, 0x66, 0x8c, 0xd2, 0xdf, 0x14, 0xfb, - 0xb7, 0x1f, 0x4e, 0x34, 0x27, 0x32, 0x4f, 0xb6, 0x67, 0x7c, 0xe7, 0xfe, 0x31, 0x02, 0x8b, 0x84, - 0x40, 0xc3, 0x5a, 0x60, 0xdb, 0xee, 0x00, 0xe8, 0x8e, 0xe2, 0x30, 0x38, 0x5d, 0x91, 0x30, 0x85, - 0x8c, 0xee, 0x70, 0x74, 0x4f, 0xd5, 0xa2, 0x13, 0xaa, 0xf6, 0x21, 0xe4, 0x29, 0xad, 0x72, 0x38, - 0xec, 0x1c, 0x63, 0xd7, 0xa1, 0x12, 0x26, 0xaa, 0x4b, 0x5c, 0xc2, 0x1c, 0xe5, 0x50, 0x65, 0x63, - 0x72, 0xce, 0x09, 0x7c, 0x4d, 0x68, 0x5f, 0x7c, 0x42, 0xfb, 0xb8, 0xe0, 0xbf, 0x8c, 0xc3, 0x8d, - 0xa7, 0xaa, 0xed, 0xea, 0x24, 0xd2, 0xea, 0x66, 0x2f, 0x20, 0xfd, 0x5d, 0xc8, 0x9a, 0x43, 0x61, - 0x90, 0x0e, 0x3f, 0x10, 0x26, 0x1f, 0x98, 0x43, 0x6e, 0x60, 0x0e, 0xfa, 0x26, 0x2c, 0x11, 0x34, - 0xbd, 0x3f, 0x30, 0xf4, 0x8e, 0xee, 0x7a, 0xf8, 0xf1, 0x00, 0x3e, 0x32, 0x87, 0xfd, 0x06, 0x47, - 0x10, 0x74, 0xbb, 0x10, 0x37, 0x74, 0xc7, 0xa5, 0x01, 0x30, 0xbb, 0xb9, 0x39, 0x43, 0x9d, 0xa6, - 0xcb, 0xb6, 0xb1, 0xab, 0x3b, 0xae, 0xd8, 0x2b, 0xc2, 0x05, 0xb5, 0x20, 0x61, 0xab, 0x66, 0x0f, - 0x53, 0x3b, 0xcb, 0x6e, 0xbe, 0x7f, 0x31, 0x76, 0x32, 0x21, 0x15, 0x69, 0x01, 0xe5, 0xb3, 0xfc, - 0xd3, 0x08, 0xc4, 0xc9, 0x2c, 0xe7, 0xb8, 0x82, 0x1b, 0x90, 0x7c, 0xa1, 0x1a, 0x43, 0xcc, 0x82, - 0x78, 0x4e, 0xe6, 0x5f, 0xe8, 0x4f, 0xa0, 0xe8, 0x0c, 0x0f, 0x07, 0x81, 0xa9, 0x78, 0x24, 0x7b, - 0xf7, 0x42, 0x52, 0x79, 0x15, 0x4f, 0x98, 0x17, 0x3b, 0xb8, 0xe5, 0x2f, 0x20, 0x41, 0xa5, 0x3e, - 0x47, 0xbe, 0xbb, 0x50, 0xe8, 0xda, 0x56, 0x5f, 0xd1, 0xcd, 0x8e, 0x31, 0x74, 0xf4, 0x17, 0x2c, - 0xa0, 0xe6, 0xe4, 0x3c, 0x81, 0x36, 0x04, 0x90, 0xe8, 0x8a, 0x6b, 0x29, 0xf8, 0xa5, 0x40, 0x8a, - 0x52, 0xa4, 0xac, 0x6b, 0xd5, 0x05, 0x28, 0xa4, 0xea, 0xff, 0x9c, 0x83, 0x22, 0x35, 0xa8, 0xb9, - 0xdc, 0xe5, 0xdd, 0x80, 0xbb, 0xbc, 0x1e, 0x72, 0x97, 0x9e, 0x55, 0x12, 0x6f, 0x79, 0x0b, 0x92, - 0x43, 0x9a, 0x65, 0x51, 0x11, 0xbd, 0xe0, 0xcb, 0x60, 0xe8, 0x21, 0xa4, 0x5e, 0x60, 0xdb, 0xd1, - 0x2d, 0xb3, 0x84, 0x28, 0xa7, 0x15, 0x5e, 0xa5, 0xde, 0x18, 0x13, 0xe4, 0x19, 0xc3, 0x92, 0x05, - 0x3a, 0x5a, 0x07, 0xe9, 0x18, 0x8f, 0x94, 0x29, 0xb6, 0x50, 0x38, 0x26, 0x25, 0x8a, 0xef, 0x8c, - 0x35, 0xb8, 0x1e, 0xc0, 0xd4, 0x74, 0x1b, 0xd3, 0xe4, 0xd3, 0x29, 0xa5, 0xd7, 0x62, 0xe7, 0x24, - 0x99, 0x63, 0x02, 0x6c, 0xd4, 0x04, 0xa1, 0x7c, 0xcd, 0x9b, 0xc0, 0x83, 0x39, 0xe8, 0xeb, 0x80, - 0x88, 0xa7, 0xc3, 0x61, 0x89, 0x12, 0x54, 0x22, 0x89, 0x8e, 0x04, 0x65, 0xaa, 0x42, 0x21, 0x20, - 0x13, 0x09, 0x12, 0x49, 0x1a, 0x24, 0x6e, 0x11, 0xeb, 0x7f, 0x22, 0xd8, 0x8f, 0xc7, 0x89, 0x9c, - 0x37, 0x31, 0x09, 0x15, 0x07, 0x6c, 0x5d, 0xce, 0xb0, 0x4b, 0xfc, 0x5c, 0x80, 0x55, 0x8a, 0xb2, - 0x2a, 0x9f, 0x9d, 0xae, 0xa2, 0x27, 0x78, 0xb4, 0x4f, 0xc7, 0xa7, 0x33, 0x44, 0xc7, 0x63, 0xe3, - 0x9a, 0x83, 0x76, 0x40, 0x0a, 0x2d, 0x84, 0x70, 0x2c, 0x50, 0x8e, 0x2b, 0x24, 0x6d, 0xd8, 0xf7, - 0x97, 0x32, 0xce, 0xad, 0x10, 0x58, 0x26, 0xe1, 0xd4, 0x86, 0x25, 0x92, 0xb3, 0x58, 0x8e, 0xee, - 0x86, 0xb8, 0xe5, 0x7d, 0xf9, 0xb6, 0xc4, 0xf8, 0x0c, 0xf9, 0x3a, 0x63, 0xe3, 0x9a, 0x83, 0xf6, - 0x21, 0xdb, 0x65, 0xf9, 0xbf, 0x72, 0x8c, 0x47, 0xb4, 0x52, 0xc8, 0x6e, 0xde, 0x9b, 0xbf, 0x52, - 0xa8, 0x26, 0x89, 0x8a, 0x95, 0x22, 0x32, 0x74, 0xbd, 0x41, 0xf4, 0x1c, 0xf2, 0x81, 0xe2, 0xee, - 0x70, 0x44, 0xd3, 0xba, 0xcb, 0xb1, 0xcd, 0xf9, 0x8c, 0xaa, 0x23, 0xf4, 0x29, 0x80, 0xee, 0xc5, - 0x4d, 0x9a, 0xc9, 0x65, 0x37, 0xdf, 0xb9, 0x40, 0x80, 0x15, 0x6e, 0xd9, 0x67, 0x82, 0x9e, 0x43, - 0xc1, 0xff, 0xa2, 0xc2, 0xe6, 0x2e, 0x2c, 0x2c, 0xe3, 0x9a, 0x0f, 0xf0, 0xa9, 0x92, 0x4d, 0xc8, - 0x85, 0x5c, 0x5b, 0xf1, 0xf2, 0xae, 0x2d, 0xc4, 0x08, 0xd5, 0x79, 0xd6, 0x2f, 0xd1, 0xac, 0xef, - 0x9d, 0x39, 0x0d, 0x8e, 0x24, 0x92, 0xc2, 0xe3, 0xd0, 0x62, 0xe0, 0x7d, 0x40, 0x1d, 0x1b, 0xab, - 0x2e, 0xd6, 0x48, 0x5e, 0x4c, 0x43, 0x8e, 0x31, 0x0a, 0xe5, 0xeb, 0x8b, 0x7c, 0xbc, 0xee, 0x0d, - 0xa3, 0x1d, 0xc8, 0x63, 0xb3, 0x63, 0x69, 0xba, 0xd9, 0xa3, 0x39, 0x6c, 0xe9, 0x9a, 0x9f, 0x4c, - 0x7d, 0x79, 0xba, 0xfa, 0xb5, 0xb1, 0x59, 0xeb, 0x1c, 0x97, 0x4c, 0x2e, 0xe7, 0x70, 0xe0, 0x0b, - 0xed, 0x40, 0x4a, 0x04, 0xfc, 0x25, 0xba, 0x33, 0xeb, 0xb3, 0xd2, 0xd7, 0xf1, 0x74, 0x41, 0x64, - 0xe7, 0x9c, 0x9c, 0x54, 0x35, 0x9a, 0xee, 0x90, 0x44, 0x47, 0x2b, 0x5d, 0x0f, 0x56, 0x35, 0x02, - 0x8a, 0xb6, 0x00, 0x7a, 0xd8, 0x52, 0x58, 0x7f, 0xb0, 0x74, 0x83, 0x4e, 0xb7, 0x12, 0x98, 0xae, - 0x87, 0xad, 0x0d, 0xd1, 0x45, 0x24, 0x85, 0x5f, 0x57, 0xef, 0x89, 0xfc, 0xa3, 0x87, 0x2d, 0x06, - 0x08, 0x57, 0x7b, 0x37, 0xa7, 0x56, 0x7b, 0xe5, 0x15, 0xc8, 0x78, 0x4e, 0x0c, 0xa5, 0x20, 0x56, - 0xd9, 0xdf, 0x62, 0x2d, 0xa1, 0x5a, 0x7d, 0x7f, 0x4b, 0x8a, 0x94, 0x6f, 0x43, 0x9c, 0x2e, 0x3e, - 0x0b, 0xa9, 0xed, 0x96, 0xfc, 0xbc, 0x22, 0xd7, 0x58, 0x1b, 0xaa, 0xd1, 0x7c, 0x56, 0x97, 0xdb, - 0xf5, 0x9a, 0x24, 0x82, 0xc7, 0x69, 0x1c, 0x90, 0x5f, 0x81, 0xb6, 0x2d, 0x5e, 0xd1, 0xf7, 0xa0, - 0xd8, 0xf1, 0xa0, 0xec, 0x00, 0x22, 0x6b, 0xd1, 0xf5, 0xc2, 0xe6, 0xc3, 0xaf, 0xac, 0x62, 0x05, - 0x8f, 0x20, 0xc8, 0x57, 0x89, 0x42, 0x27, 0x04, 0x0d, 0x24, 0x5b, 0xd1, 0xb1, 0x40, 0x25, 0x43, - 0xa2, 0x73, 0x84, 0x3b, 0xc7, 0x3c, 0x54, 0x7f, 0x73, 0xc6, 0xc4, 0x34, 0x0f, 0x0d, 0xa8, 0xdf, - 0x16, 0xa1, 0xf1, 0xa7, 0x16, 0x39, 0x04, 0x65, 0x85, 0xe4, 0xb0, 0x13, 0x8a, 0x9f, 0x6b, 0xd7, - 0xd3, 0x3a, 0x67, 0xc2, 0xae, 0x03, 0x3e, 0xe8, 0x21, 0x14, 0x4d, 0xcb, 0x55, 0x48, 0x65, 0xcb, - 0xbd, 0x25, 0xad, 0x57, 0xf3, 0x55, 0x89, 0xeb, 0xaa, 0xef, 0x17, 0xf3, 0xa6, 0xe5, 0x36, 0x87, - 0x86, 0xc1, 0x00, 0xe8, 0xcf, 0x22, 0xb0, 0xca, 0x02, 0xaa, 0x72, 0xc2, 0x7a, 0x19, 0x0a, 0xcb, - 0x9d, 0xfd, 0x3d, 0xa2, 0x9d, 0x9f, 0xd9, 0xd9, 0xd3, 0x79, 0x8d, 0x10, 0x2e, 0xea, 0xad, 0xe1, - 0x39, 0x38, 0xe5, 0x36, 0x14, 0xc2, 0xc7, 0x84, 0x32, 0x90, 0xd8, 0xda, 0xa9, 0x6f, 0x3d, 0x91, - 0x16, 0x50, 0x11, 0xb2, 0xdb, 0x2d, 0xb9, 0xde, 0x78, 0xd4, 0x54, 0x9e, 0xd4, 0xbf, 0xc7, 0x3a, - 0x97, 0xcd, 0x96, 0xd7, 0xb9, 0x2c, 0xc1, 0xd2, 0x41, 0xb3, 0xf1, 0xe9, 0x41, 0x5d, 0x79, 0xde, - 0x68, 0xef, 0xb4, 0x0e, 0xda, 0x4a, 0xa3, 0x59, 0xab, 0x7f, 0x26, 0xc5, 0xbc, 0xfa, 0x2e, 0x21, - 0x25, 0xcb, 0xbf, 0x4d, 0x42, 0xe1, 0xa9, 0xad, 0xf7, 0x55, 0x7b, 0x44, 0xa2, 0xda, 0x89, 0x3a, + // 5447 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x7c, 0xcd, 0x73, 0x23, 0xc7, + 0x75, 0x38, 0xf1, 0x0d, 0x3c, 0x7c, 0x0d, 0x7b, 0xb9, 0xbb, 0x10, 0xbd, 0x22, 0xb9, 0x58, 0xad, + 0x44, 0xad, 0x2c, 0xee, 0x8a, 0x92, 0xed, 0x95, 0x64, 0xfb, 0x27, 0x80, 0x00, 0x97, 0xd8, 0x25, + 0x81, 0xd5, 0x10, 0xdc, 0x95, 0xed, 0x5f, 0x3c, 0x1e, 0x62, 0x1a, 0xe0, 0x88, 0x83, 0x19, 0x68, + 0x66, 0xb0, 0x5c, 0xb8, 0x72, 0x48, 0xf9, 0x94, 0x53, 0x92, 0x43, 0x6e, 0x29, 0x57, 0x5c, 0x29, + 0x57, 0xc5, 0x37, 0x97, 0x2b, 0x95, 0xa4, 0x2a, 0xa9, 0xca, 0x35, 0x3e, 0x3a, 0x95, 0x2a, 0x97, + 0x4f, 0xac, 0x84, 0x3e, 0x24, 0x7f, 0x40, 0x4e, 0x3a, 0xa5, 0xfa, 0x6b, 0x3e, 0xf0, 0x41, 0x81, + 0xe4, 0xc6, 0x07, 0xa9, 0x38, 0xaf, 0xdf, 0x7b, 0xfd, 0xba, 0xfb, 0x7d, 0x77, 0x63, 0xe1, 0x8e, + 0xf3, 0x85, 0x71, 0xbf, 0xa3, 0xba, 0xaa, 0x61, 0xf5, 0xee, 0x6b, 0xd8, 0xe9, 0x0c, 0x0e, 0xef, + 0x3b, 0xae, 0x3d, 0xec, 0xb8, 0x43, 0x1b, 0x6b, 0x1b, 0x03, 0xdb, 0x72, 0x2d, 0x74, 0xbd, 0x63, + 0x75, 0x8e, 0x6d, 0x4b, 0xed, 0x1c, 0x6d, 0x38, 0x5f, 0x18, 0xe4, 0xbf, 0x43, 0xd5, 0xc1, 0xcb, + 0xa5, 0xa1, 0xab, 0x1b, 0xf7, 0x8f, 0x8c, 0xce, 0x7d, 0x57, 0xef, 0x63, 0xc7, 0x55, 0xfb, 0x03, + 0x46, 0xb0, 0x5c, 0x9e, 0xc2, 0x75, 0x60, 0xeb, 0x2f, 0x74, 0x03, 0xf7, 0x30, 0xc7, 0xb9, 0x4e, + 0x70, 0xdc, 0xd1, 0x00, 0x3b, 0xec, 0xff, 0x1c, 0xfc, 0x5a, 0x0f, 0x5b, 0xf7, 0x7b, 0xd8, 0xd2, + 0x4d, 0x0d, 0xbf, 0xbc, 0xdf, 0xb1, 0xcc, 0xae, 0xde, 0xe3, 0x43, 0x4b, 0x3d, 0xab, 0x67, 0xd1, + 0x3f, 0xef, 0x93, 0xbf, 0x18, 0xb4, 0xfc, 0x93, 0x04, 0x5c, 0xdb, 0xb6, 0x6c, 0xac, 0xf7, 0xcc, + 0x27, 0x78, 0x24, 0xe3, 0x2e, 0xb6, 0xb1, 0xd9, 0xc1, 0x68, 0x0d, 0x12, 0xae, 0x7a, 0x68, 0xe0, + 0x52, 0x64, 0x2d, 0xb2, 0x9e, 0xaf, 0xc2, 0xaf, 0x4f, 0x57, 0x17, 0xbe, 0x3c, 0x5d, 0x8d, 0x36, + 0x6a, 0x32, 0x1b, 0x40, 0x77, 0x21, 0x41, 0x67, 0x29, 0x45, 0x29, 0x46, 0x91, 0x63, 0xa4, 0x1a, + 0x04, 0x48, 0xd0, 0xe8, 0x28, 0x2a, 0x41, 0xdc, 0x54, 0xfb, 0xb8, 0x14, 0x5b, 0x8b, 0xac, 0x67, + 0xaa, 0x71, 0x82, 0x25, 0x53, 0x08, 0x7a, 0x02, 0xe9, 0x17, 0xaa, 0xa1, 0x6b, 0xba, 0x3b, 0x2a, + 0xc5, 0xd7, 0x22, 0xeb, 0x85, 0xcd, 0xb7, 0x37, 0xa6, 0x6e, 0xd5, 0xc6, 0x96, 0x65, 0x3a, 0xae, + 0xad, 0xea, 0xa6, 0xfb, 0x8c, 0x13, 0x70, 0x46, 0x1e, 0x03, 0xf4, 0x00, 0x16, 0x9d, 0x23, 0xd5, + 0xc6, 0x9a, 0x32, 0xb0, 0x71, 0x57, 0x7f, 0xa9, 0x18, 0xd8, 0x2c, 0x25, 0xd6, 0x22, 0xeb, 0x09, + 0x8e, 0x5a, 0x64, 0xc3, 0x4f, 0xe9, 0xe8, 0x2e, 0x36, 0x51, 0x1b, 0x32, 0x96, 0xa9, 0x68, 0xd8, + 0xc0, 0x2e, 0x2e, 0x25, 0xe9, 0xfc, 0xef, 0xcd, 0x98, 0x7f, 0xca, 0x06, 0x6d, 0x54, 0x3a, 0xae, + 0x6e, 0x99, 0x42, 0x0e, 0xcb, 0xac, 0x51, 0x46, 0x9c, 0xeb, 0x70, 0xa0, 0xa9, 0x2e, 0x2e, 0xa5, + 0xae, 0xcc, 0xf5, 0x80, 0x32, 0x42, 0xbb, 0x90, 0xe8, 0xab, 0x6e, 0xe7, 0xa8, 0x94, 0xa6, 0x1c, + 0x1f, 0x5c, 0x80, 0xe3, 0x1e, 0xa1, 0xe3, 0x0c, 0x19, 0x93, 0xf2, 0x73, 0x48, 0xb2, 0x79, 0x50, + 0x1e, 0x32, 0xcd, 0x96, 0x52, 0xd9, 0x6a, 0x37, 0x5a, 0x4d, 0x69, 0x01, 0xe5, 0x20, 0x2d, 0xd7, + 0xf7, 0xdb, 0x72, 0x63, 0xab, 0x2d, 0x45, 0xc8, 0xd7, 0x7e, 0xbd, 0xad, 0x34, 0x0f, 0x76, 0x77, + 0xa5, 0x28, 0x2a, 0x42, 0x96, 0x7c, 0xd5, 0xea, 0xdb, 0x95, 0x83, 0xdd, 0xb6, 0x14, 0x43, 0x59, + 0x48, 0x6d, 0x55, 0xf6, 0xb7, 0x2a, 0xb5, 0xba, 0x14, 0x5f, 0x8e, 0xff, 0xe2, 0xe7, 0x2b, 0x0b, + 0xe5, 0x07, 0x90, 0xa0, 0xd3, 0x21, 0x80, 0xe4, 0x7e, 0x63, 0xef, 0xe9, 0x6e, 0x5d, 0x5a, 0x40, + 0x69, 0x88, 0x6f, 0x13, 0x16, 0x11, 0x42, 0xf1, 0xb4, 0x22, 0xb7, 0x1b, 0x95, 0x5d, 0x29, 0xca, + 0x28, 0x3e, 0x8a, 0xff, 0xf7, 0xcf, 0x56, 0x23, 0xe5, 0x7f, 0x4b, 0xc0, 0x92, 0x2f, 0xbb, 0x7f, + 0xda, 0x68, 0x0b, 0x8a, 0x96, 0xad, 0xf7, 0x74, 0x53, 0xa1, 0x3a, 0xa7, 0xe8, 0x1a, 0xd7, 0xc7, + 0xaf, 0x91, 0xf5, 0x9c, 0x9d, 0xae, 0xe6, 0x5b, 0x74, 0xb8, 0x4d, 0x46, 0x1b, 0x35, 0xae, 0xa0, + 0x79, 0x2b, 0x00, 0xd4, 0xd0, 0x13, 0x58, 0xe4, 0x4c, 0x3a, 0x96, 0x31, 0xec, 0x9b, 0x8a, 0xae, + 0x39, 0xa5, 0xe8, 0x5a, 0x6c, 0x3d, 0x5f, 0x5d, 0x3d, 0x3b, 0x5d, 0x2d, 0x32, 0x16, 0x5b, 0x74, + 0xac, 0x51, 0x73, 0xbe, 0x3c, 0x5d, 0x4d, 0x8b, 0x0f, 0x99, 0x4f, 0xcf, 0xbf, 0x35, 0x07, 0x3d, + 0x87, 0xeb, 0xb6, 0xd8, 0x5b, 0x2d, 0xc8, 0x30, 0x46, 0x19, 0xde, 0x39, 0x3b, 0x5d, 0xbd, 0xe6, + 0x6d, 0xbe, 0x36, 0x9d, 0xe9, 0x35, 0x7b, 0x1c, 0x41, 0x73, 0x50, 0x0b, 0x02, 0x60, 0x7f, 0xb9, + 0x71, 0xba, 0xdc, 0x55, 0xbe, 0xdc, 0x45, 0x9f, 0x75, 0x78, 0xc9, 0x8b, 0xf6, 0xd8, 0x80, 0xe6, + 0x19, 0x5e, 0xe2, 0x5c, 0xc3, 0x4b, 0x5e, 0xd5, 0xf0, 0x42, 0x66, 0x94, 0xfa, 0x3f, 0x31, 0xa3, + 0xf4, 0x2b, 0x37, 0xa3, 0xcc, 0x2b, 0x30, 0x23, 0xa6, 0xbb, 0x8f, 0xe3, 0x69, 0x90, 0xb2, 0x8f, + 0xe3, 0xe9, 0xac, 0x94, 0x7b, 0x1c, 0x4f, 0xe7, 0xa4, 0xfc, 0xe3, 0x78, 0x3a, 0x2f, 0x15, 0xca, + 0x7f, 0x15, 0x85, 0x5b, 0x07, 0xa6, 0xfe, 0xc5, 0x10, 0x3f, 0xd7, 0xdd, 0x23, 0x6b, 0xe8, 0x52, + 0xbf, 0x18, 0xd0, 0xed, 0x07, 0x90, 0x1e, 0x53, 0xea, 0xeb, 0xfc, 0x94, 0x53, 0xe1, 0xb3, 0x4d, + 0xb9, 0xfc, 0x44, 0x1f, 0x02, 0x4c, 0x68, 0xf0, 0x6b, 0x67, 0xa7, 0xab, 0x99, 0xe9, 0x6a, 0x96, + 0xe9, 0x78, 0xca, 0xf5, 0x07, 0x72, 0xc2, 0x65, 0xc8, 0x0c, 0x6c, 0xac, 0xe9, 0x1d, 0x72, 0x6a, + 0x41, 0xbd, 0xf3, 0xc1, 0xdc, 0xe2, 0xff, 0x39, 0x01, 0x12, 0x13, 0xb4, 0x86, 0x9d, 0x8e, 0xad, + 0x0f, 0x5c, 0xcb, 0xf6, 0xa4, 0x8c, 0x4c, 0x48, 0xf9, 0x26, 0x44, 0x75, 0x8d, 0x07, 0x9a, 0x1b, + 0x7c, 0x97, 0xa2, 0x74, 0x83, 0xfc, 0xe5, 0x46, 0x75, 0x0d, 0x6d, 0x40, 0x9c, 0x44, 0x43, 0xba, + 0xce, 0xec, 0xe6, 0xf2, 0xf8, 0x4a, 0x70, 0x7f, 0x83, 0x05, 0xcb, 0xb6, 0x4c, 0xf1, 0xd0, 0x1a, + 0xa4, 0xcd, 0xa1, 0x61, 0xd0, 0x40, 0x47, 0x56, 0x9f, 0x16, 0x4b, 0x12, 0x50, 0x74, 0x1b, 0x72, + 0x1a, 0xee, 0xaa, 0x43, 0xc3, 0x55, 0xf0, 0xcb, 0x81, 0xcd, 0x56, 0x25, 0x67, 0x39, 0xac, 0xfe, + 0x72, 0x60, 0xa3, 0x5b, 0x90, 0x3c, 0xd2, 0x35, 0x0d, 0x9b, 0xd4, 0x98, 0x04, 0x0b, 0x0e, 0x43, + 0xeb, 0x90, 0xd3, 0x4d, 0xb5, 0xd3, 0xc1, 0x8e, 0xa3, 0x93, 0x69, 0x16, 0x03, 0x38, 0xa1, 0x11, + 0xb4, 0x09, 0x8b, 0x43, 0x07, 0x3b, 0x8a, 0x83, 0xbf, 0x18, 0x12, 0x9d, 0xa3, 0xa7, 0x0c, 0xf4, + 0x94, 0x93, 0x5c, 0x15, 0x8a, 0x04, 0x61, 0x9f, 0x8f, 0x93, 0x83, 0xdd, 0x84, 0x45, 0xeb, 0xc4, + 0x1c, 0xa3, 0xc9, 0x85, 0x69, 0x08, 0x42, 0x90, 0xe6, 0x36, 0xe4, 0x3a, 0x56, 0x7f, 0x30, 0x74, + 0x31, 0x5b, 0x52, 0x96, 0x2d, 0x89, 0xc3, 0xe8, 0x92, 0x56, 0x20, 0xf5, 0x42, 0xb7, 0xdd, 0xa1, + 0x6a, 0x94, 0xa4, 0x80, 0xbc, 0x02, 0x88, 0x3e, 0x01, 0x69, 0xd0, 0x53, 0x54, 0xd7, 0xb5, 0xf5, + 0x43, 0xc2, 0xc7, 0x1c, 0xf6, 0x4b, 0xf9, 0xd0, 0xe9, 0x14, 0x9e, 0x3e, 0xaa, 0x88, 0xe1, 0xe6, + 0xb0, 0x2f, 0x17, 0x06, 0xbd, 0xe0, 0x37, 0xda, 0x86, 0xd7, 0x55, 0xc3, 0xc5, 0xb6, 0x70, 0xa1, + 0xe4, 0x38, 0x14, 0xdd, 0x54, 0x06, 0xb6, 0xd5, 0xb3, 0xb1, 0xe3, 0x94, 0x0a, 0x81, 0x79, 0x5f, + 0xa3, 0xa8, 0xec, 0xa4, 0xdb, 0xa3, 0x01, 0x6e, 0x98, 0x4f, 0x39, 0x1a, 0xfa, 0x01, 0x20, 0x67, + 0xe4, 0xb8, 0xb8, 0x2f, 0x18, 0x1d, 0xeb, 0xa6, 0x56, 0x2a, 0x52, 0x4d, 0x7e, 0x6b, 0x86, 0x26, + 0xef, 0x53, 0x02, 0xc6, 0xee, 0x89, 0x6e, 0x6a, 0x7c, 0x16, 0xc9, 0x19, 0x83, 0x7b, 0x16, 0x9e, + 0x96, 0x32, 0x8f, 0xe3, 0xe9, 0x8c, 0x04, 0x8f, 0xe3, 0xe9, 0x94, 0x94, 0x2e, 0xff, 0x59, 0x14, + 0x6e, 0x30, 0xb4, 0x6d, 0xb5, 0xaf, 0x1b, 0xa3, 0xab, 0xea, 0x30, 0xe3, 0xc2, 0x75, 0x98, 0x1e, + 0x0f, 0x5d, 0x0a, 0x21, 0x63, 0x81, 0x85, 0x1e, 0x0f, 0x81, 0x35, 0x09, 0x68, 0xcc, 0x11, 0xc4, + 0x2f, 0xe0, 0x08, 0x5a, 0xb0, 0x28, 0xd4, 0xd9, 0xe3, 0x40, 0x75, 0x3a, 0x5f, 0xbd, 0xc3, 0x65, + 0x2a, 0xd6, 0x18, 0x82, 0x20, 0x0f, 0xc7, 0x43, 0x2d, 0x34, 0xc8, 0xb7, 0xa8, 0xfc, 0x8f, 0x51, + 0x58, 0x6a, 0x98, 0x2e, 0xb6, 0x0d, 0xac, 0xbe, 0xc0, 0x81, 0xed, 0xf8, 0x0c, 0x32, 0xaa, 0xd9, + 0xc1, 0x8e, 0x6b, 0xd9, 0x4e, 0x29, 0xb2, 0x16, 0x5b, 0xcf, 0x6e, 0x7e, 0x30, 0xe3, 0x54, 0xa6, + 0xd1, 0x6f, 0x54, 0x38, 0xb1, 0xf0, 0x23, 0x1e, 0xb3, 0xe5, 0x7f, 0x89, 0x40, 0x5a, 0x8c, 0x5e, + 0xc2, 0x97, 0x7e, 0x03, 0xd2, 0x34, 0x3f, 0x55, 0xbc, 0x33, 0x59, 0x16, 0x14, 0x3c, 0x81, 0x0d, + 0xe6, 0xb2, 0x29, 0x8a, 0xdb, 0xd0, 0xd0, 0xd6, 0xb4, 0x34, 0x33, 0x46, 0xe9, 0x6f, 0x8a, 0xfd, + 0xdb, 0x0f, 0x27, 0x9a, 0x13, 0x99, 0x27, 0xdb, 0x33, 0xbe, 0x73, 0xff, 0x10, 0x81, 0x45, 0x42, + 0xa0, 0x61, 0x2d, 0xb0, 0x6d, 0x77, 0x00, 0x74, 0x47, 0x71, 0x18, 0x9c, 0xae, 0x48, 0x98, 0x42, + 0x46, 0x77, 0x38, 0xba, 0xa7, 0x6a, 0xd1, 0x09, 0x55, 0xfb, 0x10, 0xf2, 0x94, 0x56, 0x39, 0x1c, + 0x76, 0x8e, 0xb1, 0xeb, 0x50, 0x09, 0x13, 0xd5, 0x25, 0x2e, 0x61, 0x8e, 0x72, 0xa8, 0xb2, 0x31, + 0x39, 0xe7, 0x04, 0xbe, 0x26, 0xb4, 0x2f, 0x3e, 0xa1, 0x7d, 0x5c, 0xf0, 0x5f, 0xc6, 0xe1, 0xc6, + 0x53, 0xd5, 0x76, 0x75, 0x12, 0x69, 0x75, 0xb3, 0x17, 0x90, 0xfe, 0x2e, 0x64, 0xcd, 0xa1, 0x30, + 0x48, 0x87, 0x1f, 0x08, 0x93, 0x0f, 0xcc, 0x21, 0x37, 0x30, 0x07, 0x7d, 0x13, 0x96, 0x08, 0x9a, + 0xde, 0x1f, 0x18, 0x7a, 0x47, 0x77, 0x3d, 0xfc, 0x78, 0x00, 0x1f, 0x99, 0xc3, 0x7e, 0x83, 0x23, + 0x08, 0xba, 0x5d, 0x88, 0x1b, 0xba, 0xe3, 0xd2, 0x00, 0x98, 0xdd, 0xdc, 0x9c, 0xa1, 0x4e, 0xd3, + 0x65, 0xdb, 0xd8, 0xd5, 0x1d, 0x57, 0xec, 0x15, 0xe1, 0x82, 0x5a, 0x90, 0xb0, 0x55, 0xb3, 0x87, + 0xa9, 0x9d, 0x65, 0x37, 0xdf, 0xbf, 0x18, 0x3b, 0x99, 0x90, 0x8a, 0xb4, 0x80, 0xf2, 0x59, 0xfe, + 0x69, 0x04, 0xe2, 0x64, 0x96, 0x73, 0x5c, 0xc1, 0x0d, 0x48, 0xbe, 0x50, 0x8d, 0x21, 0x66, 0x41, + 0x3c, 0x27, 0xf3, 0x2f, 0xf4, 0x47, 0x50, 0x74, 0x86, 0x87, 0x83, 0xc0, 0x54, 0x3c, 0x92, 0xbd, + 0x7b, 0x21, 0xa9, 0xbc, 0x8a, 0x27, 0xcc, 0x8b, 0x1d, 0xdc, 0xf2, 0x17, 0x90, 0xa0, 0x52, 0x9f, + 0x23, 0xdf, 0x5d, 0x28, 0x74, 0x6d, 0xab, 0xaf, 0xe8, 0x66, 0xc7, 0x18, 0x3a, 0xfa, 0x0b, 0x16, + 0x50, 0x73, 0x72, 0x9e, 0x40, 0x1b, 0x02, 0x48, 0x74, 0xc5, 0xb5, 0x14, 0xfc, 0x52, 0x20, 0x45, + 0x29, 0x52, 0xd6, 0xb5, 0xea, 0x02, 0x14, 0x52, 0xf5, 0x7f, 0xca, 0x41, 0x91, 0x1a, 0xd4, 0x5c, + 0xee, 0xf2, 0x6e, 0xc0, 0x5d, 0x5e, 0x0f, 0xb9, 0x4b, 0xcf, 0x2a, 0x89, 0xb7, 0xbc, 0x05, 0xc9, + 0x21, 0xcd, 0xb2, 0xa8, 0x88, 0x5e, 0xf0, 0x65, 0x30, 0xf4, 0x10, 0x52, 0x2f, 0xb0, 0xed, 0xe8, + 0x96, 0x59, 0x42, 0x94, 0xd3, 0x0a, 0xaf, 0x52, 0x6f, 0x8c, 0x09, 0xf2, 0x8c, 0x61, 0xc9, 0x02, + 0x1d, 0xad, 0x83, 0x74, 0x8c, 0x47, 0xca, 0x14, 0x5b, 0x28, 0x1c, 0x93, 0x12, 0xc5, 0x77, 0xc6, + 0x1a, 0x5c, 0x0f, 0x60, 0x6a, 0xba, 0x8d, 0x69, 0xf2, 0xe9, 0x94, 0xd2, 0x6b, 0xb1, 0x73, 0x92, + 0xcc, 0x31, 0x01, 0x36, 0x6a, 0x82, 0x50, 0xbe, 0xe6, 0x4d, 0xe0, 0xc1, 0x1c, 0xf4, 0x75, 0x40, + 0xc4, 0xd3, 0xe1, 0xb0, 0x44, 0x09, 0x2a, 0x91, 0x44, 0x47, 0x82, 0x32, 0x55, 0xa1, 0x10, 0x90, + 0x89, 0x04, 0x89, 0x24, 0x0d, 0x12, 0xb7, 0x88, 0xf5, 0x3f, 0x11, 0xec, 0xc7, 0xe3, 0x44, 0xce, + 0x9b, 0x98, 0x84, 0x8a, 0x03, 0xb6, 0x2e, 0x67, 0xd8, 0x25, 0x7e, 0x2e, 0xc0, 0x2a, 0x45, 0x59, + 0x95, 0xcf, 0x4e, 0x57, 0xd1, 0x13, 0x3c, 0xda, 0xa7, 0xe3, 0xd3, 0x19, 0xa2, 0xe3, 0xb1, 0x71, + 0xcd, 0x41, 0x3b, 0x20, 0x85, 0x16, 0x42, 0x38, 0x16, 0x28, 0xc7, 0x15, 0x92, 0x36, 0xec, 0xfb, + 0x4b, 0x19, 0xe7, 0x56, 0x08, 0x2c, 0x93, 0x70, 0x6a, 0xc3, 0x12, 0xc9, 0x59, 0x2c, 0x47, 0x77, + 0x43, 0xdc, 0xf2, 0xbe, 0x7c, 0x5b, 0x62, 0x7c, 0x86, 0x7c, 0x9d, 0xb1, 0x71, 0xcd, 0x41, 0xfb, + 0x90, 0xed, 0xb2, 0xfc, 0x5f, 0x39, 0xc6, 0x23, 0x5a, 0x29, 0x64, 0x37, 0xef, 0xcd, 0x5f, 0x29, + 0x54, 0x93, 0x44, 0xc5, 0x4a, 0x11, 0x19, 0xba, 0xde, 0x20, 0x7a, 0x0e, 0xf9, 0x40, 0x71, 0x77, + 0x38, 0xa2, 0x69, 0xdd, 0xe5, 0xd8, 0xe6, 0x7c, 0x46, 0xd5, 0x11, 0xfa, 0x14, 0x40, 0xf7, 0xe2, + 0x26, 0xcd, 0xe4, 0xb2, 0x9b, 0xef, 0x5c, 0x20, 0xc0, 0x0a, 0xb7, 0xec, 0x33, 0x41, 0xcf, 0xa1, + 0xe0, 0x7f, 0x51, 0x61, 0x73, 0x17, 0x16, 0x96, 0x71, 0xcd, 0x07, 0xf8, 0x54, 0xc9, 0x26, 0xe4, + 0x42, 0xae, 0xad, 0x78, 0x79, 0xd7, 0x16, 0x62, 0x84, 0xea, 0x3c, 0xeb, 0x97, 0x68, 0xd6, 0xf7, + 0xce, 0x9c, 0x06, 0x47, 0x12, 0x49, 0xe1, 0x71, 0x68, 0x31, 0xf0, 0x3e, 0xa0, 0x8e, 0x8d, 0x55, + 0x17, 0x6b, 0x24, 0x2f, 0xa6, 0x21, 0xc7, 0x18, 0x85, 0xf2, 0xf5, 0x45, 0x3e, 0x5e, 0xf7, 0x86, + 0xd1, 0x0e, 0xe4, 0xb1, 0xd9, 0xb1, 0x34, 0xdd, 0xec, 0xd1, 0x1c, 0xb6, 0x74, 0xcd, 0x4f, 0xa6, + 0xbe, 0x3c, 0x5d, 0xfd, 0xda, 0xd8, 0xac, 0x75, 0x8e, 0x4b, 0x26, 0x97, 0x73, 0x38, 0xf0, 0x85, + 0x76, 0x20, 0x25, 0x02, 0xfe, 0x12, 0xdd, 0x99, 0xf5, 0x59, 0xe9, 0xeb, 0x78, 0xba, 0x20, 0xb2, + 0x73, 0x4e, 0x4e, 0xaa, 0x1a, 0x4d, 0x77, 0x48, 0xa2, 0xa3, 0x95, 0xae, 0x07, 0xab, 0x1a, 0x01, + 0x45, 0x5b, 0x00, 0x3d, 0x6c, 0x29, 0xac, 0x3f, 0x58, 0xba, 0x41, 0xa7, 0x5b, 0x09, 0x4c, 0xd7, + 0xc3, 0xd6, 0x86, 0xe8, 0x22, 0x92, 0xc2, 0xaf, 0xab, 0xf7, 0x44, 0xfe, 0xd1, 0xc3, 0x16, 0x03, + 0x84, 0xab, 0xbd, 0x9b, 0x53, 0xab, 0xbd, 0xf2, 0x0a, 0x64, 0x3c, 0x27, 0x86, 0x52, 0x10, 0xab, + 0xec, 0x6f, 0xb1, 0x96, 0x50, 0xad, 0xbe, 0xbf, 0x25, 0x45, 0xca, 0xb7, 0x21, 0x4e, 0x17, 0x9f, + 0x85, 0xd4, 0x76, 0x4b, 0x7e, 0x5e, 0x91, 0x6b, 0xac, 0x0d, 0xd5, 0x68, 0x3e, 0xab, 0xcb, 0xed, + 0x7a, 0x4d, 0x12, 0xc1, 0xe3, 0x34, 0x0e, 0xc8, 0xaf, 0x40, 0xdb, 0x16, 0xaf, 0xe8, 0x7b, 0x50, + 0xec, 0x78, 0x50, 0x76, 0x00, 0x91, 0xb5, 0xe8, 0x7a, 0x61, 0xf3, 0xe1, 0x57, 0x56, 0xb1, 0x82, + 0x47, 0x10, 0xe4, 0xab, 0x44, 0xa1, 0x13, 0x82, 0x06, 0x92, 0xad, 0xe8, 0x58, 0xa0, 0x92, 0x21, + 0xd1, 0x39, 0xc2, 0x9d, 0x63, 0x1e, 0xaa, 0xbf, 0x39, 0x63, 0x62, 0x9a, 0x87, 0x06, 0xd4, 0x6f, + 0x8b, 0xd0, 0xf8, 0x53, 0x8b, 0x1c, 0x82, 0xb2, 0x42, 0x72, 0xd8, 0x09, 0xc5, 0xcf, 0xb5, 0xeb, + 0x69, 0x9d, 0x33, 0x61, 0xd7, 0x01, 0x1f, 0xf4, 0x10, 0x8a, 0xa6, 0xe5, 0x2a, 0xa4, 0xb2, 0xe5, + 0xde, 0x92, 0xd6, 0xab, 0xf9, 0xaa, 0xc4, 0x75, 0xd5, 0xf7, 0x8b, 0x79, 0xd3, 0x72, 0x9b, 0x43, + 0xc3, 0x60, 0x00, 0xf4, 0x27, 0x11, 0x58, 0x65, 0x01, 0x55, 0x39, 0x61, 0xbd, 0x0c, 0x85, 0xe5, + 0xce, 0xfe, 0x1e, 0xd1, 0xce, 0xcf, 0xec, 0xec, 0xe9, 0xbc, 0x46, 0x08, 0x17, 0xf5, 0xd6, 0xf0, + 0x1c, 0x9c, 0x72, 0x1b, 0x0a, 0xe1, 0x63, 0x42, 0x19, 0x48, 0x6c, 0xed, 0xd4, 0xb7, 0x9e, 0x48, + 0x0b, 0xa8, 0x08, 0xd9, 0xed, 0x96, 0x5c, 0x6f, 0x3c, 0x6a, 0x2a, 0x4f, 0xea, 0xdf, 0x63, 0x9d, + 0xcb, 0x66, 0xcb, 0xeb, 0x5c, 0x96, 0x60, 0xe9, 0xa0, 0xd9, 0xf8, 0xf4, 0xa0, 0xae, 0x3c, 0x6f, + 0xb4, 0x77, 0x5a, 0x07, 0x6d, 0xa5, 0xd1, 0xac, 0xd5, 0x3f, 0x93, 0x62, 0x5e, 0x7d, 0x97, 0x90, + 0x92, 0xe5, 0x7f, 0x4f, 0x42, 0xe1, 0xa9, 0xad, 0xf7, 0x55, 0x7b, 0x44, 0xa2, 0xda, 0x89, 0x3a, 0x40, 0x9f, 0xc0, 0x92, 0x65, 0x90, 0x4c, 0x9f, 0x42, 0x15, 0xaf, 0x5e, 0x88, 0x4f, 0x6f, 0x78, 0x2f, 0x5a, 0x86, 0xc6, 0x39, 0x34, 0x78, 0xb9, 0xf0, 0x09, 0x2c, 0x99, 0xf8, 0x64, 0x92, 0x43, 0x64, 0x06, 0x07, 0x13, 0x9f, 0x8c, 0x71, 0xf8, 0x3a, 0x64, 0x89, 0x0c, 0x94, 0x12, 0x8b, 0xa6, @@ -3903,7 +3912,7 @@ var fileDescriptor_12dcc21c3bcc9571 = []byte{ 0x6c, 0x13, 0x9f, 0x08, 0xec, 0x0f, 0xe1, 0xc6, 0xa4, 0x74, 0x13, 0x3d, 0xc3, 0x6b, 0x63, 0x42, 0x91, 0x0c, 0x03, 0x7d, 0x0e, 0x4b, 0x86, 0xd5, 0x51, 0x0d, 0xdd, 0x1d, 0x71, 0x2f, 0xa2, 0x38, 0x27, 0xea, 0x80, 0x6a, 0x54, 0x76, 0xa6, 0xf1, 0x85, 0xf7, 0x77, 0x63, 0x97, 0x73, 0x60, 0xfe, - 0x84, 0x80, 0x64, 0x64, 0x4c, 0xc0, 0x96, 0xff, 0x21, 0x06, 0x68, 0x12, 0x15, 0x1d, 0xc3, 0x35, + 0x84, 0x80, 0x64, 0x64, 0x4c, 0xc0, 0x96, 0xff, 0x2e, 0x06, 0x68, 0x12, 0x15, 0x1d, 0xc3, 0x35, 0xb2, 0x33, 0x63, 0x62, 0xd0, 0xad, 0xcd, 0x6e, 0x7e, 0x63, 0x4e, 0x2b, 0x0c, 0xf3, 0x15, 0x6e, 0xde, 0x32, 0xb4, 0xf0, 0x00, 0x99, 0x8c, 0x6c, 0xd5, 0xf8, 0x64, 0xd1, 0x57, 0x30, 0x99, 0x89, 0x4f, 0xc6, 0x26, 0xd3, 0xe1, 0x75, 0x32, 0x99, 0x8d, 0x7b, 0xba, 0x65, 0xaa, 0x86, 0x72, 0x38, @@ -3920,7 +3929,7 @@ var fileDescriptor_12dcc21c3bcc9571 = []byte{ 0x42, 0x75, 0x14, 0xab, 0xcb, 0x63, 0xd2, 0xeb, 0x01, 0x46, 0x43, 0x57, 0x37, 0x36, 0x8e, 0x8c, 0xce, 0x46, 0x5b, 0xdc, 0x3a, 0x8a, 0x68, 0xa6, 0x3a, 0xad, 0x2e, 0x7a, 0x17, 0x8a, 0xce, 0x91, 0x35, 0x34, 0x34, 0xe5, 0x50, 0xed, 0x1c, 0x77, 0x75, 0xc3, 0x08, 0x35, 0x46, 0x0b, 0x6c, 0xb0, - 0xca, 0xc7, 0xf8, 0x9e, 0xfd, 0x65, 0x0a, 0x90, 0x2f, 0xcf, 0xde, 0xd0, 0x55, 0x69, 0xbc, 0xaf, + 0xca, 0xc7, 0xf8, 0x9e, 0xfd, 0x79, 0x0a, 0x90, 0x2f, 0xcf, 0xde, 0xd0, 0x55, 0x69, 0xbc, 0xaf, 0x40, 0x92, 0x07, 0x1a, 0xb6, 0x47, 0x6f, 0xcd, 0x8c, 0xc9, 0xe1, 0x46, 0xf0, 0xce, 0x82, 0xcc, 0x09, 0xd1, 0x77, 0x83, 0x97, 0x8c, 0x73, 0xef, 0xc8, 0xce, 0x82, 0xb8, 0x7d, 0x7c, 0x02, 0x10, 0x08, 0x52, 0x69, 0xca, 0xe4, 0xed, 0xb9, 0x53, 0x83, 0x9d, 0x05, 0x39, 0x40, 0x8e, 0x5a, 0x50, @@ -3937,11 +3946,11 @@ var fileDescriptor_12dcc21c3bcc9571 = []byte{ 0x83, 0xeb, 0x1c, 0x50, 0x69, 0xd6, 0x94, 0xe7, 0x72, 0x43, 0x0c, 0x45, 0xcb, 0xeb, 0xc1, 0x9c, 0x39, 0x0d, 0xf1, 0x66, 0xab, 0x59, 0x97, 0x16, 0x68, 0xf6, 0x5c, 0xab, 0x49, 0x11, 0x9a, 0x3d, 0xcb, 0xad, 0xa7, 0x52, 0x94, 0x59, 0x5f, 0x35, 0x07, 0xa0, 0x79, 0xfb, 0xf0, 0x38, 0x9e, 0x4e, - 0x4a, 0xa9, 0xf2, 0xdf, 0x47, 0x20, 0x4d, 0x02, 0x75, 0xc3, 0xec, 0x5a, 0xe8, 0x7d, 0xc8, 0x0c, + 0x4a, 0xa9, 0xf2, 0xdf, 0x46, 0x20, 0x4d, 0x02, 0x75, 0xc3, 0xec, 0x5a, 0xe8, 0x7d, 0xc8, 0x0c, 0x54, 0x1b, 0x9b, 0xae, 0xef, 0x69, 0x45, 0x03, 0x3a, 0xfd, 0x94, 0x0e, 0x78, 0xfd, 0xd1, 0x34, 0x43, 0x6c, 0x68, 0x68, 0x1b, 0x24, 0x4e, 0xe4, 0x74, 0x8e, 0x70, 0x5f, 0xf5, 0xe3, 0xce, 0x2d, 0xaf, 0xc5, 0x4f, 0xc7, 0xf7, 0xe9, 0xb0, 0xc7, 0xa1, 0x30, 0x08, 0x42, 0xcf, 0xe9, 0x52, 0x72, - 0xdf, 0xf1, 0xd7, 0x6f, 0x43, 0x71, 0x2c, 0x50, 0x9e, 0xd3, 0x15, 0x5a, 0xa3, 0x5d, 0xa1, 0x98, + 0xdf, 0xf1, 0x97, 0x6f, 0x43, 0x71, 0x2c, 0x50, 0x9e, 0xd3, 0x15, 0x5a, 0xa3, 0x5d, 0xa1, 0x98, 0xef, 0xf7, 0xbd, 0xae, 0x50, 0x94, 0x37, 0x84, 0xde, 0xf7, 0x5b, 0x3e, 0xec, 0x80, 0x5f, 0xe3, 0xe1, 0x61, 0xf1, 0x9c, 0x6e, 0xcf, 0x53, 0x58, 0xec, 0x5b, 0x9a, 0xde, 0x25, 0x45, 0x0b, 0xd1, 0x0e, 0x57, 0xef, 0x63, 0x9e, 0xd2, 0xce, 0xe5, 0x3b, 0xa5, 0x20, 0x35, 0x19, 0x44, 0xbb, 0x50, @@ -4008,9 +4017,9 @@ var fileDescriptor_12dcc21c3bcc9571 = []byte{ 0xf5, 0x01, 0x5f, 0xf8, 0x61, 0x51, 0xe2, 0x02, 0x0f, 0x8b, 0x3e, 0x84, 0x1b, 0xba, 0xa3, 0x98, 0x96, 0x29, 0x2e, 0x18, 0xbc, 0xa6, 0x4b, 0xf0, 0x51, 0xdc, 0x35, 0xdd, 0x69, 0x5a, 0x26, 0xbb, 0x5a, 0xf0, 0x56, 0xed, 0xbf, 0x9f, 0x4b, 0x4d, 0xbe, 0x9f, 0xf3, 0x7a, 0xf4, 0x71, 0x29, 0xb1, - 0xfc, 0x6f, 0x11, 0xc8, 0x04, 0x9f, 0xa8, 0x47, 0xc3, 0x9d, 0xc3, 0x89, 0x5a, 0xf0, 0x92, 0x8f, + 0xfc, 0xaf, 0x11, 0xc8, 0x04, 0x9f, 0xa8, 0x47, 0xc3, 0x9d, 0xc3, 0x89, 0x5a, 0xf0, 0x92, 0x8f, 0x7c, 0xc2, 0xbb, 0x10, 0xbb, 0xc0, 0x2e, 0xdc, 0x86, 0xc4, 0xe1, 0x48, 0xd4, 0x68, 0xe9, 0x6a, - 0x8e, 0xcf, 0x16, 0xaf, 0x92, 0x7a, 0x20, 0x7e, 0x38, 0x12, 0x0f, 0xa6, 0x96, 0xff, 0x14, 0xb2, + 0x8e, 0xcf, 0x16, 0xaf, 0x92, 0x7a, 0x20, 0x7e, 0x38, 0x12, 0x0f, 0xa6, 0x96, 0xff, 0x18, 0xb2, 0x81, 0xb8, 0x3b, 0xde, 0x99, 0x88, 0x5c, 0xa2, 0x33, 0xf1, 0x06, 0x24, 0x79, 0x58, 0x60, 0xba, 0x97, 0xe7, 0xd4, 0x09, 0x16, 0x12, 0x12, 0x9f, 0x93, 0x70, 0xc0, 0x67, 0xff, 0x9f, 0x18, 0xe4, 0x82, 0x11, 0x94, 0xd8, 0xba, 0x6e, 0x76, 0x6c, 0x1a, 0xbe, 0xe8, 0xec, 0x31, 0xef, 0xb9, 0x91, @@ -4019,14 +4028,14 @@ var fileDescriptor_12dcc21c3bcc9571 = []byte{ 0xb5, 0x5d, 0xba, 0x43, 0xb1, 0x40, 0x0a, 0x6a, 0xbb, 0xc1, 0x57, 0x87, 0x89, 0x69, 0xaf, 0x0e, 0x4d, 0x28, 0xf8, 0x39, 0xc3, 0x89, 0x89, 0x6d, 0x7e, 0xdd, 0x50, 0xb9, 0x44, 0xd2, 0xe0, 0x7f, 0x10, 0x46, 0x22, 0x8a, 0x3b, 0x41, 0x20, 0xc9, 0x4a, 0x3b, 0x6a, 0xe7, 0x08, 0x2b, 0x8e, 0xfe, - 0x63, 0xd6, 0x0e, 0xf0, 0xb6, 0x85, 0xc2, 0xf7, 0xf5, 0x1f, 0xe3, 0xe5, 0xbf, 0x8b, 0x40, 0x3e, + 0x63, 0xd6, 0x0e, 0xf0, 0xb6, 0x85, 0xc2, 0xf7, 0xf5, 0x1f, 0xe3, 0xe5, 0xbf, 0x89, 0x40, 0x3e, 0xc4, 0x0b, 0x35, 0xa0, 0x48, 0xa5, 0x9b, 0x68, 0x6f, 0xdf, 0xf6, 0x1e, 0xad, 0x93, 0xe1, 0xa9, 0xc5, 0x6c, 0xde, 0x0a, 0x0c, 0x69, 0x24, 0x0f, 0x65, 0xac, 0xbc, 0xd7, 0x6d, 0x61, 0x35, 0xce, 0x51, 0x4e, 0xe1, 0x27, 0x6e, 0x39, 0xcb, 0x87, 0x69, 0xc1, 0x66, 0xfc, 0xb2, 0x09, 0xd9, 0x40, 0xe6, 0x32, 0x87, 0xfd, 0x7c, 0x0b, 0xe2, 0x9e, 0x37, 0x9b, 0xb7, 0x8b, 0xec, 0xfa, 0x2e, 0xee, 0x67, 0x11, 0x58, 0x9a, 0x96, 0x41, 0x84, 0xec, 0x92, 0x69, 0xdb, 0x5c, 0x76, 0x79, 0x27, 0x98, 0xd9, 0x31, 0x0d, 0x14, 0xaf, 0x22, 0xfc, 0xdc, 0xee, 0x4d, 0xcf, 0x0e, 0x98, 0x02, 0x16, 0x43, - 0x76, 0x40, 0x2a, 0xb8, 0xa0, 0x25, 0xfc, 0x2e, 0x06, 0x85, 0xb1, 0xdb, 0x97, 0x67, 0x90, 0xec, + 0x76, 0x40, 0x2a, 0xb8, 0xa0, 0x25, 0xfc, 0x36, 0x06, 0x85, 0xb1, 0xdb, 0x97, 0x67, 0x90, 0xec, 0x19, 0xd6, 0xa1, 0x6a, 0xf0, 0xae, 0xf5, 0xb7, 0x2f, 0x15, 0xca, 0x36, 0x1e, 0x51, 0x1e, 0x3b, 0x0b, 0x32, 0xe7, 0x86, 0x1c, 0x58, 0x0c, 0x5e, 0xb3, 0xb0, 0x5f, 0xd7, 0xb0, 0x9d, 0xad, 0x5f, 0x6e, 0x0a, 0xff, 0x1e, 0x86, 0x22, 0xee, 0x2c, 0xc8, 0x45, 0x3b, 0x0c, 0x42, 0x7d, 0x28, 0x8e, @@ -4036,64 +4045,66 @@ var fileDescriptor_12dcc21c3bcc9571 = []byte{ 0xaa, 0xec, 0x09, 0x61, 0x46, 0x8e, 0xaa, 0xfc, 0xf1, 0xe1, 0x72, 0x01, 0x92, 0x6c, 0x7f, 0x83, 0xfa, 0x5d, 0x05, 0x48, 0x8b, 0xfc, 0xa1, 0xbc, 0x0e, 0x19, 0x2f, 0x91, 0x46, 0x39, 0x48, 0xd7, 0x1a, 0xfb, 0x95, 0xea, 0x6e, 0xbd, 0x26, 0x2d, 0xa0, 0x3c, 0x64, 0xe4, 0x7a, 0xa5, 0x46, 0x7b, - 0xae, 0x52, 0xe4, 0xa3, 0xf4, 0x9f, 0xff, 0x6c, 0x35, 0xc2, 0x83, 0x4c, 0x52, 0x4a, 0x3d, 0x8e, - 0xa7, 0x91, 0x74, 0xad, 0xfc, 0xdb, 0x34, 0xa0, 0x9a, 0xea, 0xaa, 0x64, 0x53, 0x2e, 0xd0, 0x99, - 0x8c, 0x9e, 0x63, 0x4d, 0x53, 0x9b, 0x8c, 0xf1, 0xab, 0x34, 0x19, 0x2f, 0xd5, 0xeb, 0x9c, 0xec, - 0x4c, 0x26, 0xaf, 0xd0, 0x99, 0x0c, 0xf7, 0x7d, 0x62, 0x57, 0xea, 0xfb, 0x3c, 0x83, 0x14, 0xab, - 0x32, 0xd9, 0x1b, 0xb3, 0xd9, 0x6d, 0x85, 0xc9, 0x83, 0xe1, 0xdd, 0x1a, 0xa7, 0x6e, 0xba, 0xf6, - 0xc8, 0x7b, 0x0f, 0xc3, 0x60, 0x7e, 0x7b, 0x24, 0xfd, 0x2a, 0xdb, 0x23, 0x99, 0xd9, 0xed, 0x91, - 0x1f, 0x00, 0xb7, 0x0b, 0x91, 0x14, 0xc3, 0xb9, 0x4f, 0x43, 0xa6, 0x2c, 0x87, 0x19, 0x01, 0xcf, - 0x8a, 0x73, 0x76, 0xe0, 0x6b, 0xb9, 0x0d, 0xc0, 0xdb, 0xaf, 0x66, 0xd7, 0x9a, 0xc3, 0x89, 0xaf, - 0x40, 0x8a, 0x38, 0xc7, 0x01, 0x66, 0xda, 0xe9, 0x45, 0x55, 0x0e, 0xe4, 0x16, 0x35, 0x80, 0x5c, - 0x70, 0x0b, 0x91, 0x04, 0xb1, 0x63, 0x3c, 0xe2, 0x86, 0x47, 0xfe, 0x44, 0x8f, 0x21, 0xe1, 0xc7, - 0xfe, 0xd9, 0xcf, 0xb8, 0x67, 0x9e, 0x0d, 0x11, 0x57, 0x66, 0x2c, 0x3e, 0x8a, 0x3e, 0x8c, 0x2c, - 0xff, 0x57, 0x04, 0x72, 0xc1, 0x65, 0xa2, 0x26, 0xe4, 0x9d, 0xa1, 0xfd, 0x42, 0x7f, 0xa1, 0x1a, - 0x4a, 0xcf, 0x52, 0x0d, 0x3a, 0x51, 0x61, 0xf3, 0xce, 0xac, 0x67, 0x50, 0x1c, 0xf7, 0x91, 0xa5, - 0x1a, 0xa2, 0x71, 0xe1, 0x04, 0x60, 0xe8, 0x43, 0xef, 0xba, 0x8e, 0xdf, 0x6f, 0xf3, 0xab, 0x5f, - 0xc4, 0x8d, 0x24, 0xe8, 0x85, 0x44, 0x6f, 0x96, 0x81, 0x48, 0xdc, 0xe5, 0x07, 0x88, 0xe9, 0x13, - 0x65, 0xd1, 0x74, 0xf7, 0xe2, 0x2e, 0xc3, 0xab, 0x9b, 0xc3, 0xbe, 0x1f, 0x77, 0x6d, 0x1f, 0xe6, - 0xff, 0x74, 0x20, 0x22, 0x45, 0x7d, 0x0f, 0x53, 0xfe, 0x5d, 0x0e, 0x0a, 0xed, 0xd1, 0x60, 0x9a, - 0x47, 0x89, 0xcd, 0xf0, 0x28, 0xf1, 0xf9, 0xee, 0x3a, 0x32, 0x57, 0xbb, 0xeb, 0x80, 0x57, 0x7b, - 0xd7, 0x91, 0x7d, 0x65, 0x1e, 0xa5, 0x70, 0x25, 0x8f, 0xf2, 0xca, 0x6e, 0xbe, 0xa2, 0x97, 0xb8, - 0xf9, 0xfa, 0x0e, 0xe4, 0x55, 0xdb, 0x56, 0x47, 0xfc, 0xb7, 0x2d, 0x1a, 0x75, 0x3f, 0xfc, 0x8c, - 0xce, 0x4e, 0x57, 0xb3, 0x15, 0x32, 0x48, 0x7f, 0xce, 0x22, 0x38, 0x64, 0x55, 0x0f, 0xa4, 0xf9, - 0x5e, 0x2b, 0xff, 0x2a, 0xbd, 0x56, 0x71, 0xb6, 0xd7, 0xaa, 0x41, 0x9c, 0xfe, 0x78, 0x26, 0x41, - 0xe7, 0x9b, 0xb5, 0xe5, 0x61, 0xf5, 0xdd, 0x08, 0xfc, 0x7e, 0x86, 0x52, 0xa3, 0x1f, 0xc1, 0xb2, - 0x78, 0xa1, 0x4a, 0xf4, 0xc1, 0xbf, 0x99, 0x0c, 0xfc, 0x34, 0xa9, 0x7c, 0x76, 0xba, 0x5a, 0x92, - 0x7d, 0x2c, 0x9f, 0x1f, 0xab, 0xad, 0xc8, 0x5e, 0x94, 0xec, 0xa9, 0xe3, 0x9a, 0x83, 0xbe, 0x07, - 0x39, 0x6a, 0x95, 0x7d, 0xdc, 0x3f, 0xc4, 0xb6, 0x08, 0x5f, 0x0f, 0xe6, 0x93, 0x97, 0x98, 0xe7, - 0x1e, 0x25, 0x14, 0xfd, 0x28, 0xec, 0x41, 0x1c, 0xf4, 0x00, 0x12, 0xaa, 0xa1, 0xd3, 0xf8, 0xf3, - 0x55, 0x3f, 0x20, 0x63, 0x88, 0xec, 0x65, 0x6f, 0xd0, 0xd5, 0x4b, 0xe7, 0x77, 0x12, 0xc3, 0xd2, - 0x9c, 0xe3, 0xe6, 0x7f, 0x1a, 0x03, 0xf0, 0x85, 0x45, 0xdf, 0x82, 0x9b, 0x83, 0xa3, 0x91, 0xa3, - 0x77, 0x54, 0x43, 0xb1, 0xf1, 0xc0, 0xc6, 0x0e, 0x36, 0x59, 0x36, 0x4d, 0xf5, 0x3a, 0x27, 0xdf, - 0x10, 0xc3, 0x72, 0x68, 0x14, 0x7d, 0x0c, 0x37, 0x0c, 0xab, 0x37, 0x8d, 0x2e, 0xd8, 0x4b, 0xb8, - 0xce, 0x71, 0xc6, 0x88, 0x55, 0x52, 0x01, 0x0d, 0xd4, 0x43, 0xdd, 0xf0, 0xdb, 0x0b, 0x1f, 0x5f, - 0x74, 0xa3, 0x37, 0xb6, 0x3c, 0x16, 0xe2, 0xa9, 0x8a, 0xcf, 0x14, 0xfd, 0x70, 0xf2, 0xb6, 0xff, - 0xa3, 0x0b, 0xcf, 0x30, 0xfb, 0xd2, 0xbf, 0xfc, 0x06, 0x80, 0x3f, 0x3f, 0xbd, 0x44, 0xdf, 0xdd, - 0xf5, 0x93, 0x40, 0x7e, 0x1d, 0x5f, 0xbe, 0xf7, 0x15, 0x77, 0xee, 0x00, 0x49, 0xb9, 0xbe, 0xd7, - 0x7a, 0x56, 0x17, 0xb7, 0xee, 0xcb, 0xad, 0xb1, 0xe8, 0x35, 0x19, 0x6d, 0x22, 0x73, 0x46, 0x1b, - 0x7e, 0x11, 0xfe, 0x1e, 0xc4, 0x89, 0x31, 0x91, 0xd9, 0xeb, 0xcd, 0x83, 0x3d, 0x69, 0x01, 0x65, - 0x20, 0x51, 0xd9, 0x6d, 0x54, 0xf6, 0xa5, 0x08, 0x5a, 0x02, 0x69, 0xef, 0x60, 0xb7, 0xdd, 0x90, - 0xeb, 0x8f, 0x1a, 0xad, 0xa6, 0x42, 0x11, 0x82, 0x81, 0xe5, 0x6f, 0xe3, 0x20, 0x31, 0xc7, 0x33, - 0x25, 0xb4, 0x44, 0x2f, 0x71, 0x8d, 0xfe, 0x47, 0xcf, 0x99, 0xa6, 0x86, 0xa5, 0xc4, 0x2b, 0xca, - 0x8e, 0x93, 0x57, 0xc8, 0x8e, 0x53, 0xaf, 0xea, 0xde, 0x7e, 0xde, 0xf8, 0x13, 0x0e, 0x80, 0xf1, - 0xab, 0x04, 0xc0, 0x80, 0x86, 0xfc, 0x3c, 0x0a, 0x10, 0xd0, 0x8d, 0xef, 0x06, 0xff, 0x7d, 0x87, - 0xd9, 0x37, 0xc7, 0x63, 0xe5, 0xe0, 0xce, 0x82, 0xf8, 0xd7, 0x1f, 0x1e, 0x41, 0x5a, 0xe3, 0x99, - 0x1e, 0x4f, 0x08, 0xdf, 0x9e, 0x3b, 0x21, 0xdc, 0x59, 0x90, 0x3d, 0x62, 0xf4, 0x71, 0xe8, 0x27, - 0xbb, 0x77, 0xe7, 0x32, 0xfd, 0x1d, 0xf1, 0x64, 0xbf, 0x02, 0x49, 0x16, 0xa3, 0xf9, 0x36, 0xcd, - 0xfc, 0xc5, 0xe7, 0x98, 0x69, 0x90, 0xb2, 0x9c, 0x11, 0xf2, 0xd2, 0x31, 0x05, 0x89, 0xa1, 0xa9, - 0x5b, 0xe6, 0x3d, 0x39, 0xf8, 0xcc, 0x5c, 0xf4, 0x49, 0x89, 0xb7, 0xa0, 0x7f, 0xab, 0x2e, 0xd6, - 0xd8, 0x6b, 0x9e, 0x03, 0xf3, 0x85, 0x07, 0x88, 0xa0, 0x02, 0x00, 0x1f, 0xd7, 0xcd, 0x9e, 0x14, - 0xa5, 0x05, 0x27, 0x49, 0xaf, 0xc9, 0x57, 0xec, 0xde, 0x77, 0x40, 0x1a, 0xff, 0xc9, 0x69, 0xc0, - 0xc7, 0x2c, 0x42, 0x7e, 0xef, 0xd9, 0xd6, 0x56, 0xbb, 0xb1, 0x57, 0xdf, 0x6f, 0x57, 0xf6, 0x9e, - 0xb2, 0xf7, 0xcb, 0x6d, 0x52, 0xad, 0xb6, 0x1a, 0x35, 0x29, 0x7a, 0xef, 0x3b, 0x50, 0x1c, 0x33, - 0x33, 0xe2, 0x8e, 0x9e, 0x1e, 0x54, 0x77, 0x1b, 0x5b, 0x53, 0xdf, 0x05, 0xa1, 0x2c, 0xa4, 0x5a, - 0xdb, 0xdb, 0xbb, 0x8d, 0x66, 0x5d, 0x8a, 0xdd, 0xfb, 0x00, 0x72, 0xc1, 0x54, 0x19, 0x49, 0x90, - 0xfb, 0x7e, 0xab, 0x59, 0x57, 0xb6, 0x2b, 0x8d, 0xdd, 0x03, 0x99, 0x48, 0x80, 0xa0, 0xc0, 0xfd, - 0x8a, 0x80, 0x45, 0xaa, 0xeb, 0xbf, 0xfe, 0xcf, 0x95, 0x85, 0x5f, 0x9f, 0xad, 0x44, 0x7e, 0x73, - 0xb6, 0x12, 0xf9, 0xfd, 0xd9, 0x4a, 0xe4, 0x3f, 0xce, 0x56, 0x22, 0x7f, 0xf5, 0x87, 0x95, 0x85, - 0xdf, 0xfc, 0x61, 0x65, 0xe1, 0xf7, 0x7f, 0x58, 0x59, 0xf8, 0x7e, 0x92, 0xfd, 0xcb, 0x24, 0xff, - 0x1b, 0x00, 0x00, 0xff, 0xff, 0x8b, 0x2e, 0x44, 0x14, 0x04, 0x45, 0x00, 0x00, + 0xae, 0x52, 0xe4, 0xa3, 0xf4, 0x9f, 0xfe, 0x6c, 0x35, 0xc2, 0x83, 0x4c, 0x52, 0x4a, 0x3d, 0x8e, + 0xa7, 0x91, 0x74, 0xad, 0xfc, 0xf7, 0x19, 0x40, 0x35, 0xd5, 0x55, 0xc9, 0xa6, 0x5c, 0xa0, 0x33, + 0x19, 0x3d, 0xc7, 0x9a, 0xa6, 0x36, 0x19, 0xe3, 0x57, 0x69, 0x32, 0x5e, 0xaa, 0xd7, 0x39, 0xd9, + 0x99, 0x4c, 0x5e, 0xa1, 0x33, 0x19, 0xee, 0xfb, 0xc4, 0xae, 0xd4, 0xf7, 0x79, 0x06, 0x29, 0x56, + 0x65, 0xb2, 0x37, 0x66, 0xb3, 0xdb, 0x0a, 0x93, 0x07, 0xc3, 0xbb, 0x35, 0x4e, 0xdd, 0x74, 0xed, + 0x91, 0xf7, 0x1e, 0x86, 0xc1, 0xfc, 0xf6, 0x48, 0xfa, 0x55, 0xb6, 0x47, 0x32, 0xb3, 0xdb, 0x23, + 0x3f, 0x00, 0x6e, 0x17, 0x22, 0x29, 0x86, 0x73, 0x9f, 0x86, 0x4c, 0x59, 0x0e, 0x33, 0x02, 0x9e, + 0x15, 0xe7, 0xec, 0xc0, 0x17, 0xfa, 0x11, 0x20, 0x71, 0x2f, 0x1b, 0xd8, 0x79, 0x76, 0xa5, 0xf3, + 0xde, 0xcc, 0xa5, 0x51, 0x82, 0x69, 0x07, 0x20, 0xde, 0x83, 0x7b, 0x63, 0xce, 0x72, 0x1b, 0x80, + 0x37, 0x78, 0xcd, 0xae, 0x35, 0x47, 0x98, 0x58, 0x81, 0x14, 0x71, 0xbf, 0x03, 0xcc, 0xf4, 0xdf, + 0x8b, 0xdb, 0x1c, 0xc8, 0x6d, 0x76, 0x00, 0xb9, 0xe0, 0x21, 0x21, 0x09, 0x62, 0xc7, 0x78, 0xc4, + 0x4d, 0x9b, 0xfc, 0x89, 0x1e, 0x43, 0xc2, 0xcf, 0x2e, 0x66, 0x3f, 0x14, 0x9f, 0x79, 0xfa, 0x44, + 0x5c, 0x99, 0xb1, 0xf8, 0x28, 0xfa, 0x30, 0xb2, 0xfc, 0x5f, 0x11, 0xc8, 0x05, 0x37, 0x12, 0x35, + 0x21, 0xef, 0x0c, 0xed, 0x17, 0xfa, 0x0b, 0xd5, 0x50, 0x7a, 0x96, 0x6a, 0xd0, 0x89, 0x0a, 0x9b, + 0x77, 0x66, 0x3d, 0xb4, 0xe2, 0xb8, 0x8f, 0x2c, 0xd5, 0x10, 0xad, 0x11, 0x27, 0x00, 0x43, 0x1f, + 0x7a, 0x17, 0x82, 0xfc, 0x06, 0x9d, 0x5f, 0x2e, 0x23, 0x6e, 0x86, 0x41, 0x3f, 0x27, 0xba, 0xbf, + 0x0c, 0x44, 0x22, 0x3b, 0x57, 0x11, 0x4c, 0x1f, 0x41, 0x8b, 0xb6, 0xbe, 0x17, 0xd9, 0x19, 0x5e, + 0xdd, 0x1c, 0xf6, 0xfd, 0xc8, 0x6e, 0xfb, 0x30, 0xff, 0xc7, 0x09, 0x11, 0x29, 0xea, 0xfb, 0xb0, + 0xf2, 0x6f, 0x73, 0x50, 0x68, 0x8f, 0x06, 0xd3, 0x7c, 0x56, 0x6c, 0x86, 0xcf, 0x8a, 0xcf, 0x77, + 0x9b, 0x92, 0xb9, 0xda, 0x6d, 0x0a, 0xbc, 0xda, 0xdb, 0x94, 0xec, 0x2b, 0xf3, 0x59, 0x85, 0x2b, + 0xf9, 0xac, 0x57, 0x76, 0xb7, 0x16, 0xbd, 0xc4, 0xdd, 0xda, 0x77, 0x20, 0xaf, 0xda, 0xb6, 0x3a, + 0xe2, 0xbf, 0x9e, 0xd1, 0xa8, 0x83, 0xe3, 0x67, 0x74, 0x76, 0xba, 0x9a, 0xad, 0x90, 0x41, 0xfa, + 0x83, 0x19, 0xc1, 0x21, 0xab, 0x7a, 0x20, 0xcd, 0xf7, 0x8b, 0xf9, 0x57, 0xe9, 0x17, 0x8b, 0xb3, + 0xfd, 0x62, 0x0d, 0xe2, 0xf4, 0xe7, 0x39, 0x09, 0x3a, 0xdf, 0xac, 0x2d, 0x0f, 0xab, 0xef, 0x46, + 0xe0, 0x17, 0x3a, 0x94, 0x1a, 0xfd, 0x08, 0x96, 0xc5, 0x1b, 0x58, 0xa2, 0x0f, 0xfe, 0xdd, 0x67, + 0xe0, 0xc7, 0x4f, 0xe5, 0xb3, 0xd3, 0xd5, 0x92, 0xec, 0x63, 0xf9, 0xfc, 0x58, 0xf5, 0x46, 0xf6, + 0xa2, 0x64, 0x4f, 0x1d, 0xd7, 0x1c, 0xf4, 0x3d, 0xc8, 0x51, 0xab, 0xec, 0xe3, 0xfe, 0x21, 0xb6, + 0x45, 0x80, 0x7c, 0x30, 0x9f, 0xbc, 0xc4, 0x3c, 0xf7, 0x28, 0xa1, 0xe8, 0x78, 0x61, 0x0f, 0xe2, + 0xa0, 0x07, 0x90, 0x50, 0x0d, 0x9d, 0x46, 0xb8, 0xaf, 0xfa, 0x89, 0x1a, 0x43, 0x64, 0x6f, 0x87, + 0x83, 0xc1, 0x44, 0x3a, 0xbf, 0x57, 0x19, 0x96, 0x66, 0x76, 0x20, 0x59, 0xfe, 0x69, 0x0c, 0xc0, + 0x17, 0x16, 0x7d, 0x0b, 0x6e, 0x0e, 0x8e, 0x46, 0x8e, 0xde, 0x51, 0x0d, 0xc5, 0xc6, 0x03, 0x1b, + 0x3b, 0xd8, 0x64, 0xf9, 0x3a, 0xd5, 0xeb, 0x9c, 0x7c, 0x43, 0x0c, 0xcb, 0xa1, 0x51, 0xf4, 0x31, + 0xdc, 0x30, 0xac, 0xde, 0x34, 0xba, 0x60, 0xb7, 0xe2, 0x3a, 0xc7, 0x19, 0x23, 0x56, 0x49, 0x8d, + 0x35, 0x50, 0x0f, 0x75, 0xc3, 0x6f, 0x60, 0x7c, 0x7c, 0xd1, 0x8d, 0xde, 0xd8, 0xf2, 0x58, 0x88, + 0xc7, 0x30, 0x3e, 0x53, 0xf4, 0xc3, 0xc9, 0xf7, 0x04, 0x1f, 0x5d, 0x78, 0x86, 0xd9, 0xcf, 0x0a, + 0xca, 0x6f, 0x00, 0xf8, 0xf3, 0xd3, 0x6b, 0xfa, 0xdd, 0x5d, 0x3f, 0xcd, 0xe4, 0x17, 0xfe, 0xe5, + 0x7b, 0x5f, 0x71, 0xab, 0x0f, 0x90, 0x94, 0xeb, 0x7b, 0xad, 0x67, 0x75, 0x71, 0xaf, 0xbf, 0xdc, + 0x1a, 0x8b, 0x5e, 0x93, 0xd1, 0x26, 0x32, 0x67, 0xb4, 0xe1, 0x57, 0xed, 0xef, 0x41, 0x9c, 0x18, + 0x13, 0x99, 0xbd, 0xde, 0x3c, 0xd8, 0x93, 0x16, 0x50, 0x06, 0x12, 0x95, 0xdd, 0x46, 0x65, 0x5f, + 0x8a, 0xa0, 0x25, 0x90, 0xf6, 0x0e, 0x76, 0xdb, 0x0d, 0xb9, 0xfe, 0xa8, 0xd1, 0x6a, 0x2a, 0x14, + 0x21, 0x18, 0x58, 0xfe, 0x3a, 0x0e, 0x12, 0x73, 0x3c, 0x53, 0x42, 0x4b, 0xf4, 0x12, 0x17, 0xf5, + 0x7f, 0xf0, 0xac, 0x6c, 0x6a, 0x58, 0x4a, 0xbc, 0xa2, 0xfc, 0x3b, 0x79, 0x85, 0xfc, 0x3b, 0xf5, + 0xaa, 0x5e, 0x06, 0xcc, 0x1b, 0x7f, 0xc2, 0x01, 0x30, 0x7e, 0x95, 0x00, 0x18, 0xd0, 0x90, 0x9f, + 0x47, 0x01, 0x02, 0xba, 0xf1, 0xdd, 0xe0, 0xbf, 0x20, 0x31, 0xfb, 0x6e, 0x7a, 0xac, 0xe0, 0xdc, + 0x59, 0x10, 0xff, 0xbe, 0xc4, 0x23, 0x48, 0x6b, 0x3c, 0xd3, 0xe3, 0x09, 0xe1, 0xdb, 0x73, 0x27, + 0x84, 0x3b, 0x0b, 0xb2, 0x47, 0x8c, 0x3e, 0x0e, 0xfd, 0x28, 0xf8, 0xee, 0x5c, 0xa6, 0xbf, 0x23, + 0x7e, 0x14, 0x50, 0x81, 0x24, 0x8b, 0xd1, 0x7c, 0x9b, 0x66, 0xfe, 0xa6, 0x74, 0xcc, 0x34, 0x48, + 0xe1, 0xcf, 0x08, 0x79, 0x71, 0x9a, 0x82, 0xc4, 0xd0, 0xd4, 0x2d, 0xf3, 0x9e, 0x1c, 0x7c, 0xc8, + 0x2e, 0x3a, 0xb1, 0xc4, 0x5b, 0xd0, 0xbf, 0x55, 0x17, 0x6b, 0xec, 0xbd, 0xd0, 0x81, 0xf9, 0xc2, + 0x03, 0x44, 0x50, 0x01, 0x80, 0x8f, 0xeb, 0x66, 0x4f, 0x8a, 0xd2, 0x92, 0x96, 0xa4, 0xd7, 0xe4, + 0x2b, 0x76, 0xef, 0x3b, 0x20, 0x8d, 0xff, 0xa8, 0x35, 0xe0, 0x63, 0x16, 0x21, 0xbf, 0xf7, 0x6c, + 0x6b, 0xab, 0xdd, 0xd8, 0xab, 0xef, 0xb7, 0x2b, 0x7b, 0x4f, 0xd9, 0x0b, 0xe9, 0x36, 0xa9, 0x87, + 0x5b, 0x8d, 0x9a, 0x14, 0xbd, 0xf7, 0x1d, 0x28, 0x8e, 0x99, 0x19, 0x71, 0x47, 0x4f, 0x0f, 0xaa, + 0xbb, 0x8d, 0xad, 0xa9, 0x2f, 0x8f, 0x50, 0x16, 0x52, 0xad, 0xed, 0xed, 0xdd, 0x46, 0xb3, 0x2e, + 0xc5, 0xee, 0x7d, 0x00, 0xb9, 0x60, 0xaa, 0x8c, 0x24, 0xc8, 0x7d, 0xbf, 0xd5, 0xac, 0x2b, 0xdb, + 0x95, 0xc6, 0xee, 0x81, 0x4c, 0x24, 0x40, 0x50, 0xe0, 0x7e, 0x45, 0xc0, 0x22, 0xd5, 0xf5, 0x5f, + 0xff, 0xe7, 0xca, 0xc2, 0xaf, 0xcf, 0x56, 0x22, 0xbf, 0x39, 0x5b, 0x89, 0xfc, 0xee, 0x6c, 0x25, + 0xf2, 0x1f, 0x67, 0x2b, 0x91, 0xbf, 0xf8, 0xfd, 0xca, 0xc2, 0x6f, 0x7e, 0xbf, 0xb2, 0xf0, 0xbb, + 0xdf, 0xaf, 0x2c, 0x7c, 0x3f, 0xc9, 0xfe, 0xed, 0x93, 0xff, 0x0d, 0x00, 0x00, 0xff, 0xff, 0xb2, + 0x6d, 0x9c, 0x4a, 0x66, 0x45, 0x00, 0x00, } func (this *ForeignKeyReference) Equal(that interface{}) bool { @@ -5854,6 +5865,9 @@ func (this *DatabaseDescriptor) Equal(that interface{}) bool { if !this.RegionConfig.Equal(that1.RegionConfig) { return false } + if !this.DefaultPrivileges.Equal(that1.DefaultPrivileges) { + return false + } return true } func (this *DatabaseDescriptor_SchemaInfo) Equal(that interface{}) bool { @@ -8476,6 +8490,18 @@ func (m *DatabaseDescriptor) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.DefaultPrivileges != nil { + { + size, err := m.DefaultPrivileges.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintStructured(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x5a + } if m.RegionConfig != nil { { size, err := m.RegionConfig.MarshalToSizedBuffer(dAtA[:i]) @@ -9946,6 +9972,10 @@ func (m *DatabaseDescriptor) Size() (n int) { l = m.RegionConfig.Size() n += 1 + l + sovStructured(uint64(l)) } + if m.DefaultPrivileges != nil { + l = m.DefaultPrivileges.Size() + n += 1 + l + sovStructured(uint64(l)) + } return n } @@ -18062,6 +18092,42 @@ func (m *DatabaseDescriptor) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DefaultPrivileges", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStructured + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthStructured + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthStructured + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.DefaultPrivileges == nil { + m.DefaultPrivileges = &DefaultPrivilegeDescriptor{} + } + if err := m.DefaultPrivileges.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipStructured(dAtA[iNdEx:]) diff --git a/pkg/sql/catalog/descpb/structured.proto b/pkg/sql/catalog/descpb/structured.proto index a305aba36d1b..f05221d4314a 100644 --- a/pkg/sql/catalog/descpb/structured.proto +++ b/pkg/sql/catalog/descpb/structured.proto @@ -1254,6 +1254,9 @@ message DatabaseDescriptor { } // RegionConfig is only set if multi-region controls are set on the database. optional RegionConfig region_config = 10; + + // DefaultPrivileges contains the default privileges for the database. + optional DefaultPrivilegeDescriptor default_privileges = 11; } // TypeDescriptor represents a user defined type and is stored in a structured diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 0221eaa11a64..18435781c76e 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -178,6 +178,7 @@ type DatabaseDescriptor interface { ForEachSchemaInfo(func(id descpb.ID, name string, isDropped bool) error) error GetSchemaID(name string) descpb.ID GetNonDroppedSchemaName(schemaID descpb.ID) string + GetDefaultPrivilegeDescriptor() *descpb.DefaultPrivilegeDescriptor } // TableDescriptor is an interface around the table descriptor types. diff --git a/pkg/sql/catalog/tabledesc/validate_test.go b/pkg/sql/catalog/tabledesc/validate_test.go index f1388daca5f7..4517f17d4a7b 100644 --- a/pkg/sql/catalog/tabledesc/validate_test.go +++ b/pkg/sql/catalog/tabledesc/validate_test.go @@ -244,16 +244,17 @@ var validationMap = []struct { { obj: descpb.DatabaseDescriptor{}, fieldMap: map[string]validationStatusInfo{ - "Name": {status: iSolemnlySwearThisFieldIsValidated}, - "ID": {status: iSolemnlySwearThisFieldIsValidated}, - "Version": {status: thisFieldReferencesNoObjects}, - "ModificationTime": {status: thisFieldReferencesNoObjects}, - "DrainingNames": {status: thisFieldReferencesNoObjects}, - "Privileges": {status: iSolemnlySwearThisFieldIsValidated}, - "Schemas": {status: iSolemnlySwearThisFieldIsValidated}, - "State": {status: thisFieldReferencesNoObjects}, - "OfflineReason": {status: thisFieldReferencesNoObjects}, - "RegionConfig": {status: iSolemnlySwearThisFieldIsValidated}, + "Name": {status: iSolemnlySwearThisFieldIsValidated}, + "ID": {status: iSolemnlySwearThisFieldIsValidated}, + "Version": {status: thisFieldReferencesNoObjects}, + "ModificationTime": {status: thisFieldReferencesNoObjects}, + "DrainingNames": {status: thisFieldReferencesNoObjects}, + "Privileges": {status: iSolemnlySwearThisFieldIsValidated}, + "Schemas": {status: iSolemnlySwearThisFieldIsValidated}, + "State": {status: thisFieldReferencesNoObjects}, + "OfflineReason": {status: thisFieldReferencesNoObjects}, + "RegionConfig": {status: iSolemnlySwearThisFieldIsValidated}, + "DefaultPrivileges": {status: iSolemnlySwearThisFieldIsValidated}, }, }, { diff --git a/pkg/sql/create_schema.go b/pkg/sql/create_schema.go index a472341896a7..bc5e361f351d 100644 --- a/pkg/sql/create_schema.go +++ b/pkg/sql/create_schema.go @@ -31,7 +31,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" - "github.com/cockroachdb/cockroach/pkg/util/protoutil" ) type createSchemaNode struct { @@ -113,12 +112,9 @@ func CreateUserDefinedSchemaDescriptor( } } - // Inherit the parent privileges and filter out those which are not valid for - // schemas. - privs := protoutil.Clone(db.GetPrivileges()).(*descpb.PrivilegeDescriptor) - for i := range privs.Users { - privs.Users[i].Privileges &= privilege.SchemaPrivileges.ToBitField() - } + privs := descpb.CreatePrivilegesFromDefaultPrivileges( + db.GetID(), db.GetDefaultPrivilegeDescriptor(), user, tree.Schemas, db.GetPrivileges(), + ) if !n.AuthRole.Undefined() { exists, err := RoleExists(ctx, execCfg, txn, n.AuthRole) diff --git a/pkg/sql/create_sequence.go b/pkg/sql/create_sequence.go index 0133514d9139..6c7e6bfc956f 100644 --- a/pkg/sql/create_sequence.go +++ b/pkg/sql/create_sequence.go @@ -100,7 +100,10 @@ func doCreateSequence( return err } - privs := CreateInheritedPrivilegesFromDBDesc(dbDesc, params.SessionData().User()) + privs := descpb.CreatePrivilegesFromDefaultPrivileges( + dbDesc.GetID(), dbDesc.GetDefaultPrivilegeDescriptor(), + params.SessionData().User(), tree.Sequences, dbDesc.GetPrivileges(), + ) if persistence.IsTemporary() { telemetry.Inc(sqltelemetry.CreateTempSequenceCounter) diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 0a2f915f5621..c233abbcc8b6 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -21,9 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/geo/geoindex" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -41,7 +39,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" - "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -302,8 +299,10 @@ func (n *createTableNode) startExec(params runParams) error { return err } - privs := CreateInheritedPrivilegesFromDBDesc(n.dbDesc, params.SessionData().User()) - + privs := descpb.CreatePrivilegesFromDefaultPrivileges( + n.dbDesc.GetID(), n.dbDesc.GetDefaultPrivilegeDescriptor(), + params.SessionData().User(), tree.Tables, n.dbDesc.GetPrivileges(), + ) var desc *tabledesc.Mutable var affected map[descpb.ID]*tabledesc.Mutable // creationTime is initialized to a zero value and populated at read time. @@ -2777,30 +2776,6 @@ func incTelemetryForNewColumn(def *tree.ColumnTableDef, desc *descpb.ColumnDescr } } -// CreateInheritedPrivilegesFromDBDesc creates privileges for a -// table (or view/sequence) with the appropriate owner (node for system, -// the restoring user otherwise.) -func CreateInheritedPrivilegesFromDBDesc( - dbDesc catalog.DatabaseDescriptor, user security.SQLUsername, -) *descpb.PrivilegeDescriptor { - // If a new system table is being created (which should only be doable by - // an internal user account), make sure it gets the correct privileges. - if dbDesc.GetID() == keys.SystemDatabaseID { - return descpb.NewDefaultPrivilegeDescriptor(security.NodeUserName()) - } - - privs := dbDesc.GetPrivileges() - tablePrivBits := privilege.GetValidPrivilegesForObject(privilege.Table).ToBitField() - for i, u := range privs.Users { - // Remove privileges that are valid for databases but not for tables. - privs.Users[i].Privileges = u.Privileges & tablePrivBits - } - - privs.SetOwner(user) - - return privs -} - func regionalByRowRegionDefaultExpr(oid oid.Oid, region tree.Name) tree.Expr { return &tree.CastExpr{ Expr: tree.NewDString(string(region)), diff --git a/pkg/sql/create_type.go b/pkg/sql/create_type.go index 692861c62924..a72dcf3545ed 100644 --- a/pkg/sql/create_type.go +++ b/pkg/sql/create_type.go @@ -328,12 +328,13 @@ func (p *planner) createEnumWithID( } } - // Database privileges and Type privileges do not overlap so there is nothing - // to inherit. - // However having USAGE on a parent schema of the type - // gives USAGE privilege to the type. - privs := descpb.NewDefaultPrivilegeDescriptor(params.p.User()) + privs := descpb.CreatePrivilegesFromDefaultPrivileges( + dbDesc.GetID(), dbDesc.GetDefaultPrivilegeDescriptor(), + params.p.User(), tree.Types, dbDesc.GetPrivileges(), + ) + // TODO(richardjcai): Remove this once we figure out the migration from + // our current "inheritance" model to default privileges. inheritUsagePrivilegeFromSchema(schema, privs) privs.Grant(params.p.User(), privilege.List{privilege.ALL}) diff --git a/pkg/sql/create_view.go b/pkg/sql/create_view.go index 147c3add2932..6f66548ab133 100644 --- a/pkg/sql/create_view.go +++ b/pkg/sql/create_view.go @@ -160,7 +160,10 @@ func (n *createViewNode) startExec(params runParams) error { telemetry.Inc(sqltelemetry.CreateTempViewCounter) } - privs := CreateInheritedPrivilegesFromDBDesc(n.dbDesc, params.SessionData().User()) + privs := descpb.CreatePrivilegesFromDefaultPrivileges( + n.dbDesc.GetID(), n.dbDesc.GetDefaultPrivilegeDescriptor(), + params.SessionData().User(), tree.Tables, n.dbDesc.GetPrivileges(), + ) var newDesc *tabledesc.Mutable applyGlobalMultiRegionZoneConfig := false diff --git a/pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_schema b/pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_schema new file mode 100644 index 000000000000..f9958d4e41fd --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_schema @@ -0,0 +1,96 @@ +statement ok +CREATE DATABASE d + +statement ok +CREATE USER testuser2 + +statement ok +ALTER DEFAULT PRIVILEGES GRANT ALL ON SCHEMAS TO testuser, testuser2 + +statement ok +CREATE SCHEMA s + +query TTTT colnames +SHOW GRANTS ON SCHEMA s +---- +database_name schema_name grantee privilege_type +test s admin ALL +test s root ALL +test s testuser ALL +test s testuser2 ALL + +statement ok +ALTER DEFAULT PRIVILEGES REVOKE USAGE ON SCHEMAS FROM testuser, testuser2 + +statement ok +CREATE SCHEMA s2 + +query TTTT colnames +SHOW GRANTS ON SCHEMA s2 +---- +database_name schema_name grantee privilege_type +test s2 admin ALL +test s2 root ALL +test s2 testuser CREATE +test s2 testuser GRANT +test s2 testuser2 CREATE +test s2 testuser2 GRANT + +statement ok +ALTER DEFAULT PRIVILEGES REVOKE ALL ON SCHEMAS FROM testuser, testuser2 + +statement ok +CREATE SCHEMA s3 + +query TTTT colnames +SHOW GRANTS ON SCHEMA s3 +---- +database_name schema_name grantee privilege_type +test s3 admin ALL +test s3 root ALL + +statement ok +GRANT CREATE ON DATABASE d TO testuser + +user testuser +statement ok +USE d + +statement ok +ALTER DEFAULT PRIVILEGES FOR ROLE testuser REVOKE ALL ON SCHEMAS FROM testuser, testuser2 + +statement ok +CREATE SCHEMA s4 + +query TTTT colnames +SHOW GRANTS ON SCHEMA s4 +---- +database_name schema_name grantee privilege_type +d s4 admin ALL +d s4 root ALL +d s4 testuser CREATE + +user root +statement ok +USE d + +statement ok +GRANT testuser TO root + +statement ok +ALTER DEFAULT PRIVILEGES FOR ROLE testuser REVOKE ALL ON SCHEMAS FROM testuser, testuser2 + +user testuser +statement ok +USE d + +statement ok +CREATE SCHEMA s5 + +query TTTT colnames +SHOW GRANTS ON SCHEMA s5 +---- +database_name schema_name grantee privilege_type +d s5 admin ALL +d s5 root ALL +d s5 testuser CREATE diff --git a/pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_sequence b/pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_sequence new file mode 100644 index 000000000000..5e5c2f8165f1 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_sequence @@ -0,0 +1,118 @@ +statement ok +CREATE DATABASE d + +statement ok +CREATE USER testuser2 + +statement ok +ALTER DEFAULT PRIVILEGES GRANT ALL ON SEQUENCES TO testuser, testuser2 + +statement ok +CREATE SEQUENCE s + +query TTTTT colnames +SHOW GRANTS ON s +---- +database_name schema_name table_name grantee privilege_type +test public s admin ALL +test public s root ALL +test public s testuser ALL +test public s testuser2 ALL + +# Sequence DEFAULT PRIVILEGES should be separate from tables. +statement ok +CREATE TABLE t() + +query TTTTT colnames +SHOW GRANTS ON t +---- +database_name schema_name table_name grantee privilege_type +test public t admin ALL +test public t root ALL + + +statement ok +ALTER DEFAULT PRIVILEGES REVOKE SELECT ON SEQUENCES FROM testuser, testuser2 + +statement ok +CREATE SEQUENCE s2 + +query TTTTT colnames +SHOW GRANTS ON s2 +---- +database_name schema_name table_name grantee privilege_type +test public s2 admin ALL +test public s2 root ALL +test public s2 testuser CREATE +test public s2 testuser DELETE +test public s2 testuser DROP +test public s2 testuser GRANT +test public s2 testuser INSERT +test public s2 testuser UPDATE +test public s2 testuser ZONECONFIG +test public s2 testuser2 CREATE +test public s2 testuser2 DELETE +test public s2 testuser2 DROP +test public s2 testuser2 GRANT +test public s2 testuser2 INSERT +test public s2 testuser2 UPDATE +test public s2 testuser2 ZONECONFIG + +statement ok +ALTER DEFAULT PRIVILEGES REVOKE ALL ON SEQUENCES FROM testuser, testuser2 + +statement ok +CREATE SEQUENCE s3 + +query TTTTT colnames +SHOW GRANTS ON s3 +---- +database_name schema_name table_name grantee privilege_type +test public s3 admin ALL +test public s3 root ALL + +statement ok +GRANT CREATE ON DATABASE d TO testuser + +user testuser +statement ok +USE d + +statement ok +ALTER DEFAULT PRIVILEGES FOR ROLE testuser REVOKE ALL ON SEQUENCES FROM testuser, testuser2 + +statement ok +CREATE SEQUENCE s4 + +query TTTTT colnames +SHOW GRANTS ON s4 +---- +database_name schema_name table_name grantee privilege_type +d public s4 admin ALL +d public s4 root ALL +d public s4 testuser CREATE + +user root +statement ok +USE d + +statement ok +GRANT testuser TO root + +statement ok +ALTER DEFAULT PRIVILEGES FOR ROLE testuser REVOKE ALL ON SEQUENCES FROM testuser, testuser2 + +user testuser +statement ok +USE d + +statement ok +CREATE SEQUENCE s5 + +query TTTTT colnames +SHOW GRANTS ON s5 +---- +database_name schema_name table_name grantee privilege_type +d public s5 admin ALL +d public s5 root ALL +d public s5 testuser CREATE diff --git a/pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_table b/pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_table new file mode 100644 index 000000000000..8292527cba07 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_table @@ -0,0 +1,307 @@ +# Should error when a role that does not exist is provided. +statement error pq: role who does not exist +ALTER DEFAULT PRIVILEGES FOR ROLE who GRANT SELECT ON TABLES to testuser + +statement error pq: role who does not exist +ALTER DEFAULT PRIVILEGES GRANT SELECT ON TABLES to who + +statement error pq: role who does not exist +ALTER DEFAULT PRIVILEGES FOR ROLE testuser GRANT SELECT ON TABLES to who + +statement error pq: role who does not exist +ALTER DEFAULT PRIVILEGES FOR ROLE testuser GRANT SELECT ON TABLES to testuser, who + +# Should not be able to use invalid privileges. +statement error pq: invalid privilege type USAGE for table +ALTER DEFAULT PRIVILEGES GRANT USAGE ON TABLES to testuser + +# For Tables. +statement ok +CREATE DATABASE d; +USE d; + +statement ok +ALTER DEFAULT PRIVILEGES GRANT SELECT ON TABLES to testuser + +statement ok +CREATE TABLE t() + +query TTTTT colnames +SHOW GRANTS ON t +---- +database_name schema_name table_name grantee privilege_type +d public t admin ALL +d public t root ALL +d public t testuser SELECT + +statement ok +CREATE SEQUENCE s + +statement ok +CREATE VIEW vx AS SELECT 1 + +query TTTTT colnames +SHOW GRANTS ON s +---- +database_name schema_name table_name grantee privilege_type +d public s admin ALL +d public s root ALL + +query TTTTT colnames +SHOW GRANTS ON vx +---- +database_name schema_name table_name grantee privilege_type +d public vx admin ALL +d public vx root ALL +d public vx testuser SELECT + +statement ok +ALTER DEFAULT PRIVILEGES REVOKE SELECT ON TABLES FROM testuser + +statement ok +CREATE TABLE t2() + +query TTTTT colnames +SHOW GRANTS ON t2 +---- +database_name schema_name table_name grantee privilege_type +d public t2 admin ALL +d public t2 root ALL + +statement ok +CREATE SEQUENCE s2 + +query TTTTT colnames +SHOW GRANTS ON s2 +---- +database_name schema_name table_name grantee privilege_type +d public s2 admin ALL +d public s2 root ALL + + +# Multiple users. +statement ok +CREATE USER testuser2; + +statement ok +ALTER DEFAULT PRIVILEGES GRANT SELECT ON TABLES TO testuser, testuser2 + +statement ok +CREATE TABLE t3() + +query TTTTT colnames +SHOW GRANTS ON t3 +---- +database_name schema_name table_name grantee privilege_type +d public t3 admin ALL +d public t3 root ALL +d public t3 testuser SELECT +d public t3 testuser2 SELECT + +statement ok +CREATE SEQUENCE s3 + +query TTTTT colnames +SHOW GRANTS ON s3 +---- +database_name schema_name table_name grantee privilege_type +d public s3 admin ALL +d public s3 root ALL + +statement ok +ALTER DEFAULT PRIVILEGES REVOKE SELECT ON TABLES FROM testuser, testuser2 + +statement ok +CREATE TABLE t4() + +query TTTTT colnames +SHOW GRANTS ON t4 +---- +database_name schema_name table_name grantee privilege_type +d public t4 admin ALL +d public t4 root ALL + +statement ok +CREATE SEQUENCE s4 + +query TTTTT colnames +SHOW GRANTS ON s4 +---- +database_name schema_name table_name grantee privilege_type +d public s4 admin ALL +d public s4 root ALL + +# ALTER DEFAULT PRIVILEGES FOR ROLE. + +statement ok +GRANT CREATE ON DATABASE d TO testuser + +statement ok +GRANT testuser TO root + +statement ok +ALTER DEFAULT PRIVILEGES FOR ROLE testuser GRANT SELECT ON TABLES to testuser, testuser2 + +user testuser + +statement ok +USE d; + +statement ok +CREATE TABLE t5() + +query TTTTT colnames +SHOW GRANTS ON t5 +---- +database_name schema_name table_name grantee privilege_type +d public t5 admin ALL +d public t5 root ALL +d public t5 testuser CREATE +d public t5 testuser SELECT +d public t5 testuser2 SELECT + +user root + +statement ok +USE d; + +statement ok +ALTER DEFAULT PRIVILEGES FOR ROLE testuser REVOKE SELECT ON TABLES FROM testuser, testuser2 + +user testuser + +statement ok +USE d; + +statement ok +CREATE TABLE t6() + +query TTTTT colnames +SHOW GRANTS ON t6 +---- +database_name schema_name table_name grantee privilege_type +d public t6 admin ALL +d public t6 root ALL +d public t6 testuser CREATE + +user root + +statement ok +ALTER DEFAULT PRIVILEGES GRANT ALL ON TABLES TO testuser, testuser2 + +statement ok +CREATE TABLE t7() + +query TTTTT colnames +SHOW GRANTS ON t7 +---- +database_name schema_name table_name grantee privilege_type +d public t7 admin ALL +d public t7 root ALL +d public t7 testuser ALL +d public t7 testuser2 ALL + +statement ok +ALTER DEFAULT PRIVILEGES REVOKE SELECT ON TABLES FROM testuser, testuser2 + +statement ok +CREATE TABLE t8() + +query TTTTT colnames +SHOW GRANTS ON t8 +---- +database_name schema_name table_name grantee privilege_type +d public t8 admin ALL +d public t8 root ALL +d public t8 testuser CREATE +d public t8 testuser DELETE +d public t8 testuser DROP +d public t8 testuser GRANT +d public t8 testuser INSERT +d public t8 testuser UPDATE +d public t8 testuser ZONECONFIG +d public t8 testuser2 CREATE +d public t8 testuser2 DELETE +d public t8 testuser2 DROP +d public t8 testuser2 GRANT +d public t8 testuser2 INSERT +d public t8 testuser2 UPDATE +d public t8 testuser2 ZONECONFIG + +# Ensure you can only ALTER DEFAULT PRIVILEGES for a role you're a part of. +user testuser2 + +statement ok +USE d + +statement error pq: must be a member of root +ALTER DEFAULT PRIVILEGES FOR ROLE root GRANT SELECT ON TABLES TO testuser + +# Ensure you can ALTER DEFAULT PRIVILEGES for multiple roles. +user root + +statement ok +CREATE USER testuser3 + +statement ok +ALTER DEFAULT PRIVILEGES FOR ROLE root, testuser REVOKE ALL ON TABLES FROM testuser, testuser2, testuser3 + +statement ok +ALTER DEFAULT PRIVILEGES FOR ROLE root, testuser GRANT SELECT ON TABLES TO testuser2, testuser3 + +statement ok +CREATE TABLE t9() + +query TTTTT colnames +SHOW GRANTS ON t9 +---- +database_name schema_name table_name grantee privilege_type +d public t9 admin ALL +d public t9 root ALL +d public t9 testuser CREATE +d public t9 testuser2 SELECT +d public t9 testuser3 SELECT + +user testuser + +statement ok +CREATE TABLE t10() + +query TTTTT colnames +SHOW GRANTS ON t10 +---- +database_name schema_name table_name grantee privilege_type +d public t10 admin ALL +d public t10 root ALL +d public t10 testuser CREATE +d public t10 testuser2 SELECT +d public t10 testuser3 SELECT + +user root + +statement ok +ALTER DEFAULT PRIVILEGES FOR ROLE root, testuser REVOKE SELECT ON TABLES FROM testuser2, testuser3 + +statement ok +CREATE TABLE t11() + +query TTTTT colnames +SHOW GRANTS ON t11 +---- +database_name schema_name table_name grantee privilege_type +d public t11 admin ALL +d public t11 root ALL +d public t11 testuser CREATE + +user testuser + +statement ok +CREATE TABLE t12() + +query TTTTT colnames +SHOW GRANTS ON t12 +---- +database_name schema_name table_name grantee privilege_type +d public t12 admin ALL +d public t12 root ALL +d public t12 testuser CREATE diff --git a/pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_type b/pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_type new file mode 100644 index 000000000000..81ee616453f2 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/alter_default_privileges_for_type @@ -0,0 +1,99 @@ +statement ok +CREATE DATABASE d + +statement ok +CREATE USER testuser2 + +statement ok +ALTER DEFAULT PRIVILEGES GRANT ALL ON TYPES TO testuser, testuser2 + +statement ok +CREATE TYPE t AS ENUM() + +query TTTTT colnames +SHOW GRANTS ON TYPE t +---- +database_name schema_name type_name grantee privilege_type +test public t admin ALL +test public t public USAGE +test public t root ALL +test public t testuser ALL +test public t testuser2 ALL + +statement ok +ALTER DEFAULT PRIVILEGES REVOKE USAGE ON TYPES FROM testuser, testuser2 + +statement ok +CREATE TYPE t2 AS ENUM() + +query TTTTT colnames +SHOW GRANTS ON TYPE t2 +---- +database_name schema_name type_name grantee privilege_type +test public t2 admin ALL +test public t2 public USAGE +test public t2 root ALL +test public t2 testuser GRANT +test public t2 testuser2 GRANT + +statement ok +ALTER DEFAULT PRIVILEGES REVOKE ALL ON TYPES FROM testuser, testuser2 + +statement ok +CREATE TYPE t3 AS ENUM() + +query TTTTT colnames +SHOW GRANTS ON TYPE t3 +---- +database_name schema_name type_name grantee privilege_type +test public t3 admin ALL +test public t3 public USAGE +test public t3 root ALL + +statement ok +GRANT CREATE ON DATABASE d TO testuser + +user testuser +statement ok +USE d + +statement ok +ALTER DEFAULT PRIVILEGES FOR ROLE testuser REVOKE ALL ON TYPES FROM testuser, testuser2 + +statement ok +CREATE TYPE t4 AS ENUM() + +query TTTTT colnames +SHOW GRANTS ON TYPE t4 +---- +database_name schema_name type_name grantee privilege_type +d public t4 admin ALL +d public t4 public USAGE +d public t4 root ALL +d public t4 testuser ALL + +user root +statement ok +USE d + +statement ok +GRANT testuser TO root + +statement ok +ALTER DEFAULT PRIVILEGES FOR ROLE testuser REVOKE ALL ON TYPES FROM testuser, testuser2 + +user testuser +statement ok +USE d + +statement ok +CREATE TYPE t5 AS ENUM() + +query TTTTT colnames +SHOW GRANTS ON TYPE t5 +---- +database_name schema_name type_name grantee privilege_type +d public t5 admin ALL +d public t5 public USAGE +d public t5 root ALL +d public t5 testuser ALL diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index fe940392b002..e9e8642d8bdb 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -664,9 +664,6 @@ func (u *sqlSymUnion) abbreviatedGrant() tree.AbbreviatedGrant { func (u *sqlSymUnion) abbreviatedRevoke() tree.AbbreviatedRevoke { return u.val.(tree.AbbreviatedRevoke) } -func (u *sqlSymUnion) objectNamePrefixPtr() *tree.ObjectNamePrefix { - return u.val.(*tree.ObjectNamePrefix) -} func (u *sqlSymUnion) alterDefaultPrivilegesTargetObject() tree.AlterDefaultPrivilegesTargetObject { return u.val.(tree.AlterDefaultPrivilegesTargetObject) } @@ -1323,8 +1320,8 @@ func (u *sqlSymUnion) alterDefaultPrivilegesTargetObject() tree.AlterDefaultPriv %type abbreviated_grant_stmt %type abbreviated_revoke_stmt %type opt_with_grant_option -%type <*security.SQLUsername> opt_for_role -%type <*tree.ObjectNamePrefix> opt_in_schema +%type <[]security.SQLUsername> opt_for_roles +%type opt_in_schemas %type alter_default_privileges_target_object @@ -7848,21 +7845,27 @@ alter_rename_index_stmt: $$.val = &tree.RenameIndex{Index: $5.newTableIndexName(), NewName: tree.UnrestrictedName($8), IfExists: true} } +// %Help: ALTER DEFAULT PRIVILEGES - alter default privileges on an object +// %Category: DDL +// %Text: +// +// Commands: +// ALTER DEFAULT PRIVILEGES [ FOR { ROLE | USER } target_roles... ] [ IN SCHEMA schema_name...] abbreviated_grant_or_revoke alter_default_privileges_stmt: - ALTER DEFAULT PRIVILEGES opt_for_role opt_in_schema abbreviated_grant_stmt + ALTER DEFAULT PRIVILEGES opt_for_roles opt_in_schemas abbreviated_grant_stmt { $$.val = &tree.AlterDefaultPrivileges{ - Role: $4.userPtr(), - Schema: $5.objectNamePrefixPtr(), + Roles: $4.users(), + Schemas: $5.objectNamePrefixList(), Grant: $6.abbreviatedGrant(), IsGrant: true, } } -| ALTER DEFAULT PRIVILEGES opt_for_role opt_in_schema abbreviated_revoke_stmt +| ALTER DEFAULT PRIVILEGES opt_for_roles opt_in_schemas abbreviated_revoke_stmt { $$.val = &tree.AlterDefaultPrivileges{ - Role: $4.userPtr(), - Schema: $5.objectNamePrefixPtr(), + Roles: $4.users(), + Schemas: $5.objectNamePrefixList(), Revoke: $6.abbreviatedRevoke(), IsGrant: false, } @@ -7936,25 +7939,23 @@ alter_default_privileges_target_object: return unimplemented(sqllex, "ALTER DEFAULT PRIVILEGES ... ON FUNCTIONS ...") } -opt_for_role: - FOR role_or_group_or_user role_spec +opt_for_roles: + FOR role_or_group_or_user role_spec_list { - tmp := $3.user() - $$.val = &tmp + $$.val = $3.users() } | /* EMPTY */ { - $$.val = (*security.SQLUsername)(nil) + $$.val = []security.SQLUsername{} } -opt_in_schema: - IN SCHEMA qualifiable_schema_name +opt_in_schemas: + IN SCHEMA schema_name_list { - tmp := $3.objectNamePrefix() - $$.val = &tmp + $$.val = $3.objectNamePrefixList() } | /* EMPTY */ { - $$.val = (*tree.ObjectNamePrefix)(nil) + $$.val = tree.ObjectNamePrefixList{} } opt_column: diff --git a/pkg/sql/parser/testdata/alter_default_privileges b/pkg/sql/parser/testdata/alter_default_privileges index c55f37c53da5..ddbb58bd468a 100644 --- a/pkg/sql/parser/testdata/alter_default_privileges +++ b/pkg/sql/parser/testdata/alter_default_privileges @@ -1007,3 +1007,27 @@ ALTER DEFAULT PRIVILEGES FOR ROLE foo IN SCHEMA s REVOKE GRANT OPTION FOR USAGE, ALTER DEFAULT PRIVILEGES FOR ROLE foo IN SCHEMA s REVOKE GRANT OPTION FOR USAGE, CREATE ON SCHEMAS FROM foo, bar -- fully parenthesized ALTER DEFAULT PRIVILEGES FOR ROLE foo IN SCHEMA s REVOKE GRANT OPTION FOR USAGE, CREATE ON SCHEMAS FROM foo, bar -- literals removed ALTER DEFAULT PRIVILEGES FOR ROLE _ IN SCHEMA _ REVOKE GRANT OPTION FOR USAGE, CREATE ON SCHEMAS FROM _, _ -- identifiers removed + +parse +ALTER DEFAULT PRIVILEGES FOR ROLE foo, bar GRANT ALL ON TABLES TO FOO +---- +ALTER DEFAULT PRIVILEGES FOR ROLE foo, bar GRANT ALL ON TABLES TO foo -- normalized! +ALTER DEFAULT PRIVILEGES FOR ROLE foo, bar GRANT ALL ON TABLES TO foo -- fully parenthesized +ALTER DEFAULT PRIVILEGES FOR ROLE foo, bar GRANT ALL ON TABLES TO foo -- literals removed +ALTER DEFAULT PRIVILEGES FOR ROLE _, _ GRANT ALL ON TABLES TO _ -- identifiers removed + +parse +ALTER DEFAULT PRIVILEGES IN SCHEMA s1, s2 GRANT ALL ON TABLES TO FOO +---- +ALTER DEFAULT PRIVILEGES IN SCHEMA s1, s2 GRANT ALL ON TABLES TO foo -- normalized! +ALTER DEFAULT PRIVILEGES IN SCHEMA s1, s2 GRANT ALL ON TABLES TO foo -- fully parenthesized +ALTER DEFAULT PRIVILEGES IN SCHEMA s1, s2 GRANT ALL ON TABLES TO foo -- literals removed +ALTER DEFAULT PRIVILEGES IN SCHEMA _, _ GRANT ALL ON TABLES TO _ -- identifiers removed + +parse +ALTER DEFAULT PRIVILEGES FOR ROLE foo, bar IN SCHEMA s1, s2 GRANT ALL ON TABLES TO FOO +---- +ALTER DEFAULT PRIVILEGES FOR ROLE foo, bar IN SCHEMA s1, s2 GRANT ALL ON TABLES TO foo -- normalized! +ALTER DEFAULT PRIVILEGES FOR ROLE foo, bar IN SCHEMA s1, s2 GRANT ALL ON TABLES TO foo -- fully parenthesized +ALTER DEFAULT PRIVILEGES FOR ROLE foo, bar IN SCHEMA s1, s2 GRANT ALL ON TABLES TO foo -- literals removed +ALTER DEFAULT PRIVILEGES FOR ROLE _, _ IN SCHEMA _, _ GRANT ALL ON TABLES TO _ -- identifiers removed diff --git a/pkg/sql/sem/tree/BUILD.bazel b/pkg/sql/sem/tree/BUILD.bazel index ca59d69898e0..41a780de243a 100644 --- a/pkg/sql/sem/tree/BUILD.bazel +++ b/pkg/sql/sem/tree/BUILD.bazel @@ -12,6 +12,7 @@ go_library( srcs = [ "aggregate_funcs.go", "alter_database.go", + "alter_default_privileges.go", "alter_index.go", "alter_schema.go", "alter_sequence.go", diff --git a/pkg/sql/sem/tree/alter_default_privileges.go b/pkg/sql/sem/tree/alter_default_privileges.go index d0be2c7cafde..95a2911d56c0 100644 --- a/pkg/sql/sem/tree/alter_default_privileges.go +++ b/pkg/sql/sem/tree/alter_default_privileges.go @@ -17,10 +17,10 @@ import ( // AlterDefaultPrivileges represents an ALTER DEFAULT PRIVILEGES statement. type AlterDefaultPrivileges struct { - Role *security.SQLUsername + Roles []security.SQLUsername // If Schema is not specified, ALTER DEFAULT PRIVILEGES is being // run on the current database. - Schema *ObjectNamePrefix + Schemas ObjectNamePrefixList // Only one of Grant or Revoke should be set. IsGrant is used to determine // which one is set. @@ -31,14 +31,19 @@ type AlterDefaultPrivileges struct { func (n *AlterDefaultPrivileges) Format(ctx *FmtCtx) { ctx.WriteString("ALTER DEFAULT PRIVILEGES ") - if n.Role != nil { + if len(n.Roles) > 0 { ctx.WriteString("FOR ROLE ") - ctx.FormatUsername(*n.Role) + for i, role := range n.Roles { + if i > 0 { + ctx.WriteString(", ") + } + ctx.FormatUsername(role) + } ctx.WriteString(" ") } - if n.Schema != nil { + if len(n.Schemas) > 0 { ctx.WriteString("IN SCHEMA ") - ctx.FormatNode(n.Schema) + ctx.FormatNode(n.Schemas) ctx.WriteString(" ") } if n.IsGrant { @@ -50,8 +55,11 @@ func (n *AlterDefaultPrivileges) Format(ctx *FmtCtx) { // AlterDefaultPrivilegesTargetObject represents the type of object that is // having it's default privileges altered. -type AlterDefaultPrivilegesTargetObject int +type AlterDefaultPrivilegesTargetObject uint32 +// The numbers are explicitly assigned since the DefaultPrivilegesPerObject +// map defined in the DefaultPrivilegesPerRole proto requires the key value +// for the object type to remain unchanged. const ( Tables AlterDefaultPrivilegesTargetObject = 1 Sequences AlterDefaultPrivilegesTargetObject = 2 @@ -59,10 +67,6 @@ const ( Schemas AlterDefaultPrivilegesTargetObject = 4 ) -func (a AlterDefaultPrivilegesTargetObject) ToUInt32() uint32 { - return uint32(a) -} - type AbbreviatedGrant struct { Privileges privilege.List Target AlterDefaultPrivilegesTargetObject diff --git a/pkg/sql/walk.go b/pkg/sql/walk.go index ca6ebf664a63..7ee0bb03842a 100644 --- a/pkg/sql/walk.go +++ b/pkg/sql/walk.go @@ -338,6 +338,7 @@ var planNodeNames = map[reflect.Type]string{ reflect.TypeOf(&alterDatabasePrimaryRegionNode{}): "alter database primary region", reflect.TypeOf(&alterDatabaseSurvivalGoalNode{}): "alter database survive", reflect.TypeOf(&alterDatabaseDropRegionNode{}): "alter database drop region", + reflect.TypeOf(&alterDefaultPrivilegesNode{}): "alter default privileges", reflect.TypeOf(&alterIndexNode{}): "alter index", reflect.TypeOf(&alterSequenceNode{}): "alter sequence", reflect.TypeOf(&alterSchemaNode{}): "alter schema",