Skip to content

Commit

Permalink
opt: check for schema changes when validating UDTs in the query cache
Browse files Browse the repository at this point in the history
The metadata stores tracks each UDT referenced in the query by its OID. It
validates the cached data by checking that the OID still resolves to the
expected type and that the type hasn't undergone schema changes since the query
was resolved. The problem with tracking only by OID instead of by the name used
to reference the type is that the query cache can incorrectly pass validation
when the UDT's schema or database has been altered in some way. This previously
led to behavior like the following:
```
defaultdb> create type sc1.t as enum ('foo', 'bar');
CREATE TYPE

defaultdb> select 'foo'::sc1.t;
t
-------
foo
(1 row)

defaultdb> alter schema sc1 rename to sc2;
ALTER SCHEMA

defaultdb> select 'foo'::sc1.t;
t
-------
foo
(1 row)
```

This patch adds schema tracking to the metadata dependency tracking. For now,
this is only done for UDT resolution, but following commits will adapt data
source and function resolution to also take advantage of this change. This
allows catalog objects to be tracked in the metadata by ID rather than by
name, which simplifies the implementation. This ensures a cached query will
be properly invalidated if the schema or database of a UDT in the query is
altered or dropped.

Fixes cockroachdb#96674

Release note (bug fix): Fixed a bug that could cause a query involving
user-defined types to ignore schema changes when the same query was invoked
before and after the change.
  • Loading branch information
DrewKimball committed Feb 28, 2023
1 parent 3236259 commit 227f621
Show file tree
Hide file tree
Showing 8 changed files with 215 additions and 20 deletions.
75 changes: 75 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/schema
Original file line number Diff line number Diff line change
Expand Up @@ -897,3 +897,78 @@ DROP SCHEMA sc

statement ok
SET CLUSTER SETTING server.eventlog.enabled = false

# 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;
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
2 changes: 1 addition & 1 deletion pkg/sql/opt/memo/memo_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/duration"
"github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
"github.com/cockroachdb/datadriven"
)
)

