From 4a6da63957f276828127eeb721fb6c044b88a15b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Taylor?= Date: Wed, 13 Dec 2023 12:24:00 +0100 Subject: [PATCH] refactor: remove more errors from operator planning (#14767) --- .../operators/aggregation_pushing.go | 15 +- .../operators/aggregation_pushing_helper.go | 6 +- .../planbuilder/operators/apply_join.go | 15 +- .../vtgate/planbuilder/operators/ast_to_op.go | 14 +- .../vtgate/planbuilder/operators/distinct.go | 2 +- go/vt/vtgate/planbuilder/operators/helpers.go | 8 +- go/vt/vtgate/planbuilder/operators/horizon.go | 9 +- .../operators/horizon_expanding.go | 12 +- go/vt/vtgate/planbuilder/operators/insert.go | 2 +- .../planbuilder/operators/offset_planning.go | 14 +- .../planbuilder/operators/plan_query.go | 9 +- .../planbuilder/operators/projection.go | 10 - .../planbuilder/operators/query_planning.go | 34 +-- .../planbuilder/operators/queryprojection.go | 217 +++++------------- .../operators/queryprojection_test.go | 10 +- .../planbuilder/operators/route_planning.go | 6 +- .../vtgate/planbuilder/operators/subquery.go | 18 +- .../operators/subquery_planning.go | 17 +- go/vt/vtgate/planbuilder/operators/union.go | 50 ++-- go/vt/vtgate/planbuilder/operators/vindex.go | 6 +- 20 files changed, 136 insertions(+), 338 deletions(-) diff --git a/go/vt/vtgate/planbuilder/operators/aggregation_pushing.go b/go/vt/vtgate/planbuilder/operators/aggregation_pushing.go index a1eaf2d18c7..b56d4dbd869 100644 --- a/go/vt/vtgate/planbuilder/operators/aggregation_pushing.go +++ b/go/vt/vtgate/planbuilder/operators/aggregation_pushing.go @@ -403,10 +403,7 @@ func pushAggregationThroughHashJoin(ctx *plancontext.PlanningContext, rootAggr * // The grouping columns need to be pushed down as grouping columns on the respective sides for _, groupBy := range rootAggr.Grouping { - expr, err := rootAggr.QP.GetSimplifiedExpr(ctx, groupBy.Inner) - if err != nil { - panic(err) - } + expr := rootAggr.QP.GetSimplifiedExpr(ctx, groupBy.Inner) deps := ctx.SemTable.RecursiveDeps(expr) switch { case deps.IsSolvedBy(lhs.tableID): @@ -455,10 +452,7 @@ func addColumnsFromLHSInJoinPredicates(ctx *plancontext.PlanningContext, rootAgg for _, pred := range join.JoinPredicates.columns { for _, bve := range pred.LHSExprs { expr := bve.Expr - wexpr, err := rootAggr.QP.GetSimplifiedExpr(ctx, expr) - if err != nil { - panic(err) - } + wexpr := rootAggr.QP.GetSimplifiedExpr(ctx, expr) idx, found := canReuseColumn(ctx, lhs.pushed.Columns, expr, extractExpr) if !found { idx = len(lhs.pushed.Columns) @@ -489,10 +483,7 @@ func splitGroupingToLeftAndRight( columns joinColumns, ) { for _, groupBy := range rootAggr.Grouping { - expr, err := rootAggr.QP.GetSimplifiedExpr(ctx, groupBy.Inner) - if err != nil { - panic(err) - } + expr := rootAggr.QP.GetSimplifiedExpr(ctx, groupBy.Inner) deps := ctx.SemTable.RecursiveDeps(expr) switch { case deps.IsSolvedBy(lhs.tableID): diff --git a/go/vt/vtgate/planbuilder/operators/aggregation_pushing_helper.go b/go/vt/vtgate/planbuilder/operators/aggregation_pushing_helper.go index 81b1faf1d72..eb14f83b7df 100644 --- a/go/vt/vtgate/planbuilder/operators/aggregation_pushing_helper.go +++ b/go/vt/vtgate/planbuilder/operators/aggregation_pushing_helper.go @@ -142,15 +142,15 @@ func (ab *aggBuilder) handleAggr(ctx *plancontext.PlanningContext, aggr Aggr) er // TODO: this should be handled better by pushing the function down. return errAbortAggrPushing case opcode.AggregateUnassigned: - return vterrors.VT12001(fmt.Sprintf("in scatter query: aggregation function '%s'", sqlparser.String(aggr.Original))) + panic(vterrors.VT12001(fmt.Sprintf("in scatter query: aggregation function '%s'", sqlparser.String(aggr.Original)))) case opcode.AggregateGtid: // this is only used for SHOW GTID queries that will never contain joins - return vterrors.VT13001("cannot do join with vgtid") + panic(vterrors.VT13001("cannot do join with vgtid")) case opcode.AggregateSumDistinct, opcode.AggregateCountDistinct: // we are not going to see values multiple times, so we don't need to multiply with the count(*) from the other side return ab.handlePushThroughAggregation(ctx, aggr) default: - return vterrors.VT12001(fmt.Sprintf("aggregation not planned: %s", aggr.OpCode.String())) + panic(vterrors.VT12001(fmt.Sprintf("aggregation not planned: %s", aggr.OpCode.String()))) } } diff --git a/go/vt/vtgate/planbuilder/operators/apply_join.go b/go/vt/vtgate/planbuilder/operators/apply_join.go index b3838ea3e87..79b92687a49 100644 --- a/go/vt/vtgate/planbuilder/operators/apply_join.go +++ b/go/vt/vtgate/planbuilder/operators/apply_join.go @@ -158,11 +158,6 @@ func (aj *ApplyJoin) AddJoinPredicate(ctx *plancontext.PlanningContext, expr sql aj.RHS = rhs } -func (aj *ApplyJoin) pushColRight(ctx *plancontext.PlanningContext, e *sqlparser.AliasedExpr, addToGroupBy bool) (int, error) { - offset := aj.RHS.AddColumn(ctx, true, addToGroupBy, e) - return offset, nil -} - func (aj *ApplyJoin) GetColumns(*plancontext.PlanningContext) []*sqlparser.AliasedExpr { return slice.Map(aj.JoinColumns.columns, func(from applyJoinColumn) *sqlparser.AliasedExpr { return aeWrap(from.Original) @@ -309,7 +304,7 @@ func (aj *ApplyJoin) isColNameMovedFromL2R(bindVarName string) bool { // findOrAddColNameBindVarName goes through the JoinColumns and looks for the given colName coming from the LHS of the join // and returns the argument name if found. if it's not found, a new applyJoinColumn passing this through will be added -func (aj *ApplyJoin) findOrAddColNameBindVarName(ctx *plancontext.PlanningContext, col *sqlparser.ColName) (string, error) { +func (aj *ApplyJoin) findOrAddColNameBindVarName(ctx *plancontext.PlanningContext, col *sqlparser.ColName) string { for i, thisCol := range aj.JoinColumns.columns { idx := slices.IndexFunc(thisCol.LHSExprs, func(e BindVarExpr) bool { return ctx.SemTable.EqualsExpr(e.Expr, col) @@ -324,7 +319,7 @@ func (aj *ApplyJoin) findOrAddColNameBindVarName(ctx *plancontext.PlanningContex expr.Name = bvname aj.JoinColumns.columns[i].LHSExprs[idx] = expr } - return thisCol.LHSExprs[idx].Name, nil + return thisCol.LHSExprs[idx].Name } } for _, thisCol := range aj.JoinPredicates.columns { @@ -332,7 +327,7 @@ func (aj *ApplyJoin) findOrAddColNameBindVarName(ctx *plancontext.PlanningContex return ctx.SemTable.EqualsExpr(e.Expr, col) }) if idx != -1 { - return thisCol.LHSExprs[idx].Name, nil + return thisCol.LHSExprs[idx].Name } } @@ -340,7 +335,7 @@ func (aj *ApplyJoin) findOrAddColNameBindVarName(ctx *plancontext.PlanningContex return ctx.SemTable.EqualsExpr(e.Expr, col) }) if idx != -1 { - return aj.ExtraLHSVars[idx].Name, nil + return aj.ExtraLHSVars[idx].Name } // we didn't find it, so we need to add it @@ -349,7 +344,7 @@ func (aj *ApplyJoin) findOrAddColNameBindVarName(ctx *plancontext.PlanningContex Name: bvName, Expr: col, }) - return bvName, nil + return bvName } func (a *ApplyJoin) LHSColumnsNeeded(ctx *plancontext.PlanningContext) (needed sqlparser.Exprs) { diff --git a/go/vt/vtgate/planbuilder/operators/ast_to_op.go b/go/vt/vtgate/planbuilder/operators/ast_to_op.go index 6e037981ced..63dec0c84a8 100644 --- a/go/vt/vtgate/planbuilder/operators/ast_to_op.go +++ b/go/vt/vtgate/planbuilder/operators/ast_to_op.go @@ -96,7 +96,7 @@ func cloneASTAndSemState[T sqlparser.SQLNode](ctx *plancontext.PlanningContext, // findTablesContained returns the TableSet of all the contained func findTablesContained(ctx *plancontext.PlanningContext, node sqlparser.SQLNode) (result semantics.TableSet) { - _ = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) { + _ = sqlparser.Walk(func(node sqlparser.SQLNode) (bool, error) { t, ok := node.(*sqlparser.AliasedTableExpr) if !ok { return true, nil @@ -113,7 +113,7 @@ func checkForCorrelatedSubqueries( stmt sqlparser.SelectStatement, subqID semantics.TableSet, ) (correlated bool) { - _ = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) { + _ = sqlparser.Walk(func(node sqlparser.SQLNode) (bool, error) { colname, isColname := node.(*sqlparser.ColName) if !isColname { return true, nil @@ -177,9 +177,8 @@ func createOperatorFromUnion(ctx *plancontext.PlanningContext, node *sqlparser.U // 1. verifyAllFKs: For this given statement, do we need to verify validity of all the foreign keys on the vtgate level. // 2. fkToIgnore: The foreign key constraint to specifically ignore while planning the statement. This field is used in UPDATE CASCADE planning, wherein while planning the child update // query, we need to ignore the parent foreign key constraint that caused the cascade in question. -func createOpFromStmt(ctx *plancontext.PlanningContext, stmt sqlparser.Statement, verifyAllFKs bool, fkToIgnore string) Operator { - var err error - ctx, err = plancontext.CreatePlanningContext(stmt, ctx.ReservedVars, ctx.VSchema, ctx.PlannerVersion) +func createOpFromStmt(inCtx *plancontext.PlanningContext, stmt sqlparser.Statement, verifyAllFKs bool, fkToIgnore string) Operator { + ctx, err := plancontext.CreatePlanningContext(stmt, inCtx.ReservedVars, inCtx.VSchema, inCtx.PlannerVersion) if err != nil { panic(err) } @@ -284,10 +283,7 @@ func getOperatorFromAliasedTableExpr(ctx *plancontext.PlanningContext, tableExpr horizon.TableId = &tableID horizon.Alias = tableExpr.As.String() horizon.ColumnAliases = tableExpr.Columns - qp, err := CreateQPFromSelectStatement(ctx, tbl.Select) - if err != nil { - panic(err) - } + qp := CreateQPFromSelectStatement(ctx, tbl.Select) horizon.QP = qp } diff --git a/go/vt/vtgate/planbuilder/operators/distinct.go b/go/vt/vtgate/planbuilder/operators/distinct.go index 74f4495374c..1750846a961 100644 --- a/go/vt/vtgate/planbuilder/operators/distinct.go +++ b/go/vt/vtgate/planbuilder/operators/distinct.go @@ -48,7 +48,7 @@ type ( func (d *Distinct) planOffsets(ctx *plancontext.PlanningContext) Operator { columns := d.GetColumns(ctx) for idx, col := range columns { - e, err := d.QP.GetSimplifiedExpr(ctx, col.Expr) + e, err := d.QP.TryGetSimplifiedExpr(ctx, col.Expr) if err != nil { // ambiguous columns are not a problem for DISTINCT e = col.Expr diff --git a/go/vt/vtgate/planbuilder/operators/helpers.go b/go/vt/vtgate/planbuilder/operators/helpers.go index e5801f6b36f..0049a919e2a 100644 --- a/go/vt/vtgate/planbuilder/operators/helpers.go +++ b/go/vt/vtgate/planbuilder/operators/helpers.go @@ -43,14 +43,14 @@ func compact(ctx *plancontext.PlanningContext, op Operator) Operator { return newOp } -func checkValid(op Operator) error { +func checkValid(op Operator) { type checkable interface { - CheckValid() error + CheckValid() } - return Visit(op, func(this Operator) error { + _ = Visit(op, func(this Operator) error { if chk, ok := this.(checkable); ok { - return chk.CheckValid() + chk.CheckValid() } return nil }) diff --git a/go/vt/vtgate/planbuilder/operators/horizon.go b/go/vt/vtgate/planbuilder/operators/horizon.go index 1a6fc6331ea..f05abb0311b 100644 --- a/go/vt/vtgate/planbuilder/operators/horizon.go +++ b/go/vt/vtgate/planbuilder/operators/horizon.go @@ -197,14 +197,9 @@ func (h *Horizon) src() Operator { } func (h *Horizon) getQP(ctx *plancontext.PlanningContext) *QueryProjection { - if h.QP != nil { - return h.QP - } - qp, err := CreateQPFromSelectStatement(ctx, h.Query) - if err != nil { - panic(err) + if h.QP == nil { + h.QP = CreateQPFromSelectStatement(ctx, h.Query) } - h.QP = qp return h.QP } diff --git a/go/vt/vtgate/planbuilder/operators/horizon_expanding.go b/go/vt/vtgate/planbuilder/operators/horizon_expanding.go index e3ddc5d9232..bbe9323509b 100644 --- a/go/vt/vtgate/planbuilder/operators/horizon_expanding.go +++ b/go/vt/vtgate/planbuilder/operators/horizon_expanding.go @@ -136,10 +136,7 @@ func createProjectionFromSelect(ctx *plancontext.PlanningContext, horizon *Horiz return out } - aggregations, complexAggr, err := qp.AggregationExpressions(ctx, true) - if err != nil { - panic(err) - } + aggregations, complexAggr := qp.AggregationExpressions(ctx, true) a := &Aggregator{ Source: horizon.src(), @@ -249,16 +246,13 @@ func newStarProjection(src Operator, qp *QueryProjection) *Projection { cols := sqlparser.SelectExprs{} for _, expr := range qp.SelectExprs { - err := sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) { + _ = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) { _, isSubQ := node.(*sqlparser.Subquery) if !isSubQ { return true, nil } - return false, vterrors.VT09015() + panic(vterrors.VT09015()) }, expr.Col) - if err != nil { - panic(err) - } cols = append(cols, expr.Col) } diff --git a/go/vt/vtgate/planbuilder/operators/insert.go b/go/vt/vtgate/planbuilder/operators/insert.go index a25d9bc471f..194ef198772 100644 --- a/go/vt/vtgate/planbuilder/operators/insert.go +++ b/go/vt/vtgate/planbuilder/operators/insert.go @@ -333,7 +333,7 @@ func createUniqueKeyComp(ins *sqlparser.Insert, expr sqlparser.Expr, vTbl *vinde return []uComp{{idx, def}}, false } var offsets []uComp - _ = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) { + _ = sqlparser.Walk(func(node sqlparser.SQLNode) (bool, error) { col, ok := node.(*sqlparser.ColName) if !ok { return true, nil diff --git a/go/vt/vtgate/planbuilder/operators/offset_planning.go b/go/vt/vtgate/planbuilder/operators/offset_planning.go index 92c86eaf0ca..3d74059e812 100644 --- a/go/vt/vtgate/planbuilder/operators/offset_planning.go +++ b/go/vt/vtgate/planbuilder/operators/offset_planning.go @@ -64,11 +64,10 @@ func useOffsets(ctx *plancontext.PlanningContext, expr sqlparser.Expr, op Operat in := op.Inputs()[0] found := func(e sqlparser.Expr, offset int) { exprOffset = sqlparser.NewOffset(offset, e) } - notFound := func(e sqlparser.Expr) error { + notFound := func(e sqlparser.Expr) { _, addToGroupBy := e.(*sqlparser.ColName) offset := in.AddColumn(ctx, true, addToGroupBy, aeWrap(e)) exprOffset = sqlparser.NewOffset(offset, e) - return nil } visitor := getOffsetRewritingVisitor(ctx, in.FindCol, found, notFound) @@ -102,11 +101,10 @@ func addColumnsToInput(ctx *plancontext.PlanningContext, root Operator) Operator } addedColumns := false found := func(expr sqlparser.Expr, i int) {} - notFound := func(e sqlparser.Expr) error { + notFound := func(e sqlparser.Expr) { _, addToGroupBy := e.(*sqlparser.ColName) proj.addColumnWithoutPushing(ctx, aeWrap(e), addToGroupBy) addedColumns = true - return nil } visitor := getOffsetRewritingVisitor(ctx, proj.FindCol, found, notFound) @@ -151,13 +149,9 @@ func getOffsetRewritingVisitor( // this function will be called when an expression has been found on the input found func(sqlparser.Expr, int), // if we have an expression that mush be fetched, this method will be called - notFound func(sqlparser.Expr) error, + notFound func(sqlparser.Expr), ) func(node, parent sqlparser.SQLNode) bool { - var err error return func(node, parent sqlparser.SQLNode) bool { - if err != nil { - return false - } e, ok := node.(sqlparser.Expr) if !ok { return true @@ -169,7 +163,7 @@ func getOffsetRewritingVisitor( } if mustFetchFromInput(e) { - err = notFound(e) + notFound(e) return false } diff --git a/go/vt/vtgate/planbuilder/operators/plan_query.go b/go/vt/vtgate/planbuilder/operators/plan_query.go index 8f3abcdc0ec..d5794a1bcf1 100644 --- a/go/vt/vtgate/planbuilder/operators/plan_query.go +++ b/go/vt/vtgate/planbuilder/operators/plan_query.go @@ -68,13 +68,8 @@ func PlanQuery(ctx *plancontext.PlanningContext, stmt sqlparser.Statement) (resu } op = compact(ctx, op) - if err = checkValid(op); err != nil { - return nil, err - } - - if op, err = planQuery(ctx, op); err != nil { - return nil, err - } + checkValid(op) + op = planQuery(ctx, op) _, isRoute := op.(*Route) if !isRoute && ctx.SemTable.NotSingleRouteErr != nil { diff --git a/go/vt/vtgate/planbuilder/operators/projection.go b/go/vt/vtgate/planbuilder/operators/projection.go index bf9eeac0c33..b416e369ca2 100644 --- a/go/vt/vtgate/planbuilder/operators/projection.go +++ b/go/vt/vtgate/planbuilder/operators/projection.go @@ -77,7 +77,6 @@ type ( ProjCols interface { GetColumns() []*sqlparser.AliasedExpr GetSelectExprs() sqlparser.SelectExprs - AddColumn(*sqlparser.AliasedExpr) (ProjCols, int, error) } // Used when there are stars in the expressions that we were unable to expand @@ -137,10 +136,6 @@ func (sp StarProjections) GetColumns() []*sqlparser.AliasedExpr { panic(vterrors.VT09015()) } -func (sp StarProjections) AddColumn(*sqlparser.AliasedExpr) (ProjCols, int, error) { - return nil, 0, vterrors.VT09015() -} - func (sp StarProjections) GetSelectExprs() sqlparser.SelectExprs { return sqlparser.SelectExprs(sp) } @@ -157,11 +152,6 @@ func (ap AliasedProjections) GetSelectExprs() sqlparser.SelectExprs { }) } -func (ap AliasedProjections) AddColumn(col *sqlparser.AliasedExpr) (ProjCols, int, error) { - offset := len(ap) - return append(ap, newProjExpr(col)), offset, nil -} - func (pe *ProjExpr) String() string { var alias, expr, info string if pe.Original.As.NotEmpty() { diff --git a/go/vt/vtgate/planbuilder/operators/query_planning.go b/go/vt/vtgate/planbuilder/operators/query_planning.go index 658f4225756..19f6f3bf27d 100644 --- a/go/vt/vtgate/planbuilder/operators/query_planning.go +++ b/go/vt/vtgate/planbuilder/operators/query_planning.go @@ -33,12 +33,8 @@ type ( } ) -func planQuery(ctx *plancontext.PlanningContext, root Operator) (output Operator, err error) { - output, err = runPhases(ctx, root) - if err != nil { - return nil, err - } - +func planQuery(ctx *plancontext.PlanningContext, root Operator) Operator { + output := runPhases(ctx, root) output = planOffsets(ctx, output) if DebugOperatorTree { @@ -55,7 +51,7 @@ func planQuery(ctx *plancontext.PlanningContext, root Operator) (output Operator // If we can push it under a route - done. // If we can't, we will instead expand the Horizon into // smaller operators and try to push these down as far as possible -func runPhases(ctx *plancontext.PlanningContext, root Operator) (Operator, error) { +func runPhases(ctx *plancontext.PlanningContext, root Operator) Operator { op := root p := phaser{} @@ -66,20 +62,14 @@ func runPhases(ctx *plancontext.PlanningContext, root Operator) (Operator, error } op = phase.act(ctx, op) - - var err error - op, err = runRewriters(ctx, op) - if err != nil { - return nil, err - } - + op = runRewriters(ctx, op) op = compact(ctx, op) } - return addGroupByOnRHSOfJoin(op), nil + return addGroupByOnRHSOfJoin(op) } -func runRewriters(ctx *plancontext.PlanningContext, root Operator) (Operator, error) { +func runRewriters(ctx *plancontext.PlanningContext, root Operator) Operator { visitor := func(in Operator, _ semantics.TableSet, isRoot bool) (Operator, *ApplyResult) { switch in := in.(type) { case *Horizon: @@ -111,7 +101,7 @@ func runRewriters(ctx *plancontext.PlanningContext, root Operator) (Operator, er } } - return FixedPointBottomUp(root, TableID, visitor, stopAtRoute), nil + return FixedPointBottomUp(root, TableID, visitor, stopAtRoute) } func pushLockAndComment(l *LockAndComment) (Operator, *ApplyResult) { @@ -816,25 +806,25 @@ func tryPushUnion(ctx *plancontext.PlanningContext, op *Union) (Operator, *Apply } // addTruncationOrProjectionToReturnOutput uses the original Horizon to make sure that the output columns line up with what the user asked for -func addTruncationOrProjectionToReturnOutput(ctx *plancontext.PlanningContext, oldHorizon Operator, output Operator) (Operator, error) { +func addTruncationOrProjectionToReturnOutput(ctx *plancontext.PlanningContext, oldHorizon Operator, output Operator) Operator { horizon, ok := oldHorizon.(*Horizon) if !ok { - return output, nil + return output } cols := output.GetSelectExprs(ctx) sel := sqlparser.GetFirstSelect(horizon.Query) if len(sel.SelectExprs) == len(cols) { - return output, nil + return output } if tryTruncateColumnsAt(output, len(sel.SelectExprs)) { - return output, nil + return output } qp := horizon.getQP(ctx) proj := createSimpleProjection(ctx, qp, output) - return proj, nil + return proj } func stopAtRoute(operator Operator) VisitRule { diff --git a/go/vt/vtgate/planbuilder/operators/queryprojection.go b/go/vt/vtgate/planbuilder/operators/queryprojection.go index f9f6f7fa15d..163a1213985 100644 --- a/go/vt/vtgate/planbuilder/operators/queryprojection.go +++ b/go/vt/vtgate/planbuilder/operators/queryprojection.go @@ -99,9 +99,9 @@ type ( } AggrRewriter struct { - qp *QueryProjection - st *semantics.SemTable - Err error + qp *QueryProjection + st *semantics.SemTable + failed bool } ) @@ -177,35 +177,26 @@ func (s SelectExpr) GetAliasedExpr() (*sqlparser.AliasedExpr, error) { } // createQPFromSelect creates the QueryProjection for the input *sqlparser.Select -func createQPFromSelect(ctx *plancontext.PlanningContext, sel *sqlparser.Select) (*QueryProjection, error) { +func createQPFromSelect(ctx *plancontext.PlanningContext, sel *sqlparser.Select) *QueryProjection { qp := &QueryProjection{ Distinct: sel.Distinct, } - if err := qp.addSelectExpressions(sel); err != nil { - return nil, err - } - if err := qp.addGroupBy(ctx, sel.GroupBy); err != nil { - return nil, err - } - if err := qp.addOrderBy(ctx, sel.OrderBy); err != nil { - return nil, err - } + qp.addSelectExpressions(sel) + qp.addGroupBy(ctx, sel.GroupBy) + qp.addOrderBy(ctx, sel.OrderBy) if !qp.HasAggr && sel.Having != nil { qp.HasAggr = containsAggr(sel.Having.Expr) } + qp.calculateDistinct(ctx) - if err := qp.calculateDistinct(ctx); err != nil { - return nil, err - } - - return qp, nil + return qp } // RewriteDown stops the walker from entering inside aggregation functions func (ar *AggrRewriter) RewriteDown() func(sqlparser.SQLNode, sqlparser.SQLNode) bool { return func(node, _ sqlparser.SQLNode) bool { - if ar.Err != nil { + if ar.failed { return true } _, ok := node.(sqlparser.AggrFunc) @@ -216,7 +207,7 @@ func (ar *AggrRewriter) RewriteDown() func(sqlparser.SQLNode, sqlparser.SQLNode) // RewriteUp will go through an expression, add aggregations to the QP, and rewrite them to use column offset func (ar *AggrRewriter) RewriteUp() func(*sqlparser.Cursor) bool { return func(cursor *sqlparser.Cursor) bool { - if ar.Err != nil { + if ar.failed { return false } sqlNode := cursor.Node() @@ -227,7 +218,7 @@ func (ar *AggrRewriter) RewriteUp() func(*sqlparser.Cursor) bool { for offset, expr := range ar.qp.SelectExprs { ae, err := expr.GetAliasedExpr() if err != nil { - ar.Err = err + ar.failed = true return false } if ar.st.EqualsExprWithDeps(ae.Expr, fExp) { @@ -257,14 +248,11 @@ func (qp *QueryProjection) AggrRewriter(ctx *plancontext.PlanningContext) *AggrR } } -func (qp *QueryProjection) addSelectExpressions(sel *sqlparser.Select) error { +func (qp *QueryProjection) addSelectExpressions(sel *sqlparser.Select) { for _, selExp := range sel.SelectExprs { switch selExp := selExp.(type) { case *sqlparser.AliasedExpr: - err := checkForInvalidAggregations(selExp) - if err != nil { - return err - } + checkForInvalidAggregations(selExp) col := SelectExpr{ Col: selExp, } @@ -281,10 +269,9 @@ func (qp *QueryProjection) addSelectExpressions(sel *sqlparser.Select) error { } qp.SelectExprs = append(qp.SelectExprs, col) default: - return vterrors.VT13001(fmt.Sprintf("%T in select list", selExp)) + panic(vterrors.VT13001(fmt.Sprintf("%T in select list", selExp))) } } - return nil } func containsAggr(e sqlparser.SQLNode) (hasAggr bool) { @@ -307,21 +294,14 @@ func containsAggr(e sqlparser.SQLNode) (hasAggr bool) { } // createQPFromUnion creates the QueryProjection for the input *sqlparser.Union -func createQPFromUnion(ctx *plancontext.PlanningContext, union *sqlparser.Union) (*QueryProjection, error) { +func createQPFromUnion(ctx *plancontext.PlanningContext, union *sqlparser.Union) *QueryProjection { qp := &QueryProjection{} sel := sqlparser.GetFirstSelect(union) - err := qp.addSelectExpressions(sel) - if err != nil { - return nil, err - } + qp.addSelectExpressions(sel) + qp.addOrderBy(ctx, union.OrderBy) - err = qp.addOrderBy(ctx, union.OrderBy) - if err != nil { - return nil, err - } - - return qp, nil + return qp } type expressionSet struct { @@ -341,14 +321,11 @@ func (es *expressionSet) add(ctx *plancontext.PlanningContext, e sqlparser.Expr) return true } -func (qp *QueryProjection) addOrderBy(ctx *plancontext.PlanningContext, orderBy sqlparser.OrderBy) error { +func (qp *QueryProjection) addOrderBy(ctx *plancontext.PlanningContext, orderBy sqlparser.OrderBy) { canPushSorting := true es := &expressionSet{} for _, order := range orderBy { - simpleExpr, err := qp.GetSimplifiedExpr(ctx, order.Expr) - if err != nil { - return err - } + simpleExpr := qp.GetSimplifiedExpr(ctx, order.Expr) if sqlparser.IsNull(simpleExpr) { // ORDER BY null can safely be ignored continue @@ -363,15 +340,11 @@ func (qp *QueryProjection) addOrderBy(ctx *plancontext.PlanningContext, orderBy canPushSorting = canPushSorting && !containsAggr(simpleExpr) } qp.CanPushSorting = canPushSorting - return nil } -func (qp *QueryProjection) calculateDistinct(ctx *plancontext.PlanningContext) error { +func (qp *QueryProjection) calculateDistinct(ctx *plancontext.PlanningContext) { if qp.Distinct && !qp.HasAggr { - distinct, err := qp.useGroupingOverDistinct(ctx) - if err != nil { - return err - } + distinct := qp.useGroupingOverDistinct(ctx) if distinct { // if order by exists with overlap with select expressions, we can use the aggregation with ordering over distinct. qp.Distinct = false @@ -387,7 +360,7 @@ func (qp *QueryProjection) calculateDistinct(ctx *plancontext.PlanningContext) e } if !qp.Distinct || len(qp.groupByExprs) == 0 { - return nil + return } for _, gb := range qp.groupByExprs { @@ -399,27 +372,20 @@ func (qp *QueryProjection) calculateDistinct(ctx *plancontext.PlanningContext) e return getExpr }) if !found { - return nil + return } } // since we are returning all grouping expressions, we know the results are guaranteed to be unique qp.Distinct = false - return nil } -func (qp *QueryProjection) addGroupBy(ctx *plancontext.PlanningContext, groupBy sqlparser.GroupBy) error { +func (qp *QueryProjection) addGroupBy(ctx *plancontext.PlanningContext, groupBy sqlparser.GroupBy) { es := &expressionSet{} for _, group := range groupBy { selectExprIdx := qp.FindSelectExprIndexForExpr(ctx, group) - simpleExpr, err := qp.GetSimplifiedExpr(ctx, group) - if err != nil { - return err - } - - if err = checkForInvalidGroupingExpressions(simpleExpr); err != nil { - return err - } + simpleExpr := qp.GetSimplifiedExpr(ctx, group) + checkForInvalidGroupingExpressions(simpleExpr) if !es.add(ctx, simpleExpr) { continue @@ -430,7 +396,6 @@ func (qp *QueryProjection) addGroupBy(ctx *plancontext.PlanningContext, groupBy qp.groupByExprs = append(qp.groupByExprs, groupBy) } - return nil } // GetGrouping returns a copy of the grouping parameters of the QP @@ -438,15 +403,15 @@ func (qp *QueryProjection) GetGrouping() []GroupBy { return slices.Clone(qp.groupByExprs) } -func checkForInvalidAggregations(exp *sqlparser.AliasedExpr) error { - return sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) { +func checkForInvalidAggregations(exp *sqlparser.AliasedExpr) { + _ = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) { aggrFunc, isAggregate := node.(sqlparser.AggrFunc) if !isAggregate { return true, nil } args := aggrFunc.GetArgs() if args != nil && len(args) != 1 { - return false, vterrors.VT03001(sqlparser.String(node)) + panic(vterrors.VT03001(sqlparser.String(node))) } return true, nil @@ -463,7 +428,15 @@ func (qp *QueryProjection) isExprInGroupByExprs(ctx *plancontext.PlanningContext } // GetSimplifiedExpr takes an expression used in ORDER BY or GROUP BY, and returns an expression that is simpler to evaluate -func (qp *QueryProjection) GetSimplifiedExpr(ctx *plancontext.PlanningContext, e sqlparser.Expr) (found sqlparser.Expr, err error) { +func (qp *QueryProjection) GetSimplifiedExpr(ctx *plancontext.PlanningContext, e sqlparser.Expr) sqlparser.Expr { + expr, err := qp.TryGetSimplifiedExpr(ctx, e) + if err != nil { + panic(err) + } + return expr +} + +func (qp *QueryProjection) TryGetSimplifiedExpr(ctx *plancontext.PlanningContext, e sqlparser.Expr) (found sqlparser.Expr, err error) { if qp == nil { return e, nil } @@ -560,83 +533,6 @@ func (qp *QueryProjection) NeedsAggregation() bool { return qp.HasAggr || len(qp.groupByExprs) > 0 } -// NeedsProjecting returns true if we have projections that need to be evaluated at the vtgate level -// and can't be pushed down to MySQL -func (qp *QueryProjection) NeedsProjecting( - ctx *plancontext.PlanningContext, - pusher func(expr *sqlparser.AliasedExpr) (int, error), -) (needsVtGateEval bool, expressions []sqlparser.Expr, colNames []string, err error) { - for _, se := range qp.SelectExprs { - var ae *sqlparser.AliasedExpr - ae, err = se.GetAliasedExpr() - if err != nil { - return false, nil, nil, err - } - - expr := ae.Expr - colNames = append(colNames, ae.ColumnName()) - - if _, isCol := expr.(*sqlparser.ColName); isCol { - offset, err := pusher(ae) - if err != nil { - return false, nil, nil, err - } - expressions = append(expressions, sqlparser.NewOffset(offset, expr)) - continue - } - - stopOnError := func(sqlparser.SQLNode, sqlparser.SQLNode) bool { - return err == nil - } - rewriter := func(cursor *sqlparser.CopyOnWriteCursor) { - col, isCol := cursor.Node().(*sqlparser.ColName) - if !isCol { - return - } - var tableInfo semantics.TableInfo - tableInfo, err = ctx.SemTable.TableInfoForExpr(col) - if err != nil { - return - } - dt, isDT := tableInfo.(*semantics.DerivedTable) - if !isDT { - return - } - - rewritten := semantics.RewriteDerivedTableExpression(col, dt) - if containsAggr(rewritten) { - offset, tErr := pusher(&sqlparser.AliasedExpr{Expr: col}) - if tErr != nil { - err = tErr - return - } - cursor.Replace(sqlparser.NewOffset(offset, col)) - } - } - newExpr := sqlparser.CopyOnRewrite(expr, stopOnError, rewriter, nil) - - if err != nil { - return - } - - if newExpr != expr { - // if we changed the expression, it means that we have to evaluate the rest at the vtgate level - expressions = append(expressions, newExpr.(sqlparser.Expr)) - needsVtGateEval = true - continue - } - - // we did not need to push any parts of this expression down. Let's check if we can push all of it - offset, err := pusher(ae) - if err != nil { - return false, nil, nil, err - } - expressions = append(expressions, sqlparser.NewOffset(offset, expr)) - } - - return -} - func (qp *QueryProjection) onlyAggr() bool { if !qp.HasAggr { return false @@ -660,7 +556,7 @@ func (qp *QueryProjection) NeedsDistinct() bool { return true } -func (qp *QueryProjection) AggregationExpressions(ctx *plancontext.PlanningContext, allowComplexExpression bool) (out []Aggr, complex bool, err error) { +func (qp *QueryProjection) AggregationExpressions(ctx *plancontext.PlanningContext, allowComplexExpression bool) (out []Aggr, complex bool) { qp.addOrderByToSelect(ctx) addAggr := func(a Aggr) { out = append(out, a) @@ -674,7 +570,7 @@ func (qp *QueryProjection) AggregationExpressions(ctx *plancontext.PlanningConte for idx, expr := range qp.SelectExprs { aliasedExpr, err := expr.GetAliasedExpr() if err != nil { - return nil, false, err + panic(err) } idxCopy := idx @@ -682,7 +578,7 @@ func (qp *QueryProjection) AggregationExpressions(ctx *plancontext.PlanningConte if !containsAggr(expr.Col) { getExpr, err := expr.GetExpr() if err != nil { - return nil, false, err + panic(err) } if !qp.isExprInGroupByExprs(ctx, getExpr) { aggr := NewAggr(opcode.AggregateAnyValue, nil, aliasedExpr, aliasedExpr.ColumnName()) @@ -693,7 +589,7 @@ func (qp *QueryProjection) AggregationExpressions(ctx *plancontext.PlanningConte } _, isAggregate := aliasedExpr.Expr.(sqlparser.AggrFunc) if !isAggregate && !allowComplexExpression { - return nil, false, vterrors.VT12001("in scatter query: complex aggregate expression") + panic(vterrors.VT12001("in scatter query: complex aggregate expression")) } sqlparser.CopyOnRewrite(aliasedExpr.Expr, qp.extractAggr(ctx, idx, aliasedExpr, addAggr, makeComplex), nil, nil) @@ -903,21 +799,18 @@ func (qp *QueryProjection) orderByOverlapWithSelectExpr(ctx *plancontext.Plannin return false } -func (qp *QueryProjection) useGroupingOverDistinct(ctx *plancontext.PlanningContext) (bool, error) { +func (qp *QueryProjection) useGroupingOverDistinct(ctx *plancontext.PlanningContext) bool { if !qp.orderByOverlapWithSelectExpr(ctx) { - return false, nil + return false } var gbs []GroupBy for idx, selExpr := range qp.SelectExprs { ae, err := selExpr.GetAliasedExpr() if err != nil { // not an alias Expr, cannot continue forward. - return false, nil - } - sExpr, err := qp.GetSimplifiedExpr(ctx, ae.Expr) - if err != nil { - return false, err + return false } + sExpr := qp.GetSimplifiedExpr(ctx, ae.Expr) // check if the grouping already exists on that column. found := slices.IndexFunc(qp.groupByExprs, func(gb GroupBy) bool { return ctx.SemTable.EqualsExprWithDeps(gb.SimplifiedExpr, sExpr) @@ -932,18 +825,18 @@ func (qp *QueryProjection) useGroupingOverDistinct(ctx *plancontext.PlanningCont gbs = append(gbs, groupBy) } qp.groupByExprs = append(qp.groupByExprs, gbs...) - return true, nil + return true } -func checkForInvalidGroupingExpressions(expr sqlparser.Expr) error { - return sqlparser.Walk(func(node sqlparser.SQLNode) (bool, error) { +func checkForInvalidGroupingExpressions(expr sqlparser.Expr) { + _ = sqlparser.Walk(func(node sqlparser.SQLNode) (bool, error) { if _, isAggregate := node.(sqlparser.AggrFunc); isAggregate { - return false, vterrors.VT03005(sqlparser.String(expr)) + panic(vterrors.VT03005(sqlparser.String(expr))) } _, isSubQ := node.(*sqlparser.Subquery) arg, isArg := node.(*sqlparser.Argument) if isSubQ || (isArg && strings.HasPrefix(arg.Name, "__sq")) { - return false, vterrors.VT12001("subqueries in GROUP BY") + panic(vterrors.VT12001("subqueries in GROUP BY")) } return true, nil }, expr) @@ -967,12 +860,12 @@ func CompareRefInt(a *int, b *int) bool { return *a < *b } -func CreateQPFromSelectStatement(ctx *plancontext.PlanningContext, stmt sqlparser.SelectStatement) (*QueryProjection, error) { +func CreateQPFromSelectStatement(ctx *plancontext.PlanningContext, stmt sqlparser.SelectStatement) *QueryProjection { switch sel := stmt.(type) { case *sqlparser.Select: return createQPFromSelect(ctx, sel) case *sqlparser.Union: return createQPFromUnion(ctx, sel) } - return nil, vterrors.VT13001("can only create query projection from Union and Select statements") + panic(vterrors.VT13001("can only create query projection from Union and Select statements")) } diff --git a/go/vt/vtgate/planbuilder/operators/queryprojection_test.go b/go/vt/vtgate/planbuilder/operators/queryprojection_test.go index 1319ad7f9f6..517b169bcf8 100644 --- a/go/vt/vtgate/planbuilder/operators/queryprojection_test.go +++ b/go/vt/vtgate/planbuilder/operators/queryprojection_test.go @@ -86,7 +86,7 @@ func TestQP(t *testing.T) { _, err = semantics.Analyze(sel, "", &semantics.FakeSI{}) require.NoError(t, err) - qp, err := createQPFromSelect(ctx, sel) + qp, err := getQPAndError(ctx, sel) if tcase.expErr != "" { require.Error(t, err) require.Contains(t, err.Error(), tcase.expErr) @@ -103,6 +103,12 @@ func TestQP(t *testing.T) { } } +func getQPAndError(ctx *plancontext.PlanningContext, sel *sqlparser.Select) (qp *QueryProjection, err error) { + defer PanicHandler(&err) + qp = createQPFromSelect(ctx, sel) + return +} + func TestQPSimplifiedExpr(t *testing.T) { testCases := []struct { query, expected string @@ -193,7 +199,7 @@ func TestQPSimplifiedExpr(t *testing.T) { _, err = semantics.Analyze(sel, "", &semantics.FakeSI{}) require.NoError(t, err) ctx := &plancontext.PlanningContext{SemTable: semantics.EmptySemTable()} - qp, err := createQPFromSelect(ctx, sel) + qp := createQPFromSelect(ctx, sel) require.NoError(t, err) require.Equal(t, tc.expected[1:], qp.toString()) }) diff --git a/go/vt/vtgate/planbuilder/operators/route_planning.go b/go/vt/vtgate/planbuilder/operators/route_planning.go index 30c187ac955..cb33f4e1f55 100644 --- a/go/vt/vtgate/planbuilder/operators/route_planning.go +++ b/go/vt/vtgate/planbuilder/operators/route_planning.go @@ -20,7 +20,6 @@ import ( "bytes" "io" - "vitess.io/vitess/go/vt/key" querypb "vitess.io/vitess/go/vt/proto/query" topodatapb "vitess.io/vitess/go/vt/proto/topodata" "vitess.io/vitess/go/vt/sqlparser" @@ -96,15 +95,12 @@ func buildVindexTableForDML( return vindexTable, &AnyShardRouting{keyspace: vindexTable.Keyspace} } - var dest key.Destination - var typ topodatapb.TabletType - var err error tblName, ok := table.Alias.Expr.(sqlparser.TableName) if !ok { panic(vterrors.VT12001("multi shard UPDATE with LIMIT")) } - _, _, _, typ, dest, err = ctx.VSchema.FindTableOrVindex(tblName) + _, _, _, typ, dest, err := ctx.VSchema.FindTableOrVindex(tblName) if err != nil { panic(err) } diff --git a/go/vt/vtgate/planbuilder/operators/subquery.go b/go/vt/vtgate/planbuilder/operators/subquery.go index ce33e4dcb9b..2bcf1e97f74 100644 --- a/go/vt/vtgate/planbuilder/operators/subquery.go +++ b/go/vt/vtgate/planbuilder/operators/subquery.go @@ -199,20 +199,20 @@ func (sq *SubQuery) GetMergePredicates() []sqlparser.Expr { return sq.Predicates } -func (sq *SubQuery) settle(ctx *plancontext.PlanningContext, outer Operator) (Operator, error) { +func (sq *SubQuery) settle(ctx *plancontext.PlanningContext, outer Operator) Operator { if !sq.TopLevel { - return nil, subqueryNotAtTopErr + panic(subqueryNotAtTopErr) } if sq.correlated { - return nil, correlatedSubqueryErr + panic(correlatedSubqueryErr) } if sq.IsProjection { if len(sq.GetMergePredicates()) > 0 { // this means that we have a correlated subquery on our hands - return nil, correlatedSubqueryErr + panic(correlatedSubqueryErr) } sq.SubqueryValueName = sq.ArgName - return outer, nil + return outer } return sq.settleFilter(ctx, outer) } @@ -220,12 +220,12 @@ func (sq *SubQuery) settle(ctx *plancontext.PlanningContext, outer Operator) (Op var correlatedSubqueryErr = vterrors.VT12001("correlated subquery is only supported for EXISTS") var subqueryNotAtTopErr = vterrors.VT12001("unmergable subquery can not be inside complex expression") -func (sq *SubQuery) settleFilter(ctx *plancontext.PlanningContext, outer Operator) (Operator, error) { +func (sq *SubQuery) settleFilter(ctx *plancontext.PlanningContext, outer Operator) Operator { if len(sq.Predicates) > 0 { if sq.FilterType != opcode.PulloutExists { - return nil, correlatedSubqueryErr + panic(correlatedSubqueryErr) } - return outer, nil + return outer } hasValuesArg := func() string { @@ -272,7 +272,7 @@ func (sq *SubQuery) settleFilter(ctx *plancontext.PlanningContext, outer Operato return &Filter{ Source: outer, Predicates: predicates, - }, nil + } } func dontEnterSubqueries(node, _ sqlparser.SQLNode) bool { diff --git a/go/vt/vtgate/planbuilder/operators/subquery_planning.go b/go/vt/vtgate/planbuilder/operators/subquery_planning.go index ed0c6bde941..88efdadd266 100644 --- a/go/vt/vtgate/planbuilder/operators/subquery_planning.go +++ b/go/vt/vtgate/planbuilder/operators/subquery_planning.go @@ -78,11 +78,7 @@ func settleSubqueries(ctx *plancontext.PlanningContext, op Operator) Operator { case *SubQueryContainer: outer := op.Outer for _, subq := range op.Inner { - newOuter, err := subq.settle(ctx, outer) - if err != nil { - panic(err) - } - subq.Outer = newOuter + subq.Outer = subq.settle(ctx, outer) outer = subq } return outer, Rewrote("extracted subqueries from subquery container") @@ -324,7 +320,6 @@ func addSubQuery(in Operator, inner *SubQuery) Operator { // this is necessary because we are pushing the subquery into the RHS of the join, and we need to use the argument names // instead of the column names func rewriteOriginalPushedToRHS(ctx *plancontext.PlanningContext, expression sqlparser.Expr, outer *ApplyJoin) sqlparser.Expr { - var err error outerID := TableID(outer.LHS) result := sqlparser.CopyOnRewrite(expression, nil, func(cursor *sqlparser.CopyOnWriteCursor) { col, ok := cursor.Node().(*sqlparser.ColName) @@ -335,17 +330,9 @@ func rewriteOriginalPushedToRHS(ctx *plancontext.PlanningContext, expression sql // this is a dependency we are being fed from the LHS of the join, so we // need to find the argument name for it and use that instead // we can't use the column name directly, because we're in the RHS of the join - name, innerErr := outer.findOrAddColNameBindVarName(ctx, col) - if err != nil { - err = innerErr - cursor.StopTreeWalk() - return - } + name := outer.findOrAddColNameBindVarName(ctx, col) cursor.Replace(sqlparser.NewArgument(name)) }, nil) - if err != nil { - panic(err) - } return result.(sqlparser.Expr) } diff --git a/go/vt/vtgate/planbuilder/operators/union.go b/go/vt/vtgate/planbuilder/operators/union.go index 454a6370c2f..6ce5fe9a9f8 100644 --- a/go/vt/vtgate/planbuilder/operators/union.go +++ b/go/vt/vtgate/planbuilder/operators/union.go @@ -94,10 +94,7 @@ can be found on the same offset. The names of the RHS are discarded. */ func (u *Union) AddPredicate(ctx *plancontext.PlanningContext, expr sqlparser.Expr) Operator { offsets := make(map[string]int) - sel, err := u.GetSelectFor(0) - if err != nil { - panic(err) - } + sel := u.GetSelectFor(0) for i, selectExpr := range sel.SelectExprs { ae, ok := selectExpr.(*sqlparser.AliasedExpr) if !ok { @@ -106,10 +103,7 @@ func (u *Union) AddPredicate(ctx *plancontext.PlanningContext, expr sqlparser.Ex offsets[ae.ColumnName()] = i } - needsFilter, exprPerSource, err := u.predicatePerSource(expr, offsets) - if err != nil { - panic(err) - } + needsFilter, exprPerSource := u.predicatePerSource(expr, offsets) if needsFilter { return &Filter{ Source: u, @@ -124,11 +118,10 @@ func (u *Union) AddPredicate(ctx *plancontext.PlanningContext, expr sqlparser.Ex return u } -func (u *Union) predicatePerSource(expr sqlparser.Expr, offsets map[string]int) (bool, []sqlparser.Expr, error) { +func (u *Union) predicatePerSource(expr sqlparser.Expr, offsets map[string]int) (bool, []sqlparser.Expr) { needsFilter := false exprPerSource := make([]sqlparser.Expr, len(u.Sources)) for i := range u.Sources { - var err error predicate := sqlparser.CopyOnRewrite(expr, nil, func(cursor *sqlparser.CopyOnWriteCursor) { col, ok := cursor.Node().(*sqlparser.ColName) if !ok { @@ -142,39 +135,29 @@ func (u *Union) predicatePerSource(expr sqlparser.Expr, offsets map[string]int) return } - var sel *sqlparser.Select - sel, err = u.GetSelectFor(i) - if err != nil { - cursor.StopTreeWalk() - return - } - + sel := u.GetSelectFor(i) ae, ok := sel.SelectExprs[idx].(*sqlparser.AliasedExpr) if !ok { - err = vterrors.VT09015() - cursor.StopTreeWalk() - return + panic(vterrors.VT09015()) } cursor.Replace(ae.Expr) }, nil).(sqlparser.Expr) - if err != nil || needsFilter { - return needsFilter, nil, err - } + exprPerSource[i] = predicate } - return needsFilter, exprPerSource, nil + return needsFilter, exprPerSource } -func (u *Union) GetSelectFor(source int) (*sqlparser.Select, error) { +func (u *Union) GetSelectFor(source int) *sqlparser.Select { src := u.Sources[source] for { switch op := src.(type) { case *Horizon: - return sqlparser.GetFirstSelect(op.Query), nil + return sqlparser.GetFirstSelect(op.Query) case *Route: src = op.Source default: - return nil, vterrors.VT13001("expected all sources of the UNION to be horizons") + panic(vterrors.VT13001("expected all sources of the UNION to be horizons")) } } } @@ -208,24 +191,19 @@ func (u *Union) AddColumn(ctx *plancontext.PlanningContext, reuse bool, gb bool, panic(vterrors.VT13001(fmt.Sprintf("could not find the argument to the weight_string function: %s", sqlparser.String(wsArg)))) } - outputOffset, err := u.addWeightStringToOffset(ctx, argIdx, gb) - if err != nil { - panic(err) - } - - return outputOffset + return u.addWeightStringToOffset(ctx, argIdx, gb) default: panic(vterrors.VT13001(fmt.Sprintf("only weight_string function is expected - got %s", sqlparser.String(expr)))) } } -func (u *Union) addWeightStringToOffset(ctx *plancontext.PlanningContext, argIdx int, addToGroupBy bool) (outputOffset int, err error) { +func (u *Union) addWeightStringToOffset(ctx *plancontext.PlanningContext, argIdx int, addToGroupBy bool) (outputOffset int) { for i, src := range u.Sources { exprs := u.Selects[i] selectExpr := exprs[argIdx] ae, ok := selectExpr.(*sqlparser.AliasedExpr) if !ok { - return 0, vterrors.VT09015() + panic(vterrors.VT09015()) } thisOffset := src.AddColumn(ctx, false, addToGroupBy, aeWrap(weightStringFor(ae.Expr))) @@ -234,7 +212,7 @@ func (u *Union) addWeightStringToOffset(ctx *plancontext.PlanningContext, argIdx outputOffset = thisOffset } else { if thisOffset != outputOffset { - return 0, vterrors.VT12001("weight_string offsets did not line up for UNION") + panic(vterrors.VT12001("weight_string offsets did not line up for UNION")) } } } diff --git a/go/vt/vtgate/planbuilder/operators/vindex.go b/go/vt/vtgate/planbuilder/operators/vindex.go index 0b49083785a..c16944e7c49 100644 --- a/go/vt/vtgate/planbuilder/operators/vindex.go +++ b/go/vt/vtgate/planbuilder/operators/vindex.go @@ -112,12 +112,10 @@ func (v *Vindex) AddCol(col *sqlparser.ColName) { v.Columns = append(v.Columns, col) } -func (v *Vindex) CheckValid() error { +func (v *Vindex) CheckValid() { if len(v.Table.Predicates) == 0 { - return vterrors.VT09018(wrongWhereCond + " (where clause missing)") + panic(vterrors.VT09018(wrongWhereCond + " (where clause missing)")) } - - return nil } func (v *Vindex) AddPredicate(ctx *plancontext.PlanningContext, expr sqlparser.Expr) Operator {