From 7fe2820f2418a9d31d12f9bcfe324c8aa5eaaacf Mon Sep 17 00:00:00 2001 From: Miral Gadani Date: Thu, 1 Sep 2022 15:11:44 -0400 Subject: [PATCH 1/2] roachtest: Return vm createoptions to allow computed values to be reported. Some runtime values are probabilistically set or otherwise have defaults which are not evident at the time we report issues. This change returns the vm options associated with a created cluster, from which we can extract the final computed values for such fields including the type of filesystem and whether encryption is enabled. Release justification: test-only change Release note: none --- pkg/cmd/roachtest/cluster.go | 62 +++++++-------- pkg/cmd/roachtest/cluster/BUILD.bazel | 1 - .../roachtest/cluster/cluster_interface.go | 3 +- pkg/cmd/roachtest/test_runner.go | 75 ++++++++++++------- pkg/cmd/roachtest/test_test.go | 5 +- .../roachtest/tests/multitenant_fairness.go | 2 +- 6 files changed, 84 insertions(+), 64 deletions(-) diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index 51ab57da6ba9..a69c1df96f81 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -837,9 +837,9 @@ func (f *clusterFactory) clusterMock(cfg clusterConfig) *clusterImpl { // NOTE: setTest() needs to be called before a test can use this cluster. func (f *clusterFactory) newCluster( ctx context.Context, cfg clusterConfig, setStatus func(string), teeOpt logger.TeeOptType, -) (*clusterImpl, error) { +) (*clusterImpl, *vm.CreateOpts, error) { if ctx.Err() != nil { - return nil, errors.Wrap(ctx.Err(), "newCluster") + return nil, nil, errors.Wrap(ctx.Err(), "newCluster") } if overrideFlagset != nil && overrideFlagset.Changed("nodes") { @@ -850,9 +850,9 @@ func (f *clusterFactory) newCluster( // For tests, use a mock cluster. c := f.clusterMock(cfg) if err := f.r.registerCluster(c); err != nil { - return nil, err + return nil, nil, err } - return c, nil + return c, nil, nil } if cfg.localCluster { @@ -875,7 +875,7 @@ func (f *clusterFactory) newCluster( // We must release the allocation because cluster creation is not possible at this point. cfg.alloc.Release() - return nil, err + return nil, nil, err } if cfg.spec.Cloud != spec.Local { providerOptsContainer.SetProviderOpts(cfg.spec.Cloud, providerOpts) @@ -929,18 +929,18 @@ func (f *clusterFactory) newCluster( err = roachprod.Create(ctx, l, cfg.username, cfg.spec.NodeCount, createVMOpts, providerOptsContainer) if err == nil { if err := f.r.registerCluster(c); err != nil { - return nil, err + return nil, nil, err } c.status("idle") l.Close() - return c, nil + return c, &createVMOpts, nil } if errors.HasType(err, (*roachprod.ClusterAlreadyExistsError)(nil)) { // If the cluster couldn't be created because it existed already, bail. // In reality when this is hit is when running with the `local` flag // or a destroy from the previous iteration failed. - return nil, err + return nil, nil, err } l.PrintfCtx(ctx, "cluster creation failed, cleaning up in case it was partially created: %s", err) @@ -952,7 +952,7 @@ func (f *clusterFactory) newCluster( if i >= maxAttempts { // Here we have to release the alloc, as we are giving up. cfg.alloc.Release() - return nil, err + return nil, nil, err } // Try again to create the cluster. } @@ -1128,13 +1128,13 @@ func (c *clusterImpl) Node(i int) option.NodeListOption { // FetchLogs downloads the logs from the cluster using `roachprod get`. // The logs will be placed in the test's artifacts dir. -func (c *clusterImpl) FetchLogs(ctx context.Context, t test.Test) error { +func (c *clusterImpl) FetchLogs(ctx context.Context, l *logger.Logger) error { if c.spec.NodeCount == 0 { // No nodes can happen during unit tests and implies nothing to do. return nil } - t.L().Printf("fetching logs\n") + l.Printf("fetching logs\n") c.status("fetching logs") // Don't hang forever if we can't fetch the logs. @@ -1145,14 +1145,14 @@ func (c *clusterImpl) FetchLogs(ctx context.Context, t test.Test) error { } if err := c.Get(ctx, c.l, "logs" /* src */, path /* dest */); err != nil { - t.L().Printf("failed to fetch logs: %v", err) + l.Printf("failed to fetch logs: %v", err) if ctx.Err() != nil { return errors.Wrap(err, "cluster.FetchLogs") } } if err := c.RunE(ctx, c.All(), "mkdir -p logs/redacted && ./cockroach debug merge-logs --redact logs/*.log > logs/redacted/combined.log"); err != nil { - t.L().Printf("failed to redact logs: %v", err) + l.Printf("failed to redact logs: %v", err) if ctx.Err() != nil { return err } @@ -1204,17 +1204,17 @@ func (c *clusterImpl) CopyRoachprodState(ctx context.Context) error { // the first available node. They can be visualized via: // // `COCKROACH_DEBUG_TS_IMPORT_FILE=tsdump.gob ./cockroach start-single-node --insecure --store=$(mktemp -d)` -func (c *clusterImpl) FetchTimeseriesData(ctx context.Context, t test.Test) error { +func (c *clusterImpl) FetchTimeseriesData(ctx context.Context, l *logger.Logger) error { return contextutil.RunWithTimeout(ctx, "fetch tsdata", 5*time.Minute, func(ctx context.Context) error { node := 1 for ; node <= c.spec.NodeCount; node++ { - db, err := c.ConnE(ctx, t.L(), node) + db, err := c.ConnE(ctx, l, node) if err == nil { err = db.Ping() db.Close() } if err != nil { - t.L().Printf("node %d not responding to SQL, trying next one", node) + l.Printf("node %d not responding to SQL, trying next one", node) continue } break @@ -1241,7 +1241,7 @@ func (c *clusterImpl) FetchTimeseriesData(ctx context.Context, t test.Test) erro ); err != nil { return errors.Wrap(err, "cluster.FetchTimeseriesData") } - db, err := c.ConnE(ctx, t.L(), node) + db, err := c.ConnE(ctx, l, node) if err != nil { return err } @@ -1277,13 +1277,13 @@ COCKROACH_DEBUG_TS_IMPORT_FILE=tsdump.gob cockroach start-single-node --insecure // FetchDebugZip downloads the debug zip from the cluster using `roachprod ssh`. // The logs will be placed in the test's artifacts dir. -func (c *clusterImpl) FetchDebugZip(ctx context.Context, t test.Test) error { +func (c *clusterImpl) FetchDebugZip(ctx context.Context, l *logger.Logger) error { if c.spec.NodeCount == 0 { // No nodes can happen during unit tests and implies nothing to do. return nil } - t.L().Printf("fetching debug zip\n") + l.Printf("fetching debug zip\n") c.status("fetching debug zip") // Don't hang forever if we can't fetch the debug zip. @@ -1304,7 +1304,7 @@ func (c *clusterImpl) FetchDebugZip(ctx context.Context, t test.Test) error { si := strconv.Itoa(i) cmd := []string{"./cockroach", "debug", "zip", "--exclude-files='*.log,*.txt,*.pprof'", "--url", "{pgurl:" + si + "}", zipName} if err := c.RunE(ctx, c.All(), cmd...); err != nil { - t.L().Printf("./cockroach debug zip failed: %v", err) + l.Printf("./cockroach debug zip failed: %v", err) if i < c.spec.NodeCount { continue } @@ -1424,14 +1424,14 @@ func (c *clusterImpl) FailOnReplicaDivergence(ctx context.Context, t *testImpl) // FetchDmesg grabs the dmesg logs if possible. This requires being able to run // `sudo dmesg` on the remote nodes. -func (c *clusterImpl) FetchDmesg(ctx context.Context, t test.Test) error { +func (c *clusterImpl) FetchDmesg(ctx context.Context, l *logger.Logger) error { if c.spec.NodeCount == 0 || c.IsLocal() { // No nodes can happen during unit tests and implies nothing to do. // Also, don't grab dmesg on local runs. return nil } - t.L().Printf("fetching dmesg\n") + l.Printf("fetching dmesg\n") c.status("fetching dmesg") // Don't hang forever. @@ -1455,7 +1455,7 @@ func (c *clusterImpl) FetchDmesg(ctx context.Context, t test.Test) error { if result.Err != nil { // Store `Run` errors to return later (after copying files from successful nodes). combinedDmesgError = errors.CombineErrors(combinedDmesgError, result.Err) - t.L().Printf("running dmesg failed on node %d: %v", result.Node, result.Err) + l.Printf("running dmesg failed on node %d: %v", result.Node, result.Err) } else { // Only run `Get` on successful nodes to avoid pseudo-failure on `Get` caused by an earlier failure on `Run`. successfulNodes = append(successfulNodes, int(result.Node)) @@ -1464,7 +1464,7 @@ func (c *clusterImpl) FetchDmesg(ctx context.Context, t test.Test) error { // Get dmesg files from successful nodes only. if err := c.Get(ctx, c.l, name /* src */, path /* dest */, successfulNodes); err != nil { - t.L().Printf("getting dmesg files failed: %v", err) + l.Printf("getting dmesg files failed: %v", err) return errors.Wrap(err, "cluster.FetchDmesg") } @@ -1475,14 +1475,14 @@ func (c *clusterImpl) FetchDmesg(ctx context.Context, t test.Test) error { // FetchJournalctl grabs the journalctl logs if possible. This requires being // able to run `sudo journalctl` on the remote nodes. -func (c *clusterImpl) FetchJournalctl(ctx context.Context, t test.Test) error { +func (c *clusterImpl) FetchJournalctl(ctx context.Context, l *logger.Logger) error { if c.spec.NodeCount == 0 || c.IsLocal() { // No nodes can happen during unit tests and implies nothing to do. // Also, don't grab journalctl on local runs. return nil } - t.L().Printf("fetching journalctl\n") + l.Printf("fetching journalctl\n") c.status("fetching journalctl") // Don't hang forever. @@ -1506,7 +1506,7 @@ func (c *clusterImpl) FetchJournalctl(ctx context.Context, t test.Test) error { if result.Err != nil { // Store `Run` errors to return later (after copying files from successful nodes). combinedJournalctlError = errors.CombineErrors(combinedJournalctlError, result.Err) - t.L().Printf("running journalctl failed on node %d: %v", result.Node, result.Err) + l.Printf("running journalctl failed on node %d: %v", result.Node, result.Err) } else { // Only run `Get` on successful nodes to avoid pseudo-failure on `Get` caused by an earlier failure on `Run`. successfulNodes = append(successfulNodes, int(result.Node)) @@ -1515,7 +1515,7 @@ func (c *clusterImpl) FetchJournalctl(ctx context.Context, t test.Test) error { // Get files from successful nodes only. if err := c.Get(ctx, c.l, name /* src */, path /* dest */, successfulNodes); err != nil { - t.L().Printf("getting files failed: %v", err) + l.Printf("getting files failed: %v", err) return errors.Wrap(err, "cluster.FetchJournalctl") } @@ -1525,7 +1525,7 @@ func (c *clusterImpl) FetchJournalctl(ctx context.Context, t test.Test) error { } // FetchCores fetches any core files on the cluster. -func (c *clusterImpl) FetchCores(ctx context.Context, t test.Test) error { +func (c *clusterImpl) FetchCores(ctx context.Context, l *logger.Logger) error { if c.spec.NodeCount == 0 || c.IsLocal() { // No nodes can happen during unit tests and implies nothing to do. // Also, don't grab dmesg on local runs. @@ -1537,11 +1537,11 @@ func (c *clusterImpl) FetchCores(ctx context.Context, t test.Test) error { // from having the cores, but we should push them straight into a temp // bucket on S3 instead. OTOH, the ROI of this may be low; I don't know // of a recent example where we've wanted the Core dumps. - t.L().Printf("skipped fetching cores\n") + l.Printf("skipped fetching cores\n") return nil } - t.L().Printf("fetching cores\n") + l.Printf("fetching cores\n") c.status("fetching cores") // Don't hang forever. The core files can be large, so we give a generous diff --git a/pkg/cmd/roachtest/cluster/BUILD.bazel b/pkg/cmd/roachtest/cluster/BUILD.bazel index 736dab20a95c..afd151fae747 100644 --- a/pkg/cmd/roachtest/cluster/BUILD.bazel +++ b/pkg/cmd/roachtest/cluster/BUILD.bazel @@ -13,7 +13,6 @@ go_library( deps = [ "//pkg/cmd/roachtest/option", "//pkg/cmd/roachtest/spec", - "//pkg/cmd/roachtest/test", "//pkg/roachprod/install", "//pkg/roachprod/logger", "//pkg/roachprod/prometheus", diff --git a/pkg/cmd/roachtest/cluster/cluster_interface.go b/pkg/cmd/roachtest/cluster/cluster_interface.go index bf7b5d4494d3..bcfd087bc2e9 100644 --- a/pkg/cmd/roachtest/cluster/cluster_interface.go +++ b/pkg/cmd/roachtest/cluster/cluster_interface.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" - "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/roachprod/install" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" "github.com/cockroachdb/cockroach/pkg/roachprod/prometheus" @@ -131,7 +130,7 @@ type Cluster interface { ctx context.Context, l *logger.Logger, src, dest, branch string, node option.NodeListOption, ) error - FetchTimeseriesData(ctx context.Context, t test.Test) error + FetchTimeseriesData(ctx context.Context, l *logger.Logger) error RefetchCertsFromNode(ctx context.Context, node int) error StartGrafana(ctx context.Context, l *logger.Logger, promCfg *prometheus.Config) error diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index bbc6a6dc1420..8369d9da1ce8 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -37,6 +37,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/internal/team" "github.com/cockroachdb/cockroach/pkg/roachprod/config" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" + "github.com/cockroachdb/cockroach/pkg/roachprod/vm" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/quotapool" @@ -347,7 +348,7 @@ func defaultClusterAllocator( alloc *quotapool.IntAlloc, artifactsDir string, wStatus *workerStatus, - ) (*clusterImpl, error) { + ) (*clusterImpl, *vm.CreateOpts, error) { wStatus.SetStatus("creating cluster") defer wStatus.SetStatus("") @@ -357,7 +358,7 @@ func defaultClusterAllocator( logPath := filepath.Join(artifactsDir, runnerLogsDir, "cluster-create", existingClusterName+".log") clusterL, err := logger.RootLogger(logPath, lopt.tee) if err != nil { - return nil, err + return nil, nil, err } defer clusterL.Close() opt := attachOpt{ @@ -368,10 +369,10 @@ func defaultClusterAllocator( 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 { - return c, nil + return c, nil, nil } if !errors.Is(err, errClusterNotFound) { - return nil, err + return nil, nil, err } // Fall through to create new cluster with name override. lopt.l.PrintfCtx( @@ -401,7 +402,7 @@ type clusterAllocatorFn func( alloc *quotapool.IntAlloc, artifactsDir string, wStatus *workerStatus, -) (*clusterImpl, error) +) (*clusterImpl, *vm.CreateOpts, error) // runWorker runs tests in a loop until work is exhausted. // @@ -561,13 +562,14 @@ func (r *testRunner) runWorker( } var clusterCreateErr error + var vmCreateOpts *vm.CreateOpts if !testToRun.canReuseCluster { // Create a new cluster if can't reuse or reuse attempt failed. // N.B. non-reusable cluster would have been destroyed above. wStatus.SetTest(nil /* test */, testToRun) wStatus.SetStatus("creating cluster") - c, clusterCreateErr = allocateCluster(ctx, testToRun.spec, testToRun.alloc, artifactsRootDir, wStatus) + c, vmCreateOpts, clusterCreateErr = allocateCluster(ctx, testToRun.spec, testToRun.alloc, artifactsRootDir, wStatus) if clusterCreateErr != nil { atomic.AddInt32(&r.numClusterErrs, 1) shout(ctx, l, stdout, "Unable to create (or reuse) cluster for test %s due to: %s.", @@ -623,7 +625,7 @@ func (r *testRunner) runWorker( // N.B. issue title is of the form "roachtest: ${t.spec.Name} failed" (see UnitTestFormatter). t.spec.Name = "cluster_creation" t.spec.Owner = registry.OwnerDevInf - r.maybePostGithubIssue(ctx, l, t, stdout, issueOutput) + r.maybePostGithubIssue(ctx, l, t, c, vmCreateOpts, stdout, issueOutput) // Restore test name and owner. t.spec.Name = oldName t.spec.Owner = oldOwner @@ -652,7 +654,7 @@ func (r *testRunner) runWorker( wStatus.SetTest(t, testToRun) wStatus.SetStatus("running test") - err = r.runTest(ctx, t, testToRun.runNum, testToRun.runCount, c, stdout, testL) + err = r.runTest(ctx, t, testToRun.runNum, testToRun.runCount, c, vmCreateOpts, stdout, testL) } } @@ -764,6 +766,7 @@ func (r *testRunner) runTest( runNum int, runCount int, c *clusterImpl, + vmCreateOpts *vm.CreateOpts, stdout io.Writer, l *logger.Logger, ) error { @@ -821,7 +824,7 @@ func (r *testRunner) runTest( shout(ctx, l, stdout, "--- FAIL: %s (%s)\n%s", runID, durationStr, output) - r.maybePostGithubIssue(ctx, l, t, stdout, output) + r.maybePostGithubIssue(ctx, l, t, c, vmCreateOpts, stdout, output) } else { shout(ctx, l, stdout, "--- PASS: %s (%s)", runID, durationStr) // If `##teamcity[testFailed ...]` is not present before `##teamCity[testFinished ...]`, @@ -1023,7 +1026,7 @@ func (r *testRunner) teardownTest( c.FailOnReplicaDivergence(ctx, t) if timedOut || t.Failed() { - r.collectClusterArtifacts(ctx, c, t) + r.collectClusterArtifacts(ctx, c, t.L()) } }) @@ -1063,7 +1066,13 @@ func (r *testRunner) shouldPostGithubIssue(t test.Test) bool { } func (r *testRunner) maybePostGithubIssue( - ctx context.Context, l *logger.Logger, t test.Test, stdout io.Writer, output string, + ctx context.Context, + l *logger.Logger, + t test.Test, + c *clusterImpl, + vmCreateOpts *vm.CreateOpts, + stdout io.Writer, + output string, ) { if !r.shouldPostGithubIssue(t) { return @@ -1106,6 +1115,15 @@ func (r *testRunner) maybePostGithubIssue( roachtestParam("ssd"): fmt.Sprintf("%d", spec.Cluster.SSDs), } + // these params can be probabilistically set if requested + if vmCreateOpts != nil { + clusterParams[roachtestParam("fs")] = vmCreateOpts.SSDOpts.FileSystem + } + + if c != nil { + clusterParams[roachtestParam("encrypted")] = fmt.Sprintf("%v", c.encAtRest) + } + req := issues.PostRequest{ MentionOnCreate: mention, ProjectColumnID: projColID, @@ -1136,7 +1154,10 @@ func (r *testRunner) maybePostGithubIssue( } // TODO(tbg): nothing in this method should have the `t`; they should have a `Logger` only. -func (r *testRunner) collectClusterArtifacts(ctx context.Context, c *clusterImpl, t test.Test) { +// Maybe done +func (r *testRunner) collectClusterArtifacts( + ctx context.Context, c *clusterImpl, l *logger.Logger, +) { // NB: fetch the logs even when we have a debug zip because // debug zip can't ever get the logs for down nodes. // We only save artifacts for failed tests in CI, so this @@ -1145,32 +1166,32 @@ func (r *testRunner) collectClusterArtifacts(ctx context.Context, c *clusterImpl // below has problems. For example, `debug zip` is known to // hang sometimes at the time of writing, see: // https://github.com/cockroachdb/cockroach/issues/39620 - t.L().PrintfCtx(ctx, "collecting cluster logs") + l.PrintfCtx(ctx, "collecting cluster logs") // Do this before collecting logs to make sure the file gets // downloaded below. if err := saveDiskUsageToLogsDir(ctx, c); err != nil { - t.L().Printf("failed to fetch disk uage summary: %s", err) + l.Printf("failed to fetch disk uage summary: %s", err) } - if err := c.FetchLogs(ctx, t); err != nil { - t.L().Printf("failed to download logs: %s", err) + if err := c.FetchLogs(ctx, l); err != nil { + l.Printf("failed to download logs: %s", err) } - if err := c.FetchDmesg(ctx, t); err != nil { - t.L().Printf("failed to fetch dmesg: %s", err) + if err := c.FetchDmesg(ctx, l); err != nil { + l.Printf("failed to fetch dmesg: %s", err) } - if err := c.FetchJournalctl(ctx, t); err != nil { - t.L().Printf("failed to fetch journalctl: %s", err) + if err := c.FetchJournalctl(ctx, l); err != nil { + l.Printf("failed to fetch journalctl: %s", err) } - if err := c.FetchCores(ctx, t); err != nil { - t.L().Printf("failed to fetch cores: %s", err) + if err := c.FetchCores(ctx, l); err != nil { + l.Printf("failed to fetch cores: %s", err) } if err := c.CopyRoachprodState(ctx); err != nil { - t.L().Printf("failed to copy roachprod state: %s", err) + l.Printf("failed to copy roachprod state: %s", err) } - if err := c.FetchTimeseriesData(ctx, t); err != nil { - t.L().Printf("failed to fetch timeseries data: %s", err) + if err := c.FetchTimeseriesData(ctx, l); err != nil { + l.Printf("failed to fetch timeseries data: %s", err) } - if err := c.FetchDebugZip(ctx, t); err != nil { - t.L().Printf("failed to collect zip: %s", err) + if err := c.FetchDebugZip(ctx, l); err != nil { + l.Printf("failed to collect zip: %s", err) } } diff --git a/pkg/cmd/roachtest/test_test.go b/pkg/cmd/roachtest/test_test.go index 6fbbe4da3f32..0ed208df8c98 100644 --- a/pkg/cmd/roachtest/test_test.go +++ b/pkg/cmd/roachtest/test_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" + "github.com/cockroachdb/cockroach/pkg/roachprod/vm" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/quotapool" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -98,8 +99,8 @@ func alwaysFailingClusterAllocator( alloc *quotapool.IntAlloc, artifactsDir string, wStatus *workerStatus, -) (*clusterImpl, error) { - return nil, errors.New("cluster creation failed") +) (*clusterImpl, *vm.CreateOpts, error) { + return nil, nil, errors.New("cluster creation failed") } func TestRunnerRun(t *testing.T) { diff --git a/pkg/cmd/roachtest/tests/multitenant_fairness.go b/pkg/cmd/roachtest/tests/multitenant_fairness.go index a5fe0fca414b..f447b93399f8 100644 --- a/pkg/cmd/roachtest/tests/multitenant_fairness.go +++ b/pkg/cmd/roachtest/tests/multitenant_fairness.go @@ -310,7 +310,7 @@ func runMultiTenantFairness( c.Run(ctx, c.Node(1), cmd) // get cluster timeseries data into artifacts - err := c.FetchTimeseriesData(ctx, t) + err := c.FetchTimeseriesData(ctx, t.L()) require.NoError(t, err) } From 1e50f25f5087dfc54e8cc28c18870e91b5ebb386 Mon Sep 17 00:00:00 2001 From: Miral Gadani Date: Tue, 6 Sep 2022 17:49:31 -0400 Subject: [PATCH 2/2] roachtest: Refactor github posting to separate source file, add unit test. No previous coverage exists for posting issues to github. This change moves the relevant code into a separate source file with associated test. External functions dealing with actual issue posting to github, and loading the TEAMS.yaml are now injected to facilitate easier testing. Release justification: test-only change Release note: none --- pkg/cmd/roachtest/BUILD.bazel | 3 + pkg/cmd/roachtest/github.go | 143 ++++++++++++++++++++++ pkg/cmd/roachtest/github_test.go | 201 +++++++++++++++++++++++++++++++ pkg/cmd/roachtest/test_runner.go | 124 +++---------------- 4 files changed, 363 insertions(+), 108 deletions(-) create mode 100644 pkg/cmd/roachtest/github.go create mode 100644 pkg/cmd/roachtest/github_test.go diff --git a/pkg/cmd/roachtest/BUILD.bazel b/pkg/cmd/roachtest/BUILD.bazel index e240bfa6e703..2e272fc812c5 100644 --- a/pkg/cmd/roachtest/BUILD.bazel +++ b/pkg/cmd/roachtest/BUILD.bazel @@ -5,6 +5,7 @@ go_library( name = "roachtest_lib", srcs = [ "cluster.go", + "github.go", "main.go", "monitor.go", "slack.go", @@ -64,6 +65,7 @@ go_test( size = "small", srcs = [ "cluster_test.go", + "github_test.go", "main_test.go", "test_registry_test.go", "test_test.go", @@ -78,6 +80,7 @@ go_test( "//pkg/cmd/roachtest/test", "//pkg/internal/team", "//pkg/roachprod/logger", + "//pkg/roachprod/vm", "//pkg/testutils", "//pkg/util/quotapool", "//pkg/util/stop", diff --git a/pkg/cmd/roachtest/github.go b/pkg/cmd/roachtest/github.go new file mode 100644 index 000000000000..d412f79493c5 --- /dev/null +++ b/pkg/cmd/roachtest/github.go @@ -0,0 +1,143 @@ +// Copyright 2022 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 main + +import ( + "context" + "fmt" + "os" + + "github.com/cockroachdb/cockroach/pkg/cmd/internal/issues" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" + "github.com/cockroachdb/cockroach/pkg/internal/team" + "github.com/cockroachdb/cockroach/pkg/roachprod/logger" + "github.com/cockroachdb/cockroach/pkg/roachprod/vm" +) + +type githubIssues struct { + disable bool + l *logger.Logger + cluster *clusterImpl + vmCreateOpts *vm.CreateOpts + issuePoster func(ctx context.Context, formatter issues.IssueFormatter, req issues.PostRequest) error + teamLoader func() (team.Map, error) +} + +func newGithubIssues( + disable bool, c *clusterImpl, vmCreateOpts *vm.CreateOpts, l *logger.Logger, +) *githubIssues { + + return &githubIssues{ + disable: disable, + vmCreateOpts: vmCreateOpts, + cluster: c, + l: l, + issuePoster: issues.Post, + teamLoader: team.DefaultLoadTeams, + } +} + +func roachtestPrefix(p string) string { + return "ROACHTEST_" + p +} + +func (g *githubIssues) shouldPost(t test.Test) bool { + opts := issues.DefaultOptionsFromEnv() + return !g.disable && opts.CanPost() && + opts.IsReleaseBranch() && + t.Spec().(*registry.TestSpec).Run != nil && + // NB: check NodeCount > 0 to avoid posting issues from this pkg's unit tests. + t.Spec().(*registry.TestSpec).Cluster.NodeCount > 0 +} + +func (g *githubIssues) createPostRequest(t test.Test, message string) issues.PostRequest { + var mention []string + var projColID int + + teams, err := g.teamLoader() + if err != nil { + t.Fatalf("could not load teams: %v", err) + } + + if sl, ok := teams.GetAliasesForPurpose(ownerToAlias(t.Spec().(*registry.TestSpec).Owner), team.PurposeRoachtest); ok { + for _, alias := range sl { + mention = append(mention, "@"+string(alias)) + } + projColID = teams[sl[0]].TriageColumnID + } + + branch := os.Getenv("TC_BUILD_BRANCH") + if branch == "" { + branch = "" + } + + artifacts := fmt.Sprintf("/%s", t.Name()) + + // Issues posted from roachtest are identifiable as such and + // they are also release blockers (this label may be removed + // by a human upon closer investigation). + spec := t.Spec().(*registry.TestSpec) + labels := []string{"O-roachtest"} + if !spec.NonReleaseBlocker { + labels = append(labels, "release-blocker") + } + + clusterParams := map[string]string{ + roachtestPrefix("cloud"): spec.Cluster.Cloud, + roachtestPrefix("cpu"): fmt.Sprintf("%d", spec.Cluster.CPUs), + roachtestPrefix("ssd"): fmt.Sprintf("%d", spec.Cluster.SSDs), + } + + // These params can be probabilistically set, so we pass them here to + // show what their actual values are in the posted issue. + if g.vmCreateOpts != nil { + clusterParams[roachtestPrefix("fs")] = g.vmCreateOpts.SSDOpts.FileSystem + clusterParams[roachtestPrefix("localSSD")] = fmt.Sprintf("%v", g.vmCreateOpts.SSDOpts.UseLocalSSD) + } + + if g.cluster != nil { + clusterParams[roachtestPrefix("encrypted")] = fmt.Sprintf("%v", g.cluster.encAtRest) + } + + return issues.PostRequest{ + MentionOnCreate: mention, + ProjectColumnID: projColID, + PackageName: "roachtest", + TestName: t.Name(), + Message: message, + Artifacts: artifacts, + ExtraLabels: labels, + ExtraParams: clusterParams, + HelpCommand: func(renderer *issues.Renderer) { + issues.HelpCommandAsLink( + "roachtest README", + "https://github.com/cockroachdb/cockroach/blob/master/pkg/cmd/roachtest/README.md", + )(renderer) + issues.HelpCommandAsLink( + "How To Investigate (internal)", + "https://cockroachlabs.atlassian.net/l/c/SSSBr8c7", + )(renderer) + }, + } +} + +func (g *githubIssues) MaybePost(t test.Test, message string) error { + if !g.shouldPost(t) { + return nil + } + + return g.issuePoster( + context.Background(), + issues.UnitTestFormatter, + g.createPostRequest(t, message), + ) +} diff --git a/pkg/cmd/roachtest/github_test.go b/pkg/cmd/roachtest/github_test.go new file mode 100644 index 000000000000..d0188fc87ff8 --- /dev/null +++ b/pkg/cmd/roachtest/github_test.go @@ -0,0 +1,201 @@ +// Copyright 2022 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 main + +import ( + "context" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" + "github.com/cockroachdb/cockroach/pkg/internal/team" + "github.com/cockroachdb/cockroach/pkg/roachprod/vm" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +var ( + teamsYaml = `cockroachdb/unowned: + aliases: + cockroachdb/rfc-prs: other + triage_column_id: 0` + + validTeamsFn = func() (team.Map, error) { return loadYamlTeams(teamsYaml) } + invalidTeamsFn = func() (team.Map, error) { return loadYamlTeams("invalid yaml") } +) + +func loadYamlTeams(yaml string) (team.Map, error) { + return team.LoadTeams(strings.NewReader(yaml)) +} + +func prefixAll(params map[string]string) map[string]string { + updated := make(map[string]string) + + for k, v := range params { + updated[roachtestPrefix(k)] = v + } + + return updated +} + +func TestShouldPost(t *testing.T) { + testCases := []struct { + disableIssues bool + nodeCount int + envGithubAPIToken string + envTcBuildBranch string + expected bool + }{ + /* Cases 1 - 4 verify that issues are not posted if any of on the relevant criteria checks fail */ + // disable + {true, 1, "token", "master", false}, + // nodeCount + {false, 0, "token", "master", false}, + // apiToken + {false, 1, "", "master", false}, + // branch + {false, 1, "token", "", false}, + {false, 1, "token", "master", true}, + } + + reg, _ := makeTestRegistry(spec.GCE, "", "", false) + for _, c := range testCases { + t.Setenv("GITHUB_API_TOKEN", c.envGithubAPIToken) + t.Setenv("TC_BUILD_BRANCH", c.envTcBuildBranch) + + clusterSpec := reg.MakeClusterSpec(c.nodeCount) + testSpec := ®istry.TestSpec{ + Name: "githubPost", + Owner: OwnerUnitTest, + Cluster: clusterSpec, + // `shouldPost` explicitly checks to ensure that the run function is defined + Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {}, + } + + ti := &testImpl{ + spec: testSpec, + l: nilLogger(), + } + + github := &githubIssues{ + disable: c.disableIssues, + } + + require.Equal(t, c.expected, github.shouldPost(ti)) + } +} + +func TestCreatePostRequest(t *testing.T) { + testCases := []struct { + nonReleaseBlocker bool + clusterCreationFailed bool + loadTeamsFailed bool + localSSD bool + expectedPost bool + expectedParams map[string]string + }{ + {true, false, false, false, true, + prefixAll(map[string]string{ + "cloud": "gce", + "encrypted": "false", + "fs": "ext4", + "ssd": "0", + "cpu": "4", + "localSSD": "false", + }), + }, + {true, false, false, true, true, + prefixAll(map[string]string{ + "cloud": "gce", + "encrypted": "false", + "fs": "ext4", + "ssd": "0", + "cpu": "4", + "localSSD": "true", + }), + }, + // Assert that release-blocker label exists when !nonReleaseBlocker + // Also ensure that in the event of a failed cluster creation, + // nil `vmOptions` and `clusterImpl` are not dereferenced + {false, true, false, false, true, + prefixAll(map[string]string{ + "cloud": "gce", + "ssd": "0", + "cpu": "4", + }), + }, + //Simulate failure loading TEAMS.yaml + {true, false, true, false, false, nil}, + } + + reg, _ := makeTestRegistry(spec.GCE, "", "", false) + + for _, c := range testCases { + clusterSpec := reg.MakeClusterSpec(1) + + testSpec := ®istry.TestSpec{ + Name: "githubPost", + Owner: OwnerUnitTest, + Cluster: clusterSpec, + NonReleaseBlocker: c.nonReleaseBlocker, + } + + ti := &testImpl{ + spec: testSpec, + l: nilLogger(), + } + + testClusterImpl := &clusterImpl{spec: clusterSpec} + vo := vm.DefaultCreateOpts() + vmOpts := &vo + + if c.clusterCreationFailed { + testClusterImpl = nil + vmOpts = nil + } else if !c.localSSD { + // The default is true set in `vm.DefaultCreateOpts` + vmOpts.SSDOpts.UseLocalSSD = false + } + + teamLoadFn := validTeamsFn + + if c.loadTeamsFailed { + teamLoadFn = invalidTeamsFn + } + + github := &githubIssues{ + vmCreateOpts: vmOpts, + cluster: testClusterImpl, + l: nilLogger(), + teamLoader: teamLoadFn, + } + + if c.loadTeamsFailed { + // Assert that if TEAMS.yaml cannot be loaded then function panics. + assert.Panics(t, func() { github.createPostRequest(ti, "message") }) + } else { + req := github.createPostRequest(ti, "message") + + if c.expectedParams != nil { + require.Equal(t, c.expectedParams, req.ExtraParams) + } + + require.True(t, contains(req.ExtraLabels, nil, "O-roachtest")) + + if !c.nonReleaseBlocker { + require.True(t, contains(req.ExtraLabels, nil, "release-blocker")) + } + } + } +} diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index 8369d9da1ce8..1b62b7527816 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -29,12 +29,10 @@ import ( "sync/atomic" "time" - "github.com/cockroachdb/cockroach/pkg/cmd/internal/issues" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" - "github.com/cockroachdb/cockroach/pkg/internal/team" "github.com/cockroachdb/cockroach/pkg/roachprod/config" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" "github.com/cockroachdb/cockroach/pkg/roachprod/vm" @@ -51,8 +49,10 @@ import ( "github.com/petermattis/goid" ) -var errTestsFailed = fmt.Errorf("some tests failed") -var errClusterProvisioningFailed = fmt.Errorf("some clusters could not be created") +var ( + errTestsFailed = fmt.Errorf("some tests failed") + errClusterProvisioningFailed = fmt.Errorf("some clusters could not be created") +) // testRunner runs tests. type testRunner struct { @@ -611,6 +611,8 @@ func (r *testRunner) runWorker( // Now run the test. l.PrintfCtx(ctx, "starting test: %s:%d", testToRun.spec.Name, testToRun.runNum) + github := newGithubIssues(r.config.disableIssue, c, vmCreateOpts, l) + if clusterCreateErr != nil { // N.B. cluster creation must have failed... // We don't want to prematurely abort the test suite since it's likely a transient issue. @@ -625,7 +627,11 @@ func (r *testRunner) runWorker( // N.B. issue title is of the form "roachtest: ${t.spec.Name} failed" (see UnitTestFormatter). t.spec.Name = "cluster_creation" t.spec.Owner = registry.OwnerDevInf - r.maybePostGithubIssue(ctx, l, t, c, vmCreateOpts, stdout, issueOutput) + + if err := github.MaybePost(t, issueOutput); err != nil { + shout(ctx, l, stdout, "failed to post issue: %s", err) + } + // Restore test name and owner. t.spec.Name = oldName t.spec.Owner = oldOwner @@ -654,7 +660,7 @@ func (r *testRunner) runWorker( wStatus.SetTest(t, testToRun) wStatus.SetStatus("running test") - err = r.runTest(ctx, t, testToRun.runNum, testToRun.runCount, c, vmCreateOpts, stdout, testL) + err = r.runTest(ctx, t, testToRun.runNum, testToRun.runCount, c, stdout, testL, github) } } @@ -766,9 +772,9 @@ func (r *testRunner) runTest( runNum int, runCount int, c *clusterImpl, - vmCreateOpts *vm.CreateOpts, stdout io.Writer, l *logger.Logger, + github *githubIssues, ) error { if t.Spec().(*registry.TestSpec).Skip != "" { return fmt.Errorf("can't run skipped test: %s: %s", t.Name(), t.Spec().(*registry.TestSpec).Skip) @@ -824,7 +830,9 @@ func (r *testRunner) runTest( shout(ctx, l, stdout, "--- FAIL: %s (%s)\n%s", runID, durationStr, output) - r.maybePostGithubIssue(ctx, l, t, c, vmCreateOpts, stdout, output) + if err := github.MaybePost(t, output); err != nil { + shout(ctx, l, stdout, "failed to post issue: %s", err) + } } else { shout(ctx, l, stdout, "--- PASS: %s (%s)", runID, durationStr) // If `##teamcity[testFailed ...]` is not present before `##teamCity[testFinished ...]`, @@ -1055,106 +1063,6 @@ func (r *testRunner) teardownTest( return nil } -func (r *testRunner) shouldPostGithubIssue(t test.Test) bool { - opts := issues.DefaultOptionsFromEnv() - return !r.config.disableIssue && - opts.CanPost() && - opts.IsReleaseBranch() && - t.Spec().(*registry.TestSpec).Run != nil && - // NB: check NodeCount > 0 to avoid posting issues from this pkg's unit tests. - t.Spec().(*registry.TestSpec).Cluster.NodeCount > 0 -} - -func (r *testRunner) maybePostGithubIssue( - ctx context.Context, - l *logger.Logger, - t test.Test, - c *clusterImpl, - vmCreateOpts *vm.CreateOpts, - stdout io.Writer, - output string, -) { - if !r.shouldPostGithubIssue(t) { - return - } - - teams, err := team.DefaultLoadTeams() - if err != nil { - t.Fatalf("could not load teams: %v", err) - } - - var mention []string - var projColID int - if sl, ok := teams.GetAliasesForPurpose(ownerToAlias(t.Spec().(*registry.TestSpec).Owner), team.PurposeRoachtest); ok { - for _, alias := range sl { - mention = append(mention, "@"+string(alias)) - } - projColID = teams[sl[0]].TriageColumnID - } - - branch := os.Getenv("TC_BUILD_BRANCH") - if branch == "" { - branch = "" - } - - artifacts := fmt.Sprintf("/%s", t.Name()) - - // Issues posted from roachtest are identifiable as such and - // they are also release blockers (this label may be removed - // by a human upon closer investigation). - spec := t.Spec().(*registry.TestSpec) - labels := []string{"O-roachtest"} - if !spec.NonReleaseBlocker { - labels = append(labels, "release-blocker") - } - - roachtestParam := func(s string) string { return "ROACHTEST_" + s } - clusterParams := map[string]string{ - roachtestParam("cloud"): spec.Cluster.Cloud, - roachtestParam("cpu"): fmt.Sprintf("%d", spec.Cluster.CPUs), - roachtestParam("ssd"): fmt.Sprintf("%d", spec.Cluster.SSDs), - } - - // these params can be probabilistically set if requested - if vmCreateOpts != nil { - clusterParams[roachtestParam("fs")] = vmCreateOpts.SSDOpts.FileSystem - } - - if c != nil { - clusterParams[roachtestParam("encrypted")] = fmt.Sprintf("%v", c.encAtRest) - } - - req := issues.PostRequest{ - MentionOnCreate: mention, - ProjectColumnID: projColID, - PackageName: "roachtest", - TestName: t.Name(), - Message: output, - Artifacts: artifacts, - ExtraLabels: labels, - ExtraParams: clusterParams, - HelpCommand: func(renderer *issues.Renderer) { - issues.HelpCommandAsLink( - "roachtest README", - "https://github.com/cockroachdb/cockroach/blob/master/pkg/cmd/roachtest/README.md", - )(renderer) - issues.HelpCommandAsLink( - "How To Investigate (internal)", - "https://cockroachlabs.atlassian.net/l/c/SSSBr8c7", - )(renderer) - }, - } - if err := issues.Post( - context.Background(), - issues.UnitTestFormatter, - req, - ); err != nil { - shout(ctx, l, stdout, "failed to post issue: %s", err) - } -} - -// TODO(tbg): nothing in this method should have the `t`; they should have a `Logger` only. -// Maybe done func (r *testRunner) collectClusterArtifacts( ctx context.Context, c *clusterImpl, l *logger.Logger, ) {