Skip to content

Commit cbdd88b

Browse files
craig[bot]williamchoe3msbutler
committed
149479: roachtest: exit with failure on github post errors r=herkolategan,DarrylWong a=williamchoe3 Fixes #147116 ### Changes #### Highlevel Changes Added a new failure path first by * adding a new counter in `testRunner` struct which get's incremented when `github.MaybePost()` (called in `testRunner.runWorkers()` and `testRunner.runTests()` )returns an error. When this count > 0, `testRunner.Run()` will return a new error `errGithubPostFailed` and when `main()` sees that error, it will return a new exit code `12` which will fail the pipeline (unlike exit code 10, 11) * ^ 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 ``` type testRunner struct { ... // numGithubPostErrs Counts GitHub post errors across all workers numGithubPostErrs int32 ... } ... issue, err := github.MaybePost(t, issueInfo, l, output, params) // TODO add cluster specific args here if err != nil { shout(ctx, l, stdout, "failed to post issue: %s", err) atomic.AddInt32(&r.numGithubPostErrs, 1) } ``` #### Design In order to do verification via unit tests, i'm used to using something like Python's magic mock, but that's not available in GoLang so i opted for a Dependency Injection approach. (This was the best I could come up with, I wanted to avoid "if unit test, do this" logic. If anyone has any other approaches / suggestions let me know!) I made a new interface `GithubPoster` in such a way that the original `githubIssues` implements that new interface. I then pass this interface in function signatures all the way from `Run()` to `runTests()`. Then in the unit tests, I could pass a different implementation of `GithubPoster` that has a `MaybePost()` that always fails. `github.go` ``` type GithubPoster interface { MaybePost( t *testImpl, issueInfo *githubIssueInfo, l *logger.Logger, message string, params map[string]string) ( *issues.TestFailureIssue, error) } ``` Another issue with this approach is the original `githubIssues` has information that is cluster specific, but because of dependency injection, it's now a shared struct among all the workers, so it doesn't make sense to store certain fields that are worker dependent. For the fields that are worker specific, I created a new struct `githubIssueInfo` that is created in `runWorkers()`, similar to how `githubIssues` used to be created there. Note: I don't love the name `githubIssueInfo`, but i wanted to stick with a similar naming convention to `githubIssues`, open to name suggestions ``` // Original githubIssues type githubIssues struct { disable bool cluster *clusterImpl vmCreateOpts *vm.CreateOpts issuePoster func(context.Context, issues.Logger, issues.IssueFormatter, issues.PostRequest, *issues.Options) (*issues.TestFailureIssue, error) teamLoader func() (team.Map, error) } // New githubIssues type githubIssues struct { disable bool issuePoster func(context.Context, issues.Logger, issues.IssueFormatter, issues.PostRequest, *issues.Options) (*issues.TestFailureIssue, error) teamLoader func() (team.Map, error) } ``` All this was very verbose and didn't love that i had to change all the function signatures to do this, open to other ways to do verification. ### Misc Also first time writing in Go in like ~3 years very open to general go semantic feedback / best practices / design patterns ### Verification Diff of binary I used to manually confirm if you wanna see where I hardcoded to return errors: 611adcc #### Manual Test Logs > ➜ cockroach git:(wchoe/147116-github-err-will-fail-pipeline) ✗ tmp/roachtest run acceptance/build-info --cockroach /Users/wchoe/work/cockroachdb/cockroach/bin_linux/cockroach > ... > Running tests which match regex "acceptance/build-info" and are compatible with cloud "gce". > > fallback runner logs in: artifacts/roachtest.crdb.log > 2025/07/09 00:51:48 run.go:386: test runner logs in: artifacts/_runner-logs/test_runner-1752022308.log > test runner logs in: artifacts/_runner-logs/test_runner-1752022308.log > HTTP server listening on port 56238 on localhost: http://localhost:56238/ > 2025/07/09 00:51:48 run.go:148: global random seed: 1949199437086051249 > 2025/07/09 00:51:48 test_runner.go:398: test_run_id: will.choe-1752022308 > test_run_id: will.choe-1752022308 > [w0] 2025/07/09 00:51:48 work_pool.go:198: Acquired quota for 16 CPUs > [w0] 2025/07/09 00:51:48 cluster.go:3204: Using randomly chosen arch="amd64", acceptance/build-info > [w0] 2025/07/09 00:51:48 test_runner.go:798: Unable to create (or reuse) cluster for test acceptance/build-info due to: mocking. > Unable to create (or reuse) cluster for test acceptance/build-info due to: mocking. > 2025/07/09 00:51:48 test_impl.go:478: test failure #1: full stack retained in failure_1.log: (test_runner.go:873).func4: mocking [owner=test-eng] > 2025/07/09 00:51:48 test_impl.go:200: Runtime assertions disabled > [w0] 2025/07/09 00:51:48 test_runner.go:883: failed to post issue: mocking > failed to post issue: mocking > [w0] 2025/07/09 00:51:48 test_runner.go:1019: test failed: acceptance/build-info (run 1) > [w0] 2025/07/09 00:51:48 test_runner.go:732: Releasing quota for 16 CPUs > [w0] 2025/07/09 00:51:48 test_runner.go:744: No work remaining; runWorker is bailing out... > No work remaining; runWorker is bailing out... > [w0] 2025/07/09 00:51:48 test_runner.go:643: Worker exiting; no cluster to destroy. > 2025/07/09 00:51:48 test_runner.go:460: PASS > PASS > 2025/07/09 00:51:48 test_runner.go:465: 1 clusters could not be created and 1 errors occurred while posting to github > 1 clusters could not be created and 1 errors occurred while posting to github > 2025/07/09 00:51:48 run.go:200: runTests destroying all clusters > Error: some clusters could not be created > failed to POST to GitHub > ➜ cockroach git:(wchoe/147116-github-err-will-fail-pipeline) ✗ echo $? > 12 149913: crosscluster/physical: persist standby poller progress r=dt a=msbutler This patch sets the standby poller job's resolved time to the system time that standby descriptors have been updated to. This allows a reader tenant user to easily check that the poller job is running smoothly via SHOW JOB. Epic: none Release note: none Co-authored-by: William Choe <williamchoe3@gmail.com> Co-authored-by: Michael Butler <butler@cockroachlabs.com>
3 parents 5a4e4e8 + 3ac7c44 + d8d8fa4 commit cbdd88b

File tree

11 files changed

+266
-62
lines changed

11 files changed

+266
-62
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
@@ -52,6 +52,7 @@ func TestGlobalMonitorError(t *testing.T) {
5252
defer stopper.Stop(ctx)
5353
cr := newClusterRegistry()
5454
runner := newUnitTestRunner(cr, stopper)
55+
github := defaultGithub(runner.config.disableIssue)
5556

5657
var buf syncedBuffer
5758
copt := defaultClusterOpt()
@@ -75,6 +76,6 @@ func TestGlobalMonitorError(t *testing.T) {
7576
},
7677
}
7778
err := runner.Run(ctx, []registry.TestSpec{mockTest}, 1, /* count */
78-
defaultParallelism, copt, testOpts{}, lopt)
79+
defaultParallelism, copt, testOpts{}, lopt, github)
7980
require.Error(t, err)
8081
}

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)