Skip to content

Commit 3ac7c44

Browse files
committed
roachtest: return non-zero exit code from roachtest pipeline if GitHub issue
creation fails Previously, pipeline would not fail when encountering an issue posting to github, which would cause thsee failures to be "silent". Now pipelines will fail when there's an error posting to github increasing visability. New mechanism is very similar to how provisioning errors are tracked and returned to main. Does not trigger test short circuiting mechanism because testRunner.runWorkers() doesn't return an error Introduces dependency injection for GitHub in Run so GitHub failures can be mocked in testing Resolves #147116 Epic: None Release note: None
1 parent 71bcba2 commit 3ac7c44

File tree

7 files changed

+233
-61
lines changed

7 files changed

+233
-61
lines changed

pkg/cmd/roachtest/github.go

Lines changed: 34 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -23,21 +23,36 @@ import (
2323
"github.com/cockroachdb/cockroach/pkg/roachprod/vm"
2424
)
2525

26+
// GithubPoster interface allows MaybePost to be mocked in unit tests that test
27+
// failure modes.
28+
type GithubPoster interface {
29+
MaybePost(
30+
t *testImpl, issueInfo *githubIssueInfo, l *logger.Logger, message string,
31+
params map[string]string) (
32+
*issues.TestFailureIssue, error)
33+
}
34+
35+
// githubIssues struct implements GithubPoster
2636
type githubIssues struct {
27-
disable bool
37+
disable bool
38+
issuePoster func(context.Context, issues.Logger, issues.IssueFormatter, issues.PostRequest,
39+
*issues.Options) (*issues.TestFailureIssue, error)
40+
teamLoader func() (team.Map, error)
41+
}
42+
43+
// githubIssueInfo struct contains information related to this issue on this
44+
// worker / test
45+
// separate from githubIssues because githubIssues is shared amongst all workers
46+
type githubIssueInfo struct {
2847
cluster *clusterImpl
2948
vmCreateOpts *vm.CreateOpts
30-
issuePoster func(context.Context, issues.Logger, issues.IssueFormatter, issues.PostRequest, *issues.Options) (*issues.TestFailureIssue, error)
31-
teamLoader func() (team.Map, error)
3249
}
3350

