Skip to content

Commit

Permalink
opt: track UDFs in the Metadata
Browse files Browse the repository at this point in the history
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.
  • Loading branch information
DrewKimball committed Mar 17, 2023
1 parent 5552037 commit 57dacce
Show file tree
Hide file tree
Showing 10 changed files with 225 additions and 13 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 @@ -685,6 +685,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
74 changes: 74 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/schema
Original file line number Diff line number Diff line change
Expand Up @@ -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;
64 changes: 57 additions & 7 deletions pkg/sql/logictest/testdata/logic_test/udf
Original file line number Diff line number Diff line change
Expand Up @@ -1609,6 +1609,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 @@ -1661,7 +1662,7 @@ Mon
query I
SELECT sc1.f_seq()
----
5
1

query T
SELECT sc2.f_type()
Expand All @@ -1671,7 +1672,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 @@ -1688,16 +1689,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()
Expand All @@ -1707,7 +1715,7 @@ Mon
query I
SELECT sc2.f_seq()
----
8
4

statement ok
SET DATABASE = test
Expand Down Expand Up @@ -3333,3 +3341,45 @@ SELECT x, f93861_strict_setof(x) FROM t93861;
2 1
2 2
2 3

# 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;
1 change: 1 addition & 0 deletions pkg/sql/opt/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,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",
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 @@ -369,7 +369,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
67 changes: 65 additions & 2 deletions pkg/sql/opt/metadata.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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)
Expand All @@ -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
}
Expand All @@ -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...)
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -375,6 +395,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
}

Expand Down Expand Up @@ -460,6 +503,21 @@ 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 {
return
}
id := cat.StableID(catid.UserDefinedOIDToID(overload.Oid))
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
Expand Down Expand Up @@ -928,6 +986,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
Expand Down
18 changes: 16 additions & 2 deletions pkg/sql/opt/metadata_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,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"
Expand Down Expand Up @@ -99,11 +100,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)
Expand Down Expand Up @@ -165,6 +172,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]
Expand All @@ -182,7 +196,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")
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/opt/optbuilder/scalar.go
Original file line number Diff line number Diff line change
Expand Up @@ -619,6 +619,7 @@ func (b *Builder) buildUDF(
colRefs *opt.ColSet,
) (out opt.ScalarExpr) {
o := f.ResolvedOverload()
b.factory.Metadata().AddUserDefinedFunction(o, f.Func.ReferenceByName)

// Validate that the return types match the original return types defined in
// the function. Return types like user defined return types may change since
Expand Down
Loading

0 comments on commit 57dacce

Please sign in to comment.