diff --git a/go/test/endtoend/vtgate/queries/aggregation/fuzz_test.go b/go/test/endtoend/vtgate/queries/aggregation/fuzz_test.go index 25bec1a39b4..ba6fca839c4 100644 --- a/go/test/endtoend/vtgate/queries/aggregation/fuzz_test.go +++ b/go/test/endtoend/vtgate/queries/aggregation/fuzz_test.go @@ -40,6 +40,7 @@ type ( ) func TestFuzzAggregations(t *testing.T) { + t.Skip("dont run on CI for now") // This test randomizes values and queries, and checks that mysql returns the same values that Vitess does mcmp, closer := start(t) defer closer() diff --git a/go/vt/vtgate/planbuilder/operators/aggregator.go b/go/vt/vtgate/planbuilder/operators/aggregator.go index c9327b4e384..5da0c10c9fe 100644 --- a/go/vt/vtgate/planbuilder/operators/aggregator.go +++ b/go/vt/vtgate/planbuilder/operators/aggregator.go @@ -98,7 +98,13 @@ func (a *Aggregator) addColumnWithoutPushing(expr *sqlparser.AliasedExpr, addToG groupBy.ColOffset = offset a.Grouping = append(a.Grouping, groupBy) } else { - aggr := NewAggr(opcode.AggregateRandom, nil, expr, expr.As.String()) + var aggr Aggr + switch e := expr.Expr.(type) { + case sqlparser.AggrFunc: + aggr = createAggrFromAggrFunc(e, expr) + default: + aggr = NewAggr(opcode.AggregateRandom, nil, expr, expr.As.String()) + } aggr.ColOffset = offset a.Aggregations = append(a.Aggregations, aggr) } @@ -159,14 +165,19 @@ func (a *Aggregator) AddColumn(ctx *plancontext.PlanningContext, expr *sqlparser return a, offset, nil } -func (a *Aggregator) GetColumns() (columns []*sqlparser.AliasedExpr, err error) { - return a.Columns, nil -} +func (a *Aggregator) GetColumns() ([]*sqlparser.AliasedExpr, error) { + // we update the incoming columns, so we know about any new columns that have been added + columns, err := a.Source.GetColumns() + if err != nil { + return nil, err + } -func (a *Aggregator) Description() ops.OpDescription { - return ops.OpDescription{ - OperatorType: "Aggregator", + // if this operator is producing more columns than expected, we want to know about it + if len(columns) > len(a.Columns) { + a.Columns = append(a.Columns, columns[len(a.Columns):]...) } + + return a.Columns, nil } func (a *Aggregator) ShortDescription() string { @@ -174,8 +185,13 @@ func (a *Aggregator) ShortDescription() string { return sqlparser.String(from) }) + org := "" + if a.Original { + org = "ORG " + } + if len(a.Grouping) == 0 { - return strings.Join(columnns, ", ") + return fmt.Sprintf("%s%s", org, strings.Join(columnns, ", ")) } var grouping []string @@ -183,11 +199,6 @@ func (a *Aggregator) ShortDescription() string { grouping = append(grouping, sqlparser.String(gb.SimplifiedExpr)) } - org := "" - if a.Original { - org = "ORG " - } - return fmt.Sprintf("%s%s group by %s", org, strings.Join(columnns, ", "), strings.Join(grouping, ",")) } diff --git a/go/vt/vtgate/planbuilder/operators/apply_join.go b/go/vt/vtgate/planbuilder/operators/apply_join.go index 6f67e549a7a..816ce47a813 100644 --- a/go/vt/vtgate/planbuilder/operators/apply_join.go +++ b/go/vt/vtgate/planbuilder/operators/apply_join.go @@ -287,21 +287,6 @@ func (a *ApplyJoin) addOffset(offset int) { a.Columns = append(a.Columns, offset) } -func (a *ApplyJoin) Description() ops.OpDescription { - other := map[string]any{} - if len(a.Columns) > 0 { - other["OutputColumns"] = a.Columns - } - if a.Predicate != nil { - other["Predicate"] = sqlparser.String(a.Predicate) - } - return ops.OpDescription{ - OperatorType: "Join", - Variant: "Apply", - Other: other, - } -} - func (a *ApplyJoin) ShortDescription() string { pred := sqlparser.String(a.Predicate) columns := slices2.Map(a.ColumnsAST, func(from JoinColumn) string { diff --git a/go/vt/vtgate/planbuilder/operators/correlated_subquery.go b/go/vt/vtgate/planbuilder/operators/correlated_subquery.go index 40a8497126a..1e59da8e2bc 100644 --- a/go/vt/vtgate/planbuilder/operators/correlated_subquery.go +++ b/go/vt/vtgate/planbuilder/operators/correlated_subquery.go @@ -70,13 +70,6 @@ func (s *SubQueryOp) SetInputs(ops []ops.Operator) { s.Outer, s.Inner = ops[0], ops[1] } -func (s *SubQueryOp) Description() ops.OpDescription { - return ops.OpDescription{ - OperatorType: "SubQuery", - Variant: "Apply", - } -} - func (s *SubQueryOp) ShortDescription() string { return "" } @@ -114,13 +107,6 @@ func (c *CorrelatedSubQueryOp) SetInputs(ops []ops.Operator) { c.Outer, c.Inner = ops[0], ops[1] } -func (c *CorrelatedSubQueryOp) Description() ops.OpDescription { - return ops.OpDescription{ - OperatorType: "SubQuery", - Variant: "Correlated", - } -} - func (c *CorrelatedSubQueryOp) ShortDescription() string { return "" } diff --git a/go/vt/vtgate/planbuilder/operators/delete.go b/go/vt/vtgate/planbuilder/operators/delete.go index cd4f30f6d01..af1db82ad80 100644 --- a/go/vt/vtgate/planbuilder/operators/delete.go +++ b/go/vt/vtgate/planbuilder/operators/delete.go @@ -62,12 +62,6 @@ func (d *Delete) GetOrdering() ([]ops.OrderBy, error) { return nil, nil } -func (d *Delete) Description() ops.OpDescription { - return ops.OpDescription{ - OperatorType: "Delete", - } -} - func (d *Delete) ShortDescription() string { return fmt.Sprintf("%s.%s %s", d.VTable.Keyspace.Name, d.VTable.Name.String(), sqlparser.String(d.AST.Where)) } diff --git a/go/vt/vtgate/planbuilder/operators/derived.go b/go/vt/vtgate/planbuilder/operators/derived.go index fd1d57cc63a..dc6bbd79952 100644 --- a/go/vt/vtgate/planbuilder/operators/derived.go +++ b/go/vt/vtgate/planbuilder/operators/derived.go @@ -257,12 +257,6 @@ func (d *Derived) setQP(qp *QueryProjection) { d.QP = qp } -func (d *Derived) Description() ops.OpDescription { - return ops.OpDescription{ - OperatorType: "Derived", - } -} - func (d *Derived) ShortDescription() string { return d.Alias } diff --git a/go/vt/vtgate/planbuilder/operators/distinct.go b/go/vt/vtgate/planbuilder/operators/distinct.go index 9d1eea8182e..c120bef8230 100644 --- a/go/vt/vtgate/planbuilder/operators/distinct.go +++ b/go/vt/vtgate/planbuilder/operators/distinct.go @@ -112,12 +112,6 @@ func (d *Distinct) GetColumns() ([]*sqlparser.AliasedExpr, error) { return d.Source.GetColumns() } -func (d *Distinct) Description() ops.OpDescription { - return ops.OpDescription{ - OperatorType: "Distinct", - } -} - func (d *Distinct) ShortDescription() string { return "" } diff --git a/go/vt/vtgate/planbuilder/operators/filter.go b/go/vt/vtgate/planbuilder/operators/filter.go index f2f07b3ee19..21c6986d4bf 100644 --- a/go/vt/vtgate/planbuilder/operators/filter.go +++ b/go/vt/vtgate/planbuilder/operators/filter.go @@ -17,7 +17,11 @@ limitations under the License. package operators import ( + "strings" + + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/evalengine" "vitess.io/vitess/go/vt/vtgate/planbuilder/operators/ops" "vitess.io/vitess/go/vt/vtgate/planbuilder/operators/rewrite" @@ -114,22 +118,21 @@ func (f *Filter) Compact(*plancontext.PlanningContext) (ops.Operator, *rewrite.A } func (f *Filter) planOffsets(ctx *plancontext.PlanningContext) error { - resolveColumn := func(col *sqlparser.ColName) (int, error) { - newSrc, offset, err := f.Source.AddColumn(ctx, aeWrap(col), true, false) - if err != nil { - return 0, err - } - f.Source = newSrc - return offset, nil - } cfg := &evalengine.Config{ - ResolveType: ctx.SemTable.TypeForExpr, - Collation: ctx.SemTable.Collation, - ResolveColumn: resolveColumn, + ResolveType: ctx.SemTable.TypeForExpr, + Collation: ctx.SemTable.Collation, } - eexpr, err := evalengine.Translate(sqlparser.AndExpressions(f.Predicates...), cfg) + predicate := sqlparser.AndExpressions(f.Predicates...) + rewritten, err := useOffsets(ctx, predicate, f) + if err != nil { + return err + } + eexpr, err := evalengine.Translate(rewritten, cfg) if err != nil { + if strings.HasPrefix(err.Error(), evalengine.ErrTranslateExprNotSupported) { + return vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "%s: %s", evalengine.ErrTranslateExprNotSupported, sqlparser.String(predicate)) + } return err } @@ -137,15 +140,6 @@ func (f *Filter) planOffsets(ctx *plancontext.PlanningContext) error { return nil } -func (f *Filter) Description() ops.OpDescription { - return ops.OpDescription{ - OperatorType: "Filter", - Other: map[string]any{ - "Predicate": sqlparser.String(sqlparser.AndExpressions(f.Predicates...)), - }, - } -} - func (f *Filter) ShortDescription() string { return sqlparser.String(sqlparser.AndExpressions(f.Predicates...)) } diff --git a/go/vt/vtgate/planbuilder/operators/horizon.go b/go/vt/vtgate/planbuilder/operators/horizon.go index 39efe2b9956..d4c30313114 100644 --- a/go/vt/vtgate/planbuilder/operators/horizon.go +++ b/go/vt/vtgate/planbuilder/operators/horizon.go @@ -109,12 +109,6 @@ func (h *Horizon) setQP(qp *QueryProjection) { h.QP = qp } -func (h *Horizon) Description() ops.OpDescription { - return ops.OpDescription{ - OperatorType: "Horizon", - } -} - func (h *Horizon) ShortDescription() string { return "" } diff --git a/go/vt/vtgate/planbuilder/operators/horizon_expanding.go b/go/vt/vtgate/planbuilder/operators/horizon_expanding.go new file mode 100644 index 00000000000..c93318825bf --- /dev/null +++ b/go/vt/vtgate/planbuilder/operators/horizon_expanding.go @@ -0,0 +1,195 @@ +/* +Copyright 2023 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 operators + +import ( + "fmt" + + "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" +) + +func expandHorizon(ctx *plancontext.PlanningContext, horizon horizonLike) (ops.Operator, *rewrite.ApplyResult, error) { + sel, isSel := horizon.selectStatement().(*sqlparser.Select) + if !isSel { + return nil, nil, errHorizonNotPlanned() + } + + if sel.Having != nil { + return nil, nil, errHorizonNotPlanned() + } + + op, err := createProjectionFromSelect(ctx, horizon) + if err != nil { + return nil, nil, err + } + + qp, err := horizon.getQP(ctx) + if err != nil { + return nil, nil, err + } + + if qp.NeedsDistinct() { + op = &Distinct{ + Source: op, + QP: qp, + } + } + + if len(qp.OrderExprs) > 0 { + op = &Ordering{ + Source: op, + Order: qp.OrderExprs, + } + } + + if sel.Limit != nil { + op = &Limit{ + Source: op, + AST: sel.Limit, + } + } + + return op, rewrite.NewTree("expand horizon into smaller components", op), nil +} + +func checkInvalid(aggregations []Aggr, horizon horizonLike) error { + for _, aggregation := range aggregations { + if aggregation.Distinct { + return errHorizonNotPlanned() + } + } + if _, isDerived := horizon.(*Derived); isDerived { + return errHorizonNotPlanned() + } + return nil +} + +func createProjectionFromSelect(ctx *plancontext.PlanningContext, horizon horizonLike) (out ops.Operator, err error) { + qp, err := horizon.getQP(ctx) + if err != nil { + return nil, err + } + + if !qp.NeedsAggregation() { + projX, err := createProjectionWithoutAggr(qp, horizon.src()) + if err != nil { + return nil, err + } + if derived, isDerived := horizon.(*Derived); isDerived { + id := derived.TableId + projX.TableID = &id + projX.Alias = derived.Alias + } + out = projX + + return out, nil + } + + err = checkAggregationSupported(horizon) + if err != nil { + return nil, err + } + + aggregations, err := qp.AggregationExpressions(ctx) + if err != nil { + return nil, err + } + + if err := checkInvalid(aggregations, horizon); err != nil { + return nil, err + } + + a := &Aggregator{ + Source: horizon.src(), + Original: true, + QP: qp, + Grouping: qp.GetGrouping(), + Aggregations: aggregations, + } + + if derived, isDerived := horizon.(*Derived); isDerived { + id := derived.TableId + a.TableID = &id + a.Alias = derived.Alias + } + +outer: + for colIdx, expr := range qp.SelectExprs { + ae, err := expr.GetAliasedExpr() + if err != nil { + return nil, err + } + addedToCol := false + for idx, groupBy := range a.Grouping { + if ctx.SemTable.EqualsExprWithDeps(groupBy.SimplifiedExpr, ae.Expr) { + if !addedToCol { + a.Columns = append(a.Columns, ae) + addedToCol = true + } + if groupBy.ColOffset < 0 { + a.Grouping[idx].ColOffset = colIdx + } + } + } + if addedToCol { + continue + } + for idx, aggr := range a.Aggregations { + if ctx.SemTable.EqualsExprWithDeps(aggr.Original.Expr, ae.Expr) && aggr.ColOffset < 0 { + a.Columns = append(a.Columns, ae) + a.Aggregations[idx].ColOffset = colIdx + continue outer + } + } + return nil, vterrors.VT13001(fmt.Sprintf("Could not find the %s in aggregation in the original query", sqlparser.String(ae))) + } + + return a, nil +} + +func createProjectionWithoutAggr(qp *QueryProjection, src ops.Operator) (*Projection, error) { + proj := &Projection{ + Source: src, + } + + for _, e := range qp.SelectExprs { + if _, isStar := e.Col.(*sqlparser.StarExpr); isStar { + return nil, errHorizonNotPlanned() + } + ae, err := e.GetAliasedExpr() + + if err != nil { + return nil, err + } + expr := ae.Expr + if sqlparser.ContainsAggregation(expr) { + aggr, ok := expr.(sqlparser.AggrFunc) + if !ok { + // need to add logic to extract aggregations and pushed them to the top level + return nil, errHorizonNotPlanned() + } + expr = aggr.GetArg() + if expr == nil { + expr = sqlparser.NewIntLiteral("1") + } + } + + proj.addUnexploredExpr(ae, expr) + } + return proj, nil +} diff --git a/go/vt/vtgate/planbuilder/operators/horizon_planning.go b/go/vt/vtgate/planbuilder/operators/horizon_planning.go index 75067f71d69..6547d343759 100644 --- a/go/vt/vtgate/planbuilder/operators/horizon_planning.go +++ b/go/vt/vtgate/planbuilder/operators/horizon_planning.go @@ -17,7 +17,6 @@ limitations under the License. package operators import ( - "fmt" "io" "vitess.io/vitess/go/slices2" @@ -91,30 +90,50 @@ func tryHorizonPlanning(ctx *plancontext.PlanningContext, root ops.Operator) (ou return } +// Phase defines the different planning phases to go through to produce an optimized plan for the input query. +type Phase struct { + Name string + // preOptimizeAction is the action to be taken before calling plan optimization operation. + preOptimizeAction func(ctx *plancontext.PlanningContext, op ops.Operator) (ops.Operator, error) +} + +// getPhases returns the phases the planner will go through. +// It's used to control so rewriters collaborate correctly +func getPhases() []Phase { + return []Phase{{ + // Initial optimization + Name: "initial horizon planning optimization phase", + }, { + // Adding Ordering Op - Any aggregation that is performed in the VTGate needs the input to be ordered + // Adding Group by - This is needed if the grouping is performed on a join with a join condition then + // aggregation happening at route needs a group by to ensure only matching rows returns + // the aggregations otherwise returns no result. + Name: "add ORDER BY to aggregations above the route and add GROUP BY to aggregations on the RHS of join", + preOptimizeAction: addOrderBysAndGroupBysForAggregations, + }} +} + // planHorizons 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) (ops.Operator, error) { - root, err := optimizeHorizonPlanning(ctx, root) - if err != nil { - return nil, err - } +func planHorizons(ctx *plancontext.PlanningContext, root ops.Operator) (op ops.Operator, err error) { + phases := getPhases() + op = root - // Adding Ordering Op - This is needed if there is no explicit ordering and aggregation is performed on top of route. - // Adding Group by - This is needed if the grouping is performed on a join with a join condition then - // aggregation happening at route needs a group by to ensure only matching rows returns - // the aggregations otherwise returns no result. - root, err = addOrderBysAndGroupBysForAggregations(ctx, root) - if err != nil { - return nil, err - } - - root, err = optimizeHorizonPlanning(ctx, root) - if err != nil { - return nil, err + for _, phase := range phases { + if phase.preOptimizeAction != nil { + op, err = phase.preOptimizeAction(ctx, op) + if err != nil { + return nil, err + } + } + op, err = optimizeHorizonPlanning(ctx, op) + if err != nil { + return nil, err + } } - return root, nil + return op, nil } func optimizeHorizonPlanning(ctx *plancontext.PlanningContext, root ops.Operator) (ops.Operator, error) { @@ -152,20 +171,25 @@ func optimizeHorizonPlanning(ctx *plancontext.PlanningContext, root ops.Operator } func tryPushingDownFilter(ctx *plancontext.PlanningContext, in *Filter) (ops.Operator, *rewrite.ApplyResult, error) { - proj, ok := in.Source.(*Projection) - if !ok { - // we can only push filter under a projection - return in, rewrite.SameTree, nil + switch src := in.Source.(type) { + case *Projection: + return pushFilterUnderProjection(ctx, in, src) + case *Route: + return rewrite.Swap(in, src, "push filter into Route") } - for _, p := range in.Predicates { + return in, rewrite.SameTree, nil +} + +func pushFilterUnderProjection(ctx *plancontext.PlanningContext, filter *Filter, projection *Projection) (ops.Operator, *rewrite.ApplyResult, error) { + for _, p := range filter.Predicates { cantPushDown := false _ = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) { if !fetchByOffset(node) { return true, nil } - if proj.needsEvaluation(ctx, node.(sqlparser.Expr)) { + if projection.needsEvaluation(ctx, node.(sqlparser.Expr)) { cantPushDown = true return false, io.EOF } @@ -174,13 +198,12 @@ func tryPushingDownFilter(ctx *plancontext.PlanningContext, in *Filter) (ops.Ope }, p) if cantPushDown { - return in, rewrite.SameTree, nil + return filter, rewrite.SameTree, nil } } + return rewrite.Swap(filter, projection, "push filter under projection") - return rewrite.Swap(in, proj, "push filter under projection") } - func tryPushingDownDistinct(in *Distinct) (ops.Operator, *rewrite.ApplyResult, error) { if in.Pushed { return in, rewrite.SameTree, nil @@ -214,56 +237,57 @@ func tryPushingDownDistinct(in *Distinct) (ops.Operator, *rewrite.ApplyResult, e return aggr, rewrite.NewTree("replace distinct with aggregator", in), nil } -// addOrderBysAndGroupBysForAggregations runs after we have run horizonPlanning until the op tree stops changing +// addOrderBysForAggregations runs after we have run horizonPlanning until the op tree stops changing // this means that we have pushed aggregations and other ops as far down as they'll go -// addOrderBysAndGroupBysForAggregations will find Aggregators that have not been pushed under routes and +// addOrderBysForAggregations will find Aggregators that have not been pushed under routes and // add the necessary Ordering operators for them func addOrderBysAndGroupBysForAggregations(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 *Aggregator: - if in.Pushed { - // first we update the incoming columns, so we know about any new columns that have been added - columns, err := in.Source.GetColumns() - if err != nil { - return nil, nil, err - } - in.Columns = columns - } - - requireOrdering, err := needsOrdering(in, ctx) - if err != nil { - return nil, nil, err - } - if !requireOrdering { - return in, rewrite.SameTree, nil - } - in.Source = &Ordering{ - Source: in.Source, - Order: slices2.Map(in.Grouping, func(from GroupBy) ops.OrderBy { - return from.AsOrderBy() - }), - } - return in, rewrite.NewTree("added ordering before aggregation", in), nil + switch op := in.(type) { case *ApplyJoin: - _ = rewrite.Visit(in.RHS, func(op ops.Operator) error { - aggr, isAggr := op.(*Aggregator) - if !isAggr { - return nil - } - if len(aggr.Grouping) == 0 { - gb := sqlparser.NewIntLiteral(".0") - aggr.Grouping = append(aggr.Grouping, NewGroupBy(gb, gb, aeWrap(gb))) - } - return nil - }) + return addLiteralGroupingToRHS(op) + case *Aggregator: + return addOrderingForAggregation(ctx, op) + default: + return in, rewrite.SameTree, nil } - return in, rewrite.SameTree, nil } return rewrite.TopDown(root, TableID, visitor, stopAtRoute) } +func addLiteralGroupingToRHS(in *ApplyJoin) (ops.Operator, *rewrite.ApplyResult, error) { + _ = rewrite.Visit(in.RHS, func(op ops.Operator) error { + aggr, isAggr := op.(*Aggregator) + if !isAggr { + return nil + } + if len(aggr.Grouping) == 0 { + gb := sqlparser.NewIntLiteral(".0") + aggr.Grouping = append(aggr.Grouping, NewGroupBy(gb, gb, aeWrap(gb))) + } + return nil + }) + return in, rewrite.NewTree("added grouping to the RHS", in), nil +} + +func addOrderingForAggregation(ctx *plancontext.PlanningContext, in *Aggregator) (ops.Operator, *rewrite.ApplyResult, error) { + requireOrdering, err := needsOrdering(in, ctx) + if err != nil { + return nil, nil, err + } + if !requireOrdering { + return in, rewrite.SameTree, nil + } + in.Source = &Ordering{ + Source: in.Source, + Order: slices2.Map(in.Grouping, func(from GroupBy) ops.OrderBy { + return from.AsOrderBy() + }), + } + return in, rewrite.NewTree("added ordering before aggregation", in), nil +} + func needsOrdering(in *Aggregator, ctx *plancontext.PlanningContext) (bool, error) { if len(in.Grouping) == 0 { return false, nil @@ -688,185 +712,12 @@ func pushOrExpandHorizon(ctx *plancontext.PlanningContext, in horizonLike) (ops. return expandHorizon(ctx, in) } -func expandHorizon(ctx *plancontext.PlanningContext, horizon horizonLike) (ops.Operator, *rewrite.ApplyResult, error) { - sel, isSel := horizon.selectStatement().(*sqlparser.Select) - if !isSel { - return nil, nil, errHorizonNotPlanned() - } - - if sel.Having != nil { - return nil, nil, errHorizonNotPlanned() - } - - op, err := createProjectionFromSelect(ctx, horizon) - if err != nil { - return nil, nil, err - } - - qp, err := horizon.getQP(ctx) - if err != nil { - return nil, nil, err - } - - if qp.NeedsDistinct() { - op = &Distinct{ - Source: op, - QP: qp, - } - } - - if len(qp.OrderExprs) > 0 { - op = &Ordering{ - Source: op, - Order: qp.OrderExprs, - } - } - - if sel.Limit != nil { - op = &Limit{ - Source: op, - AST: sel.Limit, - } - } - - return op, rewrite.NewTree("expand horizon into smaller components", op), nil -} - -func checkInvalid(aggregations []Aggr, horizon horizonLike) error { - for _, aggregation := range aggregations { - if aggregation.Distinct { - return errHorizonNotPlanned() - } - } - if _, isDerived := horizon.(*Derived); isDerived { - return errHorizonNotPlanned() - } - return nil -} - -func createProjectionFromSelect(ctx *plancontext.PlanningContext, horizon horizonLike) (out ops.Operator, err error) { - qp, err := horizon.getQP(ctx) - if err != nil { - return nil, err - } - - if !qp.NeedsAggregation() { - projX, err := createProjectionWithoutAggr(qp, horizon.src()) - if err != nil { - return nil, err - } - if derived, isDerived := horizon.(*Derived); isDerived { - id := derived.TableId - projX.TableID = &id - projX.Alias = derived.Alias - } - out = projX - - return out, nil - } - - err = checkAggregationSupported(horizon) - if err != nil { - return nil, err - } - - aggregations, err := qp.AggregationExpressions(ctx) - if err != nil { - return nil, err - } - - if err := checkInvalid(aggregations, horizon); err != nil { - return nil, err - } - - a := &Aggregator{ - Source: horizon.src(), - Original: true, - QP: qp, - Grouping: qp.GetGrouping(), - Aggregations: aggregations, - } - - if derived, isDerived := horizon.(*Derived); isDerived { - id := derived.TableId - a.TableID = &id - a.Alias = derived.Alias - } - -outer: - for colIdx, expr := range qp.SelectExprs { - ae, err := expr.GetAliasedExpr() - if err != nil { - return nil, err - } - addedToCol := false - for idx, groupBy := range a.Grouping { - if ctx.SemTable.EqualsExprWithDeps(groupBy.SimplifiedExpr, ae.Expr) { - if !addedToCol { - a.Columns = append(a.Columns, ae) - addedToCol = true - } - if groupBy.ColOffset < 0 { - a.Grouping[idx].ColOffset = colIdx - } - } - } - if addedToCol { - continue - } - for idx, aggr := range a.Aggregations { - if ctx.SemTable.EqualsExprWithDeps(aggr.Original.Expr, ae.Expr) && aggr.ColOffset < 0 { - a.Columns = append(a.Columns, ae) - a.Aggregations[idx].ColOffset = colIdx - continue outer - } - } - return nil, vterrors.VT13001(fmt.Sprintf("Could not find the %s in aggregation in the original query", sqlparser.String(ae))) - } - - return a, nil -} - -func createProjectionWithoutAggr(qp *QueryProjection, src ops.Operator) (*Projection, error) { - proj := &Projection{ - Source: src, - } - - for _, e := range qp.SelectExprs { - if _, isStar := e.Col.(*sqlparser.StarExpr); isStar { - return nil, errHorizonNotPlanned() - } - ae, err := e.GetAliasedExpr() - - if err != nil { - return nil, err - } - expr := ae.Expr - if sqlparser.ContainsAggregation(expr) { - aggr, ok := expr.(sqlparser.AggrFunc) - if !ok { - // need to add logic to extract aggregations and pushed them to the top level - return nil, errHorizonNotPlanned() - } - expr = aggr.GetArg() - if expr == nil { - expr = sqlparser.NewIntLiteral("1") - } - } - - proj.addUnexploredExpr(ae, expr) - } - return proj, nil -} - func aeWrap(e sqlparser.Expr) *sqlparser.AliasedExpr { return &sqlparser.AliasedExpr{Expr: e} } +// makeSureOutputIsCorrect uses the original Horizon to make sure that the output columns line up with what the user asked for func makeSureOutputIsCorrect(ctx *plancontext.PlanningContext, oldHorizon ops.Operator, output ops.Operator) (ops.Operator, error) { - // next we use the original Horizon to make sure that the output columns line up with what the user asked for - // in the future, we'll tidy up the results. for now, we are just failing these queries and going back to the - // old horizon planning instead cols, err := output.GetColumns() if err != nil { return nil, err @@ -888,11 +739,7 @@ func makeSureOutputIsCorrect(ctx *plancontext.PlanningContext, oldHorizon ops.Op if err != nil { return nil, err } - proj, err := createProjectionWithoutAggr(qp, output) - if err != nil { - return nil, err - } - err = proj.passThroughAllColumns(ctx) + proj, err := createSimpleProjection(ctx, qp, output) if err != nil { return nil, err } diff --git a/go/vt/vtgate/planbuilder/operators/insert.go b/go/vt/vtgate/planbuilder/operators/insert.go index 225d285b73b..3fc70ed8998 100644 --- a/go/vt/vtgate/planbuilder/operators/insert.go +++ b/go/vt/vtgate/planbuilder/operators/insert.go @@ -86,12 +86,6 @@ type Generate struct { added bool } -func (i *Insert) Description() ops.OpDescription { - return ops.OpDescription{ - OperatorType: "Insert", - } -} - func (i *Insert) ShortDescription() string { return i.VTable.String() } diff --git a/go/vt/vtgate/planbuilder/operators/join.go b/go/vt/vtgate/planbuilder/operators/join.go index 5470f3ac378..1998454a4e2 100644 --- a/go/vt/vtgate/planbuilder/operators/join.go +++ b/go/vt/vtgate/planbuilder/operators/join.go @@ -159,15 +159,6 @@ func (j *Join) AddJoinPredicate(ctx *plancontext.PlanningContext, expr sqlparser return nil } -func (j *Join) Description() ops.OpDescription { - return ops.OpDescription{ - OperatorType: "Join", - Other: map[string]any{ - "Predicate": sqlparser.String(j.Predicate), - }, - } -} - func (j *Join) ShortDescription() string { return sqlparser.String(j.Predicate) } diff --git a/go/vt/vtgate/planbuilder/operators/limit.go b/go/vt/vtgate/planbuilder/operators/limit.go index a2531b4bde5..35108965a52 100644 --- a/go/vt/vtgate/planbuilder/operators/limit.go +++ b/go/vt/vtgate/planbuilder/operators/limit.go @@ -73,20 +73,6 @@ func (l *Limit) GetOrdering() ([]ops.OrderBy, error) { return l.Source.GetOrdering() } -func (l *Limit) Description() ops.OpDescription { - other := map[string]any{} - if l.AST.Offset != nil { - other["Offset"] = sqlparser.String(l.AST.Offset) - } - if l.AST.Rowcount != nil { - other["RowCount"] = sqlparser.String(l.AST.Rowcount) - } - return ops.OpDescription{ - OperatorType: "Limit", - Other: other, - } -} - func (l *Limit) ShortDescription() string { return sqlparser.String(l.AST) } diff --git a/go/vt/vtgate/planbuilder/operators/offset_planning.go b/go/vt/vtgate/planbuilder/operators/offset_planning.go index f4287f281ed..0fb2fbf091d 100644 --- a/go/vt/vtgate/planbuilder/operators/offset_planning.go +++ b/go/vt/vtgate/planbuilder/operators/offset_planning.go @@ -59,23 +59,6 @@ func planOffsets(ctx *plancontext.PlanningContext, root ops.Operator) (ops.Opera return op, nil } -func (p *Projection) passThroughAllColumns(ctx *plancontext.PlanningContext) error { - - for i, col := range p.Projections { - newSrc, offset, err := p.Source.AddColumn(ctx, aeWrap(col.GetExpr()), true, false) - if err != nil { - return err - } - p.Source = newSrc - p.Projections[i] = Offset{ - Expr: col.GetExpr(), - Offset: offset, - } - } - - return nil -} - func fetchByOffset(e sqlparser.SQLNode) bool { switch e.(type) { case *sqlparser.ColName, sqlparser.AggrFunc: diff --git a/go/vt/vtgate/planbuilder/operators/ops/op.go b/go/vt/vtgate/planbuilder/operators/ops/op.go index d68daed439e..57e27879861 100644 --- a/go/vt/vtgate/planbuilder/operators/ops/op.go +++ b/go/vt/vtgate/planbuilder/operators/ops/op.go @@ -49,7 +49,6 @@ type ( GetColumns() ([]*sqlparser.AliasedExpr, error) - Description() OpDescription ShortDescription() string GetOrdering() ([]OrderBy, error) @@ -62,13 +61,4 @@ type ( // See GroupBy#SimplifiedExpr for more details about this SimplifiedExpr sqlparser.Expr } - - OpDescription struct { - OperatorType string - Variant string `json:",omitempty"` - Other map[string]any `json:",omitempty"` - - // This field will be filled in by the JSON producer. No need to set it manually - Inputs []OpDescription `json:",omitempty"` - } ) diff --git a/go/vt/vtgate/planbuilder/operators/ops/to_json.go b/go/vt/vtgate/planbuilder/operators/ops/to_json.go index 2d22e27d8cc..2b8b747f433 100644 --- a/go/vt/vtgate/planbuilder/operators/ops/to_json.go +++ b/go/vt/vtgate/planbuilder/operators/ops/to_json.go @@ -17,31 +17,13 @@ limitations under the License. package ops import ( - "encoding/json" "fmt" "reflect" "github.com/xlab/treeprint" ) -// ToJSON is a debug only function. It can panic, so do not use this in production code -func ToJSON(op Operator) string { - descr := buildDescriptionTree(op) - out, err := json.MarshalIndent(descr, "", " ") - if err != nil { - panic(err) - } - return string(out) -} - -func buildDescriptionTree(op Operator) OpDescription { - descr := op.Description() - for _, in := range op.Inputs() { - descr.Inputs = append(descr.Inputs, buildDescriptionTree(in)) - } - return descr -} - +// ToTree returns the operator as ascii tree. Should only be used for debugging func ToTree(op Operator) string { tree := asTree(op, nil) return tree.String() diff --git a/go/vt/vtgate/planbuilder/operators/ordering.go b/go/vt/vtgate/planbuilder/operators/ordering.go index a0c42fc8ecb..360bf87cb23 100644 --- a/go/vt/vtgate/planbuilder/operators/ordering.go +++ b/go/vt/vtgate/planbuilder/operators/ordering.go @@ -38,10 +38,11 @@ type Ordering struct { func (o *Ordering) Clone(inputs []ops.Operator) ops.Operator { return &Ordering{ - Source: inputs[0], - Offset: slices.Clone(o.Offset), - WOffset: slices.Clone(o.WOffset), - Order: slices.Clone(o.Order), + Source: inputs[0], + Offset: slices.Clone(o.Offset), + WOffset: slices.Clone(o.WOffset), + Order: slices.Clone(o.Order), + ResultColumns: o.ResultColumns, } } @@ -105,13 +106,6 @@ func (o *Ordering) planOffsets(ctx *plancontext.PlanningContext) error { return nil } -func (o *Ordering) Description() ops.OpDescription { - return ops.OpDescription{ - OperatorType: "Ordering", - Other: map[string]any{}, - } -} - func (o *Ordering) ShortDescription() string { ordering := slices2.Map(o.Order, func(o ops.OrderBy) string { return sqlparser.String(o.Inner) diff --git a/go/vt/vtgate/planbuilder/operators/projection.go b/go/vt/vtgate/planbuilder/operators/projection.go index 1ed2e1cd681..79e3d837ff6 100644 --- a/go/vt/vtgate/planbuilder/operators/projection.go +++ b/go/vt/vtgate/planbuilder/operators/projection.go @@ -78,6 +78,33 @@ type ( var _ selectExpressions = (*Projection)(nil) +// createSimpleProjection returns a projection where all columns are offsets. +// used to change the name and order of the columns in the final output +func createSimpleProjection(ctx *plancontext.PlanningContext, qp *QueryProjection, src ops.Operator) (*Projection, error) { + p := &Projection{ + Source: src, + } + + for _, e := range qp.SelectExprs { + if _, isStar := e.Col.(*sqlparser.StarExpr); isStar { + return nil, errHorizonNotPlanned() + } + ae, err := e.GetAliasedExpr() + if err != nil { + return nil, err + } + newSrc, offset, err := p.Source.AddColumn(ctx, ae, true, false) + if err != nil { + return nil, err + } + + p.Source = newSrc + p.Projections = append(p.Projections, Offset{Expr: ae.Expr, Offset: offset}) + p.Columns = append(p.Columns, ae) + } + return p, nil +} + func (p *Projection) addUnexploredExpr(ae *sqlparser.AliasedExpr, e sqlparser.Expr) int { p.Projections = append(p.Projections, UnexploredExpression{E: e}) p.Columns = append(p.Columns, ae) @@ -164,30 +191,6 @@ func (p *Projection) AllOffsets() (cols []int) { return } -func (p *Projection) Description() ops.OpDescription { - var columns []string - for i, col := range p.Projections { - aliasExpr := p.Columns[i] - if aliasExpr.Expr == col.GetExpr() { - columns = append(columns, sqlparser.String(aliasExpr)) - } else { - columns = append(columns, fmt.Sprintf("%s AS %s", sqlparser.String(col.GetExpr()), aliasExpr.ColumnName())) - } - } - - other := map[string]any{ - "OutputColumns": strings.Join(columns, ", "), - } - if p.TableID != nil { - other["Derived"] = true - other["Alias"] = p.Alias - } - return ops.OpDescription{ - OperatorType: "Projection", - Other: other, - } -} - func (p *Projection) ShortDescription() string { var columns []string if p.Alias != "" { @@ -293,12 +296,8 @@ func (p *Projection) planOffsets(ctx *plancontext.PlanningContext) error { } // first step is to replace the expressions we expect to get from our input with the offsets for these - visitor, errCheck := offsetter(ctx, - func() ops.Operator { return p.Source }, - func(o ops.Operator) { p.Source = o }, - ) - rewritten := sqlparser.CopyOnRewrite(col.GetExpr(), stopAtAggregations, visitor, nil).(sqlparser.Expr) - if err := errCheck(); err != nil { + rewritten, err := useOffsets(ctx, col.GetExpr(), p) + if err != nil { return err } @@ -327,23 +326,62 @@ func (p *Projection) planOffsets(ctx *plancontext.PlanningContext) error { return nil } -func offsetter(ctx *plancontext.PlanningContext, src func() ops.Operator, setSource func(ops.Operator)) (func(cursor *sqlparser.CopyOnWriteCursor), func() error) { - var err error - return func(cursor *sqlparser.CopyOnWriteCursor) { - expr, ok := cursor.Node().(sqlparser.Expr) - if !ok || !fetchByOffset(expr) { - return - } +func useOffsets(ctx *plancontext.PlanningContext, expr sqlparser.Expr, op ops.Operator) (sqlparser.Expr, error) { + in := op.Inputs()[0] + columns, err := in.GetColumns() + if err != nil { + return nil, err + } - newSrc, offset, terr := src().AddColumn(ctx, aeWrap(expr), true, false) - if terr != nil { - err = terr - return + var exprOffset *sqlparser.Offset + down := func(node, parent sqlparser.SQLNode) bool { + if err != nil { + return false + } + e, ok := node.(sqlparser.Expr) + if !ok { + return true + } + offset := slices.IndexFunc(columns, func(expr *sqlparser.AliasedExpr) bool { + return ctx.SemTable.EqualsExprWithDeps(expr.Expr, e) + }) + if offset >= 0 { + // this expression can be fetched from the input - we can stop here + exprOffset = sqlparser.NewOffset(offset, e) + return false + } + + if fetchByOffset(e) { + // this expression has to be fetched from the input, but we didn't find it in the input. let's add it + _, addToGroupBy := e.(*sqlparser.ColName) + in, offset, err = in.AddColumn(ctx, aeWrap(e), true, addToGroupBy) + if err != nil { + return false + } + op.SetInputs([]ops.Operator{in}) + columns, err = in.GetColumns() + if err != nil { + return false } - setSource(newSrc) - cursor.Replace(sqlparser.NewOffset(offset, expr)) + exprOffset = sqlparser.NewOffset(offset, e) + return false + } - }, func() error { - return err + return true + } + + // The cursor replace is not available while walking `down`, so `up` is used to do the replacement. + up := func(cursor *sqlparser.CopyOnWriteCursor) { + if exprOffset != nil { + cursor.Replace(exprOffset) + exprOffset = nil } + } + + rewritten := sqlparser.CopyOnRewrite(expr, down, up, ctx.SemTable.CopyDependenciesOnSQLNodes) + if err != nil { + return nil, err + } + + return rewritten.(sqlparser.Expr), nil } diff --git a/go/vt/vtgate/planbuilder/operators/querygraph.go b/go/vt/vtgate/planbuilder/operators/querygraph.go index a764ca3db89..f9b05914e77 100644 --- a/go/vt/vtgate/planbuilder/operators/querygraph.go +++ b/go/vt/vtgate/planbuilder/operators/querygraph.go @@ -222,13 +222,6 @@ func (qt *QueryTable) Clone() *QueryTable { } } -func (qg *QueryGraph) Description() ops.OpDescription { - return ops.OpDescription{ - OperatorType: "QueryGraph", - Other: map[string]any{"Tables": qg.tableNames()}, - } -} - func (qg *QueryGraph) tableNames() (tables []string) { for _, table := range qg.Tables { tables = append(tables, sqlparser.String(table.Table)) diff --git a/go/vt/vtgate/planbuilder/operators/queryprojection.go b/go/vt/vtgate/planbuilder/operators/queryprojection.go index e3c6981ff99..99bfbe9b3c2 100644 --- a/go/vt/vtgate/planbuilder/operators/queryprojection.go +++ b/go/vt/vtgate/planbuilder/operators/queryprojection.go @@ -220,7 +220,9 @@ func CreateQPFromSelect(ctx *plancontext.PlanningContext, sel *sqlparser.Select) if err := qp.addOrderBy(ctx, sel.OrderBy); err != nil { return nil, err } - + if !qp.HasAggr && sel.Having != nil { + qp.HasAggr = sqlparser.ContainsAggregation(sel.Having.Expr) + } qp.calculateDistinct(ctx) return qp, nil @@ -665,31 +667,36 @@ orderBy: return nil, vterrors.VT12001("in scatter query: complex aggregate expression") } - code := opcode.SupportedAggregates[strings.ToLower(fnc.AggrName())] + aggr := createAggrFromAggrFunc(fnc, aliasedExpr) + aggr.Index = &idxCopy + out = append(out, aggr) + } + return +} - if code == opcode.AggregateCount { - if _, isStar := fnc.(*sqlparser.CountStar); isStar { - code = opcode.AggregateCountStar - } +func createAggrFromAggrFunc(fnc sqlparser.AggrFunc, aliasedExpr *sqlparser.AliasedExpr) Aggr { + code := opcode.SupportedAggregates[strings.ToLower(fnc.AggrName())] + + if code == opcode.AggregateCount { + if _, isStar := fnc.(*sqlparser.CountStar); isStar { + code = opcode.AggregateCountStar } + } - aggrF, _ := aliasedExpr.Expr.(sqlparser.AggrFunc) + aggrF, _ := aliasedExpr.Expr.(sqlparser.AggrFunc) - if aggrF.IsDistinct() { - switch code { - case opcode.AggregateCount: - code = opcode.AggregateCountDistinct - case opcode.AggregateSum: - code = opcode.AggregateSumDistinct - } + if aggrF.IsDistinct() { + switch code { + case opcode.AggregateCount: + code = opcode.AggregateCountDistinct + case opcode.AggregateSum: + code = opcode.AggregateSumDistinct } - - aggr := NewAggr(code, aggrF, aliasedExpr, aliasedExpr.ColumnName()) - aggr.Index = &idxCopy - aggr.Distinct = aggrF.IsDistinct() - out = append(out, aggr) } - return + + aggr := NewAggr(code, aggrF, aliasedExpr, aliasedExpr.ColumnName()) + aggr.Distinct = aggrF.IsDistinct() + return aggr } // FindSelectExprIndexForExpr returns the index of the given expression in the select expressions, if it is part of it diff --git a/go/vt/vtgate/planbuilder/operators/route.go b/go/vt/vtgate/planbuilder/operators/route.go index 785371315cc..31edd45fb78 100644 --- a/go/vt/vtgate/planbuilder/operators/route.go +++ b/go/vt/vtgate/planbuilder/operators/route.go @@ -579,6 +579,7 @@ func (r *Route) AddColumn(ctx *plancontext.PlanningContext, expr *sqlparser.Alia } type selectExpressions interface { + ops.Operator addColumnWithoutPushing(expr *sqlparser.AliasedExpr, addToGroupBy bool) int isDerived() bool } @@ -699,16 +700,6 @@ func (r *Route) getOffsetFor(ctx *plancontext.PlanningContext, order ops.OrderBy return offset, nil } -func (r *Route) Description() ops.OpDescription { - return ops.OpDescription{ - OperatorType: "Route", - Other: map[string]any{ - "OpCode": r.Routing.OpCode(), - "Keyspace": r.Routing.Keyspace(), - }, - } -} - func (r *Route) ShortDescription() string { first := r.Routing.OpCode().String() diff --git a/go/vt/vtgate/planbuilder/operators/route_planning.go b/go/vt/vtgate/planbuilder/operators/route_planning.go index 1c052ec3691..f38dcbab3a8 100644 --- a/go/vt/vtgate/planbuilder/operators/route_planning.go +++ b/go/vt/vtgate/planbuilder/operators/route_planning.go @@ -71,6 +71,7 @@ func transformToPhysical(ctx *plancontext.PlanningContext, in ops.Operator) (ops } func pushDownFilter(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") } diff --git a/go/vt/vtgate/planbuilder/operators/subquery.go b/go/vt/vtgate/planbuilder/operators/subquery.go index c004d1a9510..632042bf20e 100644 --- a/go/vt/vtgate/planbuilder/operators/subquery.go +++ b/go/vt/vtgate/planbuilder/operators/subquery.go @@ -127,18 +127,6 @@ func createSubqueryFromStatement(ctx *plancontext.PlanningContext, stmt sqlparse return subq, nil } -func (s *SubQuery) Description() ops.OpDescription { - return ops.OpDescription{ - OperatorType: "SubQuery", - } -} - -func (s *SubQueryInner) Description() ops.OpDescription { - return ops.OpDescription{ - OperatorType: "SubQueryInner", - } -} - func (s *SubQuery) ShortDescription() string { return "" } diff --git a/go/vt/vtgate/planbuilder/operators/table.go b/go/vt/vtgate/planbuilder/operators/table.go index 92735a055cd..a3eeea5e365 100644 --- a/go/vt/vtgate/planbuilder/operators/table.go +++ b/go/vt/vtgate/planbuilder/operators/table.go @@ -113,17 +113,6 @@ func addColumn(ctx *plancontext.PlanningContext, op ColNameColumns, e sqlparser. return offset, nil } -func (to *Table) Description() ops.OpDescription { - var columns []string - for _, col := range to.Columns { - columns = append(columns, sqlparser.String(col)) - } - return ops.OpDescription{ - OperatorType: "Table", - Other: map[string]any{"Columns": columns}, - } -} - func (to *Table) ShortDescription() string { return to.VTable.String() } diff --git a/go/vt/vtgate/planbuilder/operators/union.go b/go/vt/vtgate/planbuilder/operators/union.go index 4fff00ef819..7a68007ff63 100644 --- a/go/vt/vtgate/planbuilder/operators/union.go +++ b/go/vt/vtgate/planbuilder/operators/union.go @@ -190,12 +190,6 @@ func (u *Union) Compact(*plancontext.PlanningContext) (ops.Operator, *rewrite.Ap func (u *Union) NoLHSTableSet() {} -func (u *Union) Description() ops.OpDescription { - return ops.OpDescription{ - OperatorType: "Union", - } -} - func (u *Union) ShortDescription() string { return "" } diff --git a/go/vt/vtgate/planbuilder/operators/update.go b/go/vt/vtgate/planbuilder/operators/update.go index f9c831860f1..91273062c17 100644 --- a/go/vt/vtgate/planbuilder/operators/update.go +++ b/go/vt/vtgate/planbuilder/operators/update.go @@ -65,12 +65,6 @@ func (u *Update) TablesUsed() []string { return nil } -func (u *Update) Description() ops.OpDescription { - return ops.OpDescription{ - OperatorType: "Update", - } -} - func (u *Update) ShortDescription() string { return u.VTable.String() } diff --git a/go/vt/vtgate/planbuilder/operators/vindex.go b/go/vt/vtgate/planbuilder/operators/vindex.go index 79104fc7364..252cb7ba2b3 100644 --- a/go/vt/vtgate/planbuilder/operators/vindex.go +++ b/go/vt/vtgate/planbuilder/operators/vindex.go @@ -154,10 +154,6 @@ func (v *Vindex) TablesUsed() []string { return []string{v.Table.Table.Name.String()} } -func (v *Vindex) Description() ops.OpDescription { - return ops.OpDescription{OperatorType: "Vindex"} -} - func (v *Vindex) ShortDescription() string { return v.Vindex.String() }