diff --git a/pkg/sql/explain_plan.go b/pkg/sql/explain_plan.go index e3a790f95c1b..f5d317a6f7ab 100644 --- a/pkg/sql/explain_plan.go +++ b/pkg/sql/explain_plan.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/colflow" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec/explain" @@ -28,7 +29,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/util/buildutil" - "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/errors" ) @@ -178,19 +178,11 @@ func emitExplain( ) (err error) { // Guard against bugs in the explain code. defer func() { - if r := recover(); r != nil { - // This code allows us to propagate internal and runtime errors without - // having to add error checks everywhere throughout the code. This is only - // possible because the code does not update shared state and does not - // manipulate locks. - // Note that we don't catch anything in debug builds, so that failures are - // more visible. - if ok, e := errorutil.ShouldCatch(r); ok && !buildutil.CrdbTestBuild { + // Note that we don't catch anything in debug builds, so that failures are + // more visible. + if !buildutil.CrdbTestBuild { + if e := opt.CatchOptimizerError(); e != nil { err = e - } else { - // Other panic objects can't be considered "safe" and thus are - // propagated as crashes that terminate the session. - panic(r) } } }() diff --git a/pkg/sql/opt/BUILD.bazel b/pkg/sql/opt/BUILD.bazel index 1e417a36c73c..04a31eda2b5f 100644 --- a/pkg/sql/opt/BUILD.bazel +++ b/pkg/sql/opt/BUILD.bazel @@ -4,6 +4,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "opt", srcs = [ + "catch.go", "colset.go", "column_meta.go", "constants.go", diff --git a/pkg/sql/opt/catch.go b/pkg/sql/opt/catch.go new file mode 100644 index 000000000000..0a4e3ee59368 --- /dev/null +++ b/pkg/sql/opt/catch.go @@ -0,0 +1,33 @@ +package opt + +import ( + "runtime" + + "github.com/cockroachdb/errors" +) + +// CatchOptimizerError catches any runtime panics from optimizer functions and +// returns them as errors. This allows the optimizer to propagate errors +// internally as panics without adding error checks everywhere. This is only +// possible because the optimizer code does not update shared state and does not +// manipulate locks. +func CatchOptimizerError() error { + r := recover() + if r == nil { + return nil + } + err, ok := r.(error) + if !ok { + // Not an error object. For serious internal errors e.g. in the scheduler, + // bad goroutine state, allocator problem etc, the go runtime throws a + // string which does not implement error. So in this case we suspect we are + // not able to recover, and must crash. + panic(r) + } + if errors.HasInterface(err, (*runtime.Error)(nil)) { + // Convert runtime errors to assertion failures, which include stacks + // and get reported to Sentry. + return errors.HandleAsAssertionFailure(err) + } + return err +} diff --git a/pkg/sql/opt/exec/execbuilder/BUILD.bazel b/pkg/sql/opt/exec/execbuilder/BUILD.bazel index 4138ad9b41fd..e6fd66d28fc1 100644 --- a/pkg/sql/opt/exec/execbuilder/BUILD.bazel +++ b/pkg/sql/opt/exec/execbuilder/BUILD.bazel @@ -46,9 +46,7 @@ go_library( "//pkg/util", "//pkg/util/buildutil", "//pkg/util/encoding", - "//pkg/util/errorutil", "//pkg/util/errorutil/unimplemented", - "//pkg/util/log", "//pkg/util/timeutil", "//pkg/util/treeprinter", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/sql/opt/exec/execbuilder/builder.go b/pkg/sql/opt/exec/execbuilder/builder.go index 9f3e3b33f83a..29c8fdee0596 100644 --- a/pkg/sql/opt/exec/execbuilder/builder.go +++ b/pkg/sql/opt/exec/execbuilder/builder.go @@ -24,7 +24,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" ) @@ -282,16 +281,8 @@ func (b *Builder) wrapFunction(fnName string) tree.ResolvableFunctionReference { func (b *Builder) build(e opt.Expr) (_ execPlan, err error) { defer func() { - if r := recover(); r != nil { - // This code allows us to propagate errors without adding lots of checks - // for `if err != nil` throughout the construction code. This is only - // possible because the code does not update shared state and does not - // manipulate locks. - if ok, e := errorutil.ShouldCatch(r); ok { - err = e - } else { - panic(r) - } + if e := opt.CatchOptimizerError(); e != nil { + err = e } }() diff --git a/pkg/sql/opt/exec/execbuilder/relational.go b/pkg/sql/opt/exec/execbuilder/relational.go index 6ae7acbad6f4..f3b2fa104601 100644 --- a/pkg/sql/opt/exec/execbuilder/relational.go +++ b/pkg/sql/opt/exec/execbuilder/relational.go @@ -42,9 +42,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/encoding" - "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" - "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" @@ -1070,24 +1068,8 @@ func (b *Builder) buildApplyJoin(join memo.RelExpr) (execPlan, error) { var o xform.Optimizer planRightSideFn := func(ctx context.Context, ef exec.Factory, leftRow tree.Datums) (_ exec.Plan, err error) { defer func() { - if r := recover(); r != nil { - // This code allows us to propagate internal errors without having to add - // error checks everywhere throughout the code. This is only possible - // because the code does not update shared state and does not manipulate - // locks. - // - // This is the same panic-catching logic that exists in - // o.Optimize() below. It's required here because it's possible - // for factory functions to panic below, like - // CopyAndReplaceDefault. - if ok, e := errorutil.ShouldCatch(r); ok { - err = e - log.VEventf(ctx, 1, "%v", err) - } else { - // Other panic objects can't be considered "safe" and thus are - // propagated as crashes that terminate the session. - panic(r) - } + if e := opt.CatchOptimizerError(); e != nil { + err = e } }() diff --git a/pkg/sql/opt/exec/explain/BUILD.bazel b/pkg/sql/opt/exec/explain/BUILD.bazel index dc1fcbe02920..9f15cb0e6c59 100644 --- a/pkg/sql/opt/exec/explain/BUILD.bazel +++ b/pkg/sql/opt/exec/explain/BUILD.bazel @@ -33,7 +33,6 @@ go_library( "//pkg/sql/sem/tree", "//pkg/sql/types", "//pkg/util", - "//pkg/util/errorutil", "//pkg/util/humanizeutil", "//pkg/util/timeutil", "//pkg/util/treeprinter", diff --git a/pkg/sql/opt/exec/explain/plan_gist_factory.go b/pkg/sql/opt/exec/explain/plan_gist_factory.go index ff50b111becb..f3a4a6a855a4 100644 --- a/pkg/sql/opt/exec/explain/plan_gist_factory.go +++ b/pkg/sql/opt/exec/explain/plan_gist_factory.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/inverted" + "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/opt/constraint" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" @@ -30,7 +31,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/errors" ) @@ -155,18 +155,8 @@ func (f *PlanGistFactory) PlanGist() PlanGist { // DecodePlanGistToRows converts a gist to a logical plan and returns the rows. func DecodePlanGistToRows(gist string, catalog cat.Catalog) (_ []string, retErr error) { defer func() { - if r := recover(); r != nil { - // This code allows us to propagate internal errors without having - // to add error checks everywhere throughout the code. This is only - // possible because the code does not update shared state and does - // not manipulate locks. - if ok, e := errorutil.ShouldCatch(r); ok { - retErr = e - } else { - // Other panic objects can't be considered "safe" and thus are - // propagated as crashes that terminate the session. - panic(r) - } + if e := opt.CatchOptimizerError(); e != nil { + retErr = e } }() diff --git a/pkg/sql/opt/exec/explain/result_columns.go b/pkg/sql/opt/exec/explain/result_columns.go index 5197d7e75876..848417bac48b 100644 --- a/pkg/sql/opt/exec/explain/result_columns.go +++ b/pkg/sql/opt/exec/explain/result_columns.go @@ -13,10 +13,10 @@ package explain import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/errors" ) @@ -26,15 +26,11 @@ import ( func getResultColumns( op execOperator, args interface{}, inputs ...colinfo.ResultColumns, ) (out colinfo.ResultColumns, err error) { + // If we have a bug in the code below, it's easily possible to hit panic + // (like out-of-bounds). Catch these here and return as an error. defer func() { - if r := recover(); r != nil { - // If we have a bug in the code below, it's easily possible to hit panic - // (like out-of-bounds). Catch these here and return as an error. - if ok, e := errorutil.ShouldCatch(r); ok { - err = e - } else { - panic(r) - } + if e := opt.CatchOptimizerError(); e != nil { + err = e } }() diff --git a/pkg/sql/opt/norm/factory.go b/pkg/sql/opt/norm/factory.go index cfac9ad7543a..26d3338db827 100644 --- a/pkg/sql/opt/norm/factory.go +++ b/pkg/sql/opt/norm/factory.go @@ -303,16 +303,8 @@ func (f *Factory) CopyWithoutAssigningPlaceholders(e opt.Expr) opt.Expr { // exploration phase can begin. func (f *Factory) AssignPlaceholders(from *memo.Memo) (err error) { defer func() { - if r := recover(); r != nil { - // This code allows us to propagate errors without adding lots of checks - // for `if err != nil` throughout the construction code. This is only - // possible because the code does not update shared state and does not - // manipulate locks. - if ok, e := errorutil.ShouldCatch(r); ok { - err = e - } else { - panic(r) - } + if e := opt.CatchOptimizerError(); e != nil { + err = e } }() diff --git a/pkg/sql/opt/optbuilder/BUILD.bazel b/pkg/sql/opt/optbuilder/BUILD.bazel index bfd818882d89..367ff0c08585 100644 --- a/pkg/sql/opt/optbuilder/BUILD.bazel +++ b/pkg/sql/opt/optbuilder/BUILD.bazel @@ -84,7 +84,6 @@ go_library( "//pkg/sql/sqltelemetry", "//pkg/sql/types", "//pkg/util", - "//pkg/util/errorutil", "//pkg/util/errorutil/unimplemented", "//pkg/util/log", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/sql/opt/optbuilder/builder.go b/pkg/sql/opt/optbuilder/builder.go index c8e6700ff58f..131464fb3f0a 100644 --- a/pkg/sql/opt/optbuilder/builder.go +++ b/pkg/sql/opt/optbuilder/builder.go @@ -26,7 +26,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" @@ -184,17 +183,9 @@ func (b *Builder) Build() (err error) { log.VEventf(b.ctx, 1, "optbuilder start") defer log.VEventf(b.ctx, 1, "optbuilder finish") defer func() { - if r := recover(); r != nil { - // This code allows us to propagate errors without adding lots of checks - // for `if err != nil` throughout the construction code. This is only - // possible because the code does not update shared state and does not - // manipulate locks. - if ok, e := errorutil.ShouldCatch(r); ok { - err = e - log.VEventf(b.ctx, 1, "%v", err) - } else { - panic(r) - } + if e := opt.CatchOptimizerError(); e != nil { + err = e + log.VEventf(b.ctx, 1, "%v", err) } }() diff --git a/pkg/sql/opt/optbuilder/fk_cascade.go b/pkg/sql/opt/optbuilder/fk_cascade.go index 3c8a5bb94169..6d66127dbd68 100644 --- a/pkg/sql/opt/optbuilder/fk_cascade.go +++ b/pkg/sql/opt/optbuilder/fk_cascade.go @@ -23,7 +23,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/errors" ) @@ -895,12 +894,8 @@ func buildCascadeHelper( // Enact panic handling similar to Builder.Build(). defer func() { - if r := recover(); r != nil { - if ok, e := errorutil.ShouldCatch(r); ok { - err = e - } else { - panic(r) - } + if e := opt.CatchOptimizerError(); e != nil { + err = e } }() diff --git a/pkg/sql/opt/optbuilder/scalar.go b/pkg/sql/opt/optbuilder/scalar.go index 6251909e35d2..232898e9d613 100644 --- a/pkg/sql/opt/optbuilder/scalar.go +++ b/pkg/sql/opt/optbuilder/scalar.go @@ -34,7 +34,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" @@ -1017,16 +1016,8 @@ func NewScalar( // expression equivalent to expr. func (sb *ScalarBuilder) Build(expr tree.Expr) (err error) { defer func() { - if r := recover(); r != nil { - // This code allows us to propagate errors without adding lots of checks - // for `if err != nil` throughout the construction code. This is only - // possible because the code does not update shared state and does not - // manipulate locks. - if ok, e := errorutil.ShouldCatch(r); ok { - err = e - } else { - panic(r) - } + if e := opt.CatchOptimizerError(); e != nil { + err = e } }() diff --git a/pkg/sql/opt/xform/BUILD.bazel b/pkg/sql/opt/xform/BUILD.bazel index 29ff998fdbb3..3791e9c2d600 100644 --- a/pkg/sql/opt/xform/BUILD.bazel +++ b/pkg/sql/opt/xform/BUILD.bazel @@ -52,7 +52,6 @@ go_library( "//pkg/util", "//pkg/util/buildutil", "//pkg/util/cancelchecker", - "//pkg/util/errorutil", "//pkg/util/log", "//pkg/util/treeprinter", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/sql/opt/xform/optimizer.go b/pkg/sql/opt/xform/optimizer.go index 2f25a27a6a9b..3464552909fe 100644 --- a/pkg/sql/opt/xform/optimizer.go +++ b/pkg/sql/opt/xform/optimizer.go @@ -24,7 +24,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/cancelchecker" - "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -240,19 +239,9 @@ func (o *Optimizer) Optimize() (_ opt.Expr, err error) { log.VEventf(o.ctx, 1, "optimize start") defer log.VEventf(o.ctx, 1, "optimize finish") defer func() { - if r := recover(); r != nil { - // This code allows us to propagate internal errors without having to add - // error checks everywhere throughout the code. This is only possible - // because the code does not update shared state and does not manipulate - // locks. - if ok, e := errorutil.ShouldCatch(r); ok { - err = e - log.VEventf(o.ctx, 1, "%v", err) - } else { - // Other panic objects can't be considered "safe" and thus are - // propagated as crashes that terminate the session. - panic(r) - } + if e := opt.CatchOptimizerError(); e != nil { + err = e + log.VEventf(o.ctx, 1, "%v", err) } }() diff --git a/pkg/sql/opt/xform/placeholder_fast_path.go b/pkg/sql/opt/xform/placeholder_fast_path.go index 678b32399d0a..ae9084625282 100644 --- a/pkg/sql/opt/xform/placeholder_fast_path.go +++ b/pkg/sql/opt/xform/placeholder_fast_path.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/props/physical" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/buildutil" - "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/errors" ) @@ -34,18 +33,8 @@ const maxRowCountForPlaceholderFastPath = 10 // If this function succeeds, the memo will be considered fully optimized. func (o *Optimizer) TryPlaceholderFastPath() (_ opt.Expr, ok bool, err error) { defer func() { - if r := recover(); r != nil { - // This code allows us to propagate internal errors without having to add - // error checks everywhere throughout the code. This is only possible - // because the code does not update shared state and does not manipulate - // locks. - if shouldCatch, e := errorutil.ShouldCatch(r); shouldCatch { - err = e - } else { - // Other panic objects can't be considered "safe" and thus are - // propagated as crashes that terminate the session. - panic(r) - } + if e := opt.CatchOptimizerError(); e != nil { + err = e } }() diff --git a/pkg/sql/plan_opt.go b/pkg/sql/plan_opt.go index 5bb926a03160..14868e09bf76 100644 --- a/pkg/sql/plan_opt.go +++ b/pkg/sql/plan_opt.go @@ -705,7 +705,14 @@ func (p *planner) DecodeGist(gist string, external bool) ([]string, error) { // indexes hypothetically added to the table. An index recommendation for the // query is outputted based on which hypothetical indexes are helpful in the // optimal plan. -func (opc *optPlanningCtx) makeQueryIndexRecommendation(ctx context.Context) error { +func (opc *optPlanningCtx) makeQueryIndexRecommendation(ctx context.Context) (err error) { + defer func() { + if e := opt.CatchOptimizerError(); e != nil { + err = e + log.VEventf(ctx, 1, "%v", err) + } + }() + // Save the normalized memo created by the optbuilder. savedMemo := opc.optimizer.DetachMemo(ctx) @@ -723,7 +730,7 @@ func (opc *optPlanningCtx) makeQueryIndexRecommendation(ctx context.Context) err opc.optimizer.NotifyOnMatchedRule(func(ruleName opt.RuleName) bool { return ruleName.IsNormalize() }) - if _, err := opc.optimizer.Optimize(); err != nil { + if _, err = opc.optimizer.Optimize(); err != nil { return err } @@ -741,7 +748,7 @@ func (opc *optPlanningCtx) makeQueryIndexRecommendation(ctx context.Context) err f.CopyWithoutAssigningPlaceholders, ) opc.optimizer.Memo().Metadata().UpdateTableMeta(hypTables) - if _, err := opc.optimizer.Optimize(); err != nil { + if _, err = opc.optimizer.Optimize(); err != nil { return err } diff --git a/pkg/sql/show_stats.go b/pkg/sql/show_stats.go index 3ee39232016d..298a7d101166 100644 --- a/pkg/sql/show_stats.go +++ b/pkg/sql/show_stats.go @@ -18,11 +18,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/exprutil" + "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" @@ -124,18 +124,8 @@ func (p *planner) ShowTableStats(ctx context.Context, n *tree.ShowTableStats) (p // Guard against crashes in the code below (e.g. #56356). defer func() { - if r := recover(); r != nil { - // This code allows us to propagate internal errors without having to add - // error checks everywhere throughout the code. This is only possible - // because the code does not update shared state and does not manipulate - // locks. - if ok, e := errorutil.ShouldCatch(r); ok { - err = e - } else { - // Other panic objects can't be considered "safe" and thus are - // propagated as crashes that terminate the session. - panic(r) - } + if e := opt.CatchOptimizerError(); e != nil { + err = e } }() diff --git a/pkg/util/errorutil/BUILD.bazel b/pkg/util/errorutil/BUILD.bazel index dde6ae72b76d..2cc6b4645dc3 100644 --- a/pkg/util/errorutil/BUILD.bazel +++ b/pkg/util/errorutil/BUILD.bazel @@ -4,7 +4,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "errorutil", srcs = [ - "catch.go", "error.go", "tenant.go", "tenant_deprecated_wrapper.go", diff --git a/pkg/util/errorutil/catch.go b/pkg/util/errorutil/catch.go deleted file mode 100644 index 69ec07cef716..000000000000 --- a/pkg/util/errorutil/catch.go +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright 2019 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 errorutil - -import ( - "runtime" - - "github.com/cockroachdb/errors" -) - -// ShouldCatch is used for catching errors thrown as panics. Its argument is the -// object returned by recover(); it succeeds if the object is an error. If the -// error is a runtime.Error, it is converted to an internal error (see -// errors.AssertionFailedf). -func ShouldCatch(obj interface{}) (ok bool, err error) { - err, ok = obj.(error) - if ok { - if errors.HasInterface(err, (*runtime.Error)(nil)) { - // Convert runtime errors to internal errors, which display the stack and - // get reported to Sentry. - err = errors.HandleAsAssertionFailure(err) - } - } - return ok, err -}