diff --git a/go/vt/vtgate/planbuilder/aggregation_pushing.go b/go/vt/vtgate/planbuilder/aggregation_pushing.go deleted file mode 100644 index 677fabbac18..00000000000 --- a/go/vt/vtgate/planbuilder/aggregation_pushing.go +++ /dev/null @@ -1,578 +0,0 @@ -/* -Copyright 2022 The Vitess Authors. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package planbuilder - -import ( - "fmt" - "strconv" - - "vitess.io/vitess/go/vt/sqlparser" - "vitess.io/vitess/go/vt/vterrors" - popcode "vitess.io/vitess/go/vt/vtgate/engine/opcode" - "vitess.io/vitess/go/vt/vtgate/planbuilder/operators" - "vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext" -) - -// pushAggregation pushes grouping and aggregation as far down in the tree as possible -// the output `outputAggrsOffset` needs a little explaining: this is the offsets for aggregation - remember -// that aggregation can be broken down into multiple expressions that are later combined. -// this is why this output is a slice of slices -func (hp *horizonPlanning) pushAggregation( - ctx *plancontext.PlanningContext, - plan logicalPlan, - grouping []operators.GroupBy, - aggregations []operators.Aggr, - ignoreOutputOrder bool, -) (output logicalPlan, - groupingOffsets []offsets, - outputAggrsOffset [][]offsets, - pushed bool, - err error) { - pushed = true - switch plan := plan.(type) { - case *route: - output = plan - groupingOffsets, outputAggrsOffset, _, err = pushAggrOnRoute(ctx, plan, aggregations, grouping, ignoreOutputOrder) - return - - case *join: - output = plan - groupingOffsets, outputAggrsOffset, err = hp.pushAggrOnJoin(ctx, plan, grouping, aggregations) - return - - case *semiJoin: - output = plan - groupingOffsets, outputAggrsOffset, pushed, err = hp.pushAggrOnSemiJoin(ctx, plan, grouping, aggregations, ignoreOutputOrder) - return - - case *simpleProjection: - // we just remove the simpleProjection. We are doing an OA on top anyway, so no need to clean up the output columns - return hp.pushAggregation(ctx, plan.input, grouping, aggregations, ignoreOutputOrder) - - case *limit: - // if we are seeing a limit, it's because we are building on top of a derived table. - output = plan - pushed = false - - for _, grp := range grouping { - offset, wOffset, err := wrapAndPushExpr(ctx, grp.Inner, grp.SimplifiedExpr, plan.input) - if err != nil { - return nil, nil, nil, false, err - } - groupingOffsets = append(groupingOffsets, offsets{ - col: offset, - wsCol: wOffset, - }) - } - - for _, aggr := range aggregations { - var offset int - aggrExpr, ok := aggr.Original.Expr.(sqlparser.AggrFunc) - if !ok { - return nil, nil, nil, false, vterrors.VT13001(fmt.Sprintf("unexpected expression: %v", aggr.Original)) - } - - switch aggrExpr.(type) { - case *sqlparser.CountStar: - offset = 0 - default: - if len(aggrExpr.GetArgs()) != 1 { - return nil, nil, nil, false, vterrors.VT13001(fmt.Sprintf("unexpected expression: %v", aggrExpr)) - } - offset, _, err = pushProjection(ctx, &sqlparser.AliasedExpr{Expr: aggrExpr.GetArg() /*As: expr.As*/}, plan.input, true, true, false) - } - - if err != nil { - return nil, nil, nil, false, err - } - - outputAggrsOffset = append(outputAggrsOffset, []offsets{newOffset(offset)}) - } - - return - default: - err = vterrors.VT12001(fmt.Sprintf("using aggregation on top of a %T plan", plan)) - return - } -} - -func pushAggrOnRoute( - ctx *plancontext.PlanningContext, - plan *route, - aggregations []operators.Aggr, - grouping []operators.GroupBy, - ignoreOutputOrder bool, -) ( - groupingOffsets []offsets, - vtgateAggregation [][]offsets, - nonAggrOffsets []offsets, - err error, -) { - columnOrderMatters := !ignoreOutputOrder - sel, isSel := plan.Select.(*sqlparser.Select) - if !isSel { - return nil, nil, nil, vterrors.VT12001("plan aggregation on union") - } - - var groupingCols []int - var reorg = passThrough - - if columnOrderMatters { - // During this first run, we push the projections for the normal columns (not the weigh_string ones, that is) - // in the order that the user asked for it - // sortOffsets also returns a reorgFunc, - // that can be used to rearrange the produced outputs to the original order - var it *sortedIterator - var err error - grouping, reorg, it = sortOffsets(grouping, aggregations) - vtgateAggregation, groupingCols, err = pushAggrsAndGroupingInOrder(ctx, plan, it, sel, vtgateAggregation, groupingCols) - if err != nil { - return nil, nil, nil, err - } - } else { - // if we haven't already pushed the aggregations, now is the time - for _, aggregation := range aggregations { - param := addAggregationToSelect(ctx, sel, aggregation) - vtgateAggregation = append(vtgateAggregation, []offsets{param}) - } - } - - groupingOffsets = make([]offsets, 0, len(grouping)) - for idx, expr := range grouping { - sel.AddGroupBy(expr.Inner) - var pos offsets - if ignoreOutputOrder { - // we have not yet pushed anything, so we need to push the expression first - col, _, err := addExpressionToRoute(ctx, plan, &sqlparser.AliasedExpr{Expr: expr.Inner}, true) - if err != nil { - return nil, nil, nil, err - } - pos = newOffset(col) - } else { - pos = newOffset(groupingCols[idx]) - } - - if expr.SimplifiedExpr != nil && ctx.SemTable.NeedsWeightString(expr.Inner) { - wsExpr := weightStringFor(expr.SimplifiedExpr) - wsCol, _, err := addExpressionToRoute(ctx, plan, &sqlparser.AliasedExpr{Expr: wsExpr}, true) - if err != nil { - return nil, nil, nil, err - } - pos.wsCol = wsCol - sel.AddGroupBy(wsExpr) - } - groupingOffsets = append(groupingOffsets, pos) - } - - groupingOffsets, vtgateAggregation = reorg(groupingOffsets, vtgateAggregation) - return groupingOffsets, vtgateAggregation, nil, nil -} - -func pushAggrsAndGroupingInOrder( - ctx *plancontext.PlanningContext, - plan *route, - it *sortedIterator, - sel *sqlparser.Select, - vtgateAggregation [][]offsets, - groupingCols []int, -) ([][]offsets, []int, error) { - for it.next() { - groupBy, aggregation := it.current() - if aggregation != nil { - param := addAggregationToSelect(ctx, sel, *aggregation) - vtgateAggregation = append(vtgateAggregation, []offsets{param}) - continue - } - if groupBy != nil { - reuseCol := groupBy.InnerIndex == nil - col, _, err := addExpressionToRoute(ctx, plan, groupBy.AsAliasedExpr(), reuseCol) - groupingCols = append(groupingCols, col) - if err != nil { - return nil, nil, err - } - } - } - return vtgateAggregation, groupingCols, nil -} - -// addAggregationToSelect adds the aggregation to the SELECT statement and returns the AggregateParams to be used outside -func addAggregationToSelect(ctx *plancontext.PlanningContext, sel *sqlparser.Select, aggregation operators.Aggr) offsets { - // TODO: removing duplicated aggregation expression should also be done at the join level - for i, expr := range sel.SelectExprs { - aliasedExpr, isAliasedExpr := expr.(*sqlparser.AliasedExpr) - if !isAliasedExpr { - continue - } - if ctx.SemTable.EqualsExpr(aliasedExpr.Expr, aggregation.Original.Expr) { - return newOffset(i) - } - } - - sel.SelectExprs = append(sel.SelectExprs, aggregation.Original) - return newOffset(len(sel.SelectExprs) - 1) -} - -func countStarAggr() *operators.Aggr { - f := &sqlparser.CountStar{} - aggr := operators.NewAggr(popcode.AggregateCountStar, f, &sqlparser.AliasedExpr{Expr: f}, "count(*)") - return &aggr -} - -/* -We push down aggregations using the logic from the paper Orthogonal Optimization of Subqueries and Aggregation, by -Cesar A. Galindo-Legaria and Milind M. Joshi from Microsoft Corp. - -It explains how one can split an aggregation into local aggregates that depend on only one side of the join. -The local aggregates can then be gathered together to produce the global -group by/aggregate query that the user asked for. - -In Vitess, this is particularly useful because it allows us to push aggregation down to the routes, even when -we have to join the results at the vtgate level. Instead of doing all the grouping and aggregation at the -vtgate level, we can offload most of the work to MySQL, and at the vtgate just summarize the results. -*/ -func (hp *horizonPlanning) pushAggrOnJoin( - ctx *plancontext.PlanningContext, - join *join, - grouping []operators.GroupBy, - aggregations []operators.Aggr, -) ([]offsets, [][]offsets, error) { - // First we separate aggregations according to which side the dependencies are coming from - lhsAggrs, rhsAggrs, err := splitAggregationsToLeftAndRight(ctx, aggregations, join) - if err != nil { - return nil, nil, err - } - - // We need to group by the columns used in the join condition. - // If we don't, the LHS will not be able to return the column, and it can't be used to send down to the RHS - lhsCols, err := hp.createGroupingsForColumns(join.LHSColumns) - if err != nil { - return nil, nil, err - } - - // Here we split the grouping depending on if they should with the LHS or RHS of the query - // This is done by using the semantic table and checking dependencies - lhsGrouping, rhsGrouping, groupingOffsets, err := splitGroupingsToLeftAndRight(ctx, join, grouping, lhsCols) - if err != nil { - return nil, nil, err - } - - // If the rhs has no grouping column then a count(*) will return 0 from the query and will get mapped to the record from left hand side. - // This is an incorrect behaviour as the join condition has not matched, so we add a literal 1 to the select query and also group by on it. - // So that only if join condition matches the records will be mapped and returned. - if len(rhsGrouping) == 0 && len(rhsAggrs) != 0 { - l := sqlparser.NewIntLiteral("1") - aExpr := &sqlparser.AliasedExpr{ - Expr: l, - } - offset, _, err := pushProjection(ctx, aExpr, join.Right, true, true, false) - if err != nil { - return nil, nil, err - } - l = sqlparser.NewIntLiteral(strconv.Itoa(offset + 1)) - rhsGrouping = append(rhsGrouping, operators.NewGroupBy(l, nil, nil)) - } - - // Next we push the aggregations to both sides - newLHS, lhsOffsets, lhsAggrOffsets, _, err := hp.filteredPushAggregation(ctx, join.Left, lhsGrouping, lhsAggrs, true) - if err != nil { - return nil, nil, err - } - - newRHS, rhsOffsets, rhsAggrOffsets, _, err := hp.filteredPushAggregation(ctx, join.Right, rhsGrouping, rhsAggrs, true) - if err != nil { - return nil, nil, err - } - join.Left, join.Right = newLHS, newRHS - - // Next, we have to pass through the grouping values through the join and the projection we add on top - // We added new groupings to the LHS because of the join condition, so we don't want to pass through everything, - // just the groupings that are used by operators on top of this current one - wsOutputGrpOffset := len(groupingOffsets) + len(join.Cols) - outputGroupings := make([]offsets, 0, len(groupingOffsets)) - var wsOffsets []int - for _, groupBy := range groupingOffsets { - var offset offsets - var f func(i int) int - if groupBy < 0 { - offset = lhsOffsets[-groupBy-1] - f = func(i int) int { return -(i + 1) } - } else { - offset = rhsOffsets[groupBy-1] - f = func(i int) int { return i + 1 } - } - outputGrouping := newOffset(len(join.Cols)) - join.Cols = append(join.Cols, f(offset.col)) - if offset.wsCol > -1 { - // we add the weight_string calls at the end of the join columns - outputGrouping.wsCol = wsOutputGrpOffset + len(wsOffsets) - wsOffsets = append(wsOffsets, f(offset.wsCol)) - } - outputGroupings = append(outputGroupings, outputGrouping) - } - join.Cols = append(join.Cols, wsOffsets...) - - outputAggrOffsets := make([][]offsets, 0, len(aggregations)) - for idx := range aggregations { - l, r := lhsAggrOffsets[idx], rhsAggrOffsets[idx] - var offSlice []offsets - for _, off := range l { - offSlice = append(offSlice, newOffset(len(join.Cols))) - join.Cols = append(join.Cols, -(off.col + 1)) - } - for _, off := range r { - offSlice = append(offSlice, newOffset(len(join.Cols))) - join.Cols = append(join.Cols, off.col+1) - } - outputAggrOffsets = append(outputAggrOffsets, offSlice) - } - return outputGroupings, outputAggrOffsets, err -} - -/* -pushAggrOnSemiJoin works similarly to pushAggrOnJoin, but it's simpler, because we don't get any inputs from the RHS, -so there are no aggregations or groupings that have to be sent to the RHS - -We do however need to add the columns used in the subquery coming from the LHS to the grouping. -That way we get the aggregation grouped by the column we need to use to decide if the row should -*/ -func (hp *horizonPlanning) pushAggrOnSemiJoin( - ctx *plancontext.PlanningContext, - join *semiJoin, - grouping []operators.GroupBy, - aggregations []operators.Aggr, - ignoreOutputOrder bool, -) ([]offsets, [][]offsets, bool, error) { - // We need to group by the columns used in the join condition. - // If we don't, the LHS will not be able to return the column, and it can't be used to send down to the RHS - lhsCols, err := hp.createGroupingsForColumns(join.LHSColumns) - if err != nil { - return nil, nil, false, err - } - - totalGrouping := append(grouping, lhsCols...) - newLeft, groupingOffsets, aggrParams, pushed, err := hp.pushAggregation(ctx, join.lhs, totalGrouping, aggregations, ignoreOutputOrder) - if err != nil { - return nil, nil, false, err - } - join.lhs = newLeft - - outputGroupings := make([]offsets, 0, len(grouping)) - for idx := range grouping { - outputGroupings = append(outputGroupings, groupingOffsets[idx]) - } - - return outputGroupings, aggrParams, pushed, nil -} - -// this method takes a slice of aggregations that can have missing spots in the form of `nil`, -// and pushes the non-empty values down. -// during aggregation planning, it's important to know which of -// the incoming aggregations correspond to what is sent to the LHS and RHS. -// Some aggregations only need to be sent to one of the sides of the join, and in that case, -// the other side will have a nil in this offset of the aggregations -func (hp *horizonPlanning) filteredPushAggregation( - ctx *plancontext.PlanningContext, - plan logicalPlan, - grouping []operators.GroupBy, - aggregations []*operators.Aggr, - ignoreOutputOrder bool, -) (out logicalPlan, groupingOffsets []offsets, outputAggrs [][]offsets, pushed bool, err error) { - used := make([]bool, len(aggregations)) - var aggrs []operators.Aggr - - for idx, aggr := range aggregations { - if aggr != nil { - used[idx] = true - aggrs = append(aggrs, *aggr) - } - } - newplan, groupingOffsets, pushedAggrs, pushed, err := hp.pushAggregation(ctx, plan, grouping, aggrs, ignoreOutputOrder) - if err != nil { - return nil, nil, nil, pushed, err - } - idx := 0 - for _, b := range used { - if !b { - outputAggrs = append(outputAggrs, nil) - continue - } - outputAggrs = append(outputAggrs, pushedAggrs[idx]) - idx++ - } - return newplan, groupingOffsets, outputAggrs, pushed, nil -} - -func isMinOrMax(in popcode.AggregateOpcode) bool { - switch in { - case popcode.AggregateMin, popcode.AggregateMax: - return true - default: - return false - } -} - -func isAnyValue(in popcode.AggregateOpcode) bool { - return in == popcode.AggregateAnyValue -} - -func splitAggregationsToLeftAndRight( - ctx *plancontext.PlanningContext, - aggregations []operators.Aggr, - join *join, -) ([]*operators.Aggr, []*operators.Aggr, error) { - var lhsAggrs, rhsAggrs []*operators.Aggr - for _, aggr := range aggregations { - newAggr := aggr - if _, ok := aggr.Original.Expr.(*sqlparser.CountStar); ok { - lhsAggrs = append(lhsAggrs, &newAggr) - rhsAggrs = append(rhsAggrs, &newAggr) - } else { - deps := ctx.SemTable.RecursiveDeps(aggr.Original.Expr) - var other *operators.Aggr - // if we are sending down min/max/random, we don't have to multiply the results with anything - if !isMinOrMax(aggr.OpCode) && !isAnyValue(aggr.OpCode) { - other = countStarAggr() - } - switch { - case deps.IsSolvedBy(join.Left.ContainsTables()): - lhsAggrs = append(lhsAggrs, &newAggr) - rhsAggrs = append(rhsAggrs, other) - case deps.IsSolvedBy(join.Right.ContainsTables()): - rhsAggrs = append(rhsAggrs, &newAggr) - lhsAggrs = append(lhsAggrs, other) - default: - return nil, nil, vterrors.VT12001("aggregation on columns from different sources") - } - } - } - return lhsAggrs, rhsAggrs, nil -} - -func splitGroupingsToLeftAndRight( - ctx *plancontext.PlanningContext, - join *join, - grouping, lhsGrouping []operators.GroupBy, -) ([]operators.GroupBy, []operators.GroupBy, []int, error) { - var rhsGrouping []operators.GroupBy - - lhsTS := join.Left.ContainsTables() - rhsTS := join.Right.ContainsTables() - // here we store information about which side the grouping value is coming from. - // Negative values from the left operator and positive values are offsets into the RHS - var groupingOffsets []int - for _, groupBy := range grouping { - deps := ctx.SemTable.RecursiveDeps(groupBy.Inner) - switch { - case deps.IsSolvedBy(lhsTS): - groupingOffsets = append(groupingOffsets, -(len(lhsGrouping) + 1)) - lhsGrouping = append(lhsGrouping, groupBy) - case deps.IsSolvedBy(rhsTS): - groupingOffsets = append(groupingOffsets, len(rhsGrouping)+1) - rhsGrouping = append(rhsGrouping, groupBy) - default: - return nil, nil, nil, vterrors.VT12001("grouping on columns from different sources") - } - } - return lhsGrouping, rhsGrouping, groupingOffsets, nil -} - -type ( - reorgFunc = func(groupByOffsets []offsets, aggrOffsets [][]offsets) ([]offsets, [][]offsets) - sortedIterator struct { - grouping []operators.GroupBy - aggregations []operators.Aggr - valueGB *operators.GroupBy - valueA *operators.Aggr - groupbyIdx int - aggrIdx int - } -) - -func (it *sortedIterator) current() (*operators.GroupBy, *operators.Aggr) { - return it.valueGB, it.valueA -} - -func (it *sortedIterator) next() bool { - if it.aggrIdx < len(it.aggregations) && it.groupbyIdx < len(it.grouping) { - aggregation := it.aggregations[it.aggrIdx] - groupBy := it.grouping[it.groupbyIdx] - if operators.CompareRefInt(aggregation.Index, groupBy.InnerIndex) { - it.aggrIdx++ - it.valueA, it.valueGB = &aggregation, nil - return true - } - it.groupbyIdx++ - it.valueA, it.valueGB = nil, &groupBy - return true - } - - if it.groupbyIdx < len(it.grouping) { - groupBy := it.grouping[it.groupbyIdx] - it.groupbyIdx++ - it.valueA, it.valueGB = nil, &groupBy - return true - } - if it.aggrIdx < len(it.aggregations) { - aggregation := it.aggregations[it.aggrIdx] - it.aggrIdx++ - it.valueA, it.valueGB = &aggregation, nil - return true - } - return false -} - -func passThrough(groupByOffsets []offsets, aggrOffsets [][]offsets) ([]offsets, [][]offsets) { - return groupByOffsets, aggrOffsets -} - -func sortOffsets(grouping []operators.GroupBy, aggregations []operators.Aggr) ([]operators.GroupBy, reorgFunc, *sortedIterator) { - originalGrouping := make([]operators.GroupBy, len(grouping)) - originalAggr := make([]operators.Aggr, len(aggregations)) - copy(originalAggr, aggregations) - copy(originalGrouping, grouping) - operators.SortAggregations(aggregations) - operators.SortGrouping(grouping) - - reorg := func(groupByOffsets []offsets, aggrOffsets [][]offsets) ([]offsets, [][]offsets) { - orderedGroupingOffsets := make([]offsets, 0, len(originalGrouping)) - for _, og := range originalGrouping { - for i, g := range grouping { - if og.Inner == g.Inner { - orderedGroupingOffsets = append(orderedGroupingOffsets, groupByOffsets[i]) - break - } - } - } - - orderedAggrs := make([][]offsets, 0, len(originalAggr)) - for _, og := range originalAggr { - for i, g := range aggregations { - if og.Original.Expr == g.Original.Expr { - orderedAggrs = append(orderedAggrs, aggrOffsets[i]) - break - } - } - } - - return orderedGroupingOffsets, orderedAggrs - } - - return grouping, reorg, &sortedIterator{ - grouping: grouping, - aggregations: aggregations, - } -} diff --git a/go/vt/vtgate/planbuilder/delete.go b/go/vt/vtgate/planbuilder/delete.go index 5fa743e7034..b62c35e9bd1 100644 --- a/go/vt/vtgate/planbuilder/delete.go +++ b/go/vt/vtgate/planbuilder/delete.go @@ -59,7 +59,6 @@ func gen4DeleteStmtPlanner( if ks, tables := ctx.SemTable.SingleUnshardedKeyspace(); ks != nil { if fkManagementNotRequired(ctx, vschema, tables) { plan := deleteUnshardedShortcut(deleteStmt, ks, tables) - plan = pushCommentDirectivesOnPlan(plan, deleteStmt) return newPlanResult(plan.Primitive(), operators.QualifiedTables(ks, tables)...), nil } } diff --git a/go/vt/vtgate/planbuilder/doc.go b/go/vt/vtgate/planbuilder/doc.go deleted file mode 100644 index 77a4a88ed7b..00000000000 --- a/go/vt/vtgate/planbuilder/doc.go +++ /dev/null @@ -1,85 +0,0 @@ -/* -Copyright 2019 The Vitess Authors. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -/* -Package planbuilder allows you to build execution -plans that describe how to fulfill a query that may -span multiple keyspaces or shards. The main entry -points for this package are Build and BuildFromStmt. -*/ -package planbuilder - -/* -The main strategy of the planbuilder is to push down as -much of the work as possible down to the vttablets. The -special primitive for doing this is route, which can -execute any SQL on a single shard (or scatter). Any -work that cannot be done by a single route is stitched -together by VTGate using relational primitives. If -stitching is not possible using existing primitives, -then an "unsupported" error is returned. - -If a query is split into multiple parts, like a -cross-shard join, the latter parts may carry references -to the former parts. If this happens, the primitive -specifies how to build these cross-shard references as -"join variables" that will essentially be sent in -as bind vars during execution. For example: - - select ... from a join b on b.col = a.col - -will be executed as: - - select ... a.col from a (produce "a_col" from a.col) - select ... from b where b.col = :a_col - -The central design element for analyzing queries and -building plans is the symbol table (symtab). This data -structure evolves as a query is analyzed. Therefore, -searches are not repeatable. To resolve this, search -results are persisted inside the ColName as 'Metadata', -and reused as needed. - -The plan is built in two phases. In the -first phase (break-up and push-down), the query is -broken into smaller parts and pushed down into -various primitives. In the second phase (wire-up), -external references are wired up using bind vars, and -the individual ASTs are converted into actual queries. - -In current architecture, VTGate does not know the -underlying MySQL schema. Due to this, we assume that -any qualified or implicit column reference of a table -is valid and we rely on the underlying vttablet/MySQL -to eventually validate such references. - -Every 'logicalPlan' primitive must satisfy the logicalPlan -interface. This allows the planbuilder to outsource -primitive-specific handling into those implementations. - -Variable naming: The AST, planbuilder and engine -are three different worlds that use overloaded -names that are contextually similar, but different. -For example a join is: - Join is the AST node that represents the SQL construct - join is a logicalPlan in the current package - Join is a primitive in the engine package -In order to disambiguate, we'll use the 'a' prefix -for AST vars, and the 'e' prefix for engine vars. -So, 'ajoin' would be of type *sqlparser.Join, and -'ejoin' would be of type *engine.Join. For the planbuilder -join we'll use 'jb'. -*/ diff --git a/go/vt/vtgate/planbuilder/expression_converter.go b/go/vt/vtgate/planbuilder/expression_converter.go index f100d0d93e0..7a9dc374ea6 100644 --- a/go/vt/vtgate/planbuilder/expression_converter.go +++ b/go/vt/vtgate/planbuilder/expression_converter.go @@ -40,7 +40,7 @@ func booleanValues(astExpr sqlparser.Expr) evalengine.Expr { ) switch node := astExpr.(type) { case *sqlparser.Literal: - //set autocommit = 'on' + // set autocommit = 'on' if node.Type == sqlparser.StrVal { switch strings.ToLower(node.Val) { case "on": @@ -50,7 +50,7 @@ func booleanValues(astExpr sqlparser.Expr) evalengine.Expr { } } case *sqlparser.ColName: - //set autocommit = on + // set autocommit = on switch node.Name.Lowered() { case "on": return ON diff --git a/go/vt/vtgate/planbuilder/filter.go b/go/vt/vtgate/planbuilder/filter.go index 589287495a7..c3686380446 100644 --- a/go/vt/vtgate/planbuilder/filter.go +++ b/go/vt/vtgate/planbuilder/filter.go @@ -17,11 +17,7 @@ limitations under the License. package planbuilder import ( - "vitess.io/vitess/go/vt/sqlparser" - "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/engine" - "vitess.io/vitess/go/vt/vtgate/evalengine" - "vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext" ) type ( @@ -34,38 +30,6 @@ type ( var _ logicalPlan = (*filter)(nil) -func resolveFromPlan(ctx *plancontext.PlanningContext, plan logicalPlan, canPushProjection bool) evalengine.ColumnResolver { - return func(expr *sqlparser.ColName) (int, error) { - offset, added, err := pushProjection(ctx, &sqlparser.AliasedExpr{Expr: expr}, plan, true, true, false) - if err != nil { - return 0, err - } - if added && !canPushProjection { - return 0, vterrors.VT13001("column should not be pushed to projection while doing a column lookup") - } - return offset, nil - } -} - -// newFilter builds a new filter. -func newFilter(ctx *plancontext.PlanningContext, plan logicalPlan, expr sqlparser.Expr) (*filter, error) { - predicate, err := evalengine.Translate(expr, &evalengine.Config{ - ResolveColumn: resolveFromPlan(ctx, plan, false), - ResolveType: ctx.SemTable.TypeForExpr, - Collation: ctx.SemTable.Collation, - }) - if err != nil { - return nil, err - } - return &filter{ - logicalPlanCommon: newBuilderCommon(plan), - efilter: &engine.Filter{ - Predicate: predicate, - ASTPredicate: expr, - }, - }, nil -} - // Primitive implements the logicalPlan interface func (l *filter) Primitive() engine.Primitive { l.efilter.Input = l.input.Primitive() diff --git a/go/vt/vtgate/planbuilder/hash_join.go b/go/vt/vtgate/planbuilder/hash_join.go deleted file mode 100644 index 058adcf1965..00000000000 --- a/go/vt/vtgate/planbuilder/hash_join.go +++ /dev/null @@ -1,110 +0,0 @@ -/* -Copyright 2021 The Vitess Authors. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package planbuilder - -import ( - "fmt" - - "vitess.io/vitess/go/mysql/collations" - querypb "vitess.io/vitess/go/vt/proto/query" - "vitess.io/vitess/go/vt/sqlparser" - "vitess.io/vitess/go/vt/vterrors" - "vitess.io/vitess/go/vt/vtgate/engine" - "vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext" - "vitess.io/vitess/go/vt/vtgate/semantics" -) - -var _ logicalPlan = (*hashJoin)(nil) - -// hashJoin is used to build a HashJoin primitive. -type hashJoin struct { - - // Left and Right are the nodes for the join. - Left, Right logicalPlan - - Opcode engine.JoinOpcode - - Cols []int - - // The keys correspond to the column offset in the inputs where - // the join columns can be found - LHSKey, RHSKey int - - ComparisonType querypb.Type - - Collation collations.ID -} - -// Wireup implements the logicalPlan interface -func (hj *hashJoin) Wireup(ctx *plancontext.PlanningContext) error { - err := hj.Left.Wireup(ctx) - if err != nil { - return err - } - return hj.Right.Wireup(ctx) -} - -// Primitive implements the logicalPlan interface -func (hj *hashJoin) Primitive() engine.Primitive { - return &engine.HashJoin{ - Left: hj.Left.Primitive(), - Right: hj.Right.Primitive(), - Cols: hj.Cols, - Opcode: hj.Opcode, - LHSKey: hj.LHSKey, - RHSKey: hj.RHSKey, - ComparisonType: hj.ComparisonType, - Collation: hj.Collation, - } -} - -// Inputs implements the logicalPlan interface -func (hj *hashJoin) Inputs() []logicalPlan { - return []logicalPlan{hj.Left, hj.Right} -} - -// Rewrite implements the logicalPlan interface -func (hj *hashJoin) Rewrite(inputs ...logicalPlan) error { - if len(inputs) != 2 { - return vterrors.VT13001(fmt.Sprintf("wrong number of children in hashJoin rewrite: %d; should be exactly 2", len(inputs))) - } - hj.Left = inputs[0] - hj.Right = inputs[1] - return nil -} - -// ContainsTables implements the logicalPlan interface -func (hj *hashJoin) ContainsTables() semantics.TableSet { - return hj.Left.ContainsTables().Merge(hj.Right.ContainsTables()) -} - -// OutputColumns implements the logicalPlan interface -func (hj *hashJoin) OutputColumns() []sqlparser.SelectExpr { - return getOutputColumnsFromJoin(hj.Cols, hj.Left.OutputColumns(), hj.Right.OutputColumns()) -} - -func getOutputColumnsFromJoin(ints []int, lhs []sqlparser.SelectExpr, rhs []sqlparser.SelectExpr) (cols []sqlparser.SelectExpr) { - for _, col := range ints { - if col < 0 { - col *= -1 - cols = append(cols, lhs[col-1]) - } else { - cols = append(cols, rhs[col-1]) - } - } - return -} diff --git a/go/vt/vtgate/planbuilder/horizon_planning.go b/go/vt/vtgate/planbuilder/horizon_planning.go deleted file mode 100644 index ae559bdbefe..00000000000 --- a/go/vt/vtgate/planbuilder/horizon_planning.go +++ /dev/null @@ -1,1184 +0,0 @@ -/* -Copyright 2019 The Vitess Authors. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package planbuilder - -import ( - "fmt" - - "vitess.io/vitess/go/sqltypes" - "vitess.io/vitess/go/vt/sqlparser" - "vitess.io/vitess/go/vt/vterrors" - "vitess.io/vitess/go/vt/vtgate/engine" - popcode "vitess.io/vitess/go/vt/vtgate/engine/opcode" - "vitess.io/vitess/go/vt/vtgate/planbuilder/operators" - "vitess.io/vitess/go/vt/vtgate/planbuilder/operators/ops" - "vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext" - "vitess.io/vitess/go/vt/vtgate/semantics" -) - -type horizonPlanning struct { - sel *sqlparser.Select - qp *operators.QueryProjection -} - -func (hp *horizonPlanning) planHorizon(ctx *plancontext.PlanningContext, plan logicalPlan, truncateColumns bool) (logicalPlan, error) { - rb, isRoute := plan.(*route) - if !isRoute && ctx.SemTable.NotSingleRouteErr != nil { - // If we got here, we don't have a single shard plan - return nil, ctx.SemTable.NotSingleRouteErr - } - - if isRoute && rb.isSingleShard() { - err := planSingleRoutePlan(hp.sel, rb) - if err != nil { - return nil, err - } - return plan, nil - } - - // If the current plan is a simpleProjection, we want to rewrite derived expression. - // In transformDerivedPlan (operator_transformers.go), derived tables that are not - // a simple route are put behind a simpleProjection. In this simple projection, - // every Route will represent the original derived table. Thus, pushing new expressions - // to those Routes require us to rewrite them. - // On the other hand, when a derived table is a simple Route, we do not put it under - // a simpleProjection. We create a new Route that contains the derived table in the - // FROM clause. Meaning that, when we push expressions to the select list of this - // new Route, we do not want them to rewrite them. - sp, derivedTable := plan.(*simpleProjection) - if derivedTable { - oldRewriteDerivedExpr := ctx.RewriteDerivedExpr - defer func() { - ctx.RewriteDerivedExpr = oldRewriteDerivedExpr - }() - ctx.RewriteDerivedExpr = true - } - - var err error - hp.qp, err = operators.CreateQPFromSelectStatement(ctx, hp.sel) - if err != nil { - return nil, err - } - - needsOrdering := len(hp.qp.OrderExprs) > 0 - - // If we still have a HAVING clause, it's because it could not be pushed to the WHERE, - // so it probably has aggregations - canShortcut := isRoute && hp.sel.Having == nil && !needsOrdering - - switch { - case hp.qp.NeedsAggregation() || hp.sel.Having != nil: - plan, err = hp.planAggregations(ctx, plan) - if err != nil { - return nil, err - } - // if we already did sorting, we don't need to do it again - needsOrdering = needsOrdering && !hp.qp.CanPushSorting - case canShortcut: - err = planSingleRoutePlan(hp.sel, rb) - if err != nil { - return nil, err - } - case derivedTable: - pusher := func(ae *sqlparser.AliasedExpr) (int, error) { - offset, _, err := pushProjection(ctx, ae, sp.input, true, true, false) - return offset, err - } - needsVtGate, projections, colNames, err := hp.qp.NeedsProjecting(ctx, pusher) - if err != nil { - return nil, err - } - if !needsVtGate { - break - } - - // there were some expressions we could not push down entirely, - // so replace the simpleProjection with a real projection - plan = &projection{ - source: sp.input, - columns: projections, - columnNames: colNames, - } - default: - err = pushProjections(ctx, plan, hp.qp.SelectExprs) - if err != nil { - return nil, err - } - } - - // If we didn't already take care of ORDER BY during aggregation planning, we need to handle it now - if needsOrdering { - plan, err = hp.planOrderBy(ctx, hp.qp.OrderExprs, plan) - if err != nil { - return nil, err - } - } - - plan, err = hp.planDistinct(ctx, plan) - if err != nil { - return nil, err - } - - if !truncateColumns { - return plan, nil - } - - plan, err = hp.truncateColumnsIfNeeded(ctx, plan) - if err != nil { - return nil, err - } - - return plan, nil -} - -func pushProjections(ctx *plancontext.PlanningContext, plan logicalPlan, selectExprs []operators.SelectExpr) error { - for _, e := range selectExprs { - aliasExpr, err := e.GetAliasedExpr() - if err != nil { - return err - } - if _, _, err := pushProjection(ctx, aliasExpr, plan, true, false, false); err != nil { - return err - } - } - return nil -} - -func (hp *horizonPlanning) truncateColumnsIfNeeded(ctx *plancontext.PlanningContext, plan logicalPlan) (logicalPlan, error) { - if len(plan.OutputColumns()) == hp.qp.GetColumnCount() { - return plan, nil - } - switch p := plan.(type) { - case *route: - p.eroute.SetTruncateColumnCount(hp.qp.GetColumnCount()) - case *join, *semiJoin, *hashJoin: - // since this is a join, we can safely add extra columns and not need to truncate them - case *orderedAggregate: - p.truncateColumnCount = hp.qp.GetColumnCount() - case *memorySort: - p.truncater.SetTruncateColumnCount(hp.qp.GetColumnCount()) - case *uncorrelatedSubquery: - newUnderlyingPlan, err := hp.truncateColumnsIfNeeded(ctx, p.outer) - if err != nil { - return nil, err - } - p.outer = newUnderlyingPlan - default: - plan = &simpleProjection{ - logicalPlanCommon: newBuilderCommon(plan), - eSimpleProj: &engine.SimpleProjection{}, - } - - exprs := hp.qp.SelectExprs[0:hp.qp.GetColumnCount()] - err := pushProjections(ctx, plan, exprs) - if err != nil { - return nil, err - } - } - return plan, nil -} - -func checkIfAlreadyExists(expr *sqlparser.AliasedExpr, node sqlparser.SelectStatement, semTable *semantics.SemTable) int { - // Here to find if the expr already exists in the SelectStatement, we have 3 cases - // input is a Select -> In this case we want to search in the select - // input is a Union -> In this case we want to search in the First Select of the Union - // input is a Parenthesised Select -> In this case we want to search in the select - // all these three cases are handled by the call to GetFirstSelect. - sel := sqlparser.GetFirstSelect(node) - - exprCol, isExprCol := expr.Expr.(*sqlparser.ColName) - - // first pass - search for aliased expressions - for i, selectExpr := range sel.SelectExprs { - if !isExprCol { - break - } - - selectExpr, ok := selectExpr.(*sqlparser.AliasedExpr) - if ok && selectExpr.As.Equal(exprCol.Name) { - return i - } - } - - // next pass - we are searching the actual expressions and not the aliases - for i, selectExpr := range sel.SelectExprs { - selectExpr, ok := selectExpr.(*sqlparser.AliasedExpr) - if !ok { - continue - } - - if semTable.EqualsExpr(expr.Expr, selectExpr.Expr) { - return i - } - } - return -1 -} - -func (hp *horizonPlanning) planAggregations(ctx *plancontext.PlanningContext, plan logicalPlan) (logicalPlan, error) { - isPushable := !isJoin(plan) - grouping := hp.qp.GetGrouping() - vindexOverlapWithGrouping := hasUniqueVindex(ctx.SemTable, grouping) - if isPushable && vindexOverlapWithGrouping { - // If we have a plan that we can push the group by and aggregation through, we don't need to do aggregation - // at the vtgate level at all - err := hp.planAggregationWithoutOA(ctx, plan) - if err != nil { - return nil, err - } - resultPlan, err := hp.planOrderBy(ctx, hp.qp.OrderExprs, plan) - if err != nil { - return nil, err - } - - newPlan, err := hp.planHaving(ctx, resultPlan) - if err != nil { - return nil, err - } - - return newPlan, nil - } - - return hp.planAggrUsingOA(ctx, plan, grouping) -} - -func (hp *horizonPlanning) planAggrUsingOA( - ctx *plancontext.PlanningContext, - plan logicalPlan, - grouping []operators.GroupBy, -) (logicalPlan, error) { - oa := &orderedAggregate{ - groupByKeys: make([]*engine.GroupByParams, 0, len(grouping)), - } - - var order []ops.OrderBy - if hp.qp.CanPushSorting { - hp.qp.OldAlignGroupByAndOrderBy(ctx) - // the grouping order might have changed, so we reload the grouping expressions - grouping = hp.qp.GetGrouping() - order = hp.qp.OrderExprs - } else { - for _, expr := range grouping { - order = append(order, expr.AsOrderBy()) - } - } - - // here we are building up the grouping keys for the OA, - // but they are lacking the input offsets because we have yet to push the columns down - for _, expr := range grouping { - typ, col, _ := ctx.SemTable.TypeForExpr(expr.Inner) - oa.groupByKeys = append(oa.groupByKeys, &engine.GroupByParams{ - Expr: expr.Inner, - FromGroupBy: true, - Type: typ, - CollationID: col, - }) - } - - if hp.sel.Having != nil { - rewriter := hp.qp.AggrRewriter(ctx) - sqlparser.SafeRewrite(hp.sel.Having.Expr, rewriter.RewriteDown(), rewriter.RewriteUp()) - if rewriter.Err != nil { - return nil, rewriter.Err - } - } - - aggregationExprs, _, err := hp.qp.AggregationExpressions(ctx, false) - if err != nil { - return nil, err - } - - // If we have a distinct aggregating expression, - // we handle it by pushing it down to the underlying input as a grouping column - distinctGroupBy, distinctOffsets, aggrs, err := hp.handleDistinctAggr(ctx, aggregationExprs) - if err != nil { - return nil, err - } - - if len(distinctGroupBy) > 0 { - grouping = append(grouping, distinctGroupBy...) - // all the distinct grouping aggregates use the same expression, so it should be OK to just add it once - order = append(order, distinctGroupBy[0].AsOrderBy()) - } - - if err = unsupportedAggregations(aggrs); err != nil { - return nil, err - } - - newPlan, groupingOffsets, aggrParamOffsets, pushed, err := hp.pushAggregation(ctx, plan, grouping, aggrs, false) - if err != nil { - return nil, err - } - - plan = newPlan - - _, isRoute := plan.(*route) - needsProj := !isRoute - var aggPlan = plan - var proj *projection - if needsProj { - length := getLengthOfProjection(groupingOffsets, aggrs) - proj = &projection{ - source: plan, - columns: make([]sqlparser.Expr, length), - columnNames: make([]string, length), - } - aggPlan = proj - } - - aggrParams, err := generateAggregateParams(aggrs, aggrParamOffsets, proj, pushed) - if err != nil { - return nil, err - } - - if proj != nil { - groupingOffsets, err = passGroupingColumns(proj, groupingOffsets, grouping) - if err != nil { - return nil, err - } - } - - // Next we add the aggregation expressions and grouping offsets to the OA - addColumnsToOA(ctx, oa, distinctGroupBy, aggrParams, distinctOffsets, groupingOffsets, aggregationExprs) - - aggPlan, err = hp.planOrderBy(ctx, order, aggPlan) - if err != nil { - return nil, err - } - - oa.resultsBuilder = newResultsBuilder(aggPlan, nil) - - return hp.planHaving(ctx, oa) -} - -func unsupportedAggregations(aggrs []operators.Aggr) error { - for _, aggr := range aggrs { - if aggr.OpCode == popcode.AggregateGroupConcat { - return vterrors.VT12001(fmt.Sprintf("in scatter query: aggregation function '%s'", sqlparser.String(aggr.Func))) - } - } - return nil -} - -func passGroupingColumns(proj *projection, groupings []offsets, grouping []operators.GroupBy) (projGrpOffsets []offsets, err error) { - for idx, grp := range groupings { - origGrp := grouping[idx] - var offs offsets - expr := origGrp.AsAliasedExpr() - offs.col, err = proj.addColumn(origGrp.InnerIndex, sqlparser.NewOffset(grp.col, expr.Expr), expr.ColumnName()) - if err != nil { - return nil, err - } - if grp.wsCol != -1 { - offs.wsCol, err = proj.addColumn(nil, sqlparser.NewOffset(grp.wsCol, weightStringFor(expr.Expr)), "") - if err != nil { - return nil, err - } - } - projGrpOffsets = append(projGrpOffsets, offs) - } - return projGrpOffsets, nil -} - -func generateAggregateParams(aggrs []operators.Aggr, aggrParamOffsets [][]offsets, proj *projection, pushed bool) ([]*engine.AggregateParams, error) { - aggrParams := make([]*engine.AggregateParams, len(aggrs)) - for idx, paramOffset := range aggrParamOffsets { - aggr := aggrs[idx] - incomingOffset := paramOffset[0].col - var offset int - if proj != nil { - var aggrExpr sqlparser.Expr - for _, ofs := range paramOffset { - curr := sqlparser.NewOffset(ofs.col, aggr.Func) - if aggrExpr == nil { - aggrExpr = curr - } else { - aggrExpr = &sqlparser.BinaryExpr{ - Operator: sqlparser.MultOp, - Left: aggrExpr, - Right: &sqlparser.FuncExpr{ - Name: sqlparser.NewIdentifierCI("coalesce"), - Exprs: sqlparser.SelectExprs{ - &sqlparser.AliasedExpr{Expr: curr}, - &sqlparser.AliasedExpr{Expr: sqlparser.NewIntLiteral("1")}, - }, - }, - } - } - } - - pos, err := proj.addColumn(aggr.Index, aggrExpr, aggr.Alias) - if err != nil { - return nil, err - } - offset = pos - } else { - offset = incomingOffset - } - - opcode := popcode.AggregateSum - switch aggr.OpCode { - case popcode.AggregateMin, popcode.AggregateMax, popcode.AggregateAnyValue: - opcode = aggr.OpCode - case popcode.AggregateCount, popcode.AggregateCountStar, popcode.AggregateCountDistinct, popcode.AggregateSumDistinct: - if !pushed { - opcode = aggr.OpCode - } - } - - aggrParam := engine.NewAggregateParam(opcode, offset, aggr.Alias) - aggrParam.Expr = aggr.Original.Expr - aggrParam.Original = aggr.Original - aggrParam.OrigOpcode = aggr.OpCode - aggrParams[idx] = aggrParam - } - return aggrParams, nil -} - -func addColumnsToOA( - ctx *plancontext.PlanningContext, - oa *orderedAggregate, - // these are the group by expressions that where added because we have unique aggregations - distinctGroupBy []operators.GroupBy, - // these are the aggregate params we already have for non-distinct aggregations - aggrParams []*engine.AggregateParams, - // distinctOffsets mark out where we need to use the distinctGroupBy offsets - // to create *engine.AggregateParams for the distinct aggregations - distinctOffsets []int, - // these are the offsets for the group by params - groupings []offsets, - // aggregationExprs are all the original aggregation expressions the query requested - aggregationExprs []operators.Aggr, -) { - if len(distinctGroupBy) == 0 { - // no distinct aggregations - oa.aggregates = aggrParams - } else { - count := len(groupings) - len(distinctOffsets) - addDistinctAggr := func(offset int) { - // the last grouping we pushed is the one we added for the distinct aggregation - o := groupings[count] - count++ - a := aggregationExprs[offset] - aggr := engine.NewAggregateParam(a.OpCode, o.col, a.Alias) - aggr.KeyCol = o.col - aggr.WCol = o.wsCol - aggr.Original = a.Original - aggr.Type, aggr.CollationID, _ = ctx.SemTable.TypeForExpr(a.Func.GetArg()) - oa.aggregates = append(oa.aggregates, aggr) - } - lastOffset := distinctOffsets[len(distinctOffsets)-1] - distinctIdx := 0 - for i := 0; i <= lastOffset || i <= len(aggrParams); i++ { - for distinctIdx < len(distinctOffsets) && i == distinctOffsets[distinctIdx] { - // we loop here since we could be dealing with multiple distinct aggregations after each other - addDistinctAggr(i) - distinctIdx++ - } - if i < len(aggrParams) { - oa.aggregates = append(oa.aggregates, aggrParams[i]) - } - } - - // we have to remove the tail of the grouping offsets, so we only have the offsets for the GROUP BY in the query - groupings = groupings[:len(groupings)-len(distinctOffsets)] - } - - for i, grouping := range groupings { - oa.groupByKeys[i].KeyCol = grouping.col - oa.groupByKeys[i].WeightStringCol = grouping.wsCol - } -} - -// handleDistinctAggr takes in a slice of aggregations and returns GroupBy elements that replace -// the distinct aggregations in the input, along with a slice of offsets and the non-distinct aggregations left, -// so we can later reify the original aggregations -func (hp *horizonPlanning) handleDistinctAggr(ctx *plancontext.PlanningContext, exprs []operators.Aggr) ( - distincts []operators.GroupBy, offsets []int, aggrs []operators.Aggr, err error) { - var distinctExpr sqlparser.Expr - for i, expr := range exprs { - if !expr.Distinct { - aggrs = append(aggrs, expr) - continue - } - - inner := expr.Func.GetArg() - innerWS := hp.qp.GetSimplifiedExpr(inner) - if err != nil { - return nil, nil, nil, err - } - if exprHasVindex(ctx.SemTable, innerWS, false) { - aggrs = append(aggrs, expr) - continue - } - if distinctExpr == nil { - distinctExpr = innerWS - } else { - if !ctx.SemTable.EqualsExpr(distinctExpr, innerWS) { - err = vterrors.VT12001(fmt.Sprintf("only one DISTINCT aggregation is allowed in a SELECT: %s", sqlparser.String(expr.Original))) - return nil, nil, nil, err - } - } - groupBy := operators.NewGroupBy(inner, innerWS, nil) - groupBy.InnerIndex = expr.Index - distincts = append(distincts, groupBy) - offsets = append(offsets, i) - } - return -} - -func (hp *horizonPlanning) planAggregationWithoutOA(ctx *plancontext.PlanningContext, plan logicalPlan) error { - for _, expr := range hp.qp.SelectExprs { - aliasedExpr, err := expr.GetAliasedExpr() - if err != nil { - return err - } - _, _, err = pushProjection(ctx, aliasedExpr, plan, true, false, false) - if err != nil { - return err - } - } - for _, expr := range hp.qp.GetGrouping() { - // since all the grouping will be done at the mysql level, - // we know that we won't need any weight_string() calls - err := planGroupByGen4(ctx, expr, plan /*weighString*/, false) - if err != nil { - return err - } - } - return nil -} - -type offsets struct { - col, wsCol int -} - -func newOffset(col int) offsets { - return offsets{col: col, wsCol: -1} -} - -func (hp *horizonPlanning) createGroupingsForColumns(columns []*sqlparser.ColName) ([]operators.GroupBy, error) { - var lhsGrouping []operators.GroupBy - for _, lhsColumn := range columns { - wsExpr := hp.qp.GetSimplifiedExpr(lhsColumn) - - lhsGrouping = append(lhsGrouping, operators.NewGroupBy(lhsColumn, wsExpr, nil)) - } - return lhsGrouping, nil -} - -func hasUniqueVindex(semTable *semantics.SemTable, groupByExprs []operators.GroupBy) bool { - for _, groupByExpr := range groupByExprs { - if exprHasUniqueVindex(semTable, groupByExpr.SimplifiedExpr) { - return true - } - } - return false -} - -func (hp *horizonPlanning) planOrderBy(ctx *plancontext.PlanningContext, orderExprs []ops.OrderBy, plan logicalPlan) (logicalPlan, error) { - switch plan := plan.(type) { - case *route: - return planOrderByForRoute(ctx, orderExprs, plan, hp.qp.HasStar) - case *join: - return hp.planOrderByForJoin(ctx, orderExprs, plan) - case *hashJoin: - return hp.planOrderByForHashJoin(ctx, orderExprs, plan) - case *orderedAggregate: - // remove ORDER BY NULL from the list of order by expressions since we will be doing the ordering on vtgate level so NULL is not useful - var orderExprsWithoutNils []ops.OrderBy - for _, expr := range orderExprs { - if sqlparser.IsNull(expr.Inner.Expr) { - continue - } - orderExprsWithoutNils = append(orderExprsWithoutNils, expr) - } - orderExprs = orderExprsWithoutNils - - for _, order := range orderExprs { - if sqlparser.ContainsAggregation(order.SimplifiedExpr) { - ms, err := createMemorySortPlanOnAggregation(ctx, plan, orderExprs) - if err != nil { - return nil, err - } - return ms, nil - } - } - newInput, err := hp.planOrderBy(ctx, orderExprs, plan.input) - if err != nil { - return nil, err - } - plan.input = newInput - return plan, nil - case *memorySort: - return plan, nil - case *simpleProjection: - return hp.createMemorySortPlan(ctx, plan, orderExprs, true) - case *vindexFunc: - // This is evaluated at VTGate only, so weight_string function cannot be used. - return hp.createMemorySortPlan(ctx, plan, orderExprs /* useWeightStr */, false) - case *limit, *semiJoin, *filter, *uncorrelatedSubquery, *projection: - inputs := plan.Inputs() - if len(inputs) == 0 { - break - } - newFirstInput, err := hp.planOrderBy(ctx, orderExprs, inputs[0]) - if err != nil { - return nil, err - } - inputs[0] = newFirstInput - err = plan.Rewrite(inputs...) - if err != nil { - return nil, err - } - return plan, nil - } - return nil, vterrors.VT13001(fmt.Sprintf("ORDER BY in complex query %T", plan)) -} - -func isSpecialOrderBy(o ops.OrderBy) bool { - if sqlparser.IsNull(o.Inner.Expr) { - return true - } - f, isFunction := o.Inner.Expr.(*sqlparser.FuncExpr) - return isFunction && f.Name.Lowered() == "rand" -} - -func planOrderByForRoute(ctx *plancontext.PlanningContext, orderExprs []ops.OrderBy, plan *route, hasStar bool) (logicalPlan, error) { - for _, order := range orderExprs { - err := checkOrderExprCanBePlannedInScatter(ctx, plan, order, hasStar) - if err != nil { - return nil, err - } - plan.Select.AddOrder(order.Inner) - if isSpecialOrderBy(order) { - continue - } - var wsExpr sqlparser.Expr - if ctx.SemTable.NeedsWeightString(order.Inner.Expr) { - wsExpr = order.SimplifiedExpr - } - - offset, weightStringOffset, err := wrapAndPushExpr(ctx, order.Inner.Expr, wsExpr, plan) - if err != nil { - return nil, err - } - typ, col, _ := ctx.SemTable.TypeForExpr(order.Inner.Expr) - plan.eroute.OrderBy = append(plan.eroute.OrderBy, engine.OrderByParams{ - Col: offset, - WeightStringCol: weightStringOffset, - Desc: order.Inner.Direction == sqlparser.DescOrder, - Type: typ, - CollationID: col, - }) - } - return plan, nil -} - -// checkOrderExprCanBePlannedInScatter verifies that the given order by expression can be planned. -// It checks if the expression exists in the plan's select list when the query is a scatter. -func checkOrderExprCanBePlannedInScatter(ctx *plancontext.PlanningContext, plan *route, order ops.OrderBy, hasStar bool) error { - if !hasStar { - return nil - } - sel := sqlparser.GetFirstSelect(plan.Select) - found := false - for _, expr := range sel.SelectExprs { - aliasedExpr, isAliasedExpr := expr.(*sqlparser.AliasedExpr) - if isAliasedExpr && ctx.SemTable.EqualsExpr(aliasedExpr.Expr, order.Inner.Expr) { - found = true - break - } - } - if !found { - return vterrors.VT12001(fmt.Sprintf("in scatter query: ORDER BY must reference a column in the SELECT list: %s", sqlparser.String(order.Inner))) - } - return nil -} - -// wrapAndPushExpr pushes the expression and weighted_string function to the plan using semantics.SemTable -// It returns (expr offset, weight_string offset, error) -func wrapAndPushExpr(ctx *plancontext.PlanningContext, expr sqlparser.Expr, weightStrExpr sqlparser.Expr, plan logicalPlan) (int, int, error) { - offset, _, err := pushProjection(ctx, &sqlparser.AliasedExpr{Expr: expr}, plan, true, true, false) - if err != nil { - return 0, 0, err - } - if weightStrExpr == nil { - return offset, -1, nil - } - if !sqlparser.IsColName(expr) { - switch unary := expr.(type) { - case *sqlparser.CastExpr: - expr = unary.Expr - case *sqlparser.ConvertExpr: - expr = unary.Expr - } - if !sqlparser.IsColName(expr) { - return 0, 0, vterrors.VT13001(fmt.Sprintf("in scatter query: complex ORDER BY expression: %s", sqlparser.String(expr))) - } - } - qt, _, found := ctx.SemTable.TypeForExpr(expr) - wsNeeded := true - if found && sqltypes.IsNumber(qt) { - wsNeeded = false - } - - weightStringOffset := -1 - if wsNeeded { - aliasedExpr := &sqlparser.AliasedExpr{Expr: weightStringFor(weightStrExpr)} - weightStringOffset, _, err = pushProjection(ctx, aliasedExpr, plan, true, true, false) - if err != nil { - return 0, 0, err - } - } - return offset, weightStringOffset, nil -} - -func weightStringFor(expr sqlparser.Expr) sqlparser.Expr { - return &sqlparser.WeightStringFuncExpr{Expr: expr} -} - -func (hp *horizonPlanning) planOrderByForHashJoin(ctx *plancontext.PlanningContext, orderExprs []ops.OrderBy, plan *hashJoin) (logicalPlan, error) { - if len(orderExprs) == 1 && isSpecialOrderBy(orderExprs[0]) { - rhs, err := hp.planOrderBy(ctx, orderExprs, plan.Right) - if err != nil { - return nil, err - } - plan.Right = rhs - return plan, nil - } - if orderExprsDependsOnTableSet(orderExprs, ctx.SemTable, plan.Right.ContainsTables()) { - newRight, err := hp.planOrderBy(ctx, orderExprs, plan.Right) - if err != nil { - return nil, err - } - plan.Right = newRight - return plan, nil - } - sortPlan, err := hp.createMemorySortPlan(ctx, plan, orderExprs, true) - if err != nil { - return nil, err - } - return sortPlan, nil -} - -func (hp *horizonPlanning) planOrderByForJoin(ctx *plancontext.PlanningContext, orderExprs []ops.OrderBy, plan *join) (logicalPlan, error) { - if len(orderExprs) == 1 && isSpecialOrderBy(orderExprs[0]) { - lhs, err := hp.planOrderBy(ctx, orderExprs, plan.Left) - if err != nil { - return nil, err - } - rhs, err := hp.planOrderBy(ctx, orderExprs, plan.Right) - if err != nil { - return nil, err - } - plan.Left = lhs - plan.Right = rhs - return plan, nil - } - // We can only push down sorting on the LHS of the join. - // If the order is on the RHS, we need to do the sorting on the vtgate - if orderExprsDependsOnTableSet(orderExprs, ctx.SemTable, plan.Left.ContainsTables()) { - newLeft, err := hp.planOrderBy(ctx, orderExprs, plan.Left) - if err != nil { - return nil, err - } - plan.Left = newLeft - return plan, nil - } - sortPlan, err := hp.createMemorySortPlan(ctx, plan, orderExprs, true) - if err != nil { - return nil, err - } - return sortPlan, nil -} - -func createMemorySortPlanOnAggregation(ctx *plancontext.PlanningContext, plan *orderedAggregate, orderExprs []ops.OrderBy) (logicalPlan, error) { - primitive := &engine.MemorySort{} - ms := &memorySort{ - resultsBuilder: newResultsBuilder(plan, primitive), - eMemorySort: primitive, - } - - for _, order := range orderExprs { - offset, woffset, found := findExprInOrderedAggr(ctx, plan, order) - if !found { - return nil, vterrors.VT13001(fmt.Sprintf("expected to find ORDER BY expression (%s) in orderedAggregate", sqlparser.String(order.Inner))) - } - - typ, collationID, _ := ctx.SemTable.TypeForExpr(order.SimplifiedExpr) - ms.eMemorySort.OrderBy = append(ms.eMemorySort.OrderBy, engine.OrderByParams{ - Col: offset, - WeightStringCol: woffset, - Desc: order.Inner.Direction == sqlparser.DescOrder, - StarColFixedIndex: offset, - Type: typ, - CollationID: collationID, - }) - } - return ms, nil -} - -func findExprInOrderedAggr(ctx *plancontext.PlanningContext, plan *orderedAggregate, order ops.OrderBy) (keyCol int, weightStringCol int, found bool) { - for _, key := range plan.groupByKeys { - if ctx.SemTable.EqualsExpr(order.SimplifiedExpr, key.Expr) || - ctx.SemTable.EqualsExpr(order.Inner.Expr, key.Expr) { - return key.KeyCol, key.WeightStringCol, true - } - } - for _, aggregate := range plan.aggregates { - if ctx.SemTable.EqualsExpr(order.SimplifiedExpr, aggregate.Original.Expr) || - ctx.SemTable.EqualsExpr(order.Inner.Expr, aggregate.Original.Expr) { - return aggregate.Col, -1, true - } - } - return 0, 0, false -} - -func (hp *horizonPlanning) createMemorySortPlan(ctx *plancontext.PlanningContext, plan logicalPlan, orderExprs []ops.OrderBy, useWeightStr bool) (logicalPlan, error) { - primitive := &engine.MemorySort{} - ms := &memorySort{ - resultsBuilder: newResultsBuilder(plan, primitive), - eMemorySort: primitive, - } - - for _, order := range orderExprs { - wsExpr := order.SimplifiedExpr - if !useWeightStr { - wsExpr = nil - } - offset, weightStringOffset, err := wrapAndPushExpr(ctx, order.Inner.Expr, wsExpr, plan) - if err != nil { - return nil, err - } - typ, col, _ := ctx.SemTable.TypeForExpr(order.Inner.Expr) - ms.eMemorySort.OrderBy = append(ms.eMemorySort.OrderBy, engine.OrderByParams{ - Col: offset, - WeightStringCol: weightStringOffset, - Desc: order.Inner.Direction == sqlparser.DescOrder, - StarColFixedIndex: offset, - Type: typ, - CollationID: col, - }) - } - return ms, nil -} - -func orderExprsDependsOnTableSet(orderExprs []ops.OrderBy, semTable *semantics.SemTable, ts semantics.TableSet) bool { - for _, expr := range orderExprs { - exprDependencies := semTable.RecursiveDeps(expr.Inner.Expr) - if !exprDependencies.IsSolvedBy(ts) { - return false - } - } - return true -} - -func (hp *horizonPlanning) planDistinct(ctx *plancontext.PlanningContext, plan logicalPlan) (logicalPlan, error) { - if !hp.qp.NeedsDistinct() { - return plan, nil - } - switch p := plan.(type) { - case *route: - // we always make the underlying query distinct, - // and then we might also add a distinct operator on top if it is needed - p.Select.MakeDistinct() - if p.isSingleShard() || selectHasUniqueVindex(ctx.SemTable, hp.qp.SelectExprs) { - return plan, nil - } - - return hp.addDistinct(ctx, plan) - case *join, *uncorrelatedSubquery: - return hp.addDistinct(ctx, plan) - case *orderedAggregate: - return hp.planDistinctOA(ctx.SemTable, p) - default: - return nil, vterrors.VT13001(fmt.Sprintf("unknown plan type for DISTINCT %T", plan)) - } -} - -func (hp *horizonPlanning) planDistinctOA(semTable *semantics.SemTable, currPlan *orderedAggregate) (logicalPlan, error) { - oa := &orderedAggregate{ - resultsBuilder: newResultsBuilder(currPlan, nil), - } - for _, sExpr := range hp.qp.SelectExprs { - expr, err := sExpr.GetExpr() - if err != nil { - return nil, err - } - found := false - for _, grpParam := range currPlan.groupByKeys { - if semTable.EqualsExpr(expr, grpParam.Expr) { - found = true - oa.groupByKeys = append(oa.groupByKeys, grpParam) - break - } - } - if found { - continue - } - for _, aggrParam := range currPlan.aggregates { - if semTable.EqualsExpr(expr, aggrParam.Expr) { - found = true - typ, col, _ := semTable.TypeForExpr(expr) - oa.groupByKeys = append(oa.groupByKeys, &engine.GroupByParams{KeyCol: aggrParam.Col, WeightStringCol: -1, Type: typ, CollationID: col}) - break - } - } - if !found { - return nil, vterrors.VT13001(fmt.Sprintf("unable to plan DISTINCT query as the column is not projected: %s", sqlparser.String(sExpr.Col))) - } - } - return oa, nil -} - -func (hp *horizonPlanning) addDistinct(ctx *plancontext.PlanningContext, plan logicalPlan) (logicalPlan, error) { - var orderExprs []ops.OrderBy - var groupByKeys []*engine.GroupByParams - for index, sExpr := range hp.qp.SelectExprs { - aliasExpr, err := sExpr.GetAliasedExpr() - if err != nil { - return nil, err - } - if isAmbiguousOrderBy(index, aliasExpr.As, hp.qp.SelectExprs) { - return nil, vterrors.VT13001(fmt.Sprintf("generating ORDER BY clause: ambiguous symbol reference: %s", sqlparser.String(aliasExpr.As))) - } - var inner sqlparser.Expr - if aliasExpr.As.IsEmpty() { - inner = aliasExpr.Expr - } else { - // If we have an alias, we need to use the alias and not the original expression - // to make sure dependencies work correctly, - // we simply copy the dependencies of the original expression here - inner = sqlparser.NewColName(aliasExpr.As.String()) - ctx.SemTable.CopyDependencies(aliasExpr.Expr, inner) - } - typ, col, _ := ctx.SemTable.TypeForExpr(inner) - grpParam := &engine.GroupByParams{KeyCol: index, WeightStringCol: -1, Type: typ, CollationID: col, Expr: inner} - _, wOffset, err := wrapAndPushExpr(ctx, aliasExpr.Expr, aliasExpr.Expr, plan) - if err != nil { - return nil, err - } - grpParam.WeightStringCol = wOffset - groupByKeys = append(groupByKeys, grpParam) - - orderExprs = append(orderExprs, ops.OrderBy{ - Inner: &sqlparser.Order{Expr: inner}, - SimplifiedExpr: aliasExpr.Expr}, - ) - } - innerPlan, err := hp.planOrderBy(ctx, orderExprs, plan) - if err != nil { - return nil, err - } - oa := &orderedAggregate{ - resultsBuilder: newResultsBuilder(innerPlan, nil), - groupByKeys: groupByKeys, - } - return oa, nil -} - -func isAmbiguousOrderBy(index int, col sqlparser.IdentifierCI, exprs []operators.SelectExpr) bool { - if col.String() == "" { - return false - } - for i, expr := range exprs { - if i == index { - continue - } - aliasExpr, isAlias := expr.Col.(*sqlparser.AliasedExpr) - if !isAlias { - // TODO: handle star expression error - return true - } - alias := aliasExpr.As - if alias.IsEmpty() { - if col, ok := aliasExpr.Expr.(*sqlparser.ColName); ok { - alias = col.Name - } - } - if col.Equal(alias) { - return true - } - } - return false -} - -func selectHasUniqueVindex(semTable *semantics.SemTable, sel []operators.SelectExpr) bool { - for _, expr := range sel { - exp, err := expr.GetExpr() - if err != nil { - // TODO: handle star expression error - return false - } - if exprHasUniqueVindex(semTable, exp) { - return true - } - } - return false -} - -func (hp *horizonPlanning) planHaving(ctx *plancontext.PlanningContext, plan logicalPlan) (logicalPlan, error) { - if hp.sel.Having == nil { - return plan, nil - } - return pushHaving(ctx, hp.sel.Having.Expr, plan) -} - -func pushHaving(ctx *plancontext.PlanningContext, expr sqlparser.Expr, plan logicalPlan) (logicalPlan, error) { - switch node := plan.(type) { - case *route: - sel := sqlparser.GetFirstSelect(node.Select) - sel.AddHaving(expr) - return plan, nil - case *uncorrelatedSubquery: - return pushHaving(ctx, expr, node.outer) - case *simpleProjection: - return nil, vterrors.VT13001("filtering on results of cross-shard derived table") - case *orderedAggregate: - return newFilter(ctx, plan, expr) - } - return nil, vterrors.VT13001(fmt.Sprintf("unreachable %T.filtering", plan)) -} - -func isJoin(plan logicalPlan) bool { - switch plan.(type) { - case *join, *hashJoin: - return true - default: - return false - } -} - -func exprHasUniqueVindex(semTable *semantics.SemTable, expr sqlparser.Expr) bool { - return exprHasVindex(semTable, expr, true) -} - -func exprHasVindex(semTable *semantics.SemTable, expr sqlparser.Expr, hasToBeUnique bool) bool { - col, isCol := expr.(*sqlparser.ColName) - if !isCol { - return false - } - ts := semTable.RecursiveDeps(expr) - tableInfo, err := semTable.TableInfoFor(ts) - if err != nil { - return false - } - vschemaTable := tableInfo.GetVindexTable() - for _, vindex := range vschemaTable.ColumnVindexes { - if len(vindex.Columns) > 1 || hasToBeUnique && !vindex.IsUnique() { - return false - } - if col.Name.Equal(vindex.Columns[0]) { - return true - } - } - return false -} - -func planSingleRoutePlan(sel sqlparser.SelectStatement, rb *route) error { - err := stripDownQuery(sel, rb.Select) - if err != nil { - return err - } - return sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) { - if aliasedExpr, ok := node.(sqlparser.SelectExpr); ok { - removeKeyspaceFromSelectExpr(aliasedExpr) - } - return true, nil - }, rb.Select) - -} - -func removeKeyspaceFromSelectExpr(expr sqlparser.SelectExpr) { - switch expr := expr.(type) { - case *sqlparser.AliasedExpr: - sqlparser.RemoveKeyspaceFromColName(expr.Expr) - case *sqlparser.StarExpr: - expr.TableName.Qualifier = sqlparser.NewIdentifierCS("") - } -} - -func stripDownQuery(from, to sqlparser.SelectStatement) error { - var err error - - switch node := from.(type) { - case *sqlparser.Select: - toNode, ok := to.(*sqlparser.Select) - if !ok { - return vterrors.VT13001("AST did not match") - } - toNode.Distinct = node.Distinct - toNode.GroupBy = node.GroupBy - toNode.Having = node.Having - toNode.OrderBy = node.OrderBy - toNode.Comments = node.Comments - toNode.SelectExprs = node.SelectExprs - for _, expr := range toNode.SelectExprs { - removeKeyspaceFromSelectExpr(expr) - } - case *sqlparser.Union: - toNode, ok := to.(*sqlparser.Union) - if !ok { - return vterrors.VT13001("AST did not match") - } - err = stripDownQuery(node.Left, toNode.Left) - if err != nil { - return err - } - err = stripDownQuery(node.Right, toNode.Right) - if err != nil { - return err - } - toNode.OrderBy = node.OrderBy - default: - return vterrors.VT13001(fmt.Sprintf("this should not happen - we have covered all implementations of SelectStatement %T", from)) - } - return nil -} - -func planGroupByGen4(ctx *plancontext.PlanningContext, groupExpr operators.GroupBy, plan logicalPlan, wsAdded bool) error { - switch node := plan.(type) { - case *route: - sel := node.Select.(*sqlparser.Select) - sel.AddGroupBy(groupExpr.Inner) - // If a weight_string function is added to the select list, - // then we need to add that to the group by clause otherwise the query will fail on mysql with full_group_by error - // as the weight_string function might not be functionally dependent on the group by. - if wsAdded { - sel.AddGroupBy(weightStringFor(groupExpr.SimplifiedExpr)) - } - return nil - case *uncorrelatedSubquery: - return planGroupByGen4(ctx, groupExpr, node.outer, wsAdded) - case *semiJoin: - return vterrors.VT13001("GROUP BY in a query having a correlated subquery") - default: - return vterrors.VT13001(fmt.Sprintf("GROUP BY on: %T", plan)) - } -} - -func getLengthOfProjection(groupingOffsets []offsets, aggregations []operators.Aggr) int { - length := 0 - for _, groupBy := range groupingOffsets { - if groupBy.wsCol != -1 { - length++ - } - length++ - } - length += len(aggregations) - return length -} diff --git a/go/vt/vtgate/planbuilder/horizon_planning_test.go b/go/vt/vtgate/planbuilder/horizon_planning_test.go deleted file mode 100644 index 94e51b6700a..00000000000 --- a/go/vt/vtgate/planbuilder/horizon_planning_test.go +++ /dev/null @@ -1,80 +0,0 @@ -/* -Copyright 2021 The Vitess Authors. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package planbuilder - -import ( - "testing" - - "github.com/stretchr/testify/assert" - - "vitess.io/vitess/go/vt/vtgate/semantics" - - "vitess.io/vitess/go/vt/sqlparser" -) - -func TestCheckIfAlreadyExists(t *testing.T) { - tests := []struct { - name string - expr *sqlparser.AliasedExpr - sel *sqlparser.Select - want int - }{ - { - name: "No alias, both ColName", - want: 0, - expr: &sqlparser.AliasedExpr{Expr: sqlparser.NewColName("id")}, - sel: &sqlparser.Select{SelectExprs: []sqlparser.SelectExpr{&sqlparser.AliasedExpr{Expr: sqlparser.NewColName("id")}}}, - }, - { - name: "Aliased expression and ColName", - want: 0, - expr: &sqlparser.AliasedExpr{Expr: sqlparser.NewColName("user_id")}, - sel: &sqlparser.Select{SelectExprs: []sqlparser.SelectExpr{&sqlparser.AliasedExpr{As: sqlparser.NewIdentifierCI("user_id"), Expr: sqlparser.NewColName("id")}}}, - }, - { - name: "Non-ColName expressions", - want: 0, - expr: &sqlparser.AliasedExpr{Expr: sqlparser.NewStrLiteral("test")}, - sel: &sqlparser.Select{SelectExprs: []sqlparser.SelectExpr{&sqlparser.AliasedExpr{Expr: sqlparser.NewStrLiteral("test")}}}, - }, - { - name: "No alias, multiple ColName in projection", - want: 1, - expr: &sqlparser.AliasedExpr{Expr: sqlparser.NewColName("id")}, - sel: &sqlparser.Select{SelectExprs: []sqlparser.SelectExpr{&sqlparser.AliasedExpr{Expr: sqlparser.NewColName("foo")}, &sqlparser.AliasedExpr{Expr: sqlparser.NewColName("id")}}}, - }, - { - name: "No matching entry", - want: -1, - expr: &sqlparser.AliasedExpr{Expr: sqlparser.NewColName("id")}, - sel: &sqlparser.Select{SelectExprs: []sqlparser.SelectExpr{&sqlparser.AliasedExpr{Expr: sqlparser.NewColName("foo")}, &sqlparser.AliasedExpr{Expr: sqlparser.NewColName("name")}}}, - }, - { - name: "No AliasedExpr in projection", - want: -1, - expr: &sqlparser.AliasedExpr{Expr: sqlparser.NewColName("id")}, - sel: &sqlparser.Select{SelectExprs: []sqlparser.SelectExpr{&sqlparser.StarExpr{TableName: sqlparser.TableName{Name: sqlparser.NewIdentifierCS("user")}}, &sqlparser.StarExpr{TableName: sqlparser.TableName{Name: sqlparser.NewIdentifierCS("people")}}}}, - }, - } - for _, tt := range tests { - semTable := semantics.EmptySemTable() - t.Run(tt.name, func(t *testing.T) { - got := checkIfAlreadyExists(tt.expr, tt.sel, semTable) - assert.Equal(t, tt.want, got) - }) - } -} diff --git a/go/vt/vtgate/planbuilder/join.go b/go/vt/vtgate/planbuilder/join.go index e41835048af..2b438ce56a0 100644 --- a/go/vt/vtgate/planbuilder/join.go +++ b/go/vt/vtgate/planbuilder/join.go @@ -94,3 +94,15 @@ func (j *join) ContainsTables() semantics.TableSet { func (j *join) OutputColumns() []sqlparser.SelectExpr { return getOutputColumnsFromJoin(j.Cols, j.Left.OutputColumns(), j.Right.OutputColumns()) } + +func getOutputColumnsFromJoin(ints []int, lhs []sqlparser.SelectExpr, rhs []sqlparser.SelectExpr) (cols []sqlparser.SelectExpr) { + for _, col := range ints { + if col < 0 { + col *= -1 + cols = append(cols, lhs[col-1]) + } else { + cols = append(cols, rhs[col-1]) + } + } + return +} diff --git a/go/vt/vtgate/planbuilder/operator_transformers.go b/go/vt/vtgate/planbuilder/operator_transformers.go index 306b8dff7f7..6140b1b9b5d 100644 --- a/go/vt/vtgate/planbuilder/operator_transformers.go +++ b/go/vt/vtgate/planbuilder/operator_transformers.go @@ -51,7 +51,7 @@ func transformToLogicalPlan(ctx *plancontext.PlanningContext, op ops.Operator) ( case *operators.Filter: return transformFilter(ctx, op) case *operators.Horizon: - return transformHorizon(ctx, op) + panic("should have been solved in the operator") case *operators.Projection: return transformProjection(ctx, op) case *operators.Limit: @@ -340,16 +340,8 @@ func transformFilter(ctx *plancontext.PlanningContext, op *operators.Filter) (lo predicate := op.PredicateWithOffsets ast := ctx.SemTable.AndExpressions(op.Predicates...) - // this might already have been done on the operators if predicate == nil { - predicate, err = evalengine.Translate(ast, &evalengine.Config{ - ResolveType: ctx.SemTable.TypeForExpr, - ResolveColumn: resolveFromPlan(ctx, plan, true), - Collation: ctx.SemTable.Collation, - }) - if err != nil { - return nil, err - } + panic("this should have already been done") } return &filter{ @@ -362,47 +354,6 @@ func transformFilter(ctx *plancontext.PlanningContext, op *operators.Filter) (lo }, nil } -func transformHorizon(ctx *plancontext.PlanningContext, op *operators.Horizon) (logicalPlan, error) { - if op.IsDerived() { - return transformDerivedPlan(ctx, op) - } - source, err := transformToLogicalPlan(ctx, op.Source) - if err != nil { - return nil, err - } - switch node := op.Query.(type) { - case *sqlparser.Select: - hp := horizonPlanning{ - sel: node, - } - - plan, err := hp.planHorizon(ctx, source, true) - if err != nil { - return nil, err - } - return planLimit(node.Limit, plan) - case *sqlparser.Union: - var err error - rb, isRoute := source.(*route) - if !isRoute && ctx.SemTable.NotSingleRouteErr != nil { - return nil, ctx.SemTable.NotSingleRouteErr - } - var plan logicalPlan - if isRoute && rb.isSingleShard() { - err = planSingleRoutePlan(node, rb) - plan = rb - } else { - plan, err = planOrderByOnUnion(ctx, source, node) - } - if err != nil { - return nil, err - } - - return planLimit(node.Limit, plan) - } - return nil, vterrors.VT13001("only SELECT and UNION implement the SelectStatement interface") -} - func transformApplyJoinPlan(ctx *plancontext.PlanningContext, n *operators.ApplyJoin) (logicalPlan, error) { lhs, err := transformToLogicalPlan(ctx, n.LHS) if err != nil { @@ -746,52 +697,6 @@ func transformUnionPlan(ctx *plancontext.PlanningContext, op *operators.Union) ( } -func transformDerivedPlan(ctx *plancontext.PlanningContext, op *operators.Horizon) (logicalPlan, error) { - // transforming the inner part of the derived table into a logical plan - // so that we can do horizon planning on the inner. If the logical plan - // we've produced is a Route, we set its Select.From field to be an aliased - // expression containing our derived table's inner select and the derived - // table's alias. - - plan, err := transformToLogicalPlan(ctx, op.Source) - if err != nil { - return nil, err - } - - plan, err = planHorizon(ctx, plan, op.Query, false) - if err != nil { - return nil, err - } - - rb, isRoute := plan.(*route) - if !isRoute { - return &simpleProjection{ - logicalPlanCommon: newBuilderCommon(plan), - eSimpleProj: &engine.SimpleProjection{ - Cols: op.ColumnsOffset, - }, - }, nil - } - innerSelect := rb.Select - derivedTable := &sqlparser.DerivedTable{Select: innerSelect} - tblExpr := &sqlparser.AliasedTableExpr{ - Expr: derivedTable, - As: sqlparser.NewIdentifierCS(op.Alias), - Columns: op.ColumnAliases, - } - selectExprs := sqlparser.SelectExprs{} - for _, colName := range op.Columns { - selectExprs = append(selectExprs, &sqlparser.AliasedExpr{ - Expr: colName, - }) - } - rb.Select = &sqlparser.Select{ - From: []sqlparser.TableExpr{tblExpr}, - SelectExprs: selectExprs, - } - return plan, nil -} - func transformLimit(ctx *plancontext.PlanningContext, op *operators.Limit) (logicalPlan, error) { plan, err := transformToLogicalPlan(ctx, op.Source) if err != nil { diff --git a/go/vt/vtgate/planbuilder/operators/SQL_builder.go b/go/vt/vtgate/planbuilder/operators/SQL_builder.go index e522f97ab2e..9802e374d87 100644 --- a/go/vt/vtgate/planbuilder/operators/SQL_builder.go +++ b/go/vt/vtgate/planbuilder/operators/SQL_builder.go @@ -528,12 +528,12 @@ func buildProjection(op *Projection, qb *queryBuilder) error { // if the projection is on derived table, we use the select we have // created above and transform it into a derived table - if op.TableID != nil { + if op.DT != nil { sel := qb.asSelectStatement() qb.stmt = nil - qb.addTableExpr(op.Alias, op.Alias, TableID(op), &sqlparser.DerivedTable{ + qb.addTableExpr(op.DT.Alias, op.DT.Alias, TableID(op), &sqlparser.DerivedTable{ Select: sel, - }, nil, nil) + }, nil, op.DT.Columns) } if !isSel { diff --git a/go/vt/vtgate/planbuilder/operators/aggregation_pushing.go b/go/vt/vtgate/planbuilder/operators/aggregation_pushing.go index 97887ea55ba..c7fc8c790a7 100644 --- a/go/vt/vtgate/planbuilder/operators/aggregation_pushing.go +++ b/go/vt/vtgate/planbuilder/operators/aggregation_pushing.go @@ -39,15 +39,15 @@ func tryPushAggregator(ctx *plancontext.PlanningContext, aggregator *Aggregator) // if we have a single sharded route, we can push it down output, applyResult, err = pushAggregationThroughRoute(ctx, aggregator, src) case *ApplyJoin: - if ctx.DelegateAggregation { + if reachedPhase(ctx, delegateAggregation) { output, applyResult, err = pushAggregationThroughJoin(ctx, aggregator, src) } case *Filter: - if ctx.DelegateAggregation { + if reachedPhase(ctx, delegateAggregation) { output, applyResult, err = pushAggregationThroughFilter(ctx, aggregator, src) } case *SubQueryContainer: - if ctx.DelegateAggregation { + if reachedPhase(ctx, delegateAggregation) { output, applyResult, err = pushAggregationThroughSubquery(ctx, aggregator, src) } default: @@ -67,6 +67,11 @@ func tryPushAggregator(ctx *plancontext.PlanningContext, aggregator *Aggregator) return } +func reachedPhase(ctx *plancontext.PlanningContext, p Phase) bool { + b := ctx.CurrentPhase >= int(p) + return b +} + // pushAggregationThroughSubquery pushes an aggregation under a subquery. // Any columns that are needed to evaluate the subquery needs to be added as // grouping columns to the aggregation being pushed down, and then after the @@ -138,7 +143,7 @@ func pushAggregationThroughRoute( return rewrite.Swap(aggregator, route, "push down aggregation under route - remove original") } - if !ctx.DelegateAggregation { + if !reachedPhase(ctx, delegateAggregation) { return nil, nil, nil } diff --git a/go/vt/vtgate/planbuilder/operators/aggregator.go b/go/vt/vtgate/planbuilder/operators/aggregator.go index 4f7703bf5f7..b572e05fb45 100644 --- a/go/vt/vtgate/planbuilder/operators/aggregator.go +++ b/go/vt/vtgate/planbuilder/operators/aggregator.go @@ -55,9 +55,8 @@ type ( ResultColumns int QP *QueryProjection - // TableID will be non-nil for derived tables - TableID *semantics.TableSet - Alias string + + DT *DerivedTable } ) @@ -122,16 +121,13 @@ func (a *Aggregator) addColumnsWithoutPushing(ctx *plancontext.PlanningContext, } func (a *Aggregator) isDerived() bool { - return a.TableID != nil + return a.DT != nil } -func (a *Aggregator) FindCol(ctx *plancontext.PlanningContext, expr sqlparser.Expr, _ bool) (int, error) { - if a.isDerived() { - derivedTBL, err := ctx.SemTable.TableInfoFor(*a.TableID) - if err != nil { - return 0, err - } - expr = semantics.RewriteDerivedTableExpression(expr, derivedTBL) +func (a *Aggregator) FindCol(ctx *plancontext.PlanningContext, in sqlparser.Expr, _ bool) (int, error) { + expr, err := a.DT.RewriteExpression(ctx, in) + if err != nil { + return 0, err } if offset, found := canReuseColumn(ctx, a.Columns, expr, extractExpr); found { return offset, nil @@ -139,9 +135,19 @@ func (a *Aggregator) FindCol(ctx *plancontext.PlanningContext, expr sqlparser.Ex return -1, nil } -func (a *Aggregator) AddColumn(ctx *plancontext.PlanningContext, reuse bool, groupBy bool, expr *sqlparser.AliasedExpr) (int, error) { +func (a *Aggregator) AddColumn(ctx *plancontext.PlanningContext, reuse bool, groupBy bool, ae *sqlparser.AliasedExpr) (int, error) { + rewritten, err := a.DT.RewriteExpression(ctx, ae.Expr) + if err != nil { + return 0, err + } + + ae = &sqlparser.AliasedExpr{ + Expr: rewritten, + As: ae.As, + } + if reuse { - offset, err := a.findColInternal(ctx, expr, groupBy) + offset, err := a.findColInternal(ctx, ae, groupBy) if err != nil { return 0, err } @@ -153,7 +159,7 @@ func (a *Aggregator) AddColumn(ctx *plancontext.PlanningContext, reuse bool, gro // Upon receiving a weight string function from an upstream operator, check for an existing grouping on the argument expression. // If a grouping is found, continue to push the function down, marking it with 'addToGroupBy' to ensure it's correctly treated as a grouping column. // This process also sets the weight string column offset, eliminating the need for a later addition in the aggregator operator's planOffset. - if wsExpr, isWS := expr.Expr.(*sqlparser.WeightStringFuncExpr); isWS { + if wsExpr, isWS := rewritten.(*sqlparser.WeightStringFuncExpr); isWS { idx := slices.IndexFunc(a.Grouping, func(by GroupBy) bool { return ctx.SemTable.EqualsExprWithDeps(wsExpr.Expr, by.SimplifiedExpr) }) @@ -164,39 +170,37 @@ func (a *Aggregator) AddColumn(ctx *plancontext.PlanningContext, reuse bool, gro } if !groupBy { - aggr := NewAggr(opcode.AggregateAnyValue, nil, expr, expr.As.String()) + aggr := NewAggr(opcode.AggregateAnyValue, nil, ae, ae.As.String()) aggr.ColOffset = len(a.Columns) a.Aggregations = append(a.Aggregations, aggr) } offset := len(a.Columns) - a.Columns = append(a.Columns, expr) - incomingOffset, err := a.Source.AddColumn(ctx, false, groupBy, expr) + a.Columns = append(a.Columns, ae) + incomingOffset, err := a.Source.AddColumn(ctx, false, groupBy, ae) if err != nil { return 0, err } if offset != incomingOffset { - return 0, errFailedToPlan(expr) + return 0, errFailedToPlan(ae) } return offset, nil } -func (a *Aggregator) findColInternal(ctx *plancontext.PlanningContext, expr *sqlparser.AliasedExpr, addToGroupBy bool) (int, error) { - offset, err := a.FindCol(ctx, expr.Expr, false) +func (a *Aggregator) findColInternal(ctx *plancontext.PlanningContext, ae *sqlparser.AliasedExpr, addToGroupBy bool) (int, error) { + expr := ae.Expr + offset, err := a.FindCol(ctx, expr, false) if err != nil { return 0, err } if offset >= 0 { return offset, err } - if a.isDerived() { - derivedTBL, err := ctx.SemTable.TableInfoFor(*a.TableID) - if err != nil { - return 0, err - } - expr.Expr = semantics.RewriteDerivedTableExpression(expr.Expr, derivedTBL) + expr, err = a.DT.RewriteExpression(ctx, expr) + if err != nil { + return 0, err } // Aggregator is little special and cannot work if the input offset are not matched with the aggregation columns. @@ -206,10 +210,10 @@ func (a *Aggregator) findColInternal(ctx *plancontext.PlanningContext, expr *sql return 0, err } - if offset, found := canReuseColumn(ctx, a.Columns, expr.Expr, extractExpr); found { + if offset, found := canReuseColumn(ctx, a.Columns, expr, extractExpr); found { return offset, nil } - colName, isColName := expr.Expr.(*sqlparser.ColName) + colName, isColName := expr.(*sqlparser.ColName) for i, col := range a.Columns { if isColName && colName.Name.EqualString(col.As.String()) { return i, nil @@ -252,8 +256,8 @@ func (a *Aggregator) ShortDescription() string { columns := slice.Map(a.Columns, func(from *sqlparser.AliasedExpr) string { return sqlparser.String(from) }) - if a.Alias != "" { - columns = append([]string{"derived[" + a.Alias + "]"}, columns...) + if a.DT != nil { + columns = append([]string{a.DT.String()}, columns...) } org := "" @@ -466,16 +470,12 @@ func (a *Aggregator) SplitAggregatorBelowRoute(input []ops.Operator) *Aggregator newOp := a.Clone(input).(*Aggregator) newOp.Pushed = false newOp.Original = false - newOp.Alias = "" - newOp.TableID = nil + newOp.DT = nil return newOp } func (a *Aggregator) introducesTableID() semantics.TableSet { - if a.TableID == nil { - return semantics.EmptyTableSet() - } - return *a.TableID + return a.DT.introducesTableID() } var _ ops.Operator = (*Aggregator)(nil) diff --git a/go/vt/vtgate/planbuilder/operators/horizon.go b/go/vt/vtgate/planbuilder/operators/horizon.go index 9dc6e9c12cd..2f9d574d99d 100644 --- a/go/vt/vtgate/planbuilder/operators/horizon.go +++ b/go/vt/vtgate/planbuilder/operators/horizon.go @@ -37,14 +37,14 @@ type Horizon struct { Source ops.Operator // If this is a derived table, the two following fields will contain the tableID and name of it - TableId *semantics.TableSet - Alias string + TableId *semantics.TableSet + Alias string + ColumnAliases sqlparser.Columns // derived tables can have their column aliases specified outside the subquery // QP contains the QueryProjection for this op QP *QueryProjection - Query sqlparser.SelectStatement - ColumnAliases sqlparser.Columns + Query sqlparser.SelectStatement // Columns needed to feed other plans Columns []*sqlparser.ColName diff --git a/go/vt/vtgate/planbuilder/operators/horizon_expanding.go b/go/vt/vtgate/planbuilder/operators/horizon_expanding.go index 66e69bac055..2714cb73ff1 100644 --- a/go/vt/vtgate/planbuilder/operators/horizon_expanding.go +++ b/go/vt/vtgate/planbuilder/operators/horizon_expanding.go @@ -63,8 +63,11 @@ func expandUnionHorizon(ctx *plancontext.PlanningContext, horizon *Horizon, unio if horizon.TableId != nil { proj := newAliasedProjection(op) - proj.TableID = horizon.TableId - proj.Alias = horizon.Alias + proj.DT = &DerivedTable{ + TableID: *horizon.TableId, + Alias: horizon.Alias, + Columns: horizon.ColumnAliases, + } op = proj } @@ -135,13 +138,21 @@ func createProjectionFromSelect(ctx *plancontext.PlanningContext, horizon *Horiz return nil, err } + var dt *DerivedTable + if horizon.TableId != nil { + dt = &DerivedTable{ + TableID: *horizon.TableId, + Alias: horizon.Alias, + Columns: horizon.ColumnAliases, + } + } + if !qp.NeedsAggregation() { projX, err := createProjectionWithoutAggr(ctx, qp, horizon.src()) if err != nil { return nil, err } - projX.TableID = horizon.TableId - projX.Alias = horizon.Alias + projX.DT = dt out = projX return out, nil @@ -158,8 +169,7 @@ func createProjectionFromSelect(ctx *plancontext.PlanningContext, horizon *Horiz QP: qp, Grouping: qp.GetGrouping(), Aggregations: aggregations, - TableID: horizon.TableId, - Alias: horizon.Alias, + DT: dt, } if complexAggr { @@ -204,8 +214,7 @@ outer: func createProjectionForComplexAggregation(a *Aggregator, qp *QueryProjection) (ops.Operator, error) { p := newAliasedProjection(a) - p.Alias = a.Alias - p.TableID = a.TableID + p.DT = a.DT for _, expr := range qp.SelectExprs { ae, err := expr.GetAliasedExpr() if err != nil { diff --git a/go/vt/vtgate/planbuilder/operators/offset_planning.go b/go/vt/vtgate/planbuilder/operators/offset_planning.go index cb892fcd65b..502df5e9c82 100644 --- a/go/vt/vtgate/planbuilder/operators/offset_planning.go +++ b/go/vt/vtgate/planbuilder/operators/offset_planning.go @@ -59,17 +59,6 @@ func fetchByOffset(e sqlparser.SQLNode) bool { } } -func planOffsetsOnJoins(ctx *plancontext.PlanningContext, op ops.Operator) error { - err := rewrite.Visit(op, func(current ops.Operator) error { - join, ok := current.(*ApplyJoin) - if !ok { - return nil - } - return join.planOffsets(ctx) - }) - return err -} - // useOffsets rewrites an expression to use values from the input func useOffsets(ctx *plancontext.PlanningContext, expr sqlparser.Expr, op ops.Operator) (sqlparser.Expr, error) { var exprOffset *sqlparser.Offset diff --git a/go/vt/vtgate/planbuilder/operators/operator.go b/go/vt/vtgate/planbuilder/operators/operator.go index 238dae13b3d..a1cfbfd0cc0 100644 --- a/go/vt/vtgate/planbuilder/operators/operator.go +++ b/go/vt/vtgate/planbuilder/operators/operator.go @@ -17,19 +17,21 @@ limitations under the License. // Package operators contains the operators used to plan queries. /* The operators go through a few phases while planning: -1. Logical - In this first pass, we build an operator tree from the incoming parsed query. - It will contain logical joins - we still haven't decided on the join algorithm to use yet. - At the leaves, it will contain QueryGraphs - these are the tables in the FROM clause - that we can easily do join ordering on. The logical tree will represent the full query, - including projections, Grouping, ordering and so on. -2. Physical - Once the logical plan has been fully built, we go bottom up and plan which routes that will be used. - During this phase, we will also decide which join algorithms should be used on the vtgate level -3. Columns & Aggregation - Once we know which queries will be sent to the tablets, we go over the tree and decide which - columns each operator should output. At this point, we also do offset lookups, - so we know at runtime from which columns in the input table we need to read. +1. Initial plan + In this first pass, we build an operator tree from the incoming parsed query. + At the leaves, it will contain QueryGraphs - these are the tables in the FROM clause + that we can easily do join ordering on because they are all inner joins. + All the post-processing - aggregations, sorting, limit etc. are at this stage + contained in Horizon structs. We try to push these down under routes, and expand + the ones that can't be pushed down into individual operators such as Projection, + Agreggation, Limit, etc. +2. Planning + Once the initial plan has been fully built, we go through a number of phases. + recursively running rewriters on the tree in a fixed point fashion, until we've gone + over all phases and the tree has stop changing. +3. Offset planning + Now is the time to stop working with AST objects and transform remaining expressions being + used on top of vtgate to either offsets on inputs or evalengine expressions. */ package operators @@ -75,11 +77,7 @@ func PlanQuery(ctx *plancontext.PlanningContext, stmt sqlparser.Statement) (ops. return nil, err } - if op, err = transformToPhysical(ctx, op); err != nil { - return nil, err - } - - if op, err = tryHorizonPlanning(ctx, op); err != nil { + if op, err = planQuery(ctx, op); err != nil { return nil, err } diff --git a/go/vt/vtgate/planbuilder/operators/ops/op.go b/go/vt/vtgate/planbuilder/operators/ops/op.go index a13dbd51006..87bf9d9e12f 100644 --- a/go/vt/vtgate/planbuilder/operators/ops/op.go +++ b/go/vt/vtgate/planbuilder/operators/ops/op.go @@ -23,11 +23,11 @@ import ( type ( // Operator forms the tree of operators, representing the declarative query provided. - // While planning, the operator tree starts with logical operators, and later moves to physical operators. - // The difference between the two is that when we get to a physical operator, we have made decisions on in - // which order to do the joins, and how to split them up across shards and keyspaces. - // In some situation we go straight to the physical operator - when there are no options to consider, - // we can go straight to the end result. + // The operator tree is no actually runnable, it's an intermediate representation used + // while query planning + // The mental model are operators that pull data from each other, the root being the + // full query output, and the leaves are most often `Route`s, representing communication + // with one or more shards. We want to push down as much work as possible under these Routes Operator interface { // Clone will return a copy of this operator, protected so changed to the original will not impact the clone Clone(inputs []Operator) Operator diff --git a/go/vt/vtgate/planbuilder/operators/phases.go b/go/vt/vtgate/planbuilder/operators/phases.go index 0aed3c8bd71..1eecc595c8f 100644 --- a/go/vt/vtgate/planbuilder/operators/phases.go +++ b/go/vt/vtgate/planbuilder/operators/phases.go @@ -26,60 +26,83 @@ import ( ) type ( - // Phase defines the different planning phases to go through to produce an optimized plan for the input query. - Phase struct { - Name string - // action is the action to be taken before calling plan optimization operation. - action func(ctx *plancontext.PlanningContext, op ops.Operator) (ops.Operator, error) - // shouldRun checks if we should apply this phase or not. - // The phase is only applied if the function returns true - shouldRun func(semantics.QuerySignature) bool - } + Phase int +) + +const ( + physicalTransform Phase = iota + initialPlanning + pullDistinctFromUnion + delegateAggregation + addAggrOrdering + cleanOutPerfDistinct + subquerySettling + DONE ) +func (p Phase) String() string { + switch p { + case physicalTransform: + return "physicalTransform" + case initialPlanning: + return "initial horizon planning optimization" + case pullDistinctFromUnion: + return "pull distinct from UNION1" + case delegateAggregation: + return "split aggregation between vtgate and mysql" + case addAggrOrdering: + return "optimize aggregations with ORDER BY" + case cleanOutPerfDistinct: + return "optimize Distinct operations" + case subquerySettling: + return "settle subqueries" + } + + return "unknown" +} + +func (p Phase) shouldRun(s semantics.QuerySignature) bool { + switch p { + case pullDistinctFromUnion: + return s.Union + case delegateAggregation: + return s.Aggregation + case addAggrOrdering: + return s.Aggregation + case cleanOutPerfDistinct: + return s.Distinct + case subquerySettling: + return s.SubQueries + } + return true +} + +func (p Phase) act(ctx *plancontext.PlanningContext, op ops.Operator) (ops.Operator, error) { + switch p { + case pullDistinctFromUnion: + return pullDistinctFromUNION(ctx, op) + case delegateAggregation: + return enableDelegateAggregation(ctx, op) + case addAggrOrdering: + return addOrderBysForAggregations(ctx, op) + case cleanOutPerfDistinct: + return removePerformanceDistinctAboveRoute(ctx, op) + case subquerySettling: + return settleSubqueries(ctx, op) + } + + return op, nil +} + // getPhases returns the ordered phases that the planner will undergo. // These phases ensure the appropriate collaboration between rewriters. -func getPhases(ctx *plancontext.PlanningContext) []Phase { - phases := []Phase{ - { - // Initial optimization phase. - Name: "initial horizon planning optimization", - }, - { - // Convert UNION with `distinct` to UNION ALL with DISTINCT op on top. - Name: "pull distinct from UNION", - action: pullDistinctFromUNION, - shouldRun: func(s semantics.QuerySignature) bool { return s.Union }, - }, - { - // Split aggregation that has not been pushed under the routes into between work on mysql and vtgate. - Name: "split aggregation between vtgate and mysql", - action: enableDelegateAggregatiion, - shouldRun: func(s semantics.QuerySignature) bool { return s.Aggregation }, - }, - { - // Add ORDER BY for aggregations above the route. - Name: "optimize aggregations with ORDER BY", - action: addOrderBysForAggregations, - shouldRun: func(s semantics.QuerySignature) bool { return s.Aggregation }, - }, - { - // Remove unnecessary Distinct operators above routes. - Name: "optimize Distinct operations", - action: removePerformanceDistinctAboveRoute, - shouldRun: func(s semantics.QuerySignature) bool { return s.Distinct }, - }, - { - // Finalize subqueries after they've been pushed as far as possible. - Name: "settle subqueries", - action: settleSubqueries, - shouldRun: func(s semantics.QuerySignature) bool { return s.SubQueries }, - }, +func getPhases(ctx *plancontext.PlanningContext) (phases []Phase) { + for p := Phase(0); p < DONE; p++ { + if p.shouldRun(ctx.SemTable.QuerySignature) { + phases = append(phases, p) + } } - - return slice.Filter(phases, func(phase Phase) bool { - return phase.shouldRun == nil || phase.shouldRun(ctx.SemTable.QuerySignature) - }) + return } func removePerformanceDistinctAboveRoute(_ *plancontext.PlanningContext, op ops.Operator) (ops.Operator, error) { @@ -93,8 +116,7 @@ func removePerformanceDistinctAboveRoute(_ *plancontext.PlanningContext, op ops. }, stopAtRoute) } -func enableDelegateAggregatiion(ctx *plancontext.PlanningContext, op ops.Operator) (ops.Operator, error) { - ctx.DelegateAggregation = true +func enableDelegateAggregation(ctx *plancontext.PlanningContext, op ops.Operator) (ops.Operator, error) { return addColumnsToInput(ctx, op) } diff --git a/go/vt/vtgate/planbuilder/operators/projection.go b/go/vt/vtgate/planbuilder/operators/projection.go index 03709763feb..686950ba56d 100644 --- a/go/vt/vtgate/planbuilder/operators/projection.go +++ b/go/vt/vtgate/planbuilder/operators/projection.go @@ -39,13 +39,41 @@ type Projection struct { // Columns contain the expressions as viewed from the outside of this operator Columns ProjCols - // TableID will be non-nil for derived tables - TableID *semantics.TableSet - Alias string - + // DT will hold all the necessary information if this is a derived table projection + DT *DerivedTable FromAggr bool } +type ( + DerivedTable struct { + TableID semantics.TableSet + Alias string + Columns sqlparser.Columns + } +) + +func (dt *DerivedTable) String() string { + return fmt.Sprintf("DERIVED %s(%s)", dt.Alias, sqlparser.String(dt.Columns)) +} + +func (dt *DerivedTable) RewriteExpression(ctx *plancontext.PlanningContext, expr sqlparser.Expr) (sqlparser.Expr, error) { + if dt == nil { + return expr, nil + } + tableInfo, err := ctx.SemTable.TableInfoFor(dt.TableID) + if err != nil { + return nil, err + } + return semantics.RewriteDerivedTableExpression(expr, tableInfo), nil +} + +func (dt *DerivedTable) introducesTableID() semantics.TableSet { + if dt == nil { + return semantics.EmptyTableSet() + } + return dt.TableID +} + type ( // ProjCols is used to enable projections that are only valid if we can push them into a route, and we never need to ask it about offsets ProjCols interface { @@ -191,7 +219,7 @@ func createSimpleProjection(ctx *plancontext.PlanningContext, qp *QueryProjectio // been settled. Once they have settled, we know where to push the projection, but if we push too early // the projection can end up in the wrong branch of joins func (p *Projection) canPush(ctx *plancontext.PlanningContext) bool { - if ctx.SubqueriesSettled { + if reachedPhase(ctx, subquerySettling) { return true } ap, ok := p.Columns.(AliasedProjections) @@ -216,7 +244,7 @@ func (p *Projection) GetAliasedProjections() (AliasedProjections, error) { } func (p *Projection) isDerived() bool { - return p.TableID != nil + return p.DT != nil } func (p *Projection) FindCol(ctx *plancontext.PlanningContext, expr sqlparser.Expr, underRoute bool) (int, error) { @@ -290,14 +318,9 @@ func (p *Projection) addColumn( ae *sqlparser.AliasedExpr, push bool, ) (int, error) { - expr := ae.Expr - if p.isDerived() { - // For derived tables we rewrite the expression before searching for it and/or pushing it down - tableInfo, err := ctx.SemTable.TableInfoFor(*p.TableID) - if err != nil { - return 0, err - } - expr = semantics.RewriteDerivedTableExpression(expr, tableInfo) + expr, err := p.DT.RewriteExpression(ctx, ae.Expr) + if err != nil { + return 0, err } if reuse { @@ -349,8 +372,7 @@ func (p *Projection) Clone(inputs []ops.Operator) ops.Operator { return &Projection{ Source: inputs[0], Columns: p.Columns, // TODO don't think we need to deep clone here - TableID: p.TableID, - Alias: p.Alias, + DT: p.DT, FromAggr: p.FromAggr, } } @@ -422,8 +444,8 @@ func (p *Projection) AllOffsets() (cols []int) { func (p *Projection) ShortDescription() string { var result []string - if p.Alias != "" { - result = append(result, "derived["+p.Alias+"]") + if p.DT != nil { + result = append(result, p.DT.String()) } switch columns := p.Columns.(type) { @@ -588,15 +610,9 @@ func (p *Projection) planOffsets(ctx *plancontext.PlanningContext) error { } } - p.TableID = nil - p.Alias = "" - return nil } func (p *Projection) introducesTableID() semantics.TableSet { - if p.TableID == nil { - return semantics.EmptyTableSet() - } - return *p.TableID + return p.DT.introducesTableID() } diff --git a/go/vt/vtgate/planbuilder/operators/horizon_planning.go b/go/vt/vtgate/planbuilder/operators/query_planning.go similarity index 87% rename from go/vt/vtgate/planbuilder/operators/horizon_planning.go rename to go/vt/vtgate/planbuilder/operators/query_planning.go index 683930b0130..b4ecbdb4a7f 100644 --- a/go/vt/vtgate/planbuilder/operators/horizon_planning.go +++ b/go/vt/vtgate/planbuilder/operators/query_planning.go @@ -21,7 +21,6 @@ import ( "io" "vitess.io/vitess/go/vt/sqlparser" - "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/planbuilder/operators/ops" "vitess.io/vitess/go/vt/vtgate/planbuilder/operators/rewrite" "vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext" @@ -30,36 +29,14 @@ import ( type ( projector struct { - columns []*ProjExpr + columns []*ProjExpr + columnAliases sqlparser.Columns + explicitColumnAliases bool } ) -func errHorizonNotPlanned() error { - if rewrite.DebugOperatorTree { - fmt.Println("ERROR! Falling back on the old horizon planner") - } - return _errHorizonNotPlanned -} - -var _errHorizonNotPlanned = vterrors.VT12001("query cannot be fully operator planned") - -func tryHorizonPlanning(ctx *plancontext.PlanningContext, root ops.Operator) (output ops.Operator, err error) { - backup := Clone(root) - defer func() { - // If we encounter the _errHorizonNotPlanned error, we'll revert to using the old horizon planning strategy. - if err == _errHorizonNotPlanned { - // The only offset planning we did before was on joins. - // Therefore, we traverse the tree to find all joins and calculate the joinColumns offsets. - // Our fallback strategy is to clone the original operator tree, compute the join offsets, - // and allow the legacy horizonPlanner to handle this query using logical plans. - err = planOffsetsOnJoins(ctx, backup) - if err == nil { - output = backup - } - } - }() - - output, err = planHorizons(ctx, root) +func planQuery(ctx *plancontext.PlanningContext, root ops.Operator) (output ops.Operator, err error) { + output, err = runPhases(ctx, root) if err != nil { return nil, err } @@ -82,23 +59,24 @@ func tryHorizonPlanning(ctx *plancontext.PlanningContext, root ops.Operator) (ou return addTruncationOrProjectionToReturnOutput(ctx, root, output) } -// planHorizons is the process of figuring out how to perform the operations in the Horizon +// runPhases is the process of figuring out how to perform the operations in the Horizon // 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 planHorizons(ctx *plancontext.PlanningContext, root ops.Operator) (op ops.Operator, err error) { +func runPhases(ctx *plancontext.PlanningContext, root ops.Operator) (op ops.Operator, err error) { op = root for _, phase := range getPhases(ctx) { + ctx.CurrentPhase = int(phase) if rewrite.DebugOperatorTree { - fmt.Printf("PHASE: %s\n", phase.Name) + fmt.Printf("PHASE: %s\n", phase.String()) } - if phase.action != nil { - op, err = phase.action(ctx, op) - if err != nil { - return nil, err - } + + op, err = phase.act(ctx, op) + if err != nil { + return nil, err } - op, err = optimizeHorizonPlanning(ctx, op) + + op, err = runRewriters(ctx, op) if err != nil { return nil, err } @@ -112,11 +90,13 @@ func planHorizons(ctx *plancontext.PlanningContext, root ops.Operator) (op ops.O return addGroupByOnRHSOfJoin(op) } -func optimizeHorizonPlanning(ctx *plancontext.PlanningContext, root ops.Operator) (ops.Operator, error) { +func runRewriters(ctx *plancontext.PlanningContext, root ops.Operator) (ops.Operator, error) { visitor := func(in ops.Operator, _ semantics.TableSet, isRoot bool) (ops.Operator, *rewrite.ApplyResult, error) { switch in := in.(type) { case *Horizon: return pushOrExpandHorizon(ctx, in) + case *Join: + return optimizeJoin(ctx, in) case *Projection: return tryPushProjection(ctx, in) case *Limit: @@ -150,7 +130,7 @@ func pushLockAndComment(l *LockAndComment) (ops.Operator, *rewrite.ApplyResult, case *Horizon, *QueryGraph: // we want to wait until the horizons have been pushed under a route or expanded // that way we know that we've replaced the QueryGraphs with Routes - return nil, rewrite.SameTree, nil + return src, rewrite.SameTree, nil case *Route: src.Comments = l.Comments src.Lock = l.Lock @@ -170,8 +150,18 @@ func pushLockAndComment(l *LockAndComment) (ops.Operator, *rewrite.ApplyResult, } func pushOrExpandHorizon(ctx *plancontext.PlanningContext, in *Horizon) (ops.Operator, *rewrite.ApplyResult, error) { - if len(in.ColumnAliases) > 0 { - return nil, nil, errHorizonNotPlanned() + if in.IsDerived() { + newOp, result, err := pushDerived(ctx, in) + if err != nil { + return nil, nil, err + } + if result != rewrite.SameTree { + return newOp, result, nil + } + } + + if !reachedPhase(ctx, initialPlanning) { + return in, rewrite.SameTree, nil } if ctx.SemTable.QuerySignature.SubQueries { @@ -244,7 +234,7 @@ func pushProjectionToOuter(ctx *plancontext.PlanningContext, p *Projection, sq * return p, rewrite.SameTree, nil } - if !ctx.SubqueriesSettled || err != nil { + if !reachedPhase(ctx, subquerySettling) || err != nil { return p, rewrite.SameTree, nil } @@ -287,8 +277,11 @@ func pushProjectionInVindex( return src, rewrite.NewTree("push projection into vindex", p), nil } -func (p *projector) add(pe *ProjExpr) { +func (p *projector) add(pe *ProjExpr, col *sqlparser.IdentifierCI) { p.columns = append(p.columns, pe) + if col != nil { + p.columnAliases = append(p.columnAliases, *col) + } } // pushProjectionInApplyJoin pushes down a projection operation into an ApplyJoin operation. @@ -306,16 +299,24 @@ func pushProjectionInApplyJoin( return p, rewrite.SameTree, nil } lhs, rhs := &projector{}, &projector{} + if p.DT != nil && len(p.DT.Columns) > 0 { + lhs.explicitColumnAliases = true + rhs.explicitColumnAliases = true + } src.JoinColumns = nil - for _, pe := range ap { - err := splitProjectionAcrossJoin(ctx, src, lhs, rhs, pe) + for idx, pe := range ap { + var col *sqlparser.IdentifierCI + if p.DT != nil && idx < len(p.DT.Columns) { + col = &p.DT.Columns[idx] + } + err := splitProjectionAcrossJoin(ctx, src, lhs, rhs, pe, col) if err != nil { return nil, nil, err } } - if p.TableID != nil { + if p.isDerived() { err := exposeColumnsThroughDerivedTable(ctx, p, src, lhs) if err != nil { return nil, nil, err @@ -323,12 +324,12 @@ func pushProjectionInApplyJoin( } // Create and update the Projection operators for the left and right children, if needed. - src.LHS, err = createProjectionWithTheseColumns(ctx, src.LHS, lhs, p.TableID, p.Alias) + src.LHS, err = createProjectionWithTheseColumns(ctx, src.LHS, lhs, p.DT) if err != nil { return nil, nil, err } - src.RHS, err = createProjectionWithTheseColumns(ctx, src.RHS, rhs, p.TableID, p.Alias) + src.RHS, err = createProjectionWithTheseColumns(ctx, src.RHS, rhs, p.DT) if err != nil { return nil, nil, err } @@ -343,6 +344,7 @@ func splitProjectionAcrossJoin( join *ApplyJoin, lhs, rhs *projector, pe *ProjExpr, + colAlias *sqlparser.IdentifierCI, ) error { // Check if the current expression can reuse an existing column in the ApplyJoin. @@ -350,7 +352,7 @@ func splitProjectionAcrossJoin( return nil } - col, err := splitUnexploredExpression(ctx, join, lhs, rhs, pe) + col, err := splitUnexploredExpression(ctx, join, lhs, rhs, pe, colAlias) if err != nil { return err } @@ -365,6 +367,7 @@ func splitUnexploredExpression( join *ApplyJoin, lhs, rhs *projector, pe *ProjExpr, + colAlias *sqlparser.IdentifierCI, ) (JoinColumn, error) { // Get a JoinColumn for the current expression. col, err := join.getJoinColumnFor(ctx, pe.Original, pe.ColExpr, false) @@ -375,17 +378,23 @@ func splitUnexploredExpression( // Update the left and right child columns and names based on the JoinColumn type. switch { case col.IsPureLeft(): - lhs.add(pe) + lhs.add(pe, colAlias) case col.IsPureRight(): - rhs.add(pe) + rhs.add(pe, colAlias) case col.IsMixedLeftAndRight(): for _, lhsExpr := range col.LHSExprs { - lhs.add(newProjExpr(aeWrap(lhsExpr.Expr))) + var lhsAlias *sqlparser.IdentifierCI + if colAlias != nil { + // we need to add an explicit column alias here. let's try just the ColName as is first + ci := sqlparser.NewIdentifierCI(sqlparser.String(lhsExpr.Expr)) + lhsAlias = &ci + } + lhs.add(newProjExpr(aeWrap(lhsExpr.Expr)), lhsAlias) } innerPE := newProjExprWithInner(pe.Original, col.RHSExpr) innerPE.ColExpr = col.RHSExpr innerPE.Info = pe.Info - rhs.add(innerPE) + rhs.add(innerPE, colAlias) } return col, nil } @@ -403,7 +412,7 @@ func splitUnexploredExpression( // LHS expressions to include the derived table. This allows the expressions to be accessed outside // the derived table. func exposeColumnsThroughDerivedTable(ctx *plancontext.PlanningContext, p *Projection, src *ApplyJoin, lhs *projector) error { - derivedTbl, err := ctx.SemTable.TableInfoFor(*p.TableID) + derivedTbl, err := ctx.SemTable.TableInfoFor(p.DT.TableID) if err != nil { return err } @@ -429,7 +438,13 @@ func exposeColumnsThroughDerivedTable(ctx *plancontext.PlanningContext, p *Proje alias := sqlparser.UnescapedString(out) predicate.LHSExprs[idx].Expr = sqlparser.NewColNameWithQualifier(alias, derivedTblName) - lhs.add(newProjExprWithInner(&sqlparser.AliasedExpr{Expr: out, As: sqlparser.NewIdentifierCI(alias)}, out)) + identifierCI := sqlparser.NewIdentifierCI(alias) + projExpr := newProjExprWithInner(&sqlparser.AliasedExpr{Expr: out, As: identifierCI}, out) + var colAlias *sqlparser.IdentifierCI + if lhs.explicitColumnAliases { + colAlias = &identifierCI + } + lhs.add(projExpr, colAlias) } } return nil @@ -451,8 +466,7 @@ func createProjectionWithTheseColumns( ctx *plancontext.PlanningContext, src ops.Operator, p *projector, - tableID *semantics.TableSet, - alias string, + dt *DerivedTable, ) (ops.Operator, error) { if len(p.columns) == 0 { return src, nil @@ -462,8 +476,12 @@ func createProjectionWithTheseColumns( return nil, err } proj.Columns = AliasedProjections(p.columns) - proj.TableID = tableID - proj.Alias = alias + if dt != nil { + kopy := *dt + kopy.Columns = p.columnAliases + proj.DT = &kopy + } + return proj, nil } @@ -592,7 +610,7 @@ func pushOrderingUnderAggr(ctx *plancontext.PlanningContext, order *Ordering, ag // If Aggregator is a derived table, then we should rewrite the ordering before pushing. if aggregator.isDerived() { for idx, orderExpr := range order.Order { - ti, err := ctx.SemTable.TableInfoFor(*aggregator.TableID) + ti, err := ctx.SemTable.TableInfoFor(aggregator.DT.TableID) if err != nil { return nil, nil, err } @@ -664,6 +682,21 @@ func canPushLeft(ctx *plancontext.PlanningContext, aj *ApplyJoin, order []ops.Or return true } +func isOuterTable(op ops.Operator, ts semantics.TableSet) bool { + aj, ok := op.(*ApplyJoin) + if ok && aj.LeftJoin && TableID(aj.RHS).IsOverlapping(ts) { + return true + } + + for _, op := range op.Inputs() { + if isOuterTable(op, ts) { + return true + } + } + + return false +} + func tryPushFilter(ctx *plancontext.PlanningContext, in *Filter) (ops.Operator, *rewrite.ApplyResult, error) { switch src := in.Source.(type) { case *Projection: @@ -671,9 +704,13 @@ func tryPushFilter(ctx *plancontext.PlanningContext, in *Filter) (ops.Operator, case *Route: for _, pred := range in.Predicates { var err error - src.Routing, err = src.Routing.updateRoutingLogic(ctx, pred) - if err != nil { - return nil, nil, err + deps := ctx.SemTable.RecursiveDeps(pred) + if !isOuterTable(src, deps) { + // we can only update based on predicates on inner tables + src.Routing, err = src.Routing.updateRoutingLogic(ctx, pred) + if err != nil { + return nil, nil, err + } } } return rewrite.Swap(in, src, "push filter into Route") diff --git a/go/vt/vtgate/planbuilder/operators/route.go b/go/vt/vtgate/planbuilder/operators/route.go index 83fabb9dc42..cb6e2101e37 100644 --- a/go/vt/vtgate/planbuilder/operators/route.go +++ b/go/vt/vtgate/planbuilder/operators/route.go @@ -648,8 +648,10 @@ func addMultipleColumnsToInput(ctx *plancontext.PlanningContext, operator ops.Op proj := &Projection{ Source: op, Columns: AliasedProjections(slice.Map(unionColumns, newProjExpr)), - TableID: &tableID, - Alias: "dt", + DT: &DerivedTable{ + TableID: tableID, + Alias: "dt", + }, } return addMultipleColumnsToInput(ctx, proj, reuse, addToGroupBy, exprs) default: diff --git a/go/vt/vtgate/planbuilder/operators/route_planning.go b/go/vt/vtgate/planbuilder/operators/route_planning.go index 83f64013eb8..720f2f56480 100644 --- a/go/vt/vtgate/planbuilder/operators/route_planning.go +++ b/go/vt/vtgate/planbuilder/operators/route_planning.go @@ -41,43 +41,6 @@ type ( opCacheMap map[tableSetPair]ops.Operator ) -// TransformToPhysical takes an operator tree and rewrites any parts that have not yet been planned as physical operators. -// This is where a lot of the optimisations of the query plans are done. -// Here we try to merge query parts into the same route primitives. At the end of this process, -// all the operators in the tree are guaranteed to be PhysicalOperators -func transformToPhysical(ctx *plancontext.PlanningContext, in ops.Operator) (ops.Operator, error) { - op, err := rewrite.BottomUpAll(in, TableID, func(operator ops.Operator, ts semantics.TableSet, _ bool) (ops.Operator, *rewrite.ApplyResult, error) { - switch op := operator.(type) { - case *QueryGraph: - return optimizeQueryGraph(ctx, op) - case *Join: - return optimizeJoin(ctx, op) - case *Horizon: - if op.TableId != nil { - return pushDerived(ctx, op) - } - case *Filter: - return pushFilter(op) - } - return operator, rewrite.SameTree, nil - }) - - if err != nil { - return nil, err - } - - return compact(ctx, op) -} - -func pushFilter(op *Filter) (ops.Operator, *rewrite.ApplyResult, error) { - // TODO: once all horizon planning has been moved to the operators, we can remove this method - if _, ok := op.Source.(*Route); ok { - return rewrite.Swap(op, op.Source, "push filter into Route") - } - - return op, rewrite.SameTree, nil -} - func pushDerived(ctx *plancontext.PlanningContext, op *Horizon) (ops.Operator, *rewrite.ApplyResult, error) { innerRoute, ok := op.Source.(*Route) if !ok { @@ -424,7 +387,7 @@ func mergeOrJoin(ctx *plancontext.PlanningContext, lhs, rhs ops.Operator, joinPr if err != nil { return nil, nil, err } - return newOp, rewrite.NewTree("merge routes, but switch sides", newOp), nil + return newOp, rewrite.NewTree("logical join to applyJoin, switching side because derived table", newOp), nil } join := NewApplyJoin(Clone(lhs), Clone(rhs), nil, !inner) diff --git a/go/vt/vtgate/planbuilder/operators/subquery_planning.go b/go/vt/vtgate/planbuilder/operators/subquery_planning.go index 93046648744..44bce0e0f2e 100644 --- a/go/vt/vtgate/planbuilder/operators/subquery_planning.go +++ b/go/vt/vtgate/planbuilder/operators/subquery_planning.go @@ -101,7 +101,6 @@ func settleSubqueries(ctx *plancontext.PlanningContext, op ops.Operator) (ops.Op } return op, rewrite.SameTree, nil } - ctx.SubqueriesSettled = true return rewrite.BottomUp(op, TableID, visit, nil) } @@ -431,6 +430,10 @@ func rewriteColNameToArgument(ctx *plancontext.PlanningContext, in sqlparser.Exp } func pushOrMergeSubQueryContainer(ctx *plancontext.PlanningContext, in *SubQueryContainer) (ops.Operator, *rewrite.ApplyResult, error) { + if !reachedPhase(ctx, initialPlanning) { + return in, rewrite.SameTree, nil + } + var remaining []*SubQuery var result *rewrite.ApplyResult for _, inner := range in.Inner { diff --git a/go/vt/vtgate/planbuilder/other_read.go b/go/vt/vtgate/planbuilder/other_read.go index f9666ef0e4d..1f1c1a1a6ba 100644 --- a/go/vt/vtgate/planbuilder/other_read.go +++ b/go/vt/vtgate/planbuilder/other_read.go @@ -35,7 +35,7 @@ func buildOtherReadAndAdmin(sql string, vschema plancontext.VSchema) (*planResul return newPlanResult(&engine.Send{ Keyspace: keyspace, TargetDestination: destination, - Query: sql, //This is original sql query to be passed as the parser can provide partial ddl AST. + Query: sql, // This is original sql query to be passed as the parser can provide partial ddl AST. SingleShardOnly: true, }), nil } diff --git a/go/vt/vtgate/planbuilder/plancontext/planning_context.go b/go/vt/vtgate/planbuilder/plancontext/planning_context.go index 442ea25b5a1..d090a593a39 100644 --- a/go/vt/vtgate/planbuilder/plancontext/planning_context.go +++ b/go/vt/vtgate/planbuilder/plancontext/planning_context.go @@ -31,26 +31,18 @@ type PlanningContext struct { // e.g. [FROM tblA JOIN tblB ON a.colA = b.colB] will be rewritten to [FROM tblB WHERE :a_colA = b.colB], // if we assume that tblB is on the RHS of the join. This last predicate in the WHERE clause is added to the // map below - JoinPredicates map[sqlparser.Expr][]sqlparser.Expr - SkipPredicates map[sqlparser.Expr]any - PlannerVersion querypb.ExecuteOptions_PlannerVersion - RewriteDerivedExpr bool + JoinPredicates map[sqlparser.Expr][]sqlparser.Expr + SkipPredicates map[sqlparser.Expr]any + PlannerVersion querypb.ExecuteOptions_PlannerVersion // If we during planning have turned this expression into an argument name, // we can continue using the same argument name ReservedArguments map[sqlparser.Expr]string - // DelegateAggregation tells us when we are allowed to split an aggregation across vtgate and mysql - // We aggregate within a shard, and then at the vtgate level we aggregate the incoming shard aggregates - DelegateAggregation bool - // VerifyAllFKs tells whether we need verification for all the fk constraints on VTGate. // This is required for queries we are running with /*+ SET_VAR(foreign_key_checks=OFF) */ VerifyAllFKs bool - // SubqueriesSettled .. - SubqueriesSettled bool - // ParentFKToIgnore stores a specific parent foreign key that we would need to ignore while planning // a certain query. 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. @@ -58,6 +50,10 @@ type PlanningContext struct { // Projected subqueries that have been merged MergedSubqueries []*sqlparser.Subquery + + // CurrentPhase keeps track of how far we've gone in the planning process + // The type should be operators.Phase, but depending on that would lead to circular dependencies + CurrentPhase int } func CreatePlanningContext(stmt sqlparser.Statement, diff --git a/go/vt/vtgate/planbuilder/planner.go b/go/vt/vtgate/planbuilder/planner.go index ab965351ac5..984fd83c4e0 100644 --- a/go/vt/vtgate/planbuilder/planner.go +++ b/go/vt/vtgate/planbuilder/planner.go @@ -44,6 +44,8 @@ func gen4Planner(query string, plannerVersion querypb.ExecuteOptions_PlannerVers } } +// pushCommentDirectivesOnPlan adds comments to queries +// TODO: this should move to the operator side of planning func pushCommentDirectivesOnPlan(plan logicalPlan, stmt sqlparser.Statement) logicalPlan { var directives *sqlparser.CommentDirectives cmt, ok := stmt.(sqlparser.Commented) diff --git a/go/vt/vtgate/planbuilder/postprocess.go b/go/vt/vtgate/planbuilder/postprocess.go index 1e23c5921b4..850c68701f8 100644 --- a/go/vt/vtgate/planbuilder/postprocess.go +++ b/go/vt/vtgate/planbuilder/postprocess.go @@ -17,8 +17,6 @@ limitations under the License. package planbuilder import ( - "vitess.io/vitess/go/mysql/collations" - "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/evalengine" @@ -27,40 +25,6 @@ import ( // This file has functions to analyze postprocessing // clauses like ORDER BY, etc. -// make sure we have the right signature for this function -var _ planVisitor = setUpperLimit - -// setUpperLimit is an optimization hint that tells that primitive -// that it does not need to return more than the specified number of rows. -// A primitive that cannot perform this can ignore the request. -func setUpperLimit(plan logicalPlan) (bool, logicalPlan, error) { - switch node := plan.(type) { - case *join, *hashJoin: - return false, node, nil - case *memorySort: - pv := evalengine.NewBindVar("__upper_limit", sqltypes.Int64, collations.CollationBinaryID) - node.eMemorySort.UpperLimit = pv - // we don't want to go down to the rest of the tree - return false, node, nil - case *uncorrelatedSubquery: - // we control the visitation manually here - - // we don't want to visit the subQuery side of this plan - newUnderlying, err := visit(node.outer, setUpperLimit) - if err != nil { - return false, nil, err - } - - node.outer = newUnderlying - return false, node, nil - case *route: - // The route pushes the limit regardless of the plan. - // If it's a scatter query, the rows returned will be - // more than the upper limit, but enough for the limit - node.Select.SetLimit(&sqlparser.Limit{Rowcount: sqlparser.NewArgument("__upper_limit")}) - } - return true, plan, nil -} - func createLimit(input logicalPlan, limit *sqlparser.Limit) (logicalPlan, error) { plan := newLimit(input) pv, err := evalengine.Translate(limit.Rowcount, nil) diff --git a/go/vt/vtgate/planbuilder/projection.go b/go/vt/vtgate/planbuilder/projection.go index a4b4eaf7a8c..70cb1979780 100644 --- a/go/vt/vtgate/planbuilder/projection.go +++ b/go/vt/vtgate/planbuilder/projection.go @@ -22,7 +22,6 @@ import ( "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/engine" - "vitess.io/vitess/go/vt/vtgate/evalengine" "vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext" "vitess.io/vitess/go/vt/vtgate/semantics" ) @@ -41,28 +40,8 @@ var _ logicalPlan = (*projection)(nil) // Wireup implements the logicalPlan interface func (p *projection) Wireup(ctx *plancontext.PlanningContext) error { - if p.primitive != nil { - // if primitive is not nil, it means that the horizon planning in the operator phase already - // created all the needed evalengine expressions. - // we don't need to do anything here, let's just shortcut out of this call - return p.source.Wireup(ctx) - } - - columns := make([]evalengine.Expr, 0, len(p.columns)) - for _, expr := range p.columns { - convert, err := evalengine.Translate(expr, &evalengine.Config{ - ResolveColumn: resolveFromPlan(ctx, p.source, false), - ResolveType: ctx.SemTable.TypeForExpr, - Collation: ctx.SemTable.Collation, - }) - if err != nil { - return err - } - columns = append(columns, convert) - } - p.primitive = &engine.Projection{ - Cols: p.columnNames, - Exprs: columns, + if p.primitive == nil { + return vterrors.VT13001("should already be done") } return p.source.Wireup(ctx) @@ -107,21 +86,3 @@ func (p *projection) Primitive() engine.Primitive { p.primitive.Input = p.source.Primitive() return p.primitive } - -// addColumn is used to add a column output for the projection. -// This is the only function that should be used to add columns to projection -func (p *projection) addColumn(idx *int, column sqlparser.Expr, columnName string) (int, error) { - var offset int - if idx == nil { - p.unorderedColumnIdx++ - offset = len(p.columns) - p.unorderedColumnIdx - } else { - offset = *idx - } - if p.columnNames[offset] != "" || p.columns[offset] != nil { - return -1, vterrors.VT13001("overwriting columns in projection is not permitted") - } - p.columns[offset] = column - p.columnNames[offset] = columnName - return offset, nil -} diff --git a/go/vt/vtgate/planbuilder/projection_pushing.go b/go/vt/vtgate/planbuilder/projection_pushing.go deleted file mode 100644 index cf34dbfe97c..00000000000 --- a/go/vt/vtgate/planbuilder/projection_pushing.go +++ /dev/null @@ -1,332 +0,0 @@ -/* -Copyright 2022 The Vitess Authors. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package planbuilder - -import ( - "fmt" - - "vitess.io/vitess/go/vt/sqlparser" - "vitess.io/vitess/go/vt/vterrors" - "vitess.io/vitess/go/vt/vtgate/engine" - popcode "vitess.io/vitess/go/vt/vtgate/engine/opcode" - "vitess.io/vitess/go/vt/vtgate/planbuilder/operators" - "vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext" - "vitess.io/vitess/go/vt/vtgate/semantics" -) - -// pushProjection pushes a projection to the plan. -func pushProjection( - ctx *plancontext.PlanningContext, - expr *sqlparser.AliasedExpr, - plan logicalPlan, - inner, reuseCol, hasAggregation bool, -) (offset int, added bool, err error) { - switch node := plan.(type) { - case *limit, *projection, *uncorrelatedSubquery, *distinct, *filter: - // All of these either push to the single source, or push to the LHS - src := node.Inputs()[0] - return pushProjection(ctx, expr, src, inner, reuseCol, hasAggregation) - case *route: - return addExpressionToRoute(ctx, node, expr, reuseCol) - case *hashJoin: - return pushProjectionIntoHashJoin(ctx, expr, node, reuseCol, inner, hasAggregation) - case *join: - return pushProjectionIntoJoin(ctx, expr, node, reuseCol, inner, hasAggregation) - case *simpleProjection: - return pushProjectionIntoSimpleProj(ctx, expr, node, inner, hasAggregation, reuseCol) - case *orderedAggregate: - return pushProjectionIntoOA(ctx, expr, node, inner, hasAggregation) - case *vindexFunc: - return pushProjectionIntoVindexFunc(node, expr, reuseCol) - case *semiJoin: - return pushProjectionIntoSemiJoin(ctx, expr, reuseCol, node, inner, hasAggregation) - case *concatenate: - return pushProjectionIntoConcatenate(ctx, expr, hasAggregation, node, inner, reuseCol) - default: - return 0, false, vterrors.VT13001(fmt.Sprintf("push projection does not yet support: %T", node)) - } -} - -func pushProjectionIntoVindexFunc(node *vindexFunc, expr *sqlparser.AliasedExpr, reuseCol bool) (int, bool, error) { - colsBefore := len(node.eVindexFunc.Cols) - i, err := node.SupplyProjection(expr, reuseCol) - if err != nil { - return 0, false, err - } - return i /* col added */, len(node.eVindexFunc.Cols) > colsBefore, nil -} - -func pushProjectionIntoConcatenate(ctx *plancontext.PlanningContext, expr *sqlparser.AliasedExpr, hasAggregation bool, node *concatenate, inner bool, reuseCol bool) (int, bool, error) { - if hasAggregation { - return 0, false, vterrors.VT12001("aggregation on UNIONs") - } - offset, added, err := pushProjection(ctx, expr, node.sources[0], inner, reuseCol, hasAggregation) - if err != nil { - return 0, false, err - } - if added && ctx.SemTable.DirectDeps(expr.Expr).NonEmpty() { - return 0, false, vterrors.VT13001(fmt.Sprintf("pushing projection %v on concatenate should reference an existing column", sqlparser.String(expr))) - } - if added { - for _, source := range node.sources[1:] { - _, _, err := pushProjection(ctx, expr, source, inner, reuseCol, hasAggregation) - if err != nil { - return 0, false, err - } - } - } - return offset, added, nil -} - -func pushProjectionIntoSemiJoin( - ctx *plancontext.PlanningContext, - expr *sqlparser.AliasedExpr, - reuseCol bool, - node *semiJoin, - inner, hasAggregation bool, -) (int, bool, error) { - passDownReuseCol := reuseCol - if !reuseCol { - passDownReuseCol = expr.As.IsEmpty() - } - offset, added, err := pushProjection(ctx, expr, node.lhs, inner, passDownReuseCol, hasAggregation) - if err != nil { - return 0, false, err - } - column := -(offset + 1) - if reuseCol && !added { - for idx, col := range node.cols { - if column == col { - return idx, false, nil - } - } - } - node.cols = append(node.cols, column) - return len(node.cols) - 1, true, nil -} - -func pushProjectionIntoOA(ctx *plancontext.PlanningContext, expr *sqlparser.AliasedExpr, node *orderedAggregate, inner, hasAggregation bool) (int, bool, error) { - colName, isColName := expr.Expr.(*sqlparser.ColName) - for _, aggregate := range node.aggregates { - if ctx.SemTable.EqualsExpr(aggregate.Expr, expr.Expr) { - return aggregate.Col, false, nil - } - if isColName && colName.Name.EqualString(aggregate.Alias) { - return aggregate.Col, false, nil - } - } - for _, key := range node.groupByKeys { - if ctx.SemTable.EqualsExpr(key.Expr, expr.Expr) { - return key.KeyCol, false, nil - } - } - offset, _, err := pushProjection(ctx, expr, node.input, inner, true, hasAggregation) - if err != nil { - return 0, false, err - } - aggr := engine.NewAggregateParam(popcode.AggregateAnyValue, offset, expr.ColumnName()) - aggr.Expr = expr.Expr - aggr.Original = expr - node.aggregates = append(node.aggregates, aggr) - return offset, true, nil -} - -func pushProjectionIntoSimpleProj( - ctx *plancontext.PlanningContext, - expr *sqlparser.AliasedExpr, - node *simpleProjection, - inner, hasAggregation, reuseCol bool, -) (int, bool, error) { - offset, _, err := pushProjection(ctx, expr, node.input, inner, true, hasAggregation) - if err != nil { - return 0, false, err - } - for i, value := range node.eSimpleProj.Cols { - // we return early if we already have the column in the simple projection's - // output list so we do not add it again. - if reuseCol && value == offset { - return i, false, nil - } - } - node.eSimpleProj.Cols = append(node.eSimpleProj.Cols, offset) - return len(node.eSimpleProj.Cols) - 1, true, nil -} - -func pushProjectionIntoJoin( - ctx *plancontext.PlanningContext, - expr *sqlparser.AliasedExpr, - node *join, - reuseCol, inner, hasAggregation bool, -) (int, bool, error) { - lhsSolves := node.Left.ContainsTables() - rhsSolves := node.Right.ContainsTables() - deps := ctx.SemTable.RecursiveDeps(expr.Expr) - var column int - var appended bool - passDownReuseCol := reuseCol - if !reuseCol { - passDownReuseCol = expr.As.IsEmpty() - } - switch { - case deps.IsSolvedBy(lhsSolves): - offset, added, err := pushProjection(ctx, expr, node.Left, inner, passDownReuseCol, hasAggregation) - if err != nil { - return 0, false, err - } - column = -(offset + 1) - appended = added - case deps.IsSolvedBy(rhsSolves): - offset, added, err := pushProjection(ctx, expr, node.Right, inner && node.Opcode != engine.LeftJoin, passDownReuseCol, hasAggregation) - if err != nil { - return 0, false, err - } - column = offset + 1 - appended = added - default: - // if an expression has aggregation, then it should not be split up and pushed to both sides, - // for example an expression like count(*) will have dependencies on both sides, but we should not push it - // instead we should return an error - if hasAggregation { - return 0, false, vterrors.VT12001("cross-shard query with aggregates") - } - // now we break the expression into left and right side dependencies and rewrite the left ones to bind variables - joinCol, err := operators.BreakExpressionInLHSandRHS(ctx, expr.Expr, lhsSolves) - if err != nil { - return 0, false, err - } - // go over all the columns coming from the left side of the tree and push them down. While at it, also update the bind variable map. - // It is okay to reuse the columns on the left side since - // the final expression which will be selected will be pushed into the right side. - for _, col := range joinCol.LHSExprs { - colOffset, _, err := pushProjection(ctx, &sqlparser.AliasedExpr{Expr: col.Expr}, node.Left, inner, true, false) - if err != nil { - return 0, false, err - } - node.Vars[col.Name] = colOffset - } - // push the rewritten expression on the right side of the tree. Here we should take care whether we want to reuse the expression or not. - expr.Expr = joinCol.RHSExpr - offset, added, err := pushProjection(ctx, expr, node.Right, inner && node.Opcode != engine.LeftJoin, passDownReuseCol, false) - if err != nil { - return 0, false, err - } - column = offset + 1 - appended = added - } - if reuseCol && !appended { - for idx, col := range node.Cols { - if column == col { - return idx, false, nil - } - } - // the column was not appended to either child, but we could not find it in out cols list, - // so we'll still add it - } - node.Cols = append(node.Cols, column) - return len(node.Cols) - 1, true, nil -} - -func pushProjectionIntoHashJoin( - ctx *plancontext.PlanningContext, - expr *sqlparser.AliasedExpr, - node *hashJoin, - reuseCol, inner, hasAggregation bool, -) (int, bool, error) { - lhsSolves := node.Left.ContainsTables() - rhsSolves := node.Right.ContainsTables() - deps := ctx.SemTable.RecursiveDeps(expr.Expr) - var column int - var appended bool - passDownReuseCol := reuseCol - if !reuseCol { - passDownReuseCol = expr.As.IsEmpty() - } - switch { - case deps.IsSolvedBy(lhsSolves): - offset, added, err := pushProjection(ctx, expr, node.Left, inner, passDownReuseCol, hasAggregation) - if err != nil { - return 0, false, err - } - column = -(offset + 1) - appended = added - case deps.IsSolvedBy(rhsSolves): - offset, added, err := pushProjection(ctx, expr, node.Right, inner && node.Opcode != engine.LeftJoin, passDownReuseCol, hasAggregation) - if err != nil { - return 0, false, err - } - column = offset + 1 - appended = added - default: - // if an expression has aggregation, then it should not be split up and pushed to both sides, - // for example an expression like count(*) will have dependencies on both sides, but we should not push it - // instead we should return an error - if hasAggregation { - return 0, false, vterrors.VT12001("cross-shard query with aggregates") - } - return 0, false, vterrors.VT12001("hash join with projection from both sides of the join") - } - if reuseCol && !appended { - for idx, col := range node.Cols { - if column == col { - return idx, false, nil - } - } - // the column was not appended to either child, but we could not find it in out cols list, - // so we'll still add it - } - node.Cols = append(node.Cols, column) - return len(node.Cols) - 1, true, nil -} - -func addExpressionToRoute(ctx *plancontext.PlanningContext, rb *route, expr *sqlparser.AliasedExpr, reuseCol bool) (int, bool, error) { - if reuseCol { - if i := checkIfAlreadyExists(expr, rb.Select, ctx.SemTable); i != -1 { - return i, false, nil - } - } - sqlparser.RemoveKeyspaceFromColName(expr.Expr) - sel, isSel := rb.Select.(*sqlparser.Select) - if !isSel { - return 0, false, vterrors.VT12001(fmt.Sprintf("pushing projection '%s' on %T", sqlparser.String(expr), rb.Select)) - } - - if ctx.RewriteDerivedExpr { - // if we are trying to push a projection that belongs to a DerivedTable - // we rewrite that expression, so it matches the column name used inside - // that derived table. - err := rewriteProjectionOfDerivedTable(expr, ctx.SemTable) - if err != nil { - return 0, false, err - } - } - - offset := len(sel.SelectExprs) - sel.SelectExprs = append(sel.SelectExprs, expr) - return offset, true, nil -} - -func rewriteProjectionOfDerivedTable(expr *sqlparser.AliasedExpr, semTable *semantics.SemTable) error { - ti, err := semTable.TableInfoForExpr(expr.Expr) - if err != nil && err != semantics.ErrNotSingleTable { - return err - } - _, isDerivedTable := ti.(*semantics.DerivedTable) - if isDerivedTable { - expr.Expr = semantics.RewriteDerivedTableExpression(expr.Expr, ti) - } - return nil -} diff --git a/go/vt/vtgate/planbuilder/select.go b/go/vt/vtgate/planbuilder/select.go index 3c74d915f78..a6ea8a7c8b1 100644 --- a/go/vt/vtgate/planbuilder/select.go +++ b/go/vt/vtgate/planbuilder/select.go @@ -85,6 +85,7 @@ func gen4SelectStmtPlanner( if shouldRetryAfterPredicateRewriting(plan) { // by transforming the predicates to CNF, the planner will sometimes find better plans + // TODO: this should move to the operator side of planning plan2, tablesUsed := gen4PredicateRewrite(stmt, getPlan) if plan2 != nil { return newPlanResult(plan2.Primitive(), tablesUsed...), nil @@ -232,8 +233,6 @@ func newBuildSelectPlan( return nil, nil, err } - optimizePlan(plan) - if err = plan.Wireup(ctx); err != nil { return nil, nil, err } @@ -249,108 +248,6 @@ func createSelectOperator(ctx *plancontext.PlanningContext, selStmt sqlparser.Se return operators.PlanQuery(ctx, selStmt) } -// optimizePlan removes unnecessary simpleProjections that have been created while planning -func optimizePlan(plan logicalPlan) { - for _, lp := range plan.Inputs() { - optimizePlan(lp) - } - - this, ok := plan.(*simpleProjection) - if !ok { - return - } - - input, ok := this.input.(*simpleProjection) - if !ok { - return - } - - for i, col := range this.eSimpleProj.Cols { - this.eSimpleProj.Cols[i] = input.eSimpleProj.Cols[col] - } - this.input = input.input -} - -func planLimit(limit *sqlparser.Limit, plan logicalPlan) (logicalPlan, error) { - if limit == nil { - return plan, nil - } - rb, ok := plan.(*route) - if ok && rb.isSingleShard() { - rb.SetLimit(limit) - return plan, nil - } - - lPlan, err := createLimit(plan, limit) - if err != nil { - return nil, err - } - - // visit does not modify the plan. - _, err = visit(lPlan, setUpperLimit) - if err != nil { - return nil, err - } - return lPlan, nil -} - -func planHorizon(ctx *plancontext.PlanningContext, plan logicalPlan, in sqlparser.SelectStatement, truncateColumns bool) (logicalPlan, error) { - switch node := in.(type) { - case *sqlparser.Select: - hp := horizonPlanning{ - sel: node, - } - - var err error - plan, err = hp.planHorizon(ctx, plan, truncateColumns) - if err != nil { - return nil, err - } - plan, err = planLimit(node.Limit, plan) - if err != nil { - return nil, err - } - case *sqlparser.Union: - var err error - rb, isRoute := plan.(*route) - if !isRoute && ctx.SemTable.NotSingleRouteErr != nil { - return nil, ctx.SemTable.NotSingleRouteErr - } - if isRoute && rb.isSingleShard() { - err = planSingleRoutePlan(node, rb) - } else { - plan, err = planOrderByOnUnion(ctx, plan, node) - } - if err != nil { - return nil, err - } - - plan, err = planLimit(node.Limit, plan) - if err != nil { - return nil, err - } - } - return plan, nil - -} - -func planOrderByOnUnion(ctx *plancontext.PlanningContext, plan logicalPlan, union *sqlparser.Union) (logicalPlan, error) { - qp, err := operators.CreateQPFromSelectStatement(ctx, union) - if err != nil { - return nil, err - } - hp := horizonPlanning{ - qp: qp, - } - if len(qp.OrderExprs) > 0 { - plan, err = hp.planOrderBy(ctx, qp.OrderExprs, plan) - if err != nil { - return nil, err - } - } - return plan, nil -} - func isOnlyDual(sel *sqlparser.Select) bool { if sel.Where != nil || sel.GroupBy != nil || sel.Having != nil || sel.Limit != nil || sel.OrderBy != nil { // we can only deal with queries without any other subclauses - just SELECT and FROM, nothing else is allowed diff --git a/go/vt/vtgate/planbuilder/single_sharded_shortcut.go b/go/vt/vtgate/planbuilder/single_sharded_shortcut.go index 3c763ab7060..daf19ced859 100644 --- a/go/vt/vtgate/planbuilder/single_sharded_shortcut.go +++ b/go/vt/vtgate/planbuilder/single_sharded_shortcut.go @@ -102,3 +102,12 @@ func getTableNames(semTable *semantics.SemTable) ([]sqlparser.TableName, error) } return tableNames, nil } + +func removeKeyspaceFromSelectExpr(expr sqlparser.SelectExpr) { + switch expr := expr.(type) { + case *sqlparser.AliasedExpr: + sqlparser.RemoveKeyspaceFromColName(expr.Expr) + case *sqlparser.StarExpr: + expr.TableName.Qualifier = sqlparser.NewIdentifierCS("") + } +} diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.json b/go/vt/vtgate/planbuilder/testdata/from_cases.json index d63bd51703d..f8cf7dd032e 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.json @@ -3076,40 +3076,32 @@ "QueryType": "SELECT", "Original": "select i+1 from (select user.id from user join user_extra) t(i)", "Instructions": { - "OperatorType": "SimpleProjection", - "Columns": [ - 1 - ], + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0", + "TableName": "`user`_user_extra", "Inputs": [ { - "OperatorType": "Join", - "Variant": "Join", - "JoinColumnIndexes": "L:0,L:1", - "TableName": "`user`_user_extra", - "Inputs": [ - { - "OperatorType": "Route", - "Variant": "Scatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select `user`.id, `user`.id + 1 from `user` where 1 != 1", - "Query": "select `user`.id, `user`.id + 1 from `user`", - "Table": "`user`" - }, - { - "OperatorType": "Route", - "Variant": "Scatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select 1 from user_extra where 1 != 1", - "Query": "select 1 from user_extra", - "Table": "user_extra" - } - ] + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select i + 1 from (select `user`.id from `user` where 1 != 1) as t(i) where 1 != 1", + "Query": "select i + 1 from (select `user`.id from `user`) as t(i)", + "Table": "`user`" + }, + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user_extra where 1 != 1", + "Query": "select 1 from user_extra", + "Table": "user_extra" } ] }, diff --git a/go/vt/vtgate/planbuilder/testdata/tpch_cases.json b/go/vt/vtgate/planbuilder/testdata/tpch_cases.json index 98659f336df..947f9cc0f96 100644 --- a/go/vt/vtgate/planbuilder/testdata/tpch_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/tpch_cases.json @@ -1276,7 +1276,98 @@ { "comment": "TPC-H query 13", "query": "select c_count, count(*) as custdist from ( select c_custkey, count(o_orderkey) from customer left outer join orders on c_custkey = o_custkey and o_comment not like '%special%requests%' group by c_custkey ) as c_orders(c_custkey, c_count) group by c_count order by custdist desc, c_count desc", - "plan": "VT12001: unsupported: using aggregation on top of a *planbuilder.orderedAggregate plan" + "plan": { + "QueryType": "SELECT", + "Original": "select c_count, count(*) as custdist from ( select c_custkey, count(o_orderkey) from customer left outer join orders on c_custkey = o_custkey and o_comment not like '%special%requests%' group by c_custkey ) as c_orders(c_custkey, c_count) group by c_count order by custdist desc, c_count desc", + "Instructions": { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "1 DESC, 0 DESC", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count_star(1) AS custdist", + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "SimpleProjection", + "Columns": [ + 1, + 3 + ], + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count(1) AS count(o_orderkey), any_value(3)", + "GroupBy": "(0|2)", + "Inputs": [ + { + "OperatorType": "Projection", + "Expressions": [ + "[COLUMN 2] as c_custkey", + "[COLUMN 1] * [COLUMN 0] as count(o_orderkey)", + "[COLUMN 3] as weight_string(c_custkey)", + "[COLUMN 4] as 1" + ], + "Inputs": [ + { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "0 ASC, (2|3) ASC", + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "LeftJoin", + "JoinColumnIndexes": "R:0,L:0,L:1,L:2,L:3", + "JoinVars": { + "c_custkey": 1 + }, + "TableName": "customer_orders", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select count(*), c_custkey, weight_string(c_custkey), 1 from customer where 1 != 1 group by c_custkey, weight_string(c_custkey)", + "OrderBy": "(1|2) ASC", + "Query": "select count(*), c_custkey, weight_string(c_custkey), 1 from customer group by c_custkey, weight_string(c_custkey) order by c_custkey asc", + "Table": "customer" + }, + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select count(o_orderkey) from orders where 1 != 1 group by .0", + "Query": "select count(o_orderkey) from orders where o_comment not like '%special%requests%' and o_custkey = :c_custkey group by .0", + "Table": "orders" + } + ] + } + ] + } + ] + } + ] + } + ] + } + ] + } + ] + }, + "TablesUsed": [ + "main.customer", + "main.orders" + ] + } }, { "comment": "TPC-H query 14", diff --git a/go/vt/vtgate/planbuilder/testdata/union_cases.json b/go/vt/vtgate/planbuilder/testdata/union_cases.json index e6f84dd1631..baea31d84bd 100644 --- a/go/vt/vtgate/planbuilder/testdata/union_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/union_cases.json @@ -749,22 +749,29 @@ "QueryType": "SELECT", "Original": "select * from ((select id from user union select id+1 from user) union select user_id from user_extra) as t", "Instructions": { - "OperatorType": "Distinct", - "Collations": [ - "(0:1)" + "OperatorType": "SimpleProjection", + "Columns": [ + 0 ], - "ResultColumns": 1, "Inputs": [ { - "OperatorType": "Route", - "Variant": "Scatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select id, weight_string(id) from (select id from `user` where 1 != 1 union select id + 1 from `user` where 1 != 1 union select user_id from user_extra where 1 != 1) as dt where 1 != 1", - "Query": "select id, weight_string(id) from (select id from `user` union select id + 1 from `user` union select user_id from user_extra) as dt", - "Table": "`user`, user_extra" + "OperatorType": "Distinct", + "Collations": [ + "(0:1)" + ], + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, weight_string(id) from (select id from `user` where 1 != 1 union select id + 1 from `user` where 1 != 1 union select user_id from user_extra where 1 != 1) as dt where 1 != 1", + "Query": "select id, weight_string(id) from (select id from `user` union select id + 1 from `user` union select user_id from user_extra) as dt", + "Table": "`user`, user_extra" + } + ] } ] }, @@ -948,41 +955,49 @@ "TableName": "`user`_`user`", "Inputs": [ { - "OperatorType": "Concatenate", + "OperatorType": "SimpleProjection", + "Columns": [ + 0 + ], "Inputs": [ { - "OperatorType": "Limit", - "Count": "INT64(5)", + "OperatorType": "Concatenate", "Inputs": [ { - "OperatorType": "Route", - "Variant": "Scatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select id, weight_string(id) from `user` where 1 != 1", - "OrderBy": "(0|1) ASC", - "Query": "select id, weight_string(id) from `user` order by id asc limit :__upper_limit", - "Table": "`user`" - } - ] - }, - { - "OperatorType": "Limit", - "Count": "INT64(5)", - "Inputs": [ + "OperatorType": "Limit", + "Count": "INT64(5)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, weight_string(id) from `user` where 1 != 1", + "OrderBy": "(0|1) ASC", + "Query": "select id, weight_string(id) from `user` order by id asc limit :__upper_limit", + "Table": "`user`" + } + ] + }, { - "OperatorType": "Route", - "Variant": "Scatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select id, weight_string(id) from `user` where 1 != 1", - "OrderBy": "(0|1) DESC", - "Query": "select id, weight_string(id) from `user` order by id desc limit :__upper_limit", - "Table": "`user`" + "OperatorType": "Limit", + "Count": "INT64(5)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, weight_string(id) from `user` where 1 != 1", + "OrderBy": "(0|1) DESC", + "Query": "select id, weight_string(id) from `user` order by id desc limit :__upper_limit", + "Table": "`user`" + } + ] } ] } @@ -1083,35 +1098,41 @@ ], "Inputs": [ { - "OperatorType": "Distinct", - "Collations": [ - "(0:1)" - ], + "OperatorType": "Projection", + "Expressions": null, "Inputs": [ { - "OperatorType": "Concatenate", + "OperatorType": "Distinct", + "Collations": [ + "(0:1)" + ], "Inputs": [ { - "OperatorType": "Route", - "Variant": "Scatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select id + 42 as foo, weight_string(id + 42) from `user` where 1 != 1", - "Query": "select distinct id + 42 as foo, weight_string(id + 42) from `user`", - "Table": "`user`" - }, - { - "OperatorType": "Route", - "Variant": "Unsharded", - "Keyspace": { - "Name": "main", - "Sharded": false - }, - "FieldQuery": "select 1 + id as foo, weight_string(1 + id) from unsharded where 1 != 1", - "Query": "select distinct 1 + id as foo, weight_string(1 + id) from unsharded", - "Table": "unsharded" + "OperatorType": "Concatenate", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id + 42 as foo, weight_string(id + 42) from `user` where 1 != 1", + "Query": "select distinct id + 42 as foo, weight_string(id + 42) from `user`", + "Table": "`user`" + }, + { + "OperatorType": "Route", + "Variant": "Unsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select 1 + id as foo, weight_string(1 + id) from unsharded where 1 != 1", + "Query": "select distinct 1 + id as foo, weight_string(1 + id) from unsharded", + "Table": "unsharded" + } + ] } ] }