Skip to content

Commit

Permalink
roachtest: add roachtest.Operation and run-operation command
Browse files Browse the repository at this point in the history
This change adds the ability to write roachtest Operations that,
unlike a full-on roachtest.Test, do not require an ephemeral cluster
to be spun up or down. These operations are expected to have as few
logical side effects as possible and can be run on a cluster with
running workloads. Running an Operation using `roachtest run-operation`
also guarantees that the Cockroach/Workload binaries on that node
will not be swapped with local ones, and that the cluster won't be
wiped unintentionally at the end or in case of error.

This change also adds add-index and add-column as two example operations
that operate in SQL land and demonstrate the purpose of an Operation.

Release note: None.
  • Loading branch information
itsbilal committed Jan 26, 2024
1 parent 0fe4a72 commit a2581d6
Show file tree
Hide file tree
Showing 22 changed files with 1,673 additions and 29 deletions.
2 changes: 2 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -1149,6 +1149,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",
Expand Down
2 changes: 2 additions & 0 deletions pkg/cmd/roachtest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ go_library(
"github.go",
"main.go",
"monitor.go",
"operation_impl.go",
"run.go",
"slack.go",
"test_filter.go",
Expand All @@ -23,6 +24,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",
Expand Down
33 changes: 24 additions & 9 deletions pkg/cmd/roachtest/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -1815,17 +1819,22 @@ 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 := registry.StandardCockroach
switch tImpl := t.(type) {
case *testImpl:
binaryType = tImpl.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.")
Expand Down Expand Up @@ -2863,6 +2872,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)
Expand Down
111 changes: 111 additions & 0 deletions pkg/cmd/roachtest/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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 or ephemeral cockroach 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()
Expand Down Expand Up @@ -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/).
Expand Down
9 changes: 9 additions & 0 deletions pkg/cmd/roachtest/operation/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -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"],
)
20 changes: 20 additions & 0 deletions pkg/cmd/roachtest/operation/operation_interface.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
// 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 {
test.Test

GetCleanupState(string) string
SetCleanupState(string, string)
}
Loading

0 comments on commit a2581d6

Please sign in to comment.