From 28a82077097fbc8cd6cb7ba4f4780a172195cb24 Mon Sep 17 00:00:00 2001 From: David Taylor Date: Sat, 13 Jan 2024 15:20:34 +0000 Subject: [PATCH 1/4] cli/demo: expand demo://tenant to postgres:// URL The SQL connection URLs are large and somewhat unwieldy when used with \c to reconnect or in PCR control statements. One particular source of friction, beyond the sheer length of the string making statements harder to read or modify by hand, is that the connection string in it includes values that are specific to a given `demo` session, meaning a subsequent session cannot simply re-run the same commands from the recorded shell history as they contain the ephemeral ports and passwords. This change adds a small special-case to the shell when it is running in demo mode, so that it will replace any occurrence of a string of the form 'demo://' with the pgurl for the virtual cluster of that name on its first server. For example, when used reconnecting the shell itself to a different pgurl: ``` demo@127.0.0.1:26257/system/defaultdb> \c demo://demoapp using new connection URL: postgresql://demo:demo36905@127.0.0.1:26257/defaultdb?options=-ccluster%3Ddemoapp&sslmode=require&sslrootcert=%2FUsers%2Fuser%2F.cockroach-demo%2Fca.crt demo@127.0.0.1:26257/demoapp/defaultdb> \c demo://system using new connection URL: postgresql://demo:demo36905@127.0.0.1:26257/defaultdb?options=-ccluster%3Dsystem&sslmode=require&sslrootcert=%2FUsers%2Fuser%2F.cockroach-demo%2Fca.crt ``` For another example, when used in the content of a statement that will be sent to the database: ``` demo@127.0.0.1:26257/system/defaultdb> create virtual cluster t2 from replication of demoapp on 'demo://system'; CREATE VIRTUAL CLUSTER FROM REPLICATION 0 demo@127.0.0.1:26257/system/defaultdb> show source_cluster_uri FROM [SHOW VIRTUAL CLUSTER t2 WITH REPLICATION STATUS]; | source_cluster_uri | -----+------+--------------------------+--------------+--------------------+-------------------------------------------- postgresql://demo:redacted@127.0.0.1:26257/defaultdb?options=-ccluster%3Dsystem&sslmode=require&sslrootcert=redacted ``` The substitutions are performed after recording the line in the shell history, so that the history is kept in terms of the short name that remains constant across demo sessions. Release note: none. Epic: none. --- pkg/cli/clisqlshell/sql.go | 8 ++++++++ pkg/cli/democluster/api/api.go | 3 +++ pkg/cli/democluster/demo_cluster.go | 11 +++++++++++ 3 files changed, 22 insertions(+) diff --git a/pkg/cli/clisqlshell/sql.go b/pkg/cli/clisqlshell/sql.go index 8faa6bfe3dc3..47dda041b38f 100644 --- a/pkg/cli/clisqlshell/sql.go +++ b/pkg/cli/clisqlshell/sql.go @@ -1392,6 +1392,10 @@ func (c *cliState) doHandleCliCmd(loopState, nextState cliStateEnum) cliStateEnu // to handle it as a statement, so save the history. c.addHistory(c.lastInputLine) + if c.sqlCtx.DemoCluster != nil { + c.lastInputLine = c.sqlCtx.DemoCluster.ExpandShortDemoURLs(c.lastInputLine) + } + // As a convenience to the user, we strip the final semicolon, if // any, in all cases. line := strings.TrimRight(c.lastInputLine, "; ") @@ -1980,6 +1984,10 @@ func (c *cliState) doPrepareStatementLine( c.addHistory(c.concatLines) } + if c.sqlCtx.DemoCluster != nil { + c.concatLines = c.sqlCtx.DemoCluster.ExpandShortDemoURLs(c.concatLines) + } + if !c.iCtx.checkSyntax { return execState } diff --git a/pkg/cli/democluster/api/api.go b/pkg/cli/democluster/api/api.go index 492dcb111b27..e36360713ebd 100644 --- a/pkg/cli/democluster/api/api.go +++ b/pkg/cli/democluster/api/api.go @@ -26,6 +26,9 @@ type DemoCluster interface { // Listing is printed to 'w'. Errors/warnings are printed to 'ew'. ListDemoNodes(w, ew io.Writer, justOne, verbose bool) + // ExpandShortDemoURLs expands `demo://` in a string URLs to `postgres://...`. + ExpandShortDemoURLs(string) string + // AddNode creates a new node with the given locality string. AddNode(ctx context.Context, localityString string) (newNodeID int32, err error) diff --git a/pkg/cli/democluster/demo_cluster.go b/pkg/cli/democluster/demo_cluster.go index 0563819bf44e..c56a3e5ce472 100644 --- a/pkg/cli/democluster/demo_cluster.go +++ b/pkg/cli/democluster/demo_cluster.go @@ -2038,6 +2038,17 @@ func (c *transientCluster) ListDemoNodes(w, ew io.Writer, justOne, verbose bool) } } +func (c *transientCluster) ExpandShortDemoURLs(s string) string { + if !strings.Contains(s, "demo://") { + return s + } + u, err := c.getNetworkURLForServer(context.Background(), 0, false, forSystemTenant) + if err != nil { + return s + } + return regexp.MustCompile(`demo://([[:alnum:]]+)`).ReplaceAllString(s, strings.ReplaceAll(u.String(), "-ccluster%3Dsystem", "-ccluster%3D$1")) +} + func (c *transientCluster) printURLs( w, ew io.Writer, sqlURL *pgurl.URL, From f966213fc27edd3ab125a96061b3048eb332716b Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Fri, 19 Jan 2024 10:15:08 -0500 Subject: [PATCH 2/4] sql: remove unused functions The best code is no code. Release note: None --- pkg/sql/catalog/catalogkeys/keys.go | 8 ----- pkg/sql/catalog/errors.go | 5 --- pkg/sql/catalog/schemaexpr/expr.go | 35 +------------------ .../schemaexpr/select_name_resolution.go | 12 ------- pkg/sql/catalog/table_elements.go | 5 --- pkg/sql/parser/scanner.go | 15 -------- .../scdeps/sctestdeps/database_state.go | 11 ------ .../scexec/scmutationexec/helpers.go | 8 ----- .../internal/rules/release_23_1/helpers.go | 6 ---- .../internal/rules/release_23_2/helpers.go | 6 ---- pkg/sql/schemachanger/screl/scalars.go | 9 ----- pkg/sql/scrub/errors.go | 5 --- pkg/sql/sem/plpgsqltree/utils/plpg_visitor.go | 18 ---------- pkg/sql/sqlerrors/errors.go | 7 ---- 14 files changed, 1 insertion(+), 149 deletions(-) diff --git a/pkg/sql/catalog/catalogkeys/keys.go b/pkg/sql/catalog/catalogkeys/keys.go index 130864dd5385..eef40f874231 100644 --- a/pkg/sql/catalog/catalogkeys/keys.go +++ b/pkg/sql/catalog/catalogkeys/keys.go @@ -298,14 +298,6 @@ func MakeObjectCommentsMetadataPrefix( return encoding.EncodeUvarintAscending(k, uint64(descID)) } -// MakeSubObjectCommentsMetadataPrefix returns the key -func MakeSubObjectCommentsMetadataPrefix( - codec keys.SQLCodec, cmtKey CommentType, descID descpb.ID, subID uint32, -) roachpb.Key { - k := MakeObjectCommentsMetadataPrefix(codec, cmtKey, descID) - return encoding.EncodeUvarintAscending(k, uint64(subID)) -} - // DecodeCommentMetadataID decodes a CommentKey from comments metadata key. func DecodeCommentMetadataID(codec keys.SQLCodec, key roachpb.Key) ([]byte, CommentKey, error) { remaining, tableID, indexID, err := codec.DecodeIndexPrefix(key) diff --git a/pkg/sql/catalog/errors.go b/pkg/sql/catalog/errors.go index b3a259af6690..204b8e0c3098 100644 --- a/pkg/sql/catalog/errors.go +++ b/pkg/sql/catalog/errors.go @@ -177,11 +177,6 @@ func AsFunctionDescriptor(desc Descriptor) (FunctionDescriptor, error) { return typ, nil } -// WrapDescRefErr wraps an error pertaining to a descriptor id. -func WrapDescRefErr(id descpb.ID, err error) error { - return errors.Wrapf(err, "referenced descriptor ID %d", errors.Safe(id)) -} - // WrapDatabaseDescRefErr wraps an error pertaining to a database descriptor id. func WrapDatabaseDescRefErr(id descpb.ID, err error) error { return errors.Wrapf(err, "referenced database ID %d", errors.Safe(id)) diff --git a/pkg/sql/catalog/schemaexpr/expr.go b/pkg/sql/catalog/schemaexpr/expr.go index 3ecd72cf2b4a..5b62ff9eb10e 100644 --- a/pkg/sql/catalog/schemaexpr/expr.go +++ b/pkg/sql/catalog/schemaexpr/expr.go @@ -34,40 +34,6 @@ import ( "github.com/cockroachdb/errors" ) -// DequalifyAndTypeCheckExpr type checks the given expression and returns the -// type-checked expression disregarding volatility. The typed expression, which contains dummyColumns, -// does not support evaluation. -func DequalifyAndTypeCheckExpr( - ctx context.Context, - desc catalog.TableDescriptor, - expr tree.Expr, - semaCtx *tree.SemaContext, - tn *tree.TableName, -) (tree.TypedExpr, error) { - nonDropColumns := desc.NonDropColumns() - sourceInfo := colinfo.NewSourceInfoForSingleTable( - *tn, colinfo.ResultColumnsFromColumns(desc.GetID(), nonDropColumns), - ) - expr, err := dequalifyColumnRefs(ctx, sourceInfo, expr) - if err != nil { - return nil, err - } - - // Replace the column variables with dummyColumns so that they can be - // type-checked. - replacedExpr, _, err := replaceColumnVars(desc, expr) - if err != nil { - return nil, err - } - - typedExpr, err := tree.TypeCheck(ctx, replacedExpr, semaCtx, types.Any) - if err != nil { - return nil, err - } - - return typedExpr, nil -} - // DequalifyAndValidateExprImpl validates that an expression has the given type and // contains no functions with a volatility greater than maxVolatility. The // type-checked and constant-folded expression, the type of the expression, and @@ -77,6 +43,7 @@ func DequalifyAndTypeCheckExpr( // tree.TypedExpr would be dangerous. It contains dummyColumns which do not // support evaluation and are not useful outside the context of type-checking // the expression. +// TODO(mgartner): Rename this function without "Impl". func DequalifyAndValidateExprImpl( ctx context.Context, expr tree.Expr, diff --git a/pkg/sql/catalog/schemaexpr/select_name_resolution.go b/pkg/sql/catalog/schemaexpr/select_name_resolution.go index ddfcf3139def..7f15322a0e4c 100644 --- a/pkg/sql/catalog/schemaexpr/select_name_resolution.go +++ b/pkg/sql/catalog/schemaexpr/select_name_resolution.go @@ -112,15 +112,3 @@ func ResolveNamesUsingVisitor( func (v *NameResolutionVisitor) Err() error { return v.err } - -// MakeNameResolutionVisitor returns initialized name resolution visitor. -func MakeNameResolutionVisitor( - source *colinfo.DataSourceInfo, ivarHelper tree.IndexedVarHelper, -) NameResolutionVisitor { - return NameResolutionVisitor{ - iVarHelper: ivarHelper, - resolver: colinfo.ColumnResolver{ - Source: source, - }, - } -} diff --git a/pkg/sql/catalog/table_elements.go b/pkg/sql/catalog/table_elements.go index 82f4edeb8e48..1b7ed3e4f227 100644 --- a/pkg/sql/catalog/table_elements.go +++ b/pkg/sql/catalog/table_elements.go @@ -970,11 +970,6 @@ func ColumnIDToOrdinalMap(columns []Column) TableColMap { return m } -// ColumnTypes returns the types of the given columns -func ColumnTypes(columns []Column) []*types.T { - return ColumnTypesWithInvertedCol(columns, nil /* invertedCol */) -} - // ColumnTypesWithInvertedCol returns the types of all given columns, // If invertedCol is non-nil, substitutes the type of the inverted // column instead of the column with the same ID. diff --git a/pkg/sql/parser/scanner.go b/pkg/sql/parser/scanner.go index 691c765cc5d6..8a605592cb7d 100644 --- a/pkg/sql/parser/scanner.go +++ b/pkg/sql/parser/scanner.go @@ -56,21 +56,6 @@ func Tokens(sql string) (tokens []TokenString, ok bool) { return tokens, true } -// TokensIgnoreErrors decomposes the input into lexical tokens and -// ignores errors. -func TokensIgnoreErrors(sql string) (tokens []TokenString) { - s := makeSQLScanner(sql) - for { - var lval = &sqlSymType{} - s.Scan(lval) - if lval.ID() == 0 { - break - } - tokens = append(tokens, TokenString{TokenID: lval.ID(), Str: lval.Str()}) - } - return tokens -} - // TokenString is the unit value returned by Tokens. type TokenString struct { TokenID int32 diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/database_state.go b/pkg/sql/schemachanger/scdeps/sctestdeps/database_state.go index c38094bfd9b5..c97e18d92f28 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/database_state.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/database_state.go @@ -37,17 +37,6 @@ import ( "github.com/stretchr/testify/require" ) -// WaitForNoRunningSchemaChanges schema changes waits for no schema changes -// to exist. -func WaitForNoRunningSchemaChanges(t *testing.T, tdb *sqlutils.SQLRunner) { - tdb.CheckQueryResultsRetry(t, ` -SELECT count(*) -FROM [SHOW JOBS] -WHERE job_type = 'SCHEMA CHANGE' - AND status NOT IN ('succeeded', 'failed', 'aborted')`, - [][]string{{"0"}}) -} - // ReadDescriptorsFromDB reads the set of descriptors from tdb. func ReadDescriptorsFromDB( ctx context.Context, t *testing.T, tdb *sqlutils.SQLRunner, diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go b/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go index d33140458da7..74e194009e43 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go @@ -213,14 +213,6 @@ func MakeColumnIDMutationSelector(columnID descpb.ColumnID) MutationSelector { } } -// MakeMutationIDMutationSelector returns a MutationSelector which matches the -// first mutation with this ID. -func MakeMutationIDMutationSelector(mutationID descpb.MutationID) MutationSelector { - return func(mut catalog.Mutation) bool { - return mut.MutationID() == mutationID - } -} - // enqueueNonIndexMutation enqueues a non-index mutation `m` (of generic type M) // with direction `dir` without increasing the next mutation ID. // The mutation state will be DELETE_ONLY if `dir=ADD` and WRITE_ONLY if `dir=DROP`. diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_23_1/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/release_23_1/helpers.go index cc9a671b64f0..a73a84130fe7 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_23_1/helpers.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_23_1/helpers.go @@ -55,12 +55,6 @@ func isDescriptor(e scpb.Element) bool { return false } -// IsDescriptor returns true for a descriptor-element, i.e. an element which -// owns its corresponding descriptor. This is only used for exports -func IsDescriptor(e scpb.Element) bool { - return isDescriptor(e) -} - func isSubjectTo2VersionInvariant(e scpb.Element) bool { // TODO(ajwerner): This should include constraints and enum values but it // currently does not because we do not support dropping them unless we're diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_23_2/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/release_23_2/helpers.go index d618475ae049..735cb14726fc 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_23_2/helpers.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_23_2/helpers.go @@ -55,12 +55,6 @@ func isDescriptor(e scpb.Element) bool { return false } -// IsDescriptor returns true for a descriptor-element, i.e. an element which -// owns its corresponding descriptor. This is only used for exports -func IsDescriptor(e scpb.Element) bool { - return isDescriptor(e) -} - func isSubjectTo2VersionInvariant(e scpb.Element) bool { // TODO(ajwerner): This should include constraints and enum values but it // currently does not because we do not support dropping them unless we're diff --git a/pkg/sql/schemachanger/screl/scalars.go b/pkg/sql/schemachanger/screl/scalars.go index 62329060f518..c741ccea68de 100644 --- a/pkg/sql/schemachanger/screl/scalars.go +++ b/pkg/sql/schemachanger/screl/scalars.go @@ -32,15 +32,6 @@ func GetDescID(e scpb.Element) catid.DescID { return id.(catid.DescID) } -// GetIndexID retrieves the index ID from the element if it has one. -func GetIndexID(e scpb.Element) (catid.IndexID, bool) { - v, err := Schema.GetAttribute(IndexID, e) - if err != nil { - return 0, false - } - return v.(catid.IndexID), true -} - // AllTargetStateDescIDs applies AllTargetDescIDs to the whole target state. func AllTargetStateDescIDs(s scpb.TargetState) (ids catalog.DescriptorIDSet) { for i := range s.Targets { diff --git a/pkg/sql/scrub/errors.go b/pkg/sql/scrub/errors.go index 4a942c72cc93..49cda44f3138 100644 --- a/pkg/sql/scrub/errors.go +++ b/pkg/sql/scrub/errors.go @@ -73,11 +73,6 @@ func WrapError(code string, err error) *Error { } } -// IsScrubError checks if an error is a Error. -func IsScrubError(err error) bool { - return errors.HasType(err, (*Error)(nil)) -} - // UnwrapScrubError gets the underlying error if err is a scrub.Error. // If err is not a scrub.Error, the error is returned unchanged. func UnwrapScrubError(err error) error { diff --git a/pkg/sql/sem/plpgsqltree/utils/plpg_visitor.go b/pkg/sql/sem/plpgsqltree/utils/plpg_visitor.go index 43d99b0eb3a6..58fe5e261a6b 100644 --- a/pkg/sql/sem/plpgsqltree/utils/plpg_visitor.go +++ b/pkg/sql/sem/plpgsqltree/utils/plpg_visitor.go @@ -100,24 +100,6 @@ func CountPLpgSQLStmt(sql string) (PLpgSQLStmtCounter, error) { return v.StmtCnt, v.Err } -// ParseAndCollectTelemetryForPLpgSQLFunc takes a plpgsql function and parses and collects -// telemetry on the parsable statements. -func ParseAndCollectTelemetryForPLpgSQLFunc(stmt *tree.CreateRoutine) error { - // Assert that the function language is PLPGSQL. - var funcBodyStr string - for _, option := range stmt.Options { - switch opt := option.(type) { - case tree.RoutineBodyStr: - funcBodyStr = string(opt) - } - } - - if _, err := CountPLpgSQLStmt(funcBodyStr); err != nil { - return errors.Wrap(err, "plpgsql not supported in user-defined functions") - } - return unimp.New("plpgsql", "plpgsql not supported in user-defined functions") -} - // SQLStmtVisitor calls Fn for every SQL statement and expression found while // walking the PLpgSQL AST. Since PLpgSQL nodes may have statement and // expression fields that are nil, Fn should handle the nil case. diff --git a/pkg/sql/sqlerrors/errors.go b/pkg/sql/sqlerrors/errors.go index cb31e769e5d5..89c8eaebba7d 100644 --- a/pkg/sql/sqlerrors/errors.go +++ b/pkg/sql/sqlerrors/errors.go @@ -193,13 +193,6 @@ func NewUnsupportedUnvalidatedConstraintError(constraintType catconstants.Constr "%v constraints cannot be marked NOT VALID", constraintType) } -// WrapErrorWhileConstructingObjectAlreadyExistsErr is used to wrap an error -// when an error occurs while trying to get the colliding object for an -// ObjectAlreadyExistsErr. -func WrapErrorWhileConstructingObjectAlreadyExistsErr(err error) error { - return pgerror.WithCandidateCode(errors.Wrap(err, "object already exists"), pgcode.DuplicateObject) -} - // MakeObjectAlreadyExistsError creates an error for a namespace collision // with an arbitrary descriptor type. func MakeObjectAlreadyExistsError(collidingObject *descpb.Descriptor, name string) error { From 77c3dbffa6ec4fc576ae923261e6b54531b1c9a6 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Fri, 19 Jan 2024 10:19:43 -0500 Subject: [PATCH 3/4] sql/tests: add "_test" suffix to file name The commit renames a test file, `virtual_cluster_name.go`, to match the naming convention. I believe the test in the was being run in CI before, despite the misnaming. Release note: None --- pkg/sql/tests/BUILD.bazel | 6 +----- ...virtual_cluster_name.go => virtual_cluster_name_test.go} | 0 2 files changed, 1 insertion(+), 5 deletions(-) rename pkg/sql/tests/{virtual_cluster_name.go => virtual_cluster_name_test.go} (100%) diff --git a/pkg/sql/tests/BUILD.bazel b/pkg/sql/tests/BUILD.bazel index b917de110631..5e9653233e75 100644 --- a/pkg/sql/tests/BUILD.bazel +++ b/pkg/sql/tests/BUILD.bazel @@ -6,12 +6,10 @@ go_library( "command_filters.go", "data.go", "explain_test_util.go", - "virtual_cluster_name.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/tests", visibility = ["//visibility:public"], deps = [ - "//pkg/base", "//pkg/internal/sqlsmith", "//pkg/kv", "//pkg/kv/kvpb", @@ -20,10 +18,7 @@ go_library( "//pkg/sql/sem/tree", "//pkg/storage", "//pkg/testutils/serverutils", - "//pkg/testutils/sqlutils", "//pkg/testutils/storageutils", - "//pkg/util/leaktest", - "//pkg/util/log", "//pkg/util/syncutil", "@com_github_cockroachdb_errors//:errors", "@org_golang_x_text//cases", @@ -62,6 +57,7 @@ go_test( "table_split_test.go", "tracing_sql_test.go", "truncate_test.go", + "virtual_cluster_name_test.go", "virtual_table_test.go", ], data = glob(["testdata/**"]) + [ diff --git a/pkg/sql/tests/virtual_cluster_name.go b/pkg/sql/tests/virtual_cluster_name_test.go similarity index 100% rename from pkg/sql/tests/virtual_cluster_name.go rename to pkg/sql/tests/virtual_cluster_name_test.go From 3296952575865dc008fc73cf192be0b54c8b6367 Mon Sep 17 00:00:00 2001 From: Ricky Stewart Date: Fri, 19 Jan 2024 12:52:41 -0600 Subject: [PATCH 4/4] generate-logictest: skip certain CCL configurations under `race` For whatever reason these particular configurations are particularly prone to timing out under `race`. Presumably they are more sensitive to the extra load. The other configurations seem not to have this issue. Epic: none Release note: None --- pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go | 1 + .../tests/multiregion-15node-5region-3azs/generated_test.go | 1 + .../multiregion-3node-3superlongregions/generated_test.go | 1 + .../multiregion-9node-3region-3azs-no-los/generated_test.go | 1 + .../multiregion-9node-3region-3azs-tenant/generated_test.go | 1 + .../multiregion-9node-3region-3azs-vec-off/generated_test.go | 1 + .../tests/multiregion-9node-3region-3azs/generated_test.go | 1 + pkg/cmd/generate-logictest/main.go | 4 ++++ pkg/cmd/generate-logictest/templates.go | 3 ++- 9 files changed, 13 insertions(+), 1 deletion(-) diff --git a/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go b/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go index 19754e71902c..012614b4c62b 100644 --- a/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go +++ b/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go @@ -85,6 +85,7 @@ func runLogicTest(t *testing.T, file string) { logictest.RunLogicTest(t, logictest.TestServerArgs{}, configIdx, filepath.Join(logicTestDir, file)) } func runCCLLogicTest(t *testing.T, file string) { + skip.UnderRace(t, "times out and/or hangs") skip.UnderDeadlock(t, "times out and/or hangs") logictest.RunLogicTest(t, logictest.TestServerArgs{}, configIdx, filepath.Join(cclLogicTestDir, file)) } diff --git a/pkg/ccl/logictestccl/tests/multiregion-15node-5region-3azs/generated_test.go b/pkg/ccl/logictestccl/tests/multiregion-15node-5region-3azs/generated_test.go index 49fa99808619..df83831424b9 100644 --- a/pkg/ccl/logictestccl/tests/multiregion-15node-5region-3azs/generated_test.go +++ b/pkg/ccl/logictestccl/tests/multiregion-15node-5region-3azs/generated_test.go @@ -60,6 +60,7 @@ func TestMain(m *testing.M) { } func runCCLLogicTest(t *testing.T, file string) { + skip.UnderRace(t, "times out and/or hangs") skip.UnderDeadlock(t, "times out and/or hangs") logictest.RunLogicTest(t, logictest.TestServerArgs{}, configIdx, filepath.Join(cclLogicTestDir, file)) } diff --git a/pkg/ccl/logictestccl/tests/multiregion-3node-3superlongregions/generated_test.go b/pkg/ccl/logictestccl/tests/multiregion-3node-3superlongregions/generated_test.go index ef561fba7e16..2b2ce8b3cff7 100644 --- a/pkg/ccl/logictestccl/tests/multiregion-3node-3superlongregions/generated_test.go +++ b/pkg/ccl/logictestccl/tests/multiregion-3node-3superlongregions/generated_test.go @@ -60,6 +60,7 @@ func TestMain(m *testing.M) { } func runCCLLogicTest(t *testing.T, file string) { + skip.UnderRace(t, "times out and/or hangs") skip.UnderDeadlock(t, "times out and/or hangs") logictest.RunLogicTest(t, logictest.TestServerArgs{}, configIdx, filepath.Join(cclLogicTestDir, file)) } diff --git a/pkg/ccl/logictestccl/tests/multiregion-9node-3region-3azs-no-los/generated_test.go b/pkg/ccl/logictestccl/tests/multiregion-9node-3region-3azs-no-los/generated_test.go index a4f575f3f1b0..7be0e069206e 100644 --- a/pkg/ccl/logictestccl/tests/multiregion-9node-3region-3azs-no-los/generated_test.go +++ b/pkg/ccl/logictestccl/tests/multiregion-9node-3region-3azs-no-los/generated_test.go @@ -60,6 +60,7 @@ func TestMain(m *testing.M) { } func runCCLLogicTest(t *testing.T, file string) { + skip.UnderRace(t, "times out and/or hangs") skip.UnderDeadlock(t, "times out and/or hangs") logictest.RunLogicTest(t, logictest.TestServerArgs{}, configIdx, filepath.Join(cclLogicTestDir, file)) } diff --git a/pkg/ccl/logictestccl/tests/multiregion-9node-3region-3azs-tenant/generated_test.go b/pkg/ccl/logictestccl/tests/multiregion-9node-3region-3azs-tenant/generated_test.go index b3d602d7f12a..38f98c5a35dc 100644 --- a/pkg/ccl/logictestccl/tests/multiregion-9node-3region-3azs-tenant/generated_test.go +++ b/pkg/ccl/logictestccl/tests/multiregion-9node-3region-3azs-tenant/generated_test.go @@ -60,6 +60,7 @@ func TestMain(m *testing.M) { } func runCCLLogicTest(t *testing.T, file string) { + skip.UnderRace(t, "times out and/or hangs") skip.UnderDeadlock(t, "times out and/or hangs") logictest.RunLogicTest(t, logictest.TestServerArgs{}, configIdx, filepath.Join(cclLogicTestDir, file)) } diff --git a/pkg/ccl/logictestccl/tests/multiregion-9node-3region-3azs-vec-off/generated_test.go b/pkg/ccl/logictestccl/tests/multiregion-9node-3region-3azs-vec-off/generated_test.go index 366feee839d6..2e611fb5267f 100644 --- a/pkg/ccl/logictestccl/tests/multiregion-9node-3region-3azs-vec-off/generated_test.go +++ b/pkg/ccl/logictestccl/tests/multiregion-9node-3region-3azs-vec-off/generated_test.go @@ -60,6 +60,7 @@ func TestMain(m *testing.M) { } func runCCLLogicTest(t *testing.T, file string) { + skip.UnderRace(t, "times out and/or hangs") skip.UnderDeadlock(t, "times out and/or hangs") logictest.RunLogicTest(t, logictest.TestServerArgs{}, configIdx, filepath.Join(cclLogicTestDir, file)) } diff --git a/pkg/ccl/logictestccl/tests/multiregion-9node-3region-3azs/generated_test.go b/pkg/ccl/logictestccl/tests/multiregion-9node-3region-3azs/generated_test.go index 2a28c2c150f7..9238d6af4e1a 100644 --- a/pkg/ccl/logictestccl/tests/multiregion-9node-3region-3azs/generated_test.go +++ b/pkg/ccl/logictestccl/tests/multiregion-9node-3region-3azs/generated_test.go @@ -60,6 +60,7 @@ func TestMain(m *testing.M) { } func runCCLLogicTest(t *testing.T, file string) { + skip.UnderRace(t, "times out and/or hangs") skip.UnderDeadlock(t, "times out and/or hangs") logictest.RunLogicTest(t, logictest.TestServerArgs{}, configIdx, filepath.Join(cclLogicTestDir, file)) } diff --git a/pkg/cmd/generate-logictest/main.go b/pkg/cmd/generate-logictest/main.go index 984ec4dd38ec..85128b40af29 100644 --- a/pkg/cmd/generate-logictest/main.go +++ b/pkg/cmd/generate-logictest/main.go @@ -32,6 +32,7 @@ type testFileTemplateConfig struct { CockroachGoTestserverTest bool Ccl bool ForceProductionValues bool + SkipCclUnderRace bool Package, TestRuleName, RelDir string ConfigIdx int TestCount int @@ -178,6 +179,9 @@ func (t *testdir) dump() error { // allocate the tests which use 3-node clusters 2 vCPUs, and // the ones which use more a bit more. tplCfg.NumCPU = (cfg.NumNodes / 2) + 1 + if cfg.Name == "3node-tenant" || strings.HasPrefix(cfg.Name, "multiregion-") { + tplCfg.SkipCclUnderRace = true + } subdir := filepath.Join(t.dir, cfg.Name) f, buildF, cleanup, err := openTestSubdir(subdir) if err != nil { diff --git a/pkg/cmd/generate-logictest/templates.go b/pkg/cmd/generate-logictest/templates.go index 58c764d6c38a..81936b961108 100644 --- a/pkg/cmd/generate-logictest/templates.go +++ b/pkg/cmd/generate-logictest/templates.go @@ -60,7 +60,8 @@ func runLogicTest(t *testing.T, file string) { {{- define "runCCLLogicTest" }} {{- if .CclLogicTest -}} func runCCLLogicTest(t *testing.T, file string) { - skip.UnderDeadlock(t, "times out and/or hangs") + {{if .SkipCclUnderRace}}skip.UnderRace(t, "times out and/or hangs") + {{end}}skip.UnderDeadlock(t, "times out and/or hangs") logictest.RunLogicTest(t, logictest.TestServerArgs{}, configIdx, filepath.Join(cclLogicTestDir, file)) } {{ end }}