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/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/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 bbc6a6dc1420..1b62b7527816 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -29,14 +29,13 @@ 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" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/quotapool" @@ -50,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 { @@ -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.", @@ -609,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. @@ -623,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, 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 @@ -652,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, stdout, testL) + err = r.runTest(ctx, t, testToRun.runNum, testToRun.runCount, c, stdout, testL, github) } } @@ -766,6 +774,7 @@ func (r *testRunner) runTest( c *clusterImpl, 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) @@ -821,7 +830,9 @@ func (r *testRunner) runTest( shout(ctx, l, stdout, "--- FAIL: %s (%s)\n%s", runID, durationStr, output) - r.maybePostGithubIssue(ctx, l, t, 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 ...]`, @@ -1023,7 +1034,7 @@ func (r *testRunner) teardownTest( c.FailOnReplicaDivergence(ctx, t) if timedOut || t.Failed() { - r.collectClusterArtifacts(ctx, c, t) + r.collectClusterArtifacts(ctx, c, t.L()) } }) @@ -1052,91 +1063,9 @@ 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, stdout io.Writer, output string, +func (r *testRunner) collectClusterArtifacts( + ctx context.Context, c *clusterImpl, l *logger.Logger, ) { - 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), - } - - 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. -func (r *testRunner) collectClusterArtifacts(ctx context.Context, c *clusterImpl, t test.Test) { // 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 +1074,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) }