From 448abbb63fa7e421f530a4299e7b1dbb1dd761b5 Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Thu, 15 Dec 2022 17:40:31 -0500 Subject: [PATCH] descs: add ByIDGetter and ByNameGetter interfaces These pave the way for deprecating the existing Get*ByID and Get*ByName Collection methods. Informs #87753. Release note: None --- pkg/ccl/backupccl/backupresolver/targets.go | 34 +- pkg/sql/catalog/catalog.go | 2 +- pkg/sql/catalog/descriptor.go | 66 +- pkg/sql/catalog/descs/BUILD.bazel | 3 +- pkg/sql/catalog/descs/collection.go | 43 +- pkg/sql/catalog/descs/database.go | 62 +- pkg/sql/catalog/descs/descriptor.go | 244 ++++--- .../catalog/descs/dist_sql_type_resolver.go | 21 +- pkg/sql/catalog/descs/function.go | 32 +- pkg/sql/catalog/descs/getters.go | 602 ++++++++++++++++++ pkg/sql/catalog/descs/helpers.go | 83 ++- pkg/sql/catalog/descs/hydrate.go | 66 +- pkg/sql/catalog/descs/object.go | 146 ----- pkg/sql/catalog/descs/schema.go | 72 +-- pkg/sql/catalog/descs/table.go | 77 +-- pkg/sql/catalog/descs/table_name.go | 50 -- pkg/sql/catalog/descs/type.go | 80 +-- pkg/sql/catalog/descs/virtual_descriptors.go | 72 +-- pkg/sql/catalog/errors.go | 4 + pkg/sql/catalog/lease/lease.go | 10 +- pkg/sql/catalog/lease/storage.go | 7 +- pkg/sql/crdb_internal.go | 4 +- pkg/sql/drop_cascade.go | 4 +- pkg/sql/importer/BUILD.bazel | 1 + pkg/sql/importer/import_stmt_test.go | 5 + pkg/sql/pg_catalog.go | 8 +- pkg/sql/schema_resolver.go | 42 +- pkg/sql/scrub.go | 13 +- pkg/sql/sem/tree/name_resolution.go | 8 +- pkg/sql/set_zone_config.go | 5 +- pkg/sql/ttl/ttljob/ttljob.go | 2 +- pkg/sql/ttl/ttljob/ttljob_processor.go | 2 +- pkg/sql/ttl/ttlschedule/ttlschedule.go | 20 +- pkg/sql/virtual_schema.go | 29 +- pkg/sql/zone_config.go | 20 +- 35 files changed, 1070 insertions(+), 869 deletions(-) create mode 100644 pkg/sql/catalog/descs/getters.go delete mode 100644 pkg/sql/catalog/descs/object.go delete mode 100644 pkg/sql/catalog/descs/table_name.go diff --git a/pkg/ccl/backupccl/backupresolver/targets.go b/pkg/ccl/backupccl/backupresolver/targets.go index a489167d14a7..a94d9a22d395 100644 --- a/pkg/ccl/backupccl/backupresolver/targets.go +++ b/pkg/ccl/backupccl/backupresolver/targets.go @@ -377,12 +377,9 @@ func DescriptorsMatchingTargets( if _, ok := alreadyRequestedDBs[dbID]; !ok { desc := r.DescByID[dbID] // Verify that the database is in the correct state. - doesNotExistErr := errors.Errorf(`database %q does not exist`, d) - if err := catalog.FilterDescriptorState( - desc, tree.CommonLookupFlags{}, - ); err != nil { + if desc == nil || !desc.Public() { // Return a does not exist error if explicitly asking for this database. - return ret, doesNotExistErr + return ret, errors.Errorf(`database %q does not exist`, d) } ret.Descs = append(ret.Descs, desc) ret.RequestedDBs = append(ret.RequestedDBs, desc.(catalog.DatabaseDescriptor)) @@ -401,15 +398,14 @@ func DescriptorsMatchingTargets( } if _, ok := alreadyRequestedSchemas[id]; !ok { schemaDesc := r.DescByID[id] - if err := catalog.FilterDescriptorState( - schemaDesc, tree.CommonLookupFlags{IncludeOffline: !requirePublic}, - ); err != nil { + if schemaDesc == nil || !schemaDesc.Public() { if requirePublic { return errors.Wrapf(err, "schema %d was expected to be PUBLIC", id) + } else if schemaDesc == nil || !schemaDesc.Offline() { + // If the schema is not public, but we don't require it to be, ignore + // it. + return nil } - // If the schema is not public, but we don't require it to be, ignore - // it. - return nil } alreadyRequestedSchemas[id] = struct{}{} ret.Descs = append(ret.Descs, r.DescByID[id]) @@ -485,16 +481,8 @@ func DescriptorsMatchingTargets( } tableDesc, isTable := descI.(catalog.TableDescriptor) // If the type assertion didn't work, then we resolved a type instead, so - // error out. - if !isTable { - return ret, doesNotExistErr - } - - // Verify that the table is in the correct state. - if err := catalog.FilterDescriptorState( - tableDesc, tree.CommonLookupFlags{}, - ); err != nil { - // Return a does not exist error if explicitly asking for this table. + // error out. Otherwise verify that the table is in the correct state. + if !isTable || tableDesc == nil || !tableDesc.Public() { return ret, doesNotExistErr } @@ -591,9 +579,7 @@ func DescriptorsMatchingTargets( addObjectDescsInSchema := func(objectsIDs *catalog.DescriptorIDSet) error { for _, id := range objectsIDs.Ordered() { desc := r.DescByID[id] - if err := catalog.FilterDescriptorState( - desc, tree.CommonLookupFlags{IncludeOffline: true}, - ); err != nil { + if desc == nil || (!desc.Public() && !desc.Offline()) { // Don't include this object in the expansion since it's not in a valid // state. Silently fail since this object was not directly requested, // but was just part of an expansion. diff --git a/pkg/sql/catalog/catalog.go b/pkg/sql/catalog/catalog.go index 30443454fee3..ebf135bd820f 100644 --- a/pkg/sql/catalog/catalog.go +++ b/pkg/sql/catalog/catalog.go @@ -67,7 +67,7 @@ type VirtualSchema interface { Desc() SchemaDescriptor NumTables() int VisitTables(func(object VirtualObject)) - GetObjectByName(name string, flags tree.ObjectLookupFlags) (VirtualObject, error) + GetObjectByName(name string, kind tree.DesiredObjectKind) (VirtualObject, error) } // VirtualObject is a virtual schema object. diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index facf48501951..187130116b11 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -928,46 +928,38 @@ type FunctionDescriptor interface { ToCreateExpr() (*tree.CreateFunction, error) } -// FilterDescriptorState inspects the state of a given descriptor and returns an -// error if the state is anything but public. The error describes the state of -// the descriptor. -func FilterDescriptorState(desc Descriptor, flags tree.CommonLookupFlags) error { - if flags.ParentID != 0 { - parent := desc.GetParentID() - if parent != 0 && parent != flags.ParentID { - return ErrDescriptorNotFound - } +// FilterDroppedDescriptor returns an error if the descriptor state is DROP. +func FilterDroppedDescriptor(desc Descriptor) error { + if !desc.Dropped() { + return nil + } + return NewInactiveDescriptorError(ErrDescriptorDropped) +} - switch { - case desc.Dropped() && !flags.IncludeDropped: - return NewInactiveDescriptorError(ErrDescriptorDropped) - case desc.Offline() && !flags.IncludeOffline: - err := errors.Errorf("%s %q is offline", desc.DescriptorType(), desc.GetName()) - if desc.GetOfflineReason() != "" { - err = errors.Errorf("%s %q is offline: %s", desc.DescriptorType(), desc.GetName(), desc.GetOfflineReason()) - } - return NewInactiveDescriptorError(err) - case desc.Adding() && - // The ADD state is special. - // We don't want adding descriptors to be visible to DML queries, but we - // want them to be visible to schema changes: - // - when uncommitted we want them to be accessible by name for other - // schema changes, e.g. - // BEGIN; CREATE TABLE t ... ; ALTER TABLE t RENAME TO ...; - // should be possible. - // - when committed we want them to be accessible to their own schema - // change job, where they're referenced by ID. - // - // The AvoidCommittedAdding is set if and only if the lookup is by-name - // and prevents them from seeing committed adding descriptors. - !(flags.AvoidCommittedAdding && desc.IsUncommittedVersion() && (flags.AvoidLeased || flags.RequireMutable)) && - !(!flags.AvoidCommittedAdding && (desc.IsUncommittedVersion() || flags.AvoidLeased || flags.RequireMutable)): - // For the time being, only table descriptors can be in the adding state. - return pgerror.WithCandidateCode(newAddingTableError(desc.(TableDescriptor)), - pgcode.ObjectNotInPrerequisiteState) +// FilterOfflineDescriptor returns an error if the descriptor state is OFFLINE. +func FilterOfflineDescriptor(desc Descriptor) error { + if !desc.Offline() { + return nil + } + err := errors.Errorf("%s %q is offline", desc.DescriptorType(), desc.GetName()) + if desc.GetOfflineReason() != "" { + err = errors.Errorf("%s %q is offline: %s", desc.DescriptorType(), desc.GetName(), desc.GetOfflineReason()) + } + return NewInactiveDescriptorError(err) +} + +// FilterAddingDescriptor returns an error if the descriptor state is ADD. +func FilterAddingDescriptor(desc Descriptor) error { + if !desc.Adding() { + return nil + } + // For the time being, only table descriptors can be in the adding state. + tbl, err := AsTableDescriptor(desc) + if err != nil { + return errors.HandleAsAssertionFailure(err) } - return nil + return pgerror.WithCandidateCode(newAddingTableError(tbl), pgcode.ObjectNotInPrerequisiteState) } // TableLookupFn is used to resolve a table from an ID, particularly when diff --git a/pkg/sql/catalog/descs/BUILD.bazel b/pkg/sql/catalog/descs/BUILD.bazel index c6fdcae5c01b..f535455e4bbd 100644 --- a/pkg/sql/catalog/descs/BUILD.bazel +++ b/pkg/sql/catalog/descs/BUILD.bazel @@ -11,16 +11,15 @@ go_library( "errors.go", "factory.go", "function.go", + "getters.go", "helpers.go", "hydrate.go", "leased_descriptors.go", - "object.go", "schema.go", "session_data.go", "synthetic_descriptors.go", "system_table.go", "table.go", - "table_name.go", "temporary_descriptors.go", "txn.go", "type.go", diff --git a/pkg/sql/catalog/descs/collection.go b/pkg/sql/catalog/descs/collection.go index 0f8115cd955d..35449efe3bc7 100644 --- a/pkg/sql/catalog/descs/collection.go +++ b/pkg/sql/catalog/descs/collection.go @@ -585,12 +585,9 @@ func (tc *Collection) lookupDescriptorID( // First look up in-memory descriptors in collection, // except for leased descriptors. objInMemory, err := func() (catalog.Descriptor, error) { - flags := tree.CommonLookupFlags{ - Required: true, - AvoidLeased: true, - AvoidStorage: true, - IncludeOffline: true, - } + flags := defaultUnleasedFlags() + flags.layerFilters.withoutStorage = true + flags.descFilters.withoutDropped = true var db catalog.DatabaseDescriptor var sc catalog.SchemaDescriptor expectedType := catalog.Database @@ -618,13 +615,9 @@ func (tc *Collection) lookupDescriptorID( } } } - return tc.getDescriptorByName(ctx, txn, db, sc, key.Name, flags, expectedType) + return getDescriptorByName(ctx, txn, tc, db, sc, key.Name, flags, expectedType) }() - if errors.IsAny(err, catalog.ErrDescriptorNotFound, catalog.ErrDescriptorDropped) { - // Swallow these errors to fall back to storage lookup. - err = nil - } - if err != nil { + if err != nil && !errors.Is(err, catalog.ErrDescriptorNotFound) { return descpb.InvalidID, err } if objInMemory != nil { @@ -978,13 +971,10 @@ func (tc *Collection) aggregateAllLayers( tc.uncommittedZoneConfigs.addAllToCatalog(ret) // Remove deleted descriptors from consideration, re-read and add the rest. tc.deletedDescs.ForEach(descIDs.Remove) - flags := tree.CommonLookupFlags{ - AvoidLeased: true, - IncludeOffline: true, - IncludeDropped: true, - } - allDescs, err := tc.getDescriptorsByID(ctx, txn, flags, descIDs.Ordered()...) - if err != nil { + allDescs := make([]catalog.Descriptor, descIDs.Len()) + if err := getDescriptorsByID( + ctx, tc, txn, defaultUnleasedFlags(), allDescs, descIDs.Ordered()..., + ); err != nil { return nstree.MutableCatalog{}, err } for _, desc := range allDescs { @@ -1002,15 +992,12 @@ func (tc *Collection) GetAllDescriptorsForDatabase( ctx context.Context, txn *kv.Txn, db catalog.DatabaseDescriptor, ) (nstree.Catalog, error) { // Re-read database descriptor to have the freshest version. - flags := tree.CommonLookupFlags{ - AvoidLeased: true, - IncludeDropped: true, - IncludeOffline: true, - } - var err error - _, db, err = tc.GetImmutableDatabaseByID(ctx, txn, db.GetID(), flags) - if err != nil { - return nstree.Catalog{}, err + { + var err error + db, err = ByIDGetter(makeGetterBase(txn, tc, defaultUnleasedFlags())).Database(ctx, db.GetID()) + if err != nil { + return nstree.Catalog{}, err + } } c, err := tc.GetAllInDatabase(ctx, txn, db) if err != nil { diff --git a/pkg/sql/catalog/descs/database.go b/pkg/sql/catalog/descs/database.go index cb5602239ff1..77645f533ada 100644 --- a/pkg/sql/catalog/descs/database.go +++ b/pkg/sql/catalog/descs/database.go @@ -12,15 +12,12 @@ package descs import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" - "github.com/cockroachdb/errors" ) // GetMutableDatabaseByName returns a mutable database descriptor with @@ -28,12 +25,7 @@ import ( func (tc *Collection) GetMutableDatabaseByName( ctx context.Context, txn *kv.Txn, name string, flags tree.DatabaseLookupFlags, ) (*dbdesc.Mutable, error) { - flags.RequireMutable = true - desc, err := tc.getDatabaseByName(ctx, txn, name, flags) - if err != nil || desc == nil { - return nil, err - } - return desc.(*dbdesc.Mutable), nil + return tc.ByName(txn).WithFlags(flags).Mutable().Database(ctx, name) } // GetImmutableDatabaseByName returns an immutable database descriptor with @@ -41,33 +33,7 @@ func (tc *Collection) GetMutableDatabaseByName( func (tc *Collection) GetImmutableDatabaseByName( ctx context.Context, txn *kv.Txn, name string, flags tree.DatabaseLookupFlags, ) (catalog.DatabaseDescriptor, error) { - flags.RequireMutable = false - 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, -) (catalog.DatabaseDescriptor, error) { - desc, err := tc.getDescriptorByName(ctx, txn, nil /* db */, nil /* sc */, name, flags, catalog.Database) - if err != nil { - return nil, err - } - if desc == nil { - if flags.Required { - return nil, sqlerrors.NewUndefinedDatabaseError(name) - } - return nil, nil - } - db, ok := desc.(catalog.DatabaseDescriptor) - if !ok { - if flags.Required { - return nil, sqlerrors.NewUndefinedDatabaseError(name) - } - return nil, nil - } - return db, nil + return tc.ByName(txn).WithFlags(flags).Immutable().Database(ctx, name) } // GetImmutableDatabaseByID returns an immutable database descriptor with @@ -75,26 +41,6 @@ func (tc *Collection) getDatabaseByName( func (tc *Collection) GetImmutableDatabaseByID( ctx context.Context, txn *kv.Txn, dbID descpb.ID, flags tree.DatabaseLookupFlags, ) (bool, catalog.DatabaseDescriptor, error) { - flags.RequireMutable = false - return tc.getDatabaseByID(ctx, txn, dbID, flags) -} - -func (tc *Collection) getDatabaseByID( - ctx context.Context, txn *kv.Txn, dbID descpb.ID, flags tree.DatabaseLookupFlags, -) (bool, catalog.DatabaseDescriptor, error) { - descs, err := tc.getDescriptorsByID(ctx, txn, flags, dbID) - if err != nil { - if errors.Is(err, catalog.ErrDescriptorNotFound) { - if flags.Required { - return false, nil, sqlerrors.NewUndefinedDatabaseError(fmt.Sprintf("[%d]", dbID)) - } - return false, nil, nil - } - return false, nil, err - } - db, ok := descs[0].(catalog.DatabaseDescriptor) - if !ok { - return false, nil, sqlerrors.NewUndefinedDatabaseError(fmt.Sprintf("[%d]", dbID)) - } - return true, db, nil + db, err := tc.ByID(txn).WithFlags(flags).Immutable().Database(ctx, dbID) + return db != nil, db, err } diff --git a/pkg/sql/catalog/descs/descriptor.go b/pkg/sql/catalog/descs/descriptor.go index 9bfdf88e9647..160dd4c5d6a0 100644 --- a/pkg/sql/catalog/descs/descriptor.go +++ b/pkg/sql/catalog/descs/descriptor.go @@ -34,31 +34,14 @@ import ( func (tc *Collection) GetMutableDescriptorsByID( ctx context.Context, txn *kv.Txn, ids ...descpb.ID, ) ([]catalog.MutableDescriptor, error) { - flags := tree.CommonLookupFlags{ - RequireMutable: true, - IncludeOffline: true, - IncludeDropped: true, - } - descs, err := tc.getDescriptorsByID(ctx, txn, flags, ids...) - if err != nil { - return nil, err - } - ret := make([]catalog.MutableDescriptor, len(descs)) - for i, desc := range descs { - ret[i] = desc.(catalog.MutableDescriptor) - } - return ret, nil + return tc.ByID(txn).Mutable().Descs(ctx, ids) } // GetMutableDescriptorByID delegates to GetMutableDescriptorsByID. func (tc *Collection) GetMutableDescriptorByID( ctx context.Context, txn *kv.Txn, id descpb.ID, ) (catalog.MutableDescriptor, error) { - descs, err := tc.GetMutableDescriptorsByID(ctx, txn, id) - if err != nil { - return nil, err - } - return descs[0], nil + return tc.ByID(txn).Mutable().Desc(ctx, id) } // GetImmutableDescriptorsByID returns an immutable implementation of the @@ -67,43 +50,14 @@ func (tc *Collection) GetMutableDescriptorByID( func (tc *Collection) GetImmutableDescriptorsByID( ctx context.Context, txn *kv.Txn, flags tree.CommonLookupFlags, ids ...descpb.ID, ) ([]catalog.Descriptor, error) { - flags.RequireMutable = false - return tc.getDescriptorsByID(ctx, txn, flags, ids...) + return tc.ByID(txn).WithFlags(flags).Immutable().Descs(ctx, ids) } // GetImmutableDescriptorByID delegates to GetImmutableDescriptorsByID. func (tc *Collection) GetImmutableDescriptorByID( ctx context.Context, txn *kv.Txn, id descpb.ID, flags tree.CommonLookupFlags, ) (catalog.Descriptor, error) { - flags.RequireMutable = false - return tc.getDescriptorByID(ctx, txn, flags, id) -} - -// MaybeGetTable implements the catalog.ZoneConfigHydrationHelper interface. -func (tc *Collection) MaybeGetTable( - ctx context.Context, txn *kv.Txn, id descpb.ID, -) (catalog.TableDescriptor, error) { - // Ignore ids without a descriptor. - if id == keys.RootNamespaceID || keys.IsPseudoTableID(uint32(id)) { - return nil, nil - } - desc, err := tc.GetImmutableDescriptorByID( - ctx, - txn, - id, - tree.CommonLookupFlags{ - AvoidLeased: true, - IncludeOffline: true, - IncludeDropped: true, - }, - ) - if err != nil { - return nil, err - } - if desc.DescriptorType() == catalog.Table { - return desc.(catalog.TableDescriptor), nil - } - return nil, nil + return tc.ByID(txn).WithFlags(flags).Immutable().Desc(ctx, id) } // GetComment fetches comment from uncommitted cache if it exists, otherwise from storage. @@ -169,14 +123,6 @@ func (tc *Collection) GetZoneConfigs( return ret, nil } -// MaybeGetZoneConfig implements the catalog.ZoneConfigHydrationHelper -// interface. -func (tc *Collection) MaybeGetZoneConfig( - ctx context.Context, txn *kv.Txn, id descpb.ID, -) (catalog.ZoneConfig, error) { - return tc.GetZoneConfig(ctx, txn, id) -} - // AddUncommittedZoneConfig adds a zone config to the uncommitted cache. func (tc *Collection) AddUncommittedZoneConfig(id descpb.ID, zc *zonepb.ZoneConfig) error { return tc.uncommittedZoneConfigs.upsert(id, zc) @@ -202,38 +148,6 @@ func (tc *Collection) MarkUncommittedCommentDeletedForTable(tblID descpb.ID) { tc.uncommittedComments.markTableDeleted(tblID) } -// getDescriptorsByID returns a descriptor by ID according to the provided -// lookup flags. -// -// The Required flag is ignored and always overridden. -func (tc *Collection) getDescriptorByID( - ctx context.Context, txn *kv.Txn, flags tree.CommonLookupFlags, id descpb.ID, -) (catalog.Descriptor, error) { - var arr [1]catalog.Descriptor - if err := getDescriptorsByID( - ctx, tc, txn, flags, arr[:], id, - ); err != nil { - return nil, err - } - return arr[0], nil -} - -// getDescriptorsByID returns a slice of descriptors by ID according to the -// provided lookup flags. -// -// The Required flag is ignored and always overridden. -func (tc *Collection) getDescriptorsByID( - ctx context.Context, txn *kv.Txn, flags tree.CommonLookupFlags, ids ...descpb.ID, -) ([]catalog.Descriptor, error) { - descs := make([]catalog.Descriptor, len(ids)) - if err := getDescriptorsByID( - ctx, tc, txn, flags, descs, ids..., - ); err != nil { - return nil, err - } - return descs, nil -} - // getDescriptorsByID implements the Collection method of the same name. // It takes a slice into which the retrieved descriptors will be stored. // That slice must be the same length as the ids. This allows callers @@ -243,12 +157,10 @@ func getDescriptorsByID( ctx context.Context, tc *Collection, txn *kv.Txn, - flags tree.CommonLookupFlags, + flags getterFlags, descs []catalog.Descriptor, ids ...descpb.ID, ) (err error) { - // Override flags. - flags.Required = true if log.ExpensiveLogEnabled(ctx, 2) { // Copy the ids to a new slice to prevent the backing array from // escaping and forcing IDs to escape on this hot path. @@ -317,7 +229,7 @@ func getDescriptorsByID( } } if !readIDs.Empty() { - if flags.AvoidStorage { + if flags.layerFilters.withoutStorage { // Some descriptors are still missing and there's nowhere left to get // them from. return catalog.ErrDescriptorNotFound @@ -344,13 +256,54 @@ func getDescriptorsByID( return err } for _, desc := range descs { - if err := catalog.FilterDescriptorState(desc, flags); err != nil { + if err := filterDescriptor(desc, flags); err != nil { return err } } return nil } +func filterDescriptor(desc catalog.Descriptor, flags getterFlags) error { + if expected := flags.descFilters.maybeParentID; expected != descpb.InvalidID { + if actual := desc.GetParentID(); actual != descpb.InvalidID && actual != expected { + return catalog.ErrDescriptorNotFound + } + } + if flags.descFilters.withoutDropped { + if err := catalog.FilterDroppedDescriptor(desc); err != nil { + return err + } + } + if flags.descFilters.withoutOffline { + if err := catalog.FilterOfflineDescriptor(desc); err != nil { + return err + } + } + // Handle the special case of the ADD state. + // We don't want adding descriptors to be visible to DML queries, but we + // want them to be visible to schema changes: + // - when uncommitted we want them to be accessible by name for other + // schema changes, e.g. + // BEGIN; CREATE TABLE t ... ; ALTER TABLE t RENAME TO ...; + // should be possible. + // - when committed we want them to be accessible to their own schema + // change job, where they're referenced by ID. + // + // The AvoidCommittedAdding is set if and only if the lookup is by-name + // and prevents them from seeing committed adding descriptors. + if desc.IsUncommittedVersion() { + if !flags.descFilters.withoutCommittedAdding || flags.layerFilters.withoutLeased { + return nil + } + } + if !flags.descFilters.withoutCommittedAdding { + if flags.layerFilters.withoutLeased { + return nil + } + } + return catalog.FilterAddingDescriptor(desc) +} + // byIDLookupContext is a helper struct for getDescriptorsByID which contains // the parameters for looking up descriptors by ID at various levels in the // Collection. @@ -358,7 +311,7 @@ type byIDLookupContext struct { ctx context.Context txn *kv.Txn tc *Collection - flags tree.CommonLookupFlags + flags getterFlags } func (q *byIDLookupContext) lookupVirtual( @@ -366,17 +319,28 @@ func (q *byIDLookupContext) lookupVirtual( ) (catalog.Descriptor, catalog.ValidationLevel, error) { // TODO(postamar): get rid of descriptorless public schemas if id == keys.PublicSchemaID { - if q.flags.RequireMutable { + if q.flags.isMutable { err := catalog.NewMutableAccessToVirtualSchemaError(schemadesc.GetPublicSchema()) return nil, catalog.NoValidation, err } return schemadesc.GetPublicSchema(), validate.Write, nil } - desc, err := q.tc.virtual.getByID(q.ctx, id, q.flags.RequireMutable) - if err != nil || desc == nil { + if vs := q.tc.virtual.getSchemaByID(id); vs != nil { + if q.flags.isMutable { + err := catalog.NewMutableAccessToVirtualSchemaError(vs.Desc()) + return nil, catalog.NoValidation, err + } + return vs.Desc(), validate.Write, nil + } + vs, vd := q.tc.virtual.getObjectByID(id) + if vd == nil { + return nil, catalog.NoValidation, nil + } + if q.flags.isMutable { + err := catalog.NewMutableAccessToVirtualObjectError(vs, vd) return nil, catalog.NoValidation, err } - return desc, validate.Write, nil + return vd.Desc(), validate.Write, nil } func (q *byIDLookupContext) lookupTemporary( @@ -386,7 +350,7 @@ func (q *byIDLookupContext) lookupTemporary( if td == nil { return nil, catalog.NoValidation, nil } - if q.flags.RequireMutable { + if q.flags.isMutable { err := catalog.NewMutableAccessToVirtualSchemaError(schemadesc.GetPublicSchema()) return nil, catalog.NoValidation, err } @@ -396,14 +360,14 @@ func (q *byIDLookupContext) lookupTemporary( func (q *byIDLookupContext) lookupSynthetic( id descpb.ID, ) (catalog.Descriptor, catalog.ValidationLevel, error) { - if q.flags.AvoidSynthetic { + if q.flags.layerFilters.withoutSynthetic { return nil, catalog.NoValidation, nil } sd := q.tc.synthetic.getSyntheticByID(id) if sd == nil { return nil, catalog.NoValidation, nil } - if q.flags.RequireMutable { + if q.flags.isMutable { return nil, catalog.NoValidation, newMutableSyntheticDescriptorAssertionError(sd.GetID()) } return sd, validate.Write, nil @@ -432,12 +396,12 @@ func (q *byIDLookupContext) lookupUncommitted( func (q *byIDLookupContext) lookupLeased( id descpb.ID, ) (catalog.Descriptor, catalog.ValidationLevel, error) { - if q.flags.AvoidLeased || q.flags.RequireMutable || lease.TestingTableLeasesAreDisabled() { + if q.flags.layerFilters.withoutLeased || lease.TestingTableLeasesAreDisabled() { return nil, catalog.NoValidation, nil } // If we have already read all of the descriptors, use it as a negative // cache to short-circuit a lookup we know will be doomed to fail. - if q.tc.cr.IsDescIDKnownToNotExist(id, q.flags.ParentID) { + if q.tc.cr.IsDescIDKnownToNotExist(id, q.flags.descFilters.maybeParentID) { return nil, catalog.NoValidation, catalog.ErrDescriptorNotFound } desc, shouldReadFromStore, err := q.tc.leased.getByID(q.ctx, q.tc.deadlineHolder(q.txn), id) @@ -447,19 +411,18 @@ func (q *byIDLookupContext) lookupLeased( return desc, validate.ImmutableRead, nil } -// getDescriptorByName looks up a descriptor by name. -// -// The Required and AvoidCommittedAdding flags are ignored and overridden. -func (tc *Collection) getDescriptorByName( +// getDescriptorByName looks up a descriptor by name on a best-effort basis. +func getDescriptorByName( ctx context.Context, txn *kv.Txn, + tc *Collection, db catalog.DatabaseDescriptor, sc catalog.SchemaDescriptor, name string, - flags tree.CommonLookupFlags, + flags getterFlags, requestedType catalog.DescriptorType, ) (catalog.Descriptor, error) { - mustBeVirtual, vd, err := tc.getVirtualDescriptorByName(sc, name, flags.RequireMutable, requestedType) + mustBeVirtual, vd, err := tc.getVirtualDescriptorByName(sc, name, flags.isMutable, requestedType) if mustBeVirtual || vd != nil || err != nil || (db == nil && sc != nil) { return vd, err } @@ -469,13 +432,14 @@ func (tc *Collection) getDescriptorByName( } // When looking up descriptors by name, then descriptors in the adding state // must be uncommitted to be visible (among other things). - flags.AvoidCommittedAdding = true - desc, err := tc.getDescriptorByID(ctx, txn, flags, id) + flags.descFilters.withoutCommittedAdding = true + var arr [1]catalog.Descriptor + err = getDescriptorsByID(ctx, tc, txn, flags, arr[:], id) if err == nil { - return desc, nil + return arr[0], nil } - // Swallow error if the descriptor is dropped. if errors.Is(err, catalog.ErrDescriptorDropped) { + // Swallow error if the descriptor is dropped. return nil, nil } if errors.Is(err, catalog.ErrDescriptorNotFound) { @@ -484,6 +448,11 @@ func (tc *Collection) getDescriptorByName( if db != nil && sc == nil && isTemporarySchema(name) { return schemadesc.NewTemporarySchema(name, id, db.GetID()), nil } + // Special case for a descriptor which exists but which we're unable to + // retrieve. + if flags.layerFilters.withoutStorage { + return nil, err + } // In all other cases, having an ID should imply having a descriptor. return nil, errors.WithAssertionFailure(err) } @@ -508,25 +477,33 @@ func (tc *Collection) getVirtualDescriptorByName( isMutableRequired bool, requestedType catalog.DescriptorType, ) (continueOrHalt, catalog.Descriptor, error) { - objFlags := tree.ObjectLookupFlags{ - CommonLookupFlags: tree.CommonLookupFlags{ - RequireMutable: isMutableRequired, - }, - } + requestedKind := tree.TableObject switch requestedType { case catalog.Database, catalog.Function: return continueLookups, nil, nil case catalog.Schema: if vs := tc.virtual.getSchemaByName(name); vs != nil { + if isMutableRequired { + return haltLookups, nil, catalog.NewMutableAccessToVirtualSchemaError(vs) + } return haltLookups, vs, nil } case catalog.Type, catalog.Any: - objFlags.DesiredObjectKind = tree.TypeObject + requestedKind = tree.TypeObject fallthrough case catalog.Table: - isVirtual, vd, err := tc.virtual.getObjectByName(sc.GetName(), name, objFlags) - if isVirtual || vd != nil || err != nil { - return haltLookups, vd, err + vs, vd, err := tc.virtual.getObjectByName(sc.GetName(), name, requestedKind) + if err != nil { + return haltLookups, nil, err + } + if vd != nil { + if isMutableRequired { + return haltLookups, nil, catalog.NewMutableAccessToVirtualObjectError(vs, vd) + } + return haltLookups, vd.Desc(), nil + } + if vs != nil { + return haltLookups, nil, nil } } return continueLookups, nil, nil @@ -542,13 +519,8 @@ func (tc *Collection) getNonVirtualDescriptorID( db catalog.DatabaseDescriptor, sc catalog.SchemaDescriptor, name string, - flags tree.CommonLookupFlags, + flags getterFlags, ) (descpb.ID, error) { - flags = tree.CommonLookupFlags{ - AvoidLeased: flags.AvoidLeased, - RequireMutable: flags.RequireMutable, - AvoidSynthetic: flags.AvoidSynthetic, - } var parentID, parentSchemaID descpb.ID var isSchema bool if db != nil { @@ -596,7 +568,7 @@ func (tc *Collection) getNonVirtualDescriptorID( return haltLookups, descpb.InvalidID, nil } lookupSyntheticID := func() (continueOrHalt, descpb.ID, error) { - if flags.AvoidSynthetic { + if flags.layerFilters.withoutSynthetic { return continueLookups, descpb.InvalidID, nil } if sd := tc.synthetic.getSyntheticByName(parentID, parentSchemaID, name); sd != nil { @@ -624,7 +596,7 @@ func (tc *Collection) getNonVirtualDescriptorID( return continueLookups, descpb.InvalidID, nil } lookupLeasedID := func() (continueOrHalt, descpb.ID, error) { - if flags.AvoidLeased || flags.RequireMutable || lease.TestingTableLeasesAreDisabled() { + if flags.layerFilters.withoutLeased || lease.TestingTableLeasesAreDisabled() { return continueLookups, descpb.InvalidID, nil } if isSchema && isTemporarySchema(name) { @@ -642,7 +614,7 @@ func (tc *Collection) getNonVirtualDescriptorID( return haltLookups, ld.GetID(), nil } lookupStoredID := func() (continueOrHalt, descpb.ID, error) { - if flags.AvoidStorage { + if flags.layerFilters.withoutStorage { return haltLookups, descpb.InvalidID, nil } ni := descpb.NameInfo{ParentID: parentID, ParentSchemaID: parentSchemaID, Name: name} @@ -687,12 +659,12 @@ func (tc *Collection) getNonVirtualDescriptorID( func (tc *Collection) finalizeDescriptors( ctx context.Context, txn *kv.Txn, - flags tree.CommonLookupFlags, + flags getterFlags, descs []catalog.Descriptor, validationLevels []catalog.ValidationLevel, ) error { // Add the descriptors to the uncommitted layer if we want them to be mutable. - if flags.RequireMutable { + if flags.isMutable { for i, desc := range descs { mut, err := tc.uncommitted.ensureMutable(ctx, desc) if err != nil { @@ -706,7 +678,7 @@ func (tc *Collection) finalizeDescriptors( return nil } requiredLevel := validate.MutableRead - if !flags.RequireMutable && !flags.AvoidLeased { + if !flags.layerFilters.withoutLeased { requiredLevel = validate.ImmutableRead } var toValidate []catalog.Descriptor diff --git a/pkg/sql/catalog/descs/dist_sql_type_resolver.go b/pkg/sql/catalog/descs/dist_sql_type_resolver.go index c4b53b995d6d..83add64e194d 100644 --- a/pkg/sql/catalog/descs/dist_sql_type_resolver.go +++ b/pkg/sql/catalog/descs/dist_sql_type_resolver.go @@ -28,15 +28,13 @@ import ( // DistSQLTypeResolver is a TypeResolver that accesses TypeDescriptors through // a given descs.Collection and transaction. type DistSQLTypeResolver struct { - descriptors *Collection - txn *kv.Txn + g ByIDGetter } // NewDistSQLTypeResolver creates a new DistSQLTypeResolver. func NewDistSQLTypeResolver(descs *Collection, txn *kv.Txn) DistSQLTypeResolver { return DistSQLTypeResolver{ - descriptors: descs, - txn: txn, + g: descs.ByID(txn).Immutable(), } } @@ -62,30 +60,29 @@ func (dt *DistSQLTypeResolver) ResolveTypeByOID( return desc.MakeTypesT(ctx, &name, dt) } -// GetTypeDescriptor implements the sqlbase.TypeDescriptorResolver interface. +// GetTypeDescriptor implements the catalog.TypeDescriptorResolver interface. func (dt *DistSQLTypeResolver) GetTypeDescriptor( ctx context.Context, id descpb.ID, ) (tree.TypeName, catalog.TypeDescriptor, error) { - descs, err := dt.descriptors.getDescriptorsByID(ctx, dt.txn, tree.CommonLookupFlags{}, id) + desc, err := dt.g.Desc(ctx, id) if err != nil { return tree.TypeName{}, nil, err } - var typeDesc catalog.TypeDescriptor - switch t := descs[0].(type) { + name := tree.MakeUnqualifiedTypeName(desc.GetName()) + switch t := desc.(type) { case catalog.TypeDescriptor: // User-defined type. - typeDesc = t + return name, t, nil case catalog.TableDescriptor: - typeDesc, err = typedesc.CreateImplicitRecordTypeFromTableDesc(t) + typ, err := typedesc.CreateImplicitRecordTypeFromTableDesc(t) if err != nil { return tree.TypeName{}, nil, err } + return name, typ, nil default: return tree.TypeName{}, nil, pgerror.Newf(pgcode.WrongObjectType, "descriptor %d is a %s not a %s", id, t.DescriptorType(), catalog.Type) } - name := tree.MakeUnqualifiedTypeName(typeDesc.GetName()) - return name, typeDesc, nil } // HydrateTypeSlice installs metadata into a slice of types.T's. diff --git a/pkg/sql/catalog/descs/function.go b/pkg/sql/catalog/descs/function.go index fd9f4c0d906e..919cc6169b6e 100644 --- a/pkg/sql/catalog/descs/function.go +++ b/pkg/sql/catalog/descs/function.go @@ -18,46 +18,18 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/funcdesc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/errors" ) // GetImmutableFunctionByID returns a immutable function descriptor. func (tc *Collection) GetImmutableFunctionByID( ctx context.Context, txn *kv.Txn, fnID descpb.ID, flags tree.ObjectLookupFlags, ) (catalog.FunctionDescriptor, error) { - flags.RequireMutable = false - desc, err := tc.getFunctionByID(ctx, txn, fnID, flags) - if err != nil { - return nil, err - } - return desc, nil + return tc.ByID(txn).WithObjFlags(flags).Immutable().Function(ctx, fnID) } // GetMutableFunctionByID returns a mutable function descriptor. func (tc *Collection) GetMutableFunctionByID( ctx context.Context, txn *kv.Txn, fnID descpb.ID, flags tree.ObjectLookupFlags, ) (*funcdesc.Mutable, error) { - flags.RequireMutable = true - desc, err := tc.getFunctionByID(ctx, txn, fnID, flags) - if err != nil { - return nil, err - } - return desc.(*funcdesc.Mutable), nil -} - -func (tc *Collection) getFunctionByID( - ctx context.Context, txn *kv.Txn, fnID descpb.ID, flags tree.ObjectLookupFlags, -) (catalog.FunctionDescriptor, error) { - desc, err := tc.getDescriptorByID(ctx, txn, flags.CommonLookupFlags, fnID) - if err != nil { - if errors.Is(err, catalog.ErrDescriptorNotFound) { - return nil, errors.Wrapf(tree.ErrFunctionUndefined, "function %d does not exist", fnID) - } - return nil, err - } - fn, ok := desc.(catalog.FunctionDescriptor) - if !ok { - return nil, errors.Wrapf(tree.ErrFunctionUndefined, "function %d does not exist", fnID) - } - return fn, nil + return tc.ByID(txn).WithObjFlags(flags).Mutable().Function(ctx, fnID) } diff --git a/pkg/sql/catalog/descs/getters.go b/pkg/sql/catalog/descs/getters.go new file mode 100644 index 000000000000..44f43d68298e --- /dev/null +++ b/pkg/sql/catalog/descs/getters.go @@ -0,0 +1,602 @@ +// Copyright 2022 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 descs + +import ( + "context" + "fmt" + + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/funcdesc" + "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/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" + "github.com/cockroachdb/errors" +) + +// ByIDGetter looks up immutable descriptors by ID. +type ByIDGetter getterBase + +// Descs looks up immutable descriptors by ID. +func (g ByIDGetter) Descs(ctx context.Context, ids []descpb.ID) ([]catalog.Descriptor, error) { + ret := make([]catalog.Descriptor, len(ids)) + if err := getDescriptorsByID(ctx, g.Descriptors(), g.KV(), g.flags, ret, ids...); err != nil { + return nil, err + } + return ret, nil +} + +// Desc looks up an immutable descriptor by ID. +func (g ByIDGetter) Desc(ctx context.Context, id descpb.ID) (catalog.Descriptor, error) { + var arr [1]catalog.Descriptor + if err := getDescriptorsByID(ctx, g.Descriptors(), g.KV(), g.flags, arr[:], id); err != nil { + return nil, err + } + return arr[0], nil +} + +// Database looks up an immutable database descriptor by ID. +func (g ByIDGetter) Database( + ctx context.Context, id descpb.ID, +) (catalog.DatabaseDescriptor, error) { + desc, err := g.Desc(ctx, id) + if err != nil { + if errors.Is(err, catalog.ErrDescriptorNotFound) { + if g.flags.isOptional { + return nil, nil + } + return nil, sqlerrors.NewUndefinedDatabaseError(fmt.Sprintf("[%d]", id)) + } + return nil, err + } + db, ok := desc.(catalog.DatabaseDescriptor) + if !ok { + return nil, sqlerrors.NewUndefinedDatabaseError(fmt.Sprintf("[%d]", id)) + } + return db, nil +} + +// Schema looks up an immutable schema descriptor by ID. +func (g ByIDGetter) Schema(ctx context.Context, id descpb.ID) (catalog.SchemaDescriptor, error) { + desc, err := g.Desc(ctx, id) + if err != nil { + if errors.Is(err, catalog.ErrDescriptorNotFound) { + if g.flags.isOptional { + return nil, nil + } + return nil, sqlerrors.NewUndefinedSchemaError(fmt.Sprintf("[%d]", id)) + } + return nil, err + } + sc, ok := desc.(catalog.SchemaDescriptor) + if !ok { + return nil, sqlerrors.NewUndefinedSchemaError(fmt.Sprintf("[%d]", id)) + } + return sc, nil +} + +// Table looks up an immutable table descriptor by ID. +func (g ByIDGetter) Table(ctx context.Context, id descpb.ID) (catalog.TableDescriptor, error) { + desc, err := g.Desc(ctx, id) + if err != nil { + if errors.Is(err, catalog.ErrDescriptorNotFound) { + return nil, sqlerrors.NewUndefinedRelationError(&tree.TableRef{TableID: int64(id)}) + } + return nil, err + } + tbl, ok := desc.(catalog.TableDescriptor) + if !ok { + return nil, sqlerrors.NewUndefinedRelationError(&tree.TableRef{TableID: int64(id)}) + } + return tbl, nil +} + +// Type looks up an immutable type descriptor by ID. +func (g ByIDGetter) Type(ctx context.Context, id descpb.ID) (catalog.TypeDescriptor, error) { + desc, err := g.Desc(ctx, id) + if err != nil { + if errors.Is(err, catalog.ErrDescriptorNotFound) { + return nil, pgerror.Newf( + pgcode.UndefinedObject, "type with ID %d does not exist", id) + } + return nil, err + } + switch t := desc.(type) { + case catalog.TypeDescriptor: + return t, nil + case catalog.TableDescriptor: + // A given table name can resolve to either a type descriptor or a table + // descriptor, because every table descriptor also defines an implicit + // record type with the same name as the table... + if g.flags.isMutable { + // ...except if the type descriptor needs to be mutable. + // We don't have the capability of returning a mutable type + // descriptor for a table's implicit record type. + return nil, errors.Wrapf(ErrMutableTableImplicitType, + "cannot modify table record type %q", t.GetName()) + } + return typedesc.CreateImplicitRecordTypeFromTableDesc(t) + } + return nil, pgerror.Newf( + pgcode.UndefinedObject, "type with ID %d does not exist", id) +} + +// Function looks up an immutable function descriptor by ID. +func (g ByIDGetter) Function( + ctx context.Context, id descpb.ID, +) (catalog.FunctionDescriptor, error) { + desc, err := g.Desc(ctx, id) + if err != nil { + if errors.Is(err, catalog.ErrDescriptorNotFound) { + return nil, errors.Wrapf(tree.ErrFunctionUndefined, "function %d does not exist", id) + } + return nil, err + } + fn, ok := desc.(catalog.FunctionDescriptor) + if !ok { + return nil, errors.Wrapf(tree.ErrFunctionUndefined, "function %d does not exist", id) + } + return fn, nil +} + +// MutableByIDGetter looks up mutable descriptors by ID. +type MutableByIDGetter getterBase + +// AsByIDGetter returns this object as a ByIDGetter, which performs in +// exactly the same way except for the return types. +func (g MutableByIDGetter) AsByIDGetter() ByIDGetter { + return ByIDGetter(g) +} + +// Descs looks up mutable descriptors by ID. +func (g MutableByIDGetter) Descs( + ctx context.Context, ids []descpb.ID, +) ([]catalog.MutableDescriptor, error) { + descs, err := g.AsByIDGetter().Descs(ctx, ids) + if err != nil { + return nil, err + } + ret := make([]catalog.MutableDescriptor, len(descs)) + for i, desc := range descs { + ret[i] = desc.(catalog.MutableDescriptor) + } + return ret, err +} + +// Desc looks up a mutable descriptor by ID. +func (g MutableByIDGetter) Desc( + ctx context.Context, id descpb.ID, +) (catalog.MutableDescriptor, error) { + desc, err := g.AsByIDGetter().Desc(ctx, id) + if err != nil { + return nil, err + } + return desc.(catalog.MutableDescriptor), nil +} + +// Database looks up a mutable database descriptor by ID. +func (g MutableByIDGetter) Database(ctx context.Context, id descpb.ID) (*dbdesc.Mutable, error) { + db, err := g.AsByIDGetter().Database(ctx, id) + if err != nil { + return nil, err + } + return db.(*dbdesc.Mutable), nil +} + +// Schema looks up a mutable schema descriptor by ID. +func (g MutableByIDGetter) Schema(ctx context.Context, id descpb.ID) (*schemadesc.Mutable, error) { + sc, err := g.AsByIDGetter().Schema(ctx, id) + if err != nil { + return nil, err + } + return sc.(*schemadesc.Mutable), nil +} + +// Table looks up a mutable table descriptor by ID. +func (g MutableByIDGetter) Table(ctx context.Context, id descpb.ID) (*tabledesc.Mutable, error) { + tbl, err := g.AsByIDGetter().Table(ctx, id) + if err != nil { + return nil, err + } + return tbl.(*tabledesc.Mutable), nil +} + +// Type looks up a mutable type descriptor by ID. +func (g MutableByIDGetter) Type(ctx context.Context, id descpb.ID) (*typedesc.Mutable, error) { + typ, err := g.AsByIDGetter().Type(ctx, id) + if err != nil { + return nil, err + } + return typ.(*typedesc.Mutable), nil +} + +// Function looks up a mutable function descriptor by ID. +func (g MutableByIDGetter) Function(ctx context.Context, id descpb.ID) (*funcdesc.Mutable, error) { + fn, err := g.AsByIDGetter().Function(ctx, id) + if err != nil { + return nil, err + } + return fn.(*funcdesc.Mutable), nil +} + +// ByNameGetter looks up immutable descriptors by name. +type ByNameGetter getterBase + +// Database looks up an immutable database descriptor by name. +func (g ByNameGetter) Database( + ctx context.Context, name string, +) (catalog.DatabaseDescriptor, error) { + desc, err := getDescriptorByName( + ctx, g.KV(), g.Descriptors(), nil /* db */, nil /* sc */, name, g.flags, catalog.Database, + ) + if err != nil { + return nil, err + } + if desc == nil { + if g.flags.isOptional { + return nil, nil + } + return nil, sqlerrors.NewUndefinedDatabaseError(name) + } + db, ok := desc.(catalog.DatabaseDescriptor) + if !ok { + if g.flags.isOptional { + return nil, nil + } + return nil, sqlerrors.NewUndefinedDatabaseError(name) + } + return db, nil +} + +// Schema looks up an immutable schema descriptor by name. +func (g ByNameGetter) Schema( + ctx context.Context, db catalog.DatabaseDescriptor, name string, +) (catalog.SchemaDescriptor, error) { + desc, err := getDescriptorByName( + ctx, g.KV(), g.Descriptors(), db, nil /* sc */, name, g.flags, catalog.Schema, + ) + if err != nil { + return nil, err + } + if desc == nil { + if g.flags.isOptional { + return nil, nil + } + return nil, sqlerrors.NewUndefinedSchemaError(name) + } + schema, ok := desc.(catalog.SchemaDescriptor) + if !ok { + if g.flags.isOptional { + return nil, nil + } + return nil, sqlerrors.NewUndefinedSchemaError(name) + } + return schema, nil +} + +// Table looks up an immutable table descriptor by name. +func (g ByNameGetter) Table( + ctx context.Context, db catalog.DatabaseDescriptor, sc catalog.SchemaDescriptor, name string, +) (catalog.TableDescriptor, error) { + desc, err := getDescriptorByName( + ctx, g.KV(), g.Descriptors(), db, sc, name, g.flags, catalog.Table, + ) + if err != nil { + return nil, err + } + if desc == nil { + if g.flags.isOptional { + return nil, nil + } + tn := tree.MakeTableNameWithSchema( + tree.Name(db.GetName()), tree.Name(sc.GetName()), tree.Name(name), + ) + return nil, sqlerrors.NewUndefinedRelationError(&tn) + } + return catalog.AsTableDescriptor(desc) +} + +// Type looks up an immutable type descriptor by name. +func (g ByNameGetter) Type( + ctx context.Context, db catalog.DatabaseDescriptor, sc catalog.SchemaDescriptor, name string, +) (catalog.TypeDescriptor, error) { + desc, err := getDescriptorByName(ctx, g.KV(), g.Descriptors(), db, sc, name, g.flags, catalog.Any) + if err != nil { + return nil, err + } + if desc == nil { + if g.flags.isOptional { + return nil, nil + } + tn := tree.MakeTableNameWithSchema( + tree.Name(db.GetName()), tree.Name(sc.GetName()), tree.Name(name), + ) + return nil, sqlerrors.NewUndefinedRelationError(&tn) + } + if tbl, ok := desc.(catalog.TableDescriptor); ok { + // A given table name can resolve to either a type descriptor or a table + // descriptor, because every table descriptor also defines an implicit + // record type with the same name as the table... + if g.flags.isMutable { + // ...except if the type descriptor needs to be mutable. + // We don't have the capability of returning a mutable type + // descriptor for a table's implicit record type. + return nil, pgerror.Newf(pgcode.InsufficientPrivilege, + "cannot modify table record type %q", name) + } + return typedesc.CreateImplicitRecordTypeFromTableDesc(tbl) + } + return catalog.AsTypeDescriptor(desc) +} + +// MutableByNameGetter looks up mutable descriptors by name. +type MutableByNameGetter getterBase + +// AsByNameGetter returns this object as a ByNameGetter, which performs in +// exactly the same way except for the return types. +func (g MutableByNameGetter) AsByNameGetter() ByNameGetter { + return ByNameGetter(g) +} + +// Database looks up a mutable database descriptor by name. +func (g MutableByNameGetter) Database(ctx context.Context, name string) (*dbdesc.Mutable, error) { + db, err := g.AsByNameGetter().Database(ctx, name) + if err != nil || db == nil { + return nil, err + } + return db.(*dbdesc.Mutable), nil +} + +// Schema looks up a mutable schema descriptor by name. +func (g MutableByNameGetter) Schema( + ctx context.Context, db catalog.DatabaseDescriptor, name string, +) (*schemadesc.Mutable, error) { + sc, err := g.AsByNameGetter().Schema(ctx, db, name) + if err != nil || sc == nil { + return nil, err + } + return sc.(*schemadesc.Mutable), nil +} + +// Table looks up a mutable table descriptor by name. +func (g MutableByNameGetter) Table( + ctx context.Context, db catalog.DatabaseDescriptor, sc catalog.SchemaDescriptor, name string, +) (*tabledesc.Mutable, error) { + tbl, err := g.AsByNameGetter().Table(ctx, db, sc, name) + if err != nil || tbl == nil { + return nil, err + } + return tbl.(*tabledesc.Mutable), nil +} + +// Type looks up a mutable type descriptor by name. +func (g MutableByNameGetter) Type( + ctx context.Context, db catalog.DatabaseDescriptor, sc catalog.SchemaDescriptor, name string, +) (*typedesc.Mutable, error) { + typ, err := g.AsByNameGetter().Type(ctx, db, sc, name) + if err != nil || typ == nil { + return nil, err + } + return typ.(*typedesc.Mutable), nil +} + +func makeGetterBase(txn *kv.Txn, col *Collection, flags getterFlags) getterBase { + return getterBase{ + txn: &txnWrapper{Txn: txn, Collection: col}, + flags: flags, + } +} + +type getterBase struct { + txn + flags getterFlags +} + +type ( + txn interface { + KV() *kv.Txn + Descriptors() *Collection + } + txnWrapper struct { + *kv.Txn + *Collection + } +) + +var _ txn = &txnWrapper{} + +func (w *txnWrapper) KV() *kv.Txn { + return w.Txn +} + +func (w *txnWrapper) Descriptors() *Collection { + return w.Collection +} + +// getterFlags are the flags which power by-ID and by-name descriptor lookups +// in the Collection. The zero value of this struct is not a sane default. +// +// In any case, for historical reasons, some flags get overridden in +// inconsistent and sometimes bizarre ways depending on how the descriptors +// are looked up. +// TODO(postamar): clean up inconsistencies, enforce sane defaults. +type getterFlags struct { + contextFlags + layerFilters layerFilters + descFilters descFilters +} + +type contextFlags struct { + // isOptional specifies that the descriptor is being looked up on + // a best-effort basis. + // + // Presently, for historical reasons, this is overridden to true for + // all mutable by-ID lookups, and for all immutable by-ID object lookups. + // TODO(postamar): clean this up + isOptional bool + // isMutable specifies that a mutable descriptor is to be returned. + isMutable bool +} + +type layerFilters struct { + // withoutSynthetic specifies bypassing the synthetic descriptor layer. + withoutSynthetic bool + // withoutLeased specifies bypassing the leased descriptor layer. + withoutLeased bool + // withoutStorage specifies avoiding any queries to the KV layer. + withoutStorage bool + // withoutHydration specifies avoiding hydrating the descriptors. + // This can be set to true only when looking up descriptors when hydrating + // another group of descriptors. The purpose is to avoid potential infinite + // recursion loop when trying to hydrate a descriptor which would lead to + // the hydration of another descriptor which depends on it. + // TODO(postamar): untangle the hydration mess + withoutHydration bool +} + +type descFilters struct { + // withoutDropped specifies to raise an error if the looked-up descriptor + // is in the DROP state. + // + // Presently, for historical reasons, this is overridden everywhere except + // for immutable by-ID lookups: to true for by-name lookups and to false for + // mutable by-ID lookups. + // TODO(postamar): clean this up + withoutDropped bool + // withoutOffline specifies to raise an error if the looked-up descriptor + // is in the OFFLINE state. + // + // Presently, for historical reasons, this is overridden to true for mutable + // by-ID lookups. + // TODO(postamar): clean this up + withoutOffline bool + // withoutCommittedAdding specifies if committed descriptors in the + // adding state will be ignored. + withoutCommittedAdding bool + // maybeParentID specifies, when set, that the looked-up descriptor + // should have the same parent ID, when set. + maybeParentID descpb.ID +} + +func fromCommonFlags(flags tree.CommonLookupFlags) (f getterFlags) { + return getterFlags{ + contextFlags: contextFlags{ + isOptional: !flags.Required, + isMutable: flags.RequireMutable, + }, + layerFilters: layerFilters{ + withoutSynthetic: flags.AvoidSynthetic, + withoutLeased: flags.AvoidLeased, + }, + descFilters: descFilters{ + withoutDropped: !flags.IncludeDropped, + withoutOffline: !flags.IncludeOffline, + maybeParentID: flags.ParentID, + }, + } +} + +func fromObjectFlags(flags tree.ObjectLookupFlags) getterFlags { + return fromCommonFlags(flags.CommonLookupFlags) +} + +func defaultFlags() getterFlags { + return fromCommonFlags(tree.CommonLookupFlags{}) +} + +func defaultUnleasedFlags() (f getterFlags) { + f.layerFilters.withoutLeased = true + return f +} + +// ByID returns a ByIDGetterBuilder. +func (tc *Collection) ByID(txn *kv.Txn) ByIDGetterBuilder { + return ByIDGetterBuilder(makeGetterBase(txn, tc, defaultFlags())) +} + +// ByIDGetterBuilder is a builder object for ByIDGetter and MutableByIDGetter. +type ByIDGetterBuilder getterBase + +// WithFlags configures the ByIDGetterBuilder with the given flags. +func (b ByIDGetterBuilder) WithFlags(flags tree.CommonLookupFlags) ByIDGetterBuilder { + b.flags = fromCommonFlags(flags) + return b +} + +// WithObjFlags configures the ByIDGetterBuilder with the given object flags. +func (b ByIDGetterBuilder) WithObjFlags(flags tree.ObjectLookupFlags) ByIDGetterBuilder { + b.flags = fromObjectFlags(flags) + return b +} + +// Mutable builds a MutableByIDGetter. +func (b ByIDGetterBuilder) Mutable() MutableByIDGetter { + b.flags.isOptional = false + b.flags.isMutable = true + b.flags.layerFilters.withoutLeased = true + b.flags.descFilters.withoutDropped = false + b.flags.descFilters.withoutOffline = false + return MutableByIDGetter(b) +} + +// Immutable builds a ByIDGetter. +func (b ByIDGetterBuilder) Immutable() ByIDGetter { + if b.flags.isMutable { + b.flags.layerFilters.withoutLeased = true + b.flags.isMutable = false + } + return ByIDGetter(b) +} + +// ByName returns a ByNameGetterBuilder. +func (tc *Collection) ByName(txn *kv.Txn) ByNameGetterBuilder { + return ByNameGetterBuilder(makeGetterBase(txn, tc, defaultFlags())) +} + +// ByNameGetterBuilder is a builder object for ByNameGetter and MutableByNameGetter. +type ByNameGetterBuilder getterBase + +// WithFlags configures the ByIDGetterBuilder with the given flags. +func (b ByNameGetterBuilder) WithFlags(flags tree.CommonLookupFlags) ByNameGetterBuilder { + b.flags = fromCommonFlags(flags) + return b +} + +// WithObjFlags configures the ByNameGetterBuilder with the given object flags. +func (b ByNameGetterBuilder) WithObjFlags(flags tree.ObjectLookupFlags) ByNameGetterBuilder { + b.flags = fromObjectFlags(flags) + return b +} + +// Mutable builds a MutableByNameGetter. +func (b ByNameGetterBuilder) Mutable() MutableByNameGetter { + b.flags.isMutable = true + b.flags.layerFilters.withoutLeased = true + b.flags.descFilters.withoutDropped = true + return MutableByNameGetter(b) +} + +// Immutable builds a ByNameGetter. +func (b ByNameGetterBuilder) Immutable() ByNameGetter { + if b.flags.isMutable { + b.flags.layerFilters.withoutLeased = true + b.flags.isMutable = false + } + b.flags.descFilters.withoutDropped = true + return ByNameGetter(b) +} diff --git a/pkg/sql/catalog/descs/helpers.go b/pkg/sql/catalog/descs/helpers.go index 8da94fe9ada2..d11a04c2d4ee 100644 --- a/pkg/sql/catalog/descs/helpers.go +++ b/pkg/sql/catalog/descs/helpers.go @@ -13,6 +13,7 @@ package descs import ( "context" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -21,6 +22,25 @@ import ( "github.com/cockroachdb/errors" ) +// GetObjectName fetches the full name for the given table or type descriptor. +func GetObjectName( + ctx context.Context, txn *kv.Txn, tc *Collection, obj catalog.Descriptor, +) (tree.ObjectName, error) { + g := tc.ByID(txn).Immutable() + sc, err := g.Schema(ctx, obj.GetParentSchemaID()) + if err != nil { + return nil, err + } + db, err := g.Database(ctx, obj.GetParentID()) + if err != nil { + return nil, err + } + tn := tree.NewTableNameWithSchema( + tree.Name(db.GetName()), tree.Name(sc.GetName()), tree.Name(obj.GetName()), + ) + return tn, nil +} + // GetDescriptorCollidingWithObjectName returns the descriptor which collides // with the desired name if it exists. func GetDescriptorCollidingWithObjectName( @@ -31,12 +51,7 @@ func GetDescriptorCollidingWithObjectName( return nil, err } // At this point the ID is already in use by another object. - flags := tree.CommonLookupFlags{ - AvoidLeased: true, - IncludeOffline: true, - IncludeDropped: true, - } - desc, err := tc.getDescriptorByID(ctx, txn, flags, id) + desc, err := ByIDGetter(makeGetterBase(txn, tc, defaultUnleasedFlags())).Desc(ctx, id) if errors.Is(err, catalog.ErrDescriptorNotFound) { // Since the ID exists the descriptor should absolutely exist. err = errors.NewAssertionErrorWithWrappedErrf(err, @@ -64,3 +79,59 @@ func CheckObjectNameCollision( } return sqlerrors.MakeObjectAlreadyExistsError(d.DescriptorProto(), maybeQualifiedName) } + +func getObjectPrefix( + ctx context.Context, g ByNameGetter, dbName, scName string, +) (prefix catalog.ResolvedObjectPrefix, err error) { + // If we're reading the object descriptor from the store, + // we should read its parents from the store too to ensure + // that subsequent name resolution finds the latest name + // in the face of a concurrent rename. + if dbName != "" { + prefix.Database, err = g.Database(ctx, dbName) + if err != nil || prefix.Database == nil { + return prefix, err + } + } + prefix.Schema, err = g.Schema(ctx, prefix.Database, scName) + return prefix, err +} + +// AsZoneConfigHydrationHelper returns the collection as a +// catalog.ZoneConfigHydrationHelper +func AsZoneConfigHydrationHelper(tc *Collection) catalog.ZoneConfigHydrationHelper { + return &zcHelper{Collection: tc} +} + +type zcHelper struct { + *Collection +} + +var _ catalog.ZoneConfigHydrationHelper = &zcHelper{} + +// MaybeGetTable implements the catalog.ZoneConfigHydrationHelper interface. +func (tc *zcHelper) MaybeGetTable( + ctx context.Context, txn *kv.Txn, id descpb.ID, +) (catalog.TableDescriptor, error) { + // Ignore IDs without a descriptor. + if id == keys.RootNamespaceID || keys.IsPseudoTableID(uint32(id)) { + return nil, nil + } + g := ByIDGetter(makeGetterBase(txn, tc.Collection, defaultUnleasedFlags())) + desc, err := g.Desc(ctx, id) + if err != nil { + return nil, err + } + if desc.DescriptorType() == catalog.Table { + return desc.(catalog.TableDescriptor), nil + } + return nil, nil +} + +// MaybeGetZoneConfig implements the catalog.ZoneConfigHydrationHelper +// interface. +func (tc *zcHelper) MaybeGetZoneConfig( + ctx context.Context, txn *kv.Txn, id descpb.ID, +) (catalog.ZoneConfig, error) { + return tc.GetZoneConfig(ctx, txn, id) +} diff --git a/pkg/sql/catalog/descs/hydrate.go b/pkg/sql/catalog/descs/hydrate.go index cac766eab716..21822c529b85 100644 --- a/pkg/sql/catalog/descs/hydrate.go +++ b/pkg/sql/catalog/descs/hydrate.go @@ -18,9 +18,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/hydrateddesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/intsets" "github.com/cockroachdb/cockroach/pkg/util/tracing" ) @@ -50,9 +50,9 @@ import ( // TODO(ajwerner): Sort out the hydration mess; define clearly what is // hydrated where and test the API boundary accordingly. func (tc *Collection) hydrateDescriptors( - ctx context.Context, txn *kv.Txn, flags tree.CommonLookupFlags, descs []catalog.Descriptor, + ctx context.Context, txn *kv.Txn, flags getterFlags, descs []catalog.Descriptor, ) error { - if flags.SkipHydration { + if flags.layerFilters.withoutHydration { return nil } var hydratableMutableIndexes, hydratableImmutableIndexes intsets.Fast @@ -112,7 +112,7 @@ func (tc *Collection) hydrateDescriptors( func makeMutableTypeLookupFunc( tc *Collection, txn *kv.Txn, descs []catalog.Descriptor, ) typedesc.TypeLookupFunc { - var mut nstree.MutableCatalog + var mc nstree.MutableCatalog for _, desc := range descs { if desc == nil { continue @@ -120,7 +120,7 @@ func makeMutableTypeLookupFunc( if _, ok := desc.(catalog.MutableDescriptor); !ok { continue } - mut.UpsertDescriptor(desc) + mc.UpsertDescriptor(desc) } mutableLookupFunc := func(ctx context.Context, id descpb.ID, skipHydration bool) (catalog.Descriptor, error) { // This special case exists to deal with the desire to use enums in the @@ -131,28 +131,28 @@ func makeMutableTypeLookupFunc( // descriptor for the system database. We only want it for the name, so // let the caller have the immutable copy. if id == catconstants.PublicSchemaID { - return tc.GetImmutableDescriptorByID(ctx, txn, id, tree.CommonLookupFlags{ - Required: true, - AvoidLeased: true, - SkipHydration: skipHydration, - }) + return schemadesc.GetPublicSchema(), nil } - return tc.getDescriptorByID(ctx, txn, tree.CommonLookupFlags{ - Required: true, - AvoidLeased: true, - RequireMutable: true, - IncludeOffline: true, - IncludeDropped: true, - SkipHydration: skipHydration, - }, id) + flags := getterFlags{ + contextFlags: contextFlags{ + isMutable: true, + }, + layerFilters: layerFilters{ + withoutSynthetic: true, + withoutLeased: true, + withoutHydration: skipHydration, + }, + } + g := ByIDGetter(makeGetterBase(txn, tc, flags)) + return g.Desc(ctx, id) } - return hydrateddesc.MakeTypeLookupFuncForHydration(mut, mutableLookupFunc) + return hydrateddesc.MakeTypeLookupFuncForHydration(mc, mutableLookupFunc) } func makeImmutableTypeLookupFunc( - tc *Collection, txn *kv.Txn, flags tree.CommonLookupFlags, descs []catalog.Descriptor, + tc *Collection, txn *kv.Txn, flags getterFlags, descs []catalog.Descriptor, ) typedesc.TypeLookupFunc { - var imm nstree.MutableCatalog + var mc nstree.MutableCatalog for _, desc := range descs { if desc == nil { continue @@ -160,18 +160,24 @@ func makeImmutableTypeLookupFunc( if _, ok := desc.(catalog.MutableDescriptor); ok { continue } - imm.UpsertDescriptor(desc) + mc.UpsertDescriptor(desc) } immutableLookupFunc := func(ctx context.Context, id descpb.ID, skipHydration bool) (catalog.Descriptor, error) { - return tc.GetImmutableDescriptorByID(ctx, txn, id, tree.CommonLookupFlags{ - Required: true, - AvoidLeased: flags.AvoidLeased, - IncludeOffline: flags.IncludeOffline, - AvoidSynthetic: true, - SkipHydration: skipHydration, - }) + f := getterFlags{ + layerFilters: layerFilters{ + withoutSynthetic: true, + withoutLeased: flags.layerFilters.withoutLeased, + withoutHydration: skipHydration, + }, + descFilters: descFilters{ + withoutDropped: true, + withoutOffline: flags.descFilters.withoutOffline, + }, + } + g := ByIDGetter(makeGetterBase(txn, tc, f)) + return g.Desc(ctx, id) } - return hydrateddesc.MakeTypeLookupFuncForHydration(imm, immutableLookupFunc) + return hydrateddesc.MakeTypeLookupFuncForHydration(mc, immutableLookupFunc) } // HydrateCatalog installs type metadata in the type.T objects present for all diff --git a/pkg/sql/catalog/descs/object.go b/pkg/sql/catalog/descs/object.go deleted file mode 100644 index 28efb171ade3..000000000000 --- a/pkg/sql/catalog/descs/object.go +++ /dev/null @@ -1,146 +0,0 @@ -// 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 descs - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" - "github.com/cockroachdb/errors" -) - -// GetObjectByName looks up an object by name and returns both its -// descriptor and that of its parent database and schema. -// -// If the object is not found and flags.required is true, an error is returned, -// otherwise a nil reference is returned. -func (tc *Collection) GetObjectByName( - ctx context.Context, - txn *kv.Txn, - catalogName, schemaName, objectName string, - flags tree.ObjectLookupFlags, -) (prefix catalog.ResolvedObjectPrefix, desc catalog.Descriptor, err error) { - prefix, err = tc.getObjectPrefixByName(ctx, txn, catalogName, schemaName, flags) - if err != nil || prefix.Schema == nil { - return prefix, nil, err - } - // Read object descriptor and handle errors and absence. - { - var requestedType catalog.DescriptorType - switch flags.DesiredObjectKind { - case tree.TableObject: - requestedType = catalog.Table - case tree.TypeObject: - requestedType = catalog.Type - default: - return prefix, nil, errors.AssertionFailedf( - "unknown DesiredObjectKind value %v", flags.DesiredObjectKind) - } - desc, err = tc.getDescriptorByName( - ctx, txn, prefix.Database, prefix.Schema, objectName, flags.CommonLookupFlags, requestedType, - ) - if err != nil { - return prefix, nil, err - } - if desc == nil { - if flags.Required { - tn := tree.MakeTableNameWithSchema( - tree.Name(catalogName), - tree.Name(schemaName), - tree.Name(objectName)) - return prefix, nil, sqlerrors.NewUndefinedRelationError(&tn) - } - return prefix, nil, nil - } - } - // At this point the descriptor is not nil. - switch t := desc.(type) { - case catalog.TableDescriptor: - // A given table name can resolve to either a type descriptor or a table - // descriptor, because every table descriptor also defines an implicit - // record type with the same name as the table. Thus, depending on the - // requested descriptor type, we return either the table descriptor itself, - // or the table descriptor's implicit record type. - switch flags.DesiredObjectKind { - case tree.TableObject, tree.TypeObject: - default: - return prefix, nil, nil - } - if flags.DesiredObjectKind == tree.TypeObject { - // Since a type descriptor was requested, we need to return the implicitly - // created record type for the table that we found. - if flags.RequireMutable { - // ... but, we can't do it if we need a mutable descriptor - we don't - // have the capability of returning a mutable type descriptor for a - // table's implicit record type. - return prefix, nil, pgerror.Newf(pgcode.InsufficientPrivilege, - "cannot modify table record type %q", objectName) - } - desc, err = typedesc.CreateImplicitRecordTypeFromTableDesc(t) - if err != nil { - return prefix, nil, err - } - } - case catalog.TypeDescriptor: - if flags.DesiredObjectKind != tree.TypeObject { - return prefix, nil, nil - } - default: - return prefix, nil, errors.AssertionFailedf( - "unexpected object of type %T", t, - ) - } - return prefix, desc, nil -} - -func (tc *Collection) getObjectPrefixByName( - ctx context.Context, txn *kv.Txn, catalogName, schemaName string, objFlags tree.ObjectLookupFlags, -) (prefix catalog.ResolvedObjectPrefix, err error) { - // If we're reading the object descriptor from the store, - // we should read its parents from the store too to ensure - // that subsequent name resolution finds the latest name - // in the face of a concurrent rename. - flags := tree.CommonLookupFlags{ - Required: objFlags.Required, - AvoidLeased: objFlags.AvoidLeased || objFlags.RequireMutable, - IncludeDropped: objFlags.IncludeDropped, - IncludeOffline: objFlags.IncludeOffline, - } - if catalogName != "" { - prefix.Database, err = tc.GetImmutableDatabaseByName(ctx, txn, catalogName, flags) - if err != nil { - return prefix, err - } - if prefix.Database == nil { - if flags.Required { - return prefix, sqlerrors.NewUndefinedDatabaseError(catalogName) - } - return prefix, nil - } - } - prefix.Schema, err = tc.GetImmutableSchemaByName(ctx, txn, prefix.Database, schemaName, flags) - if err != nil { - return prefix, err - } - if prefix.Schema == nil { - if flags.Required { - return prefix, sqlerrors.NewUndefinedSchemaError(schemaName) - } - return prefix, nil - } - return prefix, nil -} diff --git a/pkg/sql/catalog/descs/schema.go b/pkg/sql/catalog/descs/schema.go index e49b2c0bc83d..29b14ece45b6 100644 --- a/pkg/sql/catalog/descs/schema.go +++ b/pkg/sql/catalog/descs/schema.go @@ -12,7 +12,6 @@ package descs import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" @@ -20,8 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" - "github.com/cockroachdb/errors" ) // GetMutableSchemaByName resolves the schema and returns a mutable descriptor @@ -37,12 +34,7 @@ func (tc *Collection) GetMutableSchemaByName( schemaName string, flags tree.SchemaLookupFlags, ) (*schemadesc.Mutable, error) { - flags.RequireMutable = true - sc, err := tc.getSchemaByName(ctx, txn, db, schemaName, flags) - if err != nil || sc == nil { - return nil, err - } - return sc.(*schemadesc.Mutable), nil + return tc.ByName(txn).WithFlags(flags).Mutable().Schema(ctx, db, schemaName) } // GetImmutableSchemaByName returns a catalog.SchemaDescriptor object if the @@ -58,37 +50,7 @@ func (tc *Collection) GetImmutableSchemaByName( scName string, flags tree.SchemaLookupFlags, ) (catalog.SchemaDescriptor, error) { - flags.RequireMutable = false - return tc.getSchemaByName(ctx, txn, db, 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, - db catalog.DatabaseDescriptor, - schemaName string, - flags tree.SchemaLookupFlags, -) (catalog.SchemaDescriptor, error) { - desc, err := tc.getDescriptorByName(ctx, txn, db, nil /* sc */, schemaName, flags, catalog.Schema) - if err != nil { - return nil, err - } - if desc == nil { - if flags.Required { - return nil, sqlerrors.NewUndefinedSchemaError(schemaName) - } - return nil, nil - } - schema, ok := desc.(catalog.SchemaDescriptor) - if !ok { - if flags.Required { - return nil, sqlerrors.NewUndefinedSchemaError(schemaName) - } - return nil, nil - } - return schema, nil + return tc.ByName(txn).WithFlags(flags).Immutable().Schema(ctx, db, scName) } // GetImmutableSchemaByID returns a ResolvedSchema wrapping an immutable @@ -98,8 +60,7 @@ func (tc *Collection) getSchemaByName( func (tc *Collection) GetImmutableSchemaByID( ctx context.Context, txn *kv.Txn, schemaID descpb.ID, flags tree.SchemaLookupFlags, ) (catalog.SchemaDescriptor, error) { - flags.RequireMutable = false - return tc.getSchemaByID(ctx, txn, schemaID, flags) + return tc.ByID(txn).WithFlags(flags).Immutable().Schema(ctx, schemaID) } // GetMutableSchemaByID returns a mutable schema descriptor with the given @@ -107,32 +68,7 @@ func (tc *Collection) GetImmutableSchemaByID( func (tc *Collection) GetMutableSchemaByID( ctx context.Context, txn *kv.Txn, schemaID descpb.ID, flags tree.SchemaLookupFlags, ) (*schemadesc.Mutable, error) { - flags.RequireMutable = true - desc, err := tc.getSchemaByID(ctx, txn, schemaID, flags) - if err != nil { - return nil, err - } - return desc.(*schemadesc.Mutable), nil -} - -func (tc *Collection) getSchemaByID( - ctx context.Context, txn *kv.Txn, schemaID descpb.ID, flags tree.SchemaLookupFlags, -) (catalog.SchemaDescriptor, error) { - descs, err := tc.getDescriptorsByID(ctx, txn, flags, schemaID) - if err != nil { - if errors.Is(err, catalog.ErrDescriptorNotFound) { - if flags.Required { - return nil, sqlerrors.NewUndefinedSchemaError(fmt.Sprintf("[%d]", schemaID)) - } - return nil, nil - } - return nil, err - } - schemaDesc, ok := descs[0].(catalog.SchemaDescriptor) - if !ok { - return nil, sqlerrors.NewUndefinedSchemaError(fmt.Sprintf("[%d]", schemaID)) - } - return schemaDesc, nil + return tc.ByID(txn).WithFlags(flags).Mutable().Schema(ctx, schemaID) } // InsertDescriptorlessPublicSchemaToBatch adds the creation of a new descriptorless public diff --git a/pkg/sql/catalog/descs/table.go b/pkg/sql/catalog/descs/table.go index 8efc50248fdf..a9f04b2ed51a 100644 --- a/pkg/sql/catalog/descs/table.go +++ b/pkg/sql/catalog/descs/table.go @@ -18,44 +18,34 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" - "github.com/cockroachdb/errors" ) // GetMutableTableByName returns a mutable table descriptor with properties // according to the provided lookup flags. RequireMutable is ignored. func (tc *Collection) GetMutableTableByName( - ctx context.Context, txn *kv.Txn, name tree.ObjectName, flags tree.ObjectLookupFlags, + ctx context.Context, txn *kv.Txn, name tree.ObjectName, objectFlags tree.ObjectLookupFlags, ) (found bool, _ *tabledesc.Mutable, _ error) { - flags.RequireMutable = true - found, desc, err := tc.getTableByName(ctx, txn, name, flags) - if err != nil || !found { + b := tc.ByName(txn).WithObjFlags(objectFlags) + p, err := getObjectPrefix(ctx, b.Immutable(), name.Catalog(), name.Schema()) + if err != nil || p.Schema == nil { return false, nil, err } - return true, desc.(*tabledesc.Mutable), nil + tbl, err := b.Mutable().Table(ctx, p.Database, p.Schema, name.Object()) + return tbl != nil, tbl, err } // GetImmutableTableByName returns a immutable table descriptor with properties // according to the provided lookup flags. RequireMutable is ignored. func (tc *Collection) GetImmutableTableByName( - ctx context.Context, txn *kv.Txn, name tree.ObjectName, flags tree.ObjectLookupFlags, + ctx context.Context, txn *kv.Txn, name tree.ObjectName, objectFlags tree.ObjectLookupFlags, ) (found bool, _ catalog.TableDescriptor, _ error) { - flags.RequireMutable = false - return tc.getTableByName(ctx, txn, name, flags) -} - -// getTableByName returns a table descriptor with properties according to the -// provided lookup flags. -func (tc *Collection) getTableByName( - ctx context.Context, txn *kv.Txn, name tree.ObjectName, flags tree.ObjectLookupFlags, -) (found bool, _ catalog.TableDescriptor, err error) { - flags.DesiredObjectKind = tree.TableObject - _, desc, err := tc.GetObjectByName( - ctx, txn, name.Catalog(), name.Schema(), name.Object(), flags) - if err != nil || desc == nil { + g := tc.ByName(txn).WithObjFlags(objectFlags).Immutable() + p, err := getObjectPrefix(ctx, g, name.Catalog(), name.Schema()) + if err != nil || p.Schema == nil { return false, nil, err } - return true, desc.(catalog.TableDescriptor), nil + tbl, err := g.Table(ctx, p.Database, p.Schema, name.Object()) + return tbl != nil, tbl, err } // GetLeasedImmutableTableByID returns a leased immutable table descriptor by @@ -68,7 +58,7 @@ func (tc *Collection) GetLeasedImmutableTableByID( return nil, err } descs := []catalog.Descriptor{desc} - err = tc.hydrateDescriptors(ctx, txn, tree.CommonLookupFlags{}, descs) + err = tc.hydrateDescriptors(ctx, txn, defaultFlags(), descs) if err != nil { return nil, err } @@ -104,28 +94,17 @@ func (tc *Collection) GetUncommittedMutableTableByID( func (tc *Collection) GetMutableTableByID( ctx context.Context, txn *kv.Txn, tableID descpb.ID, flags tree.ObjectLookupFlags, ) (*tabledesc.Mutable, error) { - flags.RequireMutable = true - desc, err := tc.getTableByID(ctx, txn, tableID, flags) - if err != nil { - return nil, err - } - return desc.(*tabledesc.Mutable), nil + return tc.ByID(txn).WithObjFlags(flags).Mutable().Table(ctx, tableID) } // GetMutableTableVersionByID is a variant of sqlbase.getTableDescFromID which returns a mutable // table descriptor of the table modified in the same transaction. -// Deprecated in favor of GetMutableTableByID. // TODO (lucy): Usages should be replaced with GetMutableTableByID, but this // needs a careful look at what flags should be passed in at each call site. func (tc *Collection) GetMutableTableVersionByID( ctx context.Context, tableID descpb.ID, txn *kv.Txn, ) (*tabledesc.Mutable, error) { - return tc.GetMutableTableByID(ctx, txn, tableID, tree.ObjectLookupFlags{ - CommonLookupFlags: tree.CommonLookupFlags{ - IncludeOffline: true, - IncludeDropped: true, - }, - }) + return tc.ByID(txn).Mutable().Table(ctx, tableID) } // GetImmutableTableByID returns an immutable table descriptor with @@ -135,29 +114,5 @@ func (tc *Collection) GetMutableTableVersionByID( func (tc *Collection) GetImmutableTableByID( ctx context.Context, txn *kv.Txn, tableID descpb.ID, flags tree.ObjectLookupFlags, ) (catalog.TableDescriptor, error) { - flags.RequireMutable = false - desc, err := tc.getTableByID(ctx, txn, tableID, flags) - if err != nil { - return nil, err - } - return desc, nil -} - -func (tc *Collection) getTableByID( - ctx context.Context, txn *kv.Txn, tableID descpb.ID, flags tree.ObjectLookupFlags, -) (catalog.TableDescriptor, error) { - descs, err := tc.getDescriptorsByID(ctx, txn, flags.CommonLookupFlags, tableID) - if err != nil { - if errors.Is(err, catalog.ErrDescriptorNotFound) { - return nil, sqlerrors.NewUndefinedRelationError( - &tree.TableRef{TableID: int64(tableID)}) - } - return nil, err - } - table, ok := descs[0].(catalog.TableDescriptor) - if !ok { - return nil, sqlerrors.NewUndefinedRelationError( - &tree.TableRef{TableID: int64(tableID)}) - } - return table, nil + return tc.ByID(txn).WithObjFlags(flags).Immutable().Table(ctx, tableID) } diff --git a/pkg/sql/catalog/descs/table_name.go b/pkg/sql/catalog/descs/table_name.go deleted file mode 100644 index 84d45e1ce34c..000000000000 --- a/pkg/sql/catalog/descs/table_name.go +++ /dev/null @@ -1,50 +0,0 @@ -// Copyright 2022 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 descs - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/errors" -) - -// GetTableNameByID fetches the full tree table name by the given ID. -func GetTableNameByID( - ctx context.Context, txn *kv.Txn, tc *Collection, tableID descpb.ID, -) (*tree.TableName, error) { - tbl, err := tc.GetImmutableTableByID(ctx, txn, tableID, tree.ObjectLookupFlagsWithRequired()) - if err != nil { - return nil, err - } - return GetTableNameByDesc(ctx, txn, tc, tbl) -} - -// GetTableNameByDesc fetches the full tree table name by the given table descriptor. -func GetTableNameByDesc( - ctx context.Context, txn *kv.Txn, tc *Collection, tbl catalog.TableDescriptor, -) (*tree.TableName, error) { - sc, err := tc.GetImmutableSchemaByID(ctx, txn, tbl.GetParentSchemaID(), tree.SchemaLookupFlags{Required: true}) - if err != nil { - return nil, err - } - found, db, err := tc.GetImmutableDatabaseByID(ctx, txn, tbl.GetParentID(), tree.DatabaseLookupFlags{Required: true}) - if err != nil { - return nil, err - } - if !found { - return nil, errors.AssertionFailedf("expected database %d to exist", tbl.GetParentID()) - } - return tree.NewTableNameWithSchema(tree.Name(db.GetName()), tree.Name(sc.GetName()), tree.Name(tbl.GetName())), nil -} diff --git a/pkg/sql/catalog/descs/type.go b/pkg/sql/catalog/descs/type.go index bd9eb106a5e6..86ac02040ed6 100644 --- a/pkg/sql/catalog/descs/type.go +++ b/pkg/sql/catalog/descs/type.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/errors" ) // ErrMutableTableImplicitType indicates that a table implicit type was fetched @@ -30,53 +29,39 @@ var ErrMutableTableImplicitType = pgerror.Newf(pgcode.DependentObjectsStillExist // GetMutableTypeByName returns a mutable type descriptor with properties // according to the provided lookup flags. RequireMutable is ignored. func (tc *Collection) GetMutableTypeByName( - ctx context.Context, txn *kv.Txn, name tree.ObjectName, flags tree.ObjectLookupFlags, + ctx context.Context, txn *kv.Txn, name tree.ObjectName, objectFlags tree.ObjectLookupFlags, ) (found bool, _ *typedesc.Mutable, _ error) { - flags.RequireMutable = true - found, desc, err := tc.getTypeByName(ctx, txn, name, flags) - if err != nil || !found { + b := tc.ByName(txn).WithObjFlags(objectFlags) + p, err := getObjectPrefix(ctx, b.Immutable(), name.Catalog(), name.Schema()) + if err != nil || p.Schema == nil { return false, nil, err } - return true, desc.(*typedesc.Mutable), nil + typ, err := b.Mutable().Type(ctx, p.Database, p.Schema, name.Object()) + return typ != nil, typ, err } // GetImmutableTypeByName returns a mutable type descriptor with properties // according to the provided lookup flags. RequireMutable is ignored. func (tc *Collection) GetImmutableTypeByName( - ctx context.Context, txn *kv.Txn, name tree.ObjectName, flags tree.ObjectLookupFlags, + ctx context.Context, txn *kv.Txn, name tree.ObjectName, objectFlags tree.ObjectLookupFlags, ) (found bool, _ catalog.TypeDescriptor, _ error) { - flags.RequireMutable = false - return tc.getTypeByName(ctx, txn, name, flags) -} - -// getTypeByName returns a type descriptor with properties according to the -// provided lookup flags. -func (tc *Collection) getTypeByName( - ctx context.Context, txn *kv.Txn, name tree.ObjectName, flags tree.ObjectLookupFlags, -) (found bool, _ catalog.TypeDescriptor, err error) { - flags.DesiredObjectKind = tree.TypeObject - _, desc, err := tc.GetObjectByName( - ctx, txn, name.Catalog(), name.Schema(), name.Object(), flags) - if err != nil || desc == nil { + g := tc.ByName(txn).WithObjFlags(objectFlags).Immutable() + p, err := getObjectPrefix(ctx, g, name.Catalog(), name.Schema()) + if err != nil || p.Schema == nil { return false, nil, err } - return true, desc.(catalog.TypeDescriptor), nil + typ, err := g.Type(ctx, p.Database, p.Schema, name.Object()) + return typ != nil, typ, err } // GetMutableTypeVersionByID is the equivalent of GetMutableTableDescriptorByID // but for accessing types. -// Deprecated in favor of GetMutableTypeByID. // TODO (lucy): Usages should be replaced with GetMutableTypeByID, but this // needs a careful look at what flags should be passed in at each call site. func (tc *Collection) GetMutableTypeVersionByID( ctx context.Context, txn *kv.Txn, typeID descpb.ID, ) (*typedesc.Mutable, error) { - return tc.GetMutableTypeByID(ctx, txn, typeID, tree.ObjectLookupFlags{ - CommonLookupFlags: tree.CommonLookupFlags{ - IncludeOffline: true, - IncludeDropped: true, - }, - }) + return tc.ByID(txn).Mutable().Type(ctx, typeID) } // GetMutableTypeByID returns a mutable type descriptor with @@ -86,19 +71,7 @@ func (tc *Collection) GetMutableTypeVersionByID( func (tc *Collection) GetMutableTypeByID( ctx context.Context, txn *kv.Txn, typeID descpb.ID, flags tree.ObjectLookupFlags, ) (*typedesc.Mutable, error) { - flags.RequireMutable = true - desc, err := tc.getTypeByID(ctx, txn, typeID, flags) - if err != nil { - return nil, err - } - switch t := desc.(type) { - case *typedesc.Mutable: - return t, nil - case *typedesc.TableImplicitRecordType: - return nil, errors.Wrapf(ErrMutableTableImplicitType, "cannot modify table record type %q", desc.GetName()) - } - return nil, - errors.AssertionFailedf("unhandled type descriptor type %T during GetMutableTypeByID", desc) + return tc.ByID(txn).WithObjFlags(flags).Mutable().Type(ctx, typeID) } // GetImmutableTypeByID returns an immutable type descriptor with @@ -108,28 +81,5 @@ func (tc *Collection) GetMutableTypeByID( func (tc *Collection) GetImmutableTypeByID( ctx context.Context, txn *kv.Txn, typeID descpb.ID, flags tree.ObjectLookupFlags, ) (catalog.TypeDescriptor, error) { - flags.RequireMutable = false - return tc.getTypeByID(ctx, txn, typeID, flags) -} - -func (tc *Collection) getTypeByID( - ctx context.Context, txn *kv.Txn, typeID descpb.ID, flags tree.ObjectLookupFlags, -) (catalog.TypeDescriptor, error) { - descs, err := tc.getDescriptorsByID(ctx, txn, flags.CommonLookupFlags, typeID) - if err != nil { - if errors.Is(err, catalog.ErrDescriptorNotFound) { - return nil, pgerror.Newf( - pgcode.UndefinedObject, "type with ID %d does not exist", typeID) - } - return nil, err - } - switch t := descs[0].(type) { - case catalog.TypeDescriptor: - // User-defined type. - return t, nil - case catalog.TableDescriptor: - return typedesc.CreateImplicitRecordTypeFromTableDesc(t) - } - return nil, pgerror.Newf( - pgcode.UndefinedObject, "type with ID %d does not exist", typeID) + return tc.ByID(txn).WithObjFlags(flags).Immutable().Type(ctx, typeID) } diff --git a/pkg/sql/catalog/descs/virtual_descriptors.go b/pkg/sql/catalog/descs/virtual_descriptors.go index baabee69fc95..5556530d83cf 100644 --- a/pkg/sql/catalog/descs/virtual_descriptors.go +++ b/pkg/sql/catalog/descs/virtual_descriptors.go @@ -11,8 +11,6 @@ package descs import ( - "context" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" @@ -20,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/errors" ) type virtualDescriptors struct { @@ -42,65 +39,46 @@ func (tc virtualDescriptors) getSchemaByName(schemaName string) catalog.SchemaDe } func (tc virtualDescriptors) getObjectByName( - schema string, object string, flags tree.ObjectLookupFlags, -) (isVirtual bool, _ catalog.Descriptor, _ error) { + schema string, object string, requestedKind tree.DesiredObjectKind, +) (virtualSchema catalog.VirtualSchema, virtualObject catalog.VirtualObject, _ error) { if tc.vs == nil { - return false, nil, nil - } - scEntry, ok := tc.vs.GetVirtualSchema(schema) - if !ok { - return false, nil, nil - } - obj, err := scEntry.GetObjectByName(object, flags) - if err != nil { - return true, nil, err + return nil, nil, nil } - if obj == nil { - return true, nil, nil + var found bool + virtualSchema, found = tc.vs.GetVirtualSchema(schema) + if !found { + return nil, nil, nil } - if flags.RequireMutable { - return true, nil, catalog.NewMutableAccessToVirtualObjectError(scEntry, obj) - } - return true, obj.Desc(), nil + obj, err := virtualSchema.GetObjectByName(object, requestedKind) + return virtualSchema, obj, err } -func (tc virtualDescriptors) getByID( - ctx context.Context, id descpb.ID, mutable bool, -) (catalog.Descriptor, error) { +func (tc virtualDescriptors) getObjectByID( + id descpb.ID, +) (catalog.VirtualSchema, catalog.VirtualObject) { if tc.vs == nil { return nil, nil } - if vd, found := tc.vs.GetVirtualObjectByID(id); found { - if mutable { - vs, found := tc.vs.GetVirtualSchemaByID(vd.Desc().GetParentSchemaID()) - if !found { - return nil, errors.AssertionFailedf( - "cannot resolve mutable virtual descriptor %d with unknown parent schema %d", - id, vd.Desc().GetParentSchemaID(), - ) - } - return nil, catalog.NewMutableAccessToVirtualObjectError(vs, vd) - } - return vd.Desc(), nil + vd, found := tc.vs.GetVirtualObjectByID(id) + if !found { + return nil, nil } - return tc.getSchemaByID(ctx, id, mutable) + vs, found := tc.vs.GetVirtualSchemaByID(vd.Desc().GetParentSchemaID()) + if !found { + return nil, vd + } + return vs, vd } -func (tc virtualDescriptors) getSchemaByID( - ctx context.Context, id descpb.ID, mutable bool, -) (catalog.SchemaDescriptor, error) { +func (tc virtualDescriptors) getSchemaByID(id descpb.ID) catalog.VirtualSchema { if tc.vs == nil { - return nil, nil + return nil } vs, found := tc.vs.GetVirtualSchemaByID(id) - switch { - case !found: - return nil, nil - case mutable: - return nil, catalog.NewMutableAccessToVirtualSchemaError(vs.Desc()) - default: - return vs.Desc(), nil + if !found { + return nil } + return vs } func (tc virtualDescriptors) addAllToCatalog(mc nstree.MutableCatalog) { diff --git a/pkg/sql/catalog/errors.go b/pkg/sql/catalog/errors.go index 399418cede3f..d6286e91f0b3 100644 --- a/pkg/sql/catalog/errors.go +++ b/pkg/sql/catalog/errors.go @@ -215,6 +215,10 @@ func NewMutableAccessToVirtualSchemaError(schema SchemaDescriptor) error { // NewMutableAccessToVirtualObjectError is returned when trying to mutably // access a virtual schema object. func NewMutableAccessToVirtualObjectError(schema VirtualSchema, object VirtualObject) error { + if schema == nil { + return errors.AssertionFailedf("virtual object %q (%d) does not have a virtual parent schema", + object.Desc().GetName(), object.Desc().GetID()) + } switch schema.Desc().GetName() { case "pg_catalog": return pgerror.Newf(pgcode.InsufficientPrivilege, diff --git a/pkg/sql/catalog/lease/lease.go b/pkg/sql/catalog/lease/lease.go index a79c1e8dae95..d94a2394e7e6 100644 --- a/pkg/sql/catalog/lease/lease.go +++ b/pkg/sql/catalog/lease/lease.go @@ -818,13 +818,9 @@ func (m *Manager) AcquireByName( // This logic will release the lease (the lease manager will still // cache it), and generate the offline descriptor error. validateDescriptorForReturn := func(desc LeasedDescriptor) (LeasedDescriptor, error) { - if desc.Underlying().Offline() { - if err := catalog.FilterDescriptorState( - desc.Underlying(), tree.CommonLookupFlags{}, - ); err != nil { - desc.Release(ctx) - return nil, err - } + if err := catalog.FilterOfflineDescriptor(desc.Underlying()); err != nil { + desc.Release(ctx) + return nil, err } return desc, nil } diff --git a/pkg/sql/catalog/lease/storage.go b/pkg/sql/catalog/lease/storage.go index 0a4ce3ebaefe..10d4dd7a3298 100644 --- a/pkg/sql/catalog/lease/storage.go +++ b/pkg/sql/catalog/lease/storage.go @@ -153,9 +153,10 @@ func (s storage) acquire( if err != nil { return err } - if err := catalog.FilterDescriptorState( - desc, tree.CommonLookupFlags{IncludeOffline: true}, // filter dropped only - ); err != nil { + if err := catalog.FilterAddingDescriptor(desc); err != nil { + return err + } + if err := catalog.FilterDroppedDescriptor(desc); err != nil { return err } log.VEventf(ctx, 2, "storage attempting to acquire lease %v@%v", desc, expiration) diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index c22b0e042a7f..437031f3408b 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -48,6 +48,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catprivilege" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/multiregion" "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" @@ -4031,8 +4032,9 @@ CREATE TABLE crdb_internal.zones ( // Inherit full information about this zone. fullZone := configProto + zcHelper := descs.AsZoneConfigHydrationHelper(p.Descriptors()) if err := completeZoneConfig( - ctx, &fullZone, p.Txn(), p.Descriptors(), descpb.ID(tree.MustBeDInt(r[0])), + ctx, &fullZone, p.Txn(), zcHelper, descpb.ID(tree.MustBeDInt(r[0])), ); err != nil { return err } diff --git a/pkg/sql/drop_cascade.go b/pkg/sql/drop_cascade.go index 929a78a66777..a23c47529265 100644 --- a/pkg/sql/drop_cascade.go +++ b/pkg/sql/drop_cascade.go @@ -120,7 +120,7 @@ func (d *dropCascadeState) resolveCollectedObjects(ctx context.Context, p *plann tbDesc, ok := desc.(*tabledesc.Mutable) if !ok { return errors.AssertionFailedf( - "descriptor for %q is not Mutable", + "table descriptor for %q is not Mutable", objName.Object(), ) } @@ -178,7 +178,7 @@ func (d *dropCascadeState) resolveCollectedObjects(ctx context.Context, p *plann typDesc, ok := desc.(*typedesc.Mutable) if !ok { return errors.AssertionFailedf( - "descriptor for %q is not Mutable", + "type descriptor for %q is not Mutable", objName.Object(), ) } diff --git a/pkg/sql/importer/BUILD.bazel b/pkg/sql/importer/BUILD.bazel index 05d306449c78..abc73eda1bca 100644 --- a/pkg/sql/importer/BUILD.bazel +++ b/pkg/sql/importer/BUILD.bazel @@ -199,6 +199,7 @@ go_test( "//pkg/sql/gcjob", "//pkg/sql/parser", "//pkg/sql/pgwire/pgcode", + "//pkg/sql/pgwire/pgerror", "//pkg/sql/randgen", "//pkg/sql/row", "//pkg/sql/rowenc", diff --git a/pkg/sql/importer/import_stmt_test.go b/pkg/sql/importer/import_stmt_test.go index 557d78043fe6..737f0255e645 100644 --- a/pkg/sql/importer/import_stmt_test.go +++ b/pkg/sql/importer/import_stmt_test.go @@ -53,6 +53,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/gcjob" "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/row" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -6416,6 +6418,9 @@ func TestImportPgDumpSchemas(t *testing.T) { IncludeDropped: true, IncludeOffline: true, }) + if pgerror.GetPGCode(err) == pgcode.InvalidSchemaName { + return nil + } if !testutils.IsError(err, "descriptor not found") { return err } diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index 0787f3f838fa..21442ea3685c 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -3560,7 +3560,13 @@ https://www.postgresql.org/docs/13/catalog-pg-statistic-ext.html`, h.writeUInt64(uint64(statisticsID)) statisticsOID := h.getOid() - tn, err := descs.GetTableNameByID(ctx, p.Txn(), p.descCollection, descpb.ID(tableID)) + tbl, err := p.Descriptors().GetImmutableTableByID( + ctx, p.Txn(), descpb.ID(tableID), tree.ObjectLookupFlagsWithRequired(), + ) + if err != nil { + return err + } + tn, err := descs.GetObjectName(ctx, p.Txn(), p.Descriptors(), tbl) if err != nil { return err } diff --git a/pkg/sql/schema_resolver.go b/pkg/sql/schema_resolver.go index 5e3e455f917f..1f7596c40208 100644 --- a/pkg/sql/schema_resolver.go +++ b/pkg/sql/schema_resolver.go @@ -41,7 +41,7 @@ import ( var _ resolver.SchemaResolver = &schemaResolver{} -// schemaResolve implements the resolver.SchemaResolver interface. +// schemaResolver implements the resolver.SchemaResolver interface. // Currently, this is only being embedded in the planner but also a convenience // for inejcting it into the declarative schema changer. // It holds sessionDataStack and a transaction handle which are reset when @@ -150,10 +150,47 @@ func (sr *schemaResolver) LookupObject( } } - prefix, objMeta, err = sr.descCollection.GetObjectByName(ctx, sr.txn, dbName, scName, obName, flags) + prefix, objMeta, err = sr.GetObjectByName( + ctx, dbName, scName, obName, flags, + ) return objMeta != nil, prefix, objMeta, err } +// GetObjectByName returns an object descriptor by name. +func (sr *schemaResolver) GetObjectByName( + ctx context.Context, catalogName, schemaName, objectName string, flags tree.ObjectLookupFlags, +) (prefix catalog.ResolvedObjectPrefix, desc catalog.Descriptor, err error) { + b := sr.descCollection.ByName(sr.txn).WithObjFlags(flags) + g := b.Immutable() + if catalogName != "" { + prefix.Database, err = g.Database(ctx, catalogName) + if err != nil || prefix.Database == nil { + return prefix, nil, err + } + } + prefix.Schema, err = g.Schema(ctx, prefix.Database, schemaName) + if err != nil || prefix.Schema == nil { + return prefix, nil, err + } + if flags.RequireMutable { + g = b.Mutable().AsByNameGetter() + } + switch flags.DesiredObjectKind { + case tree.TableObject: + desc, err = g.Table(ctx, prefix.Database, prefix.Schema, objectName) + case tree.TypeObject: + desc, err = g.Type(ctx, prefix.Database, prefix.Schema, objectName) + default: + return prefix, nil, errors.AssertionFailedf( + "unknown desired object kind %v", flags.DesiredObjectKind, + ) + } + if errors.Is(err, catalog.ErrDescriptorWrongType) && !flags.Required { + return prefix, nil, nil + } + return prefix, desc, err +} + // LookupSchema implements the resolver.ObjectNameTargetResolver interface. func (sr *schemaResolver) LookupSchema( ctx context.Context, dbName, scName string, @@ -560,7 +597,6 @@ func (sr *schemaResolver) ResolveFunctionByOID( } flags := sr.CommonLookupFlagsRequired() - flags.AvoidLeased = sr.skipDescriptorCache flags.ParentID = sr.typeResolutionDbID descID, err := funcdesc.UserDefinedFunctionOIDToID(oid) if err != nil { diff --git a/pkg/sql/scrub.go b/pkg/sql/scrub.go index 0f9e4fc16b6c..1ec87ff0ce3d 100644 --- a/pkg/sql/scrub.go +++ b/pkg/sql/scrub.go @@ -180,16 +180,19 @@ func (n *scrubNode) startScrubDatabase(ctx context.Context, p *planner, name *tr for i := range tbNames { tableName := &tbNames[i] - _, objDesc, err := p.descCollection.GetObjectByName( - ctx, p.txn, tableName.Catalog(), tableName.Schema(), tableName.Table(), + _, objDesc, err := p.GetObjectByName( + ctx, tableName.Catalog(), tableName.Schema(), tableName.Table(), p.ObjectLookupFlags(true /*required*/, false /*requireMutable*/), ) - if err != nil { - return err - } // Skip over descriptors that are not tables (like types). // Note: We are asking for table objects above, so It's valid to only // get a prefix, and no descriptor. + if errors.Is(err, catalog.ErrDescriptorWrongType) { + continue + } + if err != nil { + return err + } if objDesc == nil || objDesc.DescriptorType() != catalog.Table { continue } diff --git a/pkg/sql/sem/tree/name_resolution.go b/pkg/sql/sem/tree/name_resolution.go index 5699852ac5ca..2420de8bba37 100644 --- a/pkg/sql/sem/tree/name_resolution.go +++ b/pkg/sql/sem/tree/name_resolution.go @@ -158,24 +158,20 @@ func newInvTableNameError(n fmt.Stringer) error { // CommonLookupFlags is the common set of flags for the various accessor interfaces. type CommonLookupFlags struct { - // if required is set, lookup will return an error if the item is not found. + // Required specifies that the lookup will return an error if the item is + // not found. Required bool // RequireMutable specifies whether to return a mutable descriptor. RequireMutable bool // AvoidLeased, if set, avoid the leased (possibly stale) version of the // descriptor. It must be set when callers want consistent reads. AvoidLeased bool - // AvoidCommittedAdding specifies if committed descriptors in the adding state - // will be ignored. - AvoidCommittedAdding bool // IncludeOffline specifies if offline descriptors should be visible. IncludeOffline bool // IncludeOffline specifies if dropped descriptors should be visible. IncludeDropped bool // AvoidSynthetic specifies if the synthetic descriptors will be ignored. AvoidSynthetic bool - // AvoidStorage specifies if the descriptors in storage will be ignored. - AvoidStorage bool // ParentID enforces that the resolved descriptor exist with this parent // ID if non-zero. ParentID catid.DescID diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go index 6e060f4108da..33162dc3ee3d 100644 --- a/pkg/sql/set_zone_config.go +++ b/pkg/sql/set_zone_config.go @@ -592,13 +592,14 @@ func (n *setZoneConfigNode) startExec(params runParams) error { // We need to inherit zone configuration information from the correct zone, // not completeZone. { + zcHelper := descs.AsZoneConfigHydrationHelper(params.p.Descriptors()) if index == nil { // If we are operating on a zone, get all fields that the zone would // inherit from its parent. We do this by using an empty zoneConfig // and completing at the level of the current zone. zoneInheritedFields := zonepb.ZoneConfig{} if err := completeZoneConfig( - params.ctx, &zoneInheritedFields, params.p.Txn(), params.p.Descriptors(), targetID, + params.ctx, &zoneInheritedFields, params.p.Txn(), zcHelper, targetID, ); err != nil { return err } @@ -608,7 +609,7 @@ func (n *setZoneConfigNode) startExec(params runParams) error { // unset fields in its parent zone, which is partialZone. zoneInheritedFields := *partialZone if err := completeZoneConfig( - params.ctx, &zoneInheritedFields, params.p.Txn(), params.p.Descriptors(), targetID, + params.ctx, &zoneInheritedFields, params.p.Txn(), zcHelper, targetID, ); err != nil { return err } diff --git a/pkg/sql/ttl/ttljob/ttljob.go b/pkg/sql/ttl/ttljob/ttljob.go index bc86ec6fa419..65685384011c 100644 --- a/pkg/sql/ttl/ttljob/ttljob.go +++ b/pkg/sql/ttl/ttljob/ttljob.go @@ -142,7 +142,7 @@ func (t rowLevelTTLResumer) Resume(ctx context.Context, execCtx interface{}) err return errors.Newf("ttl jobs on table %s are currently paused", tree.Name(desc.GetName())) } - tn, err := descs.GetTableNameByDesc(ctx, txn, descsCol, desc) + tn, err := descs.GetObjectName(ctx, txn, descsCol, desc) if err != nil { return errors.Wrapf(err, "error fetching table relation name for TTL") } diff --git a/pkg/sql/ttl/ttljob/ttljob_processor.go b/pkg/sql/ttl/ttljob/ttljob_processor.go index 15df0351172a..4548bbbe0da1 100644 --- a/pkg/sql/ttl/ttljob/ttljob_processor.go +++ b/pkg/sql/ttl/ttljob/ttljob_processor.go @@ -96,7 +96,7 @@ func (t *ttlProcessor) work(ctx context.Context) error { rowLevelTTL := desc.GetRowLevelTTL() labelMetrics = rowLevelTTL.LabelMetrics - tn, err := descs.GetTableNameByDesc(ctx, txn, descsCol, desc) + tn, err := descs.GetObjectName(ctx, txn, descsCol, desc) if err != nil { return errors.Wrapf(err, "error fetching table relation name for TTL") } diff --git a/pkg/sql/ttl/ttlschedule/ttlschedule.go b/pkg/sql/ttl/ttlschedule/ttlschedule.go index 7dd15c9a631b..2aa58d1f6567 100644 --- a/pkg/sql/ttl/ttlschedule/ttlschedule.go +++ b/pkg/sql/ttl/ttlschedule/ttlschedule.go @@ -71,7 +71,13 @@ func (s rowLevelTTLExecutor) OnDrop( } if !canDrop { - tn, err := descs.GetTableNameByID(ctx, txn, descsCol, args.TableID) + tbl, err := descsCol.GetImmutableTableByID( + ctx, txn, args.TableID, tree.ObjectLookupFlagsWithRequired(), + ) + if err != nil { + return 0, err + } + tn, err := descs.GetObjectName(ctx, txn, descsCol, tbl) if err != nil { return 0, err } @@ -204,14 +210,20 @@ func (s rowLevelTTLExecutor) GetCreateScheduleStatement( if err := pbtypes.UnmarshalAny(sj.ExecutionArgs().Args, args); err != nil { return "", err } - tn, err := descs.GetTableNameByID(ctx, txn, descsCol, args.TableID) + tbl, err := descsCol.GetImmutableTableByID( + ctx, txn, args.TableID, tree.ObjectLookupFlagsWithRequired(), + ) + if err != nil { + return "", err + } + tn, err := descs.GetObjectName(ctx, txn, descsCol, tbl) if err != nil { return "", err } return fmt.Sprintf(`ALTER TABLE %s WITH (ttl = 'on', ...)`, tn.FQString()), nil } -func makeTTLJobDescription(tableDesc catalog.TableDescriptor, tn *tree.TableName) string { +func makeTTLJobDescription(tableDesc catalog.TableDescriptor, tn tree.ObjectName) string { pkColumns := tableDesc.GetPrimaryIndex().IndexDesc().KeyColumnNames pkColumnNamesSQL := ttlbase.MakeColumnNamesSQL(pkColumns) selectQuery := fmt.Sprintf( @@ -250,7 +262,7 @@ func createRowLevelTTLJob( if err != nil { return 0, err } - tn, err := descs.GetTableNameByDesc(ctx, txn, descsCol, tableDesc) + tn, err := descs.GetObjectName(ctx, txn, descsCol, tableDesc) if err != nil { return 0, err } diff --git a/pkg/sql/virtual_schema.go b/pkg/sql/virtual_schema.go index 4e372b9acd89..dc5331eaca0a 100644 --- a/pkg/sql/virtual_schema.go +++ b/pkg/sql/virtual_schema.go @@ -469,9 +469,9 @@ func (v *virtualSchemaEntry) VisitTables(f func(object catalog.VirtualObject)) { } func (v *virtualSchemaEntry) GetObjectByName( - name string, flags tree.ObjectLookupFlags, + name string, kind tree.DesiredObjectKind, ) (catalog.VirtualObject, error) { - switch flags.DesiredObjectKind { + switch kind { case tree.TypeObject: // Currently, we don't allow creation of types in virtual schemas, so // the only types present in the virtual schemas that have types (i.e. @@ -492,8 +492,7 @@ func (v *virtualSchemaEntry) GetObjectByName( typ, ok := tree.GetStaticallyKnownType(typRef) if ok { return &virtualTypeEntry{ - desc: typedesc.MakeSimpleAlias(typ, catconstants.PgCatalogID), - mutable: flags.RequireMutable, + desc: typedesc.MakeSimpleAlias(typ, catconstants.PgCatalogID), }, nil } } @@ -502,20 +501,13 @@ func (v *virtualSchemaEntry) GetObjectByName( fallthrough case tree.TableObject: if def, ok := v.defs[name]; ok { - if flags.RequireMutable { - return &mutableVirtualDefEntry{ - desc: tabledesc.NewBuilder(def.desc.TableDesc()).BuildExistingMutableTable(), - }, nil - } return def, nil } if _, ok := v.undefinedTables[name]; ok { return nil, newUnimplementedVirtualTableError(v.desc.GetName(), name) } - return nil, nil - default: - return nil, errors.AssertionFailedf("unknown desired object kind %d", flags.DesiredObjectKind) } + return nil, nil } type virtualDefEntry struct { @@ -537,22 +529,11 @@ func canQueryVirtualTable(evalCtx *eval.Context, e *virtualDefEntry) bool { evalCtx.SessionData().StubCatalogTablesEnabled } -type mutableVirtualDefEntry struct { - desc *tabledesc.Mutable -} - -func (e *mutableVirtualDefEntry) Desc() catalog.Descriptor { - return e.desc -} - type virtualTypeEntry struct { - desc catalog.TypeDescriptor - mutable bool + desc catalog.TypeDescriptor } func (e *virtualTypeEntry) Desc() catalog.Descriptor { - // TODO(ajwerner): Should this be allowed? I think no. Let's just store an - // ImmutableTypeDesc off of this thing. return e.desc } diff --git a/pkg/sql/zone_config.go b/pkg/sql/zone_config.go index 03cbafcff947..6a05d331eca2 100644 --- a/pkg/sql/zone_config.go +++ b/pkg/sql/zone_config.go @@ -212,13 +212,14 @@ func GetZoneConfigInTxn( partition string, getInheritedDefault bool, ) (descpb.ID, *zonepb.ZoneConfig, *zonepb.Subzone, error) { + zcHelper := descs.AsZoneConfigHydrationHelper(descriptors) zoneID, zone, placeholderID, placeholder, err := getZoneConfig( - ctx, id, txn, descriptors, getInheritedDefault, true, /* mayBeTable */ + ctx, id, txn, zcHelper, getInheritedDefault, true, /* mayBeTable */ ) if err != nil { return 0, nil, nil, err } - if err = completeZoneConfig(ctx, zone, txn, descriptors, zoneID); err != nil { + if err = completeZoneConfig(ctx, zone, txn, zcHelper, zoneID); err != nil { return 0, nil, nil, err } var subzone *zonepb.Subzone @@ -266,13 +267,14 @@ func GetHydratedZoneConfigForNamedZone( if !found { return nil, errors.AssertionFailedf("id %d does not belong to a named zone", id) } + zcHelper := descs.AsZoneConfigHydrationHelper(descriptors) zoneID, zone, _, _, err := getZoneConfig( - ctx, descpb.ID(id), txn, descriptors, false /* getInheritedDefault */, false, /* mayBeTable */ + ctx, descpb.ID(id), txn, zcHelper, false /* getInheritedDefault */, false, /* mayBeTable */ ) if err != nil { return nil, err } - if err := completeZoneConfig(ctx, zone, txn, descriptors, zoneID); err != nil { + if err := completeZoneConfig(ctx, zone, txn, zcHelper, zoneID); err != nil { return nil, err } return zone, nil @@ -283,13 +285,14 @@ func GetHydratedZoneConfigForNamedZone( func GetHydratedZoneConfigForTable( ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, id descpb.ID, ) (*zonepb.ZoneConfig, error) { + zcHelper := descs.AsZoneConfigHydrationHelper(descriptors) zoneID, zone, _, placeholder, err := getZoneConfig( - ctx, id, txn, descriptors, false /* getInheritedDefault */, true, /* mayBeTable */ + ctx, id, txn, zcHelper, false /* getInheritedDefault */, true, /* mayBeTable */ ) if err != nil { return nil, err } - if err := completeZoneConfig(ctx, zone, txn, descriptors, zoneID); err != nil { + if err := completeZoneConfig(ctx, zone, txn, zcHelper, zoneID); err != nil { return nil, err } @@ -339,13 +342,14 @@ func GetHydratedZoneConfigForTable( func GetHydratedZoneConfigForDatabase( ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, id descpb.ID, ) (*zonepb.ZoneConfig, error) { + zcHelper := descs.AsZoneConfigHydrationHelper(descriptors) zoneID, zone, _, _, err := getZoneConfig( - ctx, id, txn, descriptors, false /* getInheritedDefault */, false, /* mayBeTable */ + ctx, id, txn, zcHelper, false /* getInheritedDefault */, false, /* mayBeTable */ ) if err != nil { return nil, err } - if err := completeZoneConfig(ctx, zone, txn, descriptors, zoneID); err != nil { + if err := completeZoneConfig(ctx, zone, txn, zcHelper, zoneID); err != nil { return nil, err }