34-
func newGithubIssues(disable bool, c *clusterImpl, vmCreateOpts *vm.CreateOpts) *githubIssues {
35-
return &githubIssues{
36-
disable: disable,
51+
// newGithubIssueInfo constructor for newGithubIssueInfo
52+
func newGithubIssueInfo(cluster *clusterImpl, vmCreateOpts *vm.CreateOpts) *githubIssueInfo {
53+
return &githubIssueInfo{
54+
cluster: cluster,
3755
vmCreateOpts: vmCreateOpts,
38-
cluster: c,
39-
issuePoster: issues.Post,
40-
teamLoader: team.DefaultLoadTeams,
4156
}
4257
}
4358

@@ -177,6 +192,7 @@ func (g *githubIssues) createPostRequest(
177192
runtimeAssertionsBuild bool,
178193
coverageBuild bool,
179194
params map[string]string,
195+
issueInfo *githubIssueInfo,
180196
) (issues.PostRequest, error) {
181197
var mention []string
182198

@@ -265,8 +281,8 @@ func (g *githubIssues) createPostRequest(
265281

266282
artifacts := fmt.Sprintf("/%s", testName)
267283

268-
if g.cluster != nil {
269-
issueClusterName = g.cluster.name
284+
if issueInfo.cluster != nil {
285+
issueClusterName = issueInfo.cluster.name
270286
}
271287

272288
issueMessage := messagePrefix + message
@@ -303,8 +319,13 @@ func (g *githubIssues) createPostRequest(
303319
}, nil
304320
}
305321

322+
// MaybePost entry point for POSTing an issue to GitHub
306323
func (g *githubIssues) MaybePost(
307-
t *testImpl, l *logger.Logger, message string, params map[string]string,
324+
t *testImpl,
325+
issueInfo *githubIssueInfo,
326+
l *logger.Logger,
327+
message string,
328+
params map[string]string,
308329
) (*issues.TestFailureIssue, error) {
309330
skipReason := g.shouldPost(t)
310331
if skipReason != "" {
@@ -315,7 +336,7 @@ func (g *githubIssues) MaybePost(
315336
postRequest, err := g.createPostRequest(
316337
t.Name(), t.start, t.end, t.spec, t.failures(),
317338
message,
318-
roachtestutil.UsingRuntimeAssertions(t), t.goCoverEnabled, params,
339+
roachtestutil.UsingRuntimeAssertions(t), t.goCoverEnabled, params, issueInfo,
319340
)
320341

321342
if err != nil {

pkg/cmd/roachtest/github_test.go

Lines changed: 8 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -2,7 +2,6 @@
22
//
33
// Use of this software is governed by the CockroachDB Software License
44
// included in the /LICENSE file.
5-
65
package main
76

87
import (
@@ -33,12 +32,12 @@ import (
3332

3433
var (
3534
teamsYaml = `cockroachdb/unowned:
36-
aliases:
37-
cockroachdb/rfc-prs: other
35+
aliases:
36+
cockroachdb/rfc-prs: other
3837
cockroachdb/test-eng:
39-
label: T-testeng
38+
label: T-testeng
4039
cockroachdb/dev-inf:
41-
label: T-dev-inf`
40+
label: T-dev-inf`
4241

4342
validTeamsFn = func() (team.Map, error) { return loadYamlTeams(teamsYaml) }
4443
invalidTeamsFn = func() (team.Map, error) { return loadYamlTeams("invalid yaml") }
@@ -154,10 +153,9 @@ func TestCreatePostRequest(t *testing.T) {
154153
datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string {
155154
if d.Cmd == "post" {
156155
github := &githubIssues{
157-
vmCreateOpts: vmOpts,
158-
cluster: testClusterImpl,
159-
teamLoader: teamLoadFn,
156+
teamLoader: teamLoadFn,
160157
}
158+
issueInfo := newGithubIssueInfo(testClusterImpl, vmOpts)
161159

162160
// See: `formatFailure` which formats failures for roachtests. Try to
163161
// follow it here.
@@ -173,10 +171,11 @@ func TestCreatePostRequest(t *testing.T) {
173171
}
174172
message := b.String()
175173

176-
params := getTestParameters(ti, github.cluster, github.vmCreateOpts)
174+
params := getTestParameters(ti, issueInfo.cluster, issueInfo.vmCreateOpts)
177175
req, err := github.createPostRequest(
178176
testName, ti.start, ti.end, testSpec, testCase.failures,
179177
message, roachtestutil.UsingRuntimeAssertions(ti), ti.goCoverEnabled, params,
178+
issueInfo,
180179
)
181180
if testCase.loadTeamsFailed {
182181
// Assert that if TEAMS.yaml cannot be loaded then function errors.

pkg/cmd/roachtest/main.go

Lines changed: 11 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -47,6 +47,12 @@ const (
4747
// created due to errors during cloud hardware allocation.
4848
ExitCodeClusterProvisioningFailed = 11
4949

50+
// ExitCodeGithubPostFailed is the exit code indicating a failure in posting
51+
// results to GitHub successfully.
52+
// Note: This error masks the actual roachtest status i.e. this error can
53+
// occur with any of the other exit codes.
54+
ExitCodeGithubPostFailed = 12
55+
5056
// runnerLogsDir is the dir under the artifacts root where the test runner log
5157
// and other runner-related logs (i.e. cluster creation logs) will be written.
5258
runnerLogsDir = "_runner-logs"
@@ -241,11 +247,12 @@ Check --parallelism, --run-forever and --wait-before-next-execution flags`,
241247

242248
if err := rootCmd.Execute(); err != nil {
243249
code := 1
244-
if errors.Is(err, errTestsFailed) {
245-
code = ExitCodeTestsFailed
246-
}
247-
if errors.Is(err, errSomeClusterProvisioningFailed) {
250+
if errors.Is(err, errGithubPostFailed) {
251+
code = ExitCodeGithubPostFailed
252+
} else if errors.Is(err, errSomeClusterProvisioningFailed) {
248253
code = ExitCodeClusterProvisioningFailed
254+
} else if errors.Is(err, errTestsFailed) {
255+
code = ExitCodeTestsFailed
249256
}
250257
// Cobra has already printed the error message.
251258
os.Exit(code)

pkg/cmd/roachtest/run.go

Lines changed: 9 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ import (
2020

2121
"github.com/DataDog/datadog-api-client-go/v2/api/datadog"
2222
"github.com/DataDog/datadog-api-client-go/v2/api/datadogV1"
23+
"github.com/cockroachdb/cockroach/pkg/cmd/bazci/githubpost/issues"
2324
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
2425
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestflags"
2526
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
@@ -138,6 +139,12 @@ func runTests(register func(registry.Registry), filter *registry.TestFilter) err
138139
literalArtifactsDir: literalArtifactsDir,
139140
runnerLogPath: runnerLogPath,
140141
}
142+
143+
github := &githubIssues{
144+
disable: runner.config.disableIssue,
145+
issuePoster: issues.Post,
146+
}
147+
141148
l.Printf("global random seed: %d", roachtestflags.GlobalSeed)
142149
go func() {
143150
if err := http.ListenAndServe(
@@ -183,7 +190,8 @@ func runTests(register func(registry.Registry), filter *registry.TestFilter) err
183190
goCoverEnabled: roachtestflags.GoCoverEnabled,
184191
exportOpenMetrics: roachtestflags.ExportOpenmetrics,
185192
},
186-
lopt)
193+
lopt,
194+
github)
187195

188196
// Make sure we attempt to clean up. We run with a non-canceled ctx; the
189197
// ctx above might be canceled in case a signal was received. If that's

pkg/cmd/roachtest/test_monitor_test.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -49,6 +49,7 @@ func TestGlobalMonitorError(t *testing.T) {
4949
defer stopper.Stop(ctx)
5050
cr := newClusterRegistry()
5151
runner := newUnitTestRunner(cr, stopper)
52+
github := defaultGithub(runner.config.disableIssue)
5253

5354
var buf syncedBuffer
5455
copt := defaultClusterOpt()
@@ -72,6 +73,6 @@ func TestGlobalMonitorError(t *testing.T) {
7273
},
7374
}
7475
err := runner.Run(ctx, []registry.TestSpec{mockTest}, 1, /* count */
75-
defaultParallelism, copt, testOpts{}, lopt)
76+
defaultParallelism, copt, testOpts{}, lopt, github)
7677
require.Error(t, err)
7778
}

pkg/cmd/roachtest/test_runner.go

Lines changed: 47 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -62,9 +62,15 @@ func init() {
6262
var (
6363
errTestsFailed = fmt.Errorf("some tests failed")
6464

65-
// reference error used by main.go at the end of a run of tests
65+
// errSomeClusterProvisioningFailed error sent after a run in
66+
// [testRunner.Run] if any worker encountered a cluster provisioning error.
67+
// Used in main.go to determine the run exit code.
6668
errSomeClusterProvisioningFailed = fmt.Errorf("some clusters could not be created")
6769

70+
// errGithubPostFailed error sent after a run in [testRunner.Run] if any
71+
// worker encountered an error when trying to POST to GitHub
72+
errGithubPostFailed = fmt.Errorf("failed to POST to GitHub")
73+
6874
prometheusNameSpace = "roachtest"
6975
// prometheusScrapeInterval should be consistent with the scrape interval defined in
7076
// https://grafana.testeng.crdb.io/prometheus/config
@@ -174,8 +180,11 @@ type testRunner struct {
174180
completed []completedTestInfo
175181
}
176182

177-
// Counts cluster creation errors across all workers.
183+
// numClusterErrs Counts cluster creation errors across all workers.
178184
numClusterErrs int32
185+
186+
// numGithubPostErrs Counts GitHub post errors across all workers
187+
numGithubPostErrs int32
179188
}
180189

181190
type perfMetricsCollector struct {
@@ -308,6 +317,7 @@ func (r *testRunner) Run(
308317
clustersOpt clustersOpt,
309318
topt testOpts,
310319
lopt loggingOpt,
320+
github GithubPoster,
311321
) error {
312322
// Validate options.
313323
if len(tests) == 0 {
@@ -411,6 +421,7 @@ func (r *testRunner) Run(
411421
topt,
412422
childLogger,
413423
n*count,
424+
github,
414425
)
415426

416427
if err != nil {
@@ -448,14 +459,25 @@ func (r *testRunner) Run(
448459
passFailLine := r.generateReport()
449460
shout(ctx, l, lopt.stdout, passFailLine)
450461

462+
// For the errors that don't short-circuit the pipeline run, return a joined
463+
// error and leave case handling to the caller
464+
var err error
465+
if r.numGithubPostErrs > 0 {
466+
shout(ctx, l, lopt.stdout, "%d errors occurred while posting to github", r.numGithubPostErrs)
467+
err = errors.Join(err, errGithubPostFailed)
468+
}
451469
if r.numClusterErrs > 0 {
452470
shout(ctx, l, lopt.stdout, "%d clusters could not be created", r.numClusterErrs)
453-
return errSomeClusterProvisioningFailed
471+
err = errors.Join(err, errSomeClusterProvisioningFailed)
454472
}
455-
456473
if len(r.status.fail) > 0 {
457-
return errTestsFailed
474+
shout(ctx, l, lopt.stdout, "%d tests failed", r.status.fail)
475+
err = errors.Join(err, errTestsFailed)
476+
}
477+
if err != nil {
478+
return err
458479
}
480+
459481
// To ensure all prometheus metrics have been scraped, ensure shutdown takes
460482
// at least one scrapeInterval, unless the roachtest fails or gets cancelled.
461483
requiredShutDownTime := prometheusScrapeInterval
@@ -596,6 +618,7 @@ func (r *testRunner) runWorker(
596618
topt testOpts,
597619
l *logger.Logger,
598620
maxTotalFailures int,
621+
github GithubPoster,
599622
) error {
600623
stdout := lopt.stdout
601624

@@ -841,18 +864,22 @@ func (r *testRunner) runWorker(
841864
runID: generateRunID(clustersOpt),
842865
}
843866
t.ReplaceL(testL)
844-
github := newGithubIssues(r.config.disableIssue, c, vmCreateOpts)
845-
867+
issueInfo := newGithubIssueInfo(c, vmCreateOpts)
846868
// handleClusterCreationFailure can be called when the `err` given
847869
// occurred for reasons related to creating or setting up a
848870
// cluster for a test.
849-
handleClusterCreationFailure := func(err error) {
850-
t.Error(errClusterProvisioningFailed(err))
851-
852-
params := getTestParameters(t, github.cluster, github.vmCreateOpts)
871+
handleClusterCreationFailure := func(clusterCreateErr error) {
872+
t.Error(errClusterProvisioningFailed(clusterCreateErr))
873+
874+
// Technically don't need the issueInfo struct here because we have access
875+
// to the clusterImpl and vm.CreateOpts in runWorker()
876+
// but not in runTests() so keeping the invocation of getTestParameters()
877+
// the same in both spots
878+
params := getTestParameters(t, issueInfo.cluster, issueInfo.vmCreateOpts)
853879
logTestParameters(l, params)
854-
if _, err := github.MaybePost(t, l, t.failureMsg(), params); err != nil {
855-
shout(ctx, l, stdout, "failed to post issue: %s", err)
880+
if _, githubErr := github.MaybePost(t, issueInfo, l, t.failureMsg(), params); githubErr != nil {
881+
atomic.AddInt32(&r.numGithubPostErrs, 1)
882+
shout(ctx, l, stdout, "failed to post issue: %s", githubErr)
856883
}
857884
}
858885

@@ -978,7 +1005,8 @@ func (r *testRunner) runWorker(
9781005
wStatus.SetTest(t, testToRun)
9791006
wStatus.SetStatus("running test")
9801007

981-
r.runTest(ctx, t, testToRun.runNum, testToRun.runCount, c, stdout, testL, github)
1008+
r.runTest(ctx, t, testToRun.runNum, testToRun.runCount, c, stdout, testL,
1009+
github, issueInfo)
9821010
}
9831011
}
9841012

@@ -1135,7 +1163,8 @@ func (r *testRunner) runTest(
11351163
c *clusterImpl,
11361164
stdout io.Writer,
11371165
l *logger.Logger,
1138-
github *githubIssues,
1166+
github GithubPoster,
1167+
issueInfo *githubIssueInfo,
11391168
) {
11401169
testRunID := t.Name()
11411170
if runCount > 1 {
@@ -1238,11 +1267,12 @@ func (r *testRunner) runTest(
12381267
}
12391268

12401269
output := fmt.Sprintf("%s\ntest artifacts and logs in: %s", failureMsg, t.ArtifactsDir())
1241-
params := getTestParameters(t, github.cluster, github.vmCreateOpts)
1270+
params := getTestParameters(t, issueInfo.cluster, issueInfo.vmCreateOpts)
12421271
logTestParameters(l, params)
1243-
issue, err := github.MaybePost(t, l, output, params)
1272+
issue, err := github.MaybePost(t, issueInfo, l, output, params)
12441273
if err != nil {
12451274
shout(ctx, l, stdout, "failed to post issue: %s", err)
1275+
atomic.AddInt32(&r.numGithubPostErrs, 1)
12461276
}
12471277

12481278
// If an issue was created (or comment added) on GitHub,

0 commit comments

Comments
 (0)