From 4dc62d039b83ea81deabeb488943a84c1dba73cc Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Mon, 7 Oct 2024 19:53:44 -0600 Subject: [PATCH 1/4] opt: replace "udf" with "routine" in the metadata This commit changes the naming of some methods and data structures used in query plan metadata to refer to "routines" instead of "udfs". This clarifies that stored procedures are also valid targets for metadata tracking. Epic: None Release note: None --- pkg/sql/explain_bundle.go | 4 +- pkg/sql/opt/metadata.go | 65 ++++++++++++++++--------------- pkg/sql/opt/metadata_test.go | 6 +-- pkg/sql/opt/optbuilder/routine.go | 2 +- 4 files changed, 40 insertions(+), 37 deletions(-) diff --git a/pkg/sql/explain_bundle.go b/pkg/sql/explain_bundle.go index 597062ac22c4..d006a79d5349 100644 --- a/pkg/sql/explain_bundle.go +++ b/pkg/sql/explain_bundle.go @@ -623,8 +623,8 @@ func (b *stmtBundleBuilder) addEnv(ctx context.Context) { if err := c.PrintCreateEnum(&buf, b.flags.RedactValues); err != nil { b.printError(fmt.Sprintf("-- error getting schema for enums: %v", err), &buf) } - if mem.Metadata().HasUserDefinedFunctions() { - // Get all relevant user-defined functions. + if mem.Metadata().HasUserDefinedRoutines() { + // Get all relevant user-defined routines. blankLine() err = c.PrintRelevantCreateRoutine( &buf, strings.ToLower(b.stmt), b.flags.RedactValues, &b.errorStrings, false, /* procedure */ diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go index bf0ff3e3f650..5172915f762a 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -38,7 +38,7 @@ type SchemaID int32 // 1 << privilege.Kind, so that multiple privileges can be stored. type privilegeBitmap uint64 -type udfDep struct { +type routineDep struct { overload *tree.Overload invocationTypes []*types.T } @@ -122,9 +122,9 @@ 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 (as well as the - // invocation signature) that the query depends on. - udfDeps map[cat.StableID]udfDep + // routineDeps stores each user-defined function and stored procedure overload + // (as well as the invocation signature) that the query depends on. + routineDeps map[cat.StableID]routineDep // objectRefsByName stores each unique name that the query uses to reference // each object. It is needed because changes to the search path may change @@ -182,12 +182,12 @@ func (md *Metadata) Init() { delete(md.dataSourceDeps, id) } - udfDeps := md.udfDeps - if udfDeps == nil { - udfDeps = make(map[cat.StableID]udfDep) + routineDeps := md.routineDeps + if routineDeps == nil { + routineDeps = make(map[cat.StableID]routineDep) } - for id := range md.udfDeps { - delete(md.udfDeps, id) + for id := range md.routineDeps { + delete(md.routineDeps, id) } objectRefsByName := md.objectRefsByName @@ -223,7 +223,7 @@ func (md *Metadata) Init() { md.sequences = sequences[:0] md.views = views[:0] md.dataSourceDeps = dataSourceDeps - md.udfDeps = udfDeps + md.routineDeps = routineDeps md.objectRefsByName = objectRefsByName md.privileges = privileges md.builtinRefsByName = builtinRefsByName @@ -241,7 +241,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.udfDeps) != 0 || len(md.objectRefsByName) != 0 || len(md.privileges) != 0 || + len(md.routineDeps) != 0 || len(md.objectRefsByName) != 0 || len(md.privileges) != 0 || len(md.builtinRefsByName) != 0 { panic(errors.AssertionFailedf("CopyFrom requires empty destination")) } @@ -285,11 +285,11 @@ 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]udfDep) + for id, overload := range from.routineDeps { + if md.routineDeps == nil { + md.routineDeps = make(map[cat.StableID]routineDep) } - md.udfDeps[id] = overload + md.routineDeps[id] = overload } for id, names := range from.objectRefsByName { @@ -418,8 +418,9 @@ func (md *Metadata) CheckDependencies( } } - // Check that no referenced user defined functions have changed. - for id, dep := range md.udfDeps { + // Check that no referenced user defined functions or stored procedures have + // changed. + for id, dep := range md.routineDeps { overload := dep.overload if names, ok := md.objectRefsByName[id]; ok { for _, name := range names { @@ -503,7 +504,7 @@ func (md *Metadata) CheckDependencies( if err := md.checkDataSourcePrivileges(ctx, optCatalog); err != nil { return false, err } - for _, dep := range md.udfDeps { + for _, dep := range md.routineDeps { if err := optCatalog.CheckExecutionPrivilege(ctx, dep.overload.Oid, optCatalog.GetCurrentUser()); err != nil { return false, err } @@ -591,21 +592,23 @@ func (md *Metadata) AllUserDefinedTypes() []*types.T { return md.userDefinedTypesSlice } -// HasUserDefinedFunctions returns true if the query references a UDF. -func (md *Metadata) HasUserDefinedFunctions() bool { - return len(md.udfDeps) > 0 +// HasUserDefinedRoutines returns true if the query references a UDF or stored +// procedure. +func (md *Metadata) HasUserDefinedRoutines() bool { + return len(md.routineDeps) > 0 } -// 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( +// AddUserDefinedRoutine adds a user-defined function or stored procedure to the +// metadata for this query. If the routine was resolved by name, the name will +// also be tracked. +func (md *Metadata) AddUserDefinedRoutine( overload *tree.Overload, invocationTypes []*types.T, name *tree.UnresolvedObjectName, ) { if overload.Type != tree.UDFRoutine { return } id := cat.StableID(catid.UserDefinedOIDToID(overload.Oid)) - md.udfDeps[id] = udfDep{ + md.routineDeps[id] = routineDep{ overload: overload, invocationTypes: invocationTypes, } @@ -1174,14 +1177,14 @@ func (md *Metadata) TestingDataSourceDeps() map[cat.StableID]cat.DataSource { return md.dataSourceDeps } -// TestingUDFDepsEqual returns whether the UDF deps of the other Metadata are -// equal to the UDF deps of this Metadata. -func (md *Metadata) TestingUDFDepsEqual(other *Metadata) bool { - if len(md.udfDeps) != len(other.udfDeps) { +// TestingRoutineDepsEqual returns whether the routine deps of the other +// Metadata are equal to the routine deps of this Metadata. +func (md *Metadata) TestingRoutineDepsEqual(other *Metadata) bool { + if len(md.routineDeps) != len(other.routineDeps) { return false } - for id, otherDep := range other.udfDeps { - dep, ok := md.udfDeps[id] + for id, otherDep := range other.routineDeps { + dep, ok := md.routineDeps[id] if !ok { return false } diff --git a/pkg/sql/opt/metadata_test.go b/pkg/sql/opt/metadata_test.go index 44a249da8329..9f0e7e0bc8ff 100644 --- a/pkg/sql/opt/metadata_test.go +++ b/pkg/sql/opt/metadata_test.go @@ -104,7 +104,7 @@ func TestMetadata(t *testing.T) { } udfName := tree.MakeQualifiedRoutineName("t", "public", "udf") - md.AddUserDefinedFunction( + md.AddUserDefinedRoutine( &tree.Overload{Oid: catid.FuncIDToOID(1111)}, types.OneIntCol, udfName.ToUnresolvedObjectName(), @@ -175,8 +175,8 @@ func TestMetadata(t *testing.T) { } } - if !mdNew.TestingUDFDepsEqual(md) { - t.Fatalf("expected UDF dependency to be copied") + if !mdNew.TestingRoutineDepsEqual(md) { + t.Fatalf("expected user-defined routine dependency to be copied") } newNamesByID, oldNamesByID := mdNew.TestingObjectRefsByName(), md.TestingObjectRefsByName() diff --git a/pkg/sql/opt/optbuilder/routine.go b/pkg/sql/opt/optbuilder/routine.go index fdabfd76584d..780ac2045c49 100644 --- a/pkg/sql/opt/optbuilder/routine.go +++ b/pkg/sql/opt/optbuilder/routine.go @@ -216,7 +216,7 @@ func (b *Builder) buildRoutine( } invocationTypes[i] = texpr.ResolvedType() } - b.factory.Metadata().AddUserDefinedFunction(o, invocationTypes, f.Func.ReferenceByName) + b.factory.Metadata().AddUserDefinedRoutine(o, invocationTypes, 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 From 77781b60c29aa1f39ad907a5add39e28d2547bd9 Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Mon, 7 Oct 2024 20:06:54 -0600 Subject: [PATCH 2/4] sql: use memo metadata to add routines to statement bundles This commit updates the statement bundle logic to take advantage of the information stored in the query plan metadata so that only relevant routines are shown in `schema.sql` for a statement bundle. In addition, stored procedures are now tracked in the metadata in addition to UDFs. This has no impact on query-plan caching, since we currently do not cache plans that invoke stored procedures. Fixes #132142 Fixes #104976 Release note (bug fix): Fixed a bug that prevented the create statement for a routine from being shown in a statement bundle. This happened when the routine was created on a schema other than `public`. The bug has existed since v23.1. --- pkg/sql/explain_bundle.go | 90 ++++++++++++---------------------- pkg/sql/explain_bundle_test.go | 62 +++++++++++++++++++++++ pkg/sql/opt/metadata.go | 11 ++++- 3 files changed, 103 insertions(+), 60 deletions(-) diff --git a/pkg/sql/explain_bundle.go b/pkg/sql/explain_bundle.go index d006a79d5349..4077c260b73c 100644 --- a/pkg/sql/explain_bundle.go +++ b/pkg/sql/explain_bundle.go @@ -25,10 +25,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/exec/explain" "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/stmtdiagnostics" "github.com/cockroachdb/cockroach/pkg/util/buildutil" + "github.com/cockroachdb/cockroach/pkg/util/intsets" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/memzipper" "github.com/cockroachdb/cockroach/pkg/util/pretty" @@ -36,6 +38,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" + "github.com/lib/pq/oid" ) const noPlan = "no plan" @@ -600,12 +603,11 @@ func (b *stmtBundleBuilder) addEnv(ctx context.Context) { // Note: we do not shortcut out of this function if there is no table/sequence/view to report: // the bundle analysis tool require schema.sql to always be present, even if it's empty. - first := true blankLine := func() { - if !first { + if buf.Len() > 0 { + // Don't add newlines to the beginning of the file. buf.WriteByte('\n') } - first = false } blankLine() c.printCreateAllDatabases(&buf, dbNames) @@ -625,28 +627,20 @@ func (b *stmtBundleBuilder) addEnv(ctx context.Context) { } if mem.Metadata().HasUserDefinedRoutines() { // Get all relevant user-defined routines. - blankLine() - err = c.PrintRelevantCreateRoutine( - &buf, strings.ToLower(b.stmt), b.flags.RedactValues, &b.errorStrings, false, /* procedure */ - ) - if err != nil { - b.printError(fmt.Sprintf("-- error getting schema for udfs: %v", err), &buf) - } - } - if call, ok := mem.RootExpr().(*memo.CallExpr); ok { - // Currently, a stored procedure can only be called from a CALL statement, - // which can only be the root expression. - if proc, ok := call.Proc.(*memo.UDFCallExpr); ok { + var ids intsets.Fast + isProcedure := make(map[oid.Oid]bool) + mem.Metadata().ForEachUserDefinedRoutine(func(ol *tree.Overload) { + ids.Add(int(ol.Oid)) + isProcedure[ol.Oid] = ol.Type == tree.ProcedureRoutine + }) + ids.ForEach(func(id int) { blankLine() - err = c.PrintRelevantCreateRoutine( - &buf, strings.ToLower(proc.Def.Name), b.flags.RedactValues, &b.errorStrings, true, /* procedure */ - ) + routineOid := oid.Oid(id) + err = c.PrintCreateRoutine(&buf, routineOid, b.flags.RedactValues, isProcedure[routineOid]) if err != nil { - b.printError(fmt.Sprintf("-- error getting schema for procedure: %v", err), &buf) + b.printError(fmt.Sprintf("-- error getting schema for routine with ID %d: %v", id, err), &buf) } - } else { - b.printError("-- unexpected input expression for CALL statement", &buf) - } + }) } for i := range tables { blankLine() @@ -1030,50 +1024,28 @@ func (c *stmtEnvCollector) PrintCreateEnum(w io.Writer, redactValues bool) error return nil } -func (c *stmtEnvCollector) PrintRelevantCreateRoutine( - w io.Writer, stmt string, redactValues bool, errorStrings *[]string, procedure bool, +func (c *stmtEnvCollector) PrintCreateRoutine( + w io.Writer, id oid.Oid, redactValues bool, procedure bool, ) error { - // The select function_name returns a DOidWrapper, - // we need to cast it to string for queryRows function to process. - // TODO(#104976): consider getting the udf sql body statements from the memo metadata. - var routineTypeName, routineNameQuery string + var createRoutineQuery string + descID := catid.UserDefinedOIDToID(id) + queryTemplate := "SELECT create_statement FROM crdb_internal.create_%[1]s_statements WHERE %[1]s_id = %[2]d" if procedure { - routineTypeName = "PROCEDURE" - routineNameQuery = "SELECT procedure_name::STRING as procedure_name_str FROM [SHOW PROCEDURES]" + createRoutineQuery = fmt.Sprintf(queryTemplate, "procedure", descID) } else { - routineTypeName = "FUNCTION" - routineNameQuery = "SELECT function_name::STRING as function_name_str FROM [SHOW FUNCTIONS]" + createRoutineQuery = fmt.Sprintf(queryTemplate, "function", descID) + } + if redactValues { + createRoutineQuery = fmt.Sprintf( + "SELECT crdb_internal.redact(crdb_internal.redactable_sql_constants(create_statement)) FROM (%s)", + createRoutineQuery, + ) } - routineNames, err := c.queryRows(routineNameQuery) + createStatement, err := c.query(createRoutineQuery) if err != nil { return err } - for _, name := range routineNames { - if strings.Contains(stmt, name) { - createRoutineQuery := fmt.Sprintf( - "SELECT create_statement FROM [ SHOW CREATE %s \"%s\" ]", routineTypeName, name, - ) - if redactValues { - createRoutineQuery = fmt.Sprintf( - "SELECT crdb_internal.redact(crdb_internal.redactable_sql_constants(create_statement)) FROM [ SHOW CREATE %s \"%s\" ]", - routineTypeName, name, - ) - } - createStatement, err := c.query(createRoutineQuery) - if err != nil { - var errString string - if procedure { - errString = fmt.Sprintf("-- error getting stored procedure %s: %s", name, err) - } else { - errString = fmt.Sprintf("-- error getting user defined function %s: %s", name, err) - } - fmt.Fprint(w, errString+"\n") - *errorStrings = append(*errorStrings, errString) - continue - } - fmt.Fprintf(w, "%s\n", createStatement) - } - } + fmt.Fprintf(w, "%s;\n", createStatement) return nil } diff --git a/pkg/sql/explain_bundle_test.go b/pkg/sql/explain_bundle_test.go index 9b6a76bd659e..18056a0ae7ba 100644 --- a/pkg/sql/explain_bundle_test.go +++ b/pkg/sql/explain_bundle_test.go @@ -420,6 +420,68 @@ CREATE TABLE users(id UUID DEFAULT gen_random_uuid() PRIMARY KEY, promo_id INT R "distsql.html vec-v.txt vec.txt") }) + t.Run("different schema UDF", func(t *testing.T) { + r.Exec(t, "CREATE FUNCTION foo() RETURNS INT LANGUAGE SQL AS 'SELECT count(*) FROM abc, s.a';") + r.Exec(t, "CREATE FUNCTION s.foo() RETURNS INT LANGUAGE SQL AS 'SELECT count(*) FROM abc, s.a';") + rows := r.QueryStr(t, "EXPLAIN ANALYZE (DEBUG) SELECT s.foo();") + checkBundle( + t, fmt.Sprint(rows), "s.foo", func(name, contents string) error { + if name == "schema.sql" { + reg := regexp.MustCompile("s.foo") + if reg.FindString(contents) == "" { + return errors.Errorf("could not find definition for 's.foo' function in schema.sql") + } + reg = regexp.MustCompile("^foo") + if reg.FindString(contents) != "" { + return errors.Errorf("found irrelevant function 'foo' in schema.sql") + } + reg = regexp.MustCompile("s.a") + if reg.FindString(contents) == "" { + return errors.Errorf("could not find definition for relation 's.a' in schema.sql") + } + reg = regexp.MustCompile("abc") + if reg.FindString(contents) == "" { + return errors.Errorf("could not find definition for relation 'abc' in schema.sql") + } + } + return nil + }, + false /* expectErrors */, base, plans, + "stats-defaultdb.public.abc.sql stats-defaultdb.s.a.sql distsql.html vec-v.txt vec.txt", + ) + }) + + t.Run("different schema procedure", func(t *testing.T) { + r.Exec(t, "CREATE PROCEDURE bar() LANGUAGE SQL AS 'SELECT count(*) FROM abc, s.a';") + r.Exec(t, "CREATE PROCEDURE s.bar() LANGUAGE SQL AS 'SELECT count(*) FROM abc, s.a';") + rows := r.QueryStr(t, "EXPLAIN ANALYZE (DEBUG) CALL s.bar();") + checkBundle( + t, fmt.Sprint(rows), "s.bar", func(name, contents string) error { + if name == "schema.sql" { + reg := regexp.MustCompile("s.bar") + if reg.FindString(contents) == "" { + return errors.Errorf("could not find definition for 's.bar' procedure in schema.sql") + } + reg = regexp.MustCompile("^bar") + if reg.FindString(contents) != "" { + return errors.Errorf("Found irrelevant procedure 'bar' in schema.sql") + } + reg = regexp.MustCompile("s.a") + if reg.FindString(contents) == "" { + return errors.Errorf("could not find definition for relation 's.a' in schema.sql") + } + reg = regexp.MustCompile("abc") + if reg.FindString(contents) == "" { + return errors.Errorf("could not find definition for relation 'abc' in schema.sql") + } + } + return nil + }, + false /* expectErrors */, base, plans, + "stats-defaultdb.public.abc.sql stats-defaultdb.s.a.sql distsql.html vec-v.txt vec.txt", + ) + }) + t.Run("permission error", func(t *testing.T) { r.Exec(t, "CREATE USER test") r.Exec(t, "SET ROLE test") diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go index 5172915f762a..aec784f7e881 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -604,7 +604,7 @@ func (md *Metadata) HasUserDefinedRoutines() bool { func (md *Metadata) AddUserDefinedRoutine( overload *tree.Overload, invocationTypes []*types.T, name *tree.UnresolvedObjectName, ) { - if overload.Type != tree.UDFRoutine { + if overload.Type == tree.BuiltinRoutine { return } id := cat.StableID(catid.UserDefinedOIDToID(overload.Oid)) @@ -617,6 +617,15 @@ func (md *Metadata) AddUserDefinedRoutine( } } +// ForEachUserDefinedRoutine executes the given function for each user-defined +// routine (UDF or stored procedure) overload. The order of iteration is +// non-deterministic. +func (md *Metadata) ForEachUserDefinedRoutine(fn func(overload *tree.Overload)) { + for _, dep := range md.routineDeps { + fn(dep.overload) + } +} + // 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. From 94bd2ea18ca1079b615f122e752a7cd536b0ab87 Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Mon, 7 Oct 2024 20:34:00 -0600 Subject: [PATCH 3/4] sql: use memo metadata to add UDTs to statement bundle This commit modifies the statement bundle collection logic to use the query plan metadata to determine which types to display in `schema.sql`. This ensures that only types which are referenced by the query are shown. Informs #104976 Release note: None --- pkg/sql/explain_bundle.go | 20 +++++++++++--------- pkg/sql/explain_bundle_test.go | 20 ++++++++++++++------ 2 files changed, 25 insertions(+), 15 deletions(-) diff --git a/pkg/sql/explain_bundle.go b/pkg/sql/explain_bundle.go index 4077c260b73c..2583d78b800a 100644 --- a/pkg/sql/explain_bundle.go +++ b/pkg/sql/explain_bundle.go @@ -620,10 +620,12 @@ func (b *stmtBundleBuilder) addEnv(ctx context.Context) { b.printError(fmt.Sprintf("-- error getting schema for sequence %s: %v", sequences[i].FQString(), err), &buf) } } - // Get all user-defined types. If redaction is a - blankLine() - if err := c.PrintCreateEnum(&buf, b.flags.RedactValues); err != nil { - b.printError(fmt.Sprintf("-- error getting schema for enums: %v", err), &buf) + // Get all relevant user-defined types. + for _, t := range mem.Metadata().AllUserDefinedTypes() { + blankLine() + if err := c.PrintCreateUDT(&buf, t.Oid(), b.flags.RedactValues); err != nil { + b.printError(fmt.Sprintf("-- error getting schema for type %s: %v", t.SQLStringForError(), err), &buf) + } } if mem.Metadata().HasUserDefinedRoutines() { // Get all relevant user-defined routines. @@ -1008,13 +1010,13 @@ func (c *stmtEnvCollector) PrintCreateSequence(w io.Writer, tn *tree.TableName) return nil } -func (c *stmtEnvCollector) PrintCreateEnum(w io.Writer, redactValues bool) error { - qry := "SELECT create_statement FROM [SHOW CREATE ALL TYPES]" +func (c *stmtEnvCollector) PrintCreateUDT(w io.Writer, id oid.Oid, redactValues bool) error { + descID := catid.UserDefinedOIDToID(id) + query := fmt.Sprintf("SELECT create_statement FROM crdb_internal.create_type_statements WHERE descriptor_id = %d", descID) if redactValues { - qry = "SELECT crdb_internal.redact(crdb_internal.redactable_sql_constants(create_statement)) FROM [SHOW CREATE ALL TYPES]" - + query = fmt.Sprintf("SELECT crdb_internal.redact(crdb_internal.redactable_sql_constants(create_statement)) FROM (%s)", query) } - createStatement, err := c.queryRows(qry) + createStatement, err := c.queryRows(query) if err != nil { return err } diff --git a/pkg/sql/explain_bundle_test.go b/pkg/sql/explain_bundle_test.go index 18056a0ae7ba..0baf88d1d4ba 100644 --- a/pkg/sql/explain_bundle_test.go +++ b/pkg/sql/explain_bundle_test.go @@ -367,13 +367,21 @@ CREATE TABLE users(id UUID DEFAULT gen_random_uuid() PRIMARY KEY, promo_id INT R t.Run("types", func(t *testing.T) { r.Exec(t, "CREATE TYPE test_type1 AS ENUM ('hello','world');") r.Exec(t, "CREATE TYPE test_type2 AS ENUM ('goodbye','earth');") - rows := r.QueryStr(t, "EXPLAIN ANALYZE (DEBUG) SELECT 1;") + rows := r.QueryStr(t, "EXPLAIN ANALYZE (DEBUG) SELECT 'hello'::test_type1;") checkBundle( - t, fmt.Sprint(rows), "test_type1", nil, false, /* expectErrors */ - base, plans, "distsql.html vec.txt vec-v.txt", - ) - checkBundle( - t, fmt.Sprint(rows), "test_type2", nil, false, /* expectErrors */ + t, fmt.Sprint(rows), "test_type1", func(name, contents string) error { + if name == "schema.sql" { + reg := regexp.MustCompile("test_type1") + if reg.FindString(contents) == "" { + return errors.Errorf("could not find definition for 'test_type1' type in schema.sql") + } + reg = regexp.MustCompile("test_type2") + if reg.FindString(contents) != "" { + return errors.Errorf("Found irrelevant user defined type 'test_type2' in schema.sql") + } + } + return nil + }, false, /* expectErrors */ base, plans, "distsql.html vec.txt vec-v.txt", ) }) From 320297f4d73fbe59307ce93bf195b16002683fef Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Wed, 15 Jan 2025 17:01:11 -0800 Subject: [PATCH 4/4] sql: minor fixup of a recent change This commit addresses comments identified during the review of a backport of 3fb9f918ab48e53b2feb6ec9c31b505272be30b1. Release note: None --- pkg/sql/explain_bundle.go | 6 ++++++ pkg/sql/explain_bundle_test.go | 12 ++++++------ 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/pkg/sql/explain_bundle.go b/pkg/sql/explain_bundle.go index 2583d78b800a..b5ec5c79ef96 100644 --- a/pkg/sql/explain_bundle.go +++ b/pkg/sql/explain_bundle.go @@ -629,6 +629,12 @@ func (b *stmtBundleBuilder) addEnv(ctx context.Context) { } if mem.Metadata().HasUserDefinedRoutines() { // Get all relevant user-defined routines. + // + // Note that we first populate fast int set so that we add routines in + // increasing order of Oids to the bundle. This should allow for easier + // recreation when we have dependencies between routines since _usually_ + // smaller Oid would indicate an older routine which makes it less + // likely to depend on another routine. var ids intsets.Fast isProcedure := make(map[oid.Oid]bool) mem.Metadata().ForEachUserDefinedRoutine(func(ol *tree.Overload) { diff --git a/pkg/sql/explain_bundle_test.go b/pkg/sql/explain_bundle_test.go index 0baf88d1d4ba..ed5da51755e6 100644 --- a/pkg/sql/explain_bundle_test.go +++ b/pkg/sql/explain_bundle_test.go @@ -435,15 +435,15 @@ CREATE TABLE users(id UUID DEFAULT gen_random_uuid() PRIMARY KEY, promo_id INT R checkBundle( t, fmt.Sprint(rows), "s.foo", func(name, contents string) error { if name == "schema.sql" { - reg := regexp.MustCompile("s.foo") + reg := regexp.MustCompile(`s\.foo`) if reg.FindString(contents) == "" { return errors.Errorf("could not find definition for 's.foo' function in schema.sql") } - reg = regexp.MustCompile("^foo") + reg = regexp.MustCompile(`^CREATE FUNCTION public\.foo`) if reg.FindString(contents) != "" { return errors.Errorf("found irrelevant function 'foo' in schema.sql") } - reg = regexp.MustCompile("s.a") + reg = regexp.MustCompile(`s\.a`) if reg.FindString(contents) == "" { return errors.Errorf("could not find definition for relation 's.a' in schema.sql") } @@ -466,15 +466,15 @@ CREATE TABLE users(id UUID DEFAULT gen_random_uuid() PRIMARY KEY, promo_id INT R checkBundle( t, fmt.Sprint(rows), "s.bar", func(name, contents string) error { if name == "schema.sql" { - reg := regexp.MustCompile("s.bar") + reg := regexp.MustCompile(`s\.bar`) if reg.FindString(contents) == "" { return errors.Errorf("could not find definition for 's.bar' procedure in schema.sql") } - reg = regexp.MustCompile("^bar") + reg = regexp.MustCompile(`^CREATE PROCEDURE public\.bar`) if reg.FindString(contents) != "" { return errors.Errorf("Found irrelevant procedure 'bar' in schema.sql") } - reg = regexp.MustCompile("s.a") + reg = regexp.MustCompile(`s\.a`) if reg.FindString(contents) == "" { return errors.Errorf("could not find definition for relation 's.a' in schema.sql") }