Skip to content

Commit

Permalink
roachtest: handle panics in mixedversion
Browse files Browse the repository at this point in the history
Previously, a panic in a user function in a roachtest using the
`mixedversion` package would crash the entire roachtest process. This
is because all steps run in a separate goroutine, so if panics are not
captured, the entire process crashes.

This commit updates the test runner so that all steps (including those
that are part of the test infrastructure) run with panics captured.
The panic message is returned as a regular error which should lead to
usual GitHub error reports. The stack trace for the panic is also
logged so that we can pinpoint the exact offending line in the test.

Epic: CRDB-19321

Release note: None
  • Loading branch information
renatolabs committed Jun 27, 2023
1 parent e8c7bdc commit 8ca2fc4
Show file tree
Hide file tree
Showing 5 changed files with 104 additions and 6 deletions.
6 changes: 4 additions & 2 deletions pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -22,13 +22,15 @@ go_library(
"//pkg/util/syncutil",
"//pkg/util/timeutil",
"//pkg/util/version",
"@com_github_cockroachdb_errors//:errors",
],
)

go_test(
name = "mixedversion_test",
srcs = ["planner_test.go"],
srcs = [
"planner_test.go",
"runner_test.go",
],
args = ["-test.timeout=295s"],
embed = [":mixedversion"],
deps = [
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ var (
Stdout: io.Discard,
Stderr: io.Discard,
}
l, err := cfg.NewLogger("" /* path */)
l, err := cfg.NewLogger("/dev/null" /* path */)
if err != nil {
panic(err)
}
Expand Down
15 changes: 12 additions & 3 deletions pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"path"
"path/filepath"
"regexp"
"runtime/debug"
"strconv"
"strings"
"sync/atomic"
Expand All @@ -32,7 +33,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
)

type (
Expand Down Expand Up @@ -249,7 +249,16 @@ func (tr *testRunner) runSingleStep(ctx context.Context, ss singleStep, l *logge
prefix := fmt.Sprintf("FINISHED [%s]", timeutil.Since(start))
tr.logStep(prefix, ss, l)
}()
if err := ss.Run(ctx, l, tr.cluster, tr.newHelper(ctx, l)); err != nil {

if err := func() (retErr error) {
defer func() {
if r := recover(); r != nil {
l.Printf("panic stack trace:\n%s", string(debug.Stack()))
retErr = fmt.Errorf("panic (stack trace above): %v", r)
}
}()
return ss.Run(ctx, l, tr.cluster, tr.newHelper(ctx, l))
}(); err != nil {
if isContextCanceled(err) {
l.Printf("step terminated (context canceled)")
// Avoid creating a `stepError` (which involves querying binary
Expand Down Expand Up @@ -331,7 +340,7 @@ func (tr *testRunner) testFailure(desc string, l *logger.Logger) error {
tr.logger.Printf("could not rename failed step logger: %v", err)
}

return errors.WithStack(tf)
return tf
}

func (tr *testRunner) logStep(prefix string, step singleStep, l *logger.Logger) {
Expand Down
85 changes: 85 additions & 0 deletions pkg/cmd/roachtest/roachtestutil/mixedversion/runner_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
// Copyright 2023 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package mixedversion

import (
"context"
"fmt"
"testing"

"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/stretchr/testify/require"
)

func Test_runSingleStep(t *testing.T) {
tr := testTestRunner()

// steps that run without errors do not return errors
successStep := newTestStep(func() error {
return nil
})
err := tr.runSingleStep(ctx, successStep, nilLogger)
require.NoError(t, err)

// steps that return an error have that error surfaced
errorStep := newTestStep(func() error {
return fmt.Errorf("oops")
})
err = tr.runSingleStep(ctx, errorStep, nilLogger)
require.Error(t, err)
require.Contains(t, err.Error(), "oops")

// steps that panic cause an error to be returned
panicStep := newTestStep(func() error {
var ids []int
if ids[0] > 42 {
return nil
}
return fmt.Errorf("unreachable")
})
err = nil
require.NotPanics(t, func() {
err = tr.runSingleStep(ctx, panicStep, nilLogger)
})
require.Error(t, err)
require.Contains(t, err.Error(), "panic (stack trace above): runtime error: index out of range [0] with length 0")
}

func testTestRunner() *testRunner {
runnerCtx, cancel := context.WithCancel(ctx)
return &testRunner{
ctx: runnerCtx,
cancel: cancel,
logger: nilLogger,
crdbNodes: nodes,
background: newBackgroundRunner(runnerCtx),
seed: seed,
}
}

type testSingleStep struct {
runFunc func() error
}

func (testSingleStep) ID() int { return 42 }
func (testSingleStep) Description() string { return "testSingleStep" }
func (testSingleStep) Background() shouldStop { return nil }

func (tss testSingleStep) Run(
_ context.Context, _ *logger.Logger, _ cluster.Cluster, _ *Helper,
) error {
return tss.runFunc()
}

func newTestStep(f func() error) singleStep {
return testSingleStep{runFunc: f}
}
2 changes: 2 additions & 0 deletions pkg/testutils/lint/lint_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2306,6 +2306,8 @@ func TestLint(t *testing.T) {
// pooling, etc, then test code needs to adhere as well.
stream.GrepNot(nakedGoroutineExceptions + `:.*Use of go keyword not allowed`),
stream.GrepNot(nakedGoroutineExceptions + `:.*Illegal call to Group\.Go\(\)`),
// We purposefully dereference nil in this file to test panic handling
stream.GrepNot(`pkg/cmd/roachtest/roachtestutil/mixedversion/runner_test\.go:.*nil dereference`),
}

const vetTool = "roachvet"
Expand Down

0 comments on commit 8ca2fc4

Please sign in to comment.