diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 11385972f56e..55d2f93e4f90 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -1157,6 +1157,8 @@ GO_TARGETS = [ "//pkg/cmd/roachtest/clusterstats:clusterstats", "//pkg/cmd/roachtest/clusterstats:clusterstats_test", "//pkg/cmd/roachtest/grafana:grafana", + "//pkg/cmd/roachtest/operation:operation", + "//pkg/cmd/roachtest/operations:operations", "//pkg/cmd/roachtest/option:option", "//pkg/cmd/roachtest/option:option_test", "//pkg/cmd/roachtest/registry:registry", diff --git a/pkg/cmd/roachtest/BUILD.bazel b/pkg/cmd/roachtest/BUILD.bazel index 80e1064c25cc..63c793e40596 100644 --- a/pkg/cmd/roachtest/BUILD.bazel +++ b/pkg/cmd/roachtest/BUILD.bazel @@ -23,6 +23,7 @@ go_library( "//pkg/build", "//pkg/cmd/internal/issues", "//pkg/cmd/roachtest/cluster", + "//pkg/cmd/roachtest/operations", "//pkg/cmd/roachtest/option", "//pkg/cmd/roachtest/registry", "//pkg/cmd/roachtest/roachtestflags", diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index 24c5bb9d74af..e077ea24ed2b 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -316,11 +316,15 @@ func initBinariesAndLibraries() { cockroachPath := roachtestflags.CockroachPath cockroachEAPath := roachtestflags.CockroachEAPath workloadPath := roachtestflags.WorkloadPath - cockroach[defaultArch], _ = resolveBinary("cockroach", cockroachPath, defaultArch, true, false) - workload[defaultArch], _ = resolveBinary("workload", workloadPath, defaultArch, true, false) - cockroachEA[defaultArch], err = resolveBinary("cockroach-ea", cockroachEAPath, defaultArch, false, true) - if err != nil { - fmt.Fprintf(os.Stderr, "WARN: unable to find %q for %q: %s\n", "cockroach-ea", defaultArch, err) + // If a remote cockroach binary has been specified, we don't need to resolve + // cockroach/workload binaries. + if roachtestflags.CockroachBinaryPath == "" { + cockroach[defaultArch], _ = resolveBinary("cockroach", cockroachPath, defaultArch, true, false) + workload[defaultArch], _ = resolveBinary("workload", workloadPath, defaultArch, true, false) + cockroachEA[defaultArch], err = resolveBinary("cockroach-ea", cockroachEAPath, defaultArch, false, true) + if err != nil { + fmt.Fprintf(os.Stderr, "WARN: unable to find %q for %q: %s\n", "cockroach-ea", defaultArch, err) + } } if roachtestflags.ARM64Probability > 0 && defaultArch != vm.ArchARM64 { @@ -973,8 +977,9 @@ func (f *clusterFactory) newCluster( type attachOpt struct { skipValidation bool // Implies skipWipe. - skipStop bool - skipWipe bool + skipStop bool + skipWipe bool + tolerateRegistrationErrors bool } // attachToExistingCluster creates a cluster object based on machines that have @@ -1011,7 +1016,7 @@ func attachToExistingCluster( } } - if err := r.registerCluster(c); err != nil { + if err := r.registerCluster(c); err != nil && !opt.tolerateRegistrationErrors { return nil, err } @@ -1066,7 +1071,7 @@ func (c *clusterImpl) StopCockroachGracefullyOnNode( // Save marks the cluster as "saved" so that it doesn't get destroyed. func (c *clusterImpl) Save(ctx context.Context, msg string, l *logger.Logger) { - l.PrintfCtx(ctx, "saving cluster %s for debugging (--debug specified)", c) + l.PrintfCtx(ctx, "saving cluster %s (--debug specified or running operation)", c) c.r.markClusterAsSaved(c, msg) c.destroyState.mu.Lock() c.destroyState.mu.saved = true @@ -1820,17 +1825,18 @@ func (c *clusterImpl) PutE( // nodes in the cluster. By default, we randomly upload a binary with or without // runtime assertions enabled. Note that we upload to all nodes even if they // don't use the binary, so that the test runner can always fetch logs. -func (c *clusterImpl) PutCockroach(ctx context.Context, l *logger.Logger, t *testImpl) error { - switch t.spec.CockroachBinary { +func (c *clusterImpl) PutCockroach(ctx context.Context, l *logger.Logger, t test.Test) error { + binaryType := t.(*testImpl).spec.CockroachBinary + switch binaryType { case registry.RandomizedCockroach: if tests.UsingRuntimeAssertions(t) { - t.l.Printf("To reproduce the same set of metamorphic constants, run this test with %s=%d", test.EnvAssertionsEnabledSeed, c.cockroachRandomSeed()) + t.L().Printf("To reproduce the same set of metamorphic constants, run this test with %s=%d", test.EnvAssertionsEnabledSeed, c.cockroachRandomSeed()) } return c.PutE(ctx, l, t.Cockroach(), test.DefaultCockroachPath, c.All()) case registry.StandardCockroach: return c.PutE(ctx, l, t.StandardCockroach(), test.DefaultCockroachPath, c.All()) case registry.RuntimeAssertionsCockroach: - t.l.Printf("To reproduce the same set of metamorphic constants, run this test with %s=%d", test.EnvAssertionsEnabledSeed, c.cockroachRandomSeed()) + t.L().Printf("To reproduce the same set of metamorphic constants, run this test with %s=%d", test.EnvAssertionsEnabledSeed, c.cockroachRandomSeed()) return c.PutE(ctx, l, t.RuntimeAssertionsCockroach(), test.DefaultCockroachPath, c.All()) default: return errors.Errorf("Specified cockroach binary does not exist.") @@ -2861,6 +2867,12 @@ func (c *clusterImpl) MaybeExtendCluster( ctx context.Context, l *logger.Logger, testSpec *registry.TestSpec, ) error { timeout := testTimeout(testSpec) + return c.MaybeExtendClusterForTimeout(ctx, l, timeout) +} + +func (c *clusterImpl) MaybeExtendClusterForTimeout( + ctx context.Context, l *logger.Logger, timeout time.Duration, +) error { minExp := timeutil.Now().Add(timeout + time.Hour) if c.expiration.Before(minExp) { extend := minExp.Sub(c.expiration) diff --git a/pkg/cmd/roachtest/main.go b/pkg/cmd/roachtest/main.go index 68c7922ed82b..9ceea0849119 100644 --- a/pkg/cmd/roachtest/main.go +++ b/pkg/cmd/roachtest/main.go @@ -19,6 +19,7 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/build" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/operations" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestflags" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/tests" @@ -179,9 +180,31 @@ the cluster nodes on start. } roachtestflags.AddRunFlags(benchCmd.Flags()) + var runOperationCmd = &cobra.Command{ + // Don't display usage when tests fail. + SilenceUsage: true, + Use: "run-operation [regex...]", + Short: "run operations on cockroach cluster", + Long: `Run automated operations on existing clusters.`, + RunE: func(cmd *cobra.Command, args []string) error { + if err := initRunFlagsBinariesAndLibraries(cmd); err != nil { + return err + } + filter, err := makeTestFilter(args) + if err != nil { + return err + } + fmt.Printf("\nRunning %s.\n\n", filter.String()) + cmd.SilenceUsage = true + return runOperations(operations.RegisterOperations, filter) + }, + } + roachtestflags.AddRunOpsFlags(runOperationCmd.Flags()) + rootCmd.AddCommand(listCmd) rootCmd.AddCommand(runCmd) rootCmd.AddCommand(benchCmd) + rootCmd.AddCommand(runOperationCmd) var err error config.OSUser, err = user.Current() @@ -263,6 +286,94 @@ func testsToRun( return selectSpecs(notSkipped, selectProbability, true, print), nil } +func opsToRun( + r testRegistryImpl, + filter *registry.TestFilter, + runSkipped bool, + selectProbability float64, + print bool, +) ([]registry.OperationSpec, error) { + specs := filter.FilterOps(r.AllOperations()) + if len(specs) == 0 { + return nil, errors.New("no matching operations to run") + } + + var notSkipped []registry.OperationSpec + for _, s := range specs { + if s.Skip == "" || runSkipped { + notSkipped = append(notSkipped, s) + } else { + if print && roachtestflags.TeamCity { + fmt.Fprintf(os.Stdout, "##teamcity[testIgnored name='%s' message='%s']\n", + s.Name, TeamCityEscape(s.Skip)) + } + if print { + fmt.Fprintf(os.Stdout, "--- SKIP: %s (%s)\n\t%s\n", s.Name, "0.00s", s.Skip) + } + } + } + + if print { + // We want to show information about all operations which match the + // pattern(s) but were excluded for other reasons. + relaxedFilter := registry.TestFilter{ + Name: filter.Name, + } + for _, s := range relaxedFilter.FilterOps(r.AllOperations()) { + if matches, r := filter.MatchesOp(&s); !matches { + reason := filter.MatchFailReasonString(r) + // This test matches the "relaxed" filter but not the original filter. + if roachtestflags.TeamCity { + fmt.Fprintf(os.Stdout, "##teamcity[testIgnored name='%s' message='%s']\n", s.Name, reason) + } + fmt.Fprintf(os.Stdout, "--- SKIP: %s (%s)\n\t%s\n", s.Name, "0.00s", reason) + } + } + } + + return selectOpSpecs(notSkipped, selectProbability, print), nil +} + +func selectOpSpecs( + specs []registry.OperationSpec, samplePct float64, print bool, +) []registry.OperationSpec { + if samplePct == 1 || len(specs) == 0 { + return specs + } + + var sampled []registry.OperationSpec + var selectedIdxs []int + + // Selects one random spec from the range [start, end) and appends it to sampled. + for i, s := range specs { + if rand.Float64() < samplePct { + sampled = append(sampled, s) + selectedIdxs = append(selectedIdxs, i) + continue + } + } + + p := 0 + // This loop depends on an ordered list as we are essentially + // skipping all values in between the selected indexes. + for _, i := range selectedIdxs { + for j := p; j < i; j++ { + s := specs[j] + if print && roachtestflags.TeamCity { + fmt.Fprintf(os.Stdout, "##teamcity[testIgnored name='%s' message='excluded via sampling']\n", + s.Name) + } + + if print { + fmt.Fprintf(os.Stdout, "--- SKIP: %s (%s)\n\texcluded via sampling\n", s.Name, "0.00s") + } + } + p = i + 1 + } + + return sampled +} + // selectSpecs returns a random sample of the given test specs. // If atLeastOnePerPrefix is true, it guarantees that at least one test is // selected for each prefix (e.g. kv0/, acceptance/). diff --git a/pkg/cmd/roachtest/operation/BUILD.bazel b/pkg/cmd/roachtest/operation/BUILD.bazel new file mode 100644 index 000000000000..2c172388e181 --- /dev/null +++ b/pkg/cmd/roachtest/operation/BUILD.bazel @@ -0,0 +1,9 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "operation", + srcs = ["operation_interface.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/operation", + visibility = ["//visibility:public"], + deps = ["//pkg/cmd/roachtest/test"], +) diff --git a/pkg/cmd/roachtest/operation/operation_interface.go b/pkg/cmd/roachtest/operation/operation_interface.go new file mode 100644 index 000000000000..8fc14473c725 --- /dev/null +++ b/pkg/cmd/roachtest/operation/operation_interface.go @@ -0,0 +1,22 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package operation + +import "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" + +type Operation interface { + // TODO(bilal): Instead of encapsulating test.Test, copy over the small + // set of relevant methods, ideally moving them to a shared interface. + test.Test + + GetCleanupState(string) string + SetCleanupState(string, string) +} diff --git a/pkg/cmd/roachtest/operations/BUILD.bazel b/pkg/cmd/roachtest/operations/BUILD.bazel new file mode 100644 index 000000000000..ff83f36f0420 --- /dev/null +++ b/pkg/cmd/roachtest/operations/BUILD.bazel @@ -0,0 +1,22 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "operations", + srcs = [ + "add_column.go", + "add_index.go", + "register.go", + "utils.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/operations", + visibility = ["//visibility:public"], + deps = [ + "//pkg/cmd/roachtest/cluster", + "//pkg/cmd/roachtest/operation", + "//pkg/cmd/roachtest/option", + "//pkg/cmd/roachtest/registry", + "//pkg/cmd/roachtest/roachtestflags", + "//pkg/cmd/roachtest/test", + "//pkg/util/randutil", + ], +) diff --git a/pkg/cmd/roachtest/operations/add_column.go b/pkg/cmd/roachtest/operations/add_column.go new file mode 100644 index 000000000000..d6c0416e4782 --- /dev/null +++ b/pkg/cmd/roachtest/operations/add_column.go @@ -0,0 +1,74 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package operations + +import ( + "context" + "fmt" + "time" + + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/operation" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestflags" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +func runAddColumn(ctx context.Context, o operation.Operation, c cluster.Cluster) { + conn := c.Conn(ctx, o.L(), 1, option.TenantName(roachtestflags.VirtualCluster)) + defer conn.Close() + + rng, _ := randutil.NewPseudoRand() + dbName := pickRandomDB(ctx, o, conn) + tableName := pickRandomTable(ctx, o, conn, dbName) + o.SetCleanupState("db", dbName) + o.SetCleanupState("table", tableName) + + colName := fmt.Sprintf("add_column_op_%d", rng.Uint32()) + o.Status(fmt.Sprintf("adding column %s to table %s.%s", colName, dbName, tableName)) + addColStmt := fmt.Sprintf("ALTER TABLE %s.%s ADD COLUMN %s VARCHAR DEFAULT 'default'", dbName, tableName, colName) + _, err := conn.ExecContext(ctx, addColStmt) + if err != nil { + o.Fatal(err) + } + o.SetCleanupState("column", colName) + + o.Status(fmt.Sprintf("column %s created", colName)) +} + +func cleanupAddColumn(ctx context.Context, o operation.Operation, c cluster.Cluster) { + conn := c.Conn(ctx, o.L(), 1, option.TenantName(roachtestflags.VirtualCluster)) + defer conn.Close() + + dbName := o.GetCleanupState("db") + tableName := o.GetCleanupState("table") + columnName := o.GetCleanupState("column") + + o.Status(fmt.Sprintf("dropping column %s", columnName)) + _, err := conn.ExecContext(ctx, fmt.Sprintf("ALTER TABLE %s.%s DROP COLUMN %s CASCADE", dbName, tableName, columnName)) + if err != nil { + o.Fatal(err) + } +} + +func registerAddColumn(r registry.Registry) { + r.AddOperation(registry.OperationSpec{ + Name: "add-column", + Owner: registry.OwnerSQLFoundations, + Timeout: 24 * time.Hour, + CompatibleClouds: registry.AllClouds, + Dependency: registry.OperationRequiresDatabaseSchema, + Run: runAddColumn, + CleanupWaitTime: 5 * time.Minute, + Cleanup: cleanupAddColumn, + }) +} diff --git a/pkg/cmd/roachtest/operations/add_index.go b/pkg/cmd/roachtest/operations/add_index.go new file mode 100644 index 000000000000..fc54745931d3 --- /dev/null +++ b/pkg/cmd/roachtest/operations/add_index.go @@ -0,0 +1,87 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package operations + +import ( + "context" + "fmt" + "time" + + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/operation" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestflags" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +func runAddIndex(ctx context.Context, o operation.Operation, c cluster.Cluster) { + conn := c.Conn(ctx, o.L(), 1, option.TenantName(roachtestflags.VirtualCluster)) + defer conn.Close() + + rng, _ := randutil.NewPseudoRand() + dbName := pickRandomDB(ctx, o, conn) + tableName := pickRandomTable(ctx, o, conn, dbName) + o.SetCleanupState("db", dbName) + o.SetCleanupState("table", tableName) + + rows, err := conn.QueryContext(ctx, fmt.Sprintf("SELECT column_name FROM [SHOW COLUMNS FROM %s.%s]", dbName, tableName)) + if err != nil { + o.Fatal(err) + } + rows.Next() + if !rows.Next() { + o.Fatalf("not enough columns in table %s.%s", dbName, tableName) + } + var colName string + if err := rows.Scan(&colName); err != nil { + o.Fatal(err) + } + + indexName := fmt.Sprintf("add_index_op_%d", rng.Uint32()) + o.Status(fmt.Sprintf("adding index to column %s", colName)) + createIndexStmt := fmt.Sprintf("CREATE INDEX %s ON %s.%s (%s)", indexName, dbName, tableName, colName) + _, err = conn.ExecContext(ctx, createIndexStmt) + if err != nil { + o.Fatal(err) + } + o.SetCleanupState("index", indexName) + + o.Status(fmt.Sprintf("index %s created", indexName)) +} + +func cleanupAddIndex(ctx context.Context, o operation.Operation, c cluster.Cluster) { + conn := c.Conn(ctx, o.L(), 1, option.TenantName(roachtestflags.VirtualCluster)) + defer conn.Close() + + dbName := o.GetCleanupState("db") + tableName := o.GetCleanupState("table") + indexName := o.GetCleanupState("index") + + o.Status(fmt.Sprintf("dropping index %s", indexName)) + _, err := conn.ExecContext(ctx, fmt.Sprintf("DROP INDEX %s.%s@%s", dbName, tableName, indexName)) + if err != nil { + o.Fatal(err) + } +} + +func registerAddIndex(r registry.Registry) { + r.AddOperation(registry.OperationSpec{ + Name: "add-index", + Owner: registry.OwnerSQLFoundations, + Timeout: 24 * time.Hour, + CompatibleClouds: registry.AllClouds, + Dependency: registry.OperationRequiresDatabaseSchema, + Run: runAddIndex, + CleanupWaitTime: 5 * time.Minute, + Cleanup: cleanupAddIndex, + }) +} diff --git a/pkg/cmd/roachtest/operations/register.go b/pkg/cmd/roachtest/operations/register.go new file mode 100644 index 000000000000..8f4788fe073e --- /dev/null +++ b/pkg/cmd/roachtest/operations/register.go @@ -0,0 +1,19 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package operations + +import "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" + +// RegisterOperations registers all tests to the Registry. This powers `roachtest run-operations`. +func RegisterOperations(r registry.Registry) { + registerAddIndex(r) + registerAddColumn(r) +} diff --git a/pkg/cmd/roachtest/operations/utils.go b/pkg/cmd/roachtest/operations/utils.go new file mode 100644 index 000000000000..82cdb1b41398 --- /dev/null +++ b/pkg/cmd/roachtest/operations/utils.go @@ -0,0 +1,78 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package operations + +import ( + "context" + gosql "database/sql" + "fmt" + + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +func pickRandomDB(ctx context.Context, o test.Test, conn *gosql.DB) string { + rng, _ := randutil.NewPseudoRand() + + // Pick a random table. + dbs, err := conn.QueryContext(ctx, "SELECT database_name FROM [SHOW DATABASES]") + if err != nil { + o.Fatal(err) + return "" + } + var dbNames []string + for dbs.Next() { + var dbName string + if err := dbs.Scan(&dbName); err != nil { + o.Fatal(err) + return "" + } + if dbName == "system" || dbName == "information_schema" || dbName == "crdb_internal" || dbName == "defaultdb" || dbName == "postgres" { + continue + } + dbNames = append(dbNames, dbName) + } + if len(dbNames) == 0 { + o.Fatalf("unexpected zero active dbs found in cluster") + return "" + } + return dbNames[rng.Intn(len(dbNames))] +} + +func pickRandomTable(ctx context.Context, o test.Test, conn *gosql.DB, dbName string) string { + rng, _ := randutil.NewPseudoRand() + + // Pick a random table. + if _, err := conn.ExecContext(ctx, fmt.Sprintf("USE %s", dbName)); err != nil { + o.Fatal(err) + return "" + } + + tables, err := conn.QueryContext(ctx, "SELECT table_name FROM [SHOW TABLES]") + if err != nil { + o.Fatal(err) + return "" + } + var tableNames []string + for tables.Next() { + var tableName string + if err := tables.Scan(&tableName); err != nil { + o.Fatal(err) + return "" + } + tableNames = append(tableNames, tableName) + } + if len(tableNames) == 0 { + o.Fatalf("unexpected zero active tables found in db %s", dbName) + return "" + } + return tableNames[rng.Intn(len(tableNames))] +} diff --git a/pkg/cmd/roachtest/registry/BUILD.bazel b/pkg/cmd/roachtest/registry/BUILD.bazel index 0f4c211872d4..c3f86764a587 100644 --- a/pkg/cmd/roachtest/registry/BUILD.bazel +++ b/pkg/cmd/roachtest/registry/BUILD.bazel @@ -6,6 +6,7 @@ go_library( "encryption.go", "errors.go", "filter.go", + "operation_spec.go", "owners.go", "registry_interface.go", "tag.go", @@ -15,6 +16,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/cmd/roachtest/cluster", + "//pkg/cmd/roachtest/operation", "//pkg/cmd/roachtest/spec", "//pkg/cmd/roachtest/test", "//pkg/internal/team", diff --git a/pkg/cmd/roachtest/registry/filter.go b/pkg/cmd/roachtest/registry/filter.go index 47c3072edd0b..6b8af142ce3f 100644 --- a/pkg/cmd/roachtest/registry/filter.go +++ b/pkg/cmd/roachtest/registry/filter.go @@ -123,6 +123,17 @@ func (filter *TestFilter) Matches(t *TestSpec) (matches bool, reason MatchFailRe return reason == MatchFailReason{}, reason } +// MatchesOp returns true if the filter matches the operation. If the op doesn't +// match, return the reasons. +func (filter *TestFilter) MatchesOp(o *OperationSpec) (matches bool, reason MatchFailReason) { + reason.NameMismatch = !filter.Name.MatchString(o.Name) + reason.OwnerMismatch = filter.Owner != "" && o.Owner != filter.Owner + reason.CloudNotCompatible = filter.Cloud != "" && !o.CompatibleClouds.Contains(filter.Cloud) + + // We have a match if all fields are false. + return reason == MatchFailReason{}, reason +} + // MatchFailReasonString returns a user-friendly string describing the reason(s) // a filter failed to match a test (returned by Matches). Returns the empty // string if the reason is zero. @@ -295,6 +306,18 @@ func (filter *TestFilter) FilterWithHint(tests []TestSpec) ([]TestSpec, NoMatche return nil, NoHintAvailable } +// FilterOps returns the op specs in the given list that match the filter +// (in the same order). +func (filter *TestFilter) FilterOps(ops []OperationSpec) []OperationSpec { + var res []OperationSpec + for i := range ops { + if ok, _ := filter.MatchesOp(&ops[i]); ok { + res = append(res, ops[i]) + } + } + return res +} + // NoMatchesHintString returns a user-friendly string describing the hint. func (filter *TestFilter) NoMatchesHintString(h NoMatchesHint) string { noun := filter.noun() diff --git a/pkg/cmd/roachtest/registry/operation_spec.go b/pkg/cmd/roachtest/registry/operation_spec.go new file mode 100644 index 000000000000..2a93c57b1892 --- /dev/null +++ b/pkg/cmd/roachtest/registry/operation_spec.go @@ -0,0 +1,130 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package registry + +import ( + "context" + "fmt" + "time" + + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/operation" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" +) + +// OperationDependency specifies what an operation requires +type OperationDependency int + +const ( + OperationRequiresNodes OperationDependency = iota + OperationRequiresSqlConnection + OperationRequiresDatabaseSchema + OperationRequiresPopulatedDatabase +) + +// OperationSpec is a spec for a roachtest operation. +type OperationSpec struct { + Skip string // if non-empty, operation will be skipped + + Name string + // Owner is the name of the team responsible for signing off on failures of + // this operation that happen in the release process. This must be one of a limited + // set of values (the keys in the roachtestTeams map). + Owner Owner + // The maximum duration the operation is allowed to run before it is considered + // failed. + Timeout time.Duration + + // CompatibleClouds is the set of clouds this test can run on (e.g. AllClouds, + // OnlyGCE, etc). Must be set. + CompatibleClouds CloudSet + + // Dependency specify the types of resources required for this roachtest + // operation to work. This will be used to match this operation up with + // eligible clusters to run. + // + // TODO(bilal): Unused. + Dependency OperationDependency + + // CanRunConcurrently specifies whether this operation is safe to run + // concurrently with other operations that have CanRunConcurrently = true. + // For instance, two random-index additions are safe to run concurrently, + // while a drop would need to run on its own and will have + // CanRunConcurrently = false. + // + // TODO(bilal): Unused. + CanRunConcurrently bool + + // RequiresLicense indicates that the operation requires an + // enterprise license to run correctly. Use this to ensure + // operation will fail-early if COCKROACH_DEV_LICENSE is not set + // in the environment. + RequiresLicense bool + + // Run is the operation function. + Run func(ctx context.Context, o operation.Operation, c cluster.Cluster) + + // CleanupWaitTime is the min time to wait before running the Cleanup method. + CleanupWaitTime time.Duration + + // Cleanup is the operation cleanup function, if defined. + Cleanup func(ctx context.Context, o operation.Operation, c cluster.Cluster) +} + +// TestSpec() converts this operation to a TestSpec for use with roachtest run-operation. +func (s *OperationSpec) TestSpec() TestSpec { + return TestSpec{ + Skip: s.Skip, + Name: s.Name, + Owner: s.Owner, + Timeout: s.Timeout, + Benchmark: false, + Cluster: spec.ClusterSpec{NodeCount: 1}, + CompatibleClouds: AllClouds, + NonReleaseBlocker: true, + Operation: true, + RequiresLicense: s.RequiresLicense, + Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { + wrapper := &operationWrapper{Test: t, cleanupState: make(map[string]string)} + s.Run(ctx, wrapper, c) + + if s.Cleanup != nil { + t.Status(fmt.Sprintf("operation ran successfully; waiting %s before cleanup", s.CleanupWaitTime.String())) + if s.CleanupWaitTime != 0 { + select { + case <-ctx.Done(): + t.Status("bailing due to cancellation") + return + case <-time.After(s.CleanupWaitTime): + } + } + + s.Cleanup(ctx, wrapper, c) + } + }, + CockroachBinary: StandardCockroach, + } +} + +// operationWrapper turns a test.Test into an operation.Operation. +type operationWrapper struct { + test.Test + cleanupState map[string]string +} + +func (o *operationWrapper) SetCleanupState(key, value string) { + o.cleanupState[key] = value +} + +func (o *operationWrapper) GetCleanupState(key string) string { + return o.cleanupState[key] +} diff --git a/pkg/cmd/roachtest/registry/registry_interface.go b/pkg/cmd/roachtest/registry/registry_interface.go index 06c59c02d0cf..7b9d522f4408 100644 --- a/pkg/cmd/roachtest/registry/registry_interface.go +++ b/pkg/cmd/roachtest/registry/registry_interface.go @@ -20,5 +20,6 @@ import ( type Registry interface { MakeClusterSpec(nodeCount int, opts ...spec.Option) spec.ClusterSpec Add(TestSpec) + AddOperation(OperationSpec) PromFactory() promauto.Factory } diff --git a/pkg/cmd/roachtest/registry/test_spec.go b/pkg/cmd/roachtest/registry/test_spec.go index e05ed06e420c..7fc2eefe8734 100644 --- a/pkg/cmd/roachtest/registry/test_spec.go +++ b/pkg/cmd/roachtest/registry/test_spec.go @@ -55,6 +55,12 @@ type TestSpec struct { // Thus, they must be opted into explicitly via this field. Benchmark bool + // Denotes whether this test is a roachtest Operation. If true, the test is + // expected to not do any setup/teardown and is likely already running with a + // Cockroach process. Furthermore, a Cockroach/workload binary should *not* + // be loaded onto this node. + Operation bool + // CompatibleClouds is the set of clouds this test can run on (e.g. AllClouds, // OnlyGCE, etc). Must be set. CompatibleClouds CloudSet diff --git a/pkg/cmd/roachtest/roachtestflags/flags.go b/pkg/cmd/roachtest/roachtestflags/flags.go index 16d9d529406c..822aebf67021 100644 --- a/pkg/cmd/roachtest/roachtestflags/flags.go +++ b/pkg/cmd/roachtest/roachtestflags/flags.go @@ -65,8 +65,8 @@ var ( Usage: `Include tests that are not marked as compatible with the cloud used`, }) - ClusterNames string - _ = registerRunFlag(&ClusterNames, FlagInfo{ + ClusterNames string + clusterFlagInfo = FlagInfo{ Name: "cluster", Shorthand: "c", Usage: ` @@ -74,7 +74,9 @@ var ( tests. If fewer than --parallelism names are specified, then the parallelism is capped to the number of clusters specified. When a cluster does not exist yet, it is created according to the spec.`, - }) + } + _ = registerRunFlag(&ClusterNames, clusterFlagInfo) + _ = registerRunOpsFlag(&ClusterNames, clusterFlagInfo) Local bool _ = registerRunFlag(&Local, FlagInfo{ @@ -98,6 +100,24 @@ var ( Usage: `Absolute path to cockroach binary to use`, }) + CockroachBinaryPath string + _ = registerRunOpsFlag(&CockroachBinaryPath, FlagInfo{ + Name: "cockroach-binary", + Usage: `Relative path to cockroach binary to use, on the cluster specified in --cluster`, + }) + + CertsDir string + _ = registerRunOpsFlag(&CertsDir, FlagInfo{ + Name: "certs-dir", + Usage: `Absolute path to certificates directory, if the cluster specified in --cluster is secure`, + }) + + VirtualCluster string + _ = registerRunOpsFlag(&VirtualCluster, FlagInfo{ + Name: "virtual-cluster", + Usage: `Specifies virtual cluster to connect to, within the specified --cluster.`, + }) + CockroachEAPath string _ = registerRunFlag(&CockroachEAPath, FlagInfo{ Name: "cockroach-ea", @@ -141,11 +161,13 @@ var ( // ArtifactsDir is a path to a local dir where the test logs and artifacts // collected from cluster will be placed. - ArtifactsDir string = "artifacts" - _ = registerRunFlag(&ArtifactsDir, FlagInfo{ + ArtifactsDir string = "artifacts" + ArtifactsFlag = FlagInfo{ Name: "artifacts", Usage: `Path to artifacts directory`, - }) + } + _ = registerRunFlag(&ArtifactsDir, ArtifactsFlag) + _ = registerRunOpsFlag(&ArtifactsDir, ArtifactsFlag) // LiteralArtifactsDir is a path to the literal on-agent directory where // artifacts are stored. May be different from `artifacts`. Only used for @@ -242,11 +264,12 @@ var ( https://docs.microsoft.com/en-us/azure/virtual-machines/windows/sizes)`, }) - CPUQuota int = 300 - _ = registerRunFlag(&CPUQuota, FlagInfo{ + CPUQuota int = 300 + cpuQuotaFlagInfo = FlagInfo{ Name: "cpu-quota", Usage: `The number of cloud CPUs roachtest is allowed to use at any one time.`, - }) + } + _ = registerRunFlag(&CPUQuota, cpuQuotaFlagInfo) HTTPPort int = 0 _ = registerRunFlag(&HTTPPort, FlagInfo{ @@ -405,6 +428,12 @@ func AddRunFlags(cmdFlags *pflag.FlagSet) { globalMan.AddFlagsToCommand(runCmdID, cmdFlags) } +// AddRunOpsFlags adds all flags registered for the run-operations command to +// the given command flag set. +func AddRunOpsFlags(cmdFlags *pflag.FlagSet) { + globalMan.AddFlagsToCommand(runOpsCmdID, cmdFlags) +} + // Changed returns true if a flag associated with a given value was present. // // For example: roachtestflags.Changed(&roachtestflags.Cloud) returns true if @@ -425,3 +454,8 @@ func registerRunFlag(valPtr interface{}, info FlagInfo) struct{} { globalMan.RegisterFlag(runCmdID, valPtr, info) return struct{}{} } + +func registerRunOpsFlag(valPtr interface{}, info FlagInfo) struct{} { + globalMan.RegisterFlag(runOpsCmdID, valPtr, info) + return struct{}{} +} diff --git a/pkg/cmd/roachtest/roachtestflags/manager.go b/pkg/cmd/roachtest/roachtestflags/manager.go index a18354a1c906..cdc9719fafc6 100644 --- a/pkg/cmd/roachtest/roachtestflags/manager.go +++ b/pkg/cmd/roachtest/roachtestflags/manager.go @@ -23,6 +23,7 @@ type cmdID int const ( listCmdID cmdID = iota runCmdID + runOpsCmdID numCmdIDs ) diff --git a/pkg/cmd/roachtest/run.go b/pkg/cmd/roachtest/run.go index 9a9e5b2d14e1..24a807cee02b 100644 --- a/pkg/cmd/roachtest/run.go +++ b/pkg/cmd/roachtest/run.go @@ -230,7 +230,7 @@ func initRunFlagsBinariesAndLibraries(cmd *cobra.Command) error { return fmt.Errorf("'select-probability' must be in [0,1]") } arm64Opt := cmd.Flags().Lookup("metamorphic-arm64-probability") - if !arm64Opt.Changed && runtime.GOARCH == "arm64" && roachtestflags.Cloud == spec.Local { + if arm64Opt != nil && !arm64Opt.Changed && runtime.GOARCH == "arm64" && roachtestflags.Cloud == spec.Local { fmt.Printf("Detected 'arm64' in 'local mode', setting 'metamorphic-arm64-probability' to 1; use --metamorphic-arm64-probability to run (emulated) with other binaries\n") roachtestflags.ARM64Probability = 1 } @@ -327,3 +327,80 @@ func testRunnerLogger( shout(ctx, l, os.Stdout, "test runner logs in: %s", runnerLogPath) return l, teeOpt } + +// runOperations sequentially runs operations matched by the passed-in filter. +func runOperations(register func(registry.Registry), filter *registry.TestFilter) error { + //lint:ignore SA1019 deprecated + rand.Seed(roachtestflags.GlobalSeed) + r := makeTestRegistry() + + register(&r) + cr := newClusterRegistry() + stopper := stop.NewStopper() + defer stopper.Stop(context.Background()) + runner := newTestRunner(cr, stopper) + runner.config.skipClusterStopOnAttach = true + runner.config.skipClusterWipeOnAttach = true + + clusterType := roachprodCluster + + opt := clustersOpt{ + typ: clusterType, + clusterName: roachtestflags.ClusterNames, + user: getUser(""), + } + opt.debugMode = DebugKeepAlways + + specs, err := opsToRun(r, filter, roachtestflags.RunSkipped, roachtestflags.SelectProbability, true) + if err != nil { + return err + } + + artifactsDir := roachtestflags.ArtifactsDir + + setLogConfig(artifactsDir) + runnerDir := filepath.Join(artifactsDir, runnerLogsDir) + runnerLogPath := filepath.Join( + runnerDir, fmt.Sprintf("test_runner-%d.log", timeutil.Now().Unix())) + l, tee := testRunnerLogger(context.Background(), 1 /* parallelism */, runnerLogPath) + lopt := loggingOpt{ + l: l, + tee: tee, + stdout: os.Stdout, + stderr: os.Stderr, + artifactsDir: artifactsDir, + runnerLogPath: runnerLogPath, + } + l.Printf("global random seed: %d", roachtestflags.GlobalSeed) + go func() { + if err := http.ListenAndServe( + fmt.Sprintf(":%d", roachtestflags.PromPort), + promhttp.HandlerFor(r.promRegistry, promhttp.HandlerOpts{}), + ); err != nil { + l.Errorf("error serving prometheus: %v", err) + } + }() + // We're going to run all the workers (and thus all the tests) in a context + // that gets canceled when the Interrupt signal is received. + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + CtrlC(ctx, l, cancel, cr) + // Install goroutine leak checker and run it at the end of the entire test + // run. If a test is leaking a goroutine, then it will likely be still around. + // We could diff goroutine snapshots before/after each executed test, but that + // could yield false positives; e.g., user-specified test teardown goroutines + // may still be running long after the test has completed. + defer leaktest.AfterTest(l)() + + testSpecs := make([]registry.TestSpec, len(specs)) + for i := range specs { + testSpecs[i] = specs[i].TestSpec() + } + err = runner.Run( + ctx, testSpecs, roachtestflags.Count, 1 /* parallelism */, opt, + testOpts{ + skipInit: true, /* operations always skip init */ + }, + lopt) + return err +} diff --git a/pkg/cmd/roachtest/slack.go b/pkg/cmd/roachtest/slack.go index 085062ff2f2c..35c0c35f7e80 100644 --- a/pkg/cmd/roachtest/slack.go +++ b/pkg/cmd/roachtest/slack.go @@ -18,6 +18,7 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestflags" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/slack-go/slack" ) @@ -50,13 +51,13 @@ func findChannel(client *slack.Client, name string, nextCursor string) (string, return "", fmt.Errorf("not found") } -func sortTests(tests []*testImpl) { +func sortTests(tests []test.Test) { sort.Slice(tests, func(i, j int) bool { return tests[i].Name() < tests[j].Name() }) } -func postSlackReport(pass, fail, skip map[*testImpl]struct{}) { +func postSlackReport(pass, fail, skip map[test.Test]struct{}) { client := makeSlackClient() if client == nil { return @@ -104,7 +105,7 @@ func postSlackReport(pass, fail, skip map[*testImpl]struct{}) { } data := []struct { - tests map[*testImpl]struct{} + tests map[test.Test]struct{} title string color string }{ @@ -113,7 +114,7 @@ func postSlackReport(pass, fail, skip map[*testImpl]struct{}) { {skip, "Skipped", "warning"}, } for _, d := range data { - tests := make([]*testImpl, 0, len(d.tests)) + tests := make([]test.Test, 0, len(d.tests)) for t := range d.tests { tests = append(tests, t) } diff --git a/pkg/cmd/roachtest/test_impl.go b/pkg/cmd/roachtest/test_impl.go index cb58a6aae287..e6e818121c8f 100644 --- a/pkg/cmd/roachtest/test_impl.go +++ b/pkg/cmd/roachtest/test_impl.go @@ -21,6 +21,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestflags" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" "github.com/cockroachdb/cockroach/pkg/testutils/skip" @@ -157,6 +158,13 @@ func (t *testImpl) Cockroach() string { t.l.Printf("Benchmark test, running with standard cockroach") return t.StandardCockroach() } + if t.spec.Operation { + t.l.Printf("Operation, running with passed-in Cockroach") + if roachtestflags.CockroachBinaryPath != "" { + return roachtestflags.CockroachBinaryPath + } + return t.StandardCockroach() + } t.randomCockroachOnce.Do(func() { //TODO(SR): assertions are temporarily disabled for _all_ tests except those using t.RuntimeAssertionsCockroach() // directly, until after the stability period for 23.2. See https://github.com/cockroachdb/cockroach/issues/114615 diff --git a/pkg/cmd/roachtest/test_registry.go b/pkg/cmd/roachtest/test_registry.go index 9501dadeacdd..bd41941d594c 100644 --- a/pkg/cmd/roachtest/test_registry.go +++ b/pkg/cmd/roachtest/test_registry.go @@ -26,6 +26,7 @@ import ( type testRegistryImpl struct { m map[string]*registry.TestSpec + ops map[string]*registry.OperationSpec snapshotPrefixes map[string]struct{} promRegistry *prometheus.Registry @@ -37,6 +38,7 @@ var _ registry.Registry = (*testRegistryImpl)(nil) func makeTestRegistry() testRegistryImpl { return testRegistryImpl{ m: make(map[string]*registry.TestSpec), + ops: make(map[string]*registry.OperationSpec), snapshotPrefixes: make(map[string]struct{}), promRegistry: prometheus.NewRegistry(), } @@ -65,6 +67,19 @@ func (r *testRegistryImpl) Add(spec registry.TestSpec) { r.m[spec.Name] = &spec } +// Add adds a test to the registry. +func (r *testRegistryImpl) AddOperation(spec registry.OperationSpec) { + if _, ok := r.ops[spec.Name]; ok { + fmt.Fprintf(os.Stderr, "operation %s already registered\n", spec.Name) + os.Exit(1) + } + if err := r.prepareOpSpec(&spec); err != nil { + fmt.Fprintf(os.Stderr, "%+v\n", err) + os.Exit(1) + } + r.ops[spec.Name] = &spec +} + // MakeClusterSpec makes a cluster spec. It should be used over `spec.MakeClusterSpec` // because this method also adds options baked into the registry. func (r *testRegistryImpl) MakeClusterSpec(nodeCount int, opts ...spec.Option) spec.ClusterSpec { @@ -73,6 +88,30 @@ func (r *testRegistryImpl) MakeClusterSpec(nodeCount int, opts ...spec.Option) s const testNameRE = "^[a-zA-Z0-9-_=/,]+$" +// prepareOpSpec validates a spec and does minor massaging of its fields. +func (r *testRegistryImpl) prepareOpSpec(spec *registry.OperationSpec) error { + if matched, err := regexp.MatchString(testNameRE, spec.Name); err != nil || !matched { + return fmt.Errorf("%s: Name must match this regexp: %s", spec.Name, testNameRE) + } + + spec.CompatibleClouds.AssertInitialized() + + if spec.Run == nil { + return fmt.Errorf("%s: must specify Run", spec.Name) + } + + // All tests must have an owner so the release team knows who signs off on + // failures and so the github issue poster knows who to assign it to. + if spec.Owner == `` { + return fmt.Errorf(`%s: unspecified owner`, spec.Name) + } + if !spec.Owner.IsValid() { + return fmt.Errorf(`%s: unknown owner %q`, spec.Name, spec.Owner) + } + + return nil +} + // prepareSpec validates a spec and does minor massaging of its fields. func (r *testRegistryImpl) prepareSpec(spec *registry.TestSpec) error { if matched, err := regexp.MatchString(testNameRE, spec.Name); err != nil || !matched { @@ -130,3 +169,15 @@ func (r testRegistryImpl) AllTests() []registry.TestSpec { }) return tests } + +// AllOperations returns all the operation specs, sorted by name. +func (r testRegistryImpl) AllOperations() []registry.OperationSpec { + var ops []registry.OperationSpec + for _, t := range r.ops { + ops = append(ops, *t) + } + sort.Slice(ops, func(i, j int) bool { + return ops[i].Name < ops[j].Name + }) + return ops +} diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index 2f158a763f7e..4dd29f30aa55 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -118,10 +118,10 @@ type testRunner struct { status struct { syncutil.Mutex - running map[*testImpl]struct{} - pass map[*testImpl]struct{} - fail map[*testImpl]struct{} - skip map[*testImpl]struct{} + running map[test.Test]struct{} + pass map[test.Test]struct{} + fail map[test.Test]struct{} + skip map[test.Test]struct{} } // cr keeps track of all live clusters. @@ -316,10 +316,10 @@ func (r *testRunner) Run( parallelism = n * count } - r.status.running = make(map[*testImpl]struct{}) - r.status.pass = make(map[*testImpl]struct{}) - r.status.fail = make(map[*testImpl]struct{}) - r.status.skip = make(map[*testImpl]struct{}) + r.status.running = make(map[test.Test]struct{}) + r.status.pass = make(map[test.Test]struct{}) + r.status.fail = make(map[test.Test]struct{}) + r.status.skip = make(map[test.Test]struct{}) r.work = newWorkPool(tests, count) errs := &workerErrors{} @@ -460,12 +460,21 @@ func (r *testRunner) allocateCluster( skipStop: r.config.skipClusterStopOnAttach, skipWipe: r.config.skipClusterWipeOnAttach, } + if t.Operation { + // Operations don't require the cluster to be registered in the registry, + // so errors here are acceptable. Also allows for better cluster reuse + // as operations don't destroy() clusters. + opt.tolerateRegistrationErrors = true + } // TODO(srosenberg): we need to think about validation here. Attaching to an incompatible cluster, e.g., // using arm64 AMI with amd64 binary, would result in obscure errors. The test runner ensures compatibility // during cluster reuse, whereas attachment via CLI (e.g., via roachprod) does not. lopt.l.PrintfCtx(ctx, "Attaching to existing cluster %s for test %s", existingClusterName, t.Name) c, err := attachToExistingCluster(ctx, existingClusterName, clusterL, t.Cluster, opt, r.cr) if err == nil { + if t.Operation { + c.localCertsDir = roachtestflags.CertsDir + } // Pretend pre-existing's cluster architecture matches the desired one; see the above TODO wrt validation. c.arch = arch return c, nil, nil @@ -589,8 +598,9 @@ func (r *testRunner) runWorker( if c != nil { // Try to reuse cluster. testToRun = work.selectTestForCluster(ctx, c.spec, r.cr) - if !testToRun.noWork { - // We found a test to run on this cluster. Wipe the cluster. + if !testToRun.noWork && !testToRun.spec.Operation { + // We found a test to run on this cluster. Wipe the cluster if it's + // not an operation. if err := c.WipeForReuse(ctx, l, testToRun.spec.Cluster); err != nil { // We do not count reuse attempt error toward clusterCreateErr. If // either the Wipe or Extend failed, then destroy the cluster and attempt @@ -746,6 +756,11 @@ func (r *testRunner) runWorker( debug: clustersOpt.debugMode.IsDebug(), goCoverEnabled: topt.goCoverEnabled, } + if testToRun.spec.Operation { + // This test is an operation. Default to `cockroach` on the cluster's + // nodes. + t.cockroach = "cockroach" + } github := newGithubIssues(r.config.disableIssue, c, vmCreateOpts) // handleClusterCreationFailure can be called when the `err` given @@ -753,6 +768,10 @@ func (r *testRunner) runWorker( // cluster for a test. handleClusterCreationFailure := func(err error) { t.Error(errClusterProvisioningFailed(err)) + // Don't post issues for operations. + if t.spec.Operation { + return + } if _, err := github.MaybePost(t, l, t.failureMsg()); err != nil { shout(ctx, l, stdout, "failed to post issue: %s", err) @@ -770,11 +789,13 @@ func (r *testRunner) runWorker( c.setTest(t) var setupErr error - if c.spec.NodeCount > 0 { // skip during tests - setupErr = c.PutCockroach(ctx, l, t) - } - if setupErr == nil { - setupErr = c.PutLibraries(ctx, "./lib", t.spec.NativeLibs) + if !t.spec.Operation { + if c.spec.NodeCount > 0 { // skip during tests + setupErr = c.PutCockroach(ctx, l, t) + } + if setupErr == nil { + setupErr = c.PutLibraries(ctx, "./lib", t.spec.NativeLibs) + } } if setupErr != nil { @@ -1321,7 +1342,7 @@ func (r *testRunner) teardownTest( t.L().Printf("error collecting artifacts: %v", err) } - if timedOut { + if timedOut && !t.spec.Operation { // Shut down the cluster. We only do this on timeout to help the test terminate; // for regular failures, if the --debug flag is used, we want the cluster to stay // around so someone can poke at it. @@ -1351,7 +1372,7 @@ func (r *testRunner) teardownTest( } func (r *testRunner) collectArtifacts( - ctx context.Context, t *testImpl, c *clusterImpl, timedOut bool, timeout time.Duration, + ctx context.Context, t test.Test, c *clusterImpl, timedOut bool, timeout time.Duration, ) error { // Collecting artifacts may hang so we run it in a goroutine which is abandoned // after a timeout. diff --git a/pkg/cmd/roachtest/tests/restore_test.go b/pkg/cmd/roachtest/tests/restore_test.go index 94f539e28d44..b3dde999be93 100644 --- a/pkg/cmd/roachtest/tests/restore_test.go +++ b/pkg/cmd/roachtest/tests/restore_test.go @@ -36,6 +36,10 @@ func (m *mockRegistry) Add(spec registry.TestSpec) { m.testNames = append(m.testNames, spec.Name) } +func (m *mockRegistry) AddOperation(spec registry.OperationSpec) { + // No-op for now. +} + func (m *mockRegistry) PromFactory() promauto.Factory { return promauto.With(nil) }