Skip to content

Commit

Permalink
roachtest: Utilise structured errors for detection of SSH flakes.
Browse files Browse the repository at this point in the history
This commit utilises cockroachdb/errors markers api to mark an SSH
error with exit code of 255. The test runner, when it is posting an
issue to github, can act according to which marker may be present
in t.mu.failures[0]. In this case, we override the owning team to
protect them from having to investigate what is likely a transient
SSH issue.

A test now has the concept of a failure[] instead of error[]. Each
failure contains the original `squashedErr` and an errors[],
which are all the errors passed via t.Failf/Errorf.

We can then preserve the relationship of multiple
errors to any particular failure within a test, and match on any given
reference error (like SSH flakes from above)

This moves us away from substring matching on error messages.

Release justification: test-only change
Release note: None
  • Loading branch information
Miral Gadani committed Oct 17, 2022
1 parent 0000095 commit 4d32e19
Show file tree
Hide file tree
Showing 7 changed files with 113 additions and 76 deletions.
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ go_library(
"//pkg/internal/team",
"//pkg/roachprod",
"//pkg/roachprod/config",
"//pkg/roachprod/errors",
"//pkg/roachprod/install",
"//pkg/roachprod/logger",
"//pkg/roachprod/prometheus",
Expand Down
18 changes: 15 additions & 3 deletions pkg/cmd/roachtest/github.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/internal/team"
rperrors "github.com/cockroachdb/cockroach/pkg/roachprod/errors"
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/cockroach/pkg/roachprod/vm"
)
Expand All @@ -35,9 +36,9 @@ type githubIssues struct {
type issueCategory int

const (
clusterCreationErr issueCategory = iota
otherErr issueCategory = iota
clusterCreationErr
sshErr
otherErr
)

func newGithubIssues(
Expand Down Expand Up @@ -152,11 +153,22 @@ func (g *githubIssues) createPostRequest(
}
}

func (g *githubIssues) MaybePost(t test.Test, cat issueCategory, message string) error {
func (g *githubIssues) MaybePost(t *testImpl, message string) error {
if !g.shouldPost(t) {
return nil
}

//TODO: perhaps remove category completely and move this to
// issuePoster?
cat := otherErr

// Overrides to shield eng teams from potential flakes
firstFailure := t.firstFailure()
if failureContainsError(errClusterProvisioningFailed, firstFailure) {
cat = clusterCreationErr
} else if failureContainsError(rperrors.ErrSSH255, firstFailure) {
cat = sshErr
}
return g.issuePoster(
context.Background(),
issues.UnitTestFormatter,
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -338,7 +338,7 @@ runner itself.
if errors.Is(err, errTestsFailed) {
code = ExitCodeTestsFailed
}
if errors.Is(err, errClusterProvisioningFailed) {
if errors.Is(err, errSomeClusterProvisioningFailed) {
code = ExitCodeClusterProvisioningFailed
}
// Cobra has already printed the error message.
Expand Down
119 changes: 72 additions & 47 deletions pkg/cmd/roachtest/test_impl.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,18 @@ type testStatus struct {
progress float64
}

// Holds all error information from a single invocation of t.Error(f)/t.Fatal(f) to
// preserve any structured errors
// e.g. t.Fatalf("foo %s %s %s", "hello", err1, err2) would mean that
// failure.errors == [err1, err2], with all args (including the non error "hello")
// being captured in the squashedErr
type failure struct {
// this is the single error created from variadic args passed to t.Error(f)/t.Fatal(f)
squashedErr error
// errors are all the `errors` present in the variadic args
errors []error
}

type testImpl struct {
spec *registry.TestSpec

Expand Down Expand Up @@ -75,18 +87,12 @@ type testImpl struct {
// test is being marked as failed (i.e. when the failed field above is also
// set). This is used to cancel the context passed to t.spec.Run(), so async
// test goroutines can be notified.
cancel func()
failLoc struct {
file string
line int
}
cancel func()

// Errors are all the errors passed to `addFailure`, in order of
// these calls.
//
// NB: the first failure is not always the relevant one due to:
// https://github.com/cockroachdb/cockroach/issues/44436
errors []error
// failures added via addFailures, in order
// A test can have multiple calls to t.Fail()/Error(), with each call
// referencing 0+ errors. failure captures all the errors
failures []failure

// status is a map from goroutine id to status set by that goroutine. A
// special goroutine is indicated by runnerID; that one provides the test's
Expand All @@ -106,6 +112,10 @@ type testImpl struct {
versionsBinaryOverride map[string]string
}

func newFailure(squashedErr error, errs []error) failure {
return failure{squashedErr: squashedErr, errors: errs}
}

// BuildVersion exposes the build version of the cluster
// in this test.
func (t *testImpl) BuildVersion() *version.Version {
Expand Down Expand Up @@ -253,21 +263,20 @@ func (t *testImpl) Skipf(format string, args ...interface{}) {
panic(errTestFatal)
}

// This creates an error from the first arg, and adds each subsequent arg
// as error detail
func argsToErr(depth int, args ...interface{}) error {
// NB: we'd probably not allow multiple arguments here and we'd want
// the one remaining arg to be an `error`, but we are trying to be
// compatible with `(*testing.T).Fatal`.
var err error
for _, arg := range args {
if err == nil {
err = errors.NewWithDepthf(depth+1, "%v", arg)
continue
// collectErrors extracts any arg that is an error
func collectErrors(args ...interface{}) []error {
var errs []error
for _, a := range args {
if err, ok := a.(error); ok {
errs = append(errs, err)
}
err = errors.WithDetailf(err, "%v", arg)
}
return err
return errs
}

// FailNow implements the TestingT interface.
func (t *testImpl) FailNow() {
t.Fatal("FailNow called")
}

// Fatal marks the test as failed, prints the args to t.L(), and calls
Expand All @@ -279,61 +288,57 @@ func argsToErr(depth int, args ...interface{}) error {
// ATTENTION: Since this calls panic(errTestFatal), it should only be called
// from a test's closure. The test runner itself should never call this.
func (t *testImpl) Fatal(args ...interface{}) {
t.addFailure(argsToErr(1, args...))
t.Error(args...)
panic(errTestFatal)
}

// Fatalf is like Fatal, but takes a format string.
func (t *testImpl) Fatalf(format string, args ...interface{}) {
t.addFailure(errors.NewWithDepthf(1, format, args...))
panic(errTestFatal)
}

// FailNow implements the TestingT interface.
func (t *testImpl) FailNow() {
t.addFailure(errors.NewWithDepthf(1, "FailNow called"))
t.Errorf(format, args)
panic(errTestFatal)
}

func (t *testImpl) Error(args ...interface{}) {
t.addFailure(argsToErr(1, args...))
t.Errorf(strings.Repeat(" %v", len(args))[1:], args...)
}

// Errorf implements the TestingT interface.
func (t *testImpl) Errorf(format string, args ...interface{}) {
t.addFailure(errors.NewWithDepthf(1, format, args...))
t.addFailure(newFailure(errors.NewWithDepthf(1, format, args...), collectErrors(args...)))
}

func formatFailure(b *strings.Builder, errs ...error) {
for i, err := range errs {
// We take the first error from each failure which is the
// "squashed" error that contains all information of a failure
func formatFailure(b *strings.Builder, reportFailures ...failure) {
for i, failure := range reportFailures {
if i > 0 {
fmt.Fprintln(b)
}
file, line, fn, ok := errors.GetOneLineSource(err)
file, line, fn, ok := errors.GetOneLineSource(failure.squashedErr)
if !ok {
file, line, fn = "<unknown>", 0, "unknown"
}
fmt.Fprintf(b, "(%s:%d).%s: %v", file, line, fn, err)
fmt.Fprintf(b, "(%s:%d).%s: %v", file, line, fn, failure.squashedErr)
}
}

func (t *testImpl) addFailure(reportErr error) {
func (t *testImpl) addFailure(reportFailure failure) {
t.mu.Lock()
defer t.mu.Unlock()

t.mu.errors = append(t.mu.errors, reportErr)
t.mu.failures = append(t.mu.failures, reportFailure)

var b strings.Builder
formatFailure(&b, reportErr)
formatFailure(&b, reportFailure)
msg := b.String()

t.L().Printf("test failure #%d: %s", len(t.mu.errors), msg)
t.L().Printf("test failure #%d: %s", len(t.mu.failures), msg)
// Also dump the verbose error (incl. all stack traces) to a log file, in case
// we need it. The stacks are sometimes helpful, but we don't want them in the
// main log as they are highly verbose.
{
cl, err := t.L().ChildLogger(
fmt.Sprintf("failure_%d", len(t.mu.errors)),
fmt.Sprintf("failure_%d", len(t.mu.failures)),
logger.QuietStderr, logger.QuietStdout,
)
if err == nil {
Expand All @@ -342,7 +347,7 @@ func (t *testImpl) addFailure(reportErr error) {
// so it's better to write only it to the file to avoid confusion.
path := cl.File.Name()
cl.Close() // we just wanted the filename
_ = os.WriteFile(path, []byte(fmt.Sprintf("%+v", reportErr)), 0644)
_ = os.WriteFile(path, []byte(fmt.Sprintf("%+v", reportFailure.squashedErr)), 0644)
}
}

Expand All @@ -364,17 +369,37 @@ func (t *testImpl) Failed() bool {
}

func (t *testImpl) failedRLocked() bool {
return len(t.mu.errors) > 0
return len(t.mu.failures) > 0
}

func (t *testImpl) firstFailure() failure {
t.mu.RLock()
defer t.mu.RUnlock()
if len(t.mu.failures) <= 0 {
return failure{}
}
return t.mu.failures[0]
}

func (t *testImpl) FailureMsg() string {
func (t *testImpl) failureMsg() string {
t.mu.RLock()
defer t.mu.RUnlock()
var b strings.Builder
formatFailure(&b, t.mu.errors...)
formatFailure(&b, t.mu.failures...)
return b.String()
}

// failureContainsError returns true if any of the errors in a given failure
// matches the reference error
func failureContainsError(refError error, f failure) bool {
for _, err := range f.errors {
if errors.Is(err, refError) {
return true
}
}
return false
}

func (t *testImpl) ArtifactsDir() string {
return t.artifactsDir
}
Expand Down
40 changes: 18 additions & 22 deletions pkg/cmd/roachtest/test_runner.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,8 +50,13 @@ import (
)

var (
errTestsFailed = fmt.Errorf("some tests failed")
errClusterProvisioningFailed = fmt.Errorf("some clusters could not be created")
errTestsFailed = fmt.Errorf("some tests failed")

// reference error used by main.go at the end of a run of tests
errSomeClusterProvisioningFailed = fmt.Errorf("some clusters could not be created")

// reference error used when cluster creation fails for a test
errClusterProvisioningFailed = fmt.Errorf("cluster could not be created")
)

// testRunner runs tests.
Expand Down Expand Up @@ -309,7 +314,7 @@ func (r *testRunner) Run(

if r.numClusterErrs > 0 {
shout(ctx, l, lopt.stdout, "%d clusters could not be created", r.numClusterErrs)
return errClusterProvisioningFailed
return errSomeClusterProvisioningFailed
}

if len(r.status.fail) > 0 {
Expand Down Expand Up @@ -571,6 +576,7 @@ func (r *testRunner) runWorker(
wStatus.SetStatus("creating cluster")
c, vmCreateOpts, clusterCreateErr = allocateCluster(ctx, testToRun.spec, testToRun.alloc, artifactsRootDir, wStatus)
if clusterCreateErr != nil {
clusterCreateErr = errors.Mark(clusterCreateErr, errClusterProvisioningFailed)
atomic.AddInt32(&r.numClusterErrs, 1)
shout(ctx, l, stdout, "Unable to create (or reuse) cluster for test %s due to: %s.",
testToRun.spec.Name, clusterCreateErr)
Expand Down Expand Up @@ -620,12 +626,12 @@ func (r *testRunner) runWorker(
// Instead, let's report an infrastructure issue, mark the test as failed and continue with the next test.

// Generate failure reason and mark the test failed to preclude fetching (cluster) artifacts.
t.addFailure(clusterCreateErr)
t.Error(clusterCreateErr)
issueOutput := "test %s was skipped due to %s"
issueOutput = fmt.Sprintf(issueOutput, t.spec.Name, t.FailureMsg())
issueOutput = fmt.Sprintf(issueOutput, t.spec.Name, t.failureMsg())

// N.B. issue title is of the form "roachtest: ${t.spec.Name} failed" (see UnitTestFormatter).
if err := github.MaybePost(t, clusterCreationErr, issueOutput); err != nil {
if err := github.MaybePost(t, issueOutput); err != nil {
shout(ctx, l, stdout, "failed to post issue: %s", err)
}
} else {
Expand Down Expand Up @@ -661,7 +667,7 @@ func (r *testRunner) runWorker(
shout(ctx, l, stdout, "test returned error: %s: %s", t.Name(), err)
// Mark the test as failed if it isn't already.
if !t.Failed() {
t.addFailure(err)
t.Error(err)
}
} else {
msg := "test passed: %s (run %d)"
Expand All @@ -677,7 +683,7 @@ func (r *testRunner) runWorker(
if err != nil {
failureMsg += fmt.Sprintf("%+v", err)
} else {
failureMsg += t.FailureMsg()
failureMsg += t.failureMsg()
}
if c != nil {
if debug {
Expand Down Expand Up @@ -798,10 +804,7 @@ func (r *testRunner) runTest(
// during the post-flight checks; the test itself runs on a different
// goroutine and has similar code to terminate errTestFatal.
if err := recover(); err != nil && err != errTestFatal {
if _, ok := err.(error); !ok {
err = errors.Newf("%v", err)
}
t.addFailure(err.(error))
t.Error(err)
}

t.mu.Lock()
Expand All @@ -810,7 +813,7 @@ func (r *testRunner) runTest(

durationStr := fmt.Sprintf("%.2fs", t.duration().Seconds())
if t.Failed() {
output := fmt.Sprintf("test artifacts and logs in: %s\n%s", t.ArtifactsDir(), t.FailureMsg())
output := fmt.Sprintf("test artifacts and logs in: %s\n%s", t.ArtifactsDir(), t.failureMsg())

if teamCity {
shout(ctx, l, stdout, "##teamcity[testFailed name='%s' details='%s' flowId='%s']",
Expand All @@ -819,14 +822,7 @@ func (r *testRunner) runTest(

shout(ctx, l, stdout, "--- FAIL: %s (%s)\n%s", runID, durationStr, output)

cat := otherErr

// This will override the created issue's owner as it is likely due to an SSH flake
if strings.Contains(output, "SSH_PROBLEM") {
cat = sshErr
}

if err := github.MaybePost(t, cat, output); err != nil {
if err := github.MaybePost(t, output); err != nil {
shout(ctx, l, stdout, "failed to post issue: %s", err)
}
} else {
Expand Down Expand Up @@ -863,7 +859,7 @@ func (r *testRunner) runTest(
start: t.start,
end: t.end,
pass: !t.Failed(),
failure: t.FailureMsg(),
failure: t.failureMsg(),
})
r.status.Lock()
delete(r.status.running, t)
Expand Down
Loading

0 comments on commit 4d32e19

Please sign in to comment.