From 8b094c8e44c67faefda73eac52d3ebd052ec7734 Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Thu, 26 Jan 2023 17:49:15 -0800 Subject: [PATCH] opt: check UDFs when checking metadata dependencies This patch ensures that the metadata dependency-checking tracks user-defined functions. This ensures that a cached query with a UDF reference will be invalidated when the UDF is altered or dropped, or when the database is switched. Fixes #93082 Fixes #93321 Release note (bug fix): The query cache now checks to ensure that user-defined functions referenced in the query have been altered or dropped. This prevents a bug that could cause a query to return the same result even after a UDF was dropped or the database was switched. --- pkg/sql/catalog/funcdesc/func_desc.go | 1 + pkg/sql/logictest/testdata/logic_test/udf | 132 ++++++++++++++++++++-- pkg/sql/opt/BUILD.bazel | 1 + pkg/sql/opt/memo/memo.go | 2 +- pkg/sql/opt/metadata.go | 107 +++++++++++++++--- pkg/sql/opt/metadata_test.go | 4 +- pkg/sql/opt/optbuilder/scalar.go | 1 + pkg/sql/sem/tree/function_name.go | 17 ++- pkg/sql/sem/tree/overload.go | 3 + 9 files changed, 243 insertions(+), 25 deletions(-) diff --git a/pkg/sql/catalog/funcdesc/func_desc.go b/pkg/sql/catalog/funcdesc/func_desc.go index 860c3769e3c8..00cebb2d8a9f 100644 --- a/pkg/sql/catalog/funcdesc/func_desc.go +++ b/pkg/sql/catalog/funcdesc/func_desc.go @@ -539,6 +539,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.ParamTypes, 0, len(desc.Params)) diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index 86ea143a5c0e..0b91e183241f 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -1624,6 +1624,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 @@ -1676,7 +1677,7 @@ Mon query I SELECT sc1.f_seq() ---- -5 +1 query T SELECT sc2.f_type() @@ -1686,7 +1687,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 @@ -1703,16 +1704,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 schema "sc1" does not exist SELECT sc1.f_type() + +statement error 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() @@ -1722,7 +1730,7 @@ Mon query I SELECT sc2.f_seq() ---- -8 +4 statement ok SET DATABASE = test @@ -3061,3 +3069,113 @@ query I SELECT count(descriptor) FROM system.descriptor WHERE id = $dropped_fn_id; ---- 0 + +# Regression test for #93082 and #93321 - don't reuse a cached query with a UDF +# if the UDF has been altered or dropped, or if the database has been switched. +subtest regression_93082 + +# It shouldn't be possible to call the function after it is dropped. +statement ok +CREATE FUNCTION fn(a INT) RETURNS INT LANGUAGE SQL AS 'SELECT a'; + +query II +SELECT fn(1), fn(1); +---- +1 1 + +statement ok +DROP FUNCTION fn; + +statement error pq: unknown function: fn\(\): function undefined +SELECT fn(1), fn(1); + +# It shouldn't be possible to call the function by its old name after it is +# renamed. +statement ok +CREATE FUNCTION fn(a INT) RETURNS INT LANGUAGE SQL AS 'SELECT a'; + +query II +SELECT fn(1), fn(1); +---- +1 1 + +statement ok +ALTER FUNCTION fn RENAME TO fn2; + +statement error pq: unknown function: fn\(\): function undefined +SELECT fn(1); + +# It shouldn't be possible to call the function after switching database. +statement ok +CREATE DATABASE d; +USE d; + +statement error pq: unknown function: fn\(\): function undefined +SELECT fn(1); + +statement ok +CREATE FUNCTION fn(a INT) RETURNS INT LANGUAGE SQL AS 'SELECT a'; + +query I +SELECT fn(1); +---- +1 + +statement ok +USE defaultdb; + +statement error pq: unknown function: fn\(\): function undefined +SELECT fn(1); + +# A function call should fail after schema changes prevent resolution. +statement ok +CREATE FUNCTION fn(a INT) RETURNS INT LANGUAGE SQL AS 'SELECT 1'; + +query I +SELECT fn(1); +---- +1 + +statement ok +CREATE SCHEMA sc; +ALTER FUNCTION fn(INT) SET SCHEMA sc; + +statement error pq: unknown function: fn\(\): function undefined +SELECT fn(1); + +query I +SELECT sc.fn(1); +---- +1 + +# Changing the database name should cause a fully qualified function call to +# error, but not an unqualified one. +statement ok +USE d; + +query I +SELECT fn(1); +---- +1 + +query I +SELECT d.public.fn(1); +---- +1 + +statement ok +ALTER DATABASE d RENAME TO e; +USE e; + +query I +SELECT fn(1) +---- +1 + +statement error pq: cross-database function references not allowed +SELECT d.public.fn(1); + +query I +SELECT e.public.fn(1) +---- +1 diff --git a/pkg/sql/opt/BUILD.bazel b/pkg/sql/opt/BUILD.bazel index 2dfc60c133f3..a1cb9e3f4387 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/memo/memo.go b/pkg/sql/opt/memo/memo.go index 4362bab0b1a1..b48a2454759d 100644 --- a/pkg/sql/opt/memo/memo.go +++ b/pkg/sql/opt/memo/memo.go @@ -363,7 +363,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 8ca681bcde83..f771ce861590 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" @@ -101,6 +102,10 @@ type Metadata struct { userDefinedTypes map[oid.Oid]struct{} userDefinedTypesSlice []*types.T + // userDefinedFunctions contains all user defined functions present in the + // query. + userDefinedFunctions []funcMDDep + // 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. @@ -150,6 +155,13 @@ func (n *MDDepName) equals(other *MDDepName) bool { return n.byID == other.byID && n.byName.Equals(&other.byName) } +// funcMDDep tracks the overload for a UDF call, as well as the information +// needed to resolve the overload again to check if it has changed. +type funcMDDep struct { + reference *tree.UnresolvedName + overload *tree.Overload +} + // Init prepares the metadata for use (or reuse). func (md *Metadata) Init() { // Clear the metadata objects to release memory (this clearing pattern is @@ -206,7 +218,8 @@ func (md *Metadata) Init() { 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.userDefinedTypes) != 0 || len(md.userDefinedTypesSlice) != 0 || + len(md.userDefinedFunctions) != 0 { panic(errors.AssertionFailedf("CopyFrom requires empty destination")) } md.schemas = append(md.schemas, from.schemas...) @@ -223,6 +236,12 @@ func (md *Metadata) CopyFrom(from *Metadata, copyScalarFn func(Expr) Expr) { } } + if len(from.userDefinedFunctions) > 0 { + for i := range from.userDefinedFunctions { + md.userDefinedFunctions = append(md.userDefinedFunctions, from.userDefinedFunctions[i]) + } + } + if cap(md.tables) >= len(from.tables) { md.tables = md.tables[:len(from.tables)] } else { @@ -288,21 +307,23 @@ func (md *Metadata) AddDependency(name MDDepName, ds cat.DataSource, priv privil // objects. 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 cannot swallow arbitrary 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. Note that +// it is ok to swallow "undefined" or "dropped" object errors, since these are +// expected when dependencies are not up-to-date. 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 var toCheck cat.DataSource var err error if name.byID != 0 { - toCheck, _, err = catalog.ResolveDataSourceByID(ctx, cat.Flags{}, name.byID) + toCheck, _, err = optCatalog.ResolveDataSourceByID(ctx, cat.Flags{}, name.byID) } else { // Resolve data source object. - toCheck, _, err = catalog.ResolveDataSource(ctx, cat.Flags{}, &name.byName) + toCheck, _, err = optCatalog.ResolveDataSource(ctx, cat.Flags{}, &name.byName) } if err != nil { return false, err @@ -321,7 +342,7 @@ func (md *Metadata) CheckDependencies( // 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 { + if err := optCatalog.CheckPrivilege(ctx, toCheck, priv); err != nil { return false, err } } @@ -330,20 +351,46 @@ func (md *Metadata) CheckDependencies( privs &= ^(1 << priv) } } + // handleUndefined swallows "undefined" and "dropped object errors, since + // these are expected when an object no longer exists. + handleUndefined := func(err error) error { + if pgerror.GetPGCode(err) == pgcode.UndefinedObject || + errors.Is(err, catalog.ErrDescriptorDropped) { + return nil + } + return nil + } // Check that all of the user defined types present have not changed. for _, typ := range md.AllUserDefinedTypes() { - toCheck, err := catalog.ResolveTypeByOID(ctx, typ.Oid()) + toCheck, err := optCatalog.ResolveTypeByOID(ctx, typ.Oid()) if err != nil { - // Handle when the type no longer exists. - if pgerror.GetPGCode(err) == pgcode.UndefinedObject { - return false, nil - } - return false, err + return false, handleUndefined(err) } if typ.TypeMeta.Version != toCheck.TypeMeta.Version { return false, nil } } + // Check that all of the user defined functions have not changed. + for i := range md.userDefinedFunctions { + dep := &md.userDefinedFunctions[i] + toCheck, err := optCatalog.ResolveFunction( + ctx, dep.reference, &evalCtx.SessionData().SearchPath, + ) + if err != nil { + return false, handleUndefined(err) + } + overload, err := toCheck.MatchOverload( + dep.overload.Types.Types(), "", &evalCtx.SessionData().SearchPath, + ) + if err != nil { + return false, handleUndefined(err) + } + if dep.overload.Oid != overload.Oid || dep.overload.Version != overload.Version { + // The function call resolved to either a different overload or a + // different version of the same overload. + return false, nil + } + } return true, nil } @@ -378,6 +425,38 @@ func (md *Metadata) AllUserDefinedTypes() []*types.T { return md.userDefinedTypesSlice } +// AddUserDefinedFunc adds a user defined function call to the metadata for this +// query. +func (md *Metadata) AddUserDefinedFunc(f *tree.FuncExpr) { + overload := f.ResolvedOverload() + if !overload.IsUDF { + // We check fun.IsUDF here instead of fun.HasSQLBody() because we only care + // about user-defined functions, which can be altered or dropped, unlike + // builtin functions defined using a SQL string. + return + } + reference := f.Func.GetFunctionReference() + if reference == nil { + // TODO(drewk): Once it becomes possible to resolve functions by OID, we + // should handle this case using ResolveFunctionByOID(). + panic(errors.AssertionFailedf("attempted to add UDF with nil name")) + } + for i := range md.userDefinedFunctions { + if md.userDefinedFunctions[i].overload == overload && + md.userDefinedFunctions[i].reference == reference { + // This is a duplicate. Note that we cannot deduplicate function calls + // that reference the function in different ways even if the resolved + // overload is the same. This is because different function references may + // resolve differently during a re-check; for example, after the database + // name is changed. + return + } + } + md.userDefinedFunctions = append( + md.userDefinedFunctions, funcMDDep{reference: reference, overload: overload}, + ) +} + // 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/metadata_test.go b/pkg/sql/opt/metadata_test.go index 70c0e6f8e2fc..827aed45f8b5 100644 --- a/pkg/sql/opt/metadata_test.go +++ b/pkg/sql/opt/metadata_test.go @@ -95,7 +95,7 @@ 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") } @@ -154,7 +154,7 @@ func TestMetadata(t *testing.T) { t.Fatalf("unexpected type") } - 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 9f695e3af553..7a50b5266184 100644 --- a/pkg/sql/opt/optbuilder/scalar.go +++ b/pkg/sql/opt/optbuilder/scalar.go @@ -619,6 +619,7 @@ func (b *Builder) buildUDF( colRefs *opt.ColSet, ) (out opt.ScalarExpr) { o := f.ResolvedOverload() + b.factory.Metadata().AddUserDefinedFunc(f) // Build the argument expressions. var args memo.ScalarListExpr diff --git a/pkg/sql/sem/tree/function_name.go b/pkg/sql/sem/tree/function_name.go index d95ef511d78b..76d159873c57 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 { + // reference is the original UnresolvedName that was used to resolve this + // function call. It may be unset if the function has not yet been resolved. + reference *UnresolvedName FunctionReference } @@ -112,12 +115,24 @@ func (ref *ResolvableFunctionReference) Resolve( return nil, err } ref.FunctionReference = fd + ref.reference = t return fd, nil default: return nil, errors.AssertionFailedf("unknown resolvable function reference type %s", t) } } +// GetFunctionReference returns the UnresolvedName that was used to resolve this +// function call, if any. +// TODO(drewk): when we support referencing UDFs by OID, we will need to be able +// to support using it instead of the name. +func (ref *ResolvableFunctionReference) GetFunctionReference() *UnresolvedName { + if name, ok := ref.FunctionReference.(*UnresolvedName); ok { + return name + } + return ref.reference +} + // WrapFunction creates a new ResolvableFunctionReference holding a pre-resolved // function from a built-in function name. Helper for grammar rules and // execbuilder. @@ -129,7 +144,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 9d3a2ac93733..4b5368484b57 100644 --- a/pkg/sql/sem/tree/overload.go +++ b/pkg/sql/sem/tree/overload.go @@ -221,6 +221,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.