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 9831017fe8b3..167db16a39ae 100644 --- a/pkg/sql/logictest/testdata/logic_test/schema +++ b/pkg/sql/logictest/testdata/logic_test/schema @@ -911,3 +911,261 @@ 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; + +# 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; + +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; + +# 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/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 1e417a36c73c..42d56292d34d 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", @@ -34,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", @@ -67,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 6cd302519829..979e8632a4ba 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -16,11 +16,13 @@ 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" "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" @@ -78,7 +80,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 +104,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 +115,30 @@ 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. -} + // dataSourceDeps stores each data source object that the query depends on. + dataSourceDeps map[cat.StableID]cat.DataSource -type mdDep struct { - ds cat.DataSource + // udfDeps stores each user-defined function overload that the query depends + // on. + udfDeps map[cat.StableID]*tree.Overload - 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 -} + // 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{} -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 +165,51 @@ 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) + } + + 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) + } + 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) + } + + 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{} @@ -191,8 +217,12 @@ 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.udfDeps = udfDeps + md.objectRefsByName = objectRefsByName + md.privileges = privileges + md.builtinRefsByName = builtinRefsByName } // CopyFrom initializes the metadata with a copy of the provided metadata. @@ -205,8 +235,10 @@ 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.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...) @@ -242,8 +274,44 @@ 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, 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) + } + 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 + } + + 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.deps = append(md.deps, from.deps...) md.views = append(md.views, from.views...) md.currUniqueID = from.currUniqueID @@ -251,6 +319,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,85 +348,166 @@ 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, evalCtx *eval.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 no referenced user defined types have changed. + for _, typ := range md.AllUserDefinedTypes() { + // 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) + } } + } - 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 { + // 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 } } - - // Set the just-handled privilege bit to zero and look for next. - privs &= ^(1 << priv) + } else { + _, toCheck, err := optCatalog.ResolveFunctionByOID(ctx, overload.Oid) + if err != nil || overload.Version != toCheck.Version { + return false, maybeSwallowMetadataResolveErr(err) + } } } - // Check that all of the user defined types present have not changed. - for _, typ := range md.AllUserDefinedTypes() { - toCheck, err := catalog.ResolveTypeByOID(ctx, typ.Oid()) + + // 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 { - // Handle when the type no longer exists. - if pgerror.GetPGCode(err) == pgcode.UndefinedObject { + return false, maybeSwallowMetadataResolveErr(err) + } + for i := range definition.Overloads { + if definition.Overloads[i].IsUDF { return false, nil } - return false, err - } - if typ.TypeMeta.Version != toCheck.TypeMeta.Version { - return false, nil } } + return true, 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 + } + } + // Set the just-handled privilege bit to zero and look for next. + privs &= ^(1 << priv) + } + return nil +} + // AddSchema indexes a new reference to a schema used by the query. func (md *Metadata) AddSchema(sch cat.Schema) SchemaID { md.schemas = append(md.schemas, sch) @@ -360,7 +521,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 } @@ -371,6 +533,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. @@ -378,6 +546,36 @@ 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) + } +} + +// 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 @@ -840,3 +1038,23 @@ 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 +} + +// 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 +} + +// 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..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" @@ -42,11 +43,15 @@ 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() - 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. @@ -87,18 +92,24 @@ 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") } 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) @@ -153,7 +164,38 @@ func TestMetadata(t *testing.T) { t.Fatalf("unexpected type") } - depsUpToDate, err = md.CheckDependencies(context.Background(), testCat) + newDSDeps, oldDSDeps := mdNew.TestingDataSourceDeps(), md.TestingDataSourceDeps() + for id, dataSource := range oldDSDeps { + if newDSDeps[id] != dataSource { + t.Fatalf("expected data source dependency to be copied") + } + } + + 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] + 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(), &evalCtx, testCat) if err == nil || depsUpToDate { t.Fatalf("expected table privilege to be revoked in metadata copy") } 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 } diff --git a/pkg/sql/opt/optbuilder/scalar.go b/pkg/sql/opt/optbuilder/scalar.go index 982e51271cdb..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")) @@ -610,6 +611,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.