Skip to content

Commit

Permalink
opt: fix panic recovery for error handling
Browse files Browse the repository at this point in the history
The major entry points in the optimizer catch all panics that throw an
error and converts them to errors. Unfortunately, this also catches
runtime errors (in which case we convert them to errors and lose the
stack trace).

This change adds a `ShouldCatch` helper which determines if we should
return a thrown object as an error. If the object is a
`runtime.Error`, it gets wrapped by an AssertionFailed error which
will cause correct error handling (stack trace, sentry reporting, etc).

As part of this change, we are also removing wrappers like
`builderError`, which are no longer useful. We fix the opt tester to
fail with the full error information (using `%+v`) for assertion
errors.

Release note: None
  • Loading branch information
RaduBerinde committed Jul 9, 2019
1 parent 6299bd4 commit 525b9ff
Show file tree
Hide file tree
Showing 19 changed files with 155 additions and 150 deletions.
17 changes: 8 additions & 9 deletions pkg/sql/opt/exec/execbuilder/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/opt/exec"
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/errorutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -70,17 +71,15 @@ func (b *Builder) DisableTelemetry() {
func (b *Builder) Build() (_ 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.
if e, ok := r.(error); ok {
// 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
return
} else {
panic(r)
}
// Other panic objects can't be considered "safe" and thus are
// propagated as crashes that terminate the session.
panic(r)
}
}()

Expand Down
14 changes: 4 additions & 10 deletions pkg/sql/opt/norm/factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/opt/props/physical"
"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/log"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -45,13 +46,6 @@ type MatchedRuleFunc func(ruleName opt.RuleName) bool
// accessed by following the NextExpr links on the target expression.
type AppliedRuleFunc func(ruleName opt.RuleName, source, target opt.Expr)

// placeholderError wraps errors that occur during placeholder assignment, and
// is passed as an argument to panic. The panic is caught and converted back to
// an error by AssignPlaceholders.
type placeholderError struct {
error
}

