diff --git a/pkg/sql/colexecbase/colexecerror/BUILD.bazel b/pkg/sql/colexecbase/colexecerror/BUILD.bazel index 926b7ba5ffde..674b17e7fd5a 100644 --- a/pkg/sql/colexecbase/colexecerror/BUILD.bazel +++ b/pkg/sql/colexecbase/colexecerror/BUILD.bazel @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "colexecerror", @@ -12,3 +12,14 @@ go_library( "@com_github_gogo_protobuf//proto", ], ) + +go_test( + name = "colexecerror_test", + srcs = ["error_test.go"], + deps = [ + ":colexecerror", + "//pkg/util/leaktest", + "//pkg/util/log", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/sql/colexecbase/colexecerror/error.go b/pkg/sql/colexecbase/colexecerror/error.go index d37d2fcdcf5b..ce059e243c8b 100644 --- a/pkg/sql/colexecbase/colexecerror/error.go +++ b/pkg/sql/colexecbase/colexecerror/error.go @@ -54,9 +54,7 @@ func CatchVectorizedRuntimeError(operation func()) (retErr error) { panic(errors.AssertionFailedf("unexpectedly there is no line below the panic line in the stack trace\n%s", stackTrace)) } panicEmittedFrom := strings.TrimSpace(scanner.Text()) - if !isPanicFromVectorizedEngine(panicEmittedFrom) { - // Do not recover from the panic not related to the vectorized - // engine. + if !shouldCatchPanic(panicEmittedFrom) { panic(panicObj) } @@ -97,27 +95,37 @@ func CatchVectorizedRuntimeError(operation func()) (retErr error) { const ( colPackagesPrefix = "github.com/cockroachdb/cockroach/pkg/col" execinfraPackagePrefix = "github.com/cockroachdb/cockroach/pkg/sql/execinfra" - rowexecPackagePrefix = "github.com/cockroachdb/cockroach/pkg/sql/rowexec" sqlColPackagesPrefix = "github.com/cockroachdb/cockroach/pkg/sql/col" + sqlRowPackagesPrefix = "github.com/cockroachdb/cockroach/pkg/sql/row" treePackagePrefix = "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" ) -// isPanicFromVectorizedEngine checks whether the panic that was emitted from -// panicEmittedFrom line of code (which includes package name as well as the -// file name and the line number) came from the vectorized engine. +// shouldCatchPanic checks whether the panic that was emitted from +// panicEmittedFrom line of code (which contains the full path to the function +// where the panic originated) should be caught by the vectorized engine. +// +// The vectorized engine uses the panic-catch mechanism of error propagation, so +// we need to catch all of its errors. We also want to catch any panics that +// occur because of internal errors in some execution component (e.g. builtins). +// // panicEmittedFrom must be trimmed to not have any white spaces in the prefix. -func isPanicFromVectorizedEngine(panicEmittedFrom string) bool { - const testExceptionPrefix = "github.com/cockroachdb/cockroach/pkg/sql/colflow_test.(*testNonVectorizedPanicEmitter)" - if strings.HasPrefix(panicEmittedFrom, testExceptionPrefix) { - // Although the panic appears to be coming from the vectorized engine, it - // is intended to not be caught in order to test the panic propagation, so - // we say that the panic is not from the vectorized engine. +func shouldCatchPanic(panicEmittedFrom string) bool { + const panicFromTheCatcherItselfPrefix = "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror.CatchVectorizedRuntimeError" + if strings.HasPrefix(panicEmittedFrom, panicFromTheCatcherItselfPrefix) { + // This panic came from the catcher itself, so we will propagate it + // unchanged by the higher-level catchers. + return false + } + const nonVectorizedTestPrefix = "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror.NonVectorizedTestPanic" + if strings.HasPrefix(panicEmittedFrom, nonVectorizedTestPrefix) { + // This panic came from NonVectorizedTestPanic() method and should not + // be caught for testing purposes. return false } return strings.HasPrefix(panicEmittedFrom, colPackagesPrefix) || strings.HasPrefix(panicEmittedFrom, execinfraPackagePrefix) || - strings.HasPrefix(panicEmittedFrom, rowexecPackagePrefix) || strings.HasPrefix(panicEmittedFrom, sqlColPackagesPrefix) || + strings.HasPrefix(panicEmittedFrom, sqlRowPackagesPrefix) || strings.HasPrefix(panicEmittedFrom, treePackagePrefix) } @@ -185,3 +193,10 @@ func InternalError(err error) { func ExpectedError(err error) { panic(newNotInternalError(err)) } + +// NonVectorizedTestPanic is the equivalent of Golang's 'panic' word that should +// be used by the testing code within the vectorized engine to simulate a panic +// that occurs outside of the engine (and, thus, should not be caught). +func NonVectorizedTestPanic(object interface{}) { + panic(object) +} diff --git a/pkg/sql/colexecbase/colexecerror/error_test.go b/pkg/sql/colexecbase/colexecerror/error_test.go new file mode 100644 index 000000000000..0e12b1790e9b --- /dev/null +++ b/pkg/sql/colexecbase/colexecerror/error_test.go @@ -0,0 +1,76 @@ +// Copyright 2021 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 colexecerror_test + +import ( + "errors" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" +) + +// TestCatchVectorizedRuntimeError verifies that the panic-catcher doesn't catch +// panics that originate outside of the vectorized engine and correctly +// annotates errors that are propagated via +// colexecerror.(Internal|Expected)Error methods. +func TestCatchVectorizedRuntimeError(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + // Setup multiple levels of catchers to ensure that the panic-catcher + // doesn't fool itself into catching panics that the inner catcher emitted. + require.Panics(t, func() { + require.NoError(t, colexecerror.CatchVectorizedRuntimeError(func() { + require.NoError(t, colexecerror.CatchVectorizedRuntimeError(func() { + colexecerror.NonVectorizedTestPanic(errors.New("should not be caught")) + })) + })) + }) + + const shouldBeCaughtText = "should be caught" + shouldBeCaughtErr := errors.New(shouldBeCaughtText) + const annotationText = "unexpected error from the vectorized engine" + + // Propagate an error as an internal one (this should add annotations to the + // returned error). + annotatedErr := colexecerror.CatchVectorizedRuntimeError(func() { + colexecerror.InternalError(shouldBeCaughtErr) + }) + require.NotNil(t, annotatedErr) + require.True(t, strings.Contains(annotatedErr.Error(), shouldBeCaughtText)) + require.True(t, strings.Contains(annotatedErr.Error(), annotationText)) + + // Propagate an error as an expected one (this should *not* add annotations + // to the returned error). + notAnnotatedErr := colexecerror.CatchVectorizedRuntimeError(func() { + colexecerror.ExpectedError(shouldBeCaughtErr) + }) + require.NotNil(t, notAnnotatedErr) + require.True(t, strings.Contains(notAnnotatedErr.Error(), shouldBeCaughtText)) + require.False(t, strings.Contains(notAnnotatedErr.Error(), annotationText)) +} + +// TestNonVectorizedTestPanicIsNotCaught verifies that panics emitted via +// NonVectorizedTestPanic() method are not caught by the catcher. +func TestNonVectorizedTestPanicIsNotCaught(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + require.Panics(t, func() { + require.NoError(t, colexecerror.CatchVectorizedRuntimeError(func() { + colexecerror.NonVectorizedTestPanic("should panic") + })) + }) +} diff --git a/pkg/sql/colflow/vectorized_panic_propagation_test.go b/pkg/sql/colflow/vectorized_panic_propagation_test.go index dc8799c2c0d3..83103d469065 100644 --- a/pkg/sql/colflow/vectorized_panic_propagation_test.go +++ b/pkg/sql/colflow/vectorized_panic_propagation_test.go @@ -179,7 +179,7 @@ func (e *testNonVectorizedPanicEmitter) Init() { func (e *testNonVectorizedPanicEmitter) Next(ctx context.Context) coldata.Batch { if !e.emitBatch { e.emitBatch = true - panic("") + colexecerror.NonVectorizedTestPanic("") } e.emitBatch = false diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 6ec998fda920..308df3312a95 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1636,7 +1636,6 @@ func TestLint(t *testing.T) { // sql/colexecbase/colexecerror/error.go file. "sql/col*", ":!sql/colexecbase/colexecerror/error.go", - ":!sql/colflow/vectorized_panic_propagation_test.go", // This exception is because execgen itself uses panics during code // generation - not at execution time. The (glob,exclude) directive // (see git help gitglossary) makes * behave like a normal, single dir