diff --git a/pkg/ccl/testccl/workload/schemachange/schema_change_external_test.go b/pkg/ccl/testccl/workload/schemachange/schema_change_external_test.go index 9d532ea4d38d..a6706ae70af0 100644 --- a/pkg/ccl/testccl/workload/schemachange/schema_change_external_test.go +++ b/pkg/ccl/testccl/workload/schemachange/schema_change_external_test.go @@ -13,6 +13,7 @@ import ( gosql "database/sql" "fmt" "net/url" + "os" "strconv" "testing" "time" @@ -63,23 +64,38 @@ func TestWorkload(t *testing.T) { tdb.Exec(t, "GRANT admin TO testuser") tdb.Exec(t, "SET CLUSTER SETTING sql.log.all_statements.enabled = true") - // Grab a backup and also print the namespace and descriptor tables upon - // failure. - // It's not clear how helpful this actually is but it doesn't hurt. - printRows := func(rows *gosql.Rows) { + dumpRows := func(name string, rows *gosql.Rows) { t.Helper() mat, err := sqlutils.RowsToStrMatrix(rows) require.NoError(t, err) - fmt.Printf("rows:\n%s", sqlutils.MatrixToStr(mat)) + require.NoError(t, os.WriteFile(fmt.Sprintf("%s/%s.rows", dir, name), []byte(sqlutils.MatrixToStr(mat)), 0666)) } + + // Grab a backup, dump the namespace and descriptor tables upon failure. defer func() { if !t.Failed() { return } - printRows(tdb.Query(t, "SELECT id, encode(descriptor, 'hex') FROM system.descriptor")) - printRows(tdb.Query(t, "SELECT * FROM system.namespace")) + // Dump namespace and descriptor in their raw format. This is useful for + // processing results with some degree of scripting. + dumpRows("namespace", tdb.Query(t, `SELECT * FROM system.namespace`)) + dumpRows("descriptor", tdb.Query(t, "SELECT id, encode(descriptor, 'hex') FROM system.descriptor")) + // Dump out a more human readable version of the above as well to allow for + // easy debugging by hand. + // NB: A LEFT JOIN is used here because not all descriptors (looking at you + // functions) have namespace entries. + dumpRows("ns-desc-json", tdb.Query(t, ` + SELECT + "parentID", + "parentSchemaID", + descriptor.id, + name, + crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', descriptor) + FROM system.descriptor + LEFT JOIN system.namespace ON namespace.id = descriptor.id + `)) tdb.Exec(t, "BACKUP DATABASE schemachange TO 'nodelocal://1/backup'") - t.Logf("backup and tracing data in %s", dir) + t.Logf("backup, tracing data, and system table dumps in %s", dir) }() pgURL, cleanup := sqlutils.PGUrl(t, tc.Server(0).AdvSQLAddr(), t.Name(), url.User("testuser")) diff --git a/pkg/workload/schemachange/BUILD.bazel b/pkg/workload/schemachange/BUILD.bazel index c515646dc88f..75b0153a0cbd 100644 --- a/pkg/workload/schemachange/BUILD.bazel +++ b/pkg/workload/schemachange/BUILD.bazel @@ -22,7 +22,6 @@ go_library( deps = [ "//pkg/clusterversion", "//pkg/roachpb", - "//pkg/security/username", "//pkg/sql/catalog/catpb", "//pkg/sql/catalog/colinfo", "//pkg/sql/parser", diff --git a/pkg/workload/schemachange/operation_generator.go b/pkg/workload/schemachange/operation_generator.go index 3a98f6195248..88c8b921c1c7 100644 --- a/pkg/workload/schemachange/operation_generator.go +++ b/pkg/workload/schemachange/operation_generator.go @@ -24,7 +24,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/parser" @@ -883,6 +882,9 @@ func (og *operationGenerator) addForeignKeyConstraint( stmt.potentialExecErrors.add(pgcode.ForeignKeyViolation) og.potentialCommitErrors.add(pgcode.ForeignKeyViolation) + // TODO why did I add this?? + stmt.potentialExecErrors.add(pgcode.FeatureNotSupported) + // It's possible for the table to be dropped concurrently, while we are running // validation. In which case a potential commit error is an undefined table // error. @@ -2530,116 +2532,192 @@ func (og *operationGenerator) setColumnType(ctx context.Context, tx pgx.Tx) (*op func (og *operationGenerator) alterTableAlterPrimaryKey( ctx context.Context, tx pgx.Tx, ) (*opStmt, error) { - type Column struct { - Name tree.Name - Nullable bool - Unique bool + // Primary Keys are backed by a unique index, therefore we can only use + // columns that are of an indexable type. This information is only available + // via the colinfo package (not SQL) and is subject to change across + // versions. To eliminate the chance of flakes, rely on this allow list to do + // the filtering. As this list is static and non-exhaustive, we're trading a + // bit of coverage for stability. It may be worth while to add index-ability + // information to `SHOW COLUMNS` or an internal SQL function in the future. + indexableFamilies := []string{ + "DecimalFamily", + "IntFamily", + "StringFamily", + "UuidFamily", + } + + q := With([]CTE{ + {"descriptors", descJSONQuery}, + {"tables", tableDescQuery}, + {"columns", colDescQuery}, + }, ` + SELECT + json_array_length(table_descriptor->'mutation') > 0 AS table_undergoing_schema_change, + quote_ident(schema_id::REGNAMESPACE::TEXT) || '.' || quote_ident(table_name) AS table_name, + quote_ident("column"->>'name') AS column_name, + COALESCE(("column"->'nullable')::bool, false) AS is_nullable, + ("column"->>'computedExpr' = '') AS is_computed, + (("column"->'type'->>'family') = ANY($1)) AS is_indexable, + (NOT EXISTS( + SELECT * + FROM crdb_internal.table_indexes + JOIN crdb_internal.index_columns USING (descriptor_id) + WHERE table_indexes.is_inverted + AND table_indexes.descriptor_id = columns.table_id + AND index_columns.column_id = (columns."column"->'id')::int8 + )) AS is_in_inverted_index, + (EXISTS( + SELECT * + FROM crdb_internal.table_indexes + JOIN crdb_internal.index_columns USING (descriptor_id) + WHERE table_indexes.is_unique + AND table_indexes.descriptor_id = columns.table_id + AND index_columns.column_id = (columns."column"->'id')::int8 + )) AS is_unique + FROM columns + WHERE NOT ( + COALESCE(("column"->'hidden')::bool, false) + OR COALESCE(("column"->'inaccessible')::bool, false) + )`) + + columns, err := Collect(ctx, og, tx, pgx.RowToMap, q, indexableFamilies) + if err != nil { + return nil, err + } + + // Group columns by table for convenience in our templates. This could have + // been done within SQL but I didn't want to fight with unmarshalling nested + // JSON fields. + byTable := map[string][]map[string]any{} + for i, col := range columns { + byTable[col["table_name"].(string)] = append( + byTable[col["table_name"].(string)], + columns[i], + ) } - rowToTableName := func(row pgx.CollectableRow) (*tree.UnresolvedName, error) { - var schema string - var name string - if err := row.Scan(&schema, &name); err != nil { - return nil, err - } - return tree.NewUnresolvedName(schema, name), nil + tables := make([]map[string]any, 0, len(byTable)) + for table_name, grouped := range byTable { + tables = append(tables, map[string]any{ + "table_name": table_name, + "table_undergoing_schema_change": grouped[0]["table_undergoing_schema_change"].(bool), + "columns": grouped, + }) } - columnsFrom := func(table tree.NodeFormatter) ([]Column, error) { - query := With([]CTE{ - {"stats", fmt.Sprintf(`SELECT * FROM [SHOW STATISTICS FOR TABLE %v]`, table)}, - {"unique_columns", `SELECT column_names[1] AS name FROM stats WHERE row_count = distinct_count AND array_length(column_names, 1) = 1`}, - }, fmt.Sprintf(`SELECT column_name, is_nullable, EXISTS(SELECT * FROM unique_columns WHERE name = column_name) FROM [SHOW COLUMNS FROM %v] WHERE NOT is_hidden`, table)) + // Our big query can only check if there are any unique indexes on columns. + // We'll also want to check if any columns happen to be unique rather than + // being constrained to uniqueness. + fillIsUnique := func(table map[string]any) error { + // Cache uniqueness checks. They're expensive and might run twice in some + // weird cases with the Generate helper. + if _, ok := table["unique_check"]; ok { + return nil + } - return Collect(ctx, og, tx, pgx.RowToStructByPos[Column], query) - } + table["unique_check"] = true + + var b strings.Builder + fmt.Fprintf(&b, `SELECT * FROM VALUES (`) + for _, column := range table["columns"].([]map[string]any) { + // If this column is already known to be unique, don't bother checking + // it. This should only happen if there's a unique constraint on the + // column. + if column["is_unique"].(bool) { + fmt.Fprintf(&b, `(SELECT true)`) + } else { + fmt.Fprintf(&b, `(SELECT EXISTS(SELECT 1 FROM %s GROUP BY %s HAVING count(*) > 1))`, table["table_name"], column["column_name"]) + } + } + fmt.Fprintf(&b, `)`) - ctes := []CTE{ - {"tables", `SELECT * FROM [SHOW TABLES] WHERE type = 'table'`}, - {"descriptors", descJSONQuery}, - {"tables_undergoing_schema_changes", `SELECT id FROM descriptors WHERE descriptor ? 'table' AND json_array_length(descriptor->'table'->'mutations') > 0`}, - } + results, err := Collect(ctx, og, tx, pgx.RowTo[bool], b.String()) + if err != nil { + return err + } - tablesUndergoingSchemaChangesQuery := With(ctes, `SELECT schema_name, table_name FROM tables WHERE NOT EXISTS(SELECT * FROM tables_undergoing_schema_changes WHERE id = (schema_name || '.' || table_name)::regclass::oid)`) - tablesNotUndergoingSchemaChangesQuery := With(ctes, `SELECT schema_name, table_name FROM tables WHERE EXISTS(SELECT * FROM tables_undergoing_schema_changes WHERE id = (schema_name || '.' || table_name)::regclass::oid)`) + for i, unique := range results { + table["columns"].([]map[string]any)[i]["is_unique"] = unique + } + + return nil + } - var table *tree.UnresolvedName stmt, code, err := Generate[*tree.AlterTable](og.params.rng, og.produceError(), []GenerationCase{ // IF EXISTS should noop if the table doesn't exist. {pgcode.SuccessfulCompletion, `ALTER TABLE IF EXISTS "NonExistentTable" ALTER PRIMARY KEY USING COLUMNS ("IrrelevantColumn")`}, // Targeting a table that doesn't exist should error out. {pgcode.UndefinedTable, `ALTER TABLE "NonExistentTable" ALTER PRIMARY KEY USING COLUMNS ("IrrelevantColumn")`}, // Targeting a column that doesn't exist should error out. - {pgcode.InvalidSchemaDefinition, `ALTER TABLE {TableNotUnderGoingSchemaChange} ALTER PRIMARY KEY USING COLUMNS ("NonExistentColumn")`}, + {pgcode.UndefinedColumn, `{ with TableNotUnderGoingSchemaChange } ALTER TABLE { .table_name } ALTER PRIMARY KEY USING COLUMNS ("NonExistentColumn") { end }`}, // NonUniqueColumns can't be used as PKs. - {pgcode.UniqueViolation, `ALTER TABLE {TableNotUnderGoingSchemaChange} ALTER PRIMARY KEY USING COLUMNS ({NonUniqueColumns})`}, + {pgcode.UniqueViolation, `{ with TableNotUnderGoingSchemaChange } ALTER TABLE { .table_name } ALTER PRIMARY KEY USING COLUMNS ({ . | Unique false | Nullable false | Generated false | Indexable true | InInvertedIndex false | Columns }) { end }`}, // NullableColumns can't be used as PKs. - {pgcode.InvalidSchemaDefinition, `ALTER TABLE {TableNotUnderGoingSchemaChange} ALTER PRIMARY KEY USING COLUMNS ({NullableColumns})`}, + {pgcode.InvalidSchemaDefinition, `{ with TableNotUnderGoingSchemaChange } ALTER TABLE { .table_name } ALTER PRIMARY KEY USING COLUMNS ({ . | Unique true | Nullable true | Generated false | Indexable true | InInvertedIndex false | Columns }) { end }`}, + // UnindexableColumns can't be used as PKs. + {pgcode.InvalidSchemaDefinition, `{ with TableNotUnderGoingSchemaChange } ALTER TABLE { .table_name } ALTER PRIMARY KEY USING COLUMNS ({ . | Unique true | Nullable false | Generated false | Indexable false | InInvertedIndex false | Columns }) { end }`}, + // TODO(sql-foundations): Columns that have an inverted index can't be used + // as a primary key. This check isn't 100% correct because we only care + // about the final column in an inverted index and we're checking if + // columns are in an inverted index at all. + // {pgcode.InvalidSchemaDefinition, `{ with TableNotUnderGoingSchemaChange } ALTER TABLE { .table_name } ALTER PRIMARY KEY USING COLUMNS ({ . | Unique true | Nullable false | Generated false | Indexable true | InInvertedIndex true | Columns }) { end }`}, // Tables undergoing a schema change may not have their PK changed. - // TODO(chrisseto): This case doesn't cause errors as expected. + // TODO(sql-foundations): This case doesn't cause errors as expected. // {pgcode.Code{}, `ALTER TABLE {TableUnderGoingSchemaChange} ALTER PRIMARY KEY USING COLUMNS ({UniqueNotNullableColumns})`}, // Successful cases. - {pgcode.SuccessfulCompletion, `ALTER TABLE {TableNotUnderGoingSchemaChange} ALTER PRIMARY KEY USING COLUMNS ({UniqueNotNullableColumns})`}, - {pgcode.SuccessfulCompletion, `ALTER TABLE {TableNotUnderGoingSchemaChange} ALTER PRIMARY KEY USING COLUMNS ({UniqueNotNullableColumns}) USING HASH`}, - // TODO(chrisseto): Add support for hash parameters and storage parameters. + {pgcode.SuccessfulCompletion, `{ with TableNotUnderGoingSchemaChange } ALTER TABLE { .table_name } ALTER PRIMARY KEY USING COLUMNS ({ . | Unique true | Nullable false | Generated false | Indexable true | InInvertedIndex false | Columns }) { end }`}, + {pgcode.SuccessfulCompletion, `{ with TableNotUnderGoingSchemaChange } ALTER TABLE { .table_name } ALTER PRIMARY KEY USING COLUMNS ({ . | Unique true | Nullable false | Generated false | Indexable true | InInvertedIndex false | Columns }) USING HASH { end }`}, + // TODO(sql-foundations): Add support for hash parameters and storage parameters. }, template.FuncMap{ - "TableNotUnderGoingSchemaChange": func() (*tree.UnresolvedName, error) { - tables, err := Collect(ctx, og, tx, rowToTableName, tablesNotUndergoingSchemaChangesQuery) - if err != nil { - return nil, err - } - table, err = PickOne(og.params.rng, tables) - return table, err + "TableNotUnderGoingSchemaChange": func() (map[string]any, error) { + tbls := util.Filter(tables, func(table map[string]any) bool { + return !table["table_undergoing_schema_change"].(bool) + }) + return PickOne(og.params.rng, tbls) }, - "TableUnderGoingSchemaChange": func() (*tree.UnresolvedName, error) { - tables, err := Collect(ctx, og, tx, rowToTableName, tablesUndergoingSchemaChangesQuery) - if err != nil { - return nil, err - } - table, err = PickOne(og.params.rng, tables) - return table, err + "TableUnderGoingSchemaChange": func() (map[string]any, error) { + tbls := util.Filter(tables, func(table map[string]any) bool { + return table["table_undergoing_schema_change"].(bool) + }) + return PickOne(og.params.rng, tbls) }, - "NullableColumns": func() (Values, error) { - columns, err := columnsFrom(table) - if err != nil { - return nil, err - } - - names := util.Map(util.Filter(columns, func(c Column) bool { - return c.Nullable - }), func(c Column) *tree.Name { - return &c.Name + "Columns": func(table map[string]any) (string, error) { + selected, err := PickAtLeast(og.params.rng, 1, table["columns"].([]map[string]any)) + names := util.Map(selected, func(col map[string]any) string { return col["column_name"].(string) }) + return strings.Join(names, ", "), err + }, + "Nullable": func(nullable bool, table map[string]any) map[string]any { + table["columns"] = util.Filter(table["columns"].([]map[string]any), func(col map[string]any) bool { + return col["is_nullable"].(bool) == nullable }) - - return AsValues(PickAtLeast(og.params.rng, 1, names)) + return table }, - "NonUniqueColumns": func() (Values, error) { - columns, err := columnsFrom(table) - if err != nil { + "Unique": func(unique bool, table map[string]any) (map[string]any, error) { + if err := fillIsUnique(table); err != nil { return nil, err } - - names := util.Map(util.Filter(columns, func(c Column) bool { - return !c.Nullable && !c.Unique - }), func(c Column) *tree.Name { - return &c.Name + table["columns"] = util.Filter(table["columns"].([]map[string]any), func(col map[string]any) bool { + return col["is_unique"].(bool) == unique }) - - return AsValues(PickAtLeast(og.params.rng, 1, names)) + return table, nil }, - "UniqueNotNullableColumns": func() (Values, error) { - columns, err := columnsFrom(table) - if err != nil { - return nil, err - } - - names := util.Map(util.Filter(columns, func(c Column) bool { - return !c.Nullable && c.Unique - }), func(c Column) *tree.Name { - return &c.Name + "Generated": func(unique bool, table map[string]any) map[string]any { + table["columns"] = util.Filter(table["columns"].([]map[string]any), func(col map[string]any) bool { + return col["is_unique"].(bool) == unique }) - - return AsValues(PickAtLeast(og.params.rng, 1, names)) + return table + }, + "Indexable": func(indexable bool, table map[string]any) map[string]any { + table["columns"] = util.Filter(table["columns"].([]map[string]any), func(col map[string]any) bool { + return col["is_indexable"].(bool) == indexable + }) + return table + }, + "InInvertedIndex": func(inIndex bool, table map[string]any) map[string]any { + table["columns"] = util.Filter(table["columns"].([]map[string]any), func(col map[string]any) bool { + return col["is_in_inverted_index"].(bool) == inIndex + }) + return table }, }) if err != nil { @@ -2687,6 +2765,40 @@ func (og *operationGenerator) survive(ctx context.Context, tx pgx.Tx) (*opStmt, return stmt, nil } +func (og *operationGenerator) commentOn(ctx context.Context, tx pgx.Tx) (*opStmt, error) { + q := With([]CTE{ + {"descriptors", descJSONQuery}, + {"tables", tableDescQuery}, + {"columns", `SELECT schema_id::REGNAMESPACE::TEXT as schema_name, name AS table_name, jsonb_array_elements(descriptor->'table'->'columns') AS column FROM tables`}, + {"indexes", `SELECT schema_id::REGNAMESPACE::TEXT as schema_name, name AS table_name, jsonb_array_elements(descriptor->'table'->'indexes') AS index FROM tables`}, + {"constraints", `SELECT schema_id::REGNAMESPACE::TEXT as schema_name, name AS table_name, jsonb_array_elements(descriptor->'table'->'checks') AS constraint FROM tables`}, + }, ` + SELECT 'SCHEMA ' || quote_ident(schema_name) FROM [SHOW SCHEMAS] WHERE owner != 'node' + UNION ALL + SELECT 'TABLE ' || quote_ident(schema_name) || '.' || quote_ident(table_name) FROM [SHOW TABLES] WHERE type = 'table' + UNION ALL + SELECT 'COLUMN ' || quote_ident(schema_name) || '.' || quote_ident(table_name) || '.' || quote_ident("column"->>'name') FROM columns + UNION ALL + SELECT 'INDEX ' || quote_ident(schema_name) || '.' || quote_ident(table_name) || '@' || quote_ident("index"->>'name') FROM indexes + UNION ALL + SELECT 'CONSTRAINT ' || quote_ident("constraint"->>'name') || ' ON ' || quote_ident(schema_name) || '.' || quote_ident(table_name) FROM constraints + `) + + commentables, err := Collect(ctx, og, tx, pgx.RowTo[string], q) + if err != nil { + return nil, err + } + + picked, err := PickOne(og.params.rng, commentables) + if err != nil { + return nil, err + } + + stmt := makeOpStmt(OpStmtDDL) + stmt.sql = fmt.Sprintf(`COMMENT ON %s IS 'comment from the RSW'`, picked) + return stmt, nil +} + func (og *operationGenerator) insertRow(ctx context.Context, tx pgx.Tx) (stmt *opStmt, err error) { tableName, err := og.randTable(ctx, tx, og.pctExisting(true), "") if err != nil { @@ -3292,12 +3404,12 @@ func (og *operationGenerator) randParentColumnForFkRelation( )`, subQuery.String())).Scan(&tableSchema, &tableName, &columnName, &typName, &nullable) if err != nil { if rbErr := nestedTxn.Rollback(ctx); rbErr != nil { - err = errors.CombineErrors(err, rbErr) + err = errors.CombineErrors(err, errors.WithStack(rbErr)) } return nil, nil, err } if err = nestedTxn.Commit(ctx); err != nil { - return nil, nil, err + return nil, nil, errors.WithStack(err) } columnToReturn := column{ @@ -3675,8 +3787,12 @@ func (og *operationGenerator) createSchema(ctx context.Context, tx pgx.Tx) (*opS opStmt.expectedExecErrors.add(pgcode.DuplicateSchema) } - // TODO(jayshrivastava): Support authorization - stmt := randgen.MakeSchemaName(ifNotExists, schemaName, tree.MakeRoleSpecWithRoleName(username.RootUserName().Normalized())) + // TODO(sql-foundations): CREATE SCHEMA AUTHORIZATION is not currently + // support in the DSC. Either add support and re-enable it here or gate the + // AUTHORIZATION aspect by checking if the DSC is enabled or not. Previously, + // `username.RootUserName().Normalized()` was used for + // MakeRoleSpecWithRoleName. + stmt := randgen.MakeSchemaName(ifNotExists, schemaName, tree.MakeRoleSpecWithRoleName("")) opStmt.sql = tree.Serialize(stmt) return opStmt, nil } @@ -3835,6 +3951,152 @@ func (og *operationGenerator) createFunction(ctx context.Context, tx pgx.Tx) (*o }), nil } +func (og *operationGenerator) dropFunction(ctx context.Context, tx pgx.Tx) (*opStmt, error) { + q := With([]CTE{ + {"descriptors", descJSONQuery}, + {"functions", functionDescsQuery}, + }, `SELECT + quote_ident(schema_id::REGNAMESPACE::TEXT) || '.' || quote_ident(name) || '(' || array_to_string(funcargs, ', ') || ')' + FROM functions + JOIN LATERAL ( + SELECT + COALESCE(array_agg(quote_ident(typnamespace::REGNAMESPACE::TEXT) || '.' || quote_ident(typname)), '{}') AS funcargs + FROM pg_catalog.pg_type + JOIN LATERAL ( + SELECT unnest(proargtypes) AS oid FROM pg_catalog.pg_proc WHERE oid = (id + 100000) + ) args ON args.oid = pg_type.oid + ) funcargs ON TRUE + `, + ) + + functions, err := Collect(ctx, og, tx, pgx.RowTo[string], q) + if err != nil { + return nil, err + } + + stmt, expectedCode, err := Generate[*tree.DropRoutine](og.params.rng, og.produceError(), []GenerationCase{ + {pgcode.UndefinedFunction, `DROP FUNCTION "NoSuchFunction"`}, + {pgcode.SuccessfulCompletion, `DROP FUNCTION IF EXISTS "NoSuchFunction"`}, + {pgcode.SuccessfulCompletion, `DROP FUNCTION { Function }`}, + }, template.FuncMap{ + "Function": func() (string, error) { + return PickOne(og.params.rng, functions) + }, + }) + + if err != nil { + return nil, err + } + return newOpStmt(stmt, codesWithConditions{ + {expectedCode, true}, + }), nil +} + +func (og *operationGenerator) alterFunctionRename(ctx context.Context, tx pgx.Tx) (*opStmt, error) { + q := With([]CTE{ + {"descriptors", descJSONQuery}, + {"functions", functionDescsQuery}, + }, `SELECT + quote_ident(schema_id::REGNAMESPACE::TEXT) AS schema, + quote_ident(name) AS name, + quote_ident(schema_id::REGNAMESPACE::TEXT) || '.' || quote_ident(name) || '(' || array_to_string(funcargs, ', ') || ')' AS qualified_name + FROM functions + JOIN LATERAL ( + SELECT + COALESCE(array_agg(quote_ident(typnamespace::REGNAMESPACE::TEXT) || '.' || quote_ident(typname)), '{}') AS funcargs + FROM pg_catalog.pg_type + JOIN LATERAL ( + SELECT unnest(proargtypes) AS oid FROM pg_catalog.pg_proc WHERE oid = (id + 100000) + ) args ON args.oid = pg_type.oid + ) funcargs ON TRUE + `) + + functions, err := Collect(ctx, og, tx, pgx.RowToMap, q) + if err != nil { + return nil, err + } + + stmt, expectedCode, err := Generate[*tree.AlterRoutineRename](og.params.rng, og.produceError(), []GenerationCase{ + {pgcode.UndefinedFunction, `ALTER FUNCTION "NoSuchFunction" RENAME TO "IrrelevantFunctionName"`}, + // TODO(chrisseto): Neither of these seem to work as expected. Renaming a + // function to itself within a SQL shell results in conflicts but doesn't + // seem to reliably error in the context of the RSW. I'm guessing this has + // something to do with search paths and/or function overloads. + // {pgcode.DuplicateFunction, `{ with ExistingFunction } ALTER FUNCTION { .qualified_name } RENAME TO { ConflictingName . } { end }`}, + // {pgcode.DuplicateFunction, `{ with ExistingFunction } ALTER FUNCTION { .qualified_name } RENAME TO { .name } { end }`}, + {pgcode.SuccessfulCompletion, `ALTER FUNCTION { (ExistingFunction).qualified_name } RENAME TO { UniqueName }`}, + }, template.FuncMap{ + "UniqueName": func() *tree.Name { + name := tree.Name(fmt.Sprintf("udf_%d", og.newUniqueSeqNum())) + return &name + }, + "ExistingFunction": func() (map[string]any, error) { + return PickOne(og.params.rng, functions) + }, + "ConflictingName": func(existing map[string]any) (string, error) { + selected, err := PickOne(og.params.rng, util.Filter(functions, func(other map[string]any) bool { + return other["schema"] == existing["schema"] && other["name"] != existing["name"] + })) + if err != nil { + return "", err + } + return selected["name"].(string), nil + }, + }) + + if err != nil { + return nil, err + } + + return newOpStmt(stmt, codesWithConditions{ + {expectedCode, true}, + }), nil +} + +func (og *operationGenerator) alterFunctionSetSchema( + ctx context.Context, tx pgx.Tx, +) (*opStmt, error) { + functionsQuery := With([]CTE{ + {"descriptors", descJSONQuery}, + {"functions", functionDescsQuery}, + }, `SELECT quote_ident(schema_id::REGNAMESPACE::TEXT) || '.' || quote_ident(name) FROM functions`) + + schemasQuery := With([]CTE{ + {"descriptors", descJSONQuery}, + }, `SELECT quote_ident(name) FROM descriptors WHERE descriptor ? 'schema'`) + + functions, err := Collect(ctx, og, tx, pgx.RowTo[string], functionsQuery) + if err != nil { + return nil, err + } + + schemas, err := Collect(ctx, og, tx, pgx.RowTo[string], schemasQuery) + if err != nil { + return nil, err + } + + stmt, expectedCode, err := Generate[*tree.AlterRoutineSetSchema](og.params.rng, og.produceError(), []GenerationCase{ + {pgcode.UndefinedFunction, `ALTER FUNCTION "NoSuchFunction" SET SCHEMA "IrrelevantSchema"`}, + {pgcode.InvalidSchemaName, `ALTER FUNCTION { Function } SET SCHEMA "NoSuchSchema"`}, + // NB: It's considered valid to set a function's schema to the schema it already exists within. + {pgcode.SuccessfulCompletion, `ALTER FUNCTION { Function } SET SCHEMA { Schema }`}, + }, template.FuncMap{ + "Function": func() (string, error) { + return PickOne(og.params.rng, functions) + }, + "Schema": func() (string, error) { + return PickOne(og.params.rng, schemas) + }, + }) + + if err != nil { + return nil, err + } + return newOpStmt(stmt, codesWithConditions{ + {expectedCode, true}, + }), nil +} + func (og *operationGenerator) selectStmt(ctx context.Context, tx pgx.Tx) (stmt *opStmt, err error) { const maxTablesForSelect = 3 const maxColumnsForSelect = 16 diff --git a/pkg/workload/schemachange/optype.go b/pkg/workload/schemachange/optype.go index e3a5bd2f6916..7765061b2734 100644 --- a/pkg/workload/schemachange/optype.go +++ b/pkg/workload/schemachange/optype.go @@ -89,6 +89,10 @@ const ( alterDatabaseAddSuperRegion // ALTER DATABASE ADD SUPER REGION VALUES ... alterDatabaseDropSuperRegion // ALTER DATABASE DROP SUPER REGION + // ALTER FUNCTION ... + alterFunctionRename // ALTER FUNCTION RENAME TO + alterFunctionSetSchema // ALTER FUNCTION SET SCHEMA + // ALTER TABLE ... alterTableAddColumn // ALTER TABLE
ADD [COLUMN] @@ -122,8 +126,13 @@ const ( createView // CREATE VIEW AS createFunction // CREATE FUNCTION ... + // COMMENT ON ... + + commentOn // COMMENT ON [SCHEMA | TABLE | INDEX | COLUMN | CONSTRAINT] IS + // DROP ... + dropFunction // DROP FUNCTION dropIndex // DROP INDEX @
dropSchema // DROP SCHEMA dropSequence // DROP SEQUENCE @@ -137,9 +146,7 @@ const ( // alterDefaultPrivileges // alterFunctionDepExtension // alterFunctionOptions - // alterFunctionRename // alterFunctionSetOwner - // alterFunctionSetSchema // alterIndex // alterIndexPartitionBy // alterIndexVisible @@ -166,19 +173,13 @@ const ( // alterTypeRename // alterTypeRenameValue // alterTypeSetSchema - // commentOnColumn - // commentOnConstraint // commentOnDatabase - // commentOnIndex - // commentOnSchema - // commentOnTable // createDatabase // createRole // createStats // createStatsOptions // createType // dropDatabase - // dropFunction // dropOwnedBy // dropRole // DROP ROLE // dropType // DROP TYPE @@ -210,6 +211,8 @@ var opFuncs = []func(*operationGenerator, context.Context, pgx.Tx) (*opStmt, err alterDatabaseDropSuperRegion: (*operationGenerator).alterDatabaseDropSuperRegion, alterDatabasePrimaryRegion: (*operationGenerator).primaryRegion, alterDatabaseSurvivalGoal: (*operationGenerator).survive, + alterFunctionRename: (*operationGenerator).alterFunctionRename, + alterFunctionSetSchema: (*operationGenerator).alterFunctionSetSchema, alterTableAddColumn: (*operationGenerator).addColumn, alterTableAddConstraint: (*operationGenerator).addConstraint, alterTableAddConstraintForeignKey: (*operationGenerator).addForeignKeyConstraint, @@ -226,6 +229,7 @@ var opFuncs = []func(*operationGenerator, context.Context, pgx.Tx) (*opStmt, err alterTableSetColumnDefault: (*operationGenerator).setColumnDefault, alterTableSetColumnNotNull: (*operationGenerator).setColumnNotNull, alterTypeDropValue: (*operationGenerator).alterTypeDropValue, + commentOn: (*operationGenerator).commentOn, createFunction: (*operationGenerator).createFunction, createIndex: (*operationGenerator).createIndex, createSchema: (*operationGenerator).createSchema, @@ -234,6 +238,7 @@ var opFuncs = []func(*operationGenerator, context.Context, pgx.Tx) (*opStmt, err createTableAs: (*operationGenerator).createTableAs, createTypeEnum: (*operationGenerator).createEnum, createView: (*operationGenerator).createView, + dropFunction: (*operationGenerator).dropFunction, dropIndex: (*operationGenerator).dropIndex, dropSchema: (*operationGenerator).dropSchema, dropSequence: (*operationGenerator).dropSequence, @@ -252,43 +257,47 @@ var opWeights = []int{ validate: 2, // validate twice more often // DDL Operations - alterTableAddColumn: 1, - alterTableDropConstraint: 0, // TODO(spaskob): unimplemented - alterTableAddConstraintForeignKey: 1, // Tentatively re-enabled, see #91195. alterDatabaseAddRegion: 1, alterDatabaseAddSuperRegion: 0, // Disabled and tracked with #111299 alterDatabaseDropSuperRegion: 0, // Disabled and tracked with #111299 alterDatabasePrimaryRegion: 0, // Disabled and tracked with #83831 alterDatabaseSurvivalGoal: 0, // Disabled and tracked with #83831 + alterFunctionRename: 0, // Disabled and tracked with #116794. + alterFunctionSetSchema: 0, // Disabled and tracked with #116794. + alterTableAddColumn: 1, + alterTableAddConstraintForeignKey: 1, // Tentatively re-enabled, see #91195. alterTableAddConstraintUnique: 0, + alterTableAlterColumnType: 0, // Disabled and tracked with #66662. + alterTableAlterPrimaryKey: 1, + alterTableDropColumn: 0, + alterTableDropColumnDefault: 1, + alterTableDropConstraint: 0, // TODO(spaskob): unimplemented + alterTableDropNotNull: 1, + alterTableDropStored: 1, alterTableLocality: 1, + alterTableRenameColumn: 1, + alterTableSetColumnDefault: 1, + alterTableSetColumnNotNull: 1, + alterTypeDropValue: 0, // Disabled and tracked with #114844, #113859, and #115612. + commentOn: 0, // Disabled and tracked with #116795. + createFunction: 1, createIndex: 1, + createSchema: 1, createSequence: 1, createTable: 1, createTableAs: 1, - createView: 1, createTypeEnum: 1, - createSchema: 1, - createFunction: 1, - alterTableDropColumn: 0, - alterTableDropColumnDefault: 1, - alterTableDropNotNull: 1, - alterTableDropStored: 1, + createView: 1, + dropFunction: 0, // Disabled and tracked with #116794. dropIndex: 1, + dropSchema: 0, // Disabled and tracked with 116792. dropSequence: 1, dropTable: 1, dropView: 1, - alterTypeDropValue: 0, // Disabled and tracked with #114844, #113859, and #115612. - dropSchema: 1, - alterTableRenameColumn: 1, renameIndex: 1, renameSequence: 1, renameTable: 1, renameView: 1, - alterTableSetColumnDefault: 1, - alterTableSetColumnNotNull: 1, - alterTableAlterPrimaryKey: 1, - alterTableAlterColumnType: 0, // Disabled and tracked with #66662. } // This workload will maintain its own list of minimal supported versions for @@ -303,9 +312,10 @@ var opDeclarativeVersion = map[opType]clusterversion.Key{ alterTableDropConstraint: clusterversion.MinSupported, alterTableDropNotNull: clusterversion.MinSupported, alterTypeDropValue: clusterversion.MinSupported, + commentOn: clusterversion.MinSupported, createIndex: clusterversion.MinSupported, - createSequence: clusterversion.MinSupported, createSchema: clusterversion.V23_2, + createSequence: clusterversion.MinSupported, dropIndex: clusterversion.MinSupported, dropSchema: clusterversion.MinSupported, dropSequence: clusterversion.MinSupported, diff --git a/pkg/workload/schemachange/optype_string.go b/pkg/workload/schemachange/optype_string.go index 48df7398792d..bcdefdb4b06e 100644 --- a/pkg/workload/schemachange/optype_string.go +++ b/pkg/workload/schemachange/optype_string.go @@ -20,35 +20,39 @@ func _() { _ = x[alterDatabaseSurvivalGoal-9] _ = x[alterDatabaseAddSuperRegion-10] _ = x[alterDatabaseDropSuperRegion-11] - _ = x[alterTableAddColumn-12] - _ = x[alterTableAddConstraint-13] - _ = x[alterTableAddConstraintForeignKey-14] - _ = x[alterTableAddConstraintUnique-15] - _ = x[alterTableAlterColumnType-16] - _ = x[alterTableAlterPrimaryKey-17] - _ = x[alterTableDropColumn-18] - _ = x[alterTableDropColumnDefault-19] - _ = x[alterTableDropConstraint-20] - _ = x[alterTableDropNotNull-21] - _ = x[alterTableDropStored-22] - _ = x[alterTableLocality-23] - _ = x[alterTableRenameColumn-24] - _ = x[alterTableSetColumnDefault-25] - _ = x[alterTableSetColumnNotNull-26] - _ = x[alterTypeDropValue-27] - _ = x[createTypeEnum-28] - _ = x[createIndex-29] - _ = x[createSchema-30] - _ = x[createSequence-31] - _ = x[createTable-32] - _ = x[createTableAs-33] - _ = x[createView-34] - _ = x[createFunction-35] - _ = x[dropIndex-36] - _ = x[dropSchema-37] - _ = x[dropSequence-38] - _ = x[dropTable-39] - _ = x[dropView-40] + _ = x[alterFunctionRename-12] + _ = x[alterFunctionSetSchema-13] + _ = x[alterTableAddColumn-14] + _ = x[alterTableAddConstraint-15] + _ = x[alterTableAddConstraintForeignKey-16] + _ = x[alterTableAddConstraintUnique-17] + _ = x[alterTableAlterColumnType-18] + _ = x[alterTableAlterPrimaryKey-19] + _ = x[alterTableDropColumn-20] + _ = x[alterTableDropColumnDefault-21] + _ = x[alterTableDropConstraint-22] + _ = x[alterTableDropNotNull-23] + _ = x[alterTableDropStored-24] + _ = x[alterTableLocality-25] + _ = x[alterTableRenameColumn-26] + _ = x[alterTableSetColumnDefault-27] + _ = x[alterTableSetColumnNotNull-28] + _ = x[alterTypeDropValue-29] + _ = x[createTypeEnum-30] + _ = x[createIndex-31] + _ = x[createSchema-32] + _ = x[createSequence-33] + _ = x[createTable-34] + _ = x[createTableAs-35] + _ = x[createView-36] + _ = x[createFunction-37] + _ = x[commentOn-38] + _ = x[dropFunction-39] + _ = x[dropIndex-40] + _ = x[dropSchema-41] + _ = x[dropSequence-42] + _ = x[dropTable-43] + _ = x[dropView-44] } func (i opType) String() string { @@ -77,6 +81,10 @@ func (i opType) String() string { return "alterDatabaseAddSuperRegion" case alterDatabaseDropSuperRegion: return "alterDatabaseDropSuperRegion" + case alterFunctionRename: + return "alterFunctionRename" + case alterFunctionSetSchema: + return "alterFunctionSetSchema" case alterTableAddColumn: return "alterTableAddColumn" case alterTableAddConstraint: @@ -125,6 +133,10 @@ func (i opType) String() string { return "createView" case createFunction: return "createFunction" + case commentOn: + return "commentOn" + case dropFunction: + return "dropFunction" case dropIndex: return "dropIndex" case dropSchema: diff --git a/pkg/workload/schemachange/query_util.go b/pkg/workload/schemachange/query_util.go index e997349b839b..be129cdc3bae 100644 --- a/pkg/workload/schemachange/query_util.go +++ b/pkg/workload/schemachange/query_util.go @@ -53,6 +53,29 @@ const ( WHERE "parentID" = (SELECT id FROM system.namespace WHERE name = current_database() AND "parentID" = 0) ` + // tableDescQuery returns the JSONified version of all table descriptors in + // the current database. Views and sequences are NOT included in the result + // set. + // + // [descJSONQuery] must be bound to the name "descriptors". + // + // id::int | schema_id::int | name::text | descriptor::json + tableDescQuery = `SELECT * FROM descriptors WHERE descriptor ? 'table' AND NOT (descriptor->'table' ? 'viewQuery' OR descriptor->'table' ? 'sequenceOpts') ` + + // colDescQuery returns the JSONified version of all table columns in the current database. + // + // [descJSONQuery] must be bound to the name "descriptors". + // [tableDescQuery] must be bound to the name "tables". + // + // schema_id::int | table_id::int | table_name::text | table_descriptor::json | column::json + colDescQuery = `SELECT + schema_id, + tables.id AS table_id, + tables.name AS table_name, + tables.descriptor AS table_descriptor, + json_array_elements(descriptor->'columns') AS column + FROM tables` + // enumDescsQuery returns the JSONified version of all enum descriptors in // the current database. //