// Factory constructs a normalized expression tree within the memo. As each
// kind of expression is constructed by the factory, it transitively runs
// normalization transformations defined for that expression type. This may
Expand Down Expand Up @@ -214,8 +208,8 @@ func (f *Factory) AssignPlaceholders(from *memo.Memo) (err error) {
// 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 bldErr, ok := r.(placeholderError); ok {
err = bldErr.error
if ok, e := errorutil.ShouldCatch(r); ok {
err = e
} else {
panic(r)
}
Expand All @@ -229,7 +223,7 @@ func (f *Factory) AssignPlaceholders(from *memo.Memo) (err error) {
if placeholder, ok := e.(*memo.PlaceholderExpr); ok {
d, err := e.(*memo.PlaceholderExpr).Value.Eval(f.evalCtx)
if err != nil {
panic(placeholderError{err})
panic(err)
}
return f.ConstructConstVal(d, placeholder.DataType())
}
Expand Down
12 changes: 6 additions & 6 deletions pkg/sql/opt/optbuilder/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,11 +31,11 @@ func (b *Builder) buildAlterTableSplit(split *tree.Split, inScope *scope) (outSc
}
index, err := cat.ResolveTableIndex(b.ctx, b.catalog, flags, &split.TableOrIndex)
if err != nil {
panic(builderError{err})
panic(err)
}
table := index.Table()
if err := b.catalog.CheckPrivilege(b.ctx, table, privilege.INSERT); err != nil {
panic(builderError{err})
panic(err)
}

b.DisableMemoReuse = true
Expand Down Expand Up @@ -89,11 +89,11 @@ func (b *Builder) buildAlterTableUnsplit(unsplit *tree.Unsplit, inScope *scope)
}
index, err := cat.ResolveTableIndex(b.ctx, b.catalog, flags, &unsplit.TableOrIndex)
if err != nil {
panic(builderError{err})
panic(err)
}
table := index.Table()
if err := b.catalog.CheckPrivilege(b.ctx, table, privilege.INSERT); err != nil {
panic(builderError{err})
panic(err)
}

b.DisableMemoReuse = true
Expand Down Expand Up @@ -139,11 +139,11 @@ func (b *Builder) buildAlterTableRelocate(
}
index, err := cat.ResolveTableIndex(b.ctx, b.catalog, flags, &relocate.TableOrIndex)
if err != nil {
panic(builderError{err})
panic(err)
}
table := index.Table()
if err := b.catalog.CheckPrivilege(b.ctx, table, privilege.INSERT); err != nil {
panic(builderError{err})
panic(err)
}

b.DisableMemoReuse = true
Expand Down
50 changes: 13 additions & 37 deletions pkg/sql/opt/optbuilder/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,17 +12,15 @@ package optbuilder

import (
"context"
"fmt"
"runtime"

"github.com/cockroachdb/cockroach/pkg/sql/delegate"
"github.com/cockroachdb/cockroach/pkg/sql/opt/cat"
"github.com/cockroachdb/cockroach/pkg/sql/opt/norm"
"github.com/cockroachdb/cockroach/pkg/sql/opt/optgen/exprgen"
"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/errors"
)

// Builder holds the context needed for building a memo structure from a SQL
Expand Down Expand Up @@ -134,24 +132,20 @@ func New(
// Builder.factory from the parsed SQL statement in Builder.stmt. See the
// comment above the Builder type declaration for details.
//
// If any subroutines panic with a builderError or pgerror.Error as part of the
// build process, the panic is caught here and returned as an error.
// If any subroutines panic with a non-runtime error as part of the build
// process, the panic is caught here and returned as an error.
func (b *Builder) Build() (err error) {
defer func() {
if r := recover(); r != nil {
// This code allows us to propagate semantic and internal errors without
// adding lots of checks for `if err != nil` throughout the code. This is
// only possible because the code does not update shared state and does
// not manipulate locks.
if e, ok := r.(error); ok {
if _, x := r.(runtime.Error); !x {
err = e
return
}
// 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)
}
// Other panic objects can't be considered "safe" and thus are
// propagated as crashes that terminate the session.
panic(r)
}
}()

Expand All @@ -170,27 +164,9 @@ func (b *Builder) Build() (err error) {
return nil
}

// builderError is used to wrap errors returned by various external APIs that
// occur during the build process. It exists for us to be able to panic on these
// errors and then catch them inside Builder.Build.
type builderError struct {
error error
}

// builderError are errors.
func (b builderError) Error() string { return b.error.Error() }

// Cause implements the causer interface. This is used so that builderErrors
// can be peeked through by the common error facilities.
func (b builderError) Cause() error { return b.error }

// Format implements the fmt.Formatter interface.
func (b builderError) Format(s fmt.State, verb rune) { errors.FormatError(b, s, verb) }

// unimplementedWithIssueDetailf formats according to a format
// specifier and returns a Postgres error with the
// pg code FeatureNotSupported, wrapped in a
// builderError.
// pg code FeatureNotSupported.
func unimplementedWithIssueDetailf(issue int, detail, format string, args ...interface{}) error {
return unimplemented.NewWithIssueDetailf(issue, detail, format, args...)
}
Expand Down Expand Up @@ -254,7 +230,7 @@ func (b *Builder) buildStmt(
// delegate functionality.
newStmt, err := delegate.TryDelegate(b.ctx, b.catalog, b.evalCtx, stmt)
if err != nil {
panic(builderError{err})
panic(err)
}
if newStmt != nil {
// Many delegate implementations resolve objects. It would be tedious to
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/opt/optbuilder/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,10 +44,10 @@ func (b *Builder) buildCreateTable(ct *tree.CreateTable, inScope *scope) (outSco
numColNames := len(ct.AsColumnNames)
numColumns := len(outScope.cols)
if numColNames != 0 && numColNames != numColumns {
panic(builderError{sqlbase.NewSyntaxError(fmt.Sprintf(
panic(sqlbase.NewSyntaxError(fmt.Sprintf(
"CREATE TABLE specifies %d column name%s, but data source has %d column%s",
numColNames, util.Pluralize(int64(numColNames)),
numColumns, util.Pluralize(int64(numColumns))))})
numColumns, util.Pluralize(int64(numColumns)))))
}

// Synthesize rowid column, and append to end of column list.
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/optbuilder/explain.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import (
func (b *Builder) buildExplain(explain *tree.Explain, inScope *scope) (outScope *scope) {
opts, err := explain.ParseOptions()
if err != nil {
panic(builderError{err})
panic(err)
}

// We don't allow the statement under Explain to reference outer columns, so we
Expand Down
12 changes: 6 additions & 6 deletions pkg/sql/opt/optbuilder/mutation_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -228,7 +228,7 @@ func (mb *mutationBuilder) addTargetColsByName(names tree.NameList) {
mb.addTargetCol(ord)
continue
}
panic(builderError{sqlbase.NewUndefinedColumnError(string(name))})
panic(sqlbase.NewUndefinedColumnError(string(name)))
}
}

Expand All @@ -240,12 +240,12 @@ func (mb *mutationBuilder) addTargetCol(ord int) {

// Don't allow targeting of mutation columns.
if cat.IsMutationColumn(mb.tab, ord) {
panic(builderError{makeBackfillError(tabCol.ColName())})
panic(makeBackfillError(tabCol.ColName()))
}

// Computed columns cannot be targeted with input values.
if tabCol.IsComputed() {
panic(builderError{sqlbase.CannotWriteToComputedColError(string(tabCol.ColName()))})
panic(sqlbase.CannotWriteToComputedColError(string(tabCol.ColName())))
}

// Ensure that the name list does not contain duplicates.
Expand Down Expand Up @@ -521,7 +521,7 @@ func (mb *mutationBuilder) addCheckConstraintCols() {
for i, n := 0, mb.tab.CheckCount(); i < n; i++ {
expr, err := parser.ParseExpr(string(mb.tab.Check(i).Constraint))
if err != nil {
panic(builderError{err})
panic(err)
}

alias := fmt.Sprintf("check%d", i+1)
Expand Down Expand Up @@ -732,7 +732,7 @@ func (mb *mutationBuilder) parseDefaultOrComputedExpr(colID opt.ColumnID) tree.E

expr, err := parser.ParseExpr(exprStr)
if err != nil {
panic(builderError{err})
panic(err)
}

mb.parsedExprs[ord] = expr
Expand Down Expand Up @@ -765,7 +765,7 @@ func (mb *mutationBuilder) buildFKChecks() {

refTab, err := mb.b.catalog.ResolveDataSourceByID(mb.b.ctx, fk.ReferencedTableID())
if err != nil {
panic(builderError{err})
panic(err)
}
refOrdinals := make([]int, numCols)
for j := range refOrdinals {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/optbuilder/opaque.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ func (b *Builder) tryBuildOpaque(stmt tree.Statement, inScope *scope) (outScope
}
obj, cols, err := fn(b.ctx, b.semaCtx, b.evalCtx, stmt)
if err != nil {
panic(builderError{err})
panic(err)
}
outScope = inScope.push()
b.synthesizeResultColumns(outScope, cols)
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/opt/optbuilder/orderby.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ func (b *Builder) analyzeOrderByIndex(
tab, _ := b.resolveTable(&order.Table, privilege.SELECT)
index, err := b.findIndexByName(tab, order.Index)
if err != nil {
panic(builderError{err})
panic(err)
}

// Append each key column from the index (including the implicit primary key
Expand All @@ -125,7 +125,7 @@ func (b *Builder) analyzeOrderByIndex(
// Columns which are indexable are always orderable.
col := index.Column(i)
if err != nil {
panic(builderError{err})
panic(err)
}

desc := col.Descending
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/opt/optbuilder/project.go
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ func (b *Builder) analyzeSelectList(
// Fall back to slow path. Pre-normalize any VarName so the work is
// not done twice below.
if err := e.NormalizeTopLevelVarName(); err != nil {
panic(builderError{err})
panic(err)
}

// Special handling for "*", "<table>.*" and "(Expr).*".
Expand Down Expand Up @@ -177,7 +177,7 @@ func (b *Builder) resolveColRef(e tree.Expr, inScope *scope) tree.TypedExpr {
colName := unresolved.Parts[0]
_, srcMeta, _, err := inScope.FindSourceProvidingColumn(b.ctx, tree.Name(colName))
if err != nil {
panic(builderError{err})
panic(err)
}
return srcMeta.(tree.TypedExpr)
}
Expand All @@ -188,7 +188,7 @@ func (b *Builder) resolveColRef(e tree.Expr, inScope *scope) tree.TypedExpr {
func (b *Builder) getColName(expr tree.SelectExpr) string {
s, err := tree.GetRenderColName(b.semaCtx.SearchPath, expr)
if err != nil {
panic(builderError{err})
panic(err)
}
return s
}
Expand Down
Loading

0 comments on commit 525b9ff

Please sign in to comment.