diff --git a/pkg/cmd/roachtest/BUILD.bazel b/pkg/cmd/roachtest/BUILD.bazel index 84ebef5a0a29..c5e807c555ae 100644 --- a/pkg/cmd/roachtest/BUILD.bazel +++ b/pkg/cmd/roachtest/BUILD.bazel @@ -86,6 +86,7 @@ go_test( "//pkg/cmd/roachtest/spec", "//pkg/cmd/roachtest/test", "//pkg/internal/team", + "//pkg/roachprod/errors", "//pkg/roachprod/logger", "//pkg/roachprod/vm", "//pkg/testutils", diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index c3730b83429a..07ff5c2d8bc4 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -1608,33 +1608,31 @@ func (c *clusterImpl) HealthStatus( return results, nil } -// FailOnInvalidDescriptors fails the test if there exists any descriptors in +// assertValidDescriptors fails the test if there exists any descriptors in // the crdb_internal.invalid_objects virtual table. -func (c *clusterImpl) FailOnInvalidDescriptors(ctx context.Context, db *gosql.DB, t *testImpl) { +func (c *clusterImpl) assertValidDescriptors(ctx context.Context, db *gosql.DB, t *testImpl) error { t.L().Printf("checking for invalid descriptors") - if err := timeutil.RunWithTimeout( + return timeutil.RunWithTimeout( ctx, "invalid descriptors check", 1*time.Minute, func(ctx context.Context) error { return roachtestutil.CheckInvalidDescriptors(ctx, db) }, - ); err != nil { - t.Errorf("invalid descriptors check failed: %v", err) - } + ) } -// FailOnReplicaDivergence fails the test if +// assertConsistentReplicas fails the test if // crdb_internal.check_consistency(true, ”, ”) indicates that any ranges' // replicas are inconsistent with each other. -func (c *clusterImpl) FailOnReplicaDivergence(ctx context.Context, db *gosql.DB, t *testImpl) { +func (c *clusterImpl) assertConsistentReplicas( + ctx context.Context, db *gosql.DB, t *testImpl, +) error { t.L().Printf("checking for replica divergence") - if err := timeutil.RunWithTimeout( + return timeutil.RunWithTimeout( ctx, "consistency check", 5*time.Minute, func(ctx context.Context) error { return roachtestutil.CheckReplicaDivergenceOnDB(ctx, t.L(), db) }, - ); err != nil { - t.Errorf("consistency check failed: %v", err) - } + ) } // FetchDmesg grabs the dmesg logs if possible. This requires being able to run diff --git a/pkg/cmd/roachtest/github.go b/pkg/cmd/roachtest/github.go index cb1f96d573a8..c50713e2844e 100644 --- a/pkg/cmd/roachtest/github.go +++ b/pkg/cmd/roachtest/github.go @@ -32,14 +32,6 @@ 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) *githubIssues { return &githubIssues{ disable: disable, @@ -106,7 +98,7 @@ func (g *githubIssues) shouldPost(t test.Test) (bool, string) { } func (g *githubIssues) createPostRequest( - t test.Test, cat issueCategory, message string, + t test.Test, firstFailure failure, message string, ) issues.PostRequest { var mention []string var projColID int @@ -118,21 +110,23 @@ func (g *githubIssues) createPostRequest( messagePrefix := "" var infraFlake bool // Overrides to shield eng teams from potential flakes - if cat == clusterCreationErr { + switch { + case failureContainsError(firstFailure, errClusterProvisioningFailed): issueOwner = registry.OwnerDevInf issueName = "cluster_creation" messagePrefix = fmt.Sprintf("test %s was skipped due to ", t.Name()) infraFlake = true - } else if cat == sshErr { + case failureContainsError(firstFailure, rperrors.ErrSSH255): issueOwner = registry.OwnerTestEng issueName = "ssh_problem" messagePrefix = fmt.Sprintf("test %s failed due to ", t.Name()) infraFlake = true + case failureContainsError(firstFailure, errDuringPostAssertions): + messagePrefix = fmt.Sprintf("test %s failed during post test assertions (see test-post-assertions.log) due to ", 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). + // 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). labels := []string{"O-roachtest"} if !spec.NonReleaseBlocker && !infraFlake { labels = append(labels, "release-blocker") @@ -218,20 +212,10 @@ func (g *githubIssues) MaybePost(t *testImpl, l *logger.Logger, message string) 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(), l, issues.UnitTestFormatter, - g.createPostRequest(t, cat, message), + g.createPostRequest(t, t.firstFailure(), message), ) } diff --git a/pkg/cmd/roachtest/github_test.go b/pkg/cmd/roachtest/github_test.go index 823d9adc04e8..366431a8ee4b 100644 --- a/pkg/cmd/roachtest/github_test.go +++ b/pkg/cmd/roachtest/github_test.go @@ -12,6 +12,8 @@ package main import ( "context" + "errors" + "fmt" "strings" "testing" @@ -21,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" "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/vm" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -101,18 +104,22 @@ func TestShouldPost(t *testing.T) { } func TestCreatePostRequest(t *testing.T) { + createFailure := func(ref error) failure { + return failure{squashedErr: ref} + } + testCases := []struct { nonReleaseBlocker bool clusterCreationFailed bool loadTeamsFailed bool localSSD bool arch vm.CPUArch - category issueCategory + failure failure expectedPost bool expectedReleaseBlocker bool expectedParams map[string]string }{ - {true, false, false, false, "", otherErr, true, false, + {true, false, false, false, "", createFailure(errors.New("other")), true, false, prefixAll(map[string]string{ "cloud": "gce", "encrypted": "false", @@ -123,7 +130,7 @@ func TestCreatePostRequest(t *testing.T) { "localSSD": "false", }), }, - {true, false, false, true, vm.ArchARM64, clusterCreationErr, true, false, + {true, false, false, true, vm.ArchARM64, createFailure(errClusterProvisioningFailed), true, false, prefixAll(map[string]string{ "cloud": "gce", "encrypted": "false", @@ -138,7 +145,7 @@ func TestCreatePostRequest(t *testing.T) { // !nonReleaseBlocker and issue is an SSH flake. Also ensure that // in the event of a failed cluster creation, nil `vmOptions` and // `clusterImpl` are not dereferenced - {false, true, false, false, "", sshErr, true, false, + {false, true, false, false, "", createFailure(rperrors.ErrSSH255), true, false, prefixAll(map[string]string{ "cloud": "gce", "ssd": "0", @@ -146,84 +153,91 @@ func TestCreatePostRequest(t *testing.T) { }), }, //Simulate failure loading TEAMS.yaml - {true, false, true, false, "", otherErr, false, false, nil}, + {true, false, true, false, "", createFailure(errors.New("other")), false, false, nil}, + //Error during post test assertions + {true, false, false, false, "", createFailure(errDuringPostAssertions), false, false, nil}, } reg := makeTestRegistry(spec.GCE, "", "", false, false) - for _, c := range testCases { - clusterSpec := reg.MakeClusterSpec(1, spec.Arch(c.arch)) - - testSpec := ®istry.TestSpec{ - Name: "github_test", - Owner: OwnerUnitTest, - Cluster: clusterSpec, - NonReleaseBlocker: c.nonReleaseBlocker, - } - - ti := &testImpl{ - spec: testSpec, - l: nilLogger(), - } - - testClusterImpl := &clusterImpl{spec: clusterSpec, arch: vm.ArchAMD64} - vo := vm.DefaultCreateOpts() - vmOpts := &vo - - if c.clusterCreationFailed { - testClusterImpl = nil - vmOpts = nil - } else if !c.localSSD { - // The default is true set in `vm.DefaultCreateOpts` - vmOpts.SSDOpts.UseLocalSSD = false - } + for idx, c := range testCases { + t.Run(fmt.Sprintf("%d", idx+1), func(t *testing.T) { + clusterSpec := reg.MakeClusterSpec(1, spec.Arch(c.arch)) + + testSpec := ®istry.TestSpec{ + Name: "github_test", + Owner: OwnerUnitTest, + Cluster: clusterSpec, + NonReleaseBlocker: c.nonReleaseBlocker, + } - teamLoadFn := validTeamsFn + ti := &testImpl{ + spec: testSpec, + l: nilLogger(), + } - if c.loadTeamsFailed { - teamLoadFn = invalidTeamsFn - } + testClusterImpl := &clusterImpl{spec: clusterSpec, arch: vm.ArchAMD64} + vo := vm.DefaultCreateOpts() + vmOpts := &vo - github := &githubIssues{ - vmCreateOpts: vmOpts, - cluster: testClusterImpl, - teamLoader: teamLoadFn, - } + if c.clusterCreationFailed { + testClusterImpl = nil + vmOpts = nil + } else if !c.localSSD { + // The default is true set in `vm.DefaultCreateOpts` + vmOpts.SSDOpts.UseLocalSSD = false + } - if c.loadTeamsFailed { - // Assert that if TEAMS.yaml cannot be loaded then function panics. - assert.Panics(t, func() { github.createPostRequest(ti, c.category, "message") }) - } else { - req := github.createPostRequest(ti, c.category, "message") + teamLoadFn := validTeamsFn - if c.expectedParams != nil { - require.Equal(t, c.expectedParams, req.ExtraParams) + if c.loadTeamsFailed { + teamLoadFn = invalidTeamsFn } - require.True(t, contains(req.ExtraLabels, nil, "O-roachtest")) - require.Equal(t, c.expectedReleaseBlocker, contains(req.ExtraLabels, nil, "release-blocker")) - - expectedTeam := "@cockroachdb/unowned" - expectedName := "github_test" - expectedLabel := "" - 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" - expectedLabel = "T-testeng" - expectedName = "ssh_problem" - expectedMessagePrefix = "test github_test failed due to " + github := &githubIssues{ + vmCreateOpts: vmOpts, + cluster: testClusterImpl, + teamLoader: teamLoadFn, } - require.Contains(t, req.MentionOnCreate, expectedTeam) - require.Equal(t, expectedName, req.TestName) - require.True(t, strings.HasPrefix(req.Message, expectedMessagePrefix), req.Message) - if expectedLabel != "" { - require.Contains(t, req.ExtraLabels, expectedLabel) + if c.loadTeamsFailed { + // Assert that if TEAMS.yaml cannot be loaded then function panics. + assert.Panics(t, func() { github.createPostRequest(ti, c.failure, "message") }) + } else { + req := github.createPostRequest(ti, c.failure, "message") + + if c.expectedParams != nil { + require.Equal(t, c.expectedParams, req.ExtraParams) + } + + require.True(t, contains(req.ExtraLabels, nil, "O-roachtest")) + require.Equal(t, c.expectedReleaseBlocker, contains(req.ExtraLabels, nil, "release-blocker")) + + expectedTeam := "@cockroachdb/unowned" + expectedName := "github_test" + expectedLabel := "" + expectedMessagePrefix := "" + + if errors.Is(c.failure.squashedErr, errClusterProvisioningFailed) { + expectedTeam = "@cockroachdb/dev-inf" + expectedName = "cluster_creation" + expectedMessagePrefix = "test github_test was skipped due to " + } else if errors.Is(c.failure.squashedErr, rperrors.ErrSSH255) { + expectedTeam = "@cockroachdb/test-eng" + expectedLabel = "T-testeng" + expectedName = "ssh_problem" + expectedMessagePrefix = "test github_test failed due to " + } else if errors.Is(c.failure.squashedErr, errDuringPostAssertions) { + expectedMessagePrefix = "test github_test failed during post test assertions (see test-post-assertions.log) due to " + } + + require.Contains(t, req.MentionOnCreate, expectedTeam) + require.Equal(t, expectedName, req.TestName) + require.True(t, strings.HasPrefix(req.Message, expectedMessagePrefix), req.Message) + if expectedLabel != "" { + require.Contains(t, req.ExtraLabels, expectedLabel) + } } - } + }) + } } diff --git a/pkg/cmd/roachtest/test_impl.go b/pkg/cmd/roachtest/test_impl.go index adcb5f2e3d9f..3e5a070b9ffa 100644 --- a/pkg/cmd/roachtest/test_impl.go +++ b/pkg/cmd/roachtest/test_impl.go @@ -370,8 +370,7 @@ func (t *testImpl) addFailure(depth int, format string, args ...interface{}) { t.mu.output = append(t.mu.output, '\n') } -// We take the first error from each failure which is the -// "squashed" error that contains all information of a failure +// We take the "squashed" error that contains information of all the errors for each failure. func formatFailure(b *strings.Builder, reportFailures ...failure) { for i, failure := range reportFailures { if i > 0 { @@ -402,7 +401,7 @@ func (t *testImpl) failedRLocked() bool { func (t *testImpl) firstFailure() failure { t.mu.RLock() defer t.mu.RUnlock() - if len(t.mu.failures) <= 0 { + if len(t.mu.failures) == 0 { return failure{} } return t.mu.failures[0] diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index 125620991df7..800d05b045b5 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -61,6 +61,9 @@ var ( // reference error used when cluster creation fails for a test errClusterProvisioningFailed = fmt.Errorf("cluster could not be created") + // reference error for any failures during post test assertions + errDuringPostAssertions = fmt.Errorf("error during post test assertions") + prometheusNameSpace = "roachtest" // prometheusScrapeInterval should be consistent with the scrape interval defined in // https://grafana.testeng.crdb.io/prometheus/config @@ -878,9 +881,7 @@ fi'` // this returns. This happens when the test doesn't respond to cancellation. // // Args: -// c: The cluster on which the test will run. runTest() does not wipe or destroy -// -// the cluster. +// c: The cluster on which the test will run. runTest() does not wipe or destroy the cluster. func (r *testRunner) runTest( ctx context.Context, t *testImpl, @@ -941,7 +942,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("%s\ntest artifacts and logs in: %s", t.failureMsg(), t.ArtifactsDir()) if teamCity { // If `##teamcity[testFailed ...]` is not present before `##teamCity[testFinished ...]`, @@ -1061,11 +1062,11 @@ func (r *testRunner) runTest( select { case <-testReturnedCh: - s := "success" + s := "successfully" if t.Failed() { - s = "failure" + s = "with failure(s)" } - t.L().Printf("tearing down after %s; see teardown.log", s) + t.L().Printf("test completed %s", s) case <-time.After(timeout): // NB: We're adding the timeout failure intentionally without cancelling the context // to capture as much state as possible during artifact collection. @@ -1073,77 +1074,55 @@ func (r *testRunner) runTest( timedOut = true } - // From now on, all logging goes to teardown.log to give a clear - // separation between operations originating from the test vs the - // harness. - teardownL, err := c.l.ChildLogger("teardown", logger.QuietStderr, logger.QuietStdout) - if err != nil { - return err + // Replacing the logger is best effort. + replaceLogger := func(name string) { + logger, err := c.l.ChildLogger(name, logger.QuietStderr, logger.QuietStdout) + if err != nil { + l.Printf("unable to create logger %s: %s", name, err) + return + } + c.l = logger + t.ReplaceL(logger) } - l, c.l = teardownL, teardownL - t.ReplaceL(teardownL) - return r.teardownTest(ctx, t, c, timedOut) -} + // Awkward file name to keep it close to test.log. + l.Printf("running post test assertions (test-post-assertions.log)") + replaceLogger("test-post-assertions") -func (r *testRunner) teardownTest( - ctx context.Context, t *testImpl, c *clusterImpl, timedOut bool, -) error { - - // We still have to collect artifacts and run post-flight checks, and any of - // these might hang. So they go into a goroutine and the main goroutine - // abandons them after a timeout. We intentionally don't wait for the - // goroutines to return, as this too may hang if something doesn't respond to - // ctx cancellation. + // We still want to run the post-test assertions even if the test timed out as it + // might provide useful information about the health of the nodes. Any assertion failures + // will will be recorded against, and eventually fail, the test. + // TODO (miral): consider not running these assertions if the test has already failed + if err := r.postTestAssertions(ctx, t, c, 10*time.Minute); err != nil { + l.Printf("error during post test assertions: %v; see test-post-assertions.log for details", err) + } - artifactsCollectedCh := make(chan struct{}) - _ = r.stopper.RunAsyncTask(ctx, "collect-artifacts", func(ctx context.Context) { - // TODO(tbg): make `t` and `logger` resilient to use-after-Close to avoid - // crashes here in cases where the goroutine leaks but later gets unstuck - // and tries to log something. - defer close(artifactsCollectedCh) - if timedOut { - // Timeouts are often opaque. Improve our changes by dumping the stack - // so that at least we can piece together what the test is trying to - // do at this very moment. - // - // We're careful here to not fail the test, i.e. we don't call t.Error - // here. We want to preserve as much state as possible in the artifacts, - // and calling t.{Error,Fatal}{,f} cancels the test's main context. - // - // We make sure to fail the test later when handling the timedOut variable. - const stacksFile = "__stacks" - if cl, err := t.L().ChildLogger(stacksFile, logger.QuietStderr, logger.QuietStdout); err == nil { - sl := allstacks.Get() - if c.Spec().NodeCount == 0 { - sl = []byte("") // keep test outputs clutter-free - } - cl.PrintfCtx(ctx, "all stacks:\n\n%s\n", sl) - t.L().PrintfCtx(ctx, "dumped stacks to %s", stacksFile) - } + // From now on, all logging goes to test-teardown.log to give a clear separation between + // operations originating from the test vs the harness. The only error that can originate here + // is from artifact collection, which is best effort and for which we do not fail the test. + l.Printf("running test teardown (test-teardown.log)") + replaceLogger("test-teardown") + if err := r.teardownTest(ctx, t, c, timedOut); err != nil { + l.Printf("error during test teardown: %v; see test-teardown.log for details", err) + } + return nil +} - // Send SIGQUIT to ask all processes to dump stacks if requested (without shutting down). - // We need to do this before collectClusterArtifacts below, which will download the logs. - // Note that the debug.zip will hopefully also contain stacks, but we're just making sure - // there's something even if the debug.zip doesn't go through. - args := option.DefaultStopOpts() - if c.Spec().GatherCores { - // Need to use ABRT to get cores. - args.RoachprodOpts.Sig = 6 - } else { - args.RoachprodOpts.Sig = 3 - } - err := c.StopE(ctx, t.L(), args, c.All()) - t.L().PrintfCtx(ctx, "asked CRDB nodes to dump stacks; check their main (DEV) logs: %v", err) - // It takes a little moment for the stacks to get flushed to the logs. - // Against a real cluster they'll typically be there by the time we fetch - // logs but on local clusters this may not be true; either way better to - // not take any chances. - if c.Spec().NodeCount > 0 { // unit tests - time.Sleep(3 * time.Second) - } - } +// The assertions here are executed after each test, and may result in a test failure. Test authors +// may opt out of these assertions by setting the relevant `SkipPostValidations` flag in the test spec. +// An error caused by a timeout will not result in a failure. +func (r *testRunner) postTestAssertions( + ctx context.Context, t *testImpl, c *clusterImpl, timeout time.Duration, +) error { + assertionFailed := false + postAssertionErr := func(err error) { + assertionFailed = true + t.Error(errors.Mark(err, errDuringPostAssertions)) + } + postAssertCh := make(chan struct{}) + _ = r.stopper.RunAsyncTask(ctx, "test-post-assertions", func(ctx context.Context) { + defer close(postAssertCh) // When a dead node is detected, the subsequent post validation queries are likely // to hang (reason unclear), and eventually timeout according to the statement_timeout. // If this occurs frequently enough, we can look at skipping post validations on a node @@ -1151,7 +1130,7 @@ func (r *testRunner) teardownTest( if err := c.assertNoDeadNode(ctx, t); err != nil { // Some tests expect dead nodes, so they may opt out of this check. if t.spec.SkipPostValidations®istry.PostValidationNoDeadNodes == 0 { - t.Error(err) + postAssertionErr(err) } else { t.L().Printf("dead node(s) detected but expected") } @@ -1161,7 +1140,7 @@ func (r *testRunner) teardownTest( // and select the first one that succeeds to run the validation queries statuses, err := c.HealthStatus(ctx, t.L(), c.All()) if err != nil { - t.Error(errors.WithDetail(err, "Unable to check health status")) + postAssertionErr(errors.WithDetail(err, "Unable to check health status")) } var db *gosql.DB @@ -1198,31 +1177,45 @@ func (r *testRunner) teardownTest( // If this validation fails due to a timeout, it is very likely that // the replica divergence check below will also fail. if t.spec.SkipPostValidations®istry.PostValidationInvalidDescriptors == 0 { - c.FailOnInvalidDescriptors(ctx, db, t) + if err := c.assertValidDescriptors(ctx, db, t); err != nil { + postAssertionErr(errors.WithDetail(err, "invalid descriptors check failed")) + } } // Detect replica divergence (i.e. ranges in which replicas have arrived // at the same log position with different states). if t.spec.SkipPostValidations®istry.PostValidationReplicaDivergence == 0 { - c.FailOnReplicaDivergence(ctx, db, t) + if err := c.assertConsistentReplicas(ctx, db, t); err != nil { + postAssertionErr(errors.WithDetail(err, "consistency check failed")) + } } } else { t.L().Printf("no live node found, skipping validation checks") } - - if timedOut || t.Failed() { - r.collectClusterArtifacts(ctx, c, t.L()) - } }) - const artifactsCollectionTimeout = time.Hour select { - case <-artifactsCollectedCh: - case <-time.After(artifactsCollectionTimeout): - // Leak the artifacts collection goroutine. Note that the test may not be - // marked as failing here. We intentionally do not trigger it to fail here, - // but we could entertain doing so once we have a mechanism that can route - // such post-test problems to the test-eng team. - t.L().Printf("giving up on artifacts collection after %s", artifactsCollectionTimeout) + case <-postAssertCh: + case <-time.After(timeout): + return errors.Errorf("post test assertions timed out after %s", timeout) + } + + if assertionFailed { + return errors.New("post test assertion(s) failed") + } + return nil +} + +// teardownTest is best effort and should not fail a test. +// Errors during artifact collection will be propagated up. +func (r *testRunner) teardownTest( + ctx context.Context, t *testImpl, c *clusterImpl, timedOut bool, +) error { + var err error + if timedOut || t.Failed() { + err = r.collectArtifacts(ctx, t, c, timedOut, time.Hour) + if err != nil { + t.L().Printf("error collecting artifacts: %v", err) + } } if timedOut { @@ -1237,45 +1230,108 @@ func (r *testRunner) teardownTest( } t.L().Printf("test timed out; check __stacks.log and CRDB logs for goroutine dumps") } - return nil + + return err } -func (r *testRunner) collectClusterArtifacts( - ctx context.Context, c *clusterImpl, l *logger.Logger, -) { - // NB: fetch the logs even when we have a debug zip because - // debug zip can't ever get the logs for down nodes. - // We only save artifacts for failed tests in CI, so this - // duplication is acceptable. - // NB: fetch the logs *first* in case one of the other steps - // below has problems. - l.PrintfCtx(ctx, "collecting cluster logs") - // Do this before collecting logs to make sure the file gets - // downloaded below. - if err := saveDiskUsageToLogsDir(ctx, c); err != nil { - l.Printf("failed to fetch disk uage summary: %s", err) - } - if err := c.FetchLogs(ctx, l); err != nil { - l.Printf("failed to download logs: %s", err) - } - if err := c.FetchDmesg(ctx, l); err != nil { - l.Printf("failed to fetch dmesg: %s", err) - } - if err := c.FetchJournalctl(ctx, l); err != nil { - l.Printf("failed to fetch journalctl: %s", err) - } - if err := c.FetchCores(ctx, l); err != nil { - l.Printf("failed to fetch cores: %s", err) - } - if err := c.CopyRoachprodState(ctx); err != nil { - l.Printf("failed to copy roachprod state: %s", err) - } - if err := c.FetchTimeseriesData(ctx, l); err != nil { - l.Printf("failed to fetch timeseries data: %s", err) - } - if err := c.FetchDebugZip(ctx, l); err != nil { - l.Printf("failed to collect zip: %s", err) +func (r *testRunner) collectArtifacts( + ctx context.Context, t *testImpl, c *clusterImpl, timedOut bool, timeout time.Duration, +) error { + // Collecting artifacts may hang so we run it in a goroutine which is abandoned + // after a timeout. + artifactsCollectedCh := make(chan struct{}) + _ = r.stopper.RunAsyncTask(ctx, "collect-artifacts", func(ctx context.Context) { + // TODO(tbg): make `t` and `logger` resilient to use-after-Close to avoid + // crashes here in cases where the goroutine leaks but later gets unstuck + // and tries to log something. + defer close(artifactsCollectedCh) + if timedOut { + // Timeouts are often opaque. Improve our changes by dumping the stack + // so that at least we can piece together what the test is trying to + // do at this very moment. + // + // We're careful here to not fail the test, i.e. we don't call t.Error + // here. We want to preserve as much state as possible in the artifacts, + // and calling t.{Error,Fatal}{,f} cancels the test's main context. + // + // We make sure to fail the test later when handling the timedOut variable. + const stacksFile = "__stacks" + if cl, err := t.L().ChildLogger(stacksFile, logger.QuietStderr, logger.QuietStdout); err == nil { + sl := allstacks.Get() + if c.Spec().NodeCount == 0 { + sl = []byte("") // keep test outputs clutter-free + } + cl.PrintfCtx(ctx, "all stacks:\n\n%s\n", sl) + t.L().PrintfCtx(ctx, "dumped stacks to %s", stacksFile) + } + + // Send SIGQUIT to ask all processes to dump stacks if requested (without shutting down). + // We need to do this before collectClusterArtifacts below, which will download the logs. + // Note that the debug.zip will hopefully also contain stacks, but we're just making sure + // there's something even if the debug.zip doesn't go through. + args := option.DefaultStopOpts() + if c.Spec().GatherCores { + // Need to use ABRT to get cores. + args.RoachprodOpts.Sig = 6 + } else { + args.RoachprodOpts.Sig = 3 + } + err := c.StopE(ctx, t.L(), args, c.All()) + t.L().PrintfCtx(ctx, "asked CRDB nodes to dump stacks; check their main (DEV) logs: %v", err) + // It takes a little moment for the stacks to get flushed to the logs. + // Against a real cluster they'll typically be there by the time we fetch + // logs but on local clusters this may not be true; either way better to + // not take any chances. + if c.Spec().NodeCount > 0 { // unit tests + time.Sleep(3 * time.Second) + } + } + + // NB: fetch the logs even when we have a debug zip because + // debug zip can't ever get the logs for down nodes. + // We only save artifacts for failed tests in CI, so this + // duplication is acceptable. + // NB: fetch the logs *first* in case one of the other steps + // below has problems. + t.L().PrintfCtx(ctx, "collecting cluster logs") + // Do this before collecting logs to make sure the file gets + // downloaded below. + if err := saveDiskUsageToLogsDir(ctx, c); err != nil { + t.L().Printf("failed to fetch disk uage summary: %s", err) + } + if err := c.FetchLogs(ctx, t.L()); err != nil { + t.L().Printf("failed to download logs: %s", err) + } + if err := c.FetchDmesg(ctx, t.L()); err != nil { + t.L().Printf("failed to fetch dmesg: %s", err) + } + if err := c.FetchJournalctl(ctx, t.L()); err != nil { + t.L().Printf("failed to fetch journalctl: %s", err) + } + if err := c.FetchCores(ctx, t.L()); err != nil { + t.L().Printf("failed to fetch cores: %s", err) + } + if err := c.CopyRoachprodState(ctx); err != nil { + t.L().Printf("failed to copy roachprod state: %s", err) + } + if err := c.FetchTimeseriesData(ctx, t.L()); err != nil { + t.L().Printf("failed to fetch timeseries data: %s", err) + } + if err := c.FetchDebugZip(ctx, t.L()); err != nil { + t.L().Printf("failed to collect zip: %s", err) + } + }) + + select { + case <-artifactsCollectedCh: + case <-time.After(timeout): + // Leak the artifacts collection goroutine. Note that the test may not be + // marked as failing here. We intentionally do not trigger it to fail here, + // but we could entertain doing so once we have a mechanism that can route + // such post-test problems to the test-eng team. + return errors.Errorf("artifact collection timed out after %s", timeout) } + return nil } func callerName() string {