func TestMemo(t *testing.T) {
flags := memo.ExprFmtHideCost | memo.ExprFmtHideRuleProps | memo.ExprFmtHideQualifications |
Expand Down
81 changes: 70 additions & 11 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 @@ -121,6 +122,16 @@ type Metadata struct {
// mutation operators, used to determine the logical properties of WithScan.
withBindings map[WithID]Expr

// currentDatabase tracks the database that was used to resolve the query. It
// may be unset if the query's resolution does not depend on the current
// database.
currentDatabase string

// databaseIsUnset distinguishes between the case when the current database is
// not used to resolve any database objects and the case when the current
// database is unset.
databaseIsUnset bool

// NOTE! When adding fields here, update Init (if reusing allocated
// data structures is desired), CopyFrom and TestMetadata.
}
Expand Down Expand Up @@ -246,6 +257,8 @@ func (md *Metadata) CopyFrom(from *Metadata, copyScalarFn func(Expr) Expr) {
md.deps = append(md.deps, from.deps...)
md.views = append(md.views, from.views...)
md.currUniqueID = from.currUniqueID
md.currentDatabase = from.currentDatabase
md.databaseIsUnset = from.databaseIsUnset

// We cannot copy the bound expressions; they must be rebuilt in the new memo.
md.withBindings = nil
Expand Down Expand Up @@ -292,20 +305,52 @@ func (md *Metadata) AddDependency(name MDDepName, ds cat.DataSource, priv privil
// perform KV operations on behalf of the transaction associated with the
// provided catalog, and those errors are required to be propagated.
func (md *Metadata) CheckDependencies(
ctx context.Context, catalog cat.Catalog,
ctx context.Context, evalCtx *eval.Context, optCatalog cat.Catalog,
) (upToDate bool, err error) {
handleDescError := func(err error) error {
// 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
}
if md.databaseIsUnset {
return false, nil
}
if md.currentDatabase != "" && evalCtx.SessionData().Database != md.currentDatabase {
// The database has been switched, and the evaluation of the query is
// sensitive to this.
return false, nil
}
for _, schema := range md.schemas {
// Ensure that each schema referenced in the query resolves to the same,
// unchanged object.
toCheck, _, err := optCatalog.ResolveSchema(ctx, cat.Flags{}, schema.Name())
if err != nil {
return false, handleDescError(err)
}
if !toCheck.Equals(schema) {
return false, nil
}
}
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
return false, handleDescError(err)
}

// Ensure that it's the same object, and there were no schema or table
Expand All @@ -321,7 +366,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 @@ -332,13 +377,9 @@ func (md *Metadata) CheckDependencies(
}
// 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, handleDescError(err)
}
if typ.TypeMeta.Version != toCheck.TypeMeta.Version {
return false, nil
Expand All @@ -347,8 +388,26 @@ func (md *Metadata) CheckDependencies(
return true, nil
}

// SetCurrentDatabase records the current database that was used to resolve the
// query. It should be used when an object in the query is not qualified by
// database, since resolution of that object will change if the current database
// changes.
func (md *Metadata) SetCurrentDatabase(currentDatabase string) {
if currentDatabase == "" {
// This is possible in tests.
md.databaseIsUnset = true
}
md.currentDatabase = currentDatabase
}

// AddSchema indexes a new reference to a schema used by the query.
func (md *Metadata) AddSchema(sch cat.Schema) SchemaID {
for i := range md.schemas {
if md.schemas[i] == sch {
// This is a duplicate.
return SchemaID(i + 1)
}
}
md.schemas = append(md.schemas, sch)
return SchemaID(len(md.schemas))
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/opt/metadata_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")
}
Expand Down Expand Up @@ -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")
}
Expand Down
65 changes: 61 additions & 4 deletions pkg/sql/opt/optbuilder/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,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/catconstants"
"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 @@ -137,6 +138,8 @@ type Builder struct {
schemaDeps opt.SchemaDeps
schemaTypeDeps opt.SchemaTypeDeps

resolverHelper optTrackingResolverHelper

// If set, the data source names in the AST are rewritten to the fully
// qualified version (after resolution). Used to construct the strings for
// CREATE VIEW and CREATE TABLE AS queries.
Expand Down Expand Up @@ -213,12 +216,23 @@ func (b *Builder) Build() (err error) {
existingResolver := b.semaCtx.TypeResolver
// Ensure that the original TypeResolver is reset after.
defer func() { b.semaCtx.TypeResolver = existingResolver }()
b.resolverHelper = optTrackingResolverHelper{
metadata: b.factory.Metadata(),
optCatalog: b.catalog,
evalCtx: b.evalCtx,
}
typeTracker := &optTrackingTypeResolver{
res: b.semaCtx.TypeResolver,
metadata: b.factory.Metadata(),
res: b.semaCtx.TypeResolver,
optTrackingResolverHelper: &b.resolverHelper,
}
b.semaCtx.TypeResolver = typeTracker

if b.evalCtx.SessionData().Database == "" {
// It is possible (though unsafe) to set the database to the empty string.
// This is done in tests in some places.
b.factory.Metadata().SetCurrentDatabase("" /* currentDatabase */)
}

// Special case for CannedOptPlan.
if canned, ok := b.stmt.(*tree.CannedOptPlan); ok {
b.factory.DisableOptimizations()
Expand Down Expand Up @@ -491,11 +505,51 @@ func (b *Builder) maybeTrackUserDefinedTypeDepsForViews(texpr tree.TypedExpr) {
}
}

// optTrackingResolverHelper wraps the fields and logic common to resolving
// schema objects (types, functions, and data sources).
type optTrackingResolverHelper struct {
metadata *opt.Metadata
optCatalog cat.Catalog
evalCtx *eval.Context
}

// trackObjectPath adds the schema of the given object to the metadata
// dependency tracking.
func (o *optTrackingResolverHelper) trackObjectPath(
ctx context.Context, name *tree.UnresolvedObjectName,
) error {
if o.evalCtx.SessionData().Database == "" {
// This can happen during testing. The query will not be cached, so it is
// not necessary to track the schema.
return nil
}
if _, ok := catconstants.VirtualSchemaNames[name.Schema()]; ok {
// Ignore virtual schemas.
return nil
}
schemaName := cat.SchemaName{
CatalogName: tree.Name(name.Catalog()),
SchemaName: tree.Name(name.Schema()),
ExplicitCatalog: name.HasExplicitCatalog(),
ExplicitSchema: name.HasExplicitSchema(),
}
schema, schemaName, err := o.optCatalog.ResolveSchema(ctx, cat.Flags{}, &schemaName)
if err != nil {
return err
}
o.metadata.AddSchema(schema)
if !schemaName.ExplicitCatalog {
// Resolution of this object depends on the current database.
o.metadata.SetCurrentDatabase(o.evalCtx.SessionData().Database)
}
return nil
}

// optTrackingTypeResolver is a wrapper around a TypeReferenceResolver that
// remembers all of the resolved types in the provided Metadata.
type optTrackingTypeResolver struct {
res tree.TypeReferenceResolver
metadata *opt.Metadata
res tree.TypeReferenceResolver
*optTrackingResolverHelper
}

// ResolveType implements the TypeReferenceResolver interface.
Expand All @@ -507,6 +561,9 @@ func (o *optTrackingTypeResolver) ResolveType(
return nil, err
}
o.metadata.AddUserDefinedType(typ)
if err = o.trackObjectPath(ctx, name); err != nil {
return nil, err
}
return typ, nil
}

Expand Down
5 changes: 4 additions & 1 deletion pkg/sql/opt_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,10 @@ func (os *optSchema) PostgresDescriptorID() catid.DescID {
// Equals is part of the cat.Object interface.
func (os *optSchema) Equals(other cat.Object) bool {
otherSchema, ok := other.(*optSchema)
return ok && os.ID() == otherSchema.ID()
return ok && os.ID() == otherSchema.ID() &&
os.database.GetID() == otherSchema.database.GetID() &&
os.schema.GetVersion() == otherSchema.schema.GetVersion() &&
os.database.GetVersion() == otherSchema.database.GetVersion()
}

// Name is part of the cat.Schema interface.
Expand Down

0 comments on commit 227f621

Please sign in to comment.