From e6b6b8c1b17c89119dfcf193d59468ae936e13a8 Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Mon, 9 Oct 2023 13:36:29 -0700 Subject: [PATCH] roachtest: clean up command-line flags The code around command-line flags is pretty messy: they're defined in many places; the name and description of a flag are far away from the variable; the variable names look like local variables and in many cases it's not obvious we're accessing a global. This commit moves all flags to a separate subpackage, `roachtestflags`, making all uses of global flags obvious. We also add a bit of infrastructure to allow defining all information about a flag right next to where the variable is declared. We also provide a `Changed()` function that determines if a flag value was changed (without the caller having to use the Command or even the flag name). There should be no functional changes (just some cosmetic improvements to the flag usage texts). Epic: none Release note: None --- pkg/BUILD.bazel | 3 + pkg/cmd/roachtest/BUILD.bazel | 3 +- pkg/cmd/roachtest/cluster.go | 130 ++---- pkg/cmd/roachtest/main.go | 50 +-- pkg/cmd/roachtest/roachtestflags/BUILD.bazel | 26 ++ pkg/cmd/roachtest/roachtestflags/flags.go | 414 ++++++++++++++++++ pkg/cmd/roachtest/roachtestflags/manager.go | 117 +++++ .../roachtest/roachtestflags/manager_test.go | 110 +++++ pkg/cmd/roachtest/run.go | 229 ++-------- pkg/cmd/roachtest/slack.go | 5 +- pkg/cmd/roachtest/test_filter.go | 36 +- pkg/cmd/roachtest/test_runner.go | 19 +- pkg/cmd/roachtest/test_test.go | 7 +- 13 files changed, 795 insertions(+), 354 deletions(-) create mode 100644 pkg/cmd/roachtest/roachtestflags/BUILD.bazel create mode 100644 pkg/cmd/roachtest/roachtestflags/flags.go create mode 100644 pkg/cmd/roachtest/roachtestflags/manager.go create mode 100644 pkg/cmd/roachtest/roachtestflags/manager_test.go diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 9ed3779200ac..613d6cba5d69 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -147,6 +147,7 @@ ALL_TESTS = [ "//pkg/cmd/roachtest/clusterstats:clusterstats_test", "//pkg/cmd/roachtest/option:option_test", "//pkg/cmd/roachtest/registry:registry_test", + "//pkg/cmd/roachtest/roachtestflags:roachtestflags_test", "//pkg/cmd/roachtest/roachtestutil/mixedversion:mixedversion_test", "//pkg/cmd/roachtest/roachtestutil:roachtestutil_test", "//pkg/cmd/roachtest/tests:tests_test", @@ -1131,6 +1132,8 @@ GO_TARGETS = [ "//pkg/cmd/roachtest/option:option_test", "//pkg/cmd/roachtest/registry:registry", "//pkg/cmd/roachtest/registry:registry_test", + "//pkg/cmd/roachtest/roachtestflags:roachtestflags", + "//pkg/cmd/roachtest/roachtestflags:roachtestflags_test", "//pkg/cmd/roachtest/roachtestutil/clusterupgrade:clusterupgrade", "//pkg/cmd/roachtest/roachtestutil/mixedversion:mixedversion", "//pkg/cmd/roachtest/roachtestutil/mixedversion:mixedversion_test", diff --git a/pkg/cmd/roachtest/BUILD.bazel b/pkg/cmd/roachtest/BUILD.bazel index 15d30b457f1b..d3661610ed3e 100644 --- a/pkg/cmd/roachtest/BUILD.bazel +++ b/pkg/cmd/roachtest/BUILD.bazel @@ -25,6 +25,7 @@ go_library( "//pkg/cmd/roachtest/cluster", "//pkg/cmd/roachtest/option", "//pkg/cmd/roachtest/registry", + "//pkg/cmd/roachtest/roachtestflags", "//pkg/cmd/roachtest/roachtestutil", "//pkg/cmd/roachtest/spec", "//pkg/cmd/roachtest/test", @@ -61,7 +62,6 @@ go_library( "@com_github_prometheus_client_golang//prometheus/promhttp", "@com_github_slack_go_slack//:slack", "@com_github_spf13_cobra//:cobra", - "@com_github_spf13_pflag//:pflag", "@org_golang_x_sync//errgroup", ], ) @@ -93,6 +93,7 @@ go_test( "//pkg/cmd/roachtest/cluster", "//pkg/cmd/roachtest/option", "//pkg/cmd/roachtest/registry", + "//pkg/cmd/roachtest/roachtestflags", "//pkg/cmd/roachtest/spec", "//pkg/cmd/roachtest/test", "//pkg/internal/team", diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index 4fb5c7cfa5de..7eea5f0d376b 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster" "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/cmd/roachtest/roachtestutil" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" @@ -51,7 +52,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" _ "github.com/lib/pq" - "github.com/spf13/pflag" ) func init() { @@ -59,63 +59,14 @@ func init() { } var ( - // user-specified path to crdb binary - cockroachPath string // maps cpuArch to the corresponding crdb binary's absolute path cockroach = make(map[vm.CPUArch]string) - // user-specified path to crdb binary with runtime assertions enabled (EA) - cockroachEAPath string // maps cpuArch to the corresponding crdb binary with runtime assertions enabled (EA) cockroachEA = make(map[vm.CPUArch]string) - // user-specified path to workload binary - workloadPath string // maps cpuArch to the corresponding workload binary's absolute path workload = make(map[vm.CPUArch]string) // maps cpuArch to the corresponding dynamically-linked libraries' absolute paths libraryFilePaths = make(map[vm.CPUArch][]string) - cloud = spec.GCE - // encryptionProbability controls when encryption-at-rest is enabled - // in a cluster for tests that have opted-in to metamorphic - // encryption (EncryptionMetamorphic). - // - // Tests that have opted-in to metamorphic encryption will run with - // encryption enabled by default (probability 1). In order to run - // them with encryption disabled (perhaps to reproduce a test - // failure), roachtest can be invoked with --metamorphic-encryption-probability=0 - encryptionProbability float64 - // Total probability with which new ARM64 clusters are provisioned, modulo test specs. which are incompatible. - // N.B. if all selected tests are incompatible with ARM64, then arm64Probability is effectively 0. - // In other words, ClusterSpec.Arch takes precedence over the arm64Probability flag. - arm64Probability float64 - // Conditional probability with which new FIPS clusters are provisioned, modulo test specs. The total probability - // is the product of this and 1-arm64Probability. - // As in the case of arm64Probability, ClusterSpec.Arch takes precedence over the fipsProbability flag. - fipsProbability float64 - - instanceType string - localSSDArg bool - deprecatedRoachprodBinary string - // overrideOpts contains vm.CreateOpts override values passed from the cli. - overrideOpts vm.CreateOpts - // overrideFlagset represents the flags passed from the cli for - // `run` command (used to know if the value of a flag changed, - // for example: overrideFlagset("lifetime").Changed(). - // TODO(ahmad/healthy-pod): extract runCmd (and other commands) from main - // to make it global and operate on runCmd.Flags() directly. - overrideFlagset *pflag.FlagSet - overrideNumNodes = -1 - clusterName string - local bool - clusterWipe bool - zonesF string - teamCity bool - disableIssue bool -) - -const ( - defaultEncryptionProbability = 1 - defaultFIPSProbability = 0 - defaultARM64Probability = 0 ) type errBinaryOrLibraryNotFound struct { @@ -178,7 +129,7 @@ func findBinary( func findLibrary(libraryName string, os string, arch vm.CPUArch) (string, error) { suffix := ".so" - if cloud == spec.Local { + if roachtestflags.Cloud == spec.Local { switch runtime.GOOS { case "linux": case "freebsd": @@ -308,9 +259,9 @@ func initBinariesAndLibraries() { // see https://github.com/cockroachdb/cockroach/issues/104029. defaultOSName := "linux" defaultArch := vm.ArchAMD64 - if cloud == spec.Local { + if roachtestflags.Cloud == spec.Local { defaultOSName = runtime.GOOS - if arm64Probability == 1 { + if roachtestflags.ARM64Probability == 1 { // N.B. if arm64Probability != 1, then we're running a local cluster with both arm64 and amd64. defaultArch = vm.ArchARM64 } @@ -361,6 +312,9 @@ func initBinariesAndLibraries() { // We need to verify we have at least both the cockroach and the workload binaries. var err error + 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) @@ -368,7 +322,7 @@ func initBinariesAndLibraries() { fmt.Fprintf(os.Stderr, "WARN: unable to find %q for %q: %s\n", "cockroach-ea", defaultArch, err) } - if arm64Probability > 0 && defaultArch != vm.ArchARM64 { + if roachtestflags.ARM64Probability > 0 && defaultArch != vm.ArchARM64 { fmt.Printf("Locating and verifying binaries for os=%q, arch=%q\n", defaultOSName, vm.ArchARM64) // We need to verify we have all the required binaries for arm64. cockroach[vm.ArchARM64], _ = resolveBinary("cockroach", cockroachPath, vm.ArchARM64, true, false) @@ -378,7 +332,7 @@ func initBinariesAndLibraries() { fmt.Fprintf(os.Stderr, "WARN: unable to find %q for %q: %s\n", "cockroach-ea", vm.ArchARM64, err) } } - if fipsProbability > 0 && defaultArch != vm.ArchFIPS { + if roachtestflags.FIPSProbability > 0 && defaultArch != vm.ArchFIPS { fmt.Printf("Locating and verifying binaries for os=%q, arch=%q\n", defaultOSName, vm.ArchFIPS) // We need to verify we have all the required binaries for fips. cockroach[vm.ArchFIPS], _ = resolveBinary("cockroach", cockroachPath, vm.ArchFIPS, true, false) @@ -392,11 +346,11 @@ func initBinariesAndLibraries() { // In v20.2 or higher, optionally expect certain library files to exist. // Since they may not be found in older versions, do not hard error if they are not found. for _, arch := range []vm.CPUArch{vm.ArchAMD64, vm.ArchARM64, vm.ArchFIPS} { - if arm64Probability == 0 && defaultArch != vm.ArchARM64 && arch == vm.ArchARM64 { + if roachtestflags.ARM64Probability == 0 && defaultArch != vm.ArchARM64 && arch == vm.ArchARM64 { // arm64 isn't used, skip finding libs for it. continue } - if fipsProbability == 0 && arch == vm.ArchFIPS { + if roachtestflags.FIPSProbability == 0 && arch == vm.ArchFIPS { // fips isn't used, skip finding libs for it. continue } @@ -838,29 +792,24 @@ func (f *clusterFactory) genName(cfg clusterConfig) string { } // createFlagsOverride updates opts with the override values passed from the cli. -func createFlagsOverride(flags *pflag.FlagSet, opts *vm.CreateOpts) { - if flags != nil { - if flags.Changed("lifetime") { - opts.Lifetime = overrideOpts.Lifetime - } - if flags.Changed("roachprod-local-ssd") { - opts.SSDOpts.UseLocalSSD = overrideOpts.SSDOpts.UseLocalSSD - } - if flags.Changed("filesystem") { - opts.SSDOpts.FileSystem = overrideOpts.SSDOpts.FileSystem - } - if flags.Changed("local-ssd-no-ext4-barrier") { - opts.SSDOpts.NoExt4Barrier = overrideOpts.SSDOpts.NoExt4Barrier - } - if flags.Changed("os-volume-size") { - opts.OsVolumeSize = overrideOpts.OsVolumeSize - } - if flags.Changed("geo") { - opts.GeoDistributed = overrideOpts.GeoDistributed - } - if flags.Changed("label") { - opts.CustomLabels = overrideOpts.CustomLabels - } +func createFlagsOverride(opts *vm.CreateOpts) { + if roachtestflags.Changed(&roachtestflags.Lifetime) { + opts.Lifetime = roachtestflags.Lifetime + } + if roachtestflags.Changed(&roachtestflags.OverrideUseLocalSSD) { + opts.SSDOpts.UseLocalSSD = roachtestflags.OverrideUseLocalSSD + } + if roachtestflags.Changed(&roachtestflags.OverrideFilesystem) { + opts.SSDOpts.FileSystem = roachtestflags.OverrideFilesystem + } + if roachtestflags.Changed(&roachtestflags.OverrideNoExt4Barrier) { + opts.SSDOpts.NoExt4Barrier = roachtestflags.OverrideNoExt4Barrier + } + if roachtestflags.Changed(&roachtestflags.OverrideOSVolumeSizeGB) { + opts.OsVolumeSize = roachtestflags.OverrideOSVolumeSizeGB + } + if roachtestflags.Changed(&roachtestflags.OverrideGeoDistributed) { + opts.GeoDistributed = roachtestflags.OverrideGeoDistributed } } @@ -886,8 +835,8 @@ func (f *clusterFactory) newCluster( return nil, nil, errors.Wrap(ctx.Err(), "newCluster") } - if overrideFlagset != nil && overrideFlagset.Changed("nodes") { - cfg.spec.NodeCount = overrideNumNodes + if roachtestflags.Changed(&roachtestflags.OverrideNumNodes) { + cfg.spec.NodeCount = roachtestflags.OverrideNumNodes } if cfg.spec.NodeCount == 0 { @@ -921,9 +870,9 @@ func (f *clusterFactory) newCluster( UseIOBarrierOnLocalSSD: cfg.useIOBarrier, PreferredArch: cfg.arch, } - params.Defaults.MachineType = instanceType - params.Defaults.Zones = zonesF - params.Defaults.PreferLocalSSD = localSSDArg + params.Defaults.MachineType = roachtestflags.InstanceType + params.Defaults.Zones = roachtestflags.Zones + params.Defaults.PreferLocalSSD = roachtestflags.PreferLocalSSD // The ClusterName is set below in the retry loop to ensure // that each create attempt gets a unique cluster name. @@ -938,7 +887,8 @@ func (f *clusterFactory) newCluster( providerOptsContainer.SetProviderOpts(cfg.spec.Cloud, providerOpts) } - createFlagsOverride(overrideFlagset, &createVMOpts) + createFlagsOverride(&createVMOpts) + // Make sure expiration is changed if --lifetime override flag // is passed. cfg.spec.Lifetime = createVMOpts.Lifetime @@ -1070,7 +1020,7 @@ func attachToExistingCluster( return nil, err } if !opt.skipWipe { - if clusterWipe { + if roachtestflags.ClusterWipe { if err := c.WipeE(ctx, l, false /* preserveCerts */, c.All()); err != nil { return nil, err } @@ -1717,7 +1667,7 @@ func (c *clusterImpl) doDestroy(ctx context.Context, l *logger.Logger) <-chan st return inFlight } - if clusterWipe { + if roachtestflags.ClusterWipe { if c.destroyState.owned { l.PrintfCtx(ctx, "destroying cluster %s...", c) c.status("destroying cluster") @@ -2695,9 +2645,9 @@ func archForTest(ctx context.Context, l *logger.Logger, testSpec registry.TestSp // CPU architecture is unspecified, choose one according to the // probability distribution. var arch vm.CPUArch - if prng.Float64() < arm64Probability { + if prng.Float64() < roachtestflags.ARM64Probability { arch = vm.ArchARM64 - } else if prng.Float64() < fipsProbability { + } else if prng.Float64() < roachtestflags.FIPSProbability { // N.B. branch is taken with probability // (1 - arm64Probability) * fipsProbability // which is P(fips & amd64). diff --git a/pkg/cmd/roachtest/main.go b/pkg/cmd/roachtest/main.go index a073c4c718fc..dd9727491102 100644 --- a/pkg/cmd/roachtest/main.go +++ b/pkg/cmd/roachtest/main.go @@ -20,15 +20,13 @@ import ( "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" - "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestflags" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/tests" "github.com/cockroachdb/cockroach/pkg/roachprod" "github.com/cockroachdb/cockroach/pkg/roachprod/config" - "github.com/cockroachdb/cockroach/pkg/roachprod/vm" "github.com/cockroachdb/errors" _ "github.com/lib/pq" // register postgres driver "github.com/spf13/cobra" - "github.com/spf13/pflag" ) // Note that the custom exit codes below are not exposed when running @@ -52,28 +50,6 @@ const ( runnerLogsDir = "_runner-logs" ) -// Only used if passed otherwise refer to ClusterSpec. -// If a new flag is added here it should also be added to createFlagsOverride(). -func parseCreateOpts(flags *pflag.FlagSet, opts *vm.CreateOpts) { - // roachprod create flags - flags.DurationVar(&opts.Lifetime, - "lifetime", opts.Lifetime, "Lifetime of the cluster") - flags.BoolVar(&opts.SSDOpts.UseLocalSSD, - "roachprod-local-ssd", opts.SSDOpts.UseLocalSSD, "Use local SSD") - flags.StringVar(&opts.SSDOpts.FileSystem, - "filesystem", opts.SSDOpts.FileSystem, "The underlying file system(ext4/zfs).") - flags.BoolVar(&opts.SSDOpts.NoExt4Barrier, - "local-ssd-no-ext4-barrier", opts.SSDOpts.NoExt4Barrier, - `Mount the local SSD with the "-o nobarrier" flag. `+ - `Ignored if --local-ssd=false is specified.`) - flags.IntVarP(&overrideNumNodes, - "nodes", "n", -1, "Total number of nodes") - flags.IntVarP(&opts.OsVolumeSize, - "os-volume-size", "", opts.OsVolumeSize, "OS disk volume size in GB") - flags.BoolVar(&opts.GeoDistributed, - "geo", opts.GeoDistributed, "Create geo-distributed cluster") -} - func main() { cobra.EnableCommandSorting = false @@ -120,7 +96,7 @@ Examples: roachtest list --suite weekly --owner kv `, RunE: func(cmd *cobra.Command, args []string) error { - r := makeTestRegistry(cloud) + r := makeTestRegistry(roachtestflags.Cloud) tests.RegisterTests(&r) filter, err := makeTestFilter(args) @@ -146,11 +122,7 @@ Examples: return nil }, } - addSuiteAndOwnerFlags(listCmd) - listCmd.Flags().BoolVar( - &onlyBenchmarks, "bench", false, "Restricts to benchmarks") - listCmd.Flags().StringVar( - &cloud, "cloud", spec.GCE, "Restricts tests to those compatible with the given cloud (local, aws, azure, gce, or all)") + roachtestflags.AddListFlags(listCmd.Flags()) var runCmd = &cobra.Command{ // Don't display usage when tests fail. @@ -183,6 +155,7 @@ the cluster nodes on start. return runTests(tests.RegisterTests, filter) }, } + roachtestflags.AddRunFlags(runCmd.Flags()) var benchCmd = &cobra.Command{ // Don't display usage when tests fail. @@ -194,7 +167,7 @@ the cluster nodes on start. if err := initRunFlagsBinariesAndLibraries(cmd); err != nil { return err } - onlyBenchmarks = true + roachtestflags.OnlyBenchmarks = true filter, err := makeTestFilter(args) if err != nil { return err @@ -204,12 +177,7 @@ the cluster nodes on start. return runTests(tests.RegisterTests, filter) }, } - - addRunFlags(runCmd) - addBenchFlags(benchCmd) - - parseCreateOpts(runCmd.Flags(), &overrideOpts) - overrideFlagset = runCmd.Flags() + roachtestflags.AddRunFlags(benchCmd.Flags()) rootCmd.AddCommand(listCmd) rootCmd.AddCommand(runCmd) @@ -263,7 +231,7 @@ func testsToRun( if s.Skip == "" || runSkipped { notSkipped = append(notSkipped, s) } else { - if print && teamCity { + if print && roachtestflags.TeamCity { fmt.Fprintf(os.Stdout, "##teamcity[testIgnored name='%s' message='%s']\n", s.Name, teamCityEscape(s.Skip)) } @@ -284,7 +252,7 @@ func testsToRun( if matches, r := filter.Matches(&s); !matches { reason := filter.MatchFailReasonString(r) // This test matches the "relaxed" filter but not the original filter. - if teamCity { + 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) @@ -359,7 +327,7 @@ func selectSpecs( for _, i := range selectedIdxs { for j := p; j < i; j++ { s := specs[j] - if print && teamCity { + if print && roachtestflags.TeamCity { fmt.Fprintf(os.Stdout, "##teamcity[testIgnored name='%s' message='excluded via sampling']\n", s.Name) } diff --git a/pkg/cmd/roachtest/roachtestflags/BUILD.bazel b/pkg/cmd/roachtest/roachtestflags/BUILD.bazel new file mode 100644 index 000000000000..b64c7411bb03 --- /dev/null +++ b/pkg/cmd/roachtest/roachtestflags/BUILD.bazel @@ -0,0 +1,26 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "roachtestflags", + srcs = [ + "flags.go", + "manager.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestflags", + visibility = ["//visibility:public"], + deps = [ + "//pkg/cmd/roachtest/spec", + "@com_github_spf13_pflag//:pflag", + ], +) + +go_test( + name = "roachtestflags_test", + srcs = ["manager_test.go"], + args = ["-test.timeout=295s"], + embed = [":roachtestflags"], + deps = [ + "@com_github_spf13_cobra//:cobra", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/cmd/roachtest/roachtestflags/flags.go b/pkg/cmd/roachtest/roachtestflags/flags.go new file mode 100644 index 000000000000..fffa464e54de --- /dev/null +++ b/pkg/cmd/roachtest/roachtestflags/flags.go @@ -0,0 +1,414 @@ +// Copyright 2023 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 roachtestflags + +import ( + "os" + "time" + + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" + "github.com/spf13/pflag" +) + +// This block defines all roachtest flags (for the list and run/bench commands). +var ( + Cloud string = spec.GCE + _ = registerListFlag(&Cloud, FlagInfo{ + Name: "cloud", + Usage: `List only tests compatible with the given cloud ("local", "gce", + "aws", "azure", or "all")`, + }) + _ = registerRunFlag(&Cloud, FlagInfo{ + Name: "cloud", + Usage: `Cloud provider to use ("local", "gce", "aws", or "azure"); by + default, only tests compatible with the given cloud are run`, + }) + + Suite string + _ = registerListFlag(&Suite, FlagInfo{ + Name: "suite", + Usage: `List only tests from the given suite (e.g. "nightly")`, + }) + _ = registerRunFlag(&Suite, FlagInfo{ + Name: "suite", + Usage: `Run only tests from the given suite (e.g. "nightly")`, + }) + + Owner string + _ = registerListFlag(&Owner, FlagInfo{ + Name: "owner", + Usage: `List only tests with the given owner (e.g. "kv")`, + }) + _ = registerRunFlag(&Owner, FlagInfo{ + Name: "owner", + Usage: `Run only tests with the given owner (e.g. "kv")`, + }) + + OnlyBenchmarks bool + _ = registerListFlag(&OnlyBenchmarks, FlagInfo{ + Name: "bench", + Usage: `List only benchmarks`, + }) + + ForceCloudCompat bool + _ = registerRunFlag(&ForceCloudCompat, FlagInfo{ + Name: "force-cloud-compat", + Usage: `Include tests that are not marked as compatible with the cloud used`, + }) + + ClusterNames string + _ = registerRunFlag(&ClusterNames, FlagInfo{ + Name: "cluster", + Shorthand: "c", + Usage: ` + Comma-separated list of names of existing cluster(s) to use for running + 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.`, + }) + + Local bool + _ = registerRunFlag(&Local, FlagInfo{ + Name: "local", + Shorthand: "l", + Usage: `Run tests locally (equivalent to --cloud=local)`, + }) + + Username string = os.Getenv("ROACHPROD_USER") + _ = registerRunFlag(&Username, FlagInfo{ + Name: "user", + Shorthand: "u", + Usage: ` + Username to use as a cluster name prefix. If blank, the current OS user is + detected and specified`, + }) + + CockroachPath string + _ = registerRunFlag(&CockroachPath, FlagInfo{ + Name: "cockroach", + Usage: `Absolute path to cockroach binary to use`, + }) + + CockroachEAPath string + _ = registerRunFlag(&CockroachEAPath, FlagInfo{ + Name: "cockroach-ea", + Usage: ` + Absolute path to cockroach binary with enabled (runtime) assertions (i.e. + compiled with crdb_test)`, + }) + + WorkloadPath string + _ = registerRunFlag(&WorkloadPath, FlagInfo{ + Name: "workload", + Usage: `Absolute path to workload binary to use`, + }) + + EncryptionProbability float64 = defaultEncryptionProbability + _ = registerRunFlag(&EncryptionProbability, FlagInfo{ + Name: "metamorphic-encryption-probability", + Usage: ` + Probability that clusters will be created with encryption-at-rest enabled + for tests that support metamorphic encryption`, + }) + + FIPSProbability float64 = defaultFIPSProbability + _ = registerRunFlag(&FIPSProbability, FlagInfo{ + Name: "metamorphic-fips-probability", + Usage: ` + Conditional probability that amd64 clusters will be created with FIPS, + i.e., P(fips | amd64), for tests that support FIPS and whose CPU + architecture is 'amd64' (default 0) NOTE: amd64 clusters are created with + probability 1-P(arm64), where P(arm64) is 'metamorphic-arm64-probability'; + hence, P(fips | amd64) = P(fips) * (1 - P(arm64))`, + }) + + ARM64Probability float64 = defaultARM64Probability + _ = registerRunFlag(&ARM64Probability, FlagInfo{ + Name: "metamorphic-arm64-probability", + Usage: ` + Probability that clusters will be created with 'arm64' CPU architecture + for tests that support 'arm64' (default 0)`, + }) + + // 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{ + Name: "artifacts", + Usage: `Path to artifacts directory`, + }) + + // LiteralArtifactsDir is a path to the literal on-agent directory where + // artifacts are stored. May be different from `artifacts`. Only used for + // messages to ##teamcity[publishArtifacts] in Teamcity mode. + LiteralArtifactsDir string + _ = registerRunFlag(&LiteralArtifactsDir, FlagInfo{ + Name: "artifacts-literal", + Usage: ` + Literal path to on-agent artifacts directory. Used for messages to + ##teamcity[publishArtifacts] in --teamcity mode. May be different from + --artifacts; defaults to the value of --artifacts if not provided`, + }) + + ClusterID string + _ = registerRunFlag(&ClusterID, FlagInfo{ + Name: "cluster-id", + Usage: `An identifier to use in the name of the test cluster(s)`, + }) + + Count int = 1 + _ = registerRunFlag(&Count, FlagInfo{ + Name: "count", + Usage: `the number of times to run each test`, + }) + + DebugOnFailure bool + _ = registerRunFlag(&DebugOnFailure, FlagInfo{ + Name: "debug", + Shorthand: "d", + Usage: `Don't wipe and destroy cluster if test fails`, + }) + + DebugAlways bool + _ = registerRunFlag(&DebugAlways, FlagInfo{ + Name: "debug-always", + Usage: `Never wipe and destroy the cluster`, + }) + + RunSkipped bool + _ = registerRunFlag(&RunSkipped, FlagInfo{ + Name: "run-skipped", + Usage: `Run skipped tests`, + }) + + SkipInit bool + _ = registerRunFlag(&SkipInit, FlagInfo{ + Name: "skip-init", + Usage: ` + Skip initialization step (imports, table creation, etc.) for tests that + support it, useful when re-using clusters with --wipe=false`, + }) + + GoCoverEnabled bool + _ = registerRunFlag(&GoCoverEnabled, FlagInfo{ + Name: "go-cover", + Usage: ` + Enable collection of go coverage profiles (requires instrumented cockroach + binary)`, + }) + + Parallelism int = 10 + _ = registerRunFlag(&Parallelism, FlagInfo{ + Name: "parallelism", + Usage: `Number of tests to run in parallel`, + }) + + deprecatedRoachprodBinary string + _ = registerRunFlag(&deprecatedRoachprodBinary, FlagInfo{ + Name: "roachprod", + Usage: "DEPRECATED", + Deprecated: "roachtest now uses roachprod as a library", + }) + + ClusterWipe bool = true + _ = registerRunFlag(&ClusterWipe, FlagInfo{ + Name: "wipe", + Usage: `Wipe existing cluster before starting test (for use with --cluster)`, + }) + + Zones string + _ = registerRunFlag(&Zones, FlagInfo{ + Name: "zones", + Usage: ` + Zones for the cluster. (non-geo tests use the first zone, geo tests use + all zones; uses defaults if empty)`, + }) + + InstanceType string + _ = registerRunFlag(&InstanceType, FlagInfo{ + Name: "instance-type", + Usage: ` + The instance type to use (see https://aws.amazon.com/ec2/instance-types/, + https://cloud.google.com/compute/docs/machine-types or + https://docs.microsoft.com/en-us/azure/virtual-machines/windows/sizes)`, + }) + + CPUQuota int = 300 + _ = registerRunFlag(&CPUQuota, FlagInfo{ + Name: "cpu-quota", + Usage: `The number of cloud CPUs roachtest is allowed to use at any one time.`, + }) + + HTTPPort int = 8080 + _ = registerRunFlag(&HTTPPort, FlagInfo{ + Name: "port", + Usage: `The port on which to serve the HTTP interface`, + }) + + PreferLocalSSD bool = true + _ = registerRunFlag(&PreferLocalSSD, FlagInfo{ + Name: "local-ssd", + Usage: `Use a local SSD instead of an EBS volume, if the instance supports it`, + }) + + VersionsBinaryOverride map[string]string + _ = registerRunFlag(&VersionsBinaryOverride, FlagInfo{ + Name: "versions-binary-override", + Usage: ` + List of =. If a certain version + is present in the list, the respective binary will be used when a + mixed-version test asks for the respective binary, instead of roachprod + stage . Example: 20.1.4=cockroach-20.1,20.2.0=cockroach-20.2.`, + }) + + SlackToken string + _ = registerRunFlag(&SlackToken, FlagInfo{ + Name: "slack-token", + Usage: `Slack bot token`, + }) + + TeamCity bool + _ = registerRunFlag(&TeamCity, FlagInfo{ + Name: "teamcity", + Usage: `Include teamcity-specific markers in output`, + }) + + DisableIssue bool + _ = registerRunFlag(&DisableIssue, FlagInfo{ + Name: "disable-issue", + Usage: `Disable posting GitHub issue for failures`, + }) + + PromPort int = 2113 + _ = registerRunFlag(&PromPort, FlagInfo{ + Name: "prom-port", + Usage: ` + The http port on which to expose prom metrics from the roachtest + process`, + }) + + SelectProbability float64 = 1.0 + _ = registerRunFlag(&SelectProbability, FlagInfo{ + Name: "select-probability", + Usage: ` + The probability of a matched test being selected to run. Note: this will + run at least one test per prefix.`, + }) +) + +// The flags below override the final cluster configuration. They have no +// default values and are only effectual when they are specified (all uses are +// gated behind Changed() calls). +var ( + Lifetime time.Duration + _ = registerRunFlag(&Lifetime, FlagInfo{ + Name: "lifetime", + Usage: `Lifetime of the cluster`, + }) + + OverrideUseLocalSSD bool + _ = registerRunFlag(&OverrideUseLocalSSD, FlagInfo{ + Name: "roachprod-local-ssd", + Usage: `Override use of local SSD`, + }) + + OverrideFilesystem string + _ = registerRunFlag(&OverrideFilesystem, FlagInfo{ + Name: "filesystem", + Usage: `Override the underlying file system(ext4/zfs)`, + }) + + OverrideNoExt4Barrier bool + _ = registerRunFlag(&OverrideNoExt4Barrier, FlagInfo{ + Name: "local-ssd-no-ext4-barrier", + Usage: ` + Mount the local SSD with the "-o nobarrier" flag. Ignored if not using + local SSD`, + }) + + OverrideNumNodes int + _ = registerRunFlag(&OverrideNumNodes, FlagInfo{ + Name: "nodes", + Shorthand: "n", + Usage: `Override the number of nodes in the cluster`, + }) + + OverrideOSVolumeSizeGB int + _ = registerRunFlag(&OverrideOSVolumeSizeGB, FlagInfo{ + Name: "os-volume-size", + Usage: `Override OS disk volume size (in GB)`, + }) + + OverrideGeoDistributed bool + _ = registerRunFlag(&OverrideGeoDistributed, FlagInfo{ + Name: "geo", + Usage: `Create geo-distributed cluster`, + }) +) + +const ( + defaultEncryptionProbability = 1 + defaultFIPSProbability = 0 + defaultARM64Probability = 0 +) + +// FlagInfo contains the name and usage of a flag. Used to make the code +// defining them self-documenting. +type FlagInfo struct { + // Name of the flag (as will be passed in the command-line). + Name string + + // Shorthand is the one-letter abbreviated flag that can be used with a single + // dash (optional). + Shorthand string + + // Usage description. The string can be broken up into many lines + // arbitrarily; it is cleaned up to a single line with extra whitespace + // removed. + Usage string + + // Deprecated is used only for deprecated flags; it is the message shown when + // the flag is used. + Deprecated string +} + +// AddListFlags adds all flags registered for the list command to the given +// command flag set. +func AddListFlags(cmdFlags *pflag.FlagSet) { + globalMan.AddFlagsToCommand(listCmdID, cmdFlags) +} + +// AddRunFlags adds all flags registered for the run command to the given +// command flag set. +func AddRunFlags(cmdFlags *pflag.FlagSet) { + globalMan.AddFlagsToCommand(runCmdID, cmdFlags) +} + +// Changed returns true if a flag associated with a given value was present. +// +// For example: roachtestflags.Changed(&roachtestflags.Cloud) returns true if +// the `--cloud` flag was passed (even if the given value was the same with the +// default value). +func Changed(valPtr interface{}) bool { + return globalMan.Changed(valPtr) +} + +var globalMan manager + +func registerListFlag(valPtr interface{}, info FlagInfo) struct{} { + globalMan.RegisterFlag(listCmdID, valPtr, info) + return struct{}{} +} + +func registerRunFlag(valPtr interface{}, info FlagInfo) struct{} { + globalMan.RegisterFlag(runCmdID, valPtr, info) + return struct{}{} +} diff --git a/pkg/cmd/roachtest/roachtestflags/manager.go b/pkg/cmd/roachtest/roachtestflags/manager.go new file mode 100644 index 000000000000..ec7cd49fd2fd --- /dev/null +++ b/pkg/cmd/roachtest/roachtestflags/manager.go @@ -0,0 +1,117 @@ +// Copyright 2023 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 roachtestflags + +import ( + "fmt" + "strings" + "time" + + "github.com/spf13/pflag" +) + +type cmdID int + +const ( + listCmdID cmdID = iota + runCmdID + numCmdIDs +) + +type manager struct { + flags [numCmdIDs]map[interface{}]*flagData +} + +type flagData struct { + FlagInfo + // Flag sets to which the flag has been added. + flagSets []*pflag.FlagSet +} + +// RegisterFlag defines a flag for a type of command. +// +// It is not allowed to register the same value pointer to the same command +// multiple times. +func (m *manager) RegisterFlag(cmd cmdID, valPtr interface{}, info FlagInfo) { + if _, ok := m.flags[cmd][valPtr]; ok { + panic("flag value registered twice") + } + if m.flags[cmd] == nil { + m.flags[cmd] = make(map[interface{}]*flagData) + } + m.flags[cmd][valPtr] = &flagData{ + FlagInfo: info, + } +} + +// AddFlagsToCommand adds all flags registered to cmd to the flag set for a +// command. +func (m *manager) AddFlagsToCommand(cmd cmdID, cmdFlags *pflag.FlagSet) { + for p, f := range m.flags[cmd] { + usage := cleanupString(f.Usage) + switch p := p.(type) { + case *bool: + cmdFlags.BoolVarP(p, f.Name, f.Shorthand, *p, usage) + case *int: + cmdFlags.IntVarP(p, f.Name, f.Shorthand, *p, usage) + case *float64: + cmdFlags.Float64VarP(p, f.Name, f.Shorthand, *p, usage) + case *time.Duration: + cmdFlags.DurationVarP(p, f.Name, f.Shorthand, *p, usage) + case *string: + cmdFlags.StringVarP(p, f.Name, f.Shorthand, *p, usage) + case *map[string]string: + cmdFlags.StringToStringVarP(p, f.Name, f.Shorthand, *p, usage) + default: + panic(fmt.Sprintf("unsupported pointer type %T", p)) + } + if f.Deprecated != "" { + if err := cmdFlags.MarkDeprecated(f.Name, cleanupString(f.Deprecated)); err != nil { + // An error here is not possible, as we just defined this flag above. + panic(err) + } + } + f.flagSets = append(f.flagSets, cmdFlags) + } +} + +// Changed returns true if a flag associated with the given value was passed. +func (m *manager) Changed(valPtr interface{}) bool { + // We don't know which command we're running, but we'll only run one per + // program invocation; so check all of them. + for cmd := cmdID(0); cmd < numCmdIDs; cmd++ { + if f, ok := m.flags[cmd][valPtr]; ok { + for _, flagSet := range f.flagSets { + if flagSet.Changed(f.Name) { + return true + } + } + } + } + return false +} + +// cleanupString converts a multi-line string into a single-line string, +// removing all extra whitespace at the beginning and end of lines. +func cleanupString(s string) string { + l := strings.Split(s, "\n") + for i := range l { + l[i] = strings.TrimSpace(l[i]) + } + // Remove leading and trailing empty lines. + for len(l) > 0 && l[0] == "" { + l = l[1:] + } + for len(l) > 0 && l[len(l)-1] == "" { + l = l[:len(l)-1] + } + return strings.Join(l, " ") +} diff --git a/pkg/cmd/roachtest/roachtestflags/manager_test.go b/pkg/cmd/roachtest/roachtestflags/manager_test.go new file mode 100644 index 000000000000..3a9293533c14 --- /dev/null +++ b/pkg/cmd/roachtest/roachtestflags/manager_test.go @@ -0,0 +1,110 @@ +// Copyright 2023 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 roachtestflags + +import ( + "testing" + + "github.com/spf13/cobra" + "github.com/stretchr/testify/require" +) + +type testValues struct { + intVal int + boolVal bool + stringVal string +} + +func initTest() (*manager, *testValues) { + m := &manager{} + tv := &testValues{} + m.RegisterFlag(listCmdID, &tv.intVal, FlagInfo{ + Name: "some-int", + Usage: `usage usage usage +usage usage`, + }) + m.RegisterFlag(runCmdID, &tv.boolVal, FlagInfo{ + Name: "some-bool", + Shorthand: "b", + Usage: `usage for the bool`, + }) + m.RegisterFlag(listCmdID, &tv.stringVal, FlagInfo{ + Name: "some-string", + Usage: `usage for the string in list`, + }) + m.RegisterFlag(runCmdID, &tv.stringVal, FlagInfo{ + Name: "some-string", + Shorthand: "s", + Usage: `usage for the string in run`, + }) + return m, tv +} + +func TestManager(t *testing.T) { + t.Run("list1", func(t *testing.T) { + m, tv := initTest() + listCmd := &cobra.Command{} + m.AddFlagsToCommand(listCmdID, listCmd.Flags()) + require.NoError(t, listCmd.ParseFlags([]string{"--some-int", "123"})) + require.Equal(t, 123, tv.intVal) + require.True(t, m.Changed(&tv.intVal)) + require.False(t, m.Changed(&tv.boolVal)) + require.False(t, m.Changed(&tv.stringVal)) + }) + + t.Run("list2", func(t *testing.T) { + m, tv := initTest() + listCmd := &cobra.Command{} + m.AddFlagsToCommand(listCmdID, listCmd.Flags()) + require.NoError(t, listCmd.ParseFlags([]string{"--some-int", "123", "--some-string", "foo"})) + require.Equal(t, 123, tv.intVal) + require.Equal(t, "foo", tv.stringVal) + require.True(t, m.Changed(&tv.intVal)) + require.False(t, m.Changed(&tv.boolVal)) + require.True(t, m.Changed(&tv.stringVal)) + }) + + t.Run("run", func(t *testing.T) { + m, tv := initTest() + runCmd := &cobra.Command{} + m.AddFlagsToCommand(runCmdID, runCmd.Flags()) + require.NoError(t, runCmd.ParseFlags([]string{"-b"})) + require.True(t, tv.boolVal) + require.False(t, m.Changed(&tv.intVal)) + require.True(t, m.Changed(&tv.boolVal)) + require.False(t, m.Changed(&tv.stringVal)) + }) + + t.Run("run_and_bench", func(t *testing.T) { + m, tv := initTest() + runCmd := &cobra.Command{} + benchCmd := &cobra.Command{} + m.AddFlagsToCommand(runCmdID, runCmd.Flags()) + m.AddFlagsToCommand(runCmdID, benchCmd.Flags()) + require.NoError(t, runCmd.ParseFlags([]string{"--some-bool=false", "-s", "foo"})) + require.False(t, tv.boolVal) + require.Equal(t, "foo", tv.stringVal) + require.False(t, m.Changed(&tv.intVal)) + require.True(t, m.Changed(&tv.boolVal)) + require.True(t, m.Changed(&tv.stringVal)) + }) +} + +func TestCleanupString(t *testing.T) { + in := ` + this is + a string that has been broken up into many lines, +because why not + +` + out := cleanupString(in) + require.Equal(t, "this is a string that has been broken up into many lines, because why not", out) +} diff --git a/pkg/cmd/roachtest/run.go b/pkg/cmd/roachtest/run.go index 7edacd7de831..96438d1548c7 100644 --- a/pkg/cmd/roachtest/run.go +++ b/pkg/cmd/roachtest/run.go @@ -22,6 +22,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/spec" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" "github.com/cockroachdb/cockroach/pkg/util/allstacks" @@ -36,150 +37,10 @@ import ( "github.com/spf13/cobra" ) -var ( - username string - - parallelism int - cpuQuota int - - // Path to a local dir where the test logs and artifacts collected from - // cluster will be placed. - artifactsDir string - - // Path to the literal on-agent directory where artifacts are stored. - // May be different from `artifacts`. Only used for messages to - // ##teamcity[publishArtifacts] in Teamcity mode. - literalArtifactsDir string - - httpPort int - promPort int - - debugOnFailure bool - debugAlways bool - runSkipped bool - skipInit bool - goCoverEnabled bool - clusterID string - count int - versionsBinaryOverride map[string]string - - selectProbability float64 -) - -// addRunBenchCommonFlags adds flags that are used for the run and bench commands. -// It is called with runCmd and benchCmd. -func addRunBenchCommonFlags(cmd *cobra.Command) { - cmd.Flags().StringVarP( - &clusterName, "cluster", "c", "", - "Comma-separated list of names existing cluster to use for running 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.") - cmd.Flags().BoolVarP( - &local, "local", "l", local, "run tests locally") - cmd.Flags().StringVarP( - &username, "user", "u", os.Getenv("ROACHPROD_USER"), - "Username to use as a cluster name prefix. "+ - "If blank, the current OS user is detected and specified.") - cmd.Flags().StringVar( - &cockroachPath, "cockroach", "", "absolute path to cockroach binary to use") - cmd.Flags().StringVar( - &cockroachEAPath, "cockroach-ea", "", "absolute path to cockroach binary with enabled (runtime) assertions (i.e, compiled with crdb_test)") - cmd.Flags().StringVar( - &workloadPath, "workload", "", "absolute path to workload binary to use") - cmd.Flags().Float64Var( - &encryptionProbability, "metamorphic-encryption-probability", defaultEncryptionProbability, - "probability that clusters will be created with encryption-at-rest enabled "+ - "for tests that support metamorphic encryption (default 1.0)") - cmd.Flags().Float64Var( - &fipsProbability, "metamorphic-fips-probability", defaultFIPSProbability, - "conditional probability that amd64 clusters will be created with FIPS, i.e., P(fips | amd64), "+ - "for tests that support FIPS and whose CPU architecture is 'amd64' (default 0) "+ - "NOTE: amd64 clusters are created with probability 1-P(arm64), where P(arm64) is 'metamorphic-arm64-probability'. "+ - "Hence, P(fips | amd64) = P(fips) * (1 - P(arm64))") - cmd.Flags().Float64Var( - &arm64Probability, "metamorphic-arm64-probability", defaultARM64Probability, - "probability that clusters will be created with 'arm64' CPU architecture "+ - "for tests that support 'arm64' (default 0)") - - cmd.Flags().StringVar( - &artifactsDir, "artifacts", "artifacts", "path to artifacts directory") - cmd.Flags().StringVar( - &literalArtifactsDir, "artifacts-literal", "", "literal path to on-agent artifacts directory. Used for messages to ##teamcity[publishArtifacts] in --teamcity mode. May be different from --artifacts; defaults to the value of --artifacts if not provided") - cmd.Flags().StringVar( - &cloud, "cloud", cloud, "cloud provider to use (local, aws, azure, or gce)") - cmd.Flags().StringVar( - &clusterID, "cluster-id", "", "an identifier to use in the name of the test cluster(s)") - cmd.Flags().IntVar( - &count, "count", 1, "the number of times to run each test") - cmd.Flags().BoolVarP( - &debugOnFailure, "debug", "d", debugOnFailure, "don't wipe and destroy cluster if test fails") - cmd.Flags().BoolVar( - &debugAlways, "debug-always", debugAlways, "never wipe and destroy the cluster") - cmd.Flags().BoolVar( - &runSkipped, "run-skipped", runSkipped, "run skipped tests") - cmd.Flags().BoolVar( - &skipInit, "skip-init", false, "skip initialization step (imports, table creation, etc.) for tests that support it, useful when re-using clusters with --wipe=false") - cmd.Flags().BoolVar( - &goCoverEnabled, "go-cover", false, "enable collection of go coverage profiles (requires instrumented cockroach binary)") - cmd.Flags().IntVarP( - ¶llelism, "parallelism", "p", 10, "number of tests to run in parallel") - cmd.Flags().StringVar( - &deprecatedRoachprodBinary, "roachprod", "", "DEPRECATED") - _ = cmd.Flags().MarkDeprecated("roachprod", "roachtest now uses roachprod as a library") - cmd.Flags().BoolVar( - &clusterWipe, "wipe", true, - "wipe existing cluster before starting test (for use with --cluster)") - cmd.Flags().StringVar( - &zonesF, "zones", "", - "Zones for the cluster. (non-geo tests use the first zone, geo tests use all zones) "+ - "(uses roachprod defaults if empty)") - cmd.Flags().StringVar( - &instanceType, "instance-type", instanceType, - "the instance type to use (see https://aws.amazon.com/ec2/instance-types/, https://cloud.google.com/compute/docs/machine-types or https://docs.microsoft.com/en-us/azure/virtual-machines/windows/sizes)") - cmd.Flags().IntVar( - &cpuQuota, "cpu-quota", 300, - "The number of cloud CPUs roachtest is allowed to use at any one time.") - cmd.Flags().IntVar( - &httpPort, "port", 8080, "the port on which to serve the HTTP interface") - cmd.Flags().BoolVar( - &localSSDArg, "local-ssd", true, "Use a local SSD instead of an EBS volume (only for use with AWS) (defaults to true if instance type supports local SSDs)") - cmd.Flags().StringToStringVar( - &versionsBinaryOverride, "versions-binary-override", nil, - "List of =. If a certain version "+ - "is present in the list, the respective binary will be used when a "+ - "mixed-version test asks for the respective binary, instead of "+ - "`roachprod stage `. Example: 20.1.4=cockroach-20.1,20.2.0=cockroach-20.2.") - cmd.Flags().BoolVar( - &forceCloudCompat, "force-cloud-compat", false, "Includes tests that are not marked as compatible with the cloud used") - addSuiteAndOwnerFlags(cmd) -} - -func addRunFlags(runCmd *cobra.Command) { - addRunBenchCommonFlags(runCmd) - - runCmd.Flags().StringVar( - &slackToken, "slack-token", "", "Slack bot token") - runCmd.Flags().BoolVar( - &teamCity, "teamcity", false, "include teamcity-specific markers in output") - runCmd.Flags().BoolVar( - &disableIssue, "disable-issue", false, "disable posting GitHub issue for failures") - runCmd.Flags().IntVar( - &promPort, "prom-port", 2113, - "the http port on which to expose prom metrics from the roachtest process") - runCmd.Flags().Float64Var( - &selectProbability, "select-probability", 1.0, - "the probability of a matched test being selected to run. Note: this will return at least one test per prefix.") -} - -func addBenchFlags(benchCmd *cobra.Command) { - addRunBenchCommonFlags(benchCmd) -} - // runTests is the main function for the run and bench commands. // Assumes initRunFlagsBinariesAndLibraries was called. func runTests(register func(registry.Registry), filter *registry.TestFilter) error { - r := makeTestRegistry(cloud) + r := makeTestRegistry(roachtestflags.Cloud) // actual registering of tests // TODO: don't register if we can't run on the specified registry cloud @@ -191,7 +52,8 @@ func runTests(register func(registry.Registry), filter *registry.TestFilter) err clusterType := roachprodCluster bindTo := "" - if cloud == spec.Local { + parallelism := roachtestflags.Parallelism + if roachtestflags.Cloud == spec.Local { clusterType = localCluster // This will suppress the annoying "Allow incoming network connections" popup from @@ -207,42 +69,48 @@ func runTests(register func(registry.Registry), filter *registry.TestFilter) err opt := clustersOpt{ typ: clusterType, - clusterName: clusterName, + clusterName: roachtestflags.ClusterNames, // Precedence for resolving the user: cli arg, env.ROACHPROD_USER, current user. - user: getUser(username), - cpuQuota: cpuQuota, - clusterID: clusterID, + user: getUser(roachtestflags.Username), + cpuQuota: roachtestflags.CPUQuota, + clusterID: roachtestflags.ClusterID, } switch { - case debugAlways: + case roachtestflags.DebugAlways: opt.debugMode = DebugKeepAlways - case debugOnFailure: + case roachtestflags.DebugOnFailure: opt.debugMode = DebugKeepOnFailure default: opt.debugMode = NoDebug } - if err := runner.runHTTPServer(httpPort, os.Stdout, bindTo); err != nil { + if err := runner.runHTTPServer(roachtestflags.HTTPPort, os.Stdout, bindTo); err != nil { return err } - specs, err := testsToRun(r, filter, runSkipped, selectProbability, true) + specs, err := testsToRun(r, filter, roachtestflags.RunSkipped, roachtestflags.SelectProbability, true) if err != nil { return err } n := len(specs) - if n*count < parallelism { + if n*roachtestflags.Count < parallelism { // Don't spin up more workers than necessary. This has particular // implications for the common case of running a single test once: if // parallelism is set to 1, we'll use teeToStdout below to get logs to // stdout/stderr. - parallelism = n * count + parallelism = n * roachtestflags.Count } if opt.debugMode == DebugKeepAlways && n > 1 { return errors.Newf("--debug-always is only allowed when running a single test") } + artifactsDir := roachtestflags.ArtifactsDir + literalArtifactsDir := roachtestflags.LiteralArtifactsDir + if literalArtifactsDir == "" { + literalArtifactsDir = artifactsDir + } + setLogConfig(artifactsDir) runnerDir := filepath.Join(artifactsDir, runnerLogsDir) runnerLogPath := filepath.Join( @@ -259,7 +127,7 @@ func runTests(register func(registry.Registry), filter *registry.TestFilter) err } go func() { if err := http.ListenAndServe( - fmt.Sprintf(":%d", promPort), + fmt.Sprintf(":%d", roachtestflags.PromPort), promhttp.HandlerFor(r.promRegistry, promhttp.HandlerOpts{}), ); err != nil { l.Errorf("error serving prometheus: %v", err) @@ -278,11 +146,11 @@ func runTests(register func(registry.Registry), filter *registry.TestFilter) err defer leaktest.AfterTest(l)() err = runner.Run( - ctx, specs, count, parallelism, opt, + ctx, specs, roachtestflags.Count, parallelism, opt, testOpts{ - versionsBinaryOverride: versionsBinaryOverride, - skipInit: skipInit, - goCoverEnabled: goCoverEnabled, + versionsBinaryOverride: roachtestflags.VersionsBinaryOverride, + skipInit: roachtestflags.SkipInit, + goCoverEnabled: roachtestflags.GoCoverEnabled, }, lopt) @@ -293,7 +161,7 @@ func runTests(register func(registry.Registry), filter *registry.TestFilter) err l.PrintfCtx(ctx, "runTests destroying all clusters") cr.destroyAllClusters(context.Background(), l) - if teamCity { + if roachtestflags.TeamCity { // Collect the runner logs. fmt.Printf("##teamcity[publishArtifacts '%s']\n", filepath.Join(literalArtifactsDir, runnerLogsDir)) } @@ -328,61 +196,58 @@ func getUser(userFlag string) string { } func initRunFlagsBinariesAndLibraries(cmd *cobra.Command) error { - if local { - if clusterName != "" { + if roachtestflags.Local { + if roachtestflags.ClusterNames != "" { return fmt.Errorf( "cannot specify both an existing cluster (%s) and --local. However, if a local cluster "+ "already exists, --clusters=local will use it", - clusterName) + roachtestflags.ClusterNames) } - cloud = spec.Local + roachtestflags.Cloud = spec.Local } - if count <= 0 { - return fmt.Errorf("--count (%d) must by greater than 0", count) - } - if literalArtifactsDir == "" { - literalArtifactsDir = artifactsDir + if roachtestflags.Count <= 0 { + return fmt.Errorf("--count (%d) must by greater than 0", roachtestflags.Count) } - if !(0 <= arm64Probability && arm64Probability <= 1) { + if !(0 <= roachtestflags.ARM64Probability && roachtestflags.ARM64Probability <= 1) { return fmt.Errorf("'metamorphic-arm64-probability' must be in [0,1]") } - if !(0 <= fipsProbability && fipsProbability <= 1) { + if !(0 <= roachtestflags.FIPSProbability && roachtestflags.FIPSProbability <= 1) { return fmt.Errorf("'metamorphic-fips-probability' must be in [0,1]") } - if arm64Probability == 1 && fipsProbability != 0 { + if roachtestflags.ARM64Probability == 1 && roachtestflags.FIPSProbability != 0 { return fmt.Errorf("'metamorphic-fips-probability' must be 0 when 'metamorphic-arm64-probability' is 1") } - if fipsProbability == 1 && arm64Probability != 0 { + if roachtestflags.FIPSProbability == 1 && roachtestflags.ARM64Probability != 0 { return fmt.Errorf("'metamorphic-arm64-probability' must be 0 when 'metamorphic-fips-probability' is 1") } - if !(0 <= selectProbability && selectProbability <= 1) { + if !(0 <= roachtestflags.SelectProbability && roachtestflags.SelectProbability <= 1) { return fmt.Errorf("'select-probability' must be in [0,1]") } arm64Opt := cmd.Flags().Lookup("metamorphic-arm64-probability") - if !arm64Opt.Changed && runtime.GOARCH == "arm64" && cloud == spec.Local { + if !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") - arm64Probability = 1 + roachtestflags.ARM64Probability = 1 } // Find and validate all required binaries and libraries. initBinariesAndLibraries() - if arm64Probability > 0 { - fmt.Printf("ARM64 clusters will be provisioned with probability %.2f\n", arm64Probability) + if roachtestflags.ARM64Probability > 0 { + fmt.Printf("ARM64 clusters will be provisioned with probability %.2f\n", roachtestflags.ARM64Probability) } - amd64Probability := 1 - arm64Probability + amd64Probability := 1 - roachtestflags.ARM64Probability if amd64Probability > 0 { fmt.Printf("AMD64 clusters will be provisioned with probability %.2f\n", amd64Probability) } - if fipsProbability > 0 { - // N.B. arm64Probability < 1, otherwise fipsProbability == 0, as per above check. + if roachtestflags.FIPSProbability > 0 { + // N.B. roachtestflags.ARM64Probability < 1, otherwise roachtestflags.FIPSProbability == 0, as per above check. // Hence, amd64Probability > 0 is implied. - fmt.Printf("FIPS clusters will be provisioned with probability %.2f\n", fipsProbability*amd64Probability) + fmt.Printf("FIPS clusters will be provisioned with probability %.2f\n", roachtestflags.FIPSProbability*amd64Probability) } - if selectProbability > 0 { - fmt.Printf("Matching tests will be selected with probability %.2f\n", selectProbability) + if roachtestflags.SelectProbability > 0 && roachtestflags.SelectProbability < 1 { + fmt.Printf("Matching tests will be selected with probability %.2f\n", roachtestflags.SelectProbability) } return nil } diff --git a/pkg/cmd/roachtest/slack.go b/pkg/cmd/roachtest/slack.go index 40f2505003e0..085062ff2f2c 100644 --- a/pkg/cmd/roachtest/slack.go +++ b/pkg/cmd/roachtest/slack.go @@ -17,6 +17,7 @@ import ( "sort" "strings" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestflags" "github.com/slack-go/slack" ) @@ -73,8 +74,8 @@ func postSlackReport(pass, fail, skip map[*testImpl]struct{}) { var prefix string switch { - case cloud != "": - prefix = strings.ToUpper(cloud) + case roachtestflags.Cloud != "": + prefix = strings.ToUpper(roachtestflags.Cloud) default: prefix = "GCE" } diff --git a/pkg/cmd/roachtest/test_filter.go b/pkg/cmd/roachtest/test_filter.go index 78e7731ab656..131541a45080 100644 --- a/pkg/cmd/roachtest/test_filter.go +++ b/pkg/cmd/roachtest/test_filter.go @@ -12,41 +12,25 @@ package main import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" - "github.com/spf13/cobra" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestflags" ) -var ( - suite string - owner string - onlyBenchmarks bool - forceCloudCompat bool -) - -func addSuiteAndOwnerFlags(cmd *cobra.Command) { - cmd.Flags().StringVar( - &suite, "suite", "", - "Restrict tests to those in the given suite (e.g. nightly)", - ) - cmd.Flags().StringVar( - &owner, "owner", "", - "Restrict tests to those with the given owner (e.g. kv)", - ) -} - // makeTestFilter creates a registry.TestFilter based on the current flags and // the given regexps. func makeTestFilter(regexps []string) (*registry.TestFilter, error) { var options []registry.TestFilterOption - if !forceCloudCompat && cloud != "all" && cloud != "" { - options = append(options, registry.WithCloud(cloud)) + if !roachtestflags.ForceCloudCompat { + if cloud := roachtestflags.Cloud; cloud != "all" && cloud != "" { + options = append(options, registry.WithCloud(cloud)) + } } - if suite != "" { - options = append(options, registry.WithSuite(suite)) + if roachtestflags.Suite != "" { + options = append(options, registry.WithSuite(roachtestflags.Suite)) } - if owner != "" { - options = append(options, registry.WithOwner(registry.Owner(owner))) + if roachtestflags.Owner != "" { + options = append(options, registry.WithOwner(registry.Owner(roachtestflags.Owner))) } - if onlyBenchmarks { + if roachtestflags.OnlyBenchmarks { options = append(options, registry.OnlyBenchmarks()) } return registry.NewTestFilter(regexps, options...) diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index fa904f97fd76..e7eca5eefe63 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/build" "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/cmd/roachtest/spec" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/roachprod/config" @@ -139,8 +140,8 @@ func newTestRunner(cr *clusterRegistry, stopper *stop.Stopper) *testRunner { stopper: stopper, cr: cr, } - r.config.skipClusterWipeOnAttach = !clusterWipe - r.config.disableIssue = disableIssue + r.config.skipClusterWipeOnAttach = !roachtestflags.ClusterWipe + r.config.disableIssue = roachtestflags.DisableIssue r.workersMu.workers = make(map[string]*workerStatus) return r } @@ -389,7 +390,7 @@ func (r *testRunner) Run( // N.B. currently this value is hardcoded per cloud provider. func numConcurrentClusterCreations() int { var res int - if cloud == "aws" { + if roachtestflags.Cloud == "aws" { // AWS has ridiculous API calls limits, so we're going to create one cluster // at a time. Internally, roachprod has throttling for the calls required to // create a single cluster. @@ -742,7 +743,7 @@ func (r *testRunner) runWorker( // when tests opted-in to metamorphic testing, encryption will // be enabled according to the probability passed to // --metamorphic-encryption-probability - c.encAtRest = prng.Float64() < encryptionProbability + c.encAtRest = prng.Float64() < roachtestflags.EncryptionProbability } // Set initial cluster settings for this test. @@ -955,7 +956,7 @@ func (r *testRunner) runTest( if s.Skip != "" { // When skipping a test, we should not report ##teamcity[testStarted...] or ##teamcity[testFinished...] // service messages else the test will be reported as having run twice. - if teamCity { + if roachtestflags.TeamCity { shout(ctx, l, stdout, "##teamcity[testIgnored name='%s' message='%s' duration='%d']\n", s.Name, teamCityEscape(s.Skip), t.duration().Milliseconds()) } @@ -964,7 +965,7 @@ func (r *testRunner) runTest( // Delaying the ##teamcity[testStarted...] service message until the test is finished allows us to branch // separately for skipped tests. The duration of the test is passed to ##teamcity[testFinished...] for // accurate reporting in the TC UI. - if teamCity { + if roachtestflags.TeamCity { shout(ctx, l, stdout, "##teamcity[testStarted name='%s' flowId='%s']", t.Name(), testRunID) } @@ -972,7 +973,7 @@ func (r *testRunner) runTest( if t.Failed() { output := fmt.Sprintf("%s\ntest artifacts and logs in: %s", t.failureMsg(), t.ArtifactsDir()) - if teamCity { + if roachtestflags.TeamCity { // If `##teamcity[testFailed ...]` is not present before `##teamCity[testFinished ...]`, // TeamCity regards the test as successful. shout(ctx, l, stdout, "##teamcity[testFailed name='%s' details='%s' flowId='%s']", @@ -988,13 +989,13 @@ func (r *testRunner) runTest( shout(ctx, l, stdout, "--- PASS: %s (%s)", testRunID, durationStr) } - if teamCity { + if roachtestflags.TeamCity { shout(ctx, l, stdout, "##teamcity[testFinished name='%s' flowId='%s' duration='%d']", t.Name(), testRunID, t.duration().Milliseconds()) } } - if teamCity { + if roachtestflags.TeamCity { // Zip the artifacts. This improves the TeamCity UX where we can navigate // through zip files just fine, but we can't download subtrees of the // artifacts storage. By zipping we get this capability as we can just diff --git a/pkg/cmd/roachtest/test_test.go b/pkg/cmd/roachtest/test_test.go index 9d0b712eb354..6216c2c5d298 100644 --- a/pkg/cmd/roachtest/test_test.go +++ b/pkg/cmd/roachtest/test_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestflags" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" @@ -156,10 +157,10 @@ func TestRunnerEncryptionAtRest(t *testing.T) { // Verify that if a test opts into EncryptionMetamorphic, it will // (eventually) get a cluster that has encryption at rest enabled. { - prevProb := encryptionProbability - encryptionProbability = 0.5 // --metamorphic-encrypt-probability=0.5 + prevProb := roachtestflags.EncryptionProbability + roachtestflags.EncryptionProbability = 0.5 // --metamorphic-encrypt-probability=0.5 defer func() { - encryptionProbability = prevProb + roachtestflags.EncryptionProbability = prevProb }() } r := mkReg(t)