diff --git a/pkg/ccl/changefeedccl/changefeedbase/errors.go b/pkg/ccl/changefeedccl/changefeedbase/errors.go index 22ab91666e03..0c1384cb8bfe 100644 --- a/pkg/ccl/changefeedccl/changefeedbase/errors.go +++ b/pkg/ccl/changefeedccl/changefeedbase/errors.go @@ -124,7 +124,9 @@ func IsRetryableError(err error) bool { // that we can't recover the structure and we have to rely on this // unfortunate string comparison. errStr := err.Error() - if strings.Contains(errStr, retryableErrorString) || strings.Contains(errStr, kvcoord.SendErrorString) { + if strings.Contains(errStr, retryableErrorString) || + strings.Contains(errStr, kvcoord.SendErrorString) || + strings.Contains(errStr, "draining") { return true } diff --git a/pkg/cmd/roachtest/BUILD.bazel b/pkg/cmd/roachtest/BUILD.bazel index 490120f4a3cf..3a35d61b2c56 100644 --- a/pkg/cmd/roachtest/BUILD.bazel +++ b/pkg/cmd/roachtest/BUILD.bazel @@ -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", diff --git a/pkg/cmd/roachtest/github.go b/pkg/cmd/roachtest/github.go index d412f79493c5..e9d019900f0a 100644 --- a/pkg/cmd/roachtest/github.go +++ b/pkg/cmd/roachtest/github.go @@ -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" ) @@ -32,6 +33,14 @@ type githubIssues struct { teamLoader func() (team.Map, error) } +type issueCategory int + +const ( + otherErr issueCategory = iota + clusterCreationErr + sshErr +) + func newGithubIssues( disable bool, c *clusterImpl, vmCreateOpts *vm.CreateOpts, l *logger.Logger, ) *githubIssues { @@ -59,16 +68,33 @@ func (g *githubIssues) shouldPost(t test.Test) bool { t.Spec().(*registry.TestSpec).Cluster.NodeCount > 0 } -func (g *githubIssues) createPostRequest(t test.Test, message string) issues.PostRequest { +func (g *githubIssues) createPostRequest( + t test.Test, cat issueCategory, message string, +) issues.PostRequest { var mention []string var projColID int + issueOwner := t.Spec().(*registry.TestSpec).Owner + issueName := t.Name() + + messagePrefix := "" + // Overrides to shield eng teams from potential flakes + if cat == clusterCreationErr { + issueOwner = registry.OwnerDevInf + issueName = "cluster_creation" + messagePrefix = fmt.Sprintf("test %s was skipped due to ", t.Name()) + } else if cat == sshErr { + issueOwner = registry.OwnerTestEng + issueName = "ssh_problem" + messagePrefix = fmt.Sprintf("test %s failed due to ", t.Name()) + } + 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 { + if sl, ok := teams.GetAliasesForPurpose(ownerToAlias(issueOwner), team.PurposeRoachtest); ok { for _, alias := range sl { mention = append(mention, "@"+string(alias)) } @@ -112,8 +138,8 @@ func (g *githubIssues) createPostRequest(t test.Test, message string) issues.Pos MentionOnCreate: mention, ProjectColumnID: projColID, PackageName: "roachtest", - TestName: t.Name(), - Message: message, + TestName: issueName, + Message: messagePrefix + message, Artifacts: artifacts, ExtraLabels: labels, ExtraParams: clusterParams, @@ -130,14 +156,24 @@ func (g *githubIssues) createPostRequest(t test.Test, message string) issues.Pos } } -func (g *githubIssues) MaybePost(t test.Test, message string) error { +func (g *githubIssues) MaybePost(t *testImpl, message string) error { if !g.shouldPost(t) { return nil } + cat := otherErr + + // Overrides to shield eng teams from potential flakes + firstFailure := t.firstFailure() + if failureContainsError(firstFailure, errClusterProvisioningFailed) { + cat = clusterCreationErr + } else if failureContainsError(firstFailure, rperrors.ErrSSH255) { + cat = sshErr + } + return g.issuePoster( context.Background(), issues.UnitTestFormatter, - g.createPostRequest(t, message), + g.createPostRequest(t, cat, message), ) } diff --git a/pkg/cmd/roachtest/github_test.go b/pkg/cmd/roachtest/github_test.go index d0188fc87ff8..b5ccb5f4d73a 100644 --- a/pkg/cmd/roachtest/github_test.go +++ b/pkg/cmd/roachtest/github_test.go @@ -29,7 +29,11 @@ var ( teamsYaml = `cockroachdb/unowned: aliases: cockroachdb/rfc-prs: other - triage_column_id: 0` + triage_column_id: 0 +cockroachdb/test-eng: + triage_column_id: 14041337 +cockroachdb/dev-inf: + triage_column_id: 10210759` validTeamsFn = func() (team.Map, error) { return loadYamlTeams(teamsYaml) } invalidTeamsFn = func() (team.Map, error) { return loadYamlTeams("invalid yaml") } @@ -57,7 +61,7 @@ func TestShouldPost(t *testing.T) { envTcBuildBranch string expected bool }{ - /* Cases 1 - 4 verify that issues are not posted if any of on the relevant criteria checks fail */ + /* Cases 1 - 4 verify that issues are not posted if any of the relevant criteria checks fail */ // disable {true, 1, "token", "master", false}, // nodeCount @@ -102,10 +106,11 @@ func TestCreatePostRequest(t *testing.T) { clusterCreationFailed bool loadTeamsFailed bool localSSD bool + category issueCategory expectedPost bool expectedParams map[string]string }{ - {true, false, false, false, true, + {true, false, false, false, otherErr, true, prefixAll(map[string]string{ "cloud": "gce", "encrypted": "false", @@ -115,7 +120,7 @@ func TestCreatePostRequest(t *testing.T) { "localSSD": "false", }), }, - {true, false, false, true, true, + {true, false, false, true, clusterCreationErr, true, prefixAll(map[string]string{ "cloud": "gce", "encrypted": "false", @@ -128,7 +133,7 @@ func TestCreatePostRequest(t *testing.T) { // 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, + {false, true, false, false, sshErr, true, prefixAll(map[string]string{ "cloud": "gce", "ssd": "0", @@ -136,7 +141,7 @@ func TestCreatePostRequest(t *testing.T) { }), }, //Simulate failure loading TEAMS.yaml - {true, false, true, false, false, nil}, + {true, false, true, false, otherErr, false, nil}, } reg, _ := makeTestRegistry(spec.GCE, "", "", false) @@ -145,7 +150,7 @@ func TestCreatePostRequest(t *testing.T) { clusterSpec := reg.MakeClusterSpec(1) testSpec := ®istry.TestSpec{ - Name: "githubPost", + Name: "github_test", Owner: OwnerUnitTest, Cluster: clusterSpec, NonReleaseBlocker: c.nonReleaseBlocker, @@ -183,9 +188,9 @@ func TestCreatePostRequest(t *testing.T) { if c.loadTeamsFailed { // Assert that if TEAMS.yaml cannot be loaded then function panics. - assert.Panics(t, func() { github.createPostRequest(ti, "message") }) + assert.Panics(t, func() { github.createPostRequest(ti, c.category, "message") }) } else { - req := github.createPostRequest(ti, "message") + req := github.createPostRequest(ti, c.category, "message") if c.expectedParams != nil { require.Equal(t, c.expectedParams, req.ExtraParams) @@ -196,6 +201,24 @@ func TestCreatePostRequest(t *testing.T) { if !c.nonReleaseBlocker { require.True(t, contains(req.ExtraLabels, nil, "release-blocker")) } + + expectedTeam := "@cockroachdb/unowned" + expectedName := "github_test" + expectedMessagePrefix := "" + + if c.category == clusterCreationErr { + expectedTeam = "@cockroachdb/dev-inf" + expectedName = "cluster_creation" + expectedMessagePrefix = "test github_test was skipped due to " + } else if c.category == sshErr { + expectedTeam = "@cockroachdb/test-eng" + expectedName = "ssh_problem" + expectedMessagePrefix = "test github_test failed due to " + } + + require.Contains(t, req.MentionOnCreate, expectedTeam) + require.Equal(t, expectedName, req.TestName) + require.True(t, strings.HasPrefix(req.Message, expectedMessagePrefix), req.Message) } } } diff --git a/pkg/cmd/roachtest/main.go b/pkg/cmd/roachtest/main.go index 5e6465dfa0be..72db558c0a03 100644 --- a/pkg/cmd/roachtest/main.go +++ b/pkg/cmd/roachtest/main.go @@ -343,7 +343,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. @@ -382,8 +382,15 @@ func runTests(register func(registry.Registry), cfg cliCfg) error { filter := registry.NewTestFilter(cfg.args) clusterType := roachprodCluster + bindTo := "" if local { clusterType = localCluster + + // This will suppress the annoying "Allow incoming network connections" popup from + // OSX when running a roachtest + bindTo = "localhost" + + fmt.Printf("--local specified. Binding http listener to localhost only") if cfg.parallelism != 1 { fmt.Printf("--local specified. Overriding --parallelism to 1.\n") cfg.parallelism = 1 @@ -398,7 +405,7 @@ func runTests(register func(registry.Registry), cfg cliCfg) error { keepClustersOnTestFailure: cfg.debugEnabled, clusterID: cfg.clusterID, } - if err := runner.runHTTPServer(cfg.httpPort, os.Stdout); err != nil { + if err := runner.runHTTPServer(cfg.httpPort, os.Stdout, bindTo); err != nil { return err } diff --git a/pkg/cmd/roachtest/test_impl.go b/pkg/cmd/roachtest/test_impl.go index 7355dbb7a084..dbc96c3fc8cb 100644 --- a/pkg/cmd/roachtest/test_impl.go +++ b/pkg/cmd/roachtest/test_impl.go @@ -39,6 +39,18 @@ type testStatus struct { progress float64 } +// Holds all error information from a single invocation of t.{Fatal,Error}{,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.{Fatal,Error}{,f} + squashedErr error + // errors are all the `errors` present in the variadic args + errors []error +} + type testImpl struct { spec *registry.TestSpec @@ -71,23 +83,16 @@ type testImpl struct { syncutil.RWMutex done bool - cancel func() - failLoc struct { - file string - line int - } - - // 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 - // If len(errors)>0, this indicates whether the test timed out // cancel, if set, is called from the t.Fatal() family of functions when the // 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() + + // 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 @@ -108,6 +113,10 @@ type testImpl struct { skipInit bool } +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 { @@ -259,21 +268,15 @@ 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 } // Fatal marks the test as failed, prints the args to t.L(), and calls @@ -285,61 +288,69 @@ 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.addFailure("", 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...)) + t.addFailure(format, args...) panic(errTestFatal) } // FailNow implements the TestingT interface. func (t *testImpl) FailNow() { - t.addFailure(errors.NewWithDepthf(1, "FailNow called")) + t.addFailure("FailNow called") panic(errTestFatal) } +// Error implements the TestingT interface func (t *testImpl) Error(args ...interface{}) { - t.addFailure(argsToErr(1, args...)) + t.addFailure("", args...) } // Errorf implements the TestingT interface. func (t *testImpl) Errorf(format string, args ...interface{}) { - t.addFailure(errors.NewWithDepthf(1, format, args...)) + t.addFailure(format, 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 = "", 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(format string, args ...interface{}) { + if format == "" { + format = strings.Repeat(" %v", len(args))[1:] + } + reportFailure := newFailure(errors.NewWithDepthf(1, format, args...), collectErrors(args)) + 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 { @@ -348,7 +359,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) } } @@ -370,17 +381,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) FailureMsg() string { +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 { 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(f failure, refError error) bool { + for _, err := range f.errors { + if errors.Is(err, refError) { + return true + } + } + return errors.Is(f.squashedErr, refError) +} + func (t *testImpl) ArtifactsDir() string { return t.artifactsDir } diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index d3dbfff22c77..424a1b90c1b5 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -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. @@ -310,7 +315,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 { @@ -572,6 +577,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) @@ -620,24 +626,13 @@ func (r *testRunner) runWorker( // N.B. cluster creation must have failed... // We don't want to prematurely abort the test suite since it's likely a transient issue. // Instead, let's report an infrastructure issue, mark the test as failed and continue with the next test. - // Note, we fake the test name so that all cluster creation errors are posted to the same github issue. - oldName := t.spec.Name - oldOwner := t.spec.Owner + // Generate failure reason and mark the test failed to preclude fetching (cluster) artifacts. - t.addFailure(clusterCreateErr) - issueOutput := "test %s was skipped due to %s" - issueOutput = fmt.Sprintf(issueOutput, oldName, t.FailureMsg()) + t.Error(clusterCreateErr) // 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 - - if err := github.MaybePost(t, issueOutput); err != nil { + if err := github.MaybePost(t, t.failureMsg()); 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 } else { c.setTest(t) err = c.PutLibraries(ctx, "./lib", t.spec.NativeLibs) @@ -671,7 +666,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)" @@ -687,7 +682,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 { @@ -808,10 +803,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() @@ -820,7 +812,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']", @@ -866,7 +858,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) @@ -1194,14 +1186,15 @@ func (r *testRunner) removeWorker(ctx context.Context, name string) { // runHTTPServer starts a server running in the background. // // httpPort: The port on which to serve the web interface. Pass 0 for allocating +// bindTo: The host/ip on which to bind. Leave empty to bind on all local ips // // a port automatically (which will be printed to stdout). -func (r *testRunner) runHTTPServer(httpPort int, stdout io.Writer) error { +func (r *testRunner) runHTTPServer(httpPort int, stdout io.Writer, bindTo string) error { http.HandleFunc("/", r.serveHTTP) // Run an http server in the background. // We handle the case where httpPort is 0, which means we automatically // allocate a port. - listener, err := net.Listen("tcp", fmt.Sprintf(":%d", httpPort)) + listener, err := net.Listen("tcp", fmt.Sprintf("%s:%d", bindTo, httpPort)) if err != nil { return err } @@ -1211,7 +1204,11 @@ func (r *testRunner) runHTTPServer(httpPort int, stdout io.Writer) error { panic(err) } }() - fmt.Fprintf(stdout, "HTTP server listening on all network interfaces, port %d.\n", httpPort) + bindToDesc := "all network interfaces" + if bindTo != "" { + bindToDesc = bindTo + } + fmt.Fprintf(stdout, "HTTP server listening on %s, port %d.\n", bindToDesc, httpPort) return nil } diff --git a/pkg/roachprod/errors/errors.go b/pkg/roachprod/errors/errors.go index 544e460bbcdb..f0fab50e6396 100644 --- a/pkg/roachprod/errors/errors.go +++ b/pkg/roachprod/errors/errors.go @@ -33,6 +33,10 @@ const ( unclassifiedExitCode = 1 ) +// ErrSSH255 is a reference error used to mark an SSH error with an exit +// code of 255. This could be indicative of an SSH flake. +var ErrSSH255 = errors.New("SSH error occurred with exit code 255") + // Cmd wraps errors that result from a command run against the cluster. type Cmd struct { Err error @@ -116,7 +120,7 @@ func ClassifyCmdError(err error) Error { if exitErr, ok := asExitError(err); ok { if exitErr.ExitCode() == 255 { - return SSH{err} + return SSH{errors.Mark(err, ErrSSH255)} } return Cmd{err} } diff --git a/pkg/roachprod/install/cluster_synced.go b/pkg/roachprod/install/cluster_synced.go index c65005397392..ce8473598ddb 100644 --- a/pkg/roachprod/install/cluster_synced.go +++ b/pkg/roachprod/install/cluster_synced.go @@ -23,7 +23,6 @@ import ( "os/exec" "os/signal" "path/filepath" - "reflect" "sort" "strings" "sync" @@ -642,7 +641,7 @@ func runCmdOnSingleNode( detailMsg := fmt.Sprintf("Node %d. Command with error:\n```\n%s\n```\n", node, cmd) err = errors.WithDetail(err, detailMsg) err = rperrors.ClassifyCmdError(err) - if reflect.TypeOf(err) == reflect.TypeOf(rperrors.SSH{}) { + if errors.Is(err, rperrors.ErrSSH255) { result.RemoteExitStatus = "255" } result.Err = err diff --git a/pkg/testutils/lint/passes/fmtsafe/fmtsafe.go b/pkg/testutils/lint/passes/fmtsafe/fmtsafe.go index f7f4029f3695..59543a5979cd 100644 --- a/pkg/testutils/lint/passes/fmtsafe/fmtsafe.go +++ b/pkg/testutils/lint/passes/fmtsafe/fmtsafe.go @@ -245,8 +245,8 @@ func checkCallExpr(pass *analysis.Pass, enclosingFnName string, call *ast.CallEx return } - pass.Reportf(call.Lparen, escNl("%s(): %s argument is not a constant expression"+Tip), - enclosingFnName, argType) + pass.Reportf(call.Lparen, escNl("%s() [calling %s]: %s argument is not a constant expression"+Tip), + enclosingFnName, fnName, argType) } // Tip is exported for use in tests. diff --git a/pkg/testutils/lint/passes/fmtsafe/functions.go b/pkg/testutils/lint/passes/fmtsafe/functions.go index 5bd4fe3ff1b0..d5a2ffa82b73 100644 --- a/pkg/testutils/lint/passes/fmtsafe/functions.go +++ b/pkg/testutils/lint/passes/fmtsafe/functions.go @@ -92,13 +92,16 @@ var requireConstFmt = map[string]bool{ "(*github.com/cockroachdb/cockroach/pkg/util/grpcutil.grpcLogger).Fatalf": true, // Both of these signatures need to be included for the linter to not flag - // roachtest testImpl.Errorf since it is in the main package - "(*main.testImpl).Errorf": true, - "(*github.com/cockroachdb/cockroach/pkg/cmd/roachtest.testImpl).Errorf": true, + // roachtest testImpl.addFailure since it is in the main package + "(*github.com/cockroachdb/cockroach/pkg/cmd/roachtest.testImpl).addFailure": true, + "(*main.testImpl).addFailure": true, "(*main.testImpl).Fatalf": true, "(*github.com/cockroachdb/cockroach/pkg/cmd/roachtest.testImpl).Fatalf": true, + "(*main.testImpl).Errorf": true, + "(*github.com/cockroachdb/cockroach/pkg/cmd/roachtest.testImpl).Errorf": true, + "(*github.com/cockroachdb/cockroach/pkg/kv/kvserver.raftLogger).Debugf": true, "(*github.com/cockroachdb/cockroach/pkg/kv/kvserver.raftLogger).Infof": true, "(*github.com/cockroachdb/cockroach/pkg/kv/kvserver.raftLogger).Warningf": true,