From d7a41f5e6a737ebcf6e6ffdead41889d0d26a97b Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Wed, 12 May 2021 13:05:06 -0400 Subject: [PATCH 1/7] sql/*: remove catalog.ResolvedSchema, generalize SchemaDescriptor The `ResolvedSchema` was a blemish on the descriptor resolution APIs. It made it very hard to create a reasonable abstraction for descriptor resolution injection. This commit creates new implementations of `SchemaDescriptor` to represent virtual, public, and temporary schemas. There is likely more cleanup that could be done along the way to make the different kinds of schemas more uniform, but that is left for later or never. Release note: None --- pkg/ccl/backupccl/BUILD.bazel | 1 + pkg/ccl/backupccl/show.go | 4 +- pkg/ccl/cliccl/BUILD.bazel | 2 +- pkg/ccl/cliccl/debug_backup.go | 8 +- pkg/ccl/importccl/import_stmt.go | 4 +- pkg/server/telemetry/BUILD.bazel | 1 - pkg/server/telemetry/features.go | 7 +- pkg/sql/alter_schema.go | 4 +- pkg/sql/authorization.go | 28 ++-- pkg/sql/catalog/BUILD.bazel | 3 +- pkg/sql/catalog/accessor.go | 4 +- pkg/sql/catalog/catalog.go | 37 +----- pkg/sql/catalog/catconstants/BUILD.bazel | 6 +- pkg/sql/catalog/catconstants/constants.go | 4 - pkg/sql/catalog/catconstants/schemas.go | 56 ++++++++ pkg/sql/catalog/descpb/privilege.go | 9 ++ pkg/sql/catalog/descriptor.go | 10 +- pkg/sql/catalog/descs/BUILD.bazel | 2 + pkg/sql/catalog/descs/collection.go | 120 ++++++++---------- pkg/sql/catalog/descs/collection_test.go | 10 +- pkg/sql/catalog/resolver/resolver.go | 17 +-- pkg/sql/catalog/schema.go | 40 ++++++ pkg/sql/catalog/schemadesc/BUILD.bazel | 8 +- .../catalog/schemadesc/public_schema_desc.go | 52 ++++++++ pkg/sql/catalog/schemadesc/schema_desc.go | 8 +- .../schemadesc/synthetic_schema_desc.go | 99 +++++++++++++++ .../schemadesc/temporary_schema_desc.go | 54 ++++++++ .../catalog/schemadesc/virtual_schema_desc.go | 64 ++++++++++ pkg/sql/catalog/validate.go | 6 +- pkg/sql/crdb_internal.go | 10 +- pkg/sql/create_schema.go | 4 +- pkg/sql/create_table.go | 6 +- pkg/sql/create_type.go | 11 +- pkg/sql/delegate/BUILD.bazel | 1 - pkg/sql/delegate/show_enums.go | 4 +- pkg/sql/delegate/show_tables.go | 4 +- pkg/sql/drop_cascade.go | 8 +- pkg/sql/drop_database.go | 12 +- pkg/sql/drop_schema.go | 21 +-- pkg/sql/information_schema.go | 60 ++++----- pkg/sql/opt/optbuilder/BUILD.bazel | 2 +- pkg/sql/opt/optbuilder/util.go | 6 +- pkg/sql/opt_catalog.go | 10 +- pkg/sql/pg_catalog.go | 25 ++-- pkg/sql/pg_extension.go | 3 +- pkg/sql/rename_table.go | 4 +- pkg/sql/resolver.go | 32 ++--- pkg/sql/sem/tree/BUILD.bazel | 1 + pkg/sql/sem/tree/name_resolution.go | 9 +- pkg/sql/sessiondata/BUILD.bazel | 1 + pkg/sql/sessiondata/search_path.go | 57 ++------- pkg/sql/sessiondata/session_data.go | 18 ++- pkg/sql/set_schema.go | 6 +- pkg/sql/temporary_schema_test.go | 4 +- pkg/sql/type_change.go | 7 +- pkg/sql/virtual_schema.go | 83 ++++++------ 56 files changed, 683 insertions(+), 394 deletions(-) create mode 100644 pkg/sql/catalog/catconstants/schemas.go create mode 100644 pkg/sql/catalog/schema.go create mode 100644 pkg/sql/catalog/schemadesc/public_schema_desc.go create mode 100644 pkg/sql/catalog/schemadesc/synthetic_schema_desc.go create mode 100644 pkg/sql/catalog/schemadesc/temporary_schema_desc.go create mode 100644 pkg/sql/catalog/schemadesc/virtual_schema_desc.go diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index 0c870faf6f18..f4fe22c88248 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -59,6 +59,7 @@ go_library( "//pkg/sql/catalog", "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/catalogkv", + "//pkg/sql/catalog/catconstants", "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/dbdesc", "//pkg/sql/catalog/descpb", diff --git a/pkg/ccl/backupccl/show.go b/pkg/ccl/backupccl/show.go index 3da74eeaa195..4b7fc5807be7 100644 --- a/pkg/ccl/backupccl/show.go +++ b/pkg/ccl/backupccl/show.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" @@ -30,7 +31,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/protoreflect" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/storage/cloud" "github.com/cockroachdb/cockroach/pkg/util/encoding" @@ -269,7 +269,7 @@ func backupShowerDefault( // Map database ID to descriptor name. dbIDToName := make(map[descpb.ID]string) schemaIDToName := make(map[descpb.ID]string) - schemaIDToName[keys.PublicSchemaID] = sessiondata.PublicSchemaName + schemaIDToName[keys.PublicSchemaID] = catconstants.PublicSchemaName for i := range manifest.Descriptors { _, db, _, schema := descpb.FromDescriptor(&manifest.Descriptors[i]) if db != nil { diff --git a/pkg/ccl/cliccl/BUILD.bazel b/pkg/ccl/cliccl/BUILD.bazel index 840b02e1b104..276f70f0469c 100644 --- a/pkg/ccl/cliccl/BUILD.bazel +++ b/pkg/ccl/cliccl/BUILD.bazel @@ -29,13 +29,13 @@ go_library( "//pkg/server", "//pkg/settings/cluster", "//pkg/sql/catalog", + "//pkg/sql/catalog/catconstants", "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/tabledesc", "//pkg/sql/row", "//pkg/sql/rowenc", "//pkg/sql/sem/tree", - "//pkg/sql/sessiondata", "//pkg/storage", "//pkg/storage/cloud", "//pkg/storage/cloud/nodelocal", diff --git a/pkg/ccl/cliccl/debug_backup.go b/pkg/ccl/cliccl/debug_backup.go index 9edaf3d74f72..8ac9a0bd0dec 100644 --- a/pkg/ccl/cliccl/debug_backup.go +++ b/pkg/ccl/cliccl/debug_backup.go @@ -35,13 +35,13 @@ import ( "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/cloud" "github.com/cockroachdb/cockroach/pkg/storage/cloud/nodelocal" @@ -751,7 +751,7 @@ func (b backupMetaDisplayMsg) MarshalJSON() ([]byte, error) { dbIDToName := make(map[descpb.ID]string) schemaIDToFullyQualifiedName := make(map[descpb.ID]string) - schemaIDToFullyQualifiedName[keys.PublicSchemaID] = sessiondata.PublicSchemaName + schemaIDToFullyQualifiedName[keys.PublicSchemaID] = catconstants.PublicSchemaName typeIDToFullyQualifiedName := make(map[descpb.ID]string) tableIDToFullyQualifiedName := make(map[descpb.ID]string) @@ -767,7 +767,7 @@ func (b backupMetaDisplayMsg) MarshalJSON() ([]byte, error) { schemaIDToFullyQualifiedName[id] = dbName + "." + schemaName } else if typeDesc != nil { parentSchema := schemaIDToFullyQualifiedName[typeDesc.GetParentSchemaID()] - if parentSchema == sessiondata.PublicSchemaName { + if parentSchema == catconstants.PublicSchemaName { parentSchema = dbIDToName[typeDesc.GetParentID()] + "." + parentSchema } typeName := descpb.GetDescriptorName(d) @@ -775,7 +775,7 @@ func (b backupMetaDisplayMsg) MarshalJSON() ([]byte, error) { } else if tableDesc != nil { tbDesc := tabledesc.NewBuilder(tableDesc).BuildImmutable() parentSchema := schemaIDToFullyQualifiedName[tbDesc.GetParentSchemaID()] - if parentSchema == sessiondata.PublicSchemaName { + if parentSchema == catconstants.PublicSchemaName { parentSchema = dbIDToName[tableDesc.GetParentID()] + "." + parentSchema } tableName := descpb.GetDescriptorName(d) diff --git a/pkg/ccl/importccl/import_stmt.go b/pkg/ccl/importccl/import_stmt.go index 676b8c0ae82d..f589c31906fc 100644 --- a/pkg/ccl/importccl/import_stmt.go +++ b/pkg/ccl/importccl/import_stmt.go @@ -433,12 +433,12 @@ func importPlanHook( } } parentID = dbDesc.GetID() - switch schema.Kind { + switch schema.SchemaKind() { case catalog.SchemaVirtual: return pgerror.Newf(pgcode.InvalidSchemaName, "cannot import into schema %q", table.SchemaName) case catalog.SchemaUserDefined, catalog.SchemaPublic, catalog.SchemaTemporary: - parentSchemaID = schema.ID + parentSchemaID = schema.GetID() } } else { // No target table means we're importing whatever we find into the session diff --git a/pkg/server/telemetry/BUILD.bazel b/pkg/server/telemetry/BUILD.bazel index 10604c5c28a9..ba02d72a9546 100644 --- a/pkg/server/telemetry/BUILD.bazel +++ b/pkg/server/telemetry/BUILD.bazel @@ -9,7 +9,6 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/server/telemetry", visibility = ["//visibility:public"], deps = [ - "//pkg/sql/catalog/catconstants", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/util/metric", diff --git a/pkg/server/telemetry/features.go b/pkg/server/telemetry/features.go index 3506534a5c48..edc29054a200 100644 --- a/pkg/server/telemetry/features.go +++ b/pkg/server/telemetry/features.go @@ -16,7 +16,6 @@ import ( "strings" "sync/atomic" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/util/metric" @@ -226,6 +225,10 @@ func GetFeatureCounts(quantize QuantizeCounts, reset ResetCounters) map[string]i return m } +// ValidationTelemetryKeyPrefix is the prefix of telemetry keys pertaining to +// descriptor validation failures. +const ValidationTelemetryKeyPrefix = "sql.schema.validation_errors." + // RecordError takes an error and increments the corresponding count // for its error code, and, if it is an unimplemented or internal // error, the count for that feature or the internal error's shortened @@ -251,7 +254,7 @@ func RecordError(err error) { } for _, tk := range tkeys { prefixedTelemetryKey := prefix + tk - if strings.HasPrefix(tk, catconstants.ValidationTelemetryKeyPrefix) { + if strings.HasPrefix(tk, ValidationTelemetryKeyPrefix) { // Descriptor validation errors already have their own prefixing scheme. prefixedTelemetryKey = tk } diff --git a/pkg/sql/alter_schema.go b/pkg/sql/alter_schema.go index d04889763a1a..a1c788bb133d 100644 --- a/pkg/sql/alter_schema.go +++ b/pkg/sql/alter_schema.go @@ -65,11 +65,11 @@ func (p *planner) AlterSchema(ctx context.Context, n *tree.AlterSchema) (planNod if !found { return nil, pgerror.Newf(pgcode.InvalidSchemaName, "schema %q does not exist", n.Schema.String()) } - switch schema.Kind { + switch schema.SchemaKind() { case catalog.SchemaPublic, catalog.SchemaVirtual, catalog.SchemaTemporary: return nil, pgerror.Newf(pgcode.InvalidSchemaName, "cannot modify schema %q", n.Schema.String()) case catalog.SchemaUserDefined: - desc := schema.Desc.(*schemadesc.Mutable) + desc := schema.(*schemadesc.Mutable) // The user must be a superuser or the owner of the schema to modify it. hasAdmin, err := p.HasAdminRole(ctx) if err != nil { diff --git a/pkg/sql/authorization.go b/pkg/sql/authorization.go index 43450b89e219..d2000e854338 100644 --- a/pkg/sql/authorization.go +++ b/pkg/sql/authorization.go @@ -513,13 +513,13 @@ func (p *planner) canCreateOnSchema( user security.SQLUsername, checkPublicSchema shouldCheckPublicSchema, ) error { - resolvedSchema, err := p.Descriptors().GetImmutableSchemaByID( + scDesc, err := p.Descriptors().GetImmutableSchemaByID( ctx, p.Txn(), schemaID, tree.SchemaLookupFlags{}) if err != nil { return err } - switch resolvedSchema.Kind { + switch kind := scDesc.SchemaKind(); kind { case catalog.SchemaPublic: // The public schema is valid to create in if the parent database is. if !checkPublicSchema { @@ -537,11 +537,11 @@ func (p *planner) canCreateOnSchema( return nil case catalog.SchemaVirtual: return pgerror.Newf(pgcode.InsufficientPrivilege, - "cannot CREATE on schema %s", resolvedSchema.Name) + "cannot CREATE on schema %s", scDesc.GetName()) case catalog.SchemaUserDefined: - return p.CheckPrivilegeForUser(ctx, resolvedSchema.Desc, privilege.CREATE, user) + return p.CheckPrivilegeForUser(ctx, scDesc, privilege.CREATE, user) default: - panic(errors.AssertionFailedf("unknown schema kind %d", resolvedSchema.Kind)) + panic(errors.AssertionFailedf("unknown schema kind %d", kind)) } } @@ -554,20 +554,20 @@ func (p *planner) canResolveDescUnderSchema( if tbl, ok := desc.(catalog.TableDescriptor); ok && tbl.IsTemporary() { return nil } - resolvedSchema, err := p.Descriptors().GetImmutableSchemaByID( + scDesc, err := p.Descriptors().GetImmutableSchemaByID( ctx, p.Txn(), schemaID, tree.SchemaLookupFlags{}) if err != nil { return err } - switch resolvedSchema.Kind { + switch kind := scDesc.SchemaKind(); kind { case catalog.SchemaPublic, catalog.SchemaTemporary, catalog.SchemaVirtual: // Anyone can resolve under temporary, public or virtual schemas. return nil case catalog.SchemaUserDefined: - return p.CheckPrivilegeForUser(ctx, resolvedSchema.Desc, privilege.USAGE, p.User()) + return p.CheckPrivilegeForUser(ctx, scDesc, privilege.USAGE, p.User()) default: - panic(errors.AssertionFailedf("unknown schema kind %d", resolvedSchema.Kind)) + panic(errors.AssertionFailedf("unknown schema kind %d", kind)) } } @@ -643,7 +643,7 @@ func (p *planner) HasOwnershipOnSchema( // Only the node user has ownership over the system database. return p.User().IsNodeUser(), nil } - resolvedSchema, err := p.Descriptors().GetImmutableSchemaByID( + scDesc, err := p.Descriptors().GetImmutableSchemaByID( ctx, p.Txn(), schemaID, tree.SchemaLookupFlags{}, ) if err != nil { @@ -651,7 +651,7 @@ func (p *planner) HasOwnershipOnSchema( } hasOwnership := false - switch resolvedSchema.Kind { + switch kind := scDesc.SchemaKind(); kind { case catalog.SchemaPublic: // admin is the owner of the public schema. hasOwnership, err = p.UserHasAdminRole(ctx, p.User()) @@ -663,14 +663,14 @@ func (p *planner) HasOwnershipOnSchema( case catalog.SchemaTemporary: // The user owns all the temporary schemas that they created in the session. hasOwnership = p.SessionData() != nil && - p.SessionData().IsTemporarySchemaID(uint32(resolvedSchema.ID)) + p.SessionData().IsTemporarySchemaID(uint32(scDesc.GetID())) case catalog.SchemaUserDefined: - hasOwnership, err = p.HasOwnership(ctx, resolvedSchema.Desc) + hasOwnership, err = p.HasOwnership(ctx, scDesc) if err != nil { return false, err } default: - panic(errors.AssertionFailedf("unknown schema kind %d", resolvedSchema.Kind)) + panic(errors.AssertionFailedf("unknown schema kind %d", kind)) } return hasOwnership, nil diff --git a/pkg/sql/catalog/BUILD.bazel b/pkg/sql/catalog/BUILD.bazel index eb1dfd1eda64..8a1c48f15062 100644 --- a/pkg/sql/catalog/BUILD.bazel +++ b/pkg/sql/catalog/BUILD.bazel @@ -9,6 +9,7 @@ go_library( "descriptor.go", "descriptor_id_set.go", "errors.go", + "schema.go", "table_col_map.go", "table_col_set.go", "table_elements.go", @@ -21,7 +22,7 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/roachpb", - "//pkg/sql/catalog/catconstants", + "//pkg/server/telemetry", "//pkg/sql/catalog/descpb", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", diff --git a/pkg/sql/catalog/accessor.go b/pkg/sql/catalog/accessor.go index ebf06446e4c4..8f976f844696 100644 --- a/pkg/sql/catalog/accessor.go +++ b/pkg/sql/catalog/accessor.go @@ -40,11 +40,11 @@ type Accessor interface { ctx context.Context, txn *kv.Txn, dbName string, flags tree.DatabaseLookupFlags, ) (DatabaseDescriptor, error) - // GetSchema returns true and a ResolvedSchema object if the target schema + // GetSchemaByName returns true and a SchemaDescriptor object if the target schema // exists under the target database. GetSchemaByName( ctx context.Context, txn *kv.Txn, dbID descpb.ID, scName string, flags tree.SchemaLookupFlags, - ) (bool, ResolvedSchema, error) + ) (bool, SchemaDescriptor, error) // GetObjectNamesAndIDs returns the list of all objects in the given // database and schema. diff --git a/pkg/sql/catalog/catalog.go b/pkg/sql/catalog/catalog.go index a3aa9ee89295..b4702efd41ad 100644 --- a/pkg/sql/catalog/catalog.go +++ b/pkg/sql/catalog/catalog.go @@ -49,11 +49,12 @@ type MutableDescriptor interface { // VirtualSchemas is a collection of VirtualSchemas. type VirtualSchemas interface { GetVirtualSchema(schemaName string) (VirtualSchema, bool) + GetVirtualSchemaByID(id descpb.ID) (VirtualSchema, bool) } // VirtualSchema represents a collection of VirtualObjects. type VirtualSchema interface { - Desc() Descriptor + Desc() SchemaDescriptor NumTables() int VisitTables(func(object VirtualObject)) GetObjectByName(name string, flags tree.ObjectLookupFlags) (VirtualObject, error) @@ -70,40 +71,8 @@ type ResolvedObjectPrefix struct { // Database is the parent database descriptor. Database DatabaseDescriptor // Schema is the parent schema. - Schema ResolvedSchema + Schema SchemaDescriptor } // SchemaMeta implements the SchemaMeta interface. func (*ResolvedObjectPrefix) SchemaMeta() {} - -// ResolvedSchemaKind is an enum that represents what kind of schema -// has been resolved. -type ResolvedSchemaKind int - -const ( - // SchemaPublic represents the public schema. - SchemaPublic ResolvedSchemaKind = iota - // SchemaVirtual represents a virtual schema. - SchemaVirtual - // SchemaTemporary represents a temporary schema. - SchemaTemporary - // SchemaUserDefined represents a user defined schema. - SchemaUserDefined -) - -// ResolvedSchema represents the result of resolving a schema name, or an -// object prefix of .. Due to historical reasons, some schemas -// don't have unique IDs (public and virtual schemas), and others aren't backed -// by descriptors. The ResolvedSchema struct encapsulates the different cases. -type ResolvedSchema struct { - // Marks what kind of schema this is. It is always set. - Kind ResolvedSchemaKind - // Name of the resolved schema. It is always set. - Name string - // The ID of the resolved schema. This field is only set for schema kinds - // SchemaPublic, SchemaUserDefined and SchemaTemporary. - ID descpb.ID - // The descriptor backing the resolved schema. It is only set for - // SchemaUserDefined. - Desc SchemaDescriptor -} diff --git a/pkg/sql/catalog/catconstants/BUILD.bazel b/pkg/sql/catalog/catconstants/BUILD.bazel index 3a53d5b67670..3db8b53988e2 100644 --- a/pkg/sql/catalog/catconstants/BUILD.bazel +++ b/pkg/sql/catalog/catconstants/BUILD.bazel @@ -2,7 +2,11 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "catconstants", - srcs = ["constants.go"], + srcs = [ + "constants.go", + "schemas.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants", visibility = ["//visibility:public"], + deps = ["//pkg/keys"], ) diff --git a/pkg/sql/catalog/catconstants/constants.go b/pkg/sql/catalog/catconstants/constants.go index 4e12dd48a9d4..b5cf759410de 100644 --- a/pkg/sql/catalog/catconstants/constants.go +++ b/pkg/sql/catalog/catconstants/constants.go @@ -208,7 +208,3 @@ const ( PgExtensionSpatialRefSysTableID MinVirtualID = PgExtensionSpatialRefSysTableID ) - -// ValidationTelemetryKeyPrefix is the prefix of telemetry keys pertaining to -// descriptor validation failures. -const ValidationTelemetryKeyPrefix = "sql.schema.validation_errors." diff --git a/pkg/sql/catalog/catconstants/schemas.go b/pkg/sql/catalog/catconstants/schemas.go new file mode 100644 index 000000000000..723805e879a0 --- /dev/null +++ b/pkg/sql/catalog/catconstants/schemas.go @@ -0,0 +1,56 @@ +// 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 catconstants + +import "github.com/cockroachdb/cockroach/pkg/keys" + +// StaticSchemaIDMap is a map of statically known schema IDs. +var StaticSchemaIDMap = map[uint32]string{ + keys.PublicSchemaID: PublicSchemaName, + PgCatalogID: PgCatalogName, + InformationSchemaID: InformationSchemaName, + CrdbInternalID: CRDBInternalSchemaName, + PgExtensionSchemaID: PgExtensionSchemaName, +} + +// PgCatalogName is the name of the pg_catalog system schema. +const PgCatalogName = "pg_catalog" + +// PublicSchemaName is the name of the pg_catalog system schema. +const PublicSchemaName = "public" + +// UserSchemaName is the alias for schema names for users. +const UserSchemaName = "$user" + +// InformationSchemaName is the name of the information_schema system schema. +const InformationSchemaName = "information_schema" + +// CRDBInternalSchemaName is the name of the crdb_internal system schema. +const CRDBInternalSchemaName = "crdb_internal" + +// PgSchemaPrefix is a prefix for Postgres system schemas. Users cannot +// create schemas with this prefix. +const PgSchemaPrefix = "pg_" + +// PgTempSchemaName is the alias for temporary schemas across sessions. +const PgTempSchemaName = "pg_temp" + +// PgExtensionSchemaName is the alias for schemas which are usually "public" in postgres +// when installing an extension, but must be stored as a separate schema in CRDB. +const PgExtensionSchemaName = "pg_extension" + +// VirtualSchemaNames is a set of all virtual schema names. +var VirtualSchemaNames = map[string]struct{}{ + PgCatalogName: {}, + InformationSchemaName: {}, + CRDBInternalSchemaName: {}, + PgExtensionSchemaName: {}, +} diff --git a/pkg/sql/catalog/descpb/privilege.go b/pkg/sql/catalog/descpb/privilege.go index 50b97eeba5d9..c142bf9347f2 100644 --- a/pkg/sql/catalog/descpb/privilege.go +++ b/pkg/sql/catalog/descpb/privilege.go @@ -122,6 +122,15 @@ func NewCustomSuperuserPrivilegeDescriptor( } } +// NewPublicSelectPrivilegeDescriptor is used to construct a privilege descriptor +// owned by the node user which has SELECT privilege for the public role. It is +// used for virtual tables. +func NewPublicSelectPrivilegeDescriptor() *PrivilegeDescriptor { + return NewPrivilegeDescriptor( + security.PublicRoleName(), privilege.List{privilege.SELECT}, security.NodeUserName(), + ) +} + // NewPrivilegeDescriptor returns a privilege descriptor for the given // user with the specified list of privileges. func NewPrivilegeDescriptor( diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 9f13a1aa6c93..89fd279e68f7 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -152,8 +152,7 @@ type Descriptor interface { ValidateTxnCommit(vea ValidationErrorAccumulator, vdg ValidationDescGetter) } -// DatabaseDescriptor will eventually be called dbdesc.Descriptor. -// It is implemented by Immutable. +// DatabaseDescriptor encapsulates the concept of a database. type DatabaseDescriptor interface { Descriptor @@ -174,13 +173,6 @@ type DatabaseDescriptor interface { GetNonDroppedSchemaName(schemaID descpb.ID) string } -// SchemaDescriptor will eventually be called schemadesc.Descriptor. -// It is implemented by Immutable. -type SchemaDescriptor interface { - Descriptor - SchemaDesc() *descpb.SchemaDescriptor -} - // TableDescriptor is an interface around the table descriptor types. type TableDescriptor interface { Descriptor diff --git a/pkg/sql/catalog/descs/BUILD.bazel b/pkg/sql/catalog/descs/BUILD.bazel index c5f4c49eacac..b6b6496bae4d 100644 --- a/pkg/sql/catalog/descs/BUILD.bazel +++ b/pkg/sql/catalog/descs/BUILD.bazel @@ -18,11 +18,13 @@ go_library( "//pkg/sql/catalog/bootstrap", "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/catalogkv", + "//pkg/sql/catalog/catconstants", "//pkg/sql/catalog/dbdesc", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/hydratedtables", "//pkg/sql/catalog/lease", "//pkg/sql/catalog/resolver", + "//pkg/sql/catalog/schemadesc", "//pkg/sql/catalog/systemschema", "//pkg/sql/catalog/tabledesc", "//pkg/sql/catalog/typedesc", diff --git a/pkg/sql/catalog/descs/collection.go b/pkg/sql/catalog/descs/collection.go index e5ea4b330eac..44940a6b588c 100644 --- a/pkg/sql/catalog/descs/collection.go +++ b/pkg/sql/catalog/descs/collection.go @@ -27,11 +27,13 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/hydratedtables" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" @@ -623,7 +625,7 @@ func (tc *Collection) getObjectByName( dbID := db.GetID() // Resolve the schema. - foundSchema, resolvedSchema, err := tc.GetImmutableSchemaByName(ctx, txn, dbID, schemaName, + foundSchema, scDesc, err := tc.GetImmutableSchemaByName(ctx, txn, dbID, schemaName, tree.SchemaLookupFlags{ Required: flags.Required, AvoidCached: avoidCachedForParent, @@ -633,7 +635,7 @@ func (tc *Collection) getObjectByName( if err != nil || !foundSchema { return false, nil, err } - schemaID := resolvedSchema.ID + schemaID := scDesc.GetID() if found, refuseFurtherLookup, desc, err := tc.getSyntheticOrUncommittedDescriptor( dbID, schemaID, objectName, flags.RequireMutable, @@ -930,7 +932,7 @@ func filterDescriptorState( // mutable descriptor usable by the transaction. RequireMutable is ignored. func (tc *Collection) GetMutableSchemaByName( ctx context.Context, txn *kv.Txn, dbID descpb.ID, schemaName string, flags tree.SchemaLookupFlags, -) (bool, catalog.ResolvedSchema, error) { +) (bool, catalog.SchemaDescriptor, error) { flags.RequireMutable = true return tc.getSchemaByName(ctx, txn, dbID, schemaName, flags) } @@ -939,7 +941,7 @@ func (tc *Collection) GetMutableSchemaByName( // immutable descriptor usable by the transaction. RequireMutable is ignored. func (tc *Collection) GetImmutableSchemaByName( ctx context.Context, txn *kv.Txn, dbID descpb.ID, schemaName string, flags tree.SchemaLookupFlags, -) (bool, catalog.ResolvedSchema, error) { +) (bool, catalog.SchemaDescriptor, error) { flags.RequireMutable = false return tc.getSchemaByName(ctx, txn, dbID, schemaName, flags) } @@ -948,7 +950,7 @@ func (tc *Collection) GetImmutableSchemaByName( // exists under the target database. func (tc *Collection) GetSchemaByName( ctx context.Context, txn *kv.Txn, dbID descpb.ID, schemaName string, flags tree.SchemaLookupFlags, -) (found bool, _ catalog.ResolvedSchema, _ error) { +) (found bool, _ catalog.SchemaDescriptor, _ error) { return tc.getSchemaByName(ctx, txn, dbID, schemaName, flags) } @@ -956,68 +958,56 @@ func (tc *Collection) GetSchemaByName( // usable by the transaction. func (tc *Collection) getSchemaByName( ctx context.Context, txn *kv.Txn, dbID descpb.ID, schemaName string, flags tree.SchemaLookupFlags, -) (bool, catalog.ResolvedSchema, error) { +) (bool, catalog.SchemaDescriptor, error) { // Fast path public schema, as it is always found. if schemaName == tree.PublicSchema { - return true, catalog.ResolvedSchema{ - ID: keys.PublicSchemaID, Kind: catalog.SchemaPublic, Name: tree.PublicSchema, - }, nil + return true, schemadesc.GetPublicSchema(), nil } if tc.virtualSchemas != nil { - if _, ok := tc.virtualSchemas.GetVirtualSchema(schemaName); ok { - return true, catalog.ResolvedSchema{ - Kind: catalog.SchemaVirtual, - Name: schemaName, - }, nil + if sc, ok := tc.virtualSchemas.GetVirtualSchema(schemaName); ok { + return true, sc.Desc(), nil } } // If a temp schema is requested, check if it's for the current session, or // else fall back to reading from the store. - if strings.HasPrefix(schemaName, sessiondata.PgTempSchemaName) { + if strings.HasPrefix(schemaName, catconstants.PgTempSchemaName) { if tc.sessionData != nil { - if schemaName == sessiondata.PgTempSchemaName || + if schemaName == catconstants.PgTempSchemaName || schemaName == tc.sessionData.SearchPath.GetTemporarySchemaName() { schemaID, found := tc.sessionData.GetTemporarySchemaIDForDb(uint32(dbID)) if found { - schema := catalog.ResolvedSchema{ - Kind: catalog.SchemaTemporary, - Name: tc.sessionData.SearchPath.GetTemporarySchemaName(), - ID: descpb.ID(schemaID), - } - return true, schema, nil + return true, schemadesc.NewTemporarySchema( + tc.sessionData.SearchPath.GetTemporarySchemaName(), + descpb.ID(schemaID), + dbID, + ), nil } } } exists, schemaID, err := catalogkv.ResolveSchemaID(ctx, txn, tc.codec(), dbID, schemaName) if err != nil { - return false, catalog.ResolvedSchema{}, err + return false, nil, err } else if !exists { if flags.Required { - return false, catalog.ResolvedSchema{}, sqlerrors.NewUndefinedSchemaError(schemaName) + return false, nil, sqlerrors.NewUndefinedSchemaError(schemaName) } - return false, catalog.ResolvedSchema{}, nil - } - schema := catalog.ResolvedSchema{ - Kind: catalog.SchemaTemporary, - Name: schemaName, - ID: schemaID, + return false, nil, nil } - return true, schema, nil + return true, schemadesc.NewTemporarySchema( + schemaName, + schemaID, + dbID, + ), nil } // Otherwise, the schema is user-defined. Get the descriptor. desc, err := tc.getUserDefinedSchemaByName(ctx, txn, dbID, schemaName, flags) if err != nil || desc == nil { - return false, catalog.ResolvedSchema{}, err + return false, nil, err } - return true, catalog.ResolvedSchema{ - Kind: catalog.SchemaUserDefined, - Name: schemaName, - ID: desc.GetID(), - Desc: desc, - }, nil + return true, desc, nil } // GetMutableDatabaseByID returns a mutable database descriptor with @@ -1276,7 +1266,7 @@ func (tc *Collection) GetImmutableDescriptorByID( // the ID exists. func (tc *Collection) GetMutableSchemaByID( ctx context.Context, txn *kv.Txn, schemaID descpb.ID, flags tree.SchemaLookupFlags, -) (catalog.ResolvedSchema, error) { +) (catalog.SchemaDescriptor, error) { flags.RequireMutable = true return tc.getSchemaByID(ctx, txn, schemaID, flags) } @@ -1289,60 +1279,52 @@ var _ = (*Collection)(nil).GetMutableSchemaByID // the ID exists. func (tc *Collection) GetImmutableSchemaByID( ctx context.Context, txn *kv.Txn, schemaID descpb.ID, flags tree.SchemaLookupFlags, -) (catalog.ResolvedSchema, error) { +) (catalog.SchemaDescriptor, error) { flags.RequireMutable = false return tc.getSchemaByID(ctx, txn, schemaID, flags) } func (tc *Collection) getSchemaByID( ctx context.Context, txn *kv.Txn, schemaID descpb.ID, flags tree.SchemaLookupFlags, -) (catalog.ResolvedSchema, error) { +) (catalog.SchemaDescriptor, error) { if schemaID == keys.PublicSchemaID { - return catalog.ResolvedSchema{ - Kind: catalog.SchemaPublic, - ID: schemaID, - Name: tree.PublicSchema, - }, nil + return schemadesc.GetPublicSchema(), nil } // We have already considered if the schemaID is PublicSchemaID, // if the id appears in staticSchemaIDMap, it must map to a virtual schema. - if scName, ok := resolver.StaticSchemaIDMap[schemaID]; ok { - return catalog.ResolvedSchema{ - Kind: catalog.SchemaVirtual, - ID: schemaID, - Name: scName, - }, nil + if tc.virtualSchemas != nil { + if sc, ok := tc.virtualSchemas.GetVirtualSchemaByID(schemaID); ok { + return sc.Desc(), nil + } } // If this collection is attached to a session and the session has created // a temporary schema, then check if the schema ID matches. - if tc.sessionData != nil && tc.sessionData.IsTemporarySchemaID(uint32(schemaID)) { - return catalog.ResolvedSchema{ - Kind: catalog.SchemaTemporary, - ID: schemaID, - Name: tc.sessionData.SearchPath.GetTemporarySchemaName(), - }, nil + if tc.sessionData != nil { + dbID, exists := tc.sessionData.MaybeGetDatabaseForTemporarySchemaID(uint32(schemaID)) + if exists { + return schemadesc.NewTemporarySchema( + tc.sessionData.SearchPath.GetTemporarySchemaName(), + schemaID, + descpb.ID(dbID), + ), nil + } } // Otherwise, fall back to looking up the descriptor with the desired ID. desc, err := tc.getDescriptorByID(ctx, txn, schemaID, flags) if err != nil { - return catalog.ResolvedSchema{}, err + return nil, err } schemaDesc, ok := desc.(catalog.SchemaDescriptor) if !ok { - return catalog.ResolvedSchema{}, pgerror.Newf(pgcode.WrongObjectType, + return nil, pgerror.Newf(pgcode.WrongObjectType, "descriptor %d was not a schema", schemaID) } - return catalog.ResolvedSchema{ - Kind: catalog.SchemaUserDefined, - ID: schemaID, - Desc: schemaDesc, - Name: schemaDesc.GetName(), - }, nil + return schemaDesc, nil } // hydrateTypesInTableDesc installs user defined type metadata in all types.T @@ -1383,7 +1365,7 @@ func (tc *Collection) hydrateTypesInTableDesc( if err != nil { return tree.TypeName{}, nil, err } - name := tree.MakeNewQualifiedTypeName(dbDesc.GetName(), sc.Name, desc.Name) + name := tree.MakeNewQualifiedTypeName(dbDesc.GetName(), sc.GetName(), desc.Name) return name, desc, nil } @@ -1413,7 +1395,7 @@ func (tc *Collection) hydrateTypesInTableDesc( if err != nil { return tree.TypeName{}, nil, err } - name := tree.MakeNewQualifiedTypeName(dbDesc.GetName(), sc.Name, desc.GetName()) + name := tree.MakeNewQualifiedTypeName(dbDesc.GetName(), sc.GetName(), desc.GetName()) return name, desc, nil }) @@ -2000,7 +1982,7 @@ func (tc *Collection) GetObjectNamesAndIDs( } log.Eventf(ctx, "fetching list of objects for %q", dbDesc.GetName()) - prefix := catalogkeys.NewTableKey(dbDesc.GetID(), schema.ID, "").Key(tc.codec()) + prefix := catalogkeys.NewTableKey(dbDesc.GetID(), schema.GetID(), "").Key(tc.codec()) sr, err := txn.Scan(ctx, prefix, prefix.PrefixEnd(), 0) if err != nil { return nil, nil, err diff --git a/pkg/sql/catalog/descs/collection_test.go b/pkg/sql/catalog/descs/collection_test.go index 63eff1c46456..accdba2f811c 100644 --- a/pkg/sql/catalog/descs/collection_test.go +++ b/pkg/sql/catalog/descs/collection_test.go @@ -304,12 +304,12 @@ func TestAddUncommittedDescriptorAndMutableResolution(t *testing.T) { require.NoError(t, err) require.True(t, ok) - require.Same(t, schema.Desc, resolved.Desc) + require.Same(t, schema, resolved) - byID, err := descriptors.GetMutableDescriptorByID(ctx, schema.ID, txn) + byID, err := descriptors.GetMutableDescriptorByID(ctx, schema.GetID(), txn) require.NoError(t, err) - require.Same(t, schema.Desc, byID) + require.Same(t, schema, byID) return nil })) }) @@ -497,7 +497,7 @@ CREATE TABLE test.schema.t(x INT); return err } // Write garbage privileges into the schema desc. - privs := schemaDesc.Desc.GetPrivileges() + privs := schemaDesc.GetPrivileges() for i := range privs.Users { // SELECT is valid on a database but not a schema, however // due to issue #65697, after running ALTER DATABASE ... @@ -510,7 +510,7 @@ CREATE TABLE test.schema.t(x INT); } descsCol.SkipValidationOnWrite() - return descsCol.WriteDesc(ctx, false, schemaDesc.Desc.(catalog.MutableDescriptor), txn) + return descsCol.WriteDesc(ctx, false, schemaDesc.(catalog.MutableDescriptor), txn) }), ) diff --git a/pkg/sql/catalog/resolver/resolver.go b/pkg/sql/catalog/resolver/resolver.go index f3240173cdc3..91272d2b6cd7 100644 --- a/pkg/sql/catalog/resolver/resolver.go +++ b/pkg/sql/catalog/resolver/resolver.go @@ -239,22 +239,13 @@ func ResolveTargetObject( return nil, prefix, err } scInfo := scMeta.(*catalog.ResolvedObjectPrefix) - if scInfo.Schema.Kind == catalog.SchemaVirtual { + if scInfo.Schema.SchemaKind() == catalog.SchemaVirtual { return nil, prefix, pgerror.Newf(pgcode.InsufficientPrivilege, "schema cannot be modified: %q", tree.ErrString(&prefix)) } return scInfo, prefix, nil } -// StaticSchemaIDMap is a map of statically known schema IDs. -var StaticSchemaIDMap = map[descpb.ID]string{ - keys.PublicSchemaID: tree.PublicSchema, - catconstants.PgCatalogID: sessiondata.PgCatalogName, - catconstants.InformationSchemaID: sessiondata.InformationSchemaName, - catconstants.CrdbInternalID: sessiondata.CRDBInternalSchemaName, - catconstants.PgExtensionSchemaID: sessiondata.PgExtensionSchemaName, -} - // ResolveSchemaNameByID resolves a schema's name based on db and schema id. // Instead, we have to rely on a scan of the kv table. // TODO (SQLSchema): The remaining uses of this should be plumbed through @@ -263,10 +254,8 @@ func ResolveSchemaNameByID( ctx context.Context, txn *kv.Txn, codec keys.SQLCodec, dbID descpb.ID, schemaID descpb.ID, ) (string, error) { // Fast-path for public schema and virtual schemas, to avoid hot lookups. - for id, schemaName := range StaticSchemaIDMap { - if id == schemaID { - return schemaName, nil - } + if schemaName, ok := catconstants.StaticSchemaIDMap[uint32(schemaID)]; ok { + return schemaName, nil } schemas, err := GetForDatabase(ctx, txn, codec, dbID) if err != nil { diff --git a/pkg/sql/catalog/schema.go b/pkg/sql/catalog/schema.go new file mode 100644 index 000000000000..208c4bd469eb --- /dev/null +++ b/pkg/sql/catalog/schema.go @@ -0,0 +1,40 @@ +// 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 catalog + +import "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + +// SchemaDescriptor encapsulates the basic +type SchemaDescriptor interface { + Descriptor + + // SchemaKind indicates whether this descriptor + SchemaKind() ResolvedSchemaKind + + // SchemaDesc returns the underlying protocol buffer in the + // case that this is a real descriptor. + SchemaDesc() *descpb.SchemaDescriptor +} + +// ResolvedSchemaKind is an enum that represents what kind of schema +// has been resolved. +type ResolvedSchemaKind int + +const ( + // SchemaPublic represents the public schema. + SchemaPublic ResolvedSchemaKind = iota + // SchemaVirtual represents a virtual schema. + SchemaVirtual + // SchemaTemporary represents a temporary schema. + SchemaTemporary + // SchemaUserDefined represents a user defined schema. + SchemaUserDefined +) diff --git a/pkg/sql/catalog/schemadesc/BUILD.bazel b/pkg/sql/catalog/schemadesc/BUILD.bazel index 631654726544..85064c49f571 100644 --- a/pkg/sql/catalog/schemadesc/BUILD.bazel +++ b/pkg/sql/catalog/schemadesc/BUILD.bazel @@ -3,20 +3,26 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "schemadesc", srcs = [ + "public_schema_desc.go", "schema_desc.go", "schema_desc_builder.go", + "synthetic_schema_desc.go", + "temporary_schema_desc.go", + "virtual_schema_desc.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc", visibility = ["//visibility:public"], deps = [ "//pkg/keys", "//pkg/sql/catalog", + "//pkg/sql/catalog/catconstants", "//pkg/sql/catalog/descpb", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/privilege", - "//pkg/sql/sessiondata", + "//pkg/sql/sem/tree", "//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/schemadesc/public_schema_desc.go b/pkg/sql/catalog/schemadesc/public_schema_desc.go new file mode 100644 index 000000000000..7c54ccfb8d6f --- /dev/null +++ b/pkg/sql/catalog/schemadesc/public_schema_desc.go @@ -0,0 +1,52 @@ +// 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 schemadesc + +import ( + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) + +// GetPublicSchema returns a synthetic public schema which is +// part of every database. The public schema's implementation is a vestige +// of a time when there were no user-defined schemas. The public schema is +// interchangeable with the database itself in terms of privileges. +// +// The returned descriptor carries only a basic functionality, requiring the +// caller to check the SchemaKind to determine how to use the descriptor. The +// returned descriptor is not mapped to a database; every database has all of +// the same virtual schemas and the ParentID on the returned descriptor will be +// descpb.InvalidID. +func GetPublicSchema() catalog.SchemaDescriptor { + return publicDesc +} + +type public struct { + synthetic +} + +var _ catalog.SchemaDescriptor = public{} + +func (p public) GetParentID() descpb.ID { return descpb.InvalidID } +func (p public) GetID() descpb.ID { return keys.PublicSchemaID } +func (p public) GetName() string { return tree.PublicSchema } + +type publicBase struct{} + +func (p publicBase) kindName() string { return "public" } +func (p publicBase) kind() catalog.ResolvedSchemaKind { return catalog.SchemaPublic } + +// publicDesc is a singleton returned by GetPublicSchema. +var publicDesc catalog.SchemaDescriptor = public{ + synthetic{publicBase{}}, +} diff --git a/pkg/sql/catalog/schemadesc/schema_desc.go b/pkg/sql/catalog/schemadesc/schema_desc.go index abadad7655f3..726fcf502f48 100644 --- a/pkg/sql/catalog/schemadesc/schema_desc.go +++ b/pkg/sql/catalog/schemadesc/schema_desc.go @@ -16,11 +16,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "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/sessiondata" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" @@ -39,6 +39,10 @@ type immutable struct { isUncommittedVersion bool } +func (desc *immutable) SchemaKind() catalog.ResolvedSchemaKind { + return catalog.SchemaUserDefined +} + // SafeMessage makes immutable a SafeMessager. func (desc *immutable) SafeMessage() string { return formatSafeMessage("schemadesc.immutable", desc) @@ -286,7 +290,7 @@ func (desc *Mutable) IsUncommittedVersion() bool { // schema. func IsSchemaNameValid(name string) error { // Schemas starting with "pg_" are not allowed. - if strings.HasPrefix(name, sessiondata.PgSchemaPrefix) { + if strings.HasPrefix(name, catconstants.PgSchemaPrefix) { err := pgerror.Newf(pgcode.ReservedName, "unacceptable schema name %q", name) err = errors.WithDetail(err, `The prefix "pg_" is reserved for system schemas.`) return err diff --git a/pkg/sql/catalog/schemadesc/synthetic_schema_desc.go b/pkg/sql/catalog/schemadesc/synthetic_schema_desc.go new file mode 100644 index 000000000000..7a54a8c6a246 --- /dev/null +++ b/pkg/sql/catalog/schemadesc/synthetic_schema_desc.go @@ -0,0 +1,99 @@ +// 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 schemadesc + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +// synthetic implements many of the methods of catalog.SchemaDescriptor and +// is shared by the three "synthetic" implementations of that interface: +// virtual, temporary, and public. +type synthetic struct { + syntheticBase +} + +// syntheticBase is an interface to differentiate some of the +// behavior of synthetic. +type syntheticBase interface { + kindName() string + kind() catalog.ResolvedSchemaKind +} + +func (p synthetic) NameResolutionResult() {} +func (p synthetic) GetParentSchemaID() descpb.ID { + return descpb.InvalidID +} +func (p synthetic) IsUncommittedVersion() bool { + return false +} +func (p synthetic) GetVersion() descpb.DescriptorVersion { + return 1 +} +func (p synthetic) GetModificationTime() hlc.Timestamp { + return hlc.Timestamp{} +} +func (p synthetic) GetDrainingNames() []descpb.NameInfo { + return nil +} +func (p synthetic) GetPrivileges() *descpb.PrivilegeDescriptor { + log.Fatalf(context.TODO(), "cannot access privileges on a %s descriptor", p.kindName()) + return nil +} +func (p synthetic) DescriptorType() catalog.DescriptorType { + return catalog.Schema +} +func (p synthetic) GetAuditMode() descpb.TableDescriptor_AuditMode { + return descpb.TableDescriptor_DISABLED +} +func (p synthetic) Public() bool { + return true +} +func (p synthetic) Adding() bool { + return false +} +func (p synthetic) Dropped() bool { + return false +} +func (p synthetic) Offline() bool { + return false +} +func (p synthetic) GetOfflineReason() string { + return "" +} +func (p synthetic) DescriptorProto() *descpb.Descriptor { + log.Fatalf(context.TODO(), + "%s schema cannot be encoded", p.kindName()) + return nil // unreachable +} +func (p synthetic) GetReferencedDescIDs() (catalog.DescriptorIDSet, error) { + return catalog.DescriptorIDSet{}, nil +} +func (p synthetic) ValidateSelf(vea catalog.ValidationErrorAccumulator) {} +func (p synthetic) ValidateCrossReferences( + vea catalog.ValidationErrorAccumulator, vdg catalog.ValidationDescGetter, +) { +} +func (p synthetic) ValidateTxnCommit( + vea catalog.ValidationErrorAccumulator, vdg catalog.ValidationDescGetter, +) { +} +func (p synthetic) SchemaKind() catalog.ResolvedSchemaKind { return p.kind() } +func (p synthetic) SchemaDesc() *descpb.SchemaDescriptor { + log.Fatalf(context.TODO(), + "synthetic %s cannot be encoded", p.kindName()) + return nil // unreachable +} diff --git a/pkg/sql/catalog/schemadesc/temporary_schema_desc.go b/pkg/sql/catalog/schemadesc/temporary_schema_desc.go new file mode 100644 index 000000000000..f0c67a01b888 --- /dev/null +++ b/pkg/sql/catalog/schemadesc/temporary_schema_desc.go @@ -0,0 +1,54 @@ +// 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 schemadesc + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" +) + +// NewTemporarySchema returns a temporary schema with a given name, id, and +// parent. Temporary schemas do not have a durable descriptor in the store; +// they only have a namespace entry to indicate their existence. Given that, +// a different kind of "synthetic" descriptor is used to indicate temporary +// schemas. +// +// The returned descriptor carries only a basic functionality, requiring the +// caller to check the SchemaKind to determine how to use the descriptor. +func NewTemporarySchema(name string, id descpb.ID, parentDB descpb.ID) catalog.SchemaDescriptor { + return &temporary{ + synthetic: synthetic{temporaryBase{}}, + id: id, + name: name, + parentID: parentDB, + } +} + +// temporary represents the synthetic temporary schema. +type temporary struct { + synthetic + id descpb.ID + name string + parentID descpb.ID +} + +var _ catalog.SchemaDescriptor = temporary{} + +func (p temporary) GetID() descpb.ID { return p.id } +func (p temporary) GetName() string { return p.name } +func (p temporary) GetParentID() descpb.ID { return p.parentID } + +type temporaryBase struct{} + +func (t temporaryBase) kindName() string { return "temporary" } +func (t temporaryBase) kind() catalog.ResolvedSchemaKind { return catalog.SchemaTemporary } + +var _ syntheticBase = temporaryBase{} diff --git a/pkg/sql/catalog/schemadesc/virtual_schema_desc.go b/pkg/sql/catalog/schemadesc/virtual_schema_desc.go new file mode 100644 index 000000000000..2095e5801da7 --- /dev/null +++ b/pkg/sql/catalog/schemadesc/virtual_schema_desc.go @@ -0,0 +1,64 @@ +// 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 schemadesc + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" +) + +// GetVirtualSchemaByID returns a virtual schema with a given ID if it exists. +// +// The returned descriptor carries only a basic functionality, requiring the +// caller to check the SchemaKind to determine how to use the descriptor. The +// returned descriptor is not mapped to a database; every database has all of +// the same virtual schemas and the ParentID on the returned descriptor will be +// descpb.InvalidID. +func GetVirtualSchemaByID(id descpb.ID) (catalog.SchemaDescriptor, bool) { + sc, ok := virtualSchemasByID[id] + return sc, ok +} + +var virtualSchemasByID = func() map[descpb.ID]catalog.SchemaDescriptor { + m := make(map[descpb.ID]catalog.SchemaDescriptor, len(catconstants.StaticSchemaIDMap)) + for id, name := range catconstants.StaticSchemaIDMap { + id := descpb.ID(id) + sc := virtual{ + synthetic: synthetic{virtualBase{}}, + id: id, + name: name, + } + m[id] = sc + } + return m +}() + +// virtual represents the virtual schemas which are part of every database. +// See the commentary on GetVirtualSchemaByID. +type virtual struct { + synthetic + id descpb.ID + name string +} + +var _ catalog.SchemaDescriptor = virtual{} + +func (p virtual) GetID() descpb.ID { return p.id } +func (p virtual) GetName() string { return p.name } +func (p virtual) GetParentID() descpb.ID { return descpb.InvalidID } + +type virtualBase struct{} + +var _ syntheticBase = virtualBase{} + +func (v virtualBase) kindName() string { return "virtual" } +func (v virtualBase) kind() catalog.ResolvedSchemaKind { return catalog.SchemaVirtual } diff --git a/pkg/sql/catalog/validate.go b/pkg/sql/catalog/validate.go index bb2b6e846a2d..770ea9e3ebfd 100644 --- a/pkg/sql/catalog/validate.go +++ b/pkg/sql/catalog/validate.go @@ -14,7 +14,7 @@ import ( "context" "strings" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" + "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/errors" ) @@ -202,7 +202,7 @@ func (ve *validationErrors) CombinedError() error { // Otherwise, those not in the causal chain will be ignored. for _, err := range ve.errors { for _, key := range errors.GetTelemetryKeys(err) { - if strings.HasPrefix(key, catconstants.ValidationTelemetryKeyPrefix) { + if strings.HasPrefix(key, telemetry.ValidationTelemetryKeyPrefix) { extraTelemetryKeys = append(extraTelemetryKeys, key) } } @@ -328,7 +328,7 @@ func (vea *validationErrorAccumulator) decorate(err error) error { default: return err } - return errors.WithTelemetry(err, catconstants.ValidationTelemetryKeyPrefix+tkSuffix) + return errors.WithTelemetry(err, telemetry.ValidationTelemetryKeyPrefix+tkSuffix) } // ValidationDescGetter is used by the validation methods on Descriptor. diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index caf6540f1ee7..f9800e003062 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -69,7 +69,7 @@ import ( ) // CrdbInternalName is the name of the crdb_internal schema. -const CrdbInternalName = sessiondata.CRDBInternalSchemaName +const CrdbInternalName = catconstants.CRDBInternalSchemaName // Naming convention: // - if the response is served from memory, prefix with node_ @@ -313,7 +313,7 @@ CREATE TABLE crdb_internal.tables ( } dbNames := make(map[descpb.ID]string) scNames := make(map[descpb.ID]string) - scNames[keys.PublicSchemaID] = sessiondata.PublicSchemaName + scNames[keys.PublicSchemaID] = catconstants.PublicSchemaName // Record database descriptors for name lookups. for _, desc := range descs { if dbDesc, ok := desc.(catalog.DatabaseDescriptor); ok { @@ -402,9 +402,9 @@ CREATE TABLE crdb_internal.tables ( // Also add all the virtual descriptors. vt := p.getVirtualTabler() - vEntries := vt.getEntries() + vSchemas := vt.getSchemas() for _, virtSchemaName := range vt.getSchemaNames() { - e := vEntries[virtSchemaName] + e := vSchemas[virtSchemaName] for _, tName := range e.orderedDefNames { vTableEntry := e.defs[tName] if err := addDesc(vTableEntry.desc, tree.DNull, virtSchemaName); err != nil { @@ -4091,7 +4091,7 @@ CREATE TABLE crdb_internal.predefined_comments ( ) error { tableCommentKey := tree.NewDInt(keys.TableCommentType) vt := p.getVirtualTabler() - vEntries := vt.getEntries() + vEntries := vt.getSchemas() vSchemaNames := vt.getSchemaNames() for _, virtSchemaName := range vSchemaNames { diff --git a/pkg/sql/create_schema.go b/pkg/sql/create_schema.go index 350cfe1c244c..d5a24a1c7e6c 100644 --- a/pkg/sql/create_schema.go +++ b/pkg/sql/create_schema.go @@ -78,10 +78,10 @@ func CreateUserDefinedSchemaDescriptor( IncludeOffline: true, IncludeDropped: true, }) - if err != nil || sc.Kind != catalog.SchemaUserDefined { + if err != nil || sc.SchemaKind() != catalog.SchemaUserDefined { return nil, nil, err } - if sc.Desc.Dropped() { + if sc.Dropped() { return nil, nil, pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, "schema %q is being dropped, try again later", schemaName) diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 80d79346745c..924b412f9f6a 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -100,13 +100,13 @@ func (p *planner) getSchemaIDForCreate( if err != nil { return 0, err } - switch res.Kind { + switch res.SchemaKind() { case catalog.SchemaPublic, catalog.SchemaUserDefined: - return res.ID, nil + return res.GetID(), nil case catalog.SchemaVirtual: return 0, pgerror.Newf(pgcode.InsufficientPrivilege, "schema cannot be modified: %q", scName) default: - return 0, errors.AssertionFailedf("invalid schema kind for getSchemaIDForCreate: %d", res.Kind) + return 0, errors.AssertionFailedf("invalid schema kind for getSchemaIDForCreate: %d", res.SchemaKind()) } } diff --git a/pkg/sql/create_type.go b/pkg/sql/create_type.go index 4bb2c9938891..132350ad3fc1 100644 --- a/pkg/sql/create_type.go +++ b/pkg/sql/create_type.go @@ -419,10 +419,10 @@ func (n *createTypeNode) ReadingOwnWrites() {} // this seems to be how Postgres does it. // Add a test for this when we support granting privileges to schemas #50879. func inheritUsagePrivilegeFromSchema( - resolvedSchema catalog.ResolvedSchema, privs *descpb.PrivilegeDescriptor, + schema catalog.SchemaDescriptor, privs *descpb.PrivilegeDescriptor, ) { - switch resolvedSchema.Kind { + switch kind := schema.SchemaKind(); kind { case catalog.SchemaPublic: // If the type is in the public schema, the public role has USAGE on it. privs.Grant(security.PublicRoleName(), privilege.List{privilege.USAGE}) @@ -430,10 +430,9 @@ func inheritUsagePrivilegeFromSchema( // No types should be created in a temporary schema or a virtual schema. panic(errors.AssertionFailedf( "type being created in schema kind %d with id %d", - resolvedSchema.Kind, resolvedSchema.ID)) + kind, schema.GetID())) case catalog.SchemaUserDefined: - schemaDesc := resolvedSchema.Desc - schemaPrivs := schemaDesc.GetPrivileges() + schemaPrivs := schema.GetPrivileges() // Look for all users that have USAGE on the schema and add it to the // privilege descriptor. @@ -443,6 +442,6 @@ func inheritUsagePrivilegeFromSchema( } } default: - panic(errors.AssertionFailedf("unknown schema kind %d", resolvedSchema.Kind)) + panic(errors.AssertionFailedf("unknown schema kind %d", kind)) } } diff --git a/pkg/sql/delegate/BUILD.bazel b/pkg/sql/delegate/BUILD.bazel index b0a6fd73308c..5c739c4aa021 100644 --- a/pkg/sql/delegate/BUILD.bazel +++ b/pkg/sql/delegate/BUILD.bazel @@ -50,7 +50,6 @@ go_library( "//pkg/sql/privilege", "//pkg/sql/roleoption", "//pkg/sql/sem/tree", - "//pkg/sql/sessiondata", "//pkg/sql/sqltelemetry", "//pkg/util/errorutil/unimplemented", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/sql/delegate/show_enums.go b/pkg/sql/delegate/show_enums.go index cbbc5d610021..0d4bed7dfa2c 100644 --- a/pkg/sql/delegate/show_enums.go +++ b/pkg/sql/delegate/show_enums.go @@ -13,10 +13,10 @@ package delegate import ( "fmt" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" ) func (d *delegator) delegateShowEnums(n *tree.ShowEnums) (tree.Statement, error) { @@ -29,7 +29,7 @@ func (d *delegator) delegateShowEnums(n *tree.ShowEnums) (tree.Statement, error) schemaClause := "" if n.ExplicitSchema { schema := lex.EscapeSQLString(name.Schema()) - if name.Schema() == sessiondata.PgTempSchemaName { + if name.Schema() == catconstants.PgTempSchemaName { schema = lex.EscapeSQLString(d.evalCtx.SessionData.SearchPath.GetTemporarySchemaName()) } schemaClause = fmt.Sprintf("AND nsp.nspname = %s", schema) diff --git a/pkg/sql/delegate/show_tables.go b/pkg/sql/delegate/show_tables.go index 260b25e8f5af..fc04e95aea83 100644 --- a/pkg/sql/delegate/show_tables.go +++ b/pkg/sql/delegate/show_tables.go @@ -14,10 +14,10 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" ) var showEstimatedRowCountClusterSetting = settings.RegisterBoolSetting( @@ -49,7 +49,7 @@ func (d *delegator) delegateShowTables(n *tree.ShowTables) (tree.Statement, erro var schemaClause string if name.ExplicitSchema { schema := lex.EscapeSQLString(name.Schema()) - if name.Schema() == sessiondata.PgTempSchemaName { + if name.Schema() == catconstants.PgTempSchemaName { schema = lex.EscapeSQLString(d.evalCtx.SessionData.SearchPath.GetTemporarySchemaName()) } schemaClause = fmt.Sprintf("AND ns.nspname = %s", schema) diff --git a/pkg/sql/drop_cascade.go b/pkg/sql/drop_cascade.go index 352608df9474..bc70191f8a01 100644 --- a/pkg/sql/drop_cascade.go +++ b/pkg/sql/drop_cascade.go @@ -41,7 +41,7 @@ type dropCascadeState struct { } type schemaWithDbDesc struct { - schema *catalog.ResolvedSchema + schema catalog.SchemaDescriptor dbDesc *dbdesc.Mutable } @@ -55,9 +55,11 @@ func newDropCascadeState() *dropCascadeState { } func (d *dropCascadeState) collectObjectsInSchema( - ctx context.Context, p *planner, db *dbdesc.Mutable, schema *catalog.ResolvedSchema, + ctx context.Context, p *planner, db *dbdesc.Mutable, schema catalog.SchemaDescriptor, ) error { - names, _, err := resolver.GetObjectNamesAndIDs(ctx, p.txn, p, p.ExecCfg().Codec, db, schema.Name, true /* explicitPrefix */) + names, _, err := resolver.GetObjectNamesAndIDs( + ctx, p.txn, p, p.ExecCfg().Codec, db, schema.GetName(), true, /* explicitPrefix */ + ) if err != nil { return err } diff --git a/pkg/sql/drop_database.go b/pkg/sql/drop_database.go index 0dda22c29168..ad486cdcf3c2 100644 --- a/pkg/sql/drop_database.go +++ b/pkg/sql/drop_database.go @@ -92,7 +92,7 @@ func (p *planner) DropDatabase(ctx context.Context, n *tree.DropDatabase) (planN log.Warningf(ctx, "could not find schema %s under database %d", schema, dbDesc.ID) continue } - if err := d.collectObjectsInSchema(ctx, p, dbDesc, &res); err != nil { + if err := d.collectObjectsInSchema(ctx, p, dbDesc, res); err != nil { return nil, err } } @@ -133,7 +133,7 @@ func (n *dropDatabaseNode) startExec(params runParams) error { var schemasIDsToDelete []descpb.ID for _, schemaWithDbDesc := range n.d.schemasToDelete { schemaToDelete := schemaWithDbDesc.schema - switch schemaToDelete.Kind { + switch schemaToDelete.SchemaKind() { case catalog.SchemaTemporary, catalog.SchemaPublic: // The public schema and temporary schemas are cleaned up by just removing // the existing namespace entries. @@ -142,20 +142,20 @@ func (n *dropDatabaseNode) startExec(params runParams) error { p.txn, p.ExecCfg().Codec, n.dbDesc.GetID(), - schemaToDelete.Name, + schemaToDelete.GetName(), ); err != nil { return err } case catalog.SchemaUserDefined: // For user defined schemas, we have to do a bit more work. - mutDesc, ok := schemaToDelete.Desc.(*schemadesc.Mutable) + mutDesc, ok := schemaToDelete.(*schemadesc.Mutable) if !ok { - return errors.AssertionFailedf("expected Mutable, found %T", schemaToDelete.Desc) + return errors.AssertionFailedf("expected Mutable, found %T", schemaToDelete) } if err := params.p.dropSchemaImpl(ctx, n.dbDesc, mutDesc); err != nil { return err } - schemasIDsToDelete = append(schemasIDsToDelete, schemaToDelete.ID) + schemasIDsToDelete = append(schemasIDsToDelete, schemaToDelete.GetID()) } } diff --git a/pkg/sql/drop_schema.go b/pkg/sql/drop_schema.go index 5b286bcd9d45..f5dc3bcd09b1 100644 --- a/pkg/sql/drop_schema.go +++ b/pkg/sql/drop_schema.go @@ -79,19 +79,20 @@ func (p *planner) DropSchema(ctx context.Context, n *tree.DropSchema) (planNode, } return nil, pgerror.Newf(pgcode.InvalidSchemaName, "unknown schema %q", scName) } - switch sc.Kind { + switch sc.SchemaKind() { case catalog.SchemaPublic, catalog.SchemaVirtual, catalog.SchemaTemporary: return nil, pgerror.Newf(pgcode.InvalidSchemaName, "cannot drop schema %q", scName) case catalog.SchemaUserDefined: - hasOwnership, err := p.HasOwnership(ctx, sc.Desc) + hasOwnership, err := p.HasOwnership(ctx, sc) if err != nil { return nil, err } if !(isAdmin || hasOwnership) { - return nil, pgerror.Newf(pgcode.InsufficientPrivilege, "permission denied to drop schema %q", sc.Name) + return nil, pgerror.Newf(pgcode.InsufficientPrivilege, + "permission denied to drop schema %q", sc.GetName()) } namesBefore := len(d.objectNamesToDelete) - if err := d.collectObjectsInSchema(ctx, p, db, &sc); err != nil { + if err := d.collectObjectsInSchema(ctx, p, db, sc); err != nil { return nil, err } // We added some new objects to delete. Ensure that we have the correct @@ -102,7 +103,7 @@ func (p *planner) DropSchema(ctx context.Context, n *tree.DropSchema) (planNode, } sqltelemetry.IncrementUserDefinedSchemaCounter(sqltelemetry.UserDefinedSchemaDrop) default: - return nil, errors.AssertionFailedf("unknown schema kind %d", sc.Kind) + return nil, errors.AssertionFailedf("unknown schema kind %d", sc.SchemaKind()) } } @@ -134,10 +135,10 @@ func (n *dropSchemaNode) startExec(params runParams) error { schemaIDs := make([]descpb.ID, len(n.d.schemasToDelete)) for i := range n.d.schemasToDelete { sc := n.d.schemasToDelete[i].schema - schemaIDs[i] = sc.ID + schemaIDs[i] = sc.GetID() db := n.d.schemasToDelete[i].dbDesc - mutDesc := sc.Desc.(*schemadesc.Mutable) + mutDesc := sc.(*schemadesc.Mutable) if err := p.dropSchemaImpl(ctx, db, mutDesc); err != nil { return err } @@ -149,7 +150,7 @@ func (n *dropSchemaNode) startExec(params runParams) error { db := n.d.schemasToDelete[i].dbDesc if err := p.writeNonDropDatabaseChange( ctx, db, - fmt.Sprintf("updating parent database %s for %s", db.GetName(), sc.Name), + fmt.Sprintf("updating parent database %s for %s", db.GetName(), sc.GetName()), ); err != nil { return err } @@ -169,13 +170,13 @@ func (n *dropSchemaNode) startExec(params runParams) error { // in the same transaction as table descriptor update. for _, schemaToDelete := range n.d.schemasToDelete { sc := schemaToDelete.schema - qualifiedSchemaName, err := p.getQualifiedSchemaName(params.ctx, sc.Desc) + qualifiedSchemaName, err := p.getQualifiedSchemaName(params.ctx, sc) if err != nil { return err } if err := params.p.logEvent(params.ctx, - sc.ID, + sc.GetID(), &eventpb.DropSchema{ SchemaName: qualifiedSchemaName.String(), }); err != nil { diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index 3cad9ce04cd6..561e44fab5be 100755 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -25,11 +25,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/sql/vtable" "github.com/cockroachdb/errors" @@ -37,7 +37,7 @@ import ( ) const ( - pgCatalogName = sessiondata.PgCatalogName + pgCatalogName = catconstants.PgCatalogName ) var pgCatalogNameDString = tree.NewDString(pgCatalogName) @@ -45,7 +45,7 @@ var pgCatalogNameDString = tree.NewDString(pgCatalogName) // informationSchema lists all the table definitions for // information_schema. var informationSchema = virtualSchema{ - name: sessiondata.InformationSchemaName, + name: catconstants.InformationSchemaName, undefinedTables: buildStringSet( // Generated with: // select distinct '"'||table_name||'",' from information_schema.tables @@ -1067,13 +1067,13 @@ https://www.postgresql.org/docs/9.5/infoschema-schemata.html`, populate: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { return forEachDatabaseDesc(ctx, p, dbContext, true, /* requiresPrivileges */ func(db catalog.DatabaseDescriptor) error { - return forEachSchema(ctx, p, db, func(sc catalog.ResolvedSchema) error { + return forEachSchema(ctx, p, db, func(sc catalog.SchemaDescriptor) error { return addRow( tree.NewDString(db.GetName()), // catalog_name - tree.NewDString(sc.Name), // schema_name + tree.NewDString(sc.GetName()), // schema_name tree.DNull, // default_character_set_name tree.DNull, // sql_path - yesOrNoDatum(sc.Kind == catalog.SchemaUserDefined), // crdb_is_user_defined + yesOrNoDatum(sc.SchemaKind() == catalog.SchemaUserDefined), // crdb_is_user_defined ) }) }) @@ -1165,17 +1165,19 @@ CREATE TABLE information_schema.schema_privileges ( populate: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { return forEachDatabaseDesc(ctx, p, dbContext, true, /* requiresPrivileges */ func(db catalog.DatabaseDescriptor) error { - return forEachSchema(ctx, p, db, func(sc catalog.ResolvedSchema) error { + return forEachSchema(ctx, p, db, func(sc catalog.SchemaDescriptor) error { var privs []descpb.UserPrivilegeString - if sc.Kind == catalog.SchemaUserDefined { + if sc.SchemaKind() == catalog.SchemaUserDefined { // User defined schemas have their own privileges. - privs = sc.Desc.GetPrivileges().Show(privilege.Schema) + privs = sc.GetPrivileges().Show(privilege.Schema) } else { // Other schemas inherit from the parent database. + // TODO(ajwerner): Fix this because it's bogus for everything other + // than public. privs = db.GetPrivileges().Show(privilege.Database) } dbNameStr := tree.NewDString(db.GetName()) - scNameStr := tree.NewDString(sc.Name) + scNameStr := tree.NewDString(sc.GetName()) // TODO(knz): This should filter for the current user, see // https://github.com/cockroachdb/cockroach/issues/35572 for _, u := range privs { @@ -1185,7 +1187,7 @@ CREATE TABLE information_schema.schema_privileges ( // Non-user defined schemas inherit privileges from the database, // but the USAGE privilege is conferred by having SELECT privilege // on the database. (There is no SELECT privilege on schemas.) - if sc.Kind != catalog.SchemaUserDefined { + if sc.SchemaKind() != catalog.SchemaUserDefined { if privKind == privilege.SELECT { priv = privilege.USAGE.String() } else if !privilege.SchemaPrivileges.Contains(privKind) { @@ -1748,30 +1750,22 @@ func forEachSchema( ctx context.Context, p *planner, db catalog.DatabaseDescriptor, - fn func(sc catalog.ResolvedSchema) error, + fn func(sc catalog.SchemaDescriptor) error, ) error { schemaNames, err := getSchemaNames(ctx, p, db) if err != nil { return err } - vtableEntries := p.getVirtualTabler().getEntries() - schemas := make([]catalog.ResolvedSchema, 0, len(schemaNames)+len(vtableEntries)) + vtableEntries := p.getVirtualTabler().getSchemas() + schemas := make([]catalog.SchemaDescriptor, 0, len(schemaNames)+len(vtableEntries)) var userDefinedSchemaIDs []descpb.ID for id, name := range schemaNames { switch { - case strings.HasPrefix(name, sessiondata.PgTempSchemaName): - schemas = append(schemas, catalog.ResolvedSchema{ - Name: name, - ID: id, - Kind: catalog.SchemaTemporary, - }) + case strings.HasPrefix(name, catconstants.PgTempSchemaName): + schemas = append(schemas, schemadesc.NewTemporarySchema(name, id, db.GetID())) case name == tree.PublicSchema: - schemas = append(schemas, catalog.ResolvedSchema{ - Name: name, - ID: id, - Kind: catalog.SchemaPublic, - }) + schemas = append(schemas, schemadesc.GetPublicSchema()) default: // The default case is a user defined schema. Collect the ID to get the // descriptor later. @@ -1792,23 +1786,15 @@ func forEachSchema( if !canSeeDescriptor { continue } - schemas = append(schemas, catalog.ResolvedSchema{ - Name: desc.GetName(), - ID: desc.GetID(), - Kind: catalog.SchemaUserDefined, - Desc: desc, - }) + schemas = append(schemas, desc) } for _, schema := range vtableEntries { - schemas = append(schemas, catalog.ResolvedSchema{ - Name: schema.desc.GetName(), - Kind: catalog.SchemaVirtual, - }) + schemas = append(schemas, schema.Desc()) } sort.Slice(schemas, func(i int, j int) bool { - return schemas[i].Name < schemas[j].Name + return schemas[i].GetName() < schemas[j].GetName() }) for _, sc := range schemas { @@ -2099,7 +2085,7 @@ func forEachTableDescWithTableLookupInternalFromDescriptors( if virtualOpts == virtualMany || virtualOpts == virtualCurrentDB { // Virtual descriptors first. vt := p.getVirtualTabler() - vEntries := vt.getEntries() + vEntries := vt.getSchemas() vSchemaNames := vt.getSchemaNames() iterate := func(dbDesc catalog.DatabaseDescriptor) error { for _, virtSchemaName := range vSchemaNames { diff --git a/pkg/sql/opt/optbuilder/BUILD.bazel b/pkg/sql/opt/optbuilder/BUILD.bazel index d3c180356468..fa57d2824369 100644 --- a/pkg/sql/opt/optbuilder/BUILD.bazel +++ b/pkg/sql/opt/optbuilder/BUILD.bazel @@ -47,6 +47,7 @@ go_library( deps = [ "//pkg/server/telemetry", "//pkg/settings", + "//pkg/sql/catalog/catconstants", "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/schemaexpr", @@ -67,7 +68,6 @@ go_library( "//pkg/sql/privilege", "//pkg/sql/sem/builtins", "//pkg/sql/sem/tree", - "//pkg/sql/sessiondata", "//pkg/sql/sqlerrors", "//pkg/sql/sqltelemetry", "//pkg/sql/types", diff --git a/pkg/sql/opt/optbuilder/util.go b/pkg/sql/opt/optbuilder/util.go index 916c95aa1b67..4f19c2687232 100644 --- a/pkg/sql/opt/optbuilder/util.go +++ b/pkg/sql/opt/optbuilder/util.go @@ -11,6 +11,7 @@ package optbuilder import ( + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" @@ -19,7 +20,6 @@ import ( "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/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" @@ -436,10 +436,10 @@ func (b *Builder) resolveAndBuildScalar( func resolveTemporaryStatus(name *tree.TableName, persistence tree.Persistence) bool { // An explicit schema can only be provided in the CREATE TEMP TABLE statement // iff it is pg_temp. - if persistence.IsTemporary() && name.ExplicitSchema && name.SchemaName != sessiondata.PgTempSchemaName { + if persistence.IsTemporary() && name.ExplicitSchema && name.SchemaName != catconstants.PgTempSchemaName { panic(pgerror.New(pgcode.InvalidTableDefinition, "cannot create temporary relation in non-temporary schema")) } - return name.SchemaName == sessiondata.PgTempSchemaName || persistence.IsTemporary() + return name.SchemaName == catconstants.PgTempSchemaName || persistence.IsTemporary() } // resolveSchemaForCreate returns the schema that will contain a newly created diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 293ad01cd19c..136d4a8f8841 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -93,18 +93,18 @@ type optSchema struct { planner *planner database catalog.DatabaseDescriptor - schema catalog.ResolvedSchema + schema catalog.SchemaDescriptor name cat.SchemaName } // ID is part of the cat.Object interface. func (os *optSchema) ID() cat.StableID { - switch os.schema.Kind { + switch os.schema.SchemaKind() { case catalog.SchemaUserDefined, catalog.SchemaTemporary: // User defined schemas and the temporary schema have real ID's, so use // them here. - return cat.StableID(os.schema.ID) + return cat.StableID(os.schema.GetID()) default: // Virtual schemas and the public schema don't, so just fall back to the // parent database's ID. @@ -145,8 +145,8 @@ func (os *optSchema) GetDataSourceNames( func (os *optSchema) getDescriptorForPermissionsCheck() catalog.Descriptor { // If the schema is backed by a descriptor, then return it. - if os.schema.Kind == catalog.SchemaUserDefined { - return os.schema.Desc + if os.schema.SchemaKind() == catalog.SchemaUserDefined { + return os.schema } // Otherwise, just return the database descriptor. return os.database diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index 0528acd06725..a375b3269791 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -415,7 +415,7 @@ https://www.postgresql.org/docs/9.6/catalog-pg-cast.html`, schema: vtable.PGCatalogCast, populate: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { // TODO(someone): to populate this, we should split up the big PerformCast - // method in tree/eval.go into entries in a list. Then, this virtual table + // method in tree/eval.go into schemasByName in a list. Then, this virtual table // can simply range over the list. This would probably be better for // maintainability anyway. return nil @@ -1007,7 +1007,7 @@ func makeAllRelationsVirtualTableWithDescriptorIDIndex( if err != nil { return false, err } - if err := populateFromTable(ctx, p, h, db, sc.Name, table, scResolver, + if err := populateFromTable(ctx, p, h, db, sc.GetName(), table, scResolver, addRow); err != nil { return false, err } @@ -1712,19 +1712,22 @@ https://www.postgresql.org/docs/9.5/catalog-pg-namespace.html`, h := makeOidHasher() return forEachDatabaseDesc(ctx, p, dbContext, true, /* requiresPrivileges */ func(db catalog.DatabaseDescriptor) error { - return forEachSchema(ctx, p, db, func(sc catalog.ResolvedSchema) error { + return forEachSchema(ctx, p, db, func(sc catalog.SchemaDescriptor) error { ownerOID := tree.DNull - if sc.Kind == catalog.SchemaUserDefined { - ownerOID = getOwnerOID(sc.Desc) - } else if sc.Kind == catalog.SchemaPublic { + if sc.SchemaKind() == catalog.SchemaUserDefined { + ownerOID = getOwnerOID(sc) + } else if sc.SchemaKind() == catalog.SchemaPublic { // admin is the owner of the public schema. + // + // TODO(ajwerner): The public schema effectively carries the privileges + // of the database so consider using the database's owner for public. ownerOID = h.UserOid(security.MakeSQLUsernameFromPreNormalizedString("admin")) } return addRow( - h.NamespaceOid(db.GetID(), sc.Name), // oid - tree.NewDString(sc.Name), // nspname - ownerOID, // nspowner - tree.DNull, // nspacl + h.NamespaceOid(db.GetID(), sc.GetName()), // oid + tree.NewDString(sc.GetName()), // nspname + ownerOID, // nspowner + tree.DNull, // nspacl ) }) }) @@ -2465,7 +2468,7 @@ https://www.postgresql.org/docs/9.5/catalog-pg-type.html`, if err != nil { return false, err } - nspOid = h.NamespaceOid(db.GetID(), sc.Name) + nspOid = h.NamespaceOid(db.GetID(), sc.GetName()) typ, err = typDesc.MakeTypesT(ctx, tree.NewUnqualifiedTypeName(tree.Name(typDesc.GetName())), p) if err != nil { return false, err diff --git a/pkg/sql/pg_extension.go b/pkg/sql/pg_extension.go index fb88ad464a3f..cffbb1eb4266 100644 --- a/pkg/sql/pg_extension.go +++ b/pkg/sql/pg_extension.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -29,7 +28,7 @@ import ( // these tables and views on the public schema, but we instead do it in // our own defined virtual table / schema. var pgExtension = virtualSchema{ - name: sessiondata.PgExtensionSchemaName, + name: catconstants.PgExtensionSchemaName, tableDefs: map[descpb.ID]virtualSchemaDef{ catconstants.PgExtensionGeographyColumnsTableID: pgExtensionGeographyColumnsTable, catconstants.PgExtensionGeometryColumnsTableID: pgExtensionGeometryColumnsTable, diff --git a/pkg/sql/rename_table.go b/pkg/sql/rename_table.go index 603be9fe4be8..91ebd495fc99 100644 --- a/pkg/sql/rename_table.go +++ b/pkg/sql/rename_table.go @@ -107,7 +107,7 @@ func (n *renameTableNode) startExec(params runParams) error { prevDBID := tableDesc.ParentID var targetDbDesc catalog.DatabaseDescriptor - var targetSchemaDesc catalog.ResolvedSchema + var targetSchemaDesc catalog.SchemaDescriptor // If the target new name has no qualifications, then assume that the table // is intended to be renamed into the same database and schema. newTn := n.newTn @@ -210,7 +210,7 @@ func (n *renameTableNode) startExec(params runParams) error { params.p.txn, p.ExecCfg().Codec, targetDbDesc.GetID(), - targetSchemaDesc.ID, + targetSchemaDesc.GetID(), newTn, ) if err != nil { diff --git a/pkg/sql/resolver.go b/pkg/sql/resolver.go index 0b901ba4a06c..e32b1261f00b 100644 --- a/pkg/sql/resolver.go +++ b/pkg/sql/resolver.go @@ -52,7 +52,7 @@ func (p *planner) ResolveUncachedDatabaseByName( // ResolveUncachedSchemaDescriptor looks up a schema from the store. func (p *planner) ResolveUncachedSchemaDescriptor( ctx context.Context, dbID descpb.ID, name string, required bool, -) (found bool, schema catalog.ResolvedSchema, err error) { +) (found bool, schema catalog.SchemaDescriptor, err error) { p.runWithOptions(resolveFlags{skipCache: true}, func() { found, schema, err = p.Accessor().GetSchemaByName( ctx, p.txn, dbID, name, tree.SchemaLookupFlags{ @@ -67,7 +67,7 @@ func (p *planner) ResolveUncachedSchemaDescriptor( // from the store. func (p *planner) ResolveMutableSchemaDescriptor( ctx context.Context, dbID descpb.ID, name string, required bool, -) (found bool, schema catalog.ResolvedSchema, err error) { +) (found bool, schema catalog.SchemaDescriptor, err error) { return p.Accessor().GetSchemaByName( ctx, p.txn, dbID, name, tree.SchemaLookupFlags{ Required: required, @@ -152,7 +152,7 @@ func (p *planner) ResolveTargetObject( ctx context.Context, un *tree.UnresolvedObjectName, ) ( db catalog.DatabaseDescriptor, - schema catalog.ResolvedSchema, + schema catalog.SchemaDescriptor, namePrefix tree.ObjectNamePrefix, err error, ) { @@ -161,7 +161,7 @@ func (p *planner) ResolveTargetObject( prefix, namePrefix, err = resolver.ResolveTargetObject(ctx, p, un) }) if err != nil { - return nil, catalog.ResolvedSchema{}, namePrefix, err + return nil, nil, namePrefix, err } return prefix.Database, prefix.Schema, namePrefix, err } @@ -176,7 +176,7 @@ func (p *planner) LookupSchema( return false, nil, err } sc := p.Accessor() - var resolvedSchema catalog.ResolvedSchema + var resolvedSchema catalog.SchemaDescriptor found, resolvedSchema, err = sc.GetSchemaByName( ctx, p.txn, dbDesc.GetID(), scName, p.CommonLookupFlags(false /* required */), ) @@ -244,7 +244,7 @@ func (p *planner) IsTableVisible( if err != nil { return false, false, err } - if schemaDesc.Kind != catalog.SchemaVirtual { + if schemaDesc.SchemaKind() != catalog.SchemaVirtual { dbID := tableDesc.GetParentID() _, dbDesc, err := p.Descriptors().GetImmutableDatabaseByID(ctx, p.Txn(), dbID, tree.DatabaseLookupFlags{ @@ -261,7 +261,7 @@ func (p *planner) IsTableVisible( } iter := searchPath.Iter() for scName, ok := iter.Next(); ok; scName, ok = iter.Next() { - if schemaDesc.Name == scName { + if schemaDesc.GetName() == scName { return true, true, nil } } @@ -328,7 +328,7 @@ func (p *planner) GetTypeDescriptor( if err != nil { return tree.TypeName{}, nil, err } - name := tree.MakeNewQualifiedTypeName(dbDesc.GetName(), sc.Name, desc.GetName()) + name := tree.MakeNewQualifiedTypeName(dbDesc.GetName(), sc.GetName(), desc.GetName()) return name, desc, nil } @@ -462,12 +462,12 @@ func getDescriptorsFromTargetListForPrivilegeChange( if err != nil { return nil, err } - switch resSchema.Kind { + switch resSchema.SchemaKind() { case catalog.SchemaUserDefined: - descs = append(descs, resSchema.Desc) + descs = append(descs, resSchema) default: return nil, pgerror.Newf(pgcode.InvalidSchemaName, - "cannot change privileges on schema %q", resSchema.Name) + "cannot change privileges on schema %q", resSchema.GetName()) } } return descs, nil @@ -554,7 +554,7 @@ func (p *planner) getQualifiedTableName( return nil, err } schemaID := desc.GetParentSchemaID() - resolvedSchema, err := p.Descriptors().GetImmutableSchemaByID(ctx, p.txn, schemaID, + scDesc, err := p.Descriptors().GetImmutableSchemaByID(ctx, p.txn, schemaID, tree.SchemaLookupFlags{ IncludeOffline: true, IncludeDropped: true, @@ -564,7 +564,7 @@ func (p *planner) getQualifiedTableName( } tbName := tree.MakeTableNameWithSchema( tree.Name(dbDesc.GetName()), - tree.Name(resolvedSchema.Name), + tree.Name(scDesc.GetName()), tree.Name(desc.GetName()), ) return &tbName, nil @@ -623,14 +623,16 @@ func (p *planner) getQualifiedTypeName( } schemaID := desc.GetParentSchemaID() - resolvedSchema, err := p.Descriptors().GetImmutableSchemaByID(ctx, p.txn, schemaID, tree.SchemaLookupFlags{}) + scDesc, err := p.Descriptors().GetImmutableSchemaByID( + ctx, p.txn, schemaID, tree.SchemaLookupFlags{}, + ) if err != nil { return nil, err } typeName := tree.MakeNewQualifiedTypeName( dbDesc.GetName(), - resolvedSchema.Name, + scDesc.GetName(), desc.GetName(), ) diff --git a/pkg/sql/sem/tree/BUILD.bazel b/pkg/sql/sem/tree/BUILD.bazel index e10b789739fe..578781e8f487 100644 --- a/pkg/sql/sem/tree/BUILD.bazel +++ b/pkg/sql/sem/tree/BUILD.bazel @@ -122,6 +122,7 @@ go_library( "//pkg/server/telemetry", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/sql/catalog/catconstants", "//pkg/sql/lex", "//pkg/sql/lexbase", "//pkg/sql/pgwire/pgcode", diff --git a/pkg/sql/sem/tree/name_resolution.go b/pkg/sql/sem/tree/name_resolution.go index 80d0724e2567..5f5d0fa35b76 100644 --- a/pkg/sql/sem/tree/name_resolution.go +++ b/pkg/sql/sem/tree/name_resolution.go @@ -15,6 +15,7 @@ import ( "fmt" "strings" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -113,7 +114,7 @@ func classifyColumnItem(n *UnresolvedName) (VarName, error) { const ( // PublicSchema is the name of the physical schema in every // database/catalog. - PublicSchema string = sessiondata.PublicSchemaName + PublicSchema string = catconstants.PublicSchemaName // PublicSchemaName is the same, typed as Name. PublicSchemaName Name = Name(PublicSchema) ) @@ -320,7 +321,7 @@ func ResolveExisting( // database set. Therefore, we test this even if curDb == "", as long as the // schema name is for a virtual schema. - if _, isVirtualSchema := sessiondata.VirtualSchemaNames[scName]; isVirtualSchema || curDb != "" { + if _, isVirtualSchema := catconstants.VirtualSchemaNames[scName]; isVirtualSchema || curDb != "" { if found, objMeta, err := r.LookupObject(ctx, lookupFlags, curDb, scName, u.Object()); found || err != nil { if err == nil { namePrefix.CatalogName = Name(curDb) @@ -520,12 +521,12 @@ func (n *UnresolvedName) ResolveFunction( fullName := function - if prefix == sessiondata.PgCatalogName { + if prefix == catconstants.PgCatalogName { // If the user specified e.g. `pg_catalog.max()` we want to find // it in the global namespace. prefix = "" } - if prefix == sessiondata.PublicSchemaName { + if prefix == catconstants.PublicSchemaName { // If the user specified public, it may be from a PostgreSQL extension. // Double check the function definition allows resolution on the public // schema, and resolve as such if appropriate. diff --git a/pkg/sql/sessiondata/BUILD.bazel b/pkg/sql/sessiondata/BUILD.bazel index 89f9fb7777ee..88ccdbc162cb 100644 --- a/pkg/sql/sessiondata/BUILD.bazel +++ b/pkg/sql/sessiondata/BUILD.bazel @@ -13,6 +13,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/security", + "//pkg/sql/catalog/catconstants", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/pgwire/pgnotice", diff --git a/pkg/sql/sessiondata/search_path.go b/pkg/sql/sessiondata/search_path.go index 155564c0cf3e..866a2ded94be 100644 --- a/pkg/sql/sessiondata/search_path.go +++ b/pkg/sql/sessiondata/search_path.go @@ -14,47 +14,14 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" ) -// PgCatalogName is the name of the pg_catalog system schema. -const PgCatalogName = "pg_catalog" - -// PublicSchemaName is the name of the pg_catalog system schema. -const PublicSchemaName = "public" - -// UserSchemaName is the alias for schema names for users. -const UserSchemaName = "$user" - -// InformationSchemaName is the name of the information_schema system schema. -const InformationSchemaName = "information_schema" - -// CRDBInternalSchemaName is the name of the crdb_internal system schema. -const CRDBInternalSchemaName = "crdb_internal" - -// PgSchemaPrefix is a prefix for Postgres system schemas. Users cannot -// create schemas with this prefix. -const PgSchemaPrefix = "pg_" - -// PgTempSchemaName is the alias for temporary schemas across sessions. -const PgTempSchemaName = "pg_temp" - -// PgExtensionSchemaName is the alias for schemas which are usually "public" in postgres -// when installing an extension, but must be stored as a separate schema in CRDB. -const PgExtensionSchemaName = "pg_extension" - -// VirtualSchemaNames is a set of all virtual schema names. -var VirtualSchemaNames = map[string]struct{}{ - PgCatalogName: {}, - InformationSchemaName: {}, - CRDBInternalSchemaName: {}, - PgExtensionSchemaName: {}, -} - // DefaultSearchPath is the search path used by virgin sessions. var DefaultSearchPath = MakeSearchPath( - []string{UserSchemaName, PublicSchemaName}, + []string{catconstants.UserSchemaName, catconstants.PublicSchemaName}, ) // SearchPath represents a list of namespaces to search builtins in. @@ -85,11 +52,11 @@ func MakeSearchPath(paths []string) SearchPath { containsPgTempSchema := false for _, e := range paths { switch e { - case PgCatalogName: + case catconstants.PgCatalogName: containsPgCatalog = true - case PgTempSchemaName: + case catconstants.PgTempSchemaName: containsPgTempSchema = true - case PgExtensionSchemaName: + case catconstants.PgExtensionSchemaName: containsPgExtension = true } } @@ -140,12 +107,12 @@ func (s SearchPath) UpdatePaths(paths []string) SearchPath { // through for all other schema names. func (s SearchPath) MaybeResolveTemporarySchema(schemaName string) (string, error) { // Only allow access to the session specific temporary schema. - if strings.HasPrefix(schemaName, PgTempSchemaName) && schemaName != PgTempSchemaName && schemaName != s.tempSchemaName { + if strings.HasPrefix(schemaName, catconstants.PgTempSchemaName) && schemaName != catconstants.PgTempSchemaName && schemaName != s.tempSchemaName { return schemaName, pgerror.New(pgcode.FeatureNotSupported, "cannot access temporary tables of other sessions") } // If the schemaName is pg_temp and the tempSchemaName has been set, pg_temp // is an alias the session specific temp schema. - if schemaName == PgTempSchemaName && s.tempSchemaName != "" { + if schemaName == catconstants.PgTempSchemaName && s.tempSchemaName != "" { return s.tempSchemaName, nil } return schemaName, nil @@ -270,7 +237,7 @@ func (iter *SearchPathIter) Next() (path string, ok bool) { } if iter.implicitPgCatalog { iter.implicitPgCatalog = false - return PgCatalogName, true + return catconstants.PgCatalogName, true } if iter.i < len(iter.paths) { @@ -278,7 +245,7 @@ func (iter *SearchPathIter) Next() (path string, ok bool) { // If pg_temp is explicitly present in the paths, it must be resolved to the // session specific temp schema (if one exists). tempSchemaName is set in the // iterator iff the session has created a temporary schema. - if iter.paths[iter.i-1] == PgTempSchemaName { + if iter.paths[iter.i-1] == catconstants.PgTempSchemaName { // If the session specific temporary schema has not been created we can // preempt the resolution failure and iterate to the next entry. if iter.tempSchemaName == "" { @@ -286,7 +253,7 @@ func (iter *SearchPathIter) Next() (path string, ok bool) { } return iter.tempSchemaName, true } - if iter.paths[iter.i-1] == UserSchemaName { + if iter.paths[iter.i-1] == catconstants.UserSchemaName { // In case the user schema name is unset, we simply iterate to the next // entry. if iter.userSchemaName == "" { @@ -295,11 +262,11 @@ func (iter *SearchPathIter) Next() (path string, ok bool) { return iter.userSchemaName, true } // pg_extension should be read before delving into the schema. - if iter.paths[iter.i-1] == PublicSchemaName && iter.implicitPgExtension { + if iter.paths[iter.i-1] == catconstants.PublicSchemaName && iter.implicitPgExtension { iter.implicitPgExtension = false // Go back one so `public` can be found again next. iter.i-- - return PgExtensionSchemaName, true + return catconstants.PgExtensionSchemaName, true } return iter.paths[iter.i-1], true } diff --git a/pkg/sql/sessiondata/session_data.go b/pkg/sql/sessiondata/session_data.go index f8c4e6623fed..e1af485e76f0 100644 --- a/pkg/sql/sessiondata/session_data.go +++ b/pkg/sql/sessiondata/session_data.go @@ -264,13 +264,21 @@ type LocalOnlySessionData struct { // IsTemporarySchemaID returns true if the given ID refers to any of the temp // schemas created by the session. -func (s *SessionData) IsTemporarySchemaID(ID uint32) bool { - for _, tempSchemaID := range s.DatabaseIDToTempSchemaID { - if tempSchemaID == ID { - return true +func (s *SessionData) IsTemporarySchemaID(schemaID uint32) bool { + _, exists := s.MaybeGetDatabaseForTemporarySchemaID(schemaID) + return exists +} + +// MaybeGetDatabaseForTemporarySchemaID returns the corresponding database and +// true if the schemaID refers to any of the temp schemas created by this +// session. +func (s *SessionData) MaybeGetDatabaseForTemporarySchemaID(schemaID uint32) (uint32, bool) { + for dbID, tempSchemaID := range s.DatabaseIDToTempSchemaID { + if tempSchemaID == schemaID { + return dbID, true } } - return false + return 0, false } // GetTemporarySchemaIDForDb returns the schemaID for the temporary schema if diff --git a/pkg/sql/set_schema.go b/pkg/sql/set_schema.go index d1d2a0e13dea..e4a7be520599 100644 --- a/pkg/sql/set_schema.go +++ b/pkg/sql/set_schema.go @@ -51,7 +51,7 @@ func (p *planner) prepareSetSchema( return 0, err } - switch res.Kind { + switch res.SchemaKind() { case catalog.SchemaTemporary: return 0, pgerror.Newf(pgcode.FeatureNotSupported, "cannot move objects into or out of temporary schemas") @@ -63,13 +63,13 @@ func (p *planner) prepareSetSchema( default: // The user needs CREATE privilege on the target schema to move an object // to the schema. - err = p.CheckPrivilege(ctx, res.Desc, privilege.CREATE) + err = p.CheckPrivilege(ctx, res, privilege.CREATE) if err != nil { return 0, err } } - desiredSchemaID := res.ID + desiredSchemaID := res.GetID() // If the schema being changed to is the same as the current schema a no-op // will happen so we don't have to check if there is an object in the schema diff --git a/pkg/sql/temporary_schema_test.go b/pkg/sql/temporary_schema_test.go index 7c2afaf5b21a..104976b08f7b 100644 --- a/pkg/sql/temporary_schema_test.go +++ b/pkg/sql/temporary_schema_test.go @@ -22,10 +22,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/tests" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -396,7 +396,7 @@ func constructNameToIDMapping( require.NoError(t, err) namesToID[name] = descpb.ID(id) - if strings.HasPrefix(name, sessiondata.PgTempSchemaName) { + if strings.HasPrefix(name, catconstants.PgTempSchemaName) { tempSchemaNames = append(tempSchemaNames, name) } } diff --git a/pkg/sql/type_change.go b/pkg/sql/type_change.go index 286125c0b0d9..50e01c505115 100644 --- a/pkg/sql/type_change.go +++ b/pkg/sql/type_change.go @@ -993,8 +993,11 @@ func (t *typeSchemaChanger) canRemoveEnumValueFromArrayUsages( if err != nil { return err } - fqName := tree.MakeTableNameWithSchema(tree.Name(dbDesc.GetName()), tree.Name(parentSchema.Name), tree.Name(desc.GetName())) - + fqName := tree.MakeTableNameWithSchema( + tree.Name(dbDesc.GetName()), + tree.Name(parentSchema.GetName()), + tree.Name(desc.GetName()), + ) return pgerror.Newf(pgcode.DependentObjectsStillExist, "could not remove enum value %q as it is being used by table %q", member.LogicalRepresentation, fqName.FQString(), ) diff --git a/pkg/sql/virtual_schema.go b/pkg/sql/virtual_schema.go index 52e3f3738ecb..d2efb9dd3665 100644 --- a/pkg/sql/virtual_schema.go +++ b/pkg/sql/virtual_schema.go @@ -17,20 +17,18 @@ import ( "sort" "time" - "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" "github.com/cockroachdb/cockroach/pkg/sql/opt/constraint" "github.com/cockroachdb/cockroach/pkg/sql/parser" "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/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" @@ -179,7 +177,7 @@ func (t virtualSchemaTable) initVirtualTableDesc( id, nil, /* regionConfig */ startTime, /* creationTime */ - publicSelectPrivileges, + descpb.NewPublicSelectPrivilegeDescriptor(), nil, /* affected */ nil, /* semaCtx */ nil, /* evalCtx */ @@ -248,7 +246,7 @@ func (v virtualSchemaView) initVirtualTableDesc( id, columns, startTime, /* creationTime */ - publicSelectPrivileges, + descpb.NewPublicSelectPrivilegeDescriptor(), nil, /* semaCtx */ nil, /* evalCtx */ tree.PersistencePermanent, @@ -294,32 +292,37 @@ var startTime = hlc.Timestamp{ // created directly, but instead will be populated in a post-startup hook // on an Executor. type VirtualSchemaHolder struct { - entries map[string]*virtualSchemaEntry - defsByID map[descpb.ID]*virtualDefEntry - orderedNames []string + schemasByName map[string]*virtualSchemaEntry + schemasByID map[descpb.ID]*virtualSchemaEntry + defsByID map[descpb.ID]*virtualDefEntry + orderedNames []string } var _ VirtualTabler = (*VirtualSchemaHolder)(nil) -// GetVirtualSchema makes VirtualSchemaHolder implement schema.VirtualSchemas. +// GetVirtualSchema makes VirtualSchemaHolder implement catalog.VirtualSchemas. func (vs *VirtualSchemaHolder) GetVirtualSchema(schemaName string) (catalog.VirtualSchema, bool) { - virtualSchema, ok := vs.entries[schemaName] - return virtualSchema, ok + sc, ok := vs.schemasByName[schemaName] + return sc, ok +} + +// GetVirtualSchemaByID makes VirtualSchemaHolder implement catalog.VirtualSchemas. +func (vs *VirtualSchemaHolder) GetVirtualSchemaByID(id descpb.ID) (catalog.VirtualSchema, bool) { + sc, ok := vs.schemasByID[id] + return sc, ok } var _ catalog.VirtualSchemas = (*VirtualSchemaHolder)(nil) type virtualSchemaEntry struct { - // TODO(ajwerner): Use a descpb.SchemaDescriptor here as part of the - // user-defined schema work. - desc catalog.DatabaseDescriptor + desc catalog.SchemaDescriptor defs map[string]*virtualDefEntry orderedDefNames []string undefinedTables map[string]struct{} containsTypes bool } -func (v *virtualSchemaEntry) Desc() catalog.Descriptor { +func (v *virtualSchemaEntry) Desc() catalog.SchemaDescriptor { return v.desc } @@ -642,15 +645,22 @@ func NewVirtualSchemaHolder( ctx context.Context, st *cluster.Settings, ) (*VirtualSchemaHolder, error) { vs := &VirtualSchemaHolder{ - entries: make(map[string]*virtualSchemaEntry, len(virtualSchemas)), - orderedNames: make([]string, len(virtualSchemas)), - defsByID: make(map[descpb.ID]*virtualDefEntry, math.MaxUint32-catconstants.MinVirtualID), + schemasByName: make(map[string]*virtualSchemaEntry, len(virtualSchemas)), + schemasByID: make(map[descpb.ID]*virtualSchemaEntry, len(virtualSchemas)), + orderedNames: make([]string, len(virtualSchemas)), + defsByID: make(map[descpb.ID]*virtualDefEntry, math.MaxUint32-catconstants.MinVirtualID), } order := 0 for schemaID, schema := range virtualSchemas { - dbName := schema.name - dbDesc := initVirtualDatabaseDesc(schemaID, dbName) + scDesc, ok := schemadesc.GetVirtualSchemaByID(schemaID) + if !ok { + return nil, errors.AssertionFailedf("failed to find virtual schema %d (%s)", schemaID, schema.name) + } + if scDesc.GetName() != schema.name { + return nil, errors.AssertionFailedf("schema name mismatch for virtual schema %d: expected %s, found %s", + schemaID, schema.name, scDesc.GetName()) + } defs := make(map[string]*virtualDefEntry, len(schema.tableDefs)) orderedDefNames := make([]string, 0, len(schema.tableDefs)) @@ -687,37 +697,22 @@ func NewVirtualSchemaHolder( sort.Strings(orderedDefNames) - vs.entries[dbName] = &virtualSchemaEntry{ - desc: dbDesc, + vse := &virtualSchemaEntry{ + desc: scDesc, defs: defs, orderedDefNames: orderedDefNames, undefinedTables: schema.undefinedTables, containsTypes: schema.containsTypes, } - vs.orderedNames[order] = dbName + vs.schemasByName[scDesc.GetName()] = vse + vs.schemasByID[scDesc.GetID()] = vse + vs.orderedNames[order] = scDesc.GetName() order++ } sort.Strings(vs.orderedNames) return vs, nil } -// Virtual databases and tables each have SELECT privileges for "public", which includes -// all users. However, virtual schemas have more fine-grained access control. -// For instance, information_schema will only expose rows to a given user which that -// user has access to. -var publicSelectPrivileges = descpb.NewPrivilegeDescriptor( - security.PublicRoleName(), privilege.List{privilege.SELECT}, security.NodeUserName(), -) - -func initVirtualDatabaseDesc(id descpb.ID, name string) catalog.DatabaseDescriptor { - return dbdesc.NewBuilder(&descpb.DatabaseDescriptor{ - Name: name, - ID: id, - Version: 1, - Privileges: publicSelectPrivileges, - }).BuildImmutableDatabase() -} - func newUnimplementedVirtualTableError(schema, tableName string) error { return unimplemented.Newf( fmt.Sprintf("%s.%s", schema, tableName), @@ -728,8 +723,8 @@ func newUnimplementedVirtualTableError(schema, tableName string) error { } // getEntries is part of the VirtualTabler interface. -func (vs *VirtualSchemaHolder) getEntries() map[string]*virtualSchemaEntry { - return vs.entries +func (vs *VirtualSchemaHolder) getSchemas() map[string]*virtualSchemaEntry { + return vs.schemasByName } // getSchemaNames is part of the VirtualTabler interface. @@ -740,7 +735,7 @@ func (vs *VirtualSchemaHolder) getSchemaNames() []string { // getVirtualSchemaEntry retrieves a virtual schema entry given a database name. // getVirtualSchemaEntry is part of the VirtualTabler interface. func (vs *VirtualSchemaHolder) getVirtualSchemaEntry(name string) (*virtualSchemaEntry, bool) { - e, ok := vs.entries[name] + e, ok := vs.schemasByName[name] return e, ok } @@ -784,7 +779,7 @@ type VirtualTabler interface { getVirtualSchemaEntry(name string) (*virtualSchemaEntry, bool) getVirtualTableEntry(tn *tree.TableName) (*virtualDefEntry, error) getVirtualTableEntryByID(id descpb.ID) (*virtualDefEntry, error) - getEntries() map[string]*virtualSchemaEntry + getSchemas() map[string]*virtualSchemaEntry getSchemaNames() []string } From affbbd636a82e6c56e1ba014f19fc9782e7230b5 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Tue, 1 Jun 2021 09:15:40 -0400 Subject: [PATCH 2/7] sql: remove boolean return arg from database retrieval methods Broadly part of clean up. Release note: None --- pkg/sql/alter_database.go | 10 +++--- pkg/sql/alter_schema.go | 2 +- pkg/sql/catalog/descs/collection.go | 40 ++++++++++++------------ pkg/sql/catalog/descs/collection_test.go | 14 ++++----- pkg/sql/comment_on_database.go | 2 +- pkg/sql/create_schema.go | 2 +- pkg/sql/drop_database.go | 4 +-- pkg/sql/drop_schema.go | 2 +- pkg/sql/reassign_owned_by.go | 2 +- pkg/sql/region_util.go | 6 ++-- pkg/sql/rename_database.go | 2 +- pkg/sql/rename_table.go | 2 +- pkg/sql/reparent_database.go | 4 +-- pkg/sql/resolver.go | 10 +++--- pkg/sql/scrub.go | 2 +- pkg/sql/set_zone_config.go | 2 +- pkg/sql/show_zone_config.go | 2 +- pkg/sql/vars.go | 2 +- pkg/sql/virtual_schema.go | 2 +- pkg/sql/virtual_table.go | 2 +- 20 files changed, 57 insertions(+), 57 deletions(-) diff --git a/pkg/sql/alter_database.go b/pkg/sql/alter_database.go index 7f5c73536395..ca1c1455af4b 100644 --- a/pkg/sql/alter_database.go +++ b/pkg/sql/alter_database.go @@ -52,7 +52,7 @@ func (p *planner) AlterDatabaseOwner( return nil, err } - _, dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name), + dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name), tree.DatabaseLookupFlags{Required: true}) if err != nil { return nil, err @@ -133,7 +133,7 @@ func (p *planner) AlterDatabaseAddRegion( return nil, err } - _, dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name), + dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name), tree.DatabaseLookupFlags{Required: true}, ) if err != nil { @@ -292,7 +292,7 @@ func (p *planner) AlterDatabaseDropRegion( return nil, err } - _, dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name), + dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name), tree.DatabaseLookupFlags{Required: true}) if err != nil { return nil, err @@ -643,7 +643,7 @@ func (p *planner) AlterDatabasePrimaryRegion( return nil, err } - _, dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name), + dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name), tree.DatabaseLookupFlags{Required: true}, ) if err != nil { @@ -957,7 +957,7 @@ func (p *planner) AlterDatabaseSurvivalGoal( return nil, err } - _, dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name), + dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name), tree.DatabaseLookupFlags{Required: true}, ) if err != nil { diff --git a/pkg/sql/alter_schema.go b/pkg/sql/alter_schema.go index a1c788bb133d..d4e6bf1d7284 100644 --- a/pkg/sql/alter_schema.go +++ b/pkg/sql/alter_schema.go @@ -53,7 +53,7 @@ func (p *planner) AlterSchema(ctx context.Context, n *tree.AlterSchema) (planNod if n.Schema.ExplicitCatalog { dbName = n.Schema.Catalog() } - _, db, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, dbName, + db, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, dbName, tree.DatabaseLookupFlags{Required: true}) if err != nil { return nil, err diff --git a/pkg/sql/catalog/descs/collection.go b/pkg/sql/catalog/descs/collection.go index 44940a6b588c..ec9fe99cef64 100644 --- a/pkg/sql/catalog/descs/collection.go +++ b/pkg/sql/catalog/descs/collection.go @@ -443,20 +443,20 @@ func (tc *Collection) getDescriptorFromStore( // properties according to the provided lookup flags. RequireMutable is ignored. func (tc *Collection) GetMutableDatabaseByName( ctx context.Context, txn *kv.Txn, name string, flags tree.DatabaseLookupFlags, -) (found bool, _ *dbdesc.Mutable, _ error) { +) (*dbdesc.Mutable, error) { flags.RequireMutable = true - found, desc, err := tc.getDatabaseByName(ctx, txn, name, flags) - if err != nil || !found { - return false, nil, err + desc, err := tc.getDatabaseByName(ctx, txn, name, flags) + if err != nil || desc == nil { + return nil, err } - return true, desc.(*dbdesc.Mutable), nil + return desc.(*dbdesc.Mutable), nil } // GetImmutableDatabaseByName returns an immutable database descriptor with // properties according to the provided lookup flags. RequireMutable is ignored. func (tc *Collection) GetImmutableDatabaseByName( ctx context.Context, txn *kv.Txn, name string, flags tree.DatabaseLookupFlags, -) (found bool, _ catalog.DatabaseDescriptor, _ error) { +) (catalog.DatabaseDescriptor, error) { flags.RequireMutable = false return tc.getDatabaseByName(ctx, txn, name, flags) } @@ -468,24 +468,24 @@ func (tc *Collection) GetImmutableDatabaseByName( func (tc *Collection) GetDatabaseDesc( ctx context.Context, txn *kv.Txn, name string, flags tree.DatabaseLookupFlags, ) (desc catalog.DatabaseDescriptor, err error) { - _, desc, err = tc.getDatabaseByName(ctx, txn, name, flags) - return desc, err + return tc.getDatabaseByName(ctx, txn, name, flags) } // getDatabaseByName returns a database descriptor with properties according to // the provided lookup flags. func (tc *Collection) getDatabaseByName( ctx context.Context, txn *kv.Txn, name string, flags tree.DatabaseLookupFlags, -) (bool, catalog.DatabaseDescriptor, error) { +) (catalog.DatabaseDescriptor, error) { if name == systemschema.SystemDatabaseName { // The system database descriptor should never actually be mutated, which is // why we return the same hard-coded descriptor every time. It's assumed // that callers of this method will check the privileges on the descriptor // (like any other database) and return an error. if flags.RequireMutable { - return true, dbdesc.NewBuilder(systemschema.MakeSystemDatabaseDesc().DatabaseDesc()).BuildExistingMutableDatabase(), nil + proto := systemschema.MakeSystemDatabaseDesc().DatabaseDesc() + return dbdesc.NewBuilder(proto).BuildExistingMutableDatabase(), nil } - return true, systemschema.MakeSystemDatabaseDesc(), nil + return systemschema.MakeSystemDatabaseDesc(), nil } getDatabaseByName := func() (found bool, _ catalog.Descriptor, err error) { @@ -519,24 +519,24 @@ func (tc *Collection) getDatabaseByName( found, desc, err := getDatabaseByName() if err != nil { - return false, nil, err + return nil, err } else if !found { if flags.Required { - return false, nil, sqlerrors.NewUndefinedDatabaseError(name) + return nil, sqlerrors.NewUndefinedDatabaseError(name) } - return false, nil, nil + return nil, nil } db, ok := desc.(catalog.DatabaseDescriptor) if !ok { if flags.Required { - return false, nil, sqlerrors.NewUndefinedDatabaseError(name) + return nil, sqlerrors.NewUndefinedDatabaseError(name) } - return false, nil, nil + return nil, nil } if dropped, err := filterDescriptorState(db, flags.Required, flags); err != nil || dropped { - return false, nil, err + return nil, err } - return true, db, nil + return db, nil } // GetObjectDesc looks up an object by name and returns both its @@ -612,14 +612,14 @@ func (tc *Collection) getObjectByName( // in the face of a concurrent rename. avoidCachedForParent := flags.AvoidCached || flags.RequireMutable // Resolve the database. - found, db, err := tc.GetImmutableDatabaseByName(ctx, txn, catalogName, + db, err := tc.GetImmutableDatabaseByName(ctx, txn, catalogName, tree.DatabaseLookupFlags{ Required: flags.Required, AvoidCached: avoidCachedForParent, IncludeDropped: flags.IncludeDropped, IncludeOffline: flags.IncludeOffline, }) - if err != nil || !found { + if err != nil || db == nil { return false, nil, err } dbID := db.GetID() diff --git a/pkg/sql/catalog/descs/collection_test.go b/pkg/sql/catalog/descs/collection_test.go index accdba2f811c..e71836837df4 100644 --- a/pkg/sql/catalog/descs/collection_test.go +++ b/pkg/sql/catalog/descs/collection_test.go @@ -225,11 +225,11 @@ func TestAddUncommittedDescriptorAndMutableResolution(t *testing.T) { flags.RequireMutable = true flags.Required = true - _, db, err := descriptors.GetMutableDatabaseByName(ctx, txn, "db", flags) + db, err := descriptors.GetMutableDatabaseByName(ctx, txn, "db", flags) require.NoError(t, err) dbID = db.GetID() - _, resolved, err := descriptors.GetMutableDatabaseByName(ctx, txn, "db", flags) + resolved, err := descriptors.GetMutableDatabaseByName(ctx, txn, "db", flags) require.NoError(t, err) require.Same(t, db, resolved) @@ -244,7 +244,7 @@ func TestAddUncommittedDescriptorAndMutableResolution(t *testing.T) { flags.RequireMutable = false - _, immByName, err := descriptors.GetImmutableDatabaseByName(ctx, txn, "db", flags) + immByName, err := descriptors.GetImmutableDatabaseByName(ctx, txn, "db", flags) require.NoError(t, err) require.Equal(t, mut.OriginalVersion(), immByName.GetVersion()) @@ -263,20 +263,20 @@ func TestAddUncommittedDescriptorAndMutableResolution(t *testing.T) { require.NoError(t, err) // Try to get the database descriptor by the old name and fail. - _, failedToResolve, err := descriptors.GetImmutableDatabaseByName(ctx, txn, "db", flags) + failedToResolve, err := descriptors.GetImmutableDatabaseByName(ctx, txn, "db", flags) require.Regexp(t, `database "db" does not exist`, err) require.Nil(t, failedToResolve) // Try to get the database descriptor by the new name and succeed but get // the old version with the old name (this is bizarre but is the // contract now). - _, immResolvedWithNewNameButHasOldName, err := descriptors.GetImmutableDatabaseByName(ctx, txn, "new_name", flags) + immResolvedWithNewNameButHasOldName, err := descriptors.GetImmutableDatabaseByName(ctx, txn, "new_name", flags) require.NoError(t, err) require.Same(t, immByID, immResolvedWithNewNameButHasOldName) require.NoError(t, descriptors.AddUncommittedDescriptor(mut)) - _, immByNameAfter, err := descriptors.GetImmutableDatabaseByName(ctx, txn, "new_name", flags) + immByNameAfter, err := descriptors.GetImmutableDatabaseByName(ctx, txn, "new_name", flags) require.NoError(t, err) require.Equal(t, db.GetVersion(), immByNameAfter.GetVersion()) require.Equal(t, mut.ImmutableCopy(), immByNameAfter) @@ -488,7 +488,7 @@ CREATE TABLE test.schema.t(x INT); s.InternalExecutor().(sqlutil.InternalExecutor), kvDB, func(ctx context.Context, txn *kv.Txn, descsCol *descs.Collection) error { - _, dbDesc, err := descsCol.GetImmutableDatabaseByName(ctx, txn, "test", tree.DatabaseLookupFlags{Required: true}) + dbDesc, err := descsCol.GetImmutableDatabaseByName(ctx, txn, "test", tree.DatabaseLookupFlags{Required: true}) if err != nil { return err } diff --git a/pkg/sql/comment_on_database.go b/pkg/sql/comment_on_database.go index 6e0b648a4652..7f9052fbe093 100644 --- a/pkg/sql/comment_on_database.go +++ b/pkg/sql/comment_on_database.go @@ -41,7 +41,7 @@ func (p *planner) CommentOnDatabase( return nil, err } - _, dbDesc, err := p.Descriptors().GetImmutableDatabaseByName(ctx, p.txn, + dbDesc, err := p.Descriptors().GetImmutableDatabaseByName(ctx, p.txn, string(n.Name), tree.DatabaseLookupFlags{Required: true}) if err != nil { return nil, err diff --git a/pkg/sql/create_schema.go b/pkg/sql/create_schema.go index d5a24a1c7e6c..28a9a811e680 100644 --- a/pkg/sql/create_schema.go +++ b/pkg/sql/create_schema.go @@ -167,7 +167,7 @@ func (p *planner) createUserDefinedSchema(params runParams, n *tree.CreateSchema dbName = n.Schema.Catalog() } - _, db, err := p.Descriptors().GetMutableDatabaseByName(params.ctx, p.txn, dbName, + db, err := p.Descriptors().GetMutableDatabaseByName(params.ctx, p.txn, dbName, tree.DatabaseLookupFlags{Required: true}) if err != nil { return err diff --git a/pkg/sql/drop_database.go b/pkg/sql/drop_database.go index ad486cdcf3c2..57ff09c79baf 100644 --- a/pkg/sql/drop_database.go +++ b/pkg/sql/drop_database.go @@ -62,12 +62,12 @@ func (p *planner) DropDatabase(ctx context.Context, n *tree.DropDatabase) (planN } // Check that the database exists. - found, dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name), + dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name), tree.DatabaseLookupFlags{Required: !n.IfExists}) if err != nil { return nil, err } - if !found { + if dbDesc == nil { // IfExists was specified and database was not found. return newZeroNode(nil /* columns */), nil } diff --git a/pkg/sql/drop_schema.go b/pkg/sql/drop_schema.go index f5dc3bcd09b1..98b9d121cbd0 100644 --- a/pkg/sql/drop_schema.go +++ b/pkg/sql/drop_schema.go @@ -63,7 +63,7 @@ func (p *planner) DropSchema(ctx context.Context, n *tree.DropSchema) (planNode, } scName := schema.Schema() - _, db, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, dbName, + db, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, dbName, tree.DatabaseLookupFlags{Required: true}) if err != nil { return nil, err diff --git a/pkg/sql/reassign_owned_by.go b/pkg/sql/reassign_owned_by.go index de5edf36b35f..e758571576f5 100644 --- a/pkg/sql/reassign_owned_by.go +++ b/pkg/sql/reassign_owned_by.go @@ -64,7 +64,7 @@ func (n *reassignOwnedByNode) startExec(params runParams) error { // Filter for all objects in current database. currentDatabase := params.p.CurrentDatabase() - _, currentDbDesc, err := params.p.Descriptors().GetMutableDatabaseByName( + currentDbDesc, err := params.p.Descriptors().GetMutableDatabaseByName( params.ctx, params.p.txn, currentDatabase, tree.DatabaseLookupFlags{Required: true}) if err != nil { return err diff --git a/pkg/sql/region_util.go b/pkg/sql/region_util.go index c06452272882..add03583df4f 100644 --- a/pkg/sql/region_util.go +++ b/pkg/sql/region_util.go @@ -814,7 +814,7 @@ func partitionByForRegionalByRow( // ValidateAllMultiRegionZoneConfigsInCurrentDatabase is part of the tree.EvalDatabase interface. func (p *planner) ValidateAllMultiRegionZoneConfigsInCurrentDatabase(ctx context.Context) error { - _, dbDesc, err := p.Descriptors().GetImmutableDatabaseByName( + dbDesc, err := p.Descriptors().GetImmutableDatabaseByName( p.EvalContext().Ctx(), p.txn, p.CurrentDatabase(), @@ -905,7 +905,7 @@ func (p *planner) validateAllMultiRegionZoneConfigsInDatabase( func (p *planner) CurrentDatabaseRegionConfig( ctx context.Context, ) (tree.DatabaseRegionConfig, error) { - _, dbDesc, err := p.Descriptors().GetImmutableDatabaseByName( + dbDesc, err := p.Descriptors().GetImmutableDatabaseByName( p.EvalContext().Ctx(), p.txn, p.CurrentDatabase(), @@ -1112,7 +1112,7 @@ func (p *planner) CheckZoneConfigChangePermittedForMultiRegion( // Check if what we're altering is a multi-region entity. if zs.Database != "" { isDB = true - _, dbDesc, err := p.Descriptors().GetImmutableDatabaseByName( + dbDesc, err := p.Descriptors().GetImmutableDatabaseByName( ctx, p.txn, string(zs.Database), diff --git a/pkg/sql/rename_database.go b/pkg/sql/rename_database.go index 8ef58670a4e6..92d599a18492 100644 --- a/pkg/sql/rename_database.go +++ b/pkg/sql/rename_database.go @@ -58,7 +58,7 @@ func (p *planner) RenameDatabase(ctx context.Context, n *tree.RenameDatabase) (p return nil, pgerror.DangerousStatementf("RENAME DATABASE on current database") } - _, dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name), + dbDesc, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name), tree.DatabaseLookupFlags{Required: true}) if err != nil { return nil, err diff --git a/pkg/sql/rename_table.go b/pkg/sql/rename_table.go index 91ebd495fc99..fec7b3cd23f8 100644 --- a/pkg/sql/rename_table.go +++ b/pkg/sql/rename_table.go @@ -114,7 +114,7 @@ func (n *renameTableNode) startExec(params runParams) error { if !newTn.ExplicitSchema && !newTn.ExplicitCatalog { newTn.ObjectNamePrefix = oldTn.ObjectNamePrefix var err error - _, targetDbDesc, err = p.Descriptors().GetImmutableDatabaseByName(ctx, p.txn, + targetDbDesc, err = p.Descriptors().GetImmutableDatabaseByName(ctx, p.txn, string(oldTn.CatalogName), tree.DatabaseLookupFlags{Required: true}) if err != nil { return err diff --git a/pkg/sql/reparent_database.go b/pkg/sql/reparent_database.go index ff803d9af0c9..2cf784f74bb6 100644 --- a/pkg/sql/reparent_database.go +++ b/pkg/sql/reparent_database.go @@ -69,13 +69,13 @@ func (p *planner) ReparentDatabase( sqltelemetry.IncrementUserDefinedSchemaCounter(sqltelemetry.UserDefinedSchemaReparentDatabase) - _, db, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name), + db, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Name), tree.DatabaseLookupFlags{Required: true}) if err != nil { return nil, err } - _, parent, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Parent), + parent, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(n.Parent), tree.DatabaseLookupFlags{Required: true}) if err != nil { return nil, err diff --git a/pkg/sql/resolver.go b/pkg/sql/resolver.go index e32b1261f00b..1458182dc71a 100644 --- a/pkg/sql/resolver.go +++ b/pkg/sql/resolver.go @@ -44,7 +44,7 @@ var _ resolver.SchemaResolver = &planner{} func (p *planner) ResolveUncachedDatabaseByName( ctx context.Context, dbName string, required bool, ) (res catalog.DatabaseDescriptor, err error) { - _, res, err = p.Descriptors().GetImmutableDatabaseByName(ctx, p.txn, dbName, + res, err = p.Descriptors().GetImmutableDatabaseByName(ctx, p.txn, dbName, tree.DatabaseLookupFlags{Required: required, AvoidCached: true}) return res, err } @@ -170,9 +170,9 @@ func (p *planner) ResolveTargetObject( func (p *planner) LookupSchema( ctx context.Context, dbName, scName string, ) (found bool, scMeta tree.SchemaMeta, err error) { - found, dbDesc, err := p.Descriptors().GetImmutableDatabaseByName(ctx, p.txn, dbName, + dbDesc, err := p.Descriptors().GetImmutableDatabaseByName(ctx, p.txn, dbName, tree.DatabaseLookupFlags{AvoidCached: p.avoidCachedDescriptors}) - if err != nil || !found { + if err != nil || dbDesc == nil { return false, nil, err } sc := p.Accessor() @@ -398,7 +398,7 @@ func getDescriptorsFromTargetListForPrivilegeChange( } descs := make([]catalog.Descriptor, 0, len(targets.Databases)) for _, database := range targets.Databases { - _, descriptor, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, + descriptor, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, string(database), tree.DatabaseLookupFlags{Required: true}) if err != nil { return nil, err @@ -448,7 +448,7 @@ func getDescriptorsFromTargetListForPrivilegeChange( if sc.ExplicitCatalog { dbName = sc.Catalog() } - _, db, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, dbName, + db, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, dbName, tree.DatabaseLookupFlags{Required: true}) if err != nil { return nil, err diff --git a/pkg/sql/scrub.go b/pkg/sql/scrub.go index 6209be55b669..850c15f3d8b0 100644 --- a/pkg/sql/scrub.go +++ b/pkg/sql/scrub.go @@ -156,7 +156,7 @@ func (n *scrubNode) Close(ctx context.Context) { func (n *scrubNode) startScrubDatabase(ctx context.Context, p *planner, name *tree.Name) error { // Check that the database exists. database := string(*name) - _, dbDesc, err := p.Descriptors().GetImmutableDatabaseByName(ctx, p.txn, + dbDesc, err := p.Descriptors().GetImmutableDatabaseByName(ctx, p.txn, database, tree.DatabaseLookupFlags{Required: true}) if err != nil { return err diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go index c6b4dce05572..e5c5921b8fe4 100644 --- a/pkg/sql/set_zone_config.go +++ b/pkg/sql/set_zone_config.go @@ -279,7 +279,7 @@ func checkPrivilegeForSetZoneConfig(ctx context.Context, p *planner, zs tree.Zon if zs.Database == "system" { return p.RequireAdminRole(ctx, "alter the system database") } - _, dbDesc, err := p.Descriptors().GetImmutableDatabaseByName(ctx, p.txn, + dbDesc, err := p.Descriptors().GetImmutableDatabaseByName(ctx, p.txn, string(zs.Database), tree.DatabaseLookupFlags{Required: true}) if err != nil { return err diff --git a/pkg/sql/show_zone_config.go b/pkg/sql/show_zone_config.go index b63b44c380aa..98a7bc86625b 100644 --- a/pkg/sql/show_zone_config.go +++ b/pkg/sql/show_zone_config.go @@ -108,7 +108,7 @@ func getShowZoneConfigRow( return nil, err } } else if zoneSpecifier.Database != "" { - _, database, err := p.Descriptors().GetImmutableDatabaseByName( + database, err := p.Descriptors().GetImmutableDatabaseByName( ctx, p.txn, string(zoneSpecifier.Database), diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index 6fe6ea92ad20..779e2935424c 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -225,7 +225,7 @@ var varGen = map[string]sessionVar{ if len(dbName) != 0 { // Verify database descriptor exists. - if _, _, err := evalCtx.Descs.GetImmutableDatabaseByName( + if _, err := evalCtx.Descs.GetImmutableDatabaseByName( ctx, evalCtx.Txn, dbName, tree.DatabaseLookupFlags{Required: true}, ); err != nil { return "", err diff --git a/pkg/sql/virtual_schema.go b/pkg/sql/virtual_schema.go index d2efb9dd3665..df62c3ddb8fd 100644 --- a/pkg/sql/virtual_schema.go +++ b/pkg/sql/virtual_schema.go @@ -482,7 +482,7 @@ func (e *virtualDefEntry) getPlanInfo( var dbDesc catalog.DatabaseDescriptor var err error if dbName != "" { - _, dbDesc, err = p.Descriptors().GetImmutableDatabaseByName(ctx, p.txn, + dbDesc, err = p.Descriptors().GetImmutableDatabaseByName(ctx, p.txn, dbName, tree.DatabaseLookupFlags{ Required: true, AvoidCached: p.avoidCachedDescriptors, }) diff --git a/pkg/sql/virtual_table.go b/pkg/sql/virtual_table.go index 9d701b750d98..1e9d6bcf3165 100644 --- a/pkg/sql/virtual_table.go +++ b/pkg/sql/virtual_table.go @@ -256,7 +256,7 @@ func (v *vTableLookupJoinNode) startExec(params runParams) error { ) v.run.indexKeyDatums = make(tree.Datums, len(v.columns)) var err error - _, db, err := params.p.Descriptors().GetImmutableDatabaseByName( + db, err := params.p.Descriptors().GetImmutableDatabaseByName( params.ctx, params.p.txn, v.dbName, From 55ad7eb40db4953493393bfd5f19359cd4cb6a44 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Tue, 1 Jun 2021 10:44:55 -0400 Subject: [PATCH 3/7] sql/catalog: remove bool found parameter from schema access This should be inferred from the return values. Release note: None --- pkg/sql/alter_schema.go | 5 +-- pkg/sql/catalog/accessor.go | 2 +- pkg/sql/catalog/descs/collection.go | 42 +++++++++++------------- pkg/sql/catalog/descs/collection_test.go | 10 +++--- pkg/sql/create_table.go | 2 +- pkg/sql/drop_database.go | 6 +--- pkg/sql/drop_schema.go | 4 +-- pkg/sql/rename_table.go | 2 +- pkg/sql/resolver.go | 25 +++++++++----- pkg/sql/set_schema.go | 2 +- pkg/sql/sqlerrors/errors.go | 7 ---- 11 files changed, 48 insertions(+), 59 deletions(-) diff --git a/pkg/sql/alter_schema.go b/pkg/sql/alter_schema.go index d4e6bf1d7284..7e3779048064 100644 --- a/pkg/sql/alter_schema.go +++ b/pkg/sql/alter_schema.go @@ -58,13 +58,10 @@ func (p *planner) AlterSchema(ctx context.Context, n *tree.AlterSchema) (planNod if err != nil { return nil, err } - found, schema, err := p.ResolveMutableSchemaDescriptor(ctx, db.ID, string(n.Schema.SchemaName), true /* required */) + schema, err := p.ResolveMutableSchemaDescriptor(ctx, db.ID, string(n.Schema.SchemaName), true /* required */) if err != nil { return nil, err } - if !found { - return nil, pgerror.Newf(pgcode.InvalidSchemaName, "schema %q does not exist", n.Schema.String()) - } switch schema.SchemaKind() { case catalog.SchemaPublic, catalog.SchemaVirtual, catalog.SchemaTemporary: return nil, pgerror.Newf(pgcode.InvalidSchemaName, "cannot modify schema %q", n.Schema.String()) diff --git a/pkg/sql/catalog/accessor.go b/pkg/sql/catalog/accessor.go index 8f976f844696..0daf86bee5d9 100644 --- a/pkg/sql/catalog/accessor.go +++ b/pkg/sql/catalog/accessor.go @@ -44,7 +44,7 @@ type Accessor interface { // exists under the target database. GetSchemaByName( ctx context.Context, txn *kv.Txn, dbID descpb.ID, scName string, flags tree.SchemaLookupFlags, - ) (bool, SchemaDescriptor, error) + ) (SchemaDescriptor, error) // GetObjectNamesAndIDs returns the list of all objects in the given // database and schema. diff --git a/pkg/sql/catalog/descs/collection.go b/pkg/sql/catalog/descs/collection.go index ec9fe99cef64..31d5e3562449 100644 --- a/pkg/sql/catalog/descs/collection.go +++ b/pkg/sql/catalog/descs/collection.go @@ -625,14 +625,14 @@ func (tc *Collection) getObjectByName( dbID := db.GetID() // Resolve the schema. - foundSchema, scDesc, err := tc.GetImmutableSchemaByName(ctx, txn, dbID, schemaName, + scDesc, err := tc.GetImmutableSchemaByName(ctx, txn, dbID, schemaName, tree.SchemaLookupFlags{ Required: flags.Required, AvoidCached: avoidCachedForParent, IncludeDropped: flags.IncludeDropped, IncludeOffline: flags.IncludeOffline, }) - if err != nil || !foundSchema { + if err != nil || scDesc == nil { return false, nil, err } schemaID := scDesc.GetID() @@ -932,7 +932,7 @@ func filterDescriptorState( // mutable descriptor usable by the transaction. RequireMutable is ignored. func (tc *Collection) GetMutableSchemaByName( ctx context.Context, txn *kv.Txn, dbID descpb.ID, schemaName string, flags tree.SchemaLookupFlags, -) (bool, catalog.SchemaDescriptor, error) { +) (catalog.SchemaDescriptor, error) { flags.RequireMutable = true return tc.getSchemaByName(ctx, txn, dbID, schemaName, flags) } @@ -941,7 +941,7 @@ func (tc *Collection) GetMutableSchemaByName( // immutable descriptor usable by the transaction. RequireMutable is ignored. func (tc *Collection) GetImmutableSchemaByName( ctx context.Context, txn *kv.Txn, dbID descpb.ID, schemaName string, flags tree.SchemaLookupFlags, -) (bool, catalog.SchemaDescriptor, error) { +) (catalog.SchemaDescriptor, error) { flags.RequireMutable = false return tc.getSchemaByName(ctx, txn, dbID, schemaName, flags) } @@ -949,24 +949,24 @@ func (tc *Collection) GetImmutableSchemaByName( // GetSchemaByName returns true and a ResolvedSchema object if the target schema // exists under the target database. func (tc *Collection) GetSchemaByName( - ctx context.Context, txn *kv.Txn, dbID descpb.ID, schemaName string, flags tree.SchemaLookupFlags, -) (found bool, _ catalog.SchemaDescriptor, _ error) { - return tc.getSchemaByName(ctx, txn, dbID, schemaName, flags) + ctx context.Context, txn *kv.Txn, dbID descpb.ID, scName string, flags tree.SchemaLookupFlags, +) (catalog.SchemaDescriptor, error) { + return tc.getSchemaByName(ctx, txn, dbID, scName, flags) } // getSchemaByName resolves the schema and, if applicable, returns a descriptor // usable by the transaction. func (tc *Collection) getSchemaByName( ctx context.Context, txn *kv.Txn, dbID descpb.ID, schemaName string, flags tree.SchemaLookupFlags, -) (bool, catalog.SchemaDescriptor, error) { +) (catalog.SchemaDescriptor, error) { // Fast path public schema, as it is always found. if schemaName == tree.PublicSchema { - return true, schemadesc.GetPublicSchema(), nil + return schemadesc.GetPublicSchema(), nil } if tc.virtualSchemas != nil { if sc, ok := tc.virtualSchemas.GetVirtualSchema(schemaName); ok { - return true, sc.Desc(), nil + return sc.Desc(), nil } } @@ -978,7 +978,7 @@ func (tc *Collection) getSchemaByName( schemaName == tc.sessionData.SearchPath.GetTemporarySchemaName() { schemaID, found := tc.sessionData.GetTemporarySchemaIDForDb(uint32(dbID)) if found { - return true, schemadesc.NewTemporarySchema( + return schemadesc.NewTemporarySchema( tc.sessionData.SearchPath.GetTemporarySchemaName(), descpb.ID(schemaID), dbID, @@ -988,14 +988,14 @@ func (tc *Collection) getSchemaByName( } exists, schemaID, err := catalogkv.ResolveSchemaID(ctx, txn, tc.codec(), dbID, schemaName) if err != nil { - return false, nil, err + return nil, err } else if !exists { if flags.Required { - return false, nil, sqlerrors.NewUndefinedSchemaError(schemaName) + return nil, sqlerrors.NewUndefinedSchemaError(schemaName) } - return false, nil, nil + return nil, nil } - return true, schemadesc.NewTemporarySchema( + return schemadesc.NewTemporarySchema( schemaName, schemaID, dbID, @@ -1005,9 +1005,9 @@ func (tc *Collection) getSchemaByName( // Otherwise, the schema is user-defined. Get the descriptor. desc, err := tc.getUserDefinedSchemaByName(ctx, txn, dbID, schemaName, flags) if err != nil || desc == nil { - return false, nil, err + return nil, err } - return true, desc, nil + return desc, nil } // GetMutableDatabaseByID returns a mutable database descriptor with @@ -1969,15 +1969,11 @@ func (tc *Collection) GetObjectNamesAndIDs( IncludeDropped: flags.IncludeDropped, IncludeOffline: flags.IncludeOffline, } - ok, schema, err := tc.GetImmutableSchemaByName(ctx, txn, dbDesc.GetID(), scName, schemaFlags) + schema, err := tc.GetImmutableSchemaByName(ctx, txn, dbDesc.GetID(), scName, schemaFlags) if err != nil { return nil, nil, err } - if !ok { - if flags.Required { - tn := tree.MakeTableNameWithSchema(tree.Name(dbDesc.GetName()), tree.Name(scName), "") - return nil, nil, sqlerrors.NewUnsupportedSchemaUsageError(tree.ErrString(&tn.ObjectNamePrefix)) - } + if schema == nil { // required must have been false return nil, nil, nil } diff --git a/pkg/sql/catalog/descs/collection_test.go b/pkg/sql/catalog/descs/collection_test.go index e71836837df4..6f47a2018c3a 100644 --- a/pkg/sql/catalog/descs/collection_test.go +++ b/pkg/sql/catalog/descs/collection_test.go @@ -296,13 +296,13 @@ func TestAddUncommittedDescriptorAndMutableResolution(t *testing.T) { flags.RequireMutable = true flags.Required = true - ok, schema, err := descriptors.GetMutableSchemaByName(ctx, txn, dbID, "sc", flags) + schema, err := descriptors.GetMutableSchemaByName(ctx, txn, dbID, "sc", flags) require.NoError(t, err) - require.True(t, ok) + require.NotNil(t, schema) - ok, resolved, err := descriptors.GetMutableSchemaByName(ctx, txn, dbID, "sc", flags) + resolved, err := descriptors.GetMutableSchemaByName(ctx, txn, dbID, "sc", flags) require.NoError(t, err) - require.True(t, ok) + require.NotNil(t, schema) require.Same(t, schema, resolved) @@ -492,7 +492,7 @@ CREATE TABLE test.schema.t(x INT); if err != nil { return err } - _, schemaDesc, err := descsCol.GetMutableSchemaByName(ctx, txn, dbDesc.GetID(), "schema", tree.SchemaLookupFlags{Required: true}) + schemaDesc, err := descsCol.GetMutableSchemaByName(ctx, txn, dbDesc.GetID(), "schema", tree.SchemaLookupFlags{Required: true}) if err != nil { return err } diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 924b412f9f6a..8dedb7cb7ab8 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -96,7 +96,7 @@ func (n *createTableNode) ReadingOwnWrites() {} func (p *planner) getSchemaIDForCreate( ctx context.Context, codec keys.SQLCodec, dbID descpb.ID, scName string, ) (descpb.ID, error) { - _, res, err := p.ResolveUncachedSchemaDescriptor(ctx, dbID, scName, true /* required */) + res, err := p.ResolveUncachedSchemaDescriptor(ctx, dbID, scName, true /* required */) if err != nil { return 0, err } diff --git a/pkg/sql/drop_database.go b/pkg/sql/drop_database.go index 57ff09c79baf..19eff898ba89 100644 --- a/pkg/sql/drop_database.go +++ b/pkg/sql/drop_database.go @@ -84,14 +84,10 @@ func (p *planner) DropDatabase(ctx context.Context, n *tree.DropDatabase) (planN d := newDropCascadeState() for _, schema := range schemas { - found, res, err := p.ResolveMutableSchemaDescriptor(ctx, dbDesc.ID, schema, true /* required */) + res, err := p.ResolveMutableSchemaDescriptor(ctx, dbDesc.ID, schema, true /* required */) if err != nil { return nil, err } - if !found { - log.Warningf(ctx, "could not find schema %s under database %d", schema, dbDesc.ID) - continue - } if err := d.collectObjectsInSchema(ctx, p, dbDesc, res); err != nil { return nil, err } diff --git a/pkg/sql/drop_schema.go b/pkg/sql/drop_schema.go index 98b9d121cbd0..d8b11e2df482 100644 --- a/pkg/sql/drop_schema.go +++ b/pkg/sql/drop_schema.go @@ -69,11 +69,11 @@ func (p *planner) DropSchema(ctx context.Context, n *tree.DropSchema) (planNode, return nil, err } - found, sc, err := p.ResolveMutableSchemaDescriptor(ctx, db.ID, scName, false /* required */) + sc, err := p.ResolveMutableSchemaDescriptor(ctx, db.ID, scName, false /* required */) if err != nil { return nil, err } - if !found { + if sc == nil { if n.IfExists { continue } diff --git a/pkg/sql/rename_table.go b/pkg/sql/rename_table.go index fec7b3cd23f8..8e7b01e26125 100644 --- a/pkg/sql/rename_table.go +++ b/pkg/sql/rename_table.go @@ -120,7 +120,7 @@ func (n *renameTableNode) startExec(params runParams) error { return err } - _, targetSchemaDesc, err = p.ResolveUncachedSchemaDescriptor(ctx, targetDbDesc.GetID(), oldTn.Schema(), true) + targetSchemaDesc, err = p.ResolveUncachedSchemaDescriptor(ctx, targetDbDesc.GetID(), oldTn.Schema(), true) if err != nil { return err } diff --git a/pkg/sql/resolver.go b/pkg/sql/resolver.go index 1458182dc71a..d0e6d9ce803b 100644 --- a/pkg/sql/resolver.go +++ b/pkg/sql/resolver.go @@ -52,28 +52,35 @@ func (p *planner) ResolveUncachedDatabaseByName( // ResolveUncachedSchemaDescriptor looks up a schema from the store. func (p *planner) ResolveUncachedSchemaDescriptor( ctx context.Context, dbID descpb.ID, name string, required bool, -) (found bool, schema catalog.SchemaDescriptor, err error) { +) (schema catalog.SchemaDescriptor, err error) { p.runWithOptions(resolveFlags{skipCache: true}, func() { - found, schema, err = p.Accessor().GetSchemaByName( + schema, err = p.Accessor().GetSchemaByName( ctx, p.txn, dbID, name, tree.SchemaLookupFlags{ Required: required, RequireMutable: true, }, ) }) - return found, schema, err + if err != nil || schema == nil { + return nil, err + } + return schema, err } // ResolveUncachedSchemaDescriptor looks up a mutable descriptor for a schema // from the store. func (p *planner) ResolveMutableSchemaDescriptor( ctx context.Context, dbID descpb.ID, name string, required bool, -) (found bool, schema catalog.SchemaDescriptor, err error) { - return p.Accessor().GetSchemaByName( +) (schema catalog.SchemaDescriptor, err error) { + schema, err = p.Accessor().GetSchemaByName( ctx, p.txn, dbID, name, tree.SchemaLookupFlags{ Required: required, RequireMutable: true, }, ) + if err != nil || schema == nil { + return nil, err + } + return schema, nil } // runWithOptions sets the provided resolution flags for the @@ -177,13 +184,13 @@ func (p *planner) LookupSchema( } sc := p.Accessor() var resolvedSchema catalog.SchemaDescriptor - found, resolvedSchema, err = sc.GetSchemaByName( + resolvedSchema, err = sc.GetSchemaByName( ctx, p.txn, dbDesc.GetID(), scName, p.CommonLookupFlags(false /* required */), ) - if err != nil { + if err != nil || resolvedSchema == nil { return false, nil, err } - return found, &catalog.ResolvedObjectPrefix{ + return true, &catalog.ResolvedObjectPrefix{ Database: dbDesc, Schema: resolvedSchema, }, nil @@ -457,7 +464,7 @@ func getDescriptorsFromTargetListForPrivilegeChange( } for _, sc := range targetSchemas { - _, resSchema, err := p.ResolveMutableSchemaDescriptor( + resSchema, err := p.ResolveMutableSchemaDescriptor( ctx, sc.dbDesc.ID, sc.schema, true /* required */) if err != nil { return nil, err diff --git a/pkg/sql/set_schema.go b/pkg/sql/set_schema.go index e4a7be520599..2f091190fc4b 100644 --- a/pkg/sql/set_schema.go +++ b/pkg/sql/set_schema.go @@ -46,7 +46,7 @@ func (p *planner) prepareSetSchema( schemaID := desc.GetParentSchemaID() // Lookup the schema we want to set to. - _, res, err := p.ResolveUncachedSchemaDescriptor(ctx, databaseID, schema, true /* required */) + res, err := p.ResolveUncachedSchemaDescriptor(ctx, databaseID, schema, true /* required */) if err != nil { return 0, err } diff --git a/pkg/sql/sqlerrors/errors.go b/pkg/sql/sqlerrors/errors.go index d7e777f6102e..a6bf5067322f 100644 --- a/pkg/sql/sqlerrors/errors.go +++ b/pkg/sql/sqlerrors/errors.go @@ -61,13 +61,6 @@ func NewUndefinedSchemaError(name string) error { return pgerror.Newf(pgcode.InvalidSchemaName, "unknown schema %q", name) } -// NewUnsupportedSchemaUsageError creates an error for an invalid -// schema use, e.g. mydb.someschema.tbl. -func NewUnsupportedSchemaUsageError(name string) error { - return pgerror.Newf(pgcode.InvalidSchemaName, - "unsupported schema specification: %q", name) -} - // NewCCLRequiredError creates an error for when a CCL feature is used in an OSS // binary. func NewCCLRequiredError(err error) error { From e002229b6fbedb088cea64b78e480ceba1fe0ab5 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Thu, 3 Jun 2021 13:13:19 -0700 Subject: [PATCH 4/7] logcrash: update non-release Sentry URL The documentation for creating Sentry reports in non-release builds directed developers to set the COCKROACH_CRASH_REPORTS env var to an incorrect URL. Sentry reports were not created when using this URL. The documentation has been updated to refer to the correct URL. Release note: None --- pkg/util/log/logcrash/crash_reporting.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/util/log/logcrash/crash_reporting.go b/pkg/util/log/logcrash/crash_reporting.go index b3b0c1f36b14..d1009fff9ce3 100644 --- a/pkg/util/log/logcrash/crash_reporting.go +++ b/pkg/util/log/logcrash/crash_reporting.go @@ -164,7 +164,7 @@ func PanicAsError(depth int, r interface{}) error { // Non-release builds wishing to use Sentry reports // are invited to use the following URL instead: // -// https://ignored@errors.cockroachdb.com/sentrydev/v2/1111 +// https://ignored@errors.cockroachdb.com/api/sentrydev/v2/1111 // // This can be set via e.g. the env var COCKROACH_CRASH_REPORTS. // Note that the special number "1111" is important as it From d8684e47f4a9d366a4f1fcf89ddc5659caa5a65b Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Thu, 3 Jun 2021 22:51:22 -0400 Subject: [PATCH 5/7] sql: fix very slow TestSchemaChangeRetryOnVersionChange See [here](https://teamcity.cockroachdb.com/project.html?projectId=Cockroach_UnitTests&testNameId=6532733632782929224&tab=testDetails). This test was taking 10 minutes every time. This was happening because of the change to wait for the version's leases to expire in #63725. After: ``` --- PASS: TestSchemaChangeRetryOnVersionChange (1.70s) ``` Release note: None --- pkg/sql/schema_changer_test.go | 23 ++++++++++++++--------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 5e02989981bd..3e089fc89bab 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -39,12 +39,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/gcjob" "github.com/cockroachdb/cockroach/pkg/sql/sqltestutils" - "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/sql/tests" "github.com/cockroachdb/cockroach/pkg/sqlmigrations" @@ -1385,23 +1383,30 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT); upTableVersion = func() { leaseMgr := s.LeaseManager().(*lease.Manager) - ie := s.InternalExecutor().(sqlutil.InternalExecutor) var version descpb.DescriptorVersion - if err := descs.Txn(ctx, s.ClusterSettings(), leaseMgr, ie, kvDB, func( - ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, - ) error { - table, err := descsCol.GetMutableTableVersionByID(ctx, id, txn) + if err := kvDB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + table, err := catalogkv.MustGetMutableTableDescByID( + ctx, txn, keys.SystemSQLCodec, tableDesc.GetID()) if err != nil { return err } - return descsCol.WriteDesc(ctx, false /* kvTrace */, table, txn) + table.MaybeIncrementVersion() + ba := txn.NewBatch() + if err := catalogkv.WriteDescToBatch( + ctx, false /* kvTrace */, s.ClusterSettings(), ba, + keys.SystemSQLCodec, table.GetID(), table, + ); err != nil { + return err + } + version = table.GetVersion() + return txn.Run(ctx, ba) }); err != nil { t.Error(err) } // Grab a lease at the latest version so that we are confident // that all future leases will be taken at the latest version. - table, err := leaseMgr.TestingAcquireAndAssertMinVersion(ctx, s.Clock().Now(), id, version+1) + table, err := leaseMgr.TestingAcquireAndAssertMinVersion(ctx, s.Clock().Now(), id, version) if err != nil { t.Error(err) } From d3cbcfd3a51a79fbfa60d0674236346fb4436dea Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Thu, 3 Jun 2021 20:41:17 -0700 Subject: [PATCH 6/7] sql: fix statement diagnostics for EXECUTE Previously, prepared statements ran through the EXECUTE statements could not trigger collection of statement diagnostics. This is because we consider the fingerprint of the EXECUTE statement instead of the one from the prepared statement. The fix is to move up the special handling code in the executor, and replace the AST and fingerprint before setting up the instrumentation helper. Release note (bug fix): queries ran through the EXECUTE statement can now generate statement diagnostic bundles as expected. Fixes #66048. --- pkg/sql/conn_executor_exec.go | 64 ++++++++++--------- .../statement_diagnostics_test.go | 9 +++ 2 files changed, 44 insertions(+), 29 deletions(-) diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 23f20c18db70..b2d20d77cef9 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -358,6 +358,7 @@ func (ex *connExecutor) execStmtInOpenState( p.noticeSender = res ih := &p.instrumentation + // Special top-level handling for EXPLAIN ANALYZE. if e, ok := ast.(*tree.ExplainAnalyze); ok { switch e.Mode { case tree.ExplainDebug: @@ -395,6 +396,40 @@ func (ex *connExecutor) execStmtInOpenState( stmt.ExpectedTypes = nil } + // Special top-level handling for EXECUTE. This must happen after the handling + // for EXPLAIN ANALYZE (in order to support EXPLAIN ANALYZE EXECUTE) but + // before setting up the instrumentation helper. + if e, ok := ast.(*tree.Execute); ok { + // Replace the `EXECUTE foo` statement with the prepared statement, and + // continue execution. + name := e.Name.String() + ps, ok := ex.extraTxnState.prepStmtsNamespace.prepStmts[name] + if !ok { + err := pgerror.Newf( + pgcode.InvalidSQLStatementName, + "prepared statement %q does not exist", name, + ) + return makeErrEvent(err) + } + var err error + pinfo, err = fillInPlaceholders(ctx, ps, name, e.Params, ex.sessionData.SearchPath) + if err != nil { + return makeErrEvent(err) + } + + // TODO(radu): what about .SQL, .NumAnnotations, .NumPlaceholders? + stmt.Statement = ps.Statement + stmt.Prepared = ps + stmt.ExpectedTypes = ps.Columns + stmt.AnonymizedStr = ps.AnonymizedStr + res.ResetStmtType(ps.AST) + + if e.DiscardRows { + ih.SetDiscardRows() + } + ast = stmt.Statement.AST + } + var needFinish bool ctx, needFinish = ih.Setup( ctx, ex.server.cfg, ex.appStats, p, ex.stmtDiagnosticsRecorder, @@ -531,35 +566,6 @@ func (ex *connExecutor) execStmtInOpenState( return makeErrEvent(err) } return nil, nil, nil - - case *tree.Execute: - // Replace the `EXECUTE foo` statement with the prepared statement, and - // continue execution below. - name := s.Name.String() - ps, ok := ex.extraTxnState.prepStmtsNamespace.prepStmts[name] - if !ok { - err := pgerror.Newf( - pgcode.InvalidSQLStatementName, - "prepared statement %q does not exist", name, - ) - return makeErrEvent(err) - } - var err error - pinfo, err = fillInPlaceholders(ctx, ps, name, s.Params, ex.sessionData.SearchPath) - if err != nil { - return makeErrEvent(err) - } - - stmt.Statement = ps.Statement - ast = stmt.AST - stmt.Prepared = ps - stmt.ExpectedTypes = ps.Columns - stmt.AnonymizedStr = ps.AnonymizedStr - res.ResetStmtType(ps.AST) - - if s.DiscardRows { - ih.SetDiscardRows() - } } p.semaCtx.Annotations = tree.MakeAnnotations(stmt.NumAnnotations) diff --git a/pkg/sql/stmtdiagnostics/statement_diagnostics_test.go b/pkg/sql/stmtdiagnostics/statement_diagnostics_test.go index 381ae220597a..65a865e78566 100644 --- a/pkg/sql/stmtdiagnostics/statement_diagnostics_test.go +++ b/pkg/sql/stmtdiagnostics/statement_diagnostics_test.go @@ -88,6 +88,15 @@ func TestDiagnosticsRequest(t *testing.T) { _, err = db.Exec("INSERT INTO test VALUES (2)") require.NoError(t, err) checkCompleted(id1) + + // Verify that EXECUTE triggers diagnostics collection (#66048). + id4, err := registry.InsertRequestInternal(ctx, "SELECT x + $1 FROM test") + require.NoError(t, err) + _, err = db.Exec("PREPARE stmt AS SELECT x + $1 FROM test") + require.NoError(t, err) + _, err = db.Exec("EXECUTE stmt(1)") + require.NoError(t, err) + checkCompleted(id4) } // Test that a different node can service a diagnostics request. From a6ecdea15ba39f85376b62c146fbebcc8002d75d Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Wed, 26 May 2021 15:23:14 -0400 Subject: [PATCH 7/7] sql: add unit tests for planning inside the new schema changer Previously, the new schema changer did not have any unit tests covering the planning capability. This was inadequate, because we had no way of detect if plans were regressing with changes or enhancements. To address this, this patch adds basic tests to see if the operators/dependencies for a given command are sane. Release note: None --- pkg/BUILD.bazel | 1 + pkg/sql/schemachanger/scgraphviz/graphviz.go | 8 +- pkg/sql/schemachanger/scpb/attribute.go | 10 +- pkg/sql/schemachanger/scpb/attribute_test.go | 6 +- pkg/sql/schemachanger/scpb/elements.go | 8 + pkg/sql/schemachanger/scplan/BUILD.bazel | 40 +- pkg/sql/schemachanger/scplan/main_test.go | 31 + pkg/sql/schemachanger/scplan/plan.go | 23 +- pkg/sql/schemachanger/scplan/plan_test.go | 209 +++++++ .../schemachanger/scplan/testdata/alter_table | 559 ++++++++++++++++++ .../scplan/testdata/drop_sequence | 58 ++ .../schemachanger/scplan/testdata/drop_view | 103 ++++ 12 files changed, 1037 insertions(+), 19 deletions(-) create mode 100644 pkg/sql/schemachanger/scplan/main_test.go create mode 100644 pkg/sql/schemachanger/scplan/plan_test.go create mode 100644 pkg/sql/schemachanger/scplan/testdata/alter_table create mode 100644 pkg/sql/schemachanger/scplan/testdata/drop_sequence create mode 100644 pkg/sql/schemachanger/scplan/testdata/drop_view diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 1b54f651aca2..9f8ef55f0ed6 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -254,6 +254,7 @@ ALL_TESTS = [ "//pkg/sql/schemachanger/scbuild:scbuild_test", "//pkg/sql/schemachanger/scexec:scexec_test", "//pkg/sql/schemachanger/scpb:scpb_test", + "//pkg/sql/schemachanger/scplan:scplan_test", "//pkg/sql/schemachanger:schemachanger_test", "//pkg/sql/sem/builtins:builtins_test", "//pkg/sql/sem/tree/eval_test:eval_test_test", diff --git a/pkg/sql/schemachanger/scgraphviz/graphviz.go b/pkg/sql/schemachanger/scgraphviz/graphviz.go index d8cf3707d00a..2f28ad8a9cb3 100644 --- a/pkg/sql/schemachanger/scgraphviz/graphviz.go +++ b/pkg/sql/schemachanger/scgraphviz/graphviz.go @@ -161,11 +161,11 @@ func htmlLabel(o interface{}) dot.HTML { return dot.HTML(buf.String()) } -// toMap converts a struct to a map, field by field. If at any point a protobuf +// ToMap converts a struct to a map, field by field. If at any point a protobuf // message is encountered, it is converted to a map using jsonpb to marshal it // to json and then marshaling it back to a map. This approach allows zero // values to be effectively omitted. -func toMap(v interface{}) (interface{}, error) { +func ToMap(v interface{}) (interface{}, error) { if v == nil { return nil, nil } @@ -202,7 +202,7 @@ func toMap(v interface{}) (interface{}, error) { continue } var err error - if m[vt.Field(i).Name], err = toMap(vvf.Interface()); err != nil { + if m[vt.Field(i).Name], err = ToMap(vvf.Interface()); err != nil { return nil, err } } @@ -235,7 +235,7 @@ var objectTemplate = template.Must(template.New("obj").Funcs(template.FuncMap{ "isStruct": func(v interface{}) bool { return reflect.Indirect(reflect.ValueOf(v)).Kind() == reflect.Struct }, - "toMap": toMap, + "toMap": ToMap, }).Parse(` {{- define "key" -}} diff --git a/pkg/sql/schemachanger/scpb/attribute.go b/pkg/sql/schemachanger/scpb/attribute.go index fc07161310b0..4e0b173e6ad1 100644 --- a/pkg/sql/schemachanger/scpb/attribute.go +++ b/pkg/sql/schemachanger/scpb/attribute.go @@ -61,7 +61,7 @@ func (id IndexID) String() string { // String implements AttributeValue. func (id ElementTypeID) String() string { - return strconv.Itoa(int(id)) + return ElementIDToString(id) } // String implements AttributeValue. @@ -146,8 +146,12 @@ func (a Attributes) Equal(other Attributes) bool { // String seralizes attribute into a string func (a Attributes) String() string { result := strings.Builder{} - result.WriteString("[ ") + result.WriteString(a.Get(AttributeType).String()) + result.WriteString(":{") for attribIdx, attrib := range a.values { + if attrib.key == AttributeType { + continue + } result.WriteString(attrib.key.String()) result.WriteString(": ") result.WriteString(attrib.value.String()) @@ -155,7 +159,7 @@ func (a Attributes) String() string { result.WriteString(", ") } } - result.WriteString(" ]") + result.WriteString("}") return result.String() } diff --git a/pkg/sql/schemachanger/scpb/attribute_test.go b/pkg/sql/schemachanger/scpb/attribute_test.go index 241fb2aee16f..0f7c56bb45e4 100644 --- a/pkg/sql/schemachanger/scpb/attribute_test.go +++ b/pkg/sql/schemachanger/scpb/attribute_test.go @@ -30,7 +30,7 @@ func TestGetAttribute(t *testing.T) { // Sanity: Validate basic string conversion, equality, // and inequality. - expectedStr := `[ Type: 4, DescID: 3, DepID: 1, ColumnID: 2 ]` + expectedStr := `SequenceDependency:{DescID: 3, DepID: 1, ColumnID: 2}` require.Equal(t, expectedStr, seqElem.GetAttributes().String(), "Attribute string conversion is broken.") require.True(t, seqElem.GetAttributes().Equal(seqElem.GetAttributes())) require.False(t, seqElem.GetAttributes().Equal(seqElemDiff.GetAttributes())) @@ -38,7 +38,7 @@ func TestGetAttribute(t *testing.T) { // Sanity: Validate type references, then check if type comparisons // work. typeBackRef := TypeReference{TypeID: 3, DescID: 1} - expectedStr = `[ Type: 10, DescID: 3, DepID: 1 ]` + expectedStr = `TypeReference:{DescID: 3, DepID: 1}` require.Equal(t, expectedStr, typeBackRef.GetAttributes().String(), "Attribute string conversion is broken.") require.False(t, seqElem.GetAttributes().Equal(typeBackRef.GetAttributes())) require.False(t, typeBackRef.GetAttributes().Equal(seqElem.GetAttributes())) @@ -46,6 +46,6 @@ func TestGetAttribute(t *testing.T) { // Sanity: Validate attribute fetching for both types. require.Equal(t, "1", typeBackRef.GetAttributes().Get(AttributeDepID).String()) require.Equal(t, "3", typeBackRef.GetAttributes().Get(AttributeDescID).String()) - require.Equal(t, "10", typeBackRef.GetAttributes().Get(AttributeType).String()) + require.Equal(t, "TypeReference", typeBackRef.GetAttributes().Get(AttributeType).String()) require.Equal(t, "4", seqElemDiff.GetAttributes().Get(AttributeColumnID).String()) } diff --git a/pkg/sql/schemachanger/scpb/elements.go b/pkg/sql/schemachanger/scpb/elements.go index 61b8dc01a7d3..c3357aa28257 100644 --- a/pkg/sql/schemachanger/scpb/elements.go +++ b/pkg/sql/schemachanger/scpb/elements.go @@ -63,10 +63,12 @@ func NewTarget(dir Target_Direction, elem Element) *Target { type ElementTypeID int var typeToElementID map[reflect.Type]ElementTypeID +var elementIDToString map[ElementTypeID]string func init() { typ := reflect.TypeOf((*ElementProto)(nil)).Elem() typeToElementID = make(map[reflect.Type]ElementTypeID, typ.NumField()) + elementIDToString = make(map[ElementTypeID]string, typ.NumField()) for i := 0; i < typ.NumField(); i++ { f := typ.Field(i) protoFlags := strings.Split(f.Tag.Get("protobuf"), ",") @@ -75,6 +77,7 @@ func init() { panic(errors.Wrapf(err, "failed to extract ID from protobuf tag: %q", protoFlags)) } typeToElementID[f.Type] = ElementTypeID(id) + elementIDToString[ElementTypeID(id)] = strings.TrimPrefix(f.Type.String(), "*scpb.") } } @@ -83,6 +86,11 @@ func ElementType(el Element) ElementTypeID { return typeToElementID[reflect.TypeOf(el)] } +// ElementIDToString determines the type ID of a element +func ElementIDToString(id ElementTypeID) string { + return elementIDToString[id] +} + // DescriptorID implements the Element interface. func (e *Column) DescriptorID() descpb.ID { return e.TableID } diff --git a/pkg/sql/schemachanger/scplan/BUILD.bazel b/pkg/sql/schemachanger/scplan/BUILD.bazel index 1a87ff38dd43..5c7f1934eb69 100644 --- a/pkg/sql/schemachanger/scplan/BUILD.bazel +++ b/pkg/sql/schemachanger/scplan/BUILD.bazel @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "scplan", @@ -21,3 +21,41 @@ go_library( "@com_github_cockroachdb_errors//:errors", ], ) + +go_test( + name = "scplan_test", + srcs = [ + "main_test.go", + "plan_test.go", + ], + data = glob(["testdata/**"]), + deps = [ + ":scplan", + "//pkg/base", + "//pkg/kv", + "//pkg/security", + "//pkg/security/securitytest", + "//pkg/server", + "//pkg/sql", + "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/descs", + "//pkg/sql/catalog/resolver", + "//pkg/sql/parser", + "//pkg/sql/schemachanger/scbuild", + "//pkg/sql/schemachanger/scgraph", + "//pkg/sql/schemachanger/scgraphviz", + "//pkg/sql/schemachanger/scpb", + "//pkg/sql/sem/tree", + "//pkg/sql/sessiondatapb", + "//pkg/testutils/serverutils", + "//pkg/testutils/sqlutils", + "//pkg/testutils/testcluster", + "//pkg/util/leaktest", + "//pkg/util/log", + "//pkg/util/randutil", + "@com_github_cockroachdb_datadriven//:datadriven", + "@com_github_cockroachdb_errors//:errors", + "@com_github_stretchr_testify//require", + "@in_gopkg_yaml_v2//:yaml_v2", + ], +) diff --git a/pkg/sql/schemachanger/scplan/main_test.go b/pkg/sql/schemachanger/scplan/main_test.go new file mode 100644 index 000000000000..0fcc82ddb345 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/main_test.go @@ -0,0 +1,31 @@ +// 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 scplan_test + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +func TestMain(m *testing.M) { + security.SetAssetLoader(securitytest.EmbeddedAssets) + randutil.SeedForTests() + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} diff --git a/pkg/sql/schemachanger/scplan/plan.go b/pkg/sql/schemachanger/scplan/plan.go index 2b4814478c83..655ce82c9633 100644 --- a/pkg/sql/schemachanger/scplan/plan.go +++ b/pkg/sql/schemachanger/scplan/plan.go @@ -56,6 +56,9 @@ type Params struct { // // This doesn't do anything right now. CreatedDescriptorIDs catalog.DescriptorIDSet + // DisableOpRandomization disables randomization for the final set of + // operations. + DisableOpRandomization bool } // A Plan is a schema change plan, primarily containing ops to be executed that @@ -110,7 +113,7 @@ func MakePlan(initialStates []*scpb.Node, params Params) (_ Plan, err error) { }); err != nil { return Plan{}, err } - stages := buildStages(initialStates, g) + stages := buildStages(initialStates, g, params) return Plan{ Params: params, InitialNodes: initialStates, @@ -119,7 +122,7 @@ func MakePlan(initialStates []*scpb.Node, params Params) (_ Plan, err error) { }, nil } -func buildStages(init []*scpb.Node, g *scgraph.Graph) []Stage { +func buildStages(init []*scpb.Node, g *scgraph.Graph, params Params) []Stage { // TODO(ajwerner): deal with the case where the target state was // fulfilled by something that preceded the initial state. cur := init @@ -236,12 +239,15 @@ func buildStages(init []*scpb.Node, g *scgraph.Graph) []Stage { // be order independent, however we will // try to execute non-failing ones first. opsSlice := s.Ops.Slice() - rand.Seed(timeutil.Now().UnixNano()) - rand.Shuffle(len(opsSlice), func(i, j int) { - tmp := opsSlice[i] - opsSlice[i] = opsSlice[j] - opsSlice[j] = tmp - }) + if !params.DisableOpRandomization { + + rand.Seed(timeutil.Now().UnixNano()) + rand.Shuffle(len(opsSlice), func(i, j int) { + tmp := opsSlice[i] + opsSlice[i] = opsSlice[j] + opsSlice[j] = tmp + }) + } // Place non-revertible operations at the end sort.SliceStable(opsSlice, func(i, j int) bool { if opsSlice[i].Revertible() == opsSlice[j].Revertible() { @@ -251,6 +257,7 @@ func buildStages(init []*scpb.Node, g *scgraph.Graph) []Stage { }) stages = append(stages, s) cur = s.After + } return stages } diff --git a/pkg/sql/schemachanger/scplan/plan_test.go b/pkg/sql/schemachanger/scplan/plan_test.go new file mode 100644 index 000000000000..8461beb2ec53 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/plan_test.go @@ -0,0 +1,209 @@ +// 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 scplan_test + +import ( + "context" + "fmt" + "path/filepath" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" + "github.com/cockroachdb/cockroach/pkg/sql/parser" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraph" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraphviz" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/datadriven" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" + "gopkg.in/yaml.v2" +) + +func TestPlanAlterTable(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + + datadriven.Walk(t, filepath.Join("testdata"), func(t *testing.T, path string) { + s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(ctx) + + tdb := sqlutils.MakeSQLRunner(sqlDB) + run := func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "create-view", "create-sequence", "create-table", "create-type": + stmts, err := parser.Parse(d.Input) + require.NoError(t, err) + require.Len(t, stmts, 1) + tableName := "" + switch node := stmts[0].AST.(type) { + case *tree.CreateTable: + tableName = node.Table.String() + case *tree.CreateSequence: + tableName = node.Name.String() + case *tree.CreateView: + tableName = node.Name.String() + case *tree.CreateType: + tableName = "" + default: + t.Fatal("not a CREATE TABLE/SEQUENCE/VIEW statement") + } + tdb.Exec(t, d.Input) + + if len(tableName) > 0 { + var tableID descpb.ID + tdb.QueryRow(t, `SELECT $1::regclass::int`, tableName).Scan(&tableID) + if tableID == 0 { + t.Fatalf("failed to read ID of new table %s", tableName) + } + t.Logf("created relation with id %d", tableID) + } + return "" + case "ops", "deps": + deps, cleanup := newTestingPlanDeps(s) + defer cleanup() + + stmts, err := parser.Parse(d.Input) + require.NoError(t, err) + var outputNodes []*scpb.Node + for i := range stmts { + outputNodes, err = scbuild.Build(ctx, stmts[i].AST, *deps, outputNodes) + require.NoError(t, err) + } + + plan, err := scplan.MakePlan(outputNodes, + scplan.Params{ + ExecutionPhase: scplan.PostCommitPhase, + DisableOpRandomization: true, + }) + require.NoError(t, err) + + if d.Cmd == "ops" { + return marshalOps(t, &plan) + } + return marshalDeps(t, &plan) + case "unimplemented": + deps, cleanup := newTestingPlanDeps(s) + defer cleanup() + + stmts, err := parser.Parse(d.Input) + require.NoError(t, err) + require.Len(t, stmts, 1) + + stmt := stmts[0] + alter, ok := stmt.AST.(*tree.AlterTable) + require.Truef(t, ok, "not an ALTER TABLE statement: %s", stmt.SQL) + _, err = scbuild.Build(ctx, alter, *deps, nil) + require.Truef(t, scbuild.HasNotImplemented(err), "expected unimplemented, got %v", err) + return "" + + default: + return fmt.Sprintf("unknown command: %s", d.Cmd) + } + } + datadriven.RunTest(t, path, run) + }) +} + +// indentText indents text for formatting out marshaled data. +func indentText(input string, tab string) (final string) { + split := strings.Split(input, "\n") + for idx, line := range split { + if len(line) == 0 { + continue + } + final += tab + line + if idx != len(split)-1 { + final = final + "\n" + } + } + return final +} + +// marshalDeps marshals dependencies in scplan.Plan to a string. +func marshalDeps(t *testing.T, plan *scplan.Plan) string { + var stages strings.Builder + err := plan.Graph.ForEachNode(func(n *scpb.Node) error { + return plan.Graph.ForEachDepEdgeFrom(n, func(de *scgraph.DepEdge) error { + toAttr := de.To().Element().GetAttributes() + fromAttr := de.From().Element().GetAttributes() + fmt.Fprintf(&stages, "- from: [%s, %s]\n", fromAttr, de.From().State) + fmt.Fprintf(&stages, " to: [%s, %s]\n", toAttr, de.To().State) + return nil + }) + }) + if err != nil { + panic(errors.Wrap(err, "failed marshaling dependencies.")) + } + return stages.String() +} + +// marshalOps marshals operations in scplan.Plan to a string. +func marshalOps(t *testing.T, plan *scplan.Plan) string { + stages := "" + for stageIdx, stage := range plan.Stages { + stages += fmt.Sprintf("Stage %d\n", stageIdx) + stageOps := "" + for _, op := range stage.Ops.Slice() { + opMap, err := scgraphviz.ToMap(op) + require.NoError(t, err) + data, err := yaml.Marshal(opMap) + require.NoError(t, err) + stageOps += fmt.Sprintf("%T\n%s", op, indentText(string(data), " ")) + } + stages += indentText(stageOps, " ") + } + return stages +} + +func newTestingPlanDeps(s serverutils.TestServerInterface) (*scbuild.Dependencies, func()) { + execCfg := s.ExecutorConfig().(sql.ExecutorConfig) + ip, cleanup := sql.NewInternalPlanner( + "test", + kv.NewTxn(context.Background(), s.DB(), s.NodeID()), + security.RootUserName(), + &sql.MemoryMetrics{}, + &execCfg, + // Setting the database on the session data to "defaultdb" in the obvious + // way doesn't seem to do what we want. + sessiondatapb.SessionData{}, + ) + planner := ip.(interface { + resolver.SchemaResolver + SemaCtx() *tree.SemaContext + EvalContext() *tree.EvalContext + Descriptors() *descs.Collection + scbuild.AuthorizationAccessor + }) + buildDeps := scbuild.Dependencies{ + Res: planner, + SemaCtx: planner.SemaCtx(), + EvalCtx: planner.EvalContext(), + Descs: planner.Descriptors(), + AuthAccessor: planner, + } + return &buildDeps, cleanup +} diff --git a/pkg/sql/schemachanger/scplan/testdata/alter_table b/pkg/sql/schemachanger/scplan/testdata/alter_table new file mode 100644 index 000000000000..1f4cb2cce9f5 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/testdata/alter_table @@ -0,0 +1,559 @@ +create-table +CREATE TABLE defaultdb.foo (i INT PRIMARY KEY) +---- + +ops +ALTER TABLE defaultdb.foo ADD COLUMN j INT +---- +Stage 0 + *scop.MakeAddedColumnDeleteOnly + Column: + ID: 2 + Name: j + Nullable: true + Type: + family: IntFamily + oid: 20 + width: 64 + FamilyName: primary + TableID: 52 + *scop.MakeAddedIndexDeleteOnly + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + StoreColumnNames: + - j + Unique: true + Version: 3 + TableID: 52 +Stage 1 + *scop.MakeAddedColumnDeleteAndWriteOnly + ColumnID: 2 + TableID: 52 + *scop.MakeAddedIndexDeleteAndWriteOnly + IndexID: 2 + TableID: 52 +Stage 2 + scop.BackfillIndex + IndexID: 2 + TableID: 52 +Stage 3 + scop.ValidateUniqueIndex + IndexID: 2 + PrimaryIndexID: 1 + TableID: 52 +Stage 4 + *scop.MakeColumnPublic + ColumnID: 2 + TableID: 52 + *scop.MakeAddedPrimaryIndexPublic + Index: + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + Unique: true + Version: 3 + TableID: 52 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + Index: + EncodingType: 1 + ID: 1 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: primary + Unique: true + Version: 3 + TableID: 52 +Stage 5 + *scop.MakeDroppedIndexDeleteOnly + IndexID: 1 + TableID: 52 +Stage 6 + *scop.MakeIndexAbsent + IndexID: 1 + TableID: 52 + +ops +ALTER TABLE defaultdb.foo ADD COLUMN j INT DEFAULT 123 +---- +Stage 0 + *scop.MakeAddedColumnDeleteOnly + Column: + DefaultExpr: 123:::INT8 + ID: 2 + Name: j + Nullable: true + Type: + family: IntFamily + oid: 20 + width: 64 + FamilyName: primary + TableID: 52 + *scop.MakeAddedIndexDeleteOnly + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + StoreColumnNames: + - j + Unique: true + Version: 3 + TableID: 52 +Stage 1 + *scop.MakeAddedColumnDeleteAndWriteOnly + ColumnID: 2 + TableID: 52 + *scop.MakeAddedIndexDeleteAndWriteOnly + IndexID: 2 + TableID: 52 +Stage 2 + scop.BackfillIndex + IndexID: 2 + TableID: 52 +Stage 3 + scop.ValidateUniqueIndex + IndexID: 2 + PrimaryIndexID: 1 + TableID: 52 +Stage 4 + *scop.MakeColumnPublic + ColumnID: 2 + TableID: 52 + *scop.MakeAddedPrimaryIndexPublic + Index: + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + Unique: true + Version: 3 + TableID: 52 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + Index: + EncodingType: 1 + ID: 1 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: primary + Unique: true + Version: 3 + TableID: 52 +Stage 5 + *scop.MakeDroppedIndexDeleteOnly + IndexID: 1 + TableID: 52 +Stage 6 + *scop.MakeIndexAbsent + IndexID: 1 + TableID: 52 + +ops +ALTER TABLE defaultdb.foo ADD COLUMN j INT DEFAULT 123; +ALTER TABLE defaultdb.foo ADD COLUMN k INT DEFAULT 456; +---- +Stage 0 + *scop.MakeAddedColumnDeleteOnly + Column: + DefaultExpr: 123:::INT8 + ID: 2 + Name: j + Nullable: true + Type: + family: IntFamily + oid: 20 + width: 64 + FamilyName: primary + TableID: 52 + *scop.MakeAddedIndexDeleteOnly + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + - 3 + StoreColumnNames: + - j + - k + Unique: true + Version: 3 + TableID: 52 + *scop.MakeAddedColumnDeleteOnly + Column: + DefaultExpr: 456:::INT8 + ID: 3 + Name: k + Nullable: true + Type: + family: IntFamily + oid: 20 + width: 64 + FamilyName: primary + TableID: 52 +Stage 1 + *scop.MakeAddedColumnDeleteAndWriteOnly + ColumnID: 2 + TableID: 52 + *scop.MakeAddedIndexDeleteAndWriteOnly + IndexID: 2 + TableID: 52 + *scop.MakeAddedColumnDeleteAndWriteOnly + ColumnID: 3 + TableID: 52 +Stage 2 + scop.BackfillIndex + IndexID: 2 + TableID: 52 +Stage 3 + scop.ValidateUniqueIndex + IndexID: 2 + PrimaryIndexID: 1 + TableID: 52 +Stage 4 + *scop.MakeColumnPublic + ColumnID: 2 + TableID: 52 + *scop.MakeAddedPrimaryIndexPublic + Index: + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + Unique: true + Version: 3 + TableID: 52 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + Index: + EncodingType: 1 + ID: 1 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: primary + Unique: true + Version: 3 + TableID: 52 + *scop.MakeColumnPublic + ColumnID: 3 + TableID: 52 +Stage 5 + *scop.MakeDroppedIndexDeleteOnly + IndexID: 1 + TableID: 52 +Stage 6 + *scop.MakeIndexAbsent + IndexID: 1 + TableID: 52 + +ops +ALTER TABLE defaultdb.foo ADD COLUMN a INT AS (i+1) STORED +---- +Stage 0 + *scop.MakeAddedColumnDeleteOnly + Column: + ComputeExpr: i + 1:::INT8 + ID: 2 + Name: a + Nullable: true + Type: + family: IntFamily + oid: 20 + width: 64 + FamilyName: primary + TableID: 52 + *scop.MakeAddedIndexDeleteOnly + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + StoreColumnNames: + - a + Unique: true + Version: 3 + TableID: 52 +Stage 1 + *scop.MakeAddedColumnDeleteAndWriteOnly + ColumnID: 2 + TableID: 52 + *scop.MakeAddedIndexDeleteAndWriteOnly + IndexID: 2 + TableID: 52 +Stage 2 + scop.BackfillIndex + IndexID: 2 + TableID: 52 +Stage 3 + scop.ValidateUniqueIndex + IndexID: 2 + PrimaryIndexID: 1 + TableID: 52 +Stage 4 + *scop.MakeColumnPublic + ColumnID: 2 + TableID: 52 + *scop.MakeAddedPrimaryIndexPublic + Index: + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + Unique: true + Version: 3 + TableID: 52 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + Index: + EncodingType: 1 + ID: 1 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: primary + Unique: true + Version: 3 + TableID: 52 +Stage 5 + *scop.MakeDroppedIndexDeleteOnly + IndexID: 1 + TableID: 52 +Stage 6 + *scop.MakeIndexAbsent + IndexID: 1 + TableID: 52 + + +create-table +CREATE TABLE defaultdb.bar (j INT); +---- + +ops +ALTER TABLE defaultdb.foo ADD COLUMN a INT; +ALTER TABLE defaultdb.bar ADD COLUMN b INT; +---- +Stage 0 + *scop.MakeAddedColumnDeleteOnly + Column: + ID: 2 + Name: a + Nullable: true + Type: + family: IntFamily + oid: 20 + width: 64 + FamilyName: primary + TableID: 52 + *scop.MakeAddedIndexDeleteOnly + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + StoreColumnNames: + - a + Unique: true + Version: 3 + TableID: 52 + *scop.MakeAddedColumnDeleteOnly + Column: + ID: 3 + Name: b + Nullable: true + Type: + family: IntFamily + oid: 20 + width: 64 + FamilyName: primary + TableID: 53 + *scop.MakeAddedIndexDeleteOnly + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 2 + KeyColumnNames: + - rowid + Name: new_primary_key + StoreColumnIDs: + - 1 + - 3 + StoreColumnNames: + - j + - b + Unique: true + TableID: 53 +Stage 1 + *scop.MakeAddedColumnDeleteAndWriteOnly + ColumnID: 2 + TableID: 52 + *scop.MakeAddedIndexDeleteAndWriteOnly + IndexID: 2 + TableID: 52 + *scop.MakeAddedColumnDeleteAndWriteOnly + ColumnID: 3 + TableID: 53 + *scop.MakeAddedIndexDeleteAndWriteOnly + IndexID: 2 + TableID: 53 +Stage 2 + scop.BackfillIndex + IndexID: 2 + TableID: 52 + scop.BackfillIndex + IndexID: 2 + TableID: 53 +Stage 3 + scop.ValidateUniqueIndex + IndexID: 2 + PrimaryIndexID: 1 + TableID: 52 + scop.ValidateUniqueIndex + IndexID: 2 + PrimaryIndexID: 1 + TableID: 53 +Stage 4 + *scop.MakeColumnPublic + ColumnID: 2 + TableID: 52 + *scop.MakeAddedPrimaryIndexPublic + Index: + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + Unique: true + Version: 3 + TableID: 52 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + Index: + EncodingType: 1 + ID: 1 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: primary + Unique: true + Version: 3 + TableID: 52 + *scop.MakeColumnPublic + ColumnID: 3 + TableID: 53 + *scop.MakeAddedPrimaryIndexPublic + Index: + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 2 + KeyColumnNames: + - rowid + Name: new_primary_key + Unique: true + TableID: 53 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + Index: + EncodingType: 1 + ID: 1 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 2 + KeyColumnNames: + - rowid + Name: primary + StoreColumnIDs: + - 1 + StoreColumnNames: + - j + Unique: true + TableID: 53 +Stage 5 + *scop.MakeDroppedIndexDeleteOnly + IndexID: 1 + TableID: 52 + *scop.MakeDroppedIndexDeleteOnly + IndexID: 1 + TableID: 53 +Stage 6 + *scop.MakeIndexAbsent + IndexID: 1 + TableID: 52 + *scop.MakeIndexAbsent + IndexID: 1 + TableID: 53 diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_sequence b/pkg/sql/schemachanger/scplan/testdata/drop_sequence new file mode 100644 index 000000000000..1a74f8e7b294 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/testdata/drop_sequence @@ -0,0 +1,58 @@ +create-sequence +CREATE SEQUENCE defaultdb.SQ1 +---- + +ops +DROP SEQUENCE defaultdb.SQ1 CASCADE +---- +Stage 0 + *scop.CreateGcJobForDescriptor + DescID: 52 + *scop.MarkDescriptorAsDropped + TableID: 52 +Stage 1 + *scop.CreateGcJobForDescriptor + DescID: 52 + *scop.DrainDescriptorName + TableID: 52 + +create-table +CREATE TABLE defaultdb.blog_posts (id INT PRIMARY KEY, val int DEFAULT nextval('defaultdb.sq1'), title text) +---- + +create-table +CREATE TABLE defaultdb.blog_posts2 (id INT PRIMARY KEY, val int DEFAULT nextval('defaultdb.sq1'), title text) +---- + +ops +DROP SEQUENCE defaultdb.SQ1 CASCADE +---- +Stage 0 + *scop.RemoveColumnDefaultExpression + ColumnID: 2 + TableID: 53 + *scop.UpdateRelationDeps + TableID: 53 + *scop.RemoveColumnDefaultExpression + ColumnID: 2 + TableID: 54 + *scop.UpdateRelationDeps + TableID: 54 + *scop.CreateGcJobForDescriptor + DescID: 52 + *scop.MarkDescriptorAsDropped + TableID: 52 +Stage 1 + *scop.CreateGcJobForDescriptor + DescID: 52 + *scop.DrainDescriptorName + TableID: 52 + + +deps +DROP SEQUENCE defaultdb.SQ1 CASCADE +---- +- from: [Sequence:{DescID: 52}, DELETE_ONLY] + to: [DefaultExpression:{DescID: 53, ColumnID: 2}, ABSENT] +- from: [Sequence:{DescID: 52}, DELETE_ONLY] + to: [DefaultExpression:{DescID: 54, ColumnID: 2}, ABSENT] diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_view b/pkg/sql/schemachanger/scplan/testdata/drop_view new file mode 100644 index 000000000000..1e851cd64559 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/testdata/drop_view @@ -0,0 +1,103 @@ +create-table +CREATE TABLE defaultdb.t1 (id INT PRIMARY KEY, name varchar(256)) +---- + +create-view +CREATE VIEW defaultdb.v1 AS (SELECT name FROM defaultdb.t1) +---- + +ops +DROP VIEW defaultdb.v1 +---- +Stage 0 + *scop.MarkDescriptorAsDropped + TableID: 53 +Stage 1 + *scop.CreateGcJobForDescriptor + DescID: 53 + *scop.DrainDescriptorName + TableID: 53 + +deps +DROP VIEW defaultdb.v1 +---- + +create-view +CREATE VIEW defaultdb.v2 AS (SELECT name AS n1, name AS n2 FROM v1) +---- + +create-view +CREATE VIEW defaultdb.v3 AS (SELECT name, n1 FROM v1, v2); +---- + +create-view +CREATE VIEW defaultdb.v4 AS (SELECT n2, n1 FROM v2); +---- + +create-type +CREATE TYPE defaultdb.typ AS ENUM('a') +---- + +create-view +CREATE VIEW v5 AS (SELECT 'a'::defaultdb.typ::string AS k, n2, n1 from defaultdb.v4) +---- + +ops +DROP VIEW defaultdb.v1 CASCADE +---- +Stage 0 + *scop.RemoveTypeBackRef + DescID: 59 + TypeID: 57 + *scop.RemoveTypeBackRef + DescID: 59 + TypeID: 58 + *scop.MarkDescriptorAsDropped + TableID: 53 + *scop.MarkDescriptorAsDropped + TableID: 54 + *scop.MarkDescriptorAsDropped + TableID: 55 + *scop.MarkDescriptorAsDropped + TableID: 56 + *scop.MarkDescriptorAsDropped + TableID: 59 +Stage 1 + *scop.CreateGcJobForDescriptor + DescID: 53 + *scop.CreateGcJobForDescriptor + DescID: 54 + *scop.CreateGcJobForDescriptor + DescID: 55 + *scop.CreateGcJobForDescriptor + DescID: 56 + *scop.CreateGcJobForDescriptor + DescID: 59 + *scop.DrainDescriptorName + TableID: 53 + *scop.DrainDescriptorName + TableID: 54 + *scop.DrainDescriptorName + TableID: 55 + *scop.DrainDescriptorName + TableID: 56 + *scop.DrainDescriptorName + TableID: 59 + +deps +DROP VIEW defaultdb.v1 CASCADE +---- +- from: [View:{DescID: 53}, DELETE_ONLY] + to: [View:{DescID: 54}, DELETE_ONLY] +- from: [View:{DescID: 53}, DELETE_ONLY] + to: [View:{DescID: 55}, DELETE_ONLY] +- from: [View:{DescID: 54}, DELETE_ONLY] + to: [View:{DescID: 55}, DELETE_ONLY] +- from: [View:{DescID: 54}, DELETE_ONLY] + to: [View:{DescID: 56}, DELETE_ONLY] +- from: [View:{DescID: 56}, DELETE_ONLY] + to: [View:{DescID: 59}, DELETE_ONLY] +- from: [View:{DescID: 59}, DELETE_ONLY] + to: [TypeReference:{DescID: 57, DepID: 59}, ABSENT] +- from: [View:{DescID: 59}, DELETE_ONLY] + to: [TypeReference:{DescID: 58, DepID: 59}, ABSENT]