diff --git a/pkg/cmd/roachtest/BUILD.bazel b/pkg/cmd/roachtest/BUILD.bazel index 20f29bb662db..f7dc7de8adfb 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,12 +65,14 @@ go_test( size = "small", srcs = [ "cluster_test.go", + "github_test.go", "main_test.go", "test_registry_test.go", "test_test.go", ], embed = [":roachtest_lib"], deps = [ + "//pkg/cmd/internal/issues", "//pkg/cmd/roachtest/cluster", "//pkg/cmd/roachtest/option", "//pkg/cmd/roachtest/registry", @@ -77,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..f6ea8fb0871e --- /dev/null +++ b/pkg/cmd/roachtest/github.go @@ -0,0 +1,135 @@ +// Copyright 2018 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" + "io" + "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 interface { + maybePost( + ctx context.Context, + t test.Test, + stdout io.Writer, + output string, + ) +} + +type githubIssuesImpl struct { + disable bool + l *logger.Logger + cluster *clusterImpl + vmCreateOpts *vm.CreateOpts + issuePoster func(ctx context.Context, formatter issues.IssueFormatter, req issues.PostRequest) error +} + +func (g *githubIssuesImpl) 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 roachtestPrefix(p string) string { + return "ROACHTEST_" + p +} + +func (g *githubIssuesImpl) maybePost( + ctx context.Context, t test.Test, stdout io.Writer, output string, +) { + if !g.shouldPost(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") + } + + 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 if requested + if g.vmCreateOpts != nil { + clusterParams[roachtestPrefix("fs")] = g.vmCreateOpts.SSDOpts.FileSystem + } + + if g.cluster != nil { + clusterParams[roachtestPrefix("encrypted")] = fmt.Sprintf("%v", g.cluster.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 := g.issuePoster( + context.Background(), + issues.UnitTestFormatter, + req, + ); err != nil { + shout(ctx, g.l, stdout, "failed to post issue: %s", err) + } +} diff --git a/pkg/cmd/roachtest/github_test.go b/pkg/cmd/roachtest/github_test.go new file mode 100644 index 000000000000..23d7ee58727b --- /dev/null +++ b/pkg/cmd/roachtest/github_test.go @@ -0,0 +1,157 @@ +// Copyright 2018 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" + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/cmd/internal/issues" + "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/roachprod/vm" +) + +//teams by default expects to find TEAMS.yaml in the tree of the execution root +//unittests are executed from elsewhere, hence we create one where it can be found +func writeTeams(t *testing.T) { + var teams = `cockroachdb/unowned: + aliases: + cockroachdb/rfc-prs: other + triage_column_id: 0` + + data := []byte(teams) + err := os.WriteFile("TEAMS.yaml", data, 0644) + + if err != nil { + t.Fatal(err) + } +} + +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 TestMaybePost(t *testing.T) { + testCases := []struct { + disableIssues bool + nodeCount int + envGithubAPIToken string + envTcBuildBranch string + nonReleaseBlocker bool + clusterCreationFailed bool + expectedPost bool + expectedParams map[string]string + }{ + /* 1 - 4 verify that issues are not posted if any of on the relevant criteria checks fail */ + // disable + {true, 1, "token", "master", false, false, false, nil}, + // nodeCount + {false, 0, "token", "master", false, false, false, nil}, + // apiToken + {false, 1, "", "master", false, false, false, nil}, + // branch + {false, 1, "token", "", false, false, false, nil}, + {false, 1, "token", "master", false, false, true, + prefixAll(map[string]string{ + "cloud": "gce", + "encrypted": "false", + "fs": "ext4", + "ssd": "0", + "cpu": "4", + }), + }, + //assert release-blocker label + {false, 1, "token", "master", true, false, true, nil}, + // failed cluster creation - nil vmOptions and clusterImpl + {false, 1, "token", "master", true, true, true, + prefixAll(map[string]string{ + "cloud": "gce", + "ssd": "0", + "cpu": "4", + }), + }, + } + + writeTeams(t) + 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: "unowned", + Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {}, + Cluster: clusterSpec, + } + + ti := &testImpl{ + spec: testSpec, + l: nilLogger(), + } + + testClusterImpl := &clusterImpl{spec: clusterSpec} + vo := vm.DefaultCreateOpts() + vmOpts := &vo + + if c.clusterCreationFailed { + testClusterImpl = nil + vmOpts = nil + } + + github := &githubIssuesImpl{ + disable: c.disableIssues, + vmCreateOpts: vmOpts, + cluster: testClusterImpl, + l: nilLogger(), + issuePoster: func(ctx context.Context, formatter issues.IssueFormatter, req issues.PostRequest) error { + if !c.expectedPost { + t.Logf("issue should not have been posted") + t.FailNow() + } + + if c.expectedParams != nil { + if len(c.expectedParams) != len(req.ExtraParams) { + t.Logf("expected %v, actual %v", c.expectedParams, req.ExtraParams) + t.FailNow() + } + + for k, v := range c.expectedParams { + if req.ExtraParams[k] != v { + t.Logf("expected %v, actual %v", c.expectedParams, req.ExtraParams) + t.FailNow() + } + } + } + + if c.nonReleaseBlocker && req.ExtraLabels[len(req.ExtraLabels)-1] != "release-blocker" { + t.Logf("Expected label release-blocker") + t.FailNow() + } + return nil + }, + } + + github.maybePost(context.Background(), ti, nil, "body") + } +} diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index ad7ca19674c1..92b8fab887b6 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -34,7 +34,6 @@ import ( "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 +50,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 { @@ -594,6 +595,14 @@ func (r *testRunner) runWorker( // Now run the test. l.PrintfCtx(ctx, "starting test: %s:%d", testToRun.spec.Name, testToRun.runNum) + github := &githubIssuesImpl{ + disable: r.config.disableIssue, + vmCreateOpts: vmCreateOpts, + cluster: c, + l: l, + issuePoster: issues.Post, + } + 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. @@ -608,7 +617,8 @@ 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) + + github.maybePost(ctx, t, stdout, issueOutput) // Restore test name and owner. t.spec.Name = oldName t.spec.Owner = oldOwner @@ -634,7 +644,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) } if err != nil { @@ -744,9 +754,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) @@ -802,7 +812,7 @@ 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) + github.maybePost(ctx, t, stdout, output) } else { shout(ctx, l, stdout, "--- PASS: %s (%s)", runID, durationStr) // If `##teamcity[testFailed ...]` is not present before `##teamCity[testFinished ...]`, @@ -1033,104 +1043,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(