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",