From ed4d2464a8b8fbf3d75647ee7ee4affa3deeb3d8 Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Mon, 6 Mar 2023 22:40:32 -0800 Subject: [PATCH 1/4] opt: refactor metadata dependency tracking This commit performs some refactoring for the way data source objects are tracked in `opt.Metadata` in order to make future changes to UDT and UDF dependency tracking easier. More particularly, UDTs and UDFs will be able to re-resolve any references by name. This is necessary in order to handle cases where changes to the search-path cause names in the query to resolve to different objects. Release note: None --- pkg/sql/logictest/testdata/logic_test/schema | 78 +++++ pkg/sql/opt/BUILD.bazel | 1 + pkg/sql/opt/metadata.go | 284 ++++++++++++------- pkg/sql/opt/metadata_test.go | 30 +- 4 files changed, 290 insertions(+), 103 deletions(-) diff --git a/pkg/sql/logictest/testdata/logic_test/schema b/pkg/sql/logictest/testdata/logic_test/schema index 9831017fe8b3..44ee2f6face2 100644 --- a/pkg/sql/logictest/testdata/logic_test/schema +++ b/pkg/sql/logictest/testdata/logic_test/schema @@ -911,3 +911,81 @@ DROP SCHEMA sc statement ok SET CLUSTER SETTING server.eventlog.enabled = false + +subtest alter_table_schema + +# Renaming the schema should invalidate a schema-qualified table reference. +statement ok +CREATE SCHEMA sc; +CREATE TABLE sc.xy (x INT, y INT); +INSERT INTO sc.xy VALUES (1, 1); + +query II +SELECT * FROM sc.xy; +---- +1 1 + +statement ok +ALTER SCHEMA sc RENAME TO sc1; + +query error pq: relation "sc.xy" does not exist +SELECT * FROM sc.xy; + +query II +SELECT * FROM sc1.xy; +---- +1 1 + +statement ok +DROP SCHEMA sc1 CASCADE; + +# Renaming the database should invalidate a database-qualified table reference. +statement ok +CREATE DATABASE d; +USE d; +CREATE TABLE d.xy (x INT, y INT); +INSERT INTO d.xy VALUES (1, 1); + +query II +SELECT * FROM d.xy; +---- +1 1 + +statement ok +ALTER DATABASE d RENAME TO d1; +USE d1; + +query error pq: relation "d.xy" does not exist +SELECT * FROM d.xy; + +query II +SELECT * FROM d1.xy; +---- +1 1 + +statement ok +USE test; +DROP DATABASE d1 CASCADE; + +# Changing the current database should invalidate an unqualified table +# reference. +statement ok +CREATE TABLE xy (x INT, y INT); +INSERT INTO xy VALUES (1, 1); + +query II +SELECT * FROM xy; +---- +1 1 + +statement ok +CREATE DATABASE d; +USE d; + +query error pq: relation "xy" does not exist +SELECT * FROM xy; + +statement ok +USE test; +DROP DATABASE d; +DROP TABLE xy; diff --git a/pkg/sql/opt/BUILD.bazel b/pkg/sql/opt/BUILD.bazel index 1e417a36c73c..234ef1a53051 100644 --- a/pkg/sql/opt/BUILD.bazel +++ b/pkg/sql/opt/BUILD.bazel @@ -25,6 +25,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/server/telemetry", + "//pkg/sql/catalog", "//pkg/sql/catalog/catpb", "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go index 6cd302519829..34ecdefee0b7 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -16,6 +16,7 @@ import ( "math/bits" "strings" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/multiregion" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" @@ -78,7 +79,8 @@ type privilegeBitmap uint32 // In this query, `l.x` is not equivalent to `r.x` and `l.y` is not equivalent // to `r.y`. Therefore, we need to give these columns different ids. type Metadata struct { - // schemas stores each schema used in the query, indexed by SchemaID. + // schemas stores each schema used by the query if it is a CREATE statement, + // indexed by SchemaID. schemas []cat.Schema // cols stores information about each metadata column, indexed by @@ -101,15 +103,6 @@ type Metadata struct { userDefinedTypes map[oid.Oid]struct{} userDefinedTypesSlice []*types.T - // deps stores information about all data source objects depended on by the - // query, as well as the privileges required to access them. The objects are - // deduplicated: any name/object pair shows up at most once. - // Note: the same data source object can appear multiple times if different - // names were used. For example, in the query `SELECT * from t, db.t` the two - // tables might resolve to the same object now but to different objects later; - // we want to verify the resolution of both names. - deps []mdDep - // views stores the list of referenced views. This information is only // needed for EXPLAIN (opt, env). views []cat.View @@ -121,33 +114,20 @@ type Metadata struct { // mutation operators, used to determine the logical properties of WithScan. withBindings map[WithID]Expr - // NOTE! When adding fields here, update Init (if reusing allocated - // data structures is desired), CopyFrom and TestMetadata. -} - -type mdDep struct { - ds cat.DataSource + // dataSourceDeps stores each data source object that the query depends on. + dataSourceDeps map[cat.StableID]cat.DataSource - name MDDepName + // objectRefsByName stores each unique name that the query uses to reference + // each object. It is needed because changes to the search path may change + // which object a given name refers to; for example, switching the database. + objectRefsByName map[cat.StableID][]*tree.UnresolvedObjectName - // privileges is the union of all required privileges. - privileges privilegeBitmap -} + // privileges stores the privileges needed to access each object that the + // query depends on. + privileges map[cat.StableID]privilegeBitmap -// MDDepName stores either the unresolved DataSourceName or the StableID from -// the query that was used to resolve a data source. -type MDDepName struct { - // byID is non-zero if and only if the data source was looked up using the - // StableID. - byID cat.StableID - - // byName is non-zero if and only if the data source was looked up using a - // name. - byName cat.DataSourceName -} - -func (n *MDDepName) equals(other *MDDepName) bool { - return n.byID == other.byID && n.byName.Equals(&other.byName) + // NOTE! When adding fields here, update Init (if reusing allocated + // data structures is desired), CopyFrom and TestMetadata. } // Init prepares the metadata for use (or reuse). @@ -174,16 +154,35 @@ func (md *Metadata) Init() { sequences[i] = nil } - deps := md.deps - for i := range deps { - deps[i] = mdDep{} - } - views := md.views for i := range views { views[i] = nil } + dataSourceDeps := md.dataSourceDeps + if dataSourceDeps == nil { + dataSourceDeps = make(map[cat.StableID]cat.DataSource) + } + for id := range md.dataSourceDeps { + delete(md.dataSourceDeps, id) + } + + objectRefsByName := md.objectRefsByName + if objectRefsByName == nil { + objectRefsByName = make(map[cat.StableID][]*tree.UnresolvedObjectName) + } + for id := range md.objectRefsByName { + delete(md.objectRefsByName, id) + } + + privileges := md.privileges + if privileges == nil { + privileges = make(map[cat.StableID]privilegeBitmap) + } + for id := range md.privileges { + delete(md.privileges, id) + } + // This initialization pattern ensures that fields are not unwittingly // reused. Field reuse must be explicit. *md = Metadata{} @@ -191,8 +190,10 @@ func (md *Metadata) Init() { md.cols = cols[:0] md.tables = tables[:0] md.sequences = sequences[:0] - md.deps = deps[:0] md.views = views[:0] + md.dataSourceDeps = dataSourceDeps + md.objectRefsByName = objectRefsByName + md.privileges = privileges } // CopyFrom initializes the metadata with a copy of the provided metadata. @@ -205,8 +206,9 @@ func (md *Metadata) Init() { // expression. func (md *Metadata) CopyFrom(from *Metadata, copyScalarFn func(Expr) Expr) { if len(md.schemas) != 0 || len(md.cols) != 0 || len(md.tables) != 0 || - len(md.sequences) != 0 || len(md.deps) != 0 || len(md.views) != 0 || - len(md.userDefinedTypes) != 0 || len(md.userDefinedTypesSlice) != 0 { + len(md.sequences) != 0 || len(md.views) != 0 || len(md.userDefinedTypes) != 0 || + len(md.userDefinedTypesSlice) != 0 || len(md.dataSourceDeps) != 0 || + len(md.objectRefsByName) != 0 || len(md.privileges) != 0 { panic(errors.AssertionFailedf("CopyFrom requires empty destination")) } md.schemas = append(md.schemas, from.schemas...) @@ -242,8 +244,30 @@ func (md *Metadata) CopyFrom(from *Metadata, copyScalarFn func(Expr) Expr) { } } + for id, dataSource := range from.dataSourceDeps { + if md.dataSourceDeps == nil { + md.dataSourceDeps = make(map[cat.StableID]cat.DataSource) + } + md.dataSourceDeps[id] = dataSource + } + + for id, names := range from.objectRefsByName { + if md.objectRefsByName == nil { + md.objectRefsByName = make(map[cat.StableID][]*tree.UnresolvedObjectName) + } + newNames := make([]*tree.UnresolvedObjectName, len(names)) + copy(newNames, names) + md.objectRefsByName[id] = newNames + } + + for id, privilegeSet := range from.privileges { + if md.privileges == nil { + md.privileges = make(map[cat.StableID]privilegeBitmap) + } + md.privileges[id] = privilegeSet + } + md.sequences = append(md.sequences, from.sequences...) - md.deps = append(md.deps, from.deps...) md.views = append(md.views, from.views...) md.currUniqueID = from.currUniqueID @@ -251,6 +275,18 @@ func (md *Metadata) CopyFrom(from *Metadata, copyScalarFn func(Expr) Expr) { md.withBindings = nil } +// MDDepName stores either the unresolved DataSourceName or the StableID from +// the query that was used to resolve a data source. +type MDDepName struct { + // byID is non-zero if and only if the data source was looked up using the + // StableID. + byID cat.StableID + + // byName is non-zero if and only if the data source was looked up using a + // name. + byName cat.DataSourceName +} + // DepByName is used with AddDependency when the data source was looked up using a // data source name. func DepByName(name *cat.DataSourceName) MDDepName { @@ -268,83 +304,112 @@ func DepByID(id cat.StableID) MDDepName { // detect if the name resolves to a different data source now, or if changes to // schema or permissions on the data source has invalidated the cached metadata. func (md *Metadata) AddDependency(name MDDepName, ds cat.DataSource, priv privilege.Kind) { - // Search for the same name / object pair. - for i := range md.deps { - if md.deps[i].ds == ds && md.deps[i].name.equals(&name) { - md.deps[i].privileges |= (1 << priv) - return - } + id := ds.ID() + md.dataSourceDeps[id] = ds + md.privileges[id] = md.privileges[id] | (1 << priv) + if name.byID == 0 { + // This data source was referenced by name. + md.objectRefsByName[id] = append(md.objectRefsByName[id], name.byName.ToUnresolvedObjectName()) } - md.deps = append(md.deps, mdDep{ - ds: ds, - name: name, - privileges: (1 << priv), - }) } -// CheckDependencies resolves (again) each data source on which this metadata -// depends, in order to check that all data source names resolve to the same -// objects, and that the user still has sufficient privileges to access the -// objects. If the dependencies are no longer up-to-date, then CheckDependencies -// returns false. +// CheckDependencies resolves (again) each database object on which this +// metadata depends, in order to check the following conditions: +// 1. The object has not been modified. +// 2. If referenced by name, the name does not resolve to a different object. +// 3. The user still has sufficient privileges to access the object. Note that +// this point currently only applies to data sources. +// +// If the dependencies are no longer up-to-date, then CheckDependencies returns +// false. // -// This function cannot swallow errors and return only a boolean, as it may -// perform KV operations on behalf of the transaction associated with the -// provided catalog, and those errors are required to be propagated. +// This function can only swallow "undefined" or "dropped" errors, since these +// are expected. Other error types must be propagated, since CheckDependencies +// may perform KV operations on behalf of the transaction associated with the +// provided catalog. func (md *Metadata) CheckDependencies( - ctx context.Context, catalog cat.Catalog, + ctx context.Context, optCatalog cat.Catalog, ) (upToDate bool, err error) { - for i := range md.deps { - name := &md.deps[i].name + // Check that no referenced data sources have changed. + for id, dataSource := range md.dataSourceDeps { var toCheck cat.DataSource - var err error - if name.byID != 0 { - toCheck, _, err = catalog.ResolveDataSourceByID(ctx, cat.Flags{}, name.byID) + if names, ok := md.objectRefsByName[id]; ok { + // The data source was referenced by name at least once. + for _, name := range names { + tableName := name.ToTableName() + toCheck, _, err = optCatalog.ResolveDataSource(ctx, cat.Flags{}, &tableName) + if err != nil || !dataSource.Equals(toCheck) { + return false, maybeSwallowMetadataResolveErr(err) + } + } } else { - // Resolve data source object. - toCheck, _, err = catalog.ResolveDataSource(ctx, cat.Flags{}, &name.byName) + // The data source was only referenced by ID. + toCheck, _, err = optCatalog.ResolveDataSourceByID(ctx, cat.Flags{}, dataSource.ID()) + if err != nil || !dataSource.Equals(toCheck) { + return false, maybeSwallowMetadataResolveErr(err) + } } - if err != nil { + // Ensure that all required privileges for the data source are still valid. + if err := md.checkDataSourcePrivileges(ctx, optCatalog, toCheck); err != nil { return false, err } + } - // Ensure that it's the same object, and there were no schema or table - // statistics changes. - if !toCheck.Equals(md.deps[i].ds) { - return false, nil + // Check that all of the user defined types present have not changed. + for _, typ := range md.AllUserDefinedTypes() { + toCheck, err := optCatalog.ResolveTypeByOID(ctx, typ.Oid()) + if err != nil || typ.TypeMeta.Version != toCheck.TypeMeta.Version { + return false, maybeSwallowMetadataResolveErr(err) } + } - for privs := md.deps[i].privileges; privs != 0; { - // Strip off each privilege bit and make call to CheckPrivilege for it. - // Note that priv == 0 can occur when a dependency was added with - // privilege.Kind = 0 (e.g. for a table within a view, where the table - // privileges do not need to be checked). Ignore the "zero privilege". - priv := privilege.Kind(bits.TrailingZeros32(uint32(privs))) - if priv != 0 { - if err := catalog.CheckPrivilege(ctx, toCheck, priv); err != nil { - return false, err - } - } + return true, nil +} - // Set the just-handled privilege bit to zero and look for next. - privs &= ^(1 << priv) - } - } - // Check that all of the user defined types present have not changed. - for _, typ := range md.AllUserDefinedTypes() { - toCheck, err := catalog.ResolveTypeByOID(ctx, typ.Oid()) - if err != nil { - // Handle when the type no longer exists. - if pgerror.GetPGCode(err) == pgcode.UndefinedObject { - return false, nil +// handleMetadataResolveErr swallows errors that are thrown when a database +// object is dropped, since such an error potentially only means that the +// metadata is stale and should be re-resolved. +func maybeSwallowMetadataResolveErr(err error) error { + if err == nil { + return nil + } + // Handle when the object no longer exists. + switch pgerror.GetPGCode(err) { + case pgcode.UndefinedObject, pgcode.UndefinedTable, pgcode.UndefinedDatabase, + pgcode.UndefinedSchema, pgcode.UndefinedFunction, pgcode.InvalidName, + pgcode.InvalidSchemaName, pgcode.InvalidCatalogName: + return nil + } + if errors.Is(err, catalog.ErrDescriptorDropped) { + return nil + } + return err +} + +// checkDataSourcePrivileges checks that none of the privileges required by the +// query for the given data source have been revoked. +func (md *Metadata) checkDataSourcePrivileges( + ctx context.Context, optCatalog cat.Catalog, dataSource cat.DataSource, +) error { + if dataSource == nil { + panic(errors.AssertionFailedf("expected data source for privilege check to be non-nil")) + } + privileges := md.privileges[dataSource.ID()] + for privs := privileges; privs != 0; { + // Strip off each privilege bit and make call to CheckPrivilege for it. + // Note that priv == 0 can occur when a dependency was added with + // privilege.Kind = 0 (e.g. for a table within a view, where the table + // privileges do not need to be checked). Ignore the "zero privilege". + priv := privilege.Kind(bits.TrailingZeros32(uint32(privs))) + if priv != 0 { + if err := optCatalog.CheckPrivilege(ctx, dataSource, priv); err != nil { + return err } - return false, err - } - if typ.TypeMeta.Version != toCheck.TypeMeta.Version { - return false, nil } + // Set the just-handled privilege bit to zero and look for next. + privs &= ^(1 << priv) } - return true, nil + return nil } // AddSchema indexes a new reference to a schema used by the query. @@ -840,3 +905,18 @@ func (md *Metadata) ForEachWithBinding(fn func(WithID, Expr)) { fn(id, expr) } } + +// TestingDataSourceDeps exposes the dataSourceDeps for testing. +func (md *Metadata) TestingDataSourceDeps() map[cat.StableID]cat.DataSource { + return md.dataSourceDeps +} + +// TestingObjectRefsByName exposes the objectRefsByName for testing. +func (md *Metadata) TestingObjectRefsByName() map[cat.StableID][]*tree.UnresolvedObjectName { + return md.objectRefsByName +} + +// TestingPrivileges exposes the privileges for testing. +func (md *Metadata) TestingPrivileges() map[cat.StableID]privilegeBitmap { + return md.privileges +} diff --git a/pkg/sql/opt/metadata_test.go b/pkg/sql/opt/metadata_test.go index 80dac0f9eed0..5d4fe5d2a22c 100644 --- a/pkg/sql/opt/metadata_test.go +++ b/pkg/sql/opt/metadata_test.go @@ -46,7 +46,11 @@ func TestMetadata(t *testing.T) { // Call Init and add objects from catalog, verifying that IDs have been reset. testCat := testcat.New() - tab := &testcat.Table{Revoked: true} + tabName := tree.MakeTableNameWithSchema("t", "public", "tab") + tab := &testcat.Table{ + TabName: tabName, + Revoked: true, + } testCat.AddTable(tab) // Create a (col = 1) scalar expression. @@ -153,6 +157,30 @@ func TestMetadata(t *testing.T) { t.Fatalf("unexpected type") } + newDSDeps, oldDSDeps := mdNew.TestingDataSourceDeps(), md.TestingDataSourceDeps() + for id, dataSource := range oldDSDeps { + if newDSDeps[id] != dataSource { + t.Fatalf("expected data source dependency to be copied") + } + } + + newNamesByID, oldNamesByID := mdNew.TestingObjectRefsByName(), md.TestingObjectRefsByName() + for id, names := range oldNamesByID { + newNames := newNamesByID[id] + for i, name := range names { + if newNames[i] != name { + t.Fatalf("expected object name to be copied") + } + } + } + + newPrivileges, oldPrivileges := mdNew.TestingPrivileges(), md.TestingPrivileges() + for id, privileges := range oldPrivileges { + if newPrivileges[id] != privileges { + t.Fatalf("expected privileges to be copied") + } + } + depsUpToDate, err = md.CheckDependencies(context.Background(), testCat) if err == nil || depsUpToDate { t.Fatalf("expected table privilege to be revoked in metadata copy") From a5e2d3f5516cd2f8ff72f04f5e1b6ae3a6506ae3 Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Mon, 6 Mar 2023 23:09:41 -0800 Subject: [PATCH 2/4] opt: track UDT references by name in the Metadata Previously, it was possible for invalid queries to be kept in the cache after a schema change, since user-defined types were tracked only by OID. This missed cases where the search path changed which object a name in the query resolved to (or whether it resolved at all). This patch fixes this behavior by tracking UDT references by name, similar to what was already done for data sources. This ensures that the query staleness check doesn't miss changes to the search path. Fixes #96674 Release note (bug fix): Fixed a bug that could prevent a cached query with a user-defined type reference from being invalidated even after a schema change that should prevent the type from being resolved. --- pkg/sql/logictest/testdata/logic_test/schema | 75 ++++++++++++++++++++ pkg/sql/opt/BUILD.bazel | 1 + pkg/sql/opt/metadata.go | 31 ++++++-- pkg/sql/opt/metadata_test.go | 4 +- pkg/sql/opt/optbuilder/builder.go | 4 +- 5 files changed, 106 insertions(+), 9 deletions(-) diff --git a/pkg/sql/logictest/testdata/logic_test/schema b/pkg/sql/logictest/testdata/logic_test/schema index 44ee2f6face2..c1ecee289dfc 100644 --- a/pkg/sql/logictest/testdata/logic_test/schema +++ b/pkg/sql/logictest/testdata/logic_test/schema @@ -989,3 +989,78 @@ statement ok USE test; DROP DATABASE d; DROP TABLE xy; + +# Regression tests for #96674. +subtest alter_udt_schema + +# Renaming the schema should invalidate a schema-qualified UDT reference. +statement ok +CREATE SCHEMA sc; +CREATE TYPE sc.t AS ENUM ('HELLO'); + +query T +SELECT 'HELLO'::sc.t; +---- +HELLO + +statement ok +ALTER SCHEMA sc RENAME TO sc1; + +query error pq: type "sc.t" does not exist +SELECT 'HELLO'::sc.t; + +query T +SELECT 'HELLO'::sc1.t; +---- +HELLO + +statement ok +DROP SCHEMA sc1 CASCADE; + +# Renaming the database should invalidate a database-qualified UDT reference. +statement ok +CREATE DATABASE d; +USE d; +CREATE TYPE d.t AS ENUM ('HELLO'); + +query T +SELECT 'HELLO'::d.t; +---- +HELLO + +statement ok +ALTER DATABASE d RENAME TO d1; +USE d1; + +query error pq: type "d.t" does not exist +SELECT 'HELLO'::d.t; + +query T +SELECT 'HELLO'::d1.t; +---- +HELLO + +statement ok +USE test; +DROP DATABASE d1 CASCADE; + +# Changing the current database should invalidate an unqualified UDT reference. +statement ok +CREATE TYPE t AS ENUM ('HELLO'); + +query T +SELECT 'HELLO'::t; +---- +HELLO + +statement ok +CREATE DATABASE d; +USE d; + +query error pq: type "t" does not exist +SELECT 'HELLO'::t; + +statement ok +USE test; +DROP DATABASE d; +DROP TYPE t; diff --git a/pkg/sql/opt/BUILD.bazel b/pkg/sql/opt/BUILD.bazel index 234ef1a53051..693c337da93a 100644 --- a/pkg/sql/opt/BUILD.bazel +++ b/pkg/sql/opt/BUILD.bazel @@ -35,6 +35,7 @@ go_library( "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/privilege", + "//pkg/sql/sem/catid", "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", "//pkg/sql/sem/tree/treebin", diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go index 34ecdefee0b7..d6f2dd9cc183 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -355,11 +356,24 @@ func (md *Metadata) CheckDependencies( } } - // Check that all of the user defined types present have not changed. + // Check that no referenced user defined types have changed. for _, typ := range md.AllUserDefinedTypes() { - toCheck, err := optCatalog.ResolveTypeByOID(ctx, typ.Oid()) - if err != nil || typ.TypeMeta.Version != toCheck.TypeMeta.Version { - return false, maybeSwallowMetadataResolveErr(err) + // We can ignore the error because the OID is guaranteed to be user-defined. + descID, _ := catid.UserDefinedOIDToID(typ.Oid()) + id := cat.StableID(descID) + if names, ok := md.objectRefsByName[id]; ok { + for _, name := range names { + toCheck, err := optCatalog.ResolveType(ctx, name) + if err != nil || typ.Oid() != toCheck.Oid() || + typ.TypeMeta.Version != toCheck.TypeMeta.Version { + return false, maybeSwallowMetadataResolveErr(err) + } + } + } else { + toCheck, err := optCatalog.ResolveTypeByOID(ctx, typ.Oid()) + if err != nil || typ.TypeMeta.Version != toCheck.TypeMeta.Version { + return false, maybeSwallowMetadataResolveErr(err) + } } } @@ -425,7 +439,8 @@ func (md *Metadata) Schema(schID SchemaID) cat.Schema { } // AddUserDefinedType adds a user defined type to the metadata for this query. -func (md *Metadata) AddUserDefinedType(typ *types.T) { +// If the type was resolved by name, the name will be tracked as well. +func (md *Metadata) AddUserDefinedType(typ *types.T, name *tree.UnresolvedObjectName) { if !typ.UserDefined() { return } @@ -436,6 +451,12 @@ func (md *Metadata) AddUserDefinedType(typ *types.T) { md.userDefinedTypes[typ.Oid()] = struct{}{} md.userDefinedTypesSlice = append(md.userDefinedTypesSlice, typ) } + if name != nil { + // We can ignore the error because the OID is guaranteed to be user-defined. + descID, _ := catid.UserDefinedOIDToID(typ.Oid()) + id := cat.StableID(descID) + md.objectRefsByName[id] = append(md.objectRefsByName[id], name) + } } // AllUserDefinedTypes returns all user defined types contained in this query. diff --git a/pkg/sql/opt/metadata_test.go b/pkg/sql/opt/metadata_test.go index 5d4fe5d2a22c..855f48342f1e 100644 --- a/pkg/sql/opt/metadata_test.go +++ b/pkg/sql/opt/metadata_test.go @@ -42,7 +42,7 @@ func TestMetadata(t *testing.T) { tabID := md.AddTable(&testcat.Table{}, &tree.TableName{}) seqID := md.AddSequence(&testcat.Sequence{}) md.AddView(&testcat.View{}) - md.AddUserDefinedType(types.MakeEnum(152100, 154180)) + md.AddUserDefinedType(types.MakeEnum(152100, 154180), nil /* name */) // Call Init and add objects from catalog, verifying that IDs have been reset. testCat := testcat.New() @@ -91,7 +91,7 @@ func TestMetadata(t *testing.T) { t.Fatalf("unexpected views") } - md.AddUserDefinedType(types.MakeEnum(151500, 152510)) + md.AddUserDefinedType(types.MakeEnum(151500, 152510), nil /* name */) if len(md.AllUserDefinedTypes()) != 1 { fmt.Println(md) t.Fatalf("unexpected types") diff --git a/pkg/sql/opt/optbuilder/builder.go b/pkg/sql/opt/optbuilder/builder.go index 8b47e01aacb3..502bbcd07000 100644 --- a/pkg/sql/opt/optbuilder/builder.go +++ b/pkg/sql/opt/optbuilder/builder.go @@ -501,7 +501,7 @@ func (o *optTrackingTypeResolver) ResolveType( if err != nil { return nil, err } - o.metadata.AddUserDefinedType(typ) + o.metadata.AddUserDefinedType(typ, name) return typ, nil } @@ -513,6 +513,6 @@ func (o *optTrackingTypeResolver) ResolveTypeByOID( if err != nil { return nil, err } - o.metadata.AddUserDefinedType(typ) + o.metadata.AddUserDefinedType(typ, nil /* name */) return typ, nil } From 609dc268f0f21dad92a39ffb95749ebf55ee09bc Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Tue, 7 Mar 2023 00:26:37 -0800 Subject: [PATCH 3/4] opt: track UDFs in the Metadata This patch adds tracking for user-defined functions in `opt.Metadata`. This ensures that the query cache will be correctly invalidated after a schema change or search-path change that could change the query's semantics, or cause it to error. Fixes #93082 Fixes #93321 Release note (bug fix): Fixed a bug that could prevent a cached query from being invalidated when a UDF referenced by that query was altered or dropped. --- pkg/sql/catalog/funcdesc/func_desc.go | 1 + pkg/sql/logictest/testdata/logic_test/schema | 74 ++++++++++++++++++++ pkg/sql/logictest/testdata/logic_test/udf | 64 +++++++++++++++-- pkg/sql/opt/BUILD.bazel | 1 + pkg/sql/opt/memo/memo.go | 2 +- pkg/sql/opt/metadata.go | 69 +++++++++++++++++- pkg/sql/opt/metadata_test.go | 18 ++++- pkg/sql/opt/optbuilder/scalar.go | 1 + pkg/sql/sem/tree/function_name.go | 7 +- pkg/sql/sem/tree/overload.go | 3 + 10 files changed, 227 insertions(+), 13 deletions(-) diff --git a/pkg/sql/catalog/funcdesc/func_desc.go b/pkg/sql/catalog/funcdesc/func_desc.go index d77eded7ea07..aa7c63e46c3d 100644 --- a/pkg/sql/catalog/funcdesc/func_desc.go +++ b/pkg/sql/catalog/funcdesc/func_desc.go @@ -528,6 +528,7 @@ func (desc *immutable) ToOverload() (ret *tree.Overload, err error) { ReturnSet: desc.ReturnType.ReturnSet, Body: desc.FunctionBody, IsUDF: true, + Version: uint64(desc.Version), } argTypes := make(tree.ArgTypes, 0, len(desc.Args)) diff --git a/pkg/sql/logictest/testdata/logic_test/schema b/pkg/sql/logictest/testdata/logic_test/schema index c1ecee289dfc..2e9f815b5e13 100644 --- a/pkg/sql/logictest/testdata/logic_test/schema +++ b/pkg/sql/logictest/testdata/logic_test/schema @@ -1064,3 +1064,77 @@ statement ok USE test; DROP DATABASE d; DROP TYPE t; + +subtest alter_udf_schema + +# Renaming the schema should invalidate a schema-qualified UDF reference. +statement ok +CREATE SCHEMA sc; +CREATE FUNCTION sc.fn(INT) RETURNS INT LANGUAGE SQL AS 'SELECT $1'; + +query I +SELECT sc.fn(1); +---- +1 + +statement ok +ALTER SCHEMA sc RENAME TO sc1; + +query error pq: schema "sc" does not exist +SELECT sc.fn(1); + +query I +SELECT sc1.fn(1); +---- +1 + +statement ok +DROP SCHEMA sc1 CASCADE; + +# Renaming the database should invalidate a database-qualified UDF reference. +statement ok +CREATE DATABASE d; +USE d; +CREATE FUNCTION fn(INT) RETURNS INT LANGUAGE SQL AS 'SELECT $1'; + +query I +SELECT d.public.fn(1); +---- +1 + +statement ok +ALTER DATABASE d RENAME TO d1; +USE d1; + +query error cross-database function references not allowed +SELECT d.public.fn(1); + +query I +SELECT d1.public.fn(1); +---- +1 + +statement ok +USE test; +DROP DATABASE d1 CASCADE; + +# Changing the current database should invalidate an unqualified UDF reference. +statement ok +CREATE FUNCTION fn(INT) RETURNS INT LANGUAGE SQL AS 'SELECT $1'; + +query I +SELECT fn(1); +---- +1 + +statement ok +CREATE DATABASE d; +USE d; + +query error pq: unknown function: fn\(\): function undefined +SELECT fn(1); + +statement ok +USE test; +DROP DATABASE d; +DROP FUNCTION fn; diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index a45fd393ebd7..9db1bb596e22 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -1641,6 +1641,7 @@ DROP FUNCTION sc2.f_tbl() statement ok ALTER DATABASE rename_db1 RENAME TO rename_db2; +USE rename_db2; # Make sure that db renaming does not affect types and sequences in UDF. query T @@ -1693,7 +1694,7 @@ Mon query I SELECT sc1.f_seq() ---- -5 +1 query T SELECT sc2.f_type() @@ -1703,7 +1704,7 @@ Mon query I SELECT sc2.f_seq() ---- -6 +2 statement error pq: cannot rename schema because relation "rename_sc1.sc1.f_tbl" depends on relation "rename_sc1.sc1.tbl" ALTER SCHEMA sc1 RENAME TO sc1_new @@ -1720,16 +1721,23 @@ DROP FUNCTION sc2.f_tbl() statement ok ALTER SCHEMA sc1 RENAME TO sc1_new -# Make sure that db renaming does not affect types and sequences in UDF. -query T +# Cannot refer to the old schema name. +statement error pq: schema "sc1" does not exist SELECT sc1.f_type() + +statement error pq: schema "sc1" does not exist +SELECT sc1.f_seq() + +# Make sure that schema renaming does not affect types and sequences in UDF. +query T +SELECT sc1_new.f_type() ---- Mon query I -SELECT sc1.f_seq() +SELECT sc1_new.f_seq() ---- -7 +3 query T SELECT sc2.f_type() @@ -1739,7 +1747,7 @@ Mon query I SELECT sc2.f_seq() ---- -8 +4 statement ok SET DATABASE = test @@ -2733,3 +2741,45 @@ SELECT f_97130(); statement ok SET search_path = $pre_search_path + +# Regression test for #93082 - invalidate a cached query with a UDF if the UDF +# has been dropped. +subtest regression_93082 + +statement ok +CREATE FUNCTION fn(a INT) RETURNS INT LANGUAGE SQL AS 'SELECT a'; + +query I +SELECT fn(1); +---- +1 + +statement ok +DROP FUNCTION fn; + +statement error pq: unknown function: fn\(\): function undefined +SELECT fn(1); + +# Regression test for #93321 - invalidate a cached query with an unqualified UDF +# reference after the database is switched. +subtest regression_93321 + +statement ok +CREATE FUNCTION fn(a INT) RETURNS INT LANGUAGE SQL AS 'SELECT a'; + +query I +SELECT fn(1); +---- +1 + +statement ok +CREATE DATABASE d; +USE d; + +statement error pq: unknown function: fn\(\): function undefined +SELECT fn(1); + +statement ok +USE test; +DROP DATABASE d CASCADE; +DROP FUNCTION fn; diff --git a/pkg/sql/opt/BUILD.bazel b/pkg/sql/opt/BUILD.bazel index 693c337da93a..42d56292d34d 100644 --- a/pkg/sql/opt/BUILD.bazel +++ b/pkg/sql/opt/BUILD.bazel @@ -69,6 +69,7 @@ go_test( "//pkg/sql/opt/norm", "//pkg/sql/opt/testutils/testcat", "//pkg/sql/privilege", + "//pkg/sql/sem/catid", "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", "//pkg/sql/types", diff --git a/pkg/sql/opt/memo/memo.go b/pkg/sql/opt/memo/memo.go index 63e6e9d0c412..d083fddd5b51 100644 --- a/pkg/sql/opt/memo/memo.go +++ b/pkg/sql/opt/memo/memo.go @@ -366,7 +366,7 @@ func (m *Memo) IsStale( // Memo is stale if the fingerprint of any object in the memo's metadata has // changed, or if the current user no longer has sufficient privilege to // access the object. - if depsUpToDate, err := m.Metadata().CheckDependencies(ctx, catalog); err != nil { + if depsUpToDate, err := m.Metadata().CheckDependencies(ctx, evalCtx, catalog); err != nil { return true, err } else if !depsUpToDate { return true, nil diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go index d6f2dd9cc183..19386c286444 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -118,6 +118,10 @@ type Metadata struct { // dataSourceDeps stores each data source object that the query depends on. dataSourceDeps map[cat.StableID]cat.DataSource + // udfDeps stores each user-defined function overload that the query depends + // on. + udfDeps map[cat.StableID]*tree.Overload + // objectRefsByName stores each unique name that the query uses to reference // each object. It is needed because changes to the search path may change // which object a given name refers to; for example, switching the database. @@ -168,6 +172,14 @@ func (md *Metadata) Init() { delete(md.dataSourceDeps, id) } + udfDeps := md.udfDeps + if udfDeps == nil { + udfDeps = make(map[cat.StableID]*tree.Overload) + } + for id := range md.udfDeps { + delete(md.udfDeps, id) + } + objectRefsByName := md.objectRefsByName if objectRefsByName == nil { objectRefsByName = make(map[cat.StableID][]*tree.UnresolvedObjectName) @@ -193,6 +205,7 @@ func (md *Metadata) Init() { md.sequences = sequences[:0] md.views = views[:0] md.dataSourceDeps = dataSourceDeps + md.udfDeps = udfDeps md.objectRefsByName = objectRefsByName md.privileges = privileges } @@ -209,7 +222,7 @@ func (md *Metadata) CopyFrom(from *Metadata, copyScalarFn func(Expr) Expr) { if len(md.schemas) != 0 || len(md.cols) != 0 || len(md.tables) != 0 || len(md.sequences) != 0 || len(md.views) != 0 || len(md.userDefinedTypes) != 0 || len(md.userDefinedTypesSlice) != 0 || len(md.dataSourceDeps) != 0 || - len(md.objectRefsByName) != 0 || len(md.privileges) != 0 { + len(md.udfDeps) != 0 || len(md.objectRefsByName) != 0 || len(md.privileges) != 0 { panic(errors.AssertionFailedf("CopyFrom requires empty destination")) } md.schemas = append(md.schemas, from.schemas...) @@ -252,6 +265,13 @@ func (md *Metadata) CopyFrom(from *Metadata, copyScalarFn func(Expr) Expr) { md.dataSourceDeps[id] = dataSource } + for id, overload := range from.udfDeps { + if md.udfDeps == nil { + md.udfDeps = make(map[cat.StableID]*tree.Overload) + } + md.udfDeps[id] = overload + } + for id, names := range from.objectRefsByName { if md.objectRefsByName == nil { md.objectRefsByName = make(map[cat.StableID][]*tree.UnresolvedObjectName) @@ -329,7 +349,7 @@ func (md *Metadata) AddDependency(name MDDepName, ds cat.DataSource, priv privil // may perform KV operations on behalf of the transaction associated with the // provided catalog. func (md *Metadata) CheckDependencies( - ctx context.Context, optCatalog cat.Catalog, + ctx context.Context, evalCtx *eval.Context, optCatalog cat.Catalog, ) (upToDate bool, err error) { // Check that no referenced data sources have changed. for id, dataSource := range md.dataSourceDeps { @@ -377,6 +397,29 @@ func (md *Metadata) CheckDependencies( } } + // Check that no referenced user defined functions have changed. + for id, overload := range md.udfDeps { + if names, ok := md.objectRefsByName[id]; ok { + for _, name := range names { + definition, err := optCatalog.ResolveFunction( + ctx, name.ToUnresolvedName(), &evalCtx.SessionData().SearchPath, + ) + if err != nil { + return false, maybeSwallowMetadataResolveErr(err) + } + toCheck, err := definition.MatchOverload(overload.Types.Types(), name.Schema(), &evalCtx.SessionData().SearchPath) + if err != nil || toCheck.Oid != overload.Oid || toCheck.Version != overload.Version { + return false, err + } + } + } else { + _, toCheck, err := optCatalog.ResolveFunctionByOID(ctx, overload.Oid) + if err != nil || overload.Version != toCheck.Version { + return false, maybeSwallowMetadataResolveErr(err) + } + } + } + return true, nil } @@ -464,6 +507,23 @@ func (md *Metadata) AllUserDefinedTypes() []*types.T { return md.userDefinedTypesSlice } +// AddUserDefinedFunction adds a user-defined function to the metadata for this +// query. If the function was resolved by name, the name will also be tracked. +func (md *Metadata) AddUserDefinedFunction( + overload *tree.Overload, name *tree.UnresolvedObjectName, +) { + if !overload.IsUDF || !catid.IsOIDUserDefined(overload.Oid) { + return + } + // Ignore the error because we verify that the OID is user-defined above. + descID, _ := catid.UserDefinedOIDToID(overload.Oid) + id := cat.StableID(descID) + md.udfDeps[id] = overload + if name != nil { + md.objectRefsByName[id] = append(md.objectRefsByName[id], name) + } +} + // AddTable indexes a new reference to a table within the query. Separate // references to the same table are assigned different table ids (e.g. in a // self-join query). All columns are added to the metadata. If mutation columns @@ -932,6 +992,11 @@ func (md *Metadata) TestingDataSourceDeps() map[cat.StableID]cat.DataSource { return md.dataSourceDeps } +// TestingUDFDeps exposes the udfDeps for testing. +func (md *Metadata) TestingUDFDeps() map[cat.StableID]*tree.Overload { + return md.udfDeps +} + // TestingObjectRefsByName exposes the objectRefsByName for testing. func (md *Metadata) TestingObjectRefsByName() map[cat.StableID][]*tree.UnresolvedObjectName { return md.objectRefsByName diff --git a/pkg/sql/opt/metadata_test.go b/pkg/sql/opt/metadata_test.go index 855f48342f1e..f9a19e9cdab9 100644 --- a/pkg/sql/opt/metadata_test.go +++ b/pkg/sql/opt/metadata_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/norm" "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils/testcat" "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -98,11 +99,17 @@ func TestMetadata(t *testing.T) { } md.AddDependency(opt.DepByName(&tab.TabName), tab, privilege.CREATE) - depsUpToDate, err := md.CheckDependencies(context.Background(), testCat) + depsUpToDate, err := md.CheckDependencies(context.Background(), &evalCtx, testCat) if err == nil || depsUpToDate { t.Fatalf("expected table privilege to be revoked") } + udfName := tree.MakeQualifiedFunctionName("t", "public", "udf") + md.AddUserDefinedFunction( + &tree.Overload{Oid: catid.FuncIDToOID(1111)}, + udfName.ToUnresolvedObjectName(), + ) + // Call CopyFrom and verify that same objects are present in new metadata. expr := &memo.ProjectExpr{} md.AddWithBinding(1, expr) @@ -164,6 +171,13 @@ func TestMetadata(t *testing.T) { } } + newUDFDeps, oldUDFDeps := mdNew.TestingUDFDeps(), md.TestingUDFDeps() + for id, overload := range oldUDFDeps { + if newUDFDeps[id] != overload { + t.Fatalf("expected UDF dependency to be copied") + } + } + newNamesByID, oldNamesByID := mdNew.TestingObjectRefsByName(), md.TestingObjectRefsByName() for id, names := range oldNamesByID { newNames := newNamesByID[id] @@ -181,7 +195,7 @@ func TestMetadata(t *testing.T) { } } - depsUpToDate, err = md.CheckDependencies(context.Background(), testCat) + depsUpToDate, err = md.CheckDependencies(context.Background(), &evalCtx, testCat) if err == nil || depsUpToDate { t.Fatalf("expected table privilege to be revoked in metadata copy") } diff --git a/pkg/sql/opt/optbuilder/scalar.go b/pkg/sql/opt/optbuilder/scalar.go index 982e51271cdb..aeb31a43c912 100644 --- a/pkg/sql/opt/optbuilder/scalar.go +++ b/pkg/sql/opt/optbuilder/scalar.go @@ -610,6 +610,7 @@ func (b *Builder) buildUDF( colRefs *opt.ColSet, ) (out opt.ScalarExpr) { o := f.ResolvedOverload() + b.factory.Metadata().AddUserDefinedFunction(o, f.Func.ReferenceByName) // Build the input expressions. var input memo.ScalarListExpr diff --git a/pkg/sql/sem/tree/function_name.go b/pkg/sql/sem/tree/function_name.go index 572e69c62c3b..5b92b385ae87 100644 --- a/pkg/sql/sem/tree/function_name.go +++ b/pkg/sql/sem/tree/function_name.go @@ -63,6 +63,9 @@ type FunctionReferenceResolver interface { // ResolvableFunctionReference implements the editable reference call of a // FuncExpr. type ResolvableFunctionReference struct { + // ReferenceByName keeps track of the name that was used to resolve the + // function, if one was used. This is used for metadata dependency tracking. + ReferenceByName *UnresolvedObjectName FunctionReference } @@ -111,6 +114,8 @@ func (ref *ResolvableFunctionReference) Resolve( if err != nil { return nil, err } + referenceByName, _ := t.ToUnresolvedObjectName(NoAnnotation) + ref.ReferenceByName = &referenceByName ref.FunctionReference = fd return fd, nil default: @@ -146,7 +151,7 @@ func WrapFunction(n string) ResolvableFunctionReference { if !ok { panic(errors.AssertionFailedf("function %s() not defined", redact.Safe(n))) } - return ResolvableFunctionReference{fd} + return ResolvableFunctionReference{FunctionReference: fd} } // FunctionReference is the common interface to UnresolvedName and QualifiedFunctionName. diff --git a/pkg/sql/sem/tree/overload.go b/pkg/sql/sem/tree/overload.go index 34472e74af7a..8650055857c3 100644 --- a/pkg/sql/sem/tree/overload.go +++ b/pkg/sql/sem/tree/overload.go @@ -188,6 +188,9 @@ type Overload struct { // ReturnSet is set to true when a user-defined function is defined to return // a set of values. ReturnSet bool + // Version is the descriptor version of the descriptor used to construct + // this version of the function overload. Only used for UDFs. + Version uint64 } // params implements the overloadImpl interface. From 44f57e2db8d8682189f8b79dc202030d856a9097 Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Wed, 22 Mar 2023 18:07:57 -0700 Subject: [PATCH 4/4] opt: check UDF overloading a builtin invalidates the query cache It is possible to define a user-defined function with the same signature as a builtin function. Normally, an unqualified function call will resolve to the builtin function because its schema will be first in the search path. However, it is possible to modify the search path, so that the same function call can resolve to different functions on different executions. Example: ``` CREATE FUNCTION public.abs(val INT) RETURNS INT CALLED ON NULL INPUT LANGUAGE SQL AS $$ SELECT val+100 $$; SELECT abs(1); --This should resolve to the builtin abs(). SET search_path = public, pg_catalog; SELECT abs(1); --This should resolve to the udf abs(). ``` Fixes #97757 Release note (bug fix): Fixed a bug existing from when user-defined functions were introduced that could cause a function call to resolve to the wrong function after changes to the schema search path. --- pkg/sql/logictest/testdata/logic_test/schema | 31 +++++++++++ pkg/sql/opt/metadata.go | 54 +++++++++++++++++++- pkg/sql/opt/optbuilder/scalar.go | 1 + 3 files changed, 85 insertions(+), 1 deletion(-) diff --git a/pkg/sql/logictest/testdata/logic_test/schema b/pkg/sql/logictest/testdata/logic_test/schema index 2e9f815b5e13..167db16a39ae 100644 --- a/pkg/sql/logictest/testdata/logic_test/schema +++ b/pkg/sql/logictest/testdata/logic_test/schema @@ -1138,3 +1138,34 @@ statement ok USE test; DROP DATABASE d; DROP FUNCTION fn; + +# Regression test for #97757 - invalidate the query cache after changes to the +# search path cause a function call to resolve to a UDF when it previously +# resolved to a builtin function. +subtest invalidate-builtin + +statement ok +CREATE FUNCTION public.abs(val INT) RETURNS INT CALLED ON NULL INPUT LANGUAGE SQL AS $$ SELECT val+100 $$; + +query I +SELECT abs(1); +---- +1 + +statement ok +SET search_path = public, pg_catalog; + +# This should use the UDF abs which returns 101. +query I +SELECT abs(1); +---- +101 + +statement ok +RESET search_path; + +# This should use the builtin abs again. +query I +SELECT abs(1); +---- +1 diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go index 19386c286444..979e8632a4ba 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -131,6 +131,12 @@ type Metadata struct { // query depends on. privileges map[cat.StableID]privilegeBitmap + // builtinRefsByName stores the names used to reference builtin functions in + // the query. This is necessary to handle the case where changes to the search + // path cause a function call to be resolved to a UDF with the same signature + // as a builtin function. + builtinRefsByName map[tree.UnresolvedName]struct{} + // NOTE! When adding fields here, update Init (if reusing allocated // data structures is desired), CopyFrom and TestMetadata. } @@ -196,6 +202,14 @@ func (md *Metadata) Init() { delete(md.privileges, id) } + builtinRefsByName := md.builtinRefsByName + if builtinRefsByName == nil { + builtinRefsByName = make(map[tree.UnresolvedName]struct{}) + } + for name := range md.builtinRefsByName { + delete(md.builtinRefsByName, name) + } + // This initialization pattern ensures that fields are not unwittingly // reused. Field reuse must be explicit. *md = Metadata{} @@ -208,6 +222,7 @@ func (md *Metadata) Init() { md.udfDeps = udfDeps md.objectRefsByName = objectRefsByName md.privileges = privileges + md.builtinRefsByName = builtinRefsByName } // CopyFrom initializes the metadata with a copy of the provided metadata. @@ -222,7 +237,8 @@ func (md *Metadata) CopyFrom(from *Metadata, copyScalarFn func(Expr) Expr) { if len(md.schemas) != 0 || len(md.cols) != 0 || len(md.tables) != 0 || len(md.sequences) != 0 || len(md.views) != 0 || len(md.userDefinedTypes) != 0 || len(md.userDefinedTypesSlice) != 0 || len(md.dataSourceDeps) != 0 || - len(md.udfDeps) != 0 || len(md.objectRefsByName) != 0 || len(md.privileges) != 0 { + len(md.udfDeps) != 0 || len(md.objectRefsByName) != 0 || len(md.privileges) != 0 || + len(md.builtinRefsByName) != 0 { panic(errors.AssertionFailedf("CopyFrom requires empty destination")) } md.schemas = append(md.schemas, from.schemas...) @@ -288,6 +304,13 @@ func (md *Metadata) CopyFrom(from *Metadata, copyScalarFn func(Expr) Expr) { md.privileges[id] = privilegeSet } + for name := range from.builtinRefsByName { + if md.builtinRefsByName == nil { + md.builtinRefsByName = make(map[tree.UnresolvedName]struct{}) + } + md.builtinRefsByName[name] = struct{}{} + } + md.sequences = append(md.sequences, from.sequences...) md.views = append(md.views, from.views...) md.currUniqueID = from.currUniqueID @@ -420,6 +443,22 @@ func (md *Metadata) CheckDependencies( } } + // Check that any references to builtin functions do not now resolve to a UDF + // with the same signature (e.g. after changes to the search path). + for name := range md.builtinRefsByName { + definition, err := optCatalog.ResolveFunction( + ctx, &name, &evalCtx.SessionData().SearchPath, + ) + if err != nil { + return false, maybeSwallowMetadataResolveErr(err) + } + for i := range definition.Overloads { + if definition.Overloads[i].IsUDF { + return false, nil + } + } + } + return true, nil } @@ -524,6 +563,19 @@ func (md *Metadata) AddUserDefinedFunction( } } +// AddBuiltin adds a name used to resolve a builtin function to the metadata for +// this query. This is necessary to handle the case when changes to the search +// path cause a function call to resolve as a UDF instead of a builtin function. +func (md *Metadata) AddBuiltin(name *tree.UnresolvedObjectName) { + if name == nil { + return + } + if md.builtinRefsByName == nil { + md.builtinRefsByName = make(map[tree.UnresolvedName]struct{}) + } + md.builtinRefsByName[*name.ToUnresolvedName()] = struct{}{} +} + // AddTable indexes a new reference to a table within the query. Separate // references to the same table are assigned different table ids (e.g. in a // self-join query). All columns are added to the metadata. If mutation columns diff --git a/pkg/sql/opt/optbuilder/scalar.go b/pkg/sql/opt/optbuilder/scalar.go index aeb31a43c912..1016e614db0d 100644 --- a/pkg/sql/opt/optbuilder/scalar.go +++ b/pkg/sql/opt/optbuilder/scalar.go @@ -542,6 +542,7 @@ func (b *Builder) buildFunction( if overload.IsUDF { return b.buildUDF(f, def, inScope, outScope, outCol, colRefs) } + b.factory.Metadata().AddBuiltin(f.Func.ReferenceByName) if overload.Class == tree.AggregateClass { panic(errors.AssertionFailedf("aggregate function should have been replaced"))