From dcbf8360974d79334869ba2aecc80bdfc917ae0d Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Thu, 14 Mar 2019 11:30:36 -0400 Subject: [PATCH] opt: catch all pgerror.Error in optbuilder We now catch all `pgerror.Error`s in optbuilder, which means that we don't need to use the `buildError` wrapper with them. The wrapper still exists when external calls (e.g. for name resolution) return a generic error. The main motivation is that optbuilder calls into the factory which can panic internally. We will want to switch those panics to assertion errors as well, but they are in different packages. The existing approach would have required a shared, exported wrapper. Release note: None --- pkg/sql/opt/optbuilder/builder.go | 36 +++++++-------- pkg/sql/opt/optbuilder/delete.go | 6 +-- pkg/sql/opt/optbuilder/distinct.go | 8 ++-- pkg/sql/opt/optbuilder/groupby.go | 4 +- pkg/sql/opt/optbuilder/insert.go | 20 ++++----- pkg/sql/opt/optbuilder/join.go | 28 ++++++------ pkg/sql/opt/optbuilder/mutation_builder.go | 20 ++++----- pkg/sql/opt/optbuilder/project.go | 4 +- pkg/sql/opt/optbuilder/scalar.go | 10 ++--- pkg/sql/opt/optbuilder/scope.go | 20 ++++----- pkg/sql/opt/optbuilder/select.go | 51 +++++++++++----------- pkg/sql/opt/optbuilder/srfs.go | 2 +- pkg/sql/opt/optbuilder/subquery.go | 6 +-- pkg/sql/opt/optbuilder/union.go | 8 ++-- pkg/sql/opt/optbuilder/update.go | 12 ++--- pkg/sql/opt/optbuilder/util.go | 32 +++++++------- pkg/sql/opt/optbuilder/values.go | 8 ++-- pkg/sql/pgwire/pgerror/errors.go | 8 ++-- 18 files changed, 139 insertions(+), 144 deletions(-) diff --git a/pkg/sql/opt/optbuilder/builder.go b/pkg/sql/opt/optbuilder/builder.go index 981be1519885..cb2cb788074d 100644 --- a/pkg/sql/opt/optbuilder/builder.go +++ b/pkg/sql/opt/optbuilder/builder.go @@ -127,18 +127,21 @@ 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 as part of the build process, -// the panic is caught here and returned as an error. +// 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. func (b *Builder) Build() (err error) { defer func() { if r := recover(); r != nil { - // This code allows us to propagate builder errors without adding - // lots of checks for `if err != nil` throughout the code. This is + // 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 bldErr, ok := r.(builderError); ok { - err = bldErr.error - } else { + switch e := r.(type) { + case builderError: + err = e.error + case *pgerror.Error: + err = e + default: panic(r) } } @@ -159,29 +162,22 @@ func (b *Builder) Build() (err error) { return nil } -// builderError is used for semantic errors that occur during the build process -// and is passed as an argument to panic. These panics are caught and converted -// back to errors inside Builder.Build. +// 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 even if they are not +// pgerror.Error. type builderError struct { error } -// assertionErrorf formats an internal error (or assertion failure) -// using a pgerror object, so that it can be picked up by telemetry. -func assertionErrorf(format string, args ...interface{}) builderError { - return builderError{pgerror.NewAssertionErrorf(format, args...)} -} - // unimplementedWithIssueDetailf formats according to a format // specifier and returns a Postgres error with the // pgerror.CodeFeatureNotSupportedError code, wrapped in a // builderError. func unimplementedWithIssueDetailf( issue int, detail, format string, args ...interface{}, -) builderError { - return builderError{ - pgerror.UnimplementedWithIssueDetailErrorf(issue, detail, format, args...), - } +) *pgerror.Error { + return pgerror.UnimplementedWithIssueDetailErrorf(issue, detail, format, args...) } // buildStmt builds a set of memo groups that represent the given SQL diff --git a/pkg/sql/opt/optbuilder/delete.go b/pkg/sql/opt/optbuilder/delete.go index 7af9e9dd27f6..2b9d58af6b18 100644 --- a/pkg/sql/opt/optbuilder/delete.go +++ b/pkg/sql/opt/optbuilder/delete.go @@ -34,12 +34,12 @@ import ( func (b *Builder) buildDelete(del *tree.Delete, inScope *scope) (outScope *scope) { // UX friendliness safeguard. if del.Where == nil && b.evalCtx.SessionData.SafeUpdates { - panic(builderError{pgerror.NewDangerousStatementErrorf("DELETE without WHERE clause")}) + panic(pgerror.NewDangerousStatementErrorf("DELETE without WHERE clause")) } if del.OrderBy != nil && del.Limit == nil { - panic(builderError{pgerror.NewErrorf(pgerror.CodeSyntaxError, - "DELETE statement requires LIMIT when ORDER BY is used")}) + panic(pgerror.NewErrorf(pgerror.CodeSyntaxError, + "DELETE statement requires LIMIT when ORDER BY is used")) } if del.With != nil { diff --git a/pkg/sql/opt/optbuilder/distinct.go b/pkg/sql/opt/optbuilder/distinct.go index 40383e061a42..646b8c5362a6 100644 --- a/pkg/sql/opt/optbuilder/distinct.go +++ b/pkg/sql/opt/optbuilder/distinct.go @@ -38,10 +38,10 @@ func (b *Builder) constructDistinct(inScope *scope) memo.RelExpr { // Note: this behavior is consistent with PostgreSQL. for _, col := range inScope.ordering { if !private.GroupingCols.Contains(int(col.ID())) { - panic(builderError{pgerror.NewErrorf( + panic(pgerror.NewErrorf( pgerror.CodeInvalidColumnReferenceError, "for SELECT DISTINCT, ORDER BY expressions must appear in select list", - )}) + )) } } @@ -79,10 +79,10 @@ func (b *Builder) buildDistinctOn(distinctOnCols opt.ColSet, inScope *scope) (ou var seen opt.ColSet for _, col := range inScope.ordering { if !distinctOnCols.Contains(int(col.ID())) { - panic(builderError{pgerror.NewErrorf( + panic(pgerror.NewErrorf( pgerror.CodeInvalidColumnReferenceError, "SELECT DISTINCT ON expressions must match initial ORDER BY expressions", - )}) + )) } seen.Add(int(col.ID())) if seen.Equals(distinctOnCols) { diff --git a/pkg/sql/opt/optbuilder/groupby.go b/pkg/sql/opt/optbuilder/groupby.go index af446cdb3e5d..d8416bbe905d 100644 --- a/pkg/sql/opt/optbuilder/groupby.go +++ b/pkg/sql/opt/optbuilder/groupby.go @@ -114,7 +114,7 @@ func (a *aggregateInfo) TypeCheck(ctx *tree.SemaContext, desired types.T) (tree. // Eval is part of the tree.TypedExpr interface. func (a *aggregateInfo) Eval(_ *tree.EvalContext) (tree.Datum, error) { - panic(assertionErrorf("aggregateInfo must be replaced before evaluation")) + panic(pgerror.NewAssertionErrorf("aggregateInfo must be replaced before evaluation")) } var _ tree.Expr = &aggregateInfo{} @@ -145,7 +145,7 @@ func (b *Builder) constructGroupBy( if scalar == nil { // A "pass through" column (i.e. a VariableOp) is not legal as an // aggregation. - panic(assertionErrorf("variable as aggregation")) + panic(pgerror.NewAssertionErrorf("variable as aggregation")) } aggs = append(aggs, memo.AggregationsItem{ Agg: scalar, diff --git a/pkg/sql/opt/optbuilder/insert.go b/pkg/sql/opt/optbuilder/insert.go index 4b9fe62b7d66..7e91d6e31efa 100644 --- a/pkg/sql/opt/optbuilder/insert.go +++ b/pkg/sql/opt/optbuilder/insert.go @@ -342,7 +342,7 @@ func (mb *mutationBuilder) needExistingRows() bool { // list of table columns that are the target of the Insert operation. func (mb *mutationBuilder) addTargetNamedColsForInsert(names tree.NameList) { if len(mb.targetColList) != 0 { - panic(assertionErrorf("addTargetNamedColsForInsert cannot be called more than once")) + panic(pgerror.NewAssertionErrorf("addTargetNamedColsForInsert cannot be called more than once")) } // Add target table columns by the names specified in the Insert statement. @@ -376,8 +376,8 @@ func (mb *mutationBuilder) checkPrimaryKeyForInsert() { continue } - panic(builderError{pgerror.NewErrorf(pgerror.CodeInvalidForeignKeyError, - "missing %q primary key column", col.ColName())}) + panic(pgerror.NewErrorf(pgerror.CodeInvalidForeignKeyError, + "missing %q primary key column", col.ColName())) } } @@ -440,12 +440,12 @@ func (mb *mutationBuilder) checkForeignKeysForInsert() { case 0: // Do nothing. case 1: - panic(builderError{pgerror.NewErrorf(pgerror.CodeForeignKeyViolationError, - "missing value for column %q in multi-part foreign key", missingCols[0])}) + panic(pgerror.NewErrorf(pgerror.CodeForeignKeyViolationError, + "missing value for column %q in multi-part foreign key", missingCols[0])) default: sort.Strings(missingCols) - panic(builderError{pgerror.NewErrorf(pgerror.CodeForeignKeyViolationError, - "missing values for columns %q in multi-part foreign key", missingCols)}) + panic(pgerror.NewErrorf(pgerror.CodeForeignKeyViolationError, + "missing values for columns %q in multi-part foreign key", missingCols)) } } } @@ -462,7 +462,7 @@ func (mb *mutationBuilder) checkForeignKeysForInsert() { // columns. func (mb *mutationBuilder) addTargetTableColsForInsert(maxCols int) { if len(mb.targetColList) != 0 { - panic(assertionErrorf("addTargetTableColsForInsert cannot be called more than once")) + panic(pgerror.NewAssertionErrorf("addTargetTableColsForInsert cannot be called more than once")) } // Only consider non-mutation columns, since mutation columns are hidden from @@ -993,8 +993,8 @@ func (mb *mutationBuilder) ensureUniqueConflictCols(cols tree.NameList) cat.Inde return index } } - panic(builderError{pgerror.NewErrorf(pgerror.CodeInvalidColumnReferenceError, - "there is no unique or exclusion constraint matching the ON CONFLICT specification")}) + panic(pgerror.NewErrorf(pgerror.CodeInvalidColumnReferenceError, + "there is no unique or exclusion constraint matching the ON CONFLICT specification")) } // getPrimaryKeyColumnNames returns the names of all primary key columns in the diff --git a/pkg/sql/opt/optbuilder/join.go b/pkg/sql/opt/optbuilder/join.go index 8ecfb53e309c..8c2fb0adea5f 100644 --- a/pkg/sql/opt/optbuilder/join.go +++ b/pkg/sql/opt/optbuilder/join.go @@ -50,9 +50,9 @@ func (b *Builder) buildJoin(join *tree.JoinTableExpr, inScope *scope) (outScope flags.DisallowHashJoin = true flags.DisallowMergeJoin = true if joinType != sqlbase.InnerJoin && joinType != sqlbase.LeftOuterJoin { - panic(builderError{pgerror.NewErrorf(pgerror.CodeSyntaxError, + panic(pgerror.NewErrorf(pgerror.CodeSyntaxError, "%s can only be used with INNER or LEFT joins", tree.AstLookup, - )}) + )) } case tree.AstMerge: @@ -60,9 +60,9 @@ func (b *Builder) buildJoin(join *tree.JoinTableExpr, inScope *scope) (outScope flags.DisallowHashJoin = true default: - panic(builderError{pgerror.NewErrorf( + panic(pgerror.NewErrorf( pgerror.CodeFeatureNotSupportedError, "join hint %s not supported", join.Hint, - )}) + )) } switch cond := join.Cond.(type) { @@ -135,11 +135,11 @@ func (b *Builder) validateJoinTableNames(leftScope, rightScope *scope) { continue } - panic(builderError{pgerror.NewErrorf( + panic(pgerror.NewErrorf( pgerror.CodeDuplicateAliasError, "source name %q specified more than once (missing AS clause)", tree.ErrString(&leftName.TableName), - )}) + )) } } } @@ -295,8 +295,8 @@ func (jb *usingJoinBuilder) buildUsingJoin(using *tree.UsingJoinCond) { } if seenCols.Contains(int(leftCol.id)) { // Same name exists more than once in USING column name list. - panic(builderError{pgerror.NewErrorf(pgerror.CodeDuplicateColumnError, - "column %q appears more than once in USING clause", tree.ErrString(&name))}) + panic(pgerror.NewErrorf(pgerror.CodeDuplicateColumnError, + "column %q appears more than once in USING clause", tree.ErrString(&name))) } seenCols.Add(int(leftCol.id)) @@ -414,9 +414,9 @@ func (jb *usingJoinBuilder) addEqualityCondition(leftCol, rightCol *scopeColumn) // First, check if the comparison would even be valid. if !leftCol.typ.Equivalent(rightCol.typ) { if _, found := tree.FindEqualComparisonFunction(leftCol.typ, rightCol.typ); !found { - panic(builderError{pgerror.NewErrorf(pgerror.CodeDatatypeMismatchError, + panic(pgerror.NewErrorf(pgerror.CodeDatatypeMismatchError, "JOIN/USING types %s for left and %s for right cannot be matched for column %q", - leftCol.typ, rightCol.typ, tree.ErrString(&leftCol.name))}) + leftCol.typ, rightCol.typ, tree.ErrString(&leftCol.name))) } } @@ -458,11 +458,11 @@ func (jb *usingJoinBuilder) addEqualityCondition(leftCol, rightCol *scopeColumn) } func (jb *usingJoinBuilder) raiseDuplicateColError(name tree.Name) { - panic(builderError{pgerror.NewErrorf(pgerror.CodeDuplicateColumnError, - "duplicate column name: %q", tree.ErrString(&name))}) + panic(pgerror.NewErrorf(pgerror.CodeDuplicateColumnError, + "duplicate column name: %q", tree.ErrString(&name))) } func (jb *usingJoinBuilder) raiseUndefinedColError(name tree.Name, context string) { - panic(builderError{pgerror.NewErrorf(pgerror.CodeUndefinedColumnError, - "column \"%s\" specified in USING clause does not exist in %s table", name, context)}) + panic(pgerror.NewErrorf(pgerror.CodeUndefinedColumnError, + "column \"%s\" specified in USING clause does not exist in %s table", name, context)) } diff --git a/pkg/sql/opt/optbuilder/mutation_builder.go b/pkg/sql/opt/optbuilder/mutation_builder.go index 2b97da420814..31b9fb253c5d 100644 --- a/pkg/sql/opt/optbuilder/mutation_builder.go +++ b/pkg/sql/opt/optbuilder/mutation_builder.go @@ -215,8 +215,8 @@ func (mb *mutationBuilder) addTargetCol(ord int) { // Ensure that the name list does not contain duplicates. colID := mb.tabID.ColumnID(ord) if mb.targetColSet.Contains(int(colID)) { - panic(builderError{pgerror.NewErrorf(pgerror.CodeSyntaxError, - "multiple assignments to the same column %q", tabCol.ColName())}) + panic(pgerror.NewErrorf(pgerror.CodeSyntaxError, + "multiple assignments to the same column %q", tabCol.ColName())) } mb.targetColSet.Add(int(colID)) @@ -547,9 +547,9 @@ func (mb *mutationBuilder) checkNumCols(expected, actual int) { } else { kw = "UPSERT" } - panic(builderError{pgerror.NewErrorf(pgerror.CodeSyntaxError, + panic(pgerror.NewErrorf(pgerror.CodeSyntaxError, "%s has more %s than %s, %d expressions for %d targets", - kw, more, less, actual, expected)}) + kw, more, less, actual, expected)) } } @@ -597,7 +597,7 @@ func findNotNullIndexCol(index cat.Index) int { return indexCol.Ordinal } } - panic(assertionErrorf("should have found not null column in index")) + panic(pgerror.NewAssertionErrorf("should have found not null column in index")) } // resultsNeeded determines whether a statement that might have a RETURNING @@ -609,7 +609,7 @@ func resultsNeeded(r tree.ReturningClause) bool { case *tree.ReturningNothing, *tree.NoReturningClause: return false default: - panic(assertionErrorf("unexpected ReturningClause type: %T", t)) + panic(pgerror.NewAssertionErrorf("unexpected ReturningClause type: %T", t)) } } @@ -632,9 +632,9 @@ func getAliasedTableName(n tree.TableExpr) (*tree.TableName, *tree.TableName) { } tn, ok := n.(*tree.TableName) if !ok { - panic(builderError{pgerror.Unimplemented( + panic(pgerror.Unimplemented( "complex table expression in UPDATE/DELETE", - "cannot use a complex table name with DELETE/UPDATE")}) + "cannot use a complex table name with DELETE/UPDATE")) } return tn, alias } @@ -652,7 +652,7 @@ func checkDatumTypeFitsColumnType(col cat.Column, typ types.T) { } colName := string(col.ColName()) - panic(builderError{pgerror.NewErrorf(pgerror.CodeDatatypeMismatchError, + panic(pgerror.NewErrorf(pgerror.CodeDatatypeMismatchError, "value type %s doesn't match type %s of column %q", - typ, col.ColTypeStr(), tree.ErrNameString(colName))}) + typ, col.ColTypeStr(), tree.ErrNameString(colName))) } diff --git a/pkg/sql/opt/optbuilder/project.go b/pkg/sql/opt/optbuilder/project.go index 5749ac5e2396..f2b1cf0252f3 100644 --- a/pkg/sql/opt/optbuilder/project.go +++ b/pkg/sql/opt/optbuilder/project.go @@ -123,8 +123,8 @@ func (b *Builder) analyzeSelectList( switch v.(type) { case tree.UnqualifiedStar, *tree.AllColumnsSelector, *tree.TupleStar: if e.As != "" { - panic(builderError{pgerror.NewErrorf(pgerror.CodeSyntaxError, - "%q cannot be aliased", tree.ErrString(v))}) + panic(pgerror.NewErrorf(pgerror.CodeSyntaxError, + "%q cannot be aliased", tree.ErrString(v))) } aliases, exprs := b.expandStar(e.Expr, inScope) diff --git a/pkg/sql/opt/optbuilder/scalar.go b/pkg/sql/opt/optbuilder/scalar.go index 7018d0976b7a..d85f903c40fd 100644 --- a/pkg/sql/opt/optbuilder/scalar.go +++ b/pkg/sql/opt/optbuilder/scalar.go @@ -278,8 +278,8 @@ func (b *Builder) buildScalar( case *tree.IndexedVar: if t.Idx < 0 || t.Idx >= len(inScope.cols) { - panic(builderError{pgerror.NewErrorf(pgerror.CodeUndefinedColumnError, - "invalid column ordinal: @%d", t.Idx+1)}) + panic(pgerror.NewErrorf(pgerror.CodeUndefinedColumnError, + "invalid column ordinal: @%d", t.Idx+1)) } out = b.factory.ConstructVariable(inScope.cols[t.Idx].id) @@ -441,7 +441,7 @@ func (b *Builder) buildFunction( } if isAggregate(def) { - panic(assertionErrorf("aggregate function should have been replaced")) + panic(pgerror.NewAssertionErrorf("aggregate function should have been replaced")) } args := make(memo.ScalarListExpr, len(f.Exprs)) @@ -560,10 +560,10 @@ func (b *Builder) checkSubqueryOuterCols( subqueryOuterCols.DifferenceWith(inScope.groupby.aggOutScope.colSet()) colID, _ := subqueryOuterCols.Next(0) col := inScope.getColumn(opt.ColumnID(colID)) - panic(builderError{pgerror.NewErrorf( + panic(pgerror.NewErrorf( pgerror.CodeGroupingError, "subquery uses ungrouped column \"%s\" from outer query", - tree.ErrString(&col.name))}) + tree.ErrString(&col.name))) } } } diff --git a/pkg/sql/opt/optbuilder/scope.go b/pkg/sql/opt/optbuilder/scope.go index 43028dc26d75..1e5a3b64c187 100644 --- a/pkg/sql/opt/optbuilder/scope.go +++ b/pkg/sql/opt/optbuilder/scope.go @@ -1109,11 +1109,11 @@ func (s *scope) replaceSubquery( n := len(outScope.cols) switch desiredColumns { case 1: - panic(builderError{pgerror.NewErrorf(pgerror.CodeSyntaxError, - "subquery must return only one column, found %d", n)}) + panic(pgerror.NewErrorf(pgerror.CodeSyntaxError, + "subquery must return only one column, found %d", n)) default: - panic(builderError{pgerror.NewErrorf(pgerror.CodeSyntaxError, - "subquery must return %d columns, found %d", desiredColumns, n)}) + panic(pgerror.NewErrorf(pgerror.CodeSyntaxError, + "subquery must return %d columns, found %d", desiredColumns, n)) } } @@ -1143,25 +1143,25 @@ var _ tree.IndexedVarContainer = &scope{} // IndexedVarEval is part of the IndexedVarContainer interface. func (s *scope) IndexedVarEval(idx int, ctx *tree.EvalContext) (tree.Datum, error) { - panic(assertionErrorf("unimplemented: scope.IndexedVarEval")) + panic(pgerror.NewAssertionErrorf("unimplemented: scope.IndexedVarEval")) } // IndexedVarResolvedType is part of the IndexedVarContainer interface. func (s *scope) IndexedVarResolvedType(idx int) types.T { if idx >= len(s.cols) { if len(s.cols) == 0 { - panic(builderError{pgerror.NewErrorf(pgerror.CodeUndefinedColumnError, - "column reference @%d not allowed in this context", idx+1)}) + panic(pgerror.NewErrorf(pgerror.CodeUndefinedColumnError, + "column reference @%d not allowed in this context", idx+1)) } - panic(builderError{pgerror.NewErrorf(pgerror.CodeUndefinedColumnError, - "invalid column ordinal: @%d", idx+1)}) + panic(pgerror.NewErrorf(pgerror.CodeUndefinedColumnError, + "invalid column ordinal: @%d", idx+1)) } return s.cols[idx].typ } // IndexedVarNodeFormatter is part of the IndexedVarContainer interface. func (s *scope) IndexedVarNodeFormatter(idx int) tree.NodeFormatter { - panic(assertionErrorf("unimplemented: scope.IndexedVarNodeFormatter")) + panic(pgerror.NewAssertionErrorf("unimplemented: scope.IndexedVarNodeFormatter")) } // newAmbiguousColumnError returns an error with a helpful error message to be diff --git a/pkg/sql/opt/optbuilder/select.go b/pkg/sql/opt/optbuilder/select.go index b64690ca2153..d3f3aafa432e 100644 --- a/pkg/sql/opt/optbuilder/select.go +++ b/pkg/sql/opt/optbuilder/select.go @@ -97,7 +97,7 @@ func (b *Builder) buildDataSource( case cat.Sequence: return b.buildSequenceSelect(t, inScope) default: - panic(assertionErrorf("unknown DataSource type %T", ds)) + panic(pgerror.NewAssertionErrorf("unknown DataSource type %T", ds)) } case *tree.ParenTableExpr: @@ -120,8 +120,8 @@ func (b *Builder) buildDataSource( case *tree.StatementSource: outScope = b.buildStmt(source.Statement, inScope) if len(outScope.cols) == 0 { - panic(builderError{pgerror.NewErrorf(pgerror.CodeUndefinedColumnError, - "statement source \"%v\" does not return any columns", source.Statement)}) + panic(pgerror.NewErrorf(pgerror.CodeUndefinedColumnError, + "statement source \"%v\" does not return any columns", source.Statement)) } return outScope @@ -137,7 +137,7 @@ func (b *Builder) buildDataSource( return outScope default: - panic(assertionErrorf("unknown table expr: %T", texpr)) + panic(pgerror.NewAssertionErrorf("unknown table expr: %T", texpr)) } } @@ -159,7 +159,7 @@ func (b *Builder) buildView(view cat.View, inScope *scope) (outScope *scope) { sel, ok = stmt.AST.(*tree.Select) if !ok { - panic(assertionErrorf("expected SELECT statement")) + panic(pgerror.NewAssertionErrorf("expected SELECT statement")) } b.views[view] = sel @@ -226,11 +226,11 @@ func (b *Builder) renameSource(as tree.AliasClause, scope *scope) { for colIdx, aliasIdx := 0, 0; aliasIdx < len(colAlias); colIdx++ { if colIdx >= len(scope.cols) { srcName := tree.ErrString(&tableAlias) - panic(builderError{pgerror.NewErrorf( + panic(pgerror.NewErrorf( pgerror.CodeInvalidColumnReferenceError, "source %q has %d columns available but %d columns specified", srcName, aliasIdx, len(colAlias), - )}) + )) } col := &scope.cols[colIdx] if col.hidden { @@ -259,8 +259,8 @@ func (b *Builder) buildScanFromTableRef( tab cat.Table, ref *tree.TableRef, indexFlags *tree.IndexFlags, inScope *scope, ) (outScope *scope) { if ref.Columns != nil && len(ref.Columns) == 0 { - panic(builderError{pgerror.NewErrorf(pgerror.CodeSyntaxError, - "an explicit list of column IDs must include at least one column")}) + panic(pgerror.NewErrorf(pgerror.CodeSyntaxError, + "an explicit list of column IDs must include at least one column")) } // See tree.TableRef: "Note that a nil [Columns] array means 'unspecified' @@ -281,8 +281,8 @@ func (b *Builder) buildScanFromTableRef( ord++ } if ord >= cnt { - panic(builderError{pgerror.NewErrorf(pgerror.CodeUndefinedColumnError, - "column [%d] does not exist", c)}) + panic(pgerror.NewErrorf(pgerror.CodeUndefinedColumnError, + "column [%d] does not exist", c)) } ordinals[i] = ord } @@ -351,8 +351,8 @@ func (b *Builder) buildScan( if tab.IsVirtualTable() { if indexFlags != nil { - panic(builderError{pgerror.NewErrorf(pgerror.CodeSyntaxError, - "index flags not allowed with virtual tables")}) + panic(pgerror.NewErrorf(pgerror.CodeSyntaxError, + "index flags not allowed with virtual tables")) } private := memo.VirtualScanPrivate{Table: tabID, Cols: tabColIDs} outScope.expr = b.factory.ConstructVirtualScan(&private) @@ -477,8 +477,8 @@ func (b *Builder) buildCTE(ctes []*tree.CTE, inScope *scope) (outScope *scope) { } if len(cols) == 0 { - panic(builderError{pgerror.NewErrorf(pgerror.CodeFeatureNotSupportedError, - "WITH clause %q does not have a RETURNING clause", tree.ErrString(&name))}) + panic(pgerror.NewErrorf(pgerror.CodeFeatureNotSupportedError, + "WITH clause %q does not have a RETURNING clause", tree.ErrString(&name))) } outScope.ctes[ctes[i].Name.Alias.String()] = &cteSource{ @@ -498,8 +498,8 @@ func (b *Builder) buildCTE(ctes []*tree.CTE, inScope *scope) (outScope *scope) { func (b *Builder) checkCTEUsage(inScope *scope) { for alias, source := range inScope.ctes { if !source.used && source.expr.Relational().CanMutate { - panic(builderError{pgerror.UnimplementedWithIssueErrorf(24307, - "common table expression %q with side effects was not used in query", alias)}) + panic(pgerror.UnimplementedWithIssueErrorf(24307, + "common table expression %q with side effects was not used in query", alias)) } } } @@ -527,7 +527,7 @@ func (b *Builder) buildSelectStmt( return b.buildValuesClause(stmt, desiredTypes, inScope) default: - panic(assertionErrorf("unknown select statement type: %T", stmt)) + panic(pgerror.NewAssertionErrorf("unknown select statement type: %T", stmt)) } } @@ -551,25 +551,24 @@ func (b *Builder) buildSelect( // (WITH ... (WITH ...)) // Currently we are unable to nest the scopes inside ParenSelect so we // must refuse the syntax so that the query does not get invalid results. - panic(builderError{pgerror.UnimplementedWithIssueError(24303, - "multiple WITH clauses in parentheses")}) + panic(pgerror.UnimplementedWithIssueError(24303, "multiple WITH clauses in parentheses")) } with = s.Select.With } wrapped = stmt.Select if stmt.OrderBy != nil { if orderBy != nil { - panic(builderError{pgerror.NewErrorf( + panic(pgerror.NewErrorf( pgerror.CodeSyntaxError, "multiple ORDER BY clauses not allowed", - )}) + )) } orderBy = stmt.OrderBy } if stmt.Limit != nil { if limit != nil { - panic(builderError{pgerror.NewErrorf( + panic(pgerror.NewErrorf( pgerror.CodeSyntaxError, "multiple LIMIT clauses not allowed", - )}) + )) } limit = stmt.Limit } @@ -770,8 +769,8 @@ func (b *Builder) validateAsOf(asOf tree.AsOfClause) { } if b.semaCtx.AsOfTimestamp == nil { - panic(builderError{pgerror.NewErrorf(pgerror.CodeSyntaxError, - "AS OF SYSTEM TIME must be provided on a top-level statement")}) + panic(pgerror.NewErrorf(pgerror.CodeSyntaxError, + "AS OF SYSTEM TIME must be provided on a top-level statement")) } if *b.semaCtx.AsOfTimestamp != ts { diff --git a/pkg/sql/opt/optbuilder/srfs.go b/pkg/sql/opt/optbuilder/srfs.go index 658727a179ca..77a2acca3f57 100644 --- a/pkg/sql/opt/optbuilder/srfs.go +++ b/pkg/sql/opt/optbuilder/srfs.go @@ -56,7 +56,7 @@ func (s *srf) TypeCheck(ctx *tree.SemaContext, desired types.T) (tree.TypedExpr, // Eval is part of the tree.TypedExpr interface. func (s *srf) Eval(_ *tree.EvalContext) (tree.Datum, error) { - panic(assertionErrorf("srf must be replaced before evaluation")) + panic(pgerror.NewAssertionErrorf("srf must be replaced before evaluation")) } var _ tree.Expr = &srf{} diff --git a/pkg/sql/opt/optbuilder/subquery.go b/pkg/sql/opt/optbuilder/subquery.go index 893a4776fd7e..c11bb3019955 100644 --- a/pkg/sql/opt/optbuilder/subquery.go +++ b/pkg/sql/opt/optbuilder/subquery.go @@ -167,7 +167,7 @@ func (s *subquery) ResolvedType() types.T { // Eval is part of the tree.TypedExpr interface. func (s *subquery) Eval(_ *tree.EvalContext) (tree.Datum, error) { - panic(assertionErrorf("subquery must be replaced before evaluation")) + panic(pgerror.NewAssertionErrorf("subquery must be replaced before evaluation")) } // buildSubqueryProjection ensures that a subquery returns exactly one column. @@ -185,8 +185,8 @@ func (b *Builder) buildSubqueryProjection( // This can be obtained with: // CREATE TABLE t(x INT); ALTER TABLE t DROP COLUMN x; // SELECT (SELECT * FROM t) = (SELECT * FROM t); - panic(builderError{pgerror.NewErrorf(pgerror.CodeSyntaxError, - "subquery must return only one column")}) + panic(pgerror.NewErrorf(pgerror.CodeSyntaxError, + "subquery must return only one column")) case 1: outScope.cols = append(outScope.cols, s.cols[0]) diff --git a/pkg/sql/opt/optbuilder/union.go b/pkg/sql/opt/optbuilder/union.go index a27d1ed9cf01..839d690e9727 100644 --- a/pkg/sql/opt/optbuilder/union.go +++ b/pkg/sql/opt/optbuilder/union.go @@ -42,11 +42,11 @@ func (b *Builder) buildUnion( // Check that the number of columns matches. if len(leftScope.cols) != len(rightScope.cols) { - panic(builderError{pgerror.NewErrorf( + panic(pgerror.NewErrorf( pgerror.CodeSyntaxError, "each %v query must have the same number of columns: %d vs %d", clause.Type, len(leftScope.cols), len(rightScope.cols), - )}) + )) } outScope = inScope.push() @@ -80,8 +80,8 @@ func (b *Builder) buildUnion( // but Postgres is more lenient: // http://www.postgresql.org/docs/9.5/static/typeconv-union-case.html. if !(l.typ.Equivalent(r.typ) || l.typ == types.Unknown || r.typ == types.Unknown) { - panic(builderError{pgerror.NewErrorf(pgerror.CodeDatatypeMismatchError, - "%v types %s and %s cannot be matched", clause.Type, l.typ, r.typ)}) + panic(pgerror.NewErrorf(pgerror.CodeDatatypeMismatchError, + "%v types %s and %s cannot be matched", clause.Type, l.typ, r.typ)) } if l.hidden != r.hidden { // This should never happen. diff --git a/pkg/sql/opt/optbuilder/update.go b/pkg/sql/opt/optbuilder/update.go index d0eba1a5add8..ea93cbd27d8d 100644 --- a/pkg/sql/opt/optbuilder/update.go +++ b/pkg/sql/opt/optbuilder/update.go @@ -67,13 +67,13 @@ import ( // become a physical property required of the Update operator). func (b *Builder) buildUpdate(upd *tree.Update, inScope *scope) (outScope *scope) { if upd.OrderBy != nil && upd.Limit == nil { - panic(builderError{pgerror.NewErrorf(pgerror.CodeSyntaxError, - "UPDATE statement requires LIMIT when ORDER BY is used")}) + panic(pgerror.NewErrorf(pgerror.CodeSyntaxError, + "UPDATE statement requires LIMIT when ORDER BY is used")) } // UX friendliness safeguard. if upd.Where == nil && b.evalCtx.SessionData.SafeUpdates { - panic(builderError{pgerror.NewDangerousStatementErrorf("UPDATE without WHERE clause")}) + panic(pgerror.NewDangerousStatementErrorf("UPDATE without WHERE clause")) } if upd.With != nil { @@ -133,7 +133,7 @@ func (b *Builder) buildUpdate(upd *tree.Update, inScope *scope) (outScope *scope // exactly as many columns as are expected by the named SET columns. func (mb *mutationBuilder) addTargetColsForUpdate(exprs tree.UpdateExprs) { if len(mb.targetColList) != 0 { - panic(assertionErrorf("addTargetColsForUpdate cannot be called more than once")) + panic(pgerror.NewAssertionErrorf("addTargetColsForUpdate cannot be called more than once")) } for _, expr := range exprs { @@ -165,9 +165,9 @@ func (mb *mutationBuilder) addTargetColsForUpdate(exprs tree.UpdateExprs) { "source for a multiple-column UPDATE item must be a sub-SELECT or ROW() expression; not supported: %T", expr.Expr)) } if len(expr.Names) != n { - panic(builderError{pgerror.NewErrorf(pgerror.CodeSyntaxError, + panic(pgerror.NewErrorf(pgerror.CodeSyntaxError, "number of columns (%d) does not match number of values (%d)", - len(expr.Names), n)}) + len(expr.Names), n)) } } } diff --git a/pkg/sql/opt/optbuilder/util.go b/pkg/sql/opt/optbuilder/util.go index 4d5dabe8e3aa..e32e6e6b246e 100644 --- a/pkg/sql/opt/optbuilder/util.go +++ b/pkg/sql/opt/optbuilder/util.go @@ -29,8 +29,8 @@ import ( func checkFrom(expr tree.Expr, inScope *scope) { if len(inScope.cols) == 0 { - panic(builderError{pgerror.NewErrorf(pgerror.CodeInvalidNameError, - "cannot use %q without a FROM clause", tree.ErrString(expr))}) + panic(pgerror.NewErrorf(pgerror.CodeInvalidNameError, + "cannot use %q without a FROM clause", tree.ErrString(expr))) } } @@ -238,30 +238,30 @@ func colIndex(numOriginalCols int, expr tree.Expr, context string) int { } ord = val } else { - panic(builderError{pgerror.NewErrorf( + panic(pgerror.NewErrorf( pgerror.CodeSyntaxError, "non-integer constant in %s: %s", context, expr, - )}) + )) } case *tree.DInt: if *i >= 0 { ord = int64(*i) } case *tree.StrVal: - panic(builderError{pgerror.NewErrorf( + panic(pgerror.NewErrorf( pgerror.CodeSyntaxError, "non-integer constant in %s: %s", context, expr, - )}) + )) case tree.Datum: - panic(builderError{pgerror.NewErrorf( + panic(pgerror.NewErrorf( pgerror.CodeSyntaxError, "non-integer constant in %s: %s", context, expr, - )}) + )) } if ord != -1 { if ord < 1 || ord > int64(numOriginalCols) { - panic(builderError{pgerror.NewErrorf( + panic(pgerror.NewErrorf( pgerror.CodeInvalidColumnReferenceError, "%s position %s is not in select list", context, expr, - )}) + )) } ord-- } @@ -307,8 +307,8 @@ func colIdxByProjectionAlias(expr tree.Expr, op string, scope *scope) int { // `SELECT b, * FROM t ORDER BY b`. Otherwise, reject with an // ambiguity error. if scope.cols[j].getExprStr() != scope.cols[index].getExprStr() { - panic(builderError{pgerror.NewErrorf(pgerror.CodeAmbiguousAliasError, - "%s \"%s\" is ambiguous", op, target)}) + panic(pgerror.NewErrorf(pgerror.CodeAmbiguousAliasError, + "%s \"%s\" is ambiguous", op, target)) } // Use the index of the first matching column. continue @@ -402,18 +402,18 @@ func (b *Builder) resolveSchemaForCreate(name *tree.TableName) (cat.Schema, cat. // Remap invalid schema name error text so that it references the catalog // object that could not be created. if pgerr, ok := err.(*pgerror.Error); ok && pgerr.Code == pgerror.CodeInvalidSchemaNameError { - panic(builderError{pgerror.NewErrorf(pgerror.CodeInvalidSchemaNameError, + panic(pgerror.NewErrorf(pgerror.CodeInvalidSchemaNameError, "cannot create %q because the target database or schema does not exist", tree.ErrString(name)). - SetHintf("verify that the current database and search_path are valid and/or the target database exists")}) + SetHintf("verify that the current database and search_path are valid and/or the target database exists")) } panic(builderError{err}) } // Only allow creation of objects in the public schema. if resName.Schema() != tree.PublicSchema { - panic(builderError{pgerror.NewErrorf(pgerror.CodeInvalidNameError, - "schema cannot be modified: %q", tree.ErrString(&resName))}) + panic(pgerror.NewErrorf(pgerror.CodeInvalidNameError, + "schema cannot be modified: %q", tree.ErrString(&resName))) } if err := b.catalog.CheckPrivilege(b.ctx, sch, privilege.CREATE); err != nil { diff --git a/pkg/sql/opt/optbuilder/values.go b/pkg/sql/opt/optbuilder/values.go index 5587df3b6b82..f7fd8bed9c2a 100644 --- a/pkg/sql/opt/optbuilder/values.go +++ b/pkg/sql/opt/optbuilder/values.go @@ -71,8 +71,8 @@ func (b *Builder) buildValuesClause( if colTypes[i] == types.Unknown { colTypes[i] = typ } else if typ != types.Unknown && !typ.Equivalent(colTypes[i]) { - panic(builderError{pgerror.NewErrorf(pgerror.CodeDatatypeMismatchError, - "VALUES types %s and %s cannot be matched", typ, colTypes[i])}) + panic(pgerror.NewErrorf(pgerror.CodeDatatypeMismatchError, + "VALUES types %s and %s cannot be matched", typ, colTypes[i])) } } @@ -95,8 +95,8 @@ func (b *Builder) buildValuesClause( } func reportValuesLenError(expected, actual int) { - panic(builderError{pgerror.NewErrorf( + panic(pgerror.NewErrorf( pgerror.CodeSyntaxError, "VALUES lists must all be the same length, expected %d columns, found %d", - expected, actual)}) + expected, actual)) } diff --git a/pkg/sql/pgwire/pgerror/errors.go b/pkg/sql/pgwire/pgerror/errors.go index 6dbfda7ea81f..b805fd6d0a14 100644 --- a/pkg/sql/pgwire/pgerror/errors.go +++ b/pkg/sql/pgwire/pgerror/errors.go @@ -160,7 +160,7 @@ func NewAssertionErrorf(format string, args ...interface{}) *Error { // NewInternalTrackingError instantiates an error // meant for use with telemetry.ReportError directly. -func NewInternalTrackingError(issue int, detail string) error { +func NewInternalTrackingError(issue int, detail string) *Error { prefix := fmt.Sprintf("#%d.%s", issue, detail) err := NewErrorWithDepthf(1, CodeInternalError, "internal error: %s", prefix) err.InternalCommand = prefix + " " + captureTrace() @@ -221,7 +221,7 @@ func UnimplementedWithIssueDetailError(issue int, detail, msg string) *Error { // but supports message formatting. func UnimplementedWithIssueDetailErrorf( issue int, detail, format string, args ...interface{}, -) error { +) *Error { err := NewErrorWithDepthf(1, CodeFeatureNotSupportedError, "unimplemented: "+format, args...) if detail == "" { err.InternalCommand = fmt.Sprintf("#%d", issue) @@ -234,7 +234,7 @@ func UnimplementedWithIssueDetailErrorf( // UnimplementedWithIssueHintError constructs an error with the given // message, hint, and a link to the passed issue. Recorded as "#" // in tracking. -func UnimplementedWithIssueHintError(issue int, msg, hint string) error { +func UnimplementedWithIssueHintError(issue int, msg, hint string) *Error { err := NewErrorWithDepthf(1, CodeFeatureNotSupportedError, "unimplemented: %s", msg) err.InternalCommand = fmt.Sprintf("#%d", issue) return err.SetHintf("%s\nSee: https://github.com/cockroachdb/cockroach/issues/%d", hint, issue) @@ -275,7 +275,7 @@ func UnimplementedWithDepth(depth int, feature, msg string, args ...interface{}) // if the original error was not a pgerror already. The errContext // string is used to populate the InternalCommand. If InternalCommand // already exists, the errContext is prepended. -func Wrap(err error, code, errContext string) error { +func Wrap(err error, code, errContext string) *Error { var pgErr Error origErr, ok := GetPGCause(err) if ok {