Skip to content

Commit

Permalink
opt: check UDFs when checking metadata dependencies
Browse files Browse the repository at this point in the history
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.
  • Loading branch information
DrewKimball committed Feb 2, 2023
1 parent 5716851 commit 8b094c8
Show file tree
Hide file tree
Showing 9 changed files with 243 additions and 25 deletions.
1 change: 1 addition & 0 deletions pkg/sql/catalog/funcdesc/func_desc.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand Down
132 changes: 125 additions & 7 deletions pkg/sql/logictest/testdata/logic_test/udf
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -1676,7 +1677,7 @@ Mon
query I
SELECT sc1.f_seq()
----
5
1

query T
SELECT sc2.f_type()
Expand All @@ -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
Expand All @@ -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()
Expand All @@ -1722,7 +1730,7 @@ Mon
query I
SELECT sc2.f_seq()
----
8
4

statement ok
SET DATABASE = test
Expand Down Expand Up @@ -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
1 change: 1 addition & 0 deletions pkg/sql/opt/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/memo/memo.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
107 changes: 93 additions & 14 deletions pkg/sql/opt/metadata.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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.
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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...)
Expand All @@ -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 {
Expand Down Expand Up @@ -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
Expand All @@ -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
}
}
Expand All @@ -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
}

Expand Down Expand Up @@ -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
Expand Down
Loading

0 comments on commit 8b094c8

Please sign in to comment.