From f3ffc306eb4e5ebea53e060a8cd39d8276c1216a Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Tue, 15 Oct 2024 17:29:28 +0200 Subject: [PATCH 1/5] bugfix: backport of #14974 Signed-off-by: Andres Taylor --- .../vtgate/queries/subquery/subquery_test.go | 13 ++ .../planbuilder/operators/SQL_builder.go | 17 +- .../planbuilder/operators/aggregator.go | 8 + .../planbuilder/operators/apply_join.go | 50 ++++-- .../planbuilder/operators/expressions.go | 1 + .../planbuilder/operators/join_merging.go | 10 +- .../planbuilder/operators/offset_planning.go | 6 + .../planbuilder/operators/operator_funcs.go | 103 ----------- .../planbuilder/operators/projection.go | 12 +- .../planbuilder/operators/query_planning.go | 168 +++++++++++++----- go/vt/vtgate/planbuilder/operators/route.go | 62 ++++--- .../planbuilder/operators/route_planning.go | 10 +- .../planbuilder/testdata/aggr_cases.json | 86 +++++---- .../planbuilder/testdata/from_cases.json | 18 +- .../testdata/large_union_cases.json | 8 +- .../testdata/postprocess_cases.json | 2 +- .../planbuilder/testdata/reference_cases.json | 14 +- .../planbuilder/testdata/select_cases.json | 8 +- .../planbuilder/testdata/tpcc_cases.json | 4 +- .../planbuilder/testdata/tpch_cases.json | 52 +++--- .../planbuilder/testdata/union_cases.json | 64 +++---- 21 files changed, 401 insertions(+), 315 deletions(-) delete mode 100644 go/vt/vtgate/planbuilder/operators/operator_funcs.go diff --git a/go/test/endtoend/vtgate/queries/subquery/subquery_test.go b/go/test/endtoend/vtgate/queries/subquery/subquery_test.go index 0e21fa43cc1..a03948f8f40 100644 --- a/go/test/endtoend/vtgate/queries/subquery/subquery_test.go +++ b/go/test/endtoend/vtgate/queries/subquery/subquery_test.go @@ -173,3 +173,16 @@ func TestSubqueryInAggregation(t *testing.T) { // This fails as the planner adds `weight_string` method which make the query fail on MySQL. // mcmp.Exec(`SELECT max((select min(id2) from t1 where t1.id1 = t.id1)) FROM t1 t`) } + +// TestSubqueryInDerivedTable tests that subqueries and derived tables +// are handled correctly when there are joins inside the derived table +func TestSubqueryInDerivedTable(t *testing.T) { + utils.SkipIfBinaryIsBelowVersion(t, 18, "vtgate") + mcmp, closer := start(t) + defer closer() + + mcmp.Exec("INSERT INTO t1 (id1, id2) VALUES (1, 100), (2, 200), (3, 300), (4, 400), (5, 500);") + mcmp.Exec("INSERT INTO t2 (id3, id4) VALUES (10, 1), (20, 2), (30, 3), (40, 4), (50, 99)") + mcmp.Exec(`select t.a from (select t1.id2, t2.id3, (select id2 from t1 order by id2 limit 1) as a from t1 join t2 on t1.id1 = t2.id4) t`) + mcmp.Exec(`SELECT COUNT(*) FROM (SELECT DISTINCT t1.id1 FROM t1 JOIN t2 ON t1.id1 = t2.id4) dt`) +} diff --git a/go/vt/vtgate/planbuilder/operators/SQL_builder.go b/go/vt/vtgate/planbuilder/operators/SQL_builder.go index 9a55ddef1de..cd56fed05b2 100644 --- a/go/vt/vtgate/planbuilder/operators/SQL_builder.go +++ b/go/vt/vtgate/planbuilder/operators/SQL_builder.go @@ -21,6 +21,8 @@ import ( "slices" "sort" + "vitess.io/vitess/go/slice" + "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/planbuilder/operators/ops" @@ -562,6 +564,13 @@ func buildProjection(op *Projection, qb *queryBuilder) error { } func buildApplyJoin(op *ApplyJoin, qb *queryBuilder) error { + predicates := slice.Map(op.JoinPredicates, func(jc JoinColumn) sqlparser.Expr { + // since we are adding these join predicates, we need to mark to broken up version (RHSExpr) of it as done + qb.ctx.SkipPredicates[jc.RHSExpr] = nil + + return jc.Original.Expr + }) + pred := sqlparser.AndExpressions(predicates...) err := buildQuery(op.LHS, qb) if err != nil { return err @@ -569,8 +578,8 @@ func buildApplyJoin(op *ApplyJoin, qb *queryBuilder) error { // If we are going to add the predicate used in join here // We should not add the predicate's copy of when it was split into // two parts. To avoid this, we use the SkipPredicates map. - for _, expr := range qb.ctx.JoinPredicates[op.Predicate] { - qb.ctx.SkipPredicates[expr] = nil + for _, pred := range op.JoinPredicates { + qb.ctx.SkipPredicates[pred.RHSExpr] = nil } qbR := &queryBuilder{ctx: qb.ctx} err = buildQuery(op.RHS, qbR) @@ -578,9 +587,9 @@ func buildApplyJoin(op *ApplyJoin, qb *queryBuilder) error { return err } if op.LeftJoin { - qb.joinOuterWith(qbR, op.Predicate) + qb.joinOuterWith(qbR, pred) } else { - qb.joinInnerWith(qbR, op.Predicate) + qb.joinInnerWith(qbR, pred) } return nil } diff --git a/go/vt/vtgate/planbuilder/operators/aggregator.go b/go/vt/vtgate/planbuilder/operators/aggregator.go index 9e38048d957..2073dcd6119 100644 --- a/go/vt/vtgate/planbuilder/operators/aggregator.go +++ b/go/vt/vtgate/planbuilder/operators/aggregator.go @@ -124,6 +124,14 @@ func (a *Aggregator) isDerived() bool { return a.DT != nil } +func (a *Aggregator) derivedName() string { + if a.DT == nil { + return "" + } + + return a.DT.Alias +} + func (a *Aggregator) FindCol(ctx *plancontext.PlanningContext, in sqlparser.Expr, underRoute bool) (int, error) { if underRoute && a.isDerived() { // We don't want to use columns on this operator if it's a derived table under a route. diff --git a/go/vt/vtgate/planbuilder/operators/apply_join.go b/go/vt/vtgate/planbuilder/operators/apply_join.go index 5e48fb4d5e3..279357d98ed 100644 --- a/go/vt/vtgate/planbuilder/operators/apply_join.go +++ b/go/vt/vtgate/planbuilder/operators/apply_join.go @@ -38,9 +38,6 @@ type ( // LeftJoin will be true in the case of an outer join LeftJoin bool - // Before offset planning - Predicate sqlparser.Expr - // JoinColumns keeps track of what AST expression is represented in the Columns array JoinColumns []JoinColumn @@ -86,14 +83,19 @@ type ( } ) -func NewApplyJoin(lhs, rhs ops.Operator, predicate sqlparser.Expr, leftOuterJoin bool) *ApplyJoin { - return &ApplyJoin{ - LHS: lhs, - RHS: rhs, - Vars: map[string]int{}, - Predicate: predicate, - LeftJoin: leftOuterJoin, +func NewApplyJoin(ctx *plancontext.PlanningContext, lhs, rhs ops.Operator, predicate sqlparser.Expr, leftOuterJoin bool) (*ApplyJoin, error) { + aj := &ApplyJoin{ + LHS: lhs, + RHS: rhs, + Vars: map[string]int{}, + LeftJoin: leftOuterJoin, + } + err := aj.AddJoinPredicate(ctx, predicate) + if err != nil { + return nil, err } + + return aj, nil } // Clone implements the Operator interface @@ -105,7 +107,6 @@ func (aj *ApplyJoin) Clone(inputs []ops.Operator) ops.Operator { kopy.JoinColumns = slices.Clone(aj.JoinColumns) kopy.JoinPredicates = slices.Clone(aj.JoinPredicates) kopy.Vars = maps.Clone(aj.Vars) - kopy.Predicate = sqlparser.CloneExpr(aj.Predicate) kopy.ExtraLHSVars = slices.Clone(aj.ExtraLHSVars) return &kopy } @@ -149,8 +150,9 @@ func (aj *ApplyJoin) IsInner() bool { } func (aj *ApplyJoin) AddJoinPredicate(ctx *plancontext.PlanningContext, expr sqlparser.Expr) error { - aj.Predicate = ctx.SemTable.AndExpressions(expr, aj.Predicate) - + if expr == nil { + return nil + } col, err := BreakExpressionInLHSandRHS(ctx, expr, TableID(aj.LHS)) if err != nil { return err @@ -312,11 +314,15 @@ func (aj *ApplyJoin) addOffset(offset int) { } func (aj *ApplyJoin) ShortDescription() string { - pred := sqlparser.String(aj.Predicate) - columns := slice.Map(aj.JoinColumns, func(from JoinColumn) string { - return sqlparser.String(from.Original) - }) - firstPart := fmt.Sprintf("on %s columns: %s", pred, strings.Join(columns, ", ")) + fn := func(cols []JoinColumn) string { + out := slice.Map(cols, func(jc JoinColumn) string { + return jc.String() + }) + return strings.Join(out, ", ") + } + + firstPart := fmt.Sprintf("on %s columns: %s", fn(aj.JoinPredicates), fn(aj.JoinColumns)) + if len(aj.ExtraLHSVars) == 0 { return firstPart } @@ -419,6 +425,14 @@ func (jc JoinColumn) IsMixedLeftAndRight() bool { return len(jc.LHSExprs) > 0 && jc.RHSExpr != nil } +func (jc JoinColumn) String() string { + rhs := sqlparser.String(jc.RHSExpr) + lhs := slice.Map(jc.LHSExprs, func(e BindVarExpr) string { + return sqlparser.String(e.Expr) + }) + return fmt.Sprintf("[%s | %s | %s]", strings.Join(lhs, ", "), rhs, sqlparser.String(jc.Original)) +} + func (bve BindVarExpr) String() string { if bve.Name == "" { return sqlparser.String(bve.Expr) diff --git a/go/vt/vtgate/planbuilder/operators/expressions.go b/go/vt/vtgate/planbuilder/operators/expressions.go index 7ab27e787e8..4c03490317e 100644 --- a/go/vt/vtgate/planbuilder/operators/expressions.go +++ b/go/vt/vtgate/planbuilder/operators/expressions.go @@ -29,6 +29,7 @@ func BreakExpressionInLHSandRHS( expr sqlparser.Expr, lhs semantics.TableSet, ) (col JoinColumn, err error) { + col.Original = aeWrap(expr) rewrittenExpr := sqlparser.CopyOnRewrite(expr, nil, func(cursor *sqlparser.CopyOnWriteCursor) { nodeExpr, ok := cursor.Node().(sqlparser.Expr) if !ok || !fetchByOffset(nodeExpr) { diff --git a/go/vt/vtgate/planbuilder/operators/join_merging.go b/go/vt/vtgate/planbuilder/operators/join_merging.go index c43b7b4c87e..a8f401211e6 100644 --- a/go/vt/vtgate/planbuilder/operators/join_merging.go +++ b/go/vt/vtgate/planbuilder/operators/join_merging.go @@ -235,13 +235,17 @@ func mergeShardedRouting(r1 *ShardedRouting, r2 *ShardedRouting) *ShardedRouting return tr } -func (jm *joinMerger) getApplyJoin(ctx *plancontext.PlanningContext, op1, op2 *Route) *ApplyJoin { - return NewApplyJoin(op1.Source, op2.Source, ctx.SemTable.AndExpressions(jm.predicates...), !jm.innerJoin) +func (jm *joinMerger) getApplyJoin(ctx *plancontext.PlanningContext, op1, op2 *Route) (*ApplyJoin, error) { + return NewApplyJoin(ctx, op1.Source, op2.Source, ctx.SemTable.AndExpressions(jm.predicates...), !jm.innerJoin) } func (jm *joinMerger) merge(ctx *plancontext.PlanningContext, op1, op2 *Route, r Routing) (*Route, error) { + join, err := jm.getApplyJoin(ctx, op1, op2) + if err != nil { + return nil, err + } return &Route{ - Source: jm.getApplyJoin(ctx, op1, op2), + Source: join, MergedWith: []*Route{op2}, Routing: r, }, nil diff --git a/go/vt/vtgate/planbuilder/operators/offset_planning.go b/go/vt/vtgate/planbuilder/operators/offset_planning.go index 502df5e9c82..db9bd8d3631 100644 --- a/go/vt/vtgate/planbuilder/operators/offset_planning.go +++ b/go/vt/vtgate/planbuilder/operators/offset_planning.go @@ -30,6 +30,7 @@ import ( // planOffsets will walk the tree top down, adding offset information to columns in the tree for use in further optimization, func planOffsets(ctx *plancontext.PlanningContext, root ops.Operator) (ops.Operator, error) { type offsettable interface { + ops.Operator planOffsets(ctx *plancontext.PlanningContext) error } @@ -40,6 +41,11 @@ func planOffsets(ctx *plancontext.PlanningContext, root ops.Operator) (ops.Opera return nil, nil, vterrors.VT13001(fmt.Sprintf("should not see %T here", in)) case offsettable: err = op.planOffsets(ctx) + if rewrite.DebugOperatorTree { + fmt.Println("Planned offsets for:") + fmt.Println(ops.ToTree(op)) + } + } if err != nil { return nil, nil, err diff --git a/go/vt/vtgate/planbuilder/operators/operator_funcs.go b/go/vt/vtgate/planbuilder/operators/operator_funcs.go deleted file mode 100644 index 7f7aaff29c5..00000000000 --- a/go/vt/vtgate/planbuilder/operators/operator_funcs.go +++ /dev/null @@ -1,103 +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 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/plancontext" -) - -// RemovePredicate is used when we turn a predicate into a plan operator, -// and the predicate needs to be removed as an AST construct -func RemovePredicate(ctx *plancontext.PlanningContext, expr sqlparser.Expr, op ops.Operator) (ops.Operator, error) { - switch op := op.(type) { - case *Route: - newSrc, err := RemovePredicate(ctx, expr, op.Source) - if err != nil { - return nil, err - } - op.Source = newSrc - return op, err - case *ApplyJoin: - isRemoved := false - deps := ctx.SemTable.RecursiveDeps(expr) - if deps.IsSolvedBy(TableID(op.LHS)) { - newSrc, err := RemovePredicate(ctx, expr, op.LHS) - if err != nil { - return nil, err - } - op.LHS = newSrc - isRemoved = true - } - - if deps.IsSolvedBy(TableID(op.RHS)) { - newSrc, err := RemovePredicate(ctx, expr, op.RHS) - if err != nil { - return nil, err - } - op.RHS = newSrc - isRemoved = true - } - - var keep []sqlparser.Expr - for _, e := range sqlparser.SplitAndExpression(nil, op.Predicate) { - if ctx.SemTable.EqualsExprWithDeps(expr, e) { - isRemoved = true - } else { - keep = append(keep, e) - } - } - - if !isRemoved { - return nil, vterrors.VT12001(fmt.Sprintf("remove '%s' predicate on cross-shard join query", sqlparser.String(expr))) - } - - op.Predicate = ctx.SemTable.AndExpressions(keep...) - return op, nil - case *Filter: - idx := -1 - for i, predicate := range op.Predicates { - if ctx.SemTable.EqualsExprWithDeps(predicate, expr) { - idx = i - } - } - if idx == -1 { - // the predicate is not here. let's remove it from our source - newSrc, err := RemovePredicate(ctx, expr, op.Source) - if err != nil { - return nil, err - } - op.Source = newSrc - return op, nil - } - if len(op.Predicates) == 1 { - // no predicates left on this operator, so we just remove it - return op.Source, nil - } - - // remove the predicate from this filter - op.Predicates = append(op.Predicates[:idx], op.Predicates[idx+1:]...) - return op, nil - - default: - return nil, vterrors.VT13001("this should not happen - tried to remove predicate from the operator table") - } -} diff --git a/go/vt/vtgate/planbuilder/operators/projection.go b/go/vt/vtgate/planbuilder/operators/projection.go index 686950ba56d..e83dbbecdc3 100644 --- a/go/vt/vtgate/planbuilder/operators/projection.go +++ b/go/vt/vtgate/planbuilder/operators/projection.go @@ -149,7 +149,10 @@ func (sp StarProjections) GetSelectExprs() sqlparser.SelectExprs { func (ap AliasedProjections) GetColumns() ([]*sqlparser.AliasedExpr, error) { return slice.Map(ap, func(from *ProjExpr) *sqlparser.AliasedExpr { - return aeWrap(from.ColExpr) + return &sqlparser.AliasedExpr{ + As: from.Original.As, + Expr: from.ColExpr, + } }), nil } @@ -247,6 +250,13 @@ func (p *Projection) isDerived() bool { return p.DT != nil } +func (p *Projection) derivedName() string { + if p.DT == nil { + return "" + } + return p.DT.Alias +} + func (p *Projection) FindCol(ctx *plancontext.PlanningContext, expr sqlparser.Expr, underRoute bool) (int, error) { ap, err := p.GetAliasedProjections() if err != nil { diff --git a/go/vt/vtgate/planbuilder/operators/query_planning.go b/go/vt/vtgate/planbuilder/operators/query_planning.go index 0af4270d71b..adff05e55cb 100644 --- a/go/vt/vtgate/planbuilder/operators/query_planning.go +++ b/go/vt/vtgate/planbuilder/operators/query_planning.go @@ -19,6 +19,10 @@ package operators import ( "fmt" "io" + "slices" + "strconv" + + "vitess.io/vitess/go/slice" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vtgate/planbuilder/operators/ops" @@ -30,8 +34,9 @@ import ( type ( projector struct { columns []*ProjExpr - columnAliases sqlparser.Columns + columnAliases []string explicitColumnAliases bool + tableName sqlparser.TableName } ) @@ -277,11 +282,48 @@ func pushProjectionInVindex( return src, rewrite.NewTree("push projection into vindex", p), nil } -func (p *projector) add(pe *ProjExpr, col *sqlparser.IdentifierCI) { +func (p *projector) add(pe *ProjExpr, alias string) { + p.columns = append(p.columns, pe) + if alias != "" && slices.Index(p.columnAliases, alias) > -1 { + panic("alias already used") + } +} + +// get finds or adds an expression in the projector, returning its SQL representation with the appropriate alias +func (p *projector) get(ctx *plancontext.PlanningContext, expr sqlparser.Expr) sqlparser.Expr { + for _, column := range p.columns { + if ctx.SemTable.EqualsExprWithDeps(expr, column.ColExpr) { + alias := p.claimUnusedAlias(column.Original) + out := sqlparser.NewColName(alias) + out.Qualifier = p.tableName + + ctx.SemTable.CopySemanticInfo(expr, out) + return out + } + } + + // we could not find the expression, so we add it + alias := sqlparser.UnescapedString(expr) + pe := newProjExpr(sqlparser.NewAliasedExpr(expr, alias)) p.columns = append(p.columns, pe) - if col != nil { - p.columnAliases = append(p.columnAliases, *col) + p.columnAliases = append(p.columnAliases, alias) + + out := sqlparser.NewColName(alias) + out.Qualifier = p.tableName + + ctx.SemTable.CopySemanticInfo(expr, out) + + return out +} + +// claimUnusedAlias generates a unique alias based on the provided expression, ensuring no duplication in the projector +func (p *projector) claimUnusedAlias(ae *sqlparser.AliasedExpr) string { + bare := ae.ColumnName() + alias := bare + for i := int64(0); slices.Index(p.columnAliases, alias) > -1; i++ { + alias = bare + strconv.FormatInt(i, 10) } + return alias } // pushProjectionInApplyJoin pushes down a projection operation into an ApplyJoin operation. @@ -306,18 +348,18 @@ func pushProjectionInApplyJoin( src.JoinColumns = nil for idx, pe := range ap { - var col *sqlparser.IdentifierCI + var alias string if p.DT != nil && idx < len(p.DT.Columns) { - col = &p.DT.Columns[idx] + alias = p.DT.Columns[idx].String() } - err := splitProjectionAcrossJoin(ctx, src, lhs, rhs, pe, col) + err := splitProjectionAcrossJoin(ctx, src, lhs, rhs, pe, alias) if err != nil { return nil, nil, err } } if p.isDerived() { - err := exposeColumnsThroughDerivedTable(ctx, p, src, lhs) + err := exposeColumnsThroughDerivedTable(ctx, p, src, lhs, rhs) if err != nil { return nil, nil, err } @@ -344,7 +386,7 @@ func splitProjectionAcrossJoin( join *ApplyJoin, lhs, rhs *projector, pe *ProjExpr, - colAlias *sqlparser.IdentifierCI, + alias string, ) error { // Check if the current expression can reuse an existing column in the ApplyJoin. @@ -352,7 +394,7 @@ func splitProjectionAcrossJoin( return nil } - col, err := splitUnexploredExpression(ctx, join, lhs, rhs, pe, colAlias) + col, err := splitUnexploredExpression(ctx, join, lhs, rhs, pe, alias) if err != nil { return err } @@ -367,7 +409,7 @@ func splitUnexploredExpression( join *ApplyJoin, lhs, rhs *projector, pe *ProjExpr, - colAlias *sqlparser.IdentifierCI, + alias string, ) (JoinColumn, error) { // Get a JoinColumn for the current expression. col, err := join.getJoinColumnFor(ctx, pe.Original, pe.ColExpr, false) @@ -378,23 +420,22 @@ func splitUnexploredExpression( // Update the left and right child columns and names based on the JoinColumn type. switch { case col.IsPureLeft(): - lhs.add(pe, colAlias) + lhs.add(pe, alias) case col.IsPureRight(): - rhs.add(pe, colAlias) + rhs.add(pe, alias) case col.IsMixedLeftAndRight(): for _, lhsExpr := range col.LHSExprs { - var lhsAlias *sqlparser.IdentifierCI - if colAlias != nil { + lhsAlias := "" + if alias != "" { // 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 + lhsAlias = sqlparser.String(lhsExpr.Expr) } lhs.add(newProjExpr(aeWrap(lhsExpr.Expr)), lhsAlias) } innerPE := newProjExprWithInner(pe.Original, col.RHSExpr) innerPE.ColExpr = col.RHSExpr innerPE.Info = pe.Info - rhs.add(innerPE, colAlias) + rhs.add(innerPE, alias) } return col, nil } @@ -411,7 +452,7 @@ func splitUnexploredExpression( // The function iterates through each join predicate, rewriting the expressions in the predicate's // 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 { +func exposeColumnsThroughDerivedTable(ctx *plancontext.PlanningContext, p *Projection, src *ApplyJoin, lhs, rhs *projector) error { derivedTbl, err := ctx.SemTable.TableInfoFor(p.DT.TableID) if err != nil { return err @@ -420,34 +461,75 @@ func exposeColumnsThroughDerivedTable(ctx *plancontext.PlanningContext, p *Proje if err != nil { return err } - for _, predicate := range src.JoinPredicates { - for idx, bve := range predicate.LHSExprs { - expr := bve.Expr - tbl, err := ctx.SemTable.TableInfoForExpr(expr) - if err != nil { - return err + lhs.tableName = derivedTblName + rhs.tableName = derivedTblName + + lhsIDs := TableID(src.LHS) + rhsIDs := TableID(src.RHS) + rewriteColumnsForJoin(ctx, src.JoinPredicates, lhsIDs, rhsIDs, lhs, rhs, true) + rewriteColumnsForJoin(ctx, src.JoinColumns, lhsIDs, rhsIDs, lhs, rhs, true) + return nil +} + +func rewriteColumnsForJoin( + ctx *plancontext.PlanningContext, + columns []JoinColumn, + lhsIDs, rhsIDs semantics.TableSet, + lhs, rhs *projector, + exposeRHS bool, // we only want to expose the returned columns from the RHS. + // For predicates, we don't need to expose the RHS columns +) { + for colIdx, column := range columns { + for lhsIdx, bve := range column.LHSExprs { + // since this is on the LHSExprs, we know that dependencies are from that side of the join + column.LHSExprs[lhsIdx].Expr = lhs.get(ctx, bve.Expr) + } + if column.IsPureLeft() { + continue + } + + // now we need to go over the predicate and find + var rewriteTo sqlparser.Expr + + pre := func(node, _ sqlparser.SQLNode) bool { + _, isSQ := node.(*sqlparser.Subquery) + if isSQ { + return false } - tblExpr := tbl.GetExpr() - tblName, err := tblExpr.TableName() - if err != nil { - return err + expr, ok := node.(sqlparser.Expr) + if !ok { + return true } + deps := ctx.SemTable.RecursiveDeps(expr) + + switch { + case deps.IsEmpty(): + return true + case deps.IsSolvedBy(lhsIDs): + rewriteTo = lhs.get(ctx, expr) + return false + case deps.IsSolvedBy(rhsIDs): + if exposeRHS { + rewriteTo = rhs.get(ctx, expr) + } + return false + default: + return true + } + } - expr = semantics.RewriteDerivedTableExpression(expr, derivedTbl) - out := prefixColNames(tblName, expr) - - alias := sqlparser.UnescapedString(out) - predicate.LHSExprs[idx].Expr = sqlparser.NewColNameWithQualifier(alias, derivedTblName) - identifierCI := sqlparser.NewIdentifierCI(alias) - projExpr := newProjExprWithInner(&sqlparser.AliasedExpr{Expr: out, As: identifierCI}, out) - var colAlias *sqlparser.IdentifierCI - if lhs.explicitColumnAliases { - colAlias = &identifierCI + post := func(cursor *sqlparser.CopyOnWriteCursor) { + if rewriteTo != nil { + cursor.Replace(rewriteTo) + rewriteTo = nil + return } - lhs.add(projExpr, colAlias) } + newOriginal := sqlparser.CopyOnRewrite(column.Original.Expr, pre, post, ctx.SemTable.CopySemanticInfo).(sqlparser.Expr) + column.Original.Expr = newOriginal + + columns[colIdx] = column } - return nil } // prefixColNames adds qualifier prefixes to all ColName:s. @@ -478,7 +560,9 @@ func createProjectionWithTheseColumns( proj.Columns = AliasedProjections(p.columns) if dt != nil { kopy := *dt - kopy.Columns = p.columnAliases + kopy.Columns = slice.Map(p.columnAliases, func(s string) sqlparser.IdentifierCI { + return sqlparser.NewIdentifierCI(s) + }) proj.DT = &kopy } diff --git a/go/vt/vtgate/planbuilder/operators/route.go b/go/vt/vtgate/planbuilder/operators/route.go index e90948cbe5c..96edbf7f3e4 100644 --- a/go/vt/vtgate/planbuilder/operators/route.go +++ b/go/vt/vtgate/planbuilder/operators/route.go @@ -578,7 +578,10 @@ func (r *Route) AddColumn(ctx *plancontext.PlanningContext, reuse bool, gb bool, // if at least one column is not already present, we check if we can easily find a projection // or aggregation in our source that we can add to - derived, op, ok, offsets := addMultipleColumnsToInput(ctx, r.Source, reuse, []bool{gb}, []*sqlparser.AliasedExpr{expr}) + derived, op, ok, offsets, err := addMultipleColumnsToInput(ctx, r.Source, reuse, []bool{gb}, []*sqlparser.AliasedExpr{expr}) + if err != nil { + return 0, err + } r.Source = op if ok { return offsets[0], nil @@ -599,7 +602,7 @@ type selectExpressions interface { ops.Operator addColumnWithoutPushing(ctx *plancontext.PlanningContext, expr *sqlparser.AliasedExpr, addToGroupBy bool) (int, error) addColumnsWithoutPushing(ctx *plancontext.PlanningContext, reuse bool, addToGroupBy []bool, exprs []*sqlparser.AliasedExpr) ([]int, error) - isDerived() bool + derivedName() string } // addColumnToInput adds a column to an operator without pushing it down. @@ -615,63 +618,83 @@ func addMultipleColumnsToInput( projection ops.Operator, // if an operator needed to be built, it will be returned here found bool, // whether a matching op was found or not offsets []int, // the offsets the expressions received + err error, ) { switch op := operator.(type) { case *SubQuery: - derivedName, src, added, offset := addMultipleColumnsToInput(ctx, op.Outer, reuse, addToGroupBy, exprs) + derivedName, src, added, offset, err := addMultipleColumnsToInput(ctx, op.Outer, reuse, addToGroupBy, exprs) + if err != nil { + return "", nil, true, nil, err + } if added { op.Outer = src } - return derivedName, op, added, offset + return derivedName, op, added, offset, nil case *Distinct: - derivedName, src, added, offset := addMultipleColumnsToInput(ctx, op.Source, reuse, addToGroupBy, exprs) + derivedName, src, added, offset, err := addMultipleColumnsToInput(ctx, op.Source, reuse, addToGroupBy, exprs) + if err != nil { + return "", nil, true, nil, err + } if added { op.Source = src } - return derivedName, op, added, offset + return derivedName, op, added, offset, nil case *Limit: - derivedName, src, added, offset := addMultipleColumnsToInput(ctx, op.Source, reuse, addToGroupBy, exprs) + derivedName, src, added, offset, err := addMultipleColumnsToInput(ctx, op.Source, reuse, addToGroupBy, exprs) + if err != nil { + return "", nil, true, nil, err + } if added { op.Source = src } - return derivedName, op, added, offset + return derivedName, op, added, offset, nil case *Ordering: - derivedName, src, added, offset := addMultipleColumnsToInput(ctx, op.Source, reuse, addToGroupBy, exprs) + derivedName, src, added, offset, err := addMultipleColumnsToInput(ctx, op.Source, reuse, addToGroupBy, exprs) + if err != nil { + return "", nil, true, nil, err + } if added { op.Source = src } - return derivedName, op, added, offset + return derivedName, op, added, offset, nil case *LockAndComment: - derivedName, src, added, offset := addMultipleColumnsToInput(ctx, op.Source, reuse, addToGroupBy, exprs) + derivedName, src, added, offset, err := addMultipleColumnsToInput(ctx, op.Source, reuse, addToGroupBy, exprs) + if err != nil { + return "", nil, true, nil, err + } if added { op.Source = src } - return derivedName, op, added, offset + return derivedName, op, added, offset, nil case *Horizon: // if the horizon has an alias, then it is a derived table, // we have to add a new projection and can't build on this one - return op.Alias, op, false, nil + return op.Alias, op, false, nil, nil case selectExpressions: - if op.isDerived() { + name := op.derivedName() + if name != "" { // if the only thing we can push to is a derived table, // we have to add a new projection and can't build on this one - return "", op, false, nil + return name, op, false, nil, nil + } + offset, err := op.addColumnsWithoutPushing(ctx, reuse, addToGroupBy, exprs) + if err != nil { + return "", nil, true, nil, err } - offset, _ := op.addColumnsWithoutPushing(ctx, reuse, addToGroupBy, exprs) - return "", op, true, offset + return "", op, true, offset, nil case *Union: tableID := semantics.SingleTableSet(len(ctx.SemTable.Tables)) ctx.SemTable.Tables = append(ctx.SemTable.Tables, nil) unionColumns, err := op.GetColumns(ctx) if err != nil { - return "", op, false, nil + break } proj := &Projection{ Source: op, @@ -682,9 +705,8 @@ func addMultipleColumnsToInput( }, } return addMultipleColumnsToInput(ctx, proj, reuse, addToGroupBy, exprs) - default: - return "", op, false, nil } + return "", operator, false, nil, nil } func (r *Route) FindCol(ctx *plancontext.PlanningContext, expr sqlparser.Expr, _ bool) (int, error) { diff --git a/go/vt/vtgate/planbuilder/operators/route_planning.go b/go/vt/vtgate/planbuilder/operators/route_planning.go index 93a00154dc1..710c186b9c2 100644 --- a/go/vt/vtgate/planbuilder/operators/route_planning.go +++ b/go/vt/vtgate/planbuilder/operators/route_planning.go @@ -382,7 +382,10 @@ func mergeOrJoin(ctx *plancontext.PlanningContext, lhs, rhs ops.Operator, joinPr return nil, nil, vterrors.VT12001("JOIN between derived tables with LIMIT") } - join := NewApplyJoin(Clone(rhs), Clone(lhs), nil, !inner) + join, err := NewApplyJoin(ctx, Clone(rhs), Clone(lhs), nil, !inner) + if err != nil { + return nil, nil, err + } for _, pred := range joinPredicates { err := join.AddJoinPredicate(ctx, pred) if err != nil { @@ -392,7 +395,10 @@ func mergeOrJoin(ctx *plancontext.PlanningContext, lhs, rhs ops.Operator, joinPr return join, rewrite.NewTree("logical join to applyJoin, switching side because LIMIT", join), nil } - join := NewApplyJoin(Clone(lhs), Clone(rhs), nil, !inner) + join, err := NewApplyJoin(ctx, Clone(lhs), Clone(rhs), nil, !inner) + if err != nil { + return nil, nil, err + } for _, pred := range joinPredicates { err := join.AddJoinPredicate(ctx, pred) if err != nil { diff --git a/go/vt/vtgate/planbuilder/testdata/aggr_cases.json b/go/vt/vtgate/planbuilder/testdata/aggr_cases.json index 127dd2c4837..499ee753971 100644 --- a/go/vt/vtgate/planbuilder/testdata/aggr_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/aggr_cases.json @@ -3442,8 +3442,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select phone, id, city from (select phone, id, city from `user` where 1 != 1) as x where 1 != 1", - "Query": "select phone, id, city from (select phone, id, city from `user` where id > 12) as x limit :__upper_limit", + "FieldQuery": "select x.phone, x.id, x.city from (select phone, id, city from `user` where 1 != 1) as x where 1 != 1", + "Query": "select x.phone, x.id, x.city from (select phone, id, city from `user` where id > 12) as x limit :__upper_limit", "Table": "`user`" } ] @@ -3485,8 +3485,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select phone, id, city, 1 from (select phone, id, city from `user` where 1 != 1) as x where 1 != 1", - "Query": "select phone, id, city, 1 from (select phone, id, city from `user` where id > 12) as x limit :__upper_limit", + "FieldQuery": "select x.phone, x.id, x.city, 1 from (select phone, id, city from `user` where 1 != 1) as x where 1 != 1", + "Query": "select x.phone, x.id, x.city, 1 from (select phone, id, city from `user` where id > 12) as x limit :__upper_limit", "Table": "`user`" } ] @@ -3590,9 +3590,9 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select id, val1, 1, weight_string(val1) from (select id, val1 from `user` where 1 != 1) as x where 1 != 1", + "FieldQuery": "select x.id, x.val1, 1, weight_string(x.val1) from (select id, val1 from `user` where 1 != 1) as x where 1 != 1", "OrderBy": "(1|3) ASC", - "Query": "select id, val1, 1, weight_string(val1) from (select id, val1 from `user` where val2 < 4) as x order by x.val1 asc limit :__upper_limit", + "Query": "select x.id, x.val1, 1, weight_string(x.val1) from (select id, val1 from `user` where val2 < 4) as x order by x.val1 asc limit :__upper_limit", "Table": "`user`" } ] @@ -5895,8 +5895,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select id, val2 from (select id, val2 from `user` where 1 != 1) as x where 1 != 1", - "Query": "select id, val2 from (select id, val2 from `user` where val2 is null) as x limit :__upper_limit", + "FieldQuery": "select x.id, x.val2 from (select id, val2 from `user` where 1 != 1) as x where 1 != 1", + "Query": "select x.id, x.val2 from (select id, val2 from `user` where val2 is null) as x limit :__upper_limit", "Table": "`user`" } ] @@ -6475,24 +6475,31 @@ "OperatorType": "Aggregate", "Variant": "Scalar", "Aggregates": "count_star(0) AS count(*)", - "ResultColumns": 1, "Inputs": [ { - "OperatorType": "Limit", - "Count": "INT64(25)", - "Offset": "INT64(0)", + "OperatorType": "SimpleProjection", + "Columns": [ + 2 + ], "Inputs": [ { - "OperatorType": "Route", - "Variant": "Scatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select 1, id, weight_string(id) from (select 1 as one, id from `user` where 1 != 1) as subquery_for_count where 1 != 1", - "OrderBy": "(1|2) DESC", - "Query": "select 1, id, weight_string(id) from (select 1 as one, id from `user` where `user`.is_not_deleted = true) as subquery_for_count order by subquery_for_count.id desc limit :__upper_limit", - "Table": "`user`" + "OperatorType": "Limit", + "Count": "INT64(25)", + "Offset": "INT64(0)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select subquery_for_count.one, subquery_for_count.id, 1, weight_string(subquery_for_count.id) from (select 1 as one, id from `user` where 1 != 1) as subquery_for_count where 1 != 1", + "OrderBy": "(1|3) DESC", + "Query": "select subquery_for_count.one, subquery_for_count.id, 1, weight_string(subquery_for_count.id) from (select 1 as one, id from `user` where `user`.is_not_deleted = true) as subquery_for_count order by subquery_for_count.id desc limit :__upper_limit", + "Table": "`user`" + } + ] } ] } @@ -6518,24 +6525,31 @@ "OperatorType": "Aggregate", "Variant": "Scalar", "Aggregates": "count_star(0) AS count(*)", - "ResultColumns": 1, "Inputs": [ { - "OperatorType": "Limit", - "Count": "INT64(25)", - "Offset": "INT64(0)", + "OperatorType": "SimpleProjection", + "Columns": [ + 2 + ], "Inputs": [ { - "OperatorType": "Route", - "Variant": "Scatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select 1, id, weight_string(id) from (select 1 as one, id from `user` where 1 != 1) as subquery_for_count where 1 != 1", - "OrderBy": "(1|2) DESC", - "Query": "select 1, id, weight_string(id) from (select 1 as one, id from `user` where `user`.is_not_deleted = true) as subquery_for_count order by subquery_for_count.id desc limit :__upper_limit", - "Table": "`user`" + "OperatorType": "Limit", + "Count": "INT64(25)", + "Offset": "INT64(0)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select subquery_for_count.one, subquery_for_count.id, 1, weight_string(subquery_for_count.id) from (select 1 as one, id from `user` where 1 != 1) as subquery_for_count where 1 != 1", + "OrderBy": "(1|3) DESC", + "Query": "select subquery_for_count.one, subquery_for_count.id, 1, weight_string(subquery_for_count.id) from (select 1 as one, id from `user` where `user`.is_not_deleted = true) as subquery_for_count order by subquery_for_count.id desc limit :__upper_limit", + "Table": "`user`" + } + ] } ] } diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.json b/go/vt/vtgate/planbuilder/testdata/from_cases.json index f977a5ca7a8..b5efdb18eae 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.json @@ -511,7 +511,7 @@ "Sharded": true }, "FieldQuery": "select 42 from `user` as u, user_extra as ue, music as m where 1 != 1", - "Query": "select 42 from `user` as u, user_extra as ue, music as m where u.id = ue.user_id and m.user_id = u.id and (u.foo or m.foo or ue.foo)", + "Query": "select 42 from `user` as u, user_extra as ue, music as m where u.id = ue.user_id and u.id = :m_user_id and (u.foo or :m_foo or ue.foo) and m.user_id = u.id and (u.foo or m.foo or ue.foo)", "Table": "`user`, music, user_extra" }, "TablesUsed": [ @@ -1598,7 +1598,7 @@ "Sharded": true }, "FieldQuery": "select t.id from (select id from `user` where 1 != 1) as t, user_extra where 1 != 1", - "Query": "select t.id from (select id from `user` where id = 5) as t, user_extra where t.id = user_extra.user_id", + "Query": "select t.id from (select id from `user` where id = 5 and id = :user_extra_user_id) as t, user_extra where t.id = user_extra.user_id", "Table": "`user`, user_extra", "Values": [ "INT64(5)" @@ -1736,7 +1736,7 @@ "Sharded": true }, "FieldQuery": "select t.id from (select id, textcol1 as baz from `user` as route1 where 1 != 1) as t, (select id, textcol1 + textcol1 as baz from `user` where 1 != 1) as s where 1 != 1", - "Query": "select t.id from (select id, textcol1 as baz from `user` as route1 where textcol1 = '3') as t, (select id, textcol1 + textcol1 as baz from `user` where textcol1 + textcol1 = '3') as s where t.id = s.id", + "Query": "select t.id from (select id, textcol1 as baz from `user` as route1 where textcol1 = '3') as t, (select id, textcol1 + textcol1 as baz from `user` where textcol1 + textcol1 = '3' and id = :t_id) as s where t.id = s.id", "Table": "`user`" }, "TablesUsed": [ @@ -1989,8 +1989,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select t.id, t.`user.col` from (select `user`.id, `user`.col1, `user`.col as `user.col` from `user` where 1 != 1) as t where 1 != 1", - "Query": "select t.id, t.`user.col` from (select `user`.id, `user`.col1, `user`.col as `user.col` from `user`) as t", + "FieldQuery": "select t.id, t.`user.col` from (select `user`.id, `user`.col1, `user`.col as `user.col` from `user` where 1 != 1) as t(`user.col`) where 1 != 1", + "Query": "select t.id, t.`user.col` from (select `user`.id, `user`.col1, `user`.col as `user.col` from `user`) as t(`user.col`)", "Table": "`user`" }, { @@ -2000,8 +2000,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select 1 from user_extra where 1 != 1", - "Query": "select 1 from user_extra where user_extra.col = :user_col", + "FieldQuery": "select 1 from (select user_extra.col as `user_extra.col` from user_extra where 1 != 1) as t(`user_extra.col`) where 1 != 1", + "Query": "select 1 from (select user_extra.col as `user_extra.col` from user_extra where user_extra.col = :user_col) as t(`user_extra.col`)", "Table": "user_extra" } ] @@ -3187,8 +3187,8 @@ "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)", + "FieldQuery": "select i + 1 from (select `user`.id from `user` where 1 != 1) as t where 1 != 1", + "Query": "select i + 1 from (select `user`.id from `user`) as t", "Table": "`user`" }, { diff --git a/go/vt/vtgate/planbuilder/testdata/large_union_cases.json b/go/vt/vtgate/planbuilder/testdata/large_union_cases.json index 1ad7b33d589..bcf68e30e57 100644 --- a/go/vt/vtgate/planbuilder/testdata/large_union_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/large_union_cases.json @@ -23,8 +23,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from ((select content, user_id from music where 1 != 1) union (select content, user_id from music where 1 != 1)) as dt where 1 != 1", - "Query": "select content, user_id, weight_string(content), weight_string(user_id) from ((select content, user_id from music where user_id = 1270698330 order by created_at asc, id asc limit 11) union (select content, user_id from music where user_id = 1270698330 order by created_at asc, id asc limit 11)) as dt", + "FieldQuery": "select dt.content, dt.user_id, weight_string(content), weight_string(user_id) from ((select content, user_id from music where 1 != 1) union (select content, user_id from music where 1 != 1)) as dt where 1 != 1", + "Query": "select dt.content, dt.user_id, weight_string(content), weight_string(user_id) from ((select content, user_id from music where user_id = 1270698330 order by created_at asc, id asc limit 11) union (select content, user_id from music where user_id = 1270698330 order by created_at asc, id asc limit 11)) as dt", "Table": "music", "Values": [ "INT64(1270698330)" @@ -38,8 +38,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from ((select content, user_id from music where 1 != 1) union (select content, user_id from music where 1 != 1)) as dt where 1 != 1", - "Query": "select content, user_id, weight_string(content), weight_string(user_id) from ((select content, user_id from music where user_id = 1270699497 order by created_at asc, id asc limit 11) union (select content, user_id from music where user_id = 1270699497 order by created_at asc, id asc limit 11)) as dt", + "FieldQuery": "select dt.content, dt.user_id, weight_string(content), weight_string(user_id) from ((select content, user_id from music where 1 != 1) union (select content, user_id from music where 1 != 1)) as dt where 1 != 1", + "Query": "select dt.content, dt.user_id, weight_string(content), weight_string(user_id) from ((select content, user_id from music where user_id = 1270699497 order by created_at asc, id asc limit 11) union (select content, user_id from music where user_id = 1270699497 order by created_at asc, id asc limit 11)) as dt", "Table": "music", "Values": [ "INT64(1270699497)" diff --git a/go/vt/vtgate/planbuilder/testdata/postprocess_cases.json b/go/vt/vtgate/planbuilder/testdata/postprocess_cases.json index a36ad580dc4..7f573227e65 100644 --- a/go/vt/vtgate/planbuilder/testdata/postprocess_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/postprocess_cases.json @@ -1052,7 +1052,7 @@ "Sharded": true }, "FieldQuery": "select * from (select user_id from user_extra where 1 != 1) as eu, `user` as u where 1 != 1", - "Query": "select * from (select user_id from user_extra where user_id = 5) as eu, `user` as u where u.id = 5 and u.id = eu.user_id order by eu.user_id asc", + "Query": "select * from (select user_id from user_extra where user_id = 5 and user_id = :u_id) as eu, `user` as u where u.id = 5 and u.id = eu.user_id order by eu.user_id asc", "Table": "`user`, user_extra", "Values": [ "INT64(5)" diff --git a/go/vt/vtgate/planbuilder/testdata/reference_cases.json b/go/vt/vtgate/planbuilder/testdata/reference_cases.json index 53019a48e12..140ce9f7849 100644 --- a/go/vt/vtgate/planbuilder/testdata/reference_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/reference_cases.json @@ -876,19 +876,19 @@ "QueryType": "SELECT", "Original": "select * from user.ref_with_source ref, `user`.`user` u where ref.id = u.ref_id and u.id = 2", "Instructions": { - "FieldQuery": "select * from ref_with_source as ref, `user` as u where 1 != 1", "OperatorType": "Route", "Variant": "EqualUnique", - "Vindex": "user_index", "Keyspace": { "Name": "user", "Sharded": true }, + "FieldQuery": "select * from ref_with_source as ref, `user` as u where 1 != 1", "Query": "select * from ref_with_source as ref, `user` as u where u.id = 2 and ref.id = u.ref_id", "Table": "`user`, ref_with_source", "Values": [ "INT64(2)" - ] + ], + "Vindex": "user_index" }, "TablesUsed": [ "user.ref_with_source", @@ -903,19 +903,19 @@ "QueryType": "SELECT", "Original": "select * from source_of_ref ref, `user`.`user` u where ref.id = u.ref_id and u.id = 2", "Instructions": { - "FieldQuery": "select * from ref_with_source as ref, `user` as u where 1 != 1", "OperatorType": "Route", "Variant": "EqualUnique", - "Vindex": "user_index", "Keyspace": { "Name": "user", "Sharded": true }, + "FieldQuery": "select * from ref_with_source as ref, `user` as u where 1 != 1", "Query": "select * from ref_with_source as ref, `user` as u where u.id = 2 and ref.id = u.ref_id", "Table": "`user`, ref_with_source", "Values": [ "INT64(2)" - ] + ], + "Vindex": "user_index" }, "TablesUsed": [ "user.ref_with_source", @@ -937,7 +937,7 @@ "Sharded": true }, "FieldQuery": "select 1 from `user` as u, user_extra as ue, ref_with_source as sr, ref as rr where 1 != 1", - "Query": "select 1 from `user` as u, user_extra as ue, ref_with_source as sr, ref as rr where rr.bar = sr.bar and u.id = ue.user_id and sr.foo = ue.foo", + "Query": "select 1 from `user` as u, user_extra as ue, ref_with_source as sr, ref as rr where sr.foo = :ue_foo and rr.bar = sr.bar and u.id = ue.user_id and sr.foo = ue.foo", "Table": "`user`, ref, ref_with_source, user_extra" }, "TablesUsed": [ diff --git a/go/vt/vtgate/planbuilder/testdata/select_cases.json b/go/vt/vtgate/planbuilder/testdata/select_cases.json index d8f0d09a64e..5aa9b95b922 100644 --- a/go/vt/vtgate/planbuilder/testdata/select_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/select_cases.json @@ -4110,7 +4110,7 @@ "Sharded": true }, "FieldQuery": "select music.id from (select id from music where 1 != 1) as other, music where 1 != 1", - "Query": "select music.id from (select id from music where music.user_id = 5) as other, music where other.id = music.id", + "Query": "select music.id from (select id from music where music.user_id = 5 and id = :music_id) as other, music where other.id = music.id", "Table": "music", "Values": [ "INT64(5)" @@ -4136,7 +4136,7 @@ "Sharded": true }, "FieldQuery": "select music.id from (select id from music where 1 != 1) as other, music where 1 != 1", - "Query": "select music.id from (select id from music where music.user_id in ::__vals) as other, music where other.id = music.id", + "Query": "select music.id from (select id from music where music.user_id in ::__vals and id = :music_id) as other, music where other.id = music.id", "Table": "music", "Values": [ "(INT64(5), INT64(6), INT64(7))" @@ -4174,8 +4174,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select user_id from (select user_id from user_extra where 1 != 1) as ue where 1 != 1", - "Query": "select user_id from (select user_id from user_extra) as ue limit :__upper_limit", + "FieldQuery": "select ue.user_id from (select user_id from user_extra where 1 != 1) as ue where 1 != 1", + "Query": "select ue.user_id from (select user_id from user_extra) as ue limit :__upper_limit", "Table": "user_extra" } ] diff --git a/go/vt/vtgate/planbuilder/testdata/tpcc_cases.json b/go/vt/vtgate/planbuilder/testdata/tpcc_cases.json index 2677deb2cab..fa823b0ae59 100644 --- a/go/vt/vtgate/planbuilder/testdata/tpcc_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/tpcc_cases.json @@ -13,7 +13,7 @@ "Sharded": true }, "FieldQuery": "select c_discount, c_last, c_credit, w_tax from customer1 as c, warehouse1 as w where 1 != 1", - "Query": "select c_discount, c_last, c_credit, w_tax from customer1 as c, warehouse1 as w where c_d_id = 15 and c_id = 10 and w_id = 1 and c_w_id = w_id", + "Query": "select c_discount, c_last, c_credit, w_tax from customer1 as c, warehouse1 as w where c_w_id = :w_id and c_d_id = 15 and c_id = 10 and w_id = 1 and c_w_id = w_id", "Table": "customer1, warehouse1", "Values": [ "INT64(1)" @@ -947,7 +947,7 @@ "Sharded": true }, "FieldQuery": "select o.o_id, o.o_d_id from (select o_c_id, o_w_id, o_d_id, count(distinct o_w_id), o_id from orders1 where 1 != 1 group by o_c_id, o_d_id, o_w_id) as t, orders1 as o where 1 != 1", - "Query": "select o.o_id, o.o_d_id from (select o_c_id, o_w_id, o_d_id, count(distinct o_w_id), o_id from orders1 where o_w_id = 1 and o_id > 2100 and o_id < 11153 group by o_c_id, o_d_id, o_w_id having count(distinct o_id) > 1 limit 1) as t, orders1 as o where t.o_w_id = o.o_w_id and t.o_d_id = o.o_d_id and t.o_c_id = o.o_c_id limit 1", + "Query": "select o.o_id, o.o_d_id from (select o_c_id, o_w_id, o_d_id, count(distinct o_w_id), o_id from orders1 where o_w_id = 1 and o_id > 2100 and o_id < 11153 and o_w_id = :o_o_w_id and o_d_id = :o_o_d_id and o_c_id = :o_o_c_id group by o_c_id, o_d_id, o_w_id having count(distinct o_id) > 1 limit 1) as t, orders1 as o where t.o_w_id = o.o_w_id and t.o_d_id = o.o_d_id and t.o_c_id = o.o_c_id limit 1", "Table": "orders1", "Values": [ "INT64(1)" diff --git a/go/vt/vtgate/planbuilder/testdata/tpch_cases.json b/go/vt/vtgate/planbuilder/testdata/tpch_cases.json index 0a20b7e06da..107b26cc055 100644 --- a/go/vt/vtgate/planbuilder/testdata/tpch_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/tpch_cases.json @@ -535,9 +535,9 @@ { "OperatorType": "Join", "Variant": "Join", - "JoinColumnIndexes": "L:0,R:0,L:1,R:1,L:2,L:5,R:2,L:6", + "JoinColumnIndexes": "L:0,R:0,L:1,R:1,L:2,L:4,R:2,L:5", "JoinVars": { - "n1_n_name": 4, + "n1_n_name": 1, "o_custkey": 3 }, "TableName": "lineitem_orders_supplier_nation_customer_nation", @@ -548,18 +548,17 @@ "[COLUMN 0] * [COLUMN 1] as revenue", "[COLUMN 2] as supp_nation", "[COLUMN 3] as l_year", - "[COLUMN 4] as orders.o_custkey", - "[COLUMN 5] as n1.n_name", - "[COLUMN 6] as weight_string(supp_nation)", - "[COLUMN 7] as weight_string(l_year)" + "[COLUMN 4] as o_custkey", + "[COLUMN 5] as weight_string(supp_nation)", + "[COLUMN 6] as weight_string(l_year)" ], "Inputs": [ { "OperatorType": "Join", "Variant": "Join", - "JoinColumnIndexes": "L:0,R:0,R:1,L:1,L:2,L:3,R:2,L:5", + "JoinColumnIndexes": "L:0,R:0,R:1,L:1,L:2,R:2,L:4", "JoinVars": { - "l_suppkey": 4 + "l_suppkey": 3 }, "TableName": "lineitem_orders_supplier_nation", "Inputs": [ @@ -568,18 +567,17 @@ "Expressions": [ "[COLUMN 0] * [COLUMN 1] as revenue", "[COLUMN 2] as l_year", - "[COLUMN 3] as orders.o_custkey", - "[COLUMN 4] as n1.n_name", - "[COLUMN 5] as lineitem.l_suppkey", - "[COLUMN 6] as weight_string(l_year)" + "[COLUMN 3] as o_custkey", + "[COLUMN 4] as l_suppkey", + "[COLUMN 5] as weight_string(l_year)" ], "Inputs": [ { "OperatorType": "Join", "Variant": "Join", - "JoinColumnIndexes": "L:0,R:0,L:1,L:2,L:3,L:4,L:6", + "JoinColumnIndexes": "L:0,R:0,L:1,R:1,L:2,L:4", "JoinVars": { - "l_orderkey": 5 + "l_orderkey": 3 }, "TableName": "lineitem_orders", "Inputs": [ @@ -590,9 +588,9 @@ "Name": "main", "Sharded": true }, - "FieldQuery": "select sum(volume) as revenue, l_year, shipping.`orders.o_custkey`, shipping.`n1.n_name`, shipping.`lineitem.l_suppkey`, shipping.`lineitem.l_orderkey`, weight_string(l_year), shipping.supp_nation, weight_string(shipping.supp_nation), shipping.cust_nation, weight_string(shipping.cust_nation) from (select extract(year from l_shipdate) as l_year, l_extendedprice * (1 - l_discount) as volume, orders.o_custkey as `orders.o_custkey`, lineitem.l_suppkey as `lineitem.l_suppkey`, lineitem.l_orderkey as `lineitem.l_orderkey` from lineitem where 1 != 1) as shipping where 1 != 1 group by l_year, shipping.`orders.o_custkey`, shipping.`n1.n_name`, shipping.`lineitem.l_suppkey`, shipping.`lineitem.l_orderkey`, weight_string(l_year)", - "OrderBy": "(7|8) ASC, (9|10) ASC, (1|6) ASC", - "Query": "select sum(volume) as revenue, l_year, shipping.`orders.o_custkey`, shipping.`n1.n_name`, shipping.`lineitem.l_suppkey`, shipping.`lineitem.l_orderkey`, weight_string(l_year), shipping.supp_nation, weight_string(shipping.supp_nation), shipping.cust_nation, weight_string(shipping.cust_nation) from (select extract(year from l_shipdate) as l_year, l_extendedprice * (1 - l_discount) as volume, orders.o_custkey as `orders.o_custkey`, lineitem.l_suppkey as `lineitem.l_suppkey`, lineitem.l_orderkey as `lineitem.l_orderkey` from lineitem where l_shipdate between date('1995-01-01') and date('1996-12-31')) as shipping group by l_year, shipping.`orders.o_custkey`, shipping.`n1.n_name`, shipping.`lineitem.l_suppkey`, shipping.`lineitem.l_orderkey`, weight_string(l_year) order by shipping.supp_nation asc, shipping.cust_nation asc, shipping.l_year asc", + "FieldQuery": "select sum(volume) as revenue, l_year, shipping.l_suppkey, shipping.l_orderkey, weight_string(l_year), shipping.supp_nation, weight_string(shipping.supp_nation), shipping.cust_nation, weight_string(shipping.cust_nation) from (select extract(year from l_shipdate) as l_year, l_extendedprice * (1 - l_discount) as volume, l_suppkey as l_suppkey, l_orderkey as l_orderkey from lineitem where 1 != 1) as shipping(l_orderkey) where 1 != 1 group by l_year, shipping.l_suppkey, shipping.l_orderkey, weight_string(l_year)", + "OrderBy": "(5|6) ASC, (7|8) ASC, (1|4) ASC", + "Query": "select sum(volume) as revenue, l_year, shipping.l_suppkey, shipping.l_orderkey, weight_string(l_year), shipping.supp_nation, weight_string(shipping.supp_nation), shipping.cust_nation, weight_string(shipping.cust_nation) from (select extract(year from l_shipdate) as l_year, l_extendedprice * (1 - l_discount) as volume, l_suppkey as l_suppkey, l_orderkey as l_orderkey from lineitem where l_shipdate between date('1995-01-01') and date('1996-12-31')) as shipping(l_orderkey) group by l_year, shipping.l_suppkey, shipping.l_orderkey, weight_string(l_year) order by shipping.supp_nation asc, shipping.cust_nation asc, shipping.l_year asc", "Table": "lineitem" }, { @@ -602,8 +600,8 @@ "Name": "main", "Sharded": true }, - "FieldQuery": "select count(*) from orders where 1 != 1 group by .0", - "Query": "select count(*) from orders where o_orderkey = :l_orderkey group by .0", + "FieldQuery": "select count(*), shipping.o_custkey from (select o_custkey as o_custkey, o_orderkey as o_orderkey from orders where 1 != 1) as shipping(o_orderkey) where 1 != 1 group by shipping.o_custkey", + "Query": "select count(*), shipping.o_custkey from (select o_custkey as o_custkey, o_orderkey as o_orderkey from orders where o_orderkey = :l_orderkey) as shipping(o_orderkey) group by shipping.o_custkey", "Table": "orders", "Values": [ ":l_orderkey" @@ -638,8 +636,8 @@ "Name": "main", "Sharded": true }, - "FieldQuery": "select count(*), shipping.`supplier.s_nationkey` from (select supplier.s_nationkey as `supplier.s_nationkey` from supplier where 1 != 1) as shipping where 1 != 1 group by shipping.`supplier.s_nationkey`", - "Query": "select count(*), shipping.`supplier.s_nationkey` from (select supplier.s_nationkey as `supplier.s_nationkey` from supplier where s_suppkey = :l_suppkey) as shipping group by shipping.`supplier.s_nationkey`", + "FieldQuery": "select count(*), shipping.s_nationkey from (select s_suppkey as s_suppkey, s_nationkey as s_nationkey from supplier where 1 != 1) as shipping(s_nationkey) where 1 != 1 group by shipping.s_nationkey", + "Query": "select count(*), shipping.s_nationkey from (select s_suppkey as s_suppkey, s_nationkey as s_nationkey from supplier where s_suppkey = :l_suppkey) as shipping(s_nationkey) group by shipping.s_nationkey", "Table": "supplier", "Values": [ ":l_suppkey" @@ -653,8 +651,8 @@ "Name": "main", "Sharded": true }, - "FieldQuery": "select count(*), supp_nation, weight_string(supp_nation) from (select n1.n_name as supp_nation from nation as n1 where 1 != 1) as shipping where 1 != 1 group by supp_nation, weight_string(supp_nation)", - "Query": "select count(*), supp_nation, weight_string(supp_nation) from (select n1.n_name as supp_nation from nation as n1 where n1.n_nationkey = :s_nationkey) as shipping group by supp_nation, weight_string(supp_nation)", + "FieldQuery": "select count(*), supp_nation, weight_string(supp_nation), n1.n_nationkey as `n1.n_nationkey`, shipping.supp_nation as `shipping.supp_nation`, shipping.`n1.n_name = 'FRANCE'` as `shipping.n1.n_name = 'FRANCE'`, shipping.`n1.n_name = 'GERMANY'` as `shipping.n1.n_name = 'GERMANY'` from (select n1.n_name as supp_nation, n1.n_name = 'FRANCE' as `n1.n_name = 'FRANCE'`, n1.n_name = 'GERMANY' as `n1.n_name = 'GERMANY'`, n1.n_nationkey as `n1.n_nationkey`, shipping.supp_nation as `shipping.supp_nation`, shipping.`n1.n_name = 'FRANCE'` as `shipping.n1.n_name = 'FRANCE'`, shipping.`n1.n_name = 'GERMANY'` as `shipping.n1.n_name = 'GERMANY'` from nation as n1 where 1 != 1) as shipping(`n1.n_nationkey`, `shipping.supp_nation`, `shipping.n1.n_name = 'FRANCE'`, `shipping.n1.n_name = 'GERMANY'`) where 1 != 1 group by supp_nation, weight_string(supp_nation)", + "Query": "select count(*), supp_nation, weight_string(supp_nation), n1.n_nationkey as `n1.n_nationkey`, shipping.supp_nation as `shipping.supp_nation`, shipping.`n1.n_name = 'FRANCE'` as `shipping.n1.n_name = 'FRANCE'`, shipping.`n1.n_name = 'GERMANY'` as `shipping.n1.n_name = 'GERMANY'` from (select n1.n_name as supp_nation, n1.n_name = 'FRANCE' as `n1.n_name = 'FRANCE'`, n1.n_name = 'GERMANY' as `n1.n_name = 'GERMANY'`, n1.n_nationkey as `n1.n_nationkey`, shipping.supp_nation as `shipping.supp_nation`, shipping.`n1.n_name = 'FRANCE'` as `shipping.n1.n_name = 'FRANCE'`, shipping.`n1.n_name = 'GERMANY'` as `shipping.n1.n_name = 'GERMANY'` from nation as n1 where n1.n_nationkey = :s_nationkey) as shipping(`n1.n_nationkey`, `shipping.supp_nation`, `shipping.n1.n_name = 'FRANCE'`, `shipping.n1.n_name = 'GERMANY'`) group by supp_nation, weight_string(supp_nation)", "Table": "nation", "Values": [ ":s_nationkey" @@ -693,8 +691,8 @@ "Name": "main", "Sharded": true }, - "FieldQuery": "select count(*), shipping.`customer.c_nationkey` from (select customer.c_nationkey as `customer.c_nationkey` from customer where 1 != 1) as shipping where 1 != 1 group by shipping.`customer.c_nationkey`", - "Query": "select count(*), shipping.`customer.c_nationkey` from (select customer.c_nationkey as `customer.c_nationkey` from customer where c_custkey = :o_custkey) as shipping group by shipping.`customer.c_nationkey`", + "FieldQuery": "select count(*), shipping.c_nationkey from (select c_custkey as c_custkey, c_nationkey as c_nationkey from customer where 1 != 1) as shipping(c_nationkey) where 1 != 1 group by shipping.c_nationkey", + "Query": "select count(*), shipping.c_nationkey from (select c_custkey as c_custkey, c_nationkey as c_nationkey from customer where c_custkey = :o_custkey) as shipping(c_nationkey) group by shipping.c_nationkey", "Table": "customer", "Values": [ ":o_custkey" @@ -708,8 +706,8 @@ "Name": "main", "Sharded": true }, - "FieldQuery": "select count(*), cust_nation, weight_string(cust_nation) from (select n2.n_name as cust_nation from nation as n2 where 1 != 1) as shipping where 1 != 1 group by cust_nation, weight_string(cust_nation)", - "Query": "select count(*), cust_nation, weight_string(cust_nation) from (select n2.n_name as cust_nation from nation as n2 where (:n1_n_name = 'FRANCE' and n2.n_name = 'GERMANY' or :n1_n_name = 'GERMANY' and n2.n_name = 'FRANCE') and n2.n_nationkey = :c_nationkey) as shipping group by cust_nation, weight_string(cust_nation)", + "FieldQuery": "select count(*), cust_nation, weight_string(cust_nation), n2.n_nationkey as `n2.n_nationkey`, shipping.cust_nation as `shipping.cust_nation` from (select n2.n_name as cust_nation, n2.n_name = 'GERMANY' as `n2.n_name = 'GERMANY'`, n2.n_name = 'FRANCE' as `n2.n_name = 'FRANCE'`, n2.n_nationkey as `n2.n_nationkey`, shipping.cust_nation as `shipping.cust_nation` from nation as n2 where 1 != 1) as shipping(`n2.n_nationkey`, `shipping.cust_nation`) where 1 != 1 group by cust_nation, weight_string(cust_nation)", + "Query": "select count(*), cust_nation, weight_string(cust_nation), n2.n_nationkey as `n2.n_nationkey`, shipping.cust_nation as `shipping.cust_nation` from (select n2.n_name as cust_nation, n2.n_name = 'GERMANY' as `n2.n_name = 'GERMANY'`, n2.n_name = 'FRANCE' as `n2.n_name = 'FRANCE'`, n2.n_nationkey as `n2.n_nationkey`, shipping.cust_nation as `shipping.cust_nation` from nation as n2 where (:n1_n_name = 'FRANCE' and n2.n_name = 'GERMANY' or :n1_n_name = 'GERMANY' and n2.n_name = 'FRANCE') and n2.n_nationkey = :c_nationkey) as shipping(`n2.n_nationkey`, `shipping.cust_nation`) group by cust_nation, weight_string(cust_nation)", "Table": "nation", "Values": [ ":c_nationkey" diff --git a/go/vt/vtgate/planbuilder/testdata/union_cases.json b/go/vt/vtgate/planbuilder/testdata/union_cases.json index 8b4a2d91cdb..17510c62d57 100644 --- a/go/vt/vtgate/planbuilder/testdata/union_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/union_cases.json @@ -42,8 +42,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select id, weight_string(id) from (select id from `user` where 1 != 1 union select id from music where 1 != 1) as dt where 1 != 1", - "Query": "select id, weight_string(id) from (select id from `user` union select id from music) as dt", + "FieldQuery": "select dt.id, weight_string(dt.id) from (select id from `user` where 1 != 1 union select id from music where 1 != 1) as dt where 1 != 1", + "Query": "select dt.id, weight_string(dt.id) from (select id from `user` union select id from music) as dt", "Table": "`user`, music" } ] @@ -384,8 +384,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select id, weight_string(id) from (select id from `user` where 1 != 1 union select id from music where 1 != 1 union select 1 from dual where 1 != 1) as dt where 1 != 1", - "Query": "select id, weight_string(id) from (select id from `user` union select id from music union select 1 from dual) as dt", + "FieldQuery": "select dt.id, weight_string(dt.id) from (select id from `user` where 1 != 1 union select id from music where 1 != 1 union select 1 from dual where 1 != 1) as dt where 1 != 1", + "Query": "select dt.id, weight_string(dt.id) from (select id from `user` union select id from music union select 1 from dual) as dt", "Table": "`user`, dual, music" } ] @@ -503,8 +503,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select id, weight_string(id) from ((select id from `user` where 1 != 1) union (select id from `user` where 1 != 1)) as dt where 1 != 1", - "Query": "select id, weight_string(id) from ((select id from `user` order by id desc) union (select id from `user` order by id asc)) as dt", + "FieldQuery": "select dt.id, weight_string(dt.id) from ((select id from `user` where 1 != 1) union (select id from `user` where 1 != 1)) as dt where 1 != 1", + "Query": "select dt.id, weight_string(dt.id) from ((select id from `user` order by id desc) union (select id from `user` order by id asc)) as dt", "Table": "`user`" } ] @@ -534,8 +534,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select `1`, weight_string(`1`) from (select 1 from dual where 1 != 1 union select null from dual where 1 != 1 union select 1.0 from dual where 1 != 1 union select '1' from dual where 1 != 1 union select 2 from dual where 1 != 1 union select 2.0 from `user` where 1 != 1) as dt where 1 != 1", - "Query": "select `1`, weight_string(`1`) from (select 1 from dual union select null from dual union select 1.0 from dual union select '1' from dual union select 2 from dual union select 2.0 from `user`) as dt", + "FieldQuery": "select dt.`1`, weight_string(dt.`1`) from (select 1 from dual where 1 != 1 union select null from dual where 1 != 1 union select 1.0 from dual where 1 != 1 union select '1' from dual where 1 != 1 union select 2 from dual where 1 != 1 union select 2.0 from `user` where 1 != 1) as dt where 1 != 1", + "Query": "select dt.`1`, weight_string(dt.`1`) from (select 1 from dual union select null from dual union select 1.0 from dual union select '1' from dual union select 2 from dual union select 2.0 from `user`) as dt", "Table": "`user`, dual" } ] @@ -767,8 +767,8 @@ "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", + "FieldQuery": "select dt.id, weight_string(dt.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 dt.id, weight_string(dt.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" } ] @@ -804,8 +804,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select id, weight_string(id) from (select id from `user` where 1 != 1 union select id from music where 1 != 1) as dt where 1 != 1", - "Query": "select id, weight_string(id) from (select id from `user` union select id from music) as dt", + "FieldQuery": "select id, weight_string(dt.id) from (select id from `user` where 1 != 1 union select id from music where 1 != 1) as dt where 1 != 1", + "Query": "select id, weight_string(dt.id) from (select id from `user` union select id from music) as dt", "Table": "`user`, music" }, { @@ -854,8 +854,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select id, weight_string(id) from (select id from `user` where 1 != 1 union select 3 from dual where 1 != 1) as dt where 1 != 1", - "Query": "select id, weight_string(id) from (select id from `user` union select 3 from dual limit :__upper_limit) as dt", + "FieldQuery": "select id, weight_string(dt.id) from (select id from `user` where 1 != 1 union select 3 from dual where 1 != 1) as dt where 1 != 1", + "Query": "select id, weight_string(dt.id) from (select id from `user` union select 3 from dual limit :__upper_limit) as dt", "Table": "`user`, dual" } ] @@ -913,8 +913,8 @@ "Name": "main", "Sharded": false }, - "FieldQuery": "select col, weight_string(col) from (select col from unsharded where 1 != 1 union select col2 from unsharded where 1 != 1) as dt where 1 != 1", - "Query": "select col, weight_string(col) from (select col from unsharded union select col2 from unsharded) as dt", + "FieldQuery": "select dt.col, weight_string(col) from (select col from unsharded where 1 != 1 union select col2 from unsharded where 1 != 1) as dt where 1 != 1", + "Query": "select dt.col, weight_string(col) from (select col from unsharded union select col2 from unsharded) as dt", "Table": "unsharded" }, { @@ -1071,8 +1071,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select id, weight_string(id) from (select id from `user` where 1 != 1 union select 3 from dual where 1 != 1) as dt where 1 != 1", - "Query": "select id, weight_string(id) from (select id from `user` union select 3 from dual) as dt", + "FieldQuery": "select dt.id, weight_string(dt.id) from (select id from `user` where 1 != 1 union select 3 from dual where 1 != 1) as dt where 1 != 1", + "Query": "select dt.id, weight_string(dt.id) from (select id from `user` union select 3 from dual) as dt", "Table": "`user`, dual" } ] @@ -1450,8 +1450,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select bar, baz, toto, weight_string(bar), weight_string(baz), weight_string(toto) from (select bar, baz, toto from music where 1 != 1 union select foo, foo, foo from `user` where 1 != 1) as dt where 1 != 1", - "Query": "select bar, baz, toto, weight_string(bar), weight_string(baz), weight_string(toto) from (select bar, baz, toto from music union select foo, foo, foo from `user`) as dt", + "FieldQuery": "select dt.bar, dt.baz, dt.toto, weight_string(dt.bar), weight_string(dt.baz), weight_string(dt.toto) from (select bar, baz, toto from music where 1 != 1 union select foo, foo, foo from `user` where 1 != 1) as dt where 1 != 1", + "Query": "select dt.bar, dt.baz, dt.toto, weight_string(dt.bar), weight_string(dt.baz), weight_string(dt.toto) from (select bar, baz, toto from music union select foo, foo, foo from `user`) as dt", "Table": "`user`, music" } ] @@ -1484,8 +1484,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select foo, foo, foo, weight_string(foo) from (select foo, foo, foo from `user` where 1 != 1 union select bar, baz, toto from music where 1 != 1) as dt where 1 != 1", - "Query": "select foo, foo, foo, weight_string(foo) from (select foo, foo, foo from `user` union select bar, baz, toto from music) as dt", + "FieldQuery": "select dt.foo, dt.foo, dt.foo, weight_string(dt.foo) from (select foo, foo, foo from `user` where 1 != 1 union select bar, baz, toto from music where 1 != 1) as dt where 1 != 1", + "Query": "select dt.foo, dt.foo, dt.foo, weight_string(dt.foo) from (select foo, foo, foo from `user` union select bar, baz, toto from music) as dt", "Table": "`user`, music" } ] @@ -1546,8 +1546,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select col1, weight_string(col1) from (select col1 from `user` where 1 != 1 union select 3 from `user` where 1 != 1) as dt where 1 != 1", - "Query": "select col1, weight_string(col1) from (select col1 from `user` union select 3 from `user`) as dt", + "FieldQuery": "select dt.col1, weight_string(dt.col1) from (select col1 from `user` where 1 != 1 union select 3 from `user` where 1 != 1) as dt where 1 != 1", + "Query": "select dt.col1, weight_string(dt.col1) from (select col1 from `user` union select 3 from `user`) as dt", "Table": "`user`" } ] @@ -1577,8 +1577,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select `3`, weight_string(`3`) from (select 3 from `user` where 1 != 1 union select col1 from `user` where 1 != 1) as dt where 1 != 1", - "Query": "select `3`, weight_string(`3`) from (select 3 from `user` union select col1 from `user`) as dt", + "FieldQuery": "select dt.`3`, weight_string(dt.`3`) from (select 3 from `user` where 1 != 1 union select col1 from `user` where 1 != 1) as dt where 1 != 1", + "Query": "select dt.`3`, weight_string(dt.`3`) from (select 3 from `user` union select col1 from `user`) as dt", "Table": "`user`" } ] @@ -1608,8 +1608,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select `3`, weight_string(`3`) from (select 3 from `user` where 1 != 1 union select now() from `user` where 1 != 1) as dt where 1 != 1", - "Query": "select `3`, weight_string(`3`) from (select 3 from `user` union select now() from `user`) as dt", + "FieldQuery": "select dt.`3`, weight_string(dt.`3`) from (select 3 from `user` where 1 != 1 union select now() from `user` where 1 != 1) as dt where 1 != 1", + "Query": "select dt.`3`, weight_string(dt.`3`) from (select 3 from `user` union select now() from `user`) as dt", "Table": "`user`" } ] @@ -1639,8 +1639,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select `now()`, weight_string(`now()`) from (select now() from `user` where 1 != 1 union select 3 from `user` where 1 != 1) as dt where 1 != 1", - "Query": "select `now()`, weight_string(`now()`) from (select now() from `user` union select 3 from `user`) as dt", + "FieldQuery": "select dt.`now()`, weight_string(dt.`now()`) from (select now() from `user` where 1 != 1 union select 3 from `user` where 1 != 1) as dt where 1 != 1", + "Query": "select dt.`now()`, weight_string(dt.`now()`) from (select now() from `user` union select 3 from `user`) as dt", "Table": "`user`" } ] @@ -1670,8 +1670,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select `now()`, weight_string(`now()`) from (select now() from `user` where 1 != 1 union select id from `user` where 1 != 1) as dt where 1 != 1", - "Query": "select `now()`, weight_string(`now()`) from (select now() from `user` union select id from `user`) as dt", + "FieldQuery": "select dt.`now()`, weight_string(dt.`now()`) from (select now() from `user` where 1 != 1 union select id from `user` where 1 != 1) as dt where 1 != 1", + "Query": "select dt.`now()`, weight_string(dt.`now()`) from (select now() from `user` union select id from `user`) as dt", "Table": "`user`" } ] From 608ee651f98dda0abc84c6ee09dfc28f6f156917 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Tue, 15 Oct 2024 21:11:40 +0200 Subject: [PATCH 2/5] test: update test expectations Signed-off-by: Andres Taylor --- go/vt/vtgate/executor_select_test.go | 36 ++++++++++++++-------------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/go/vt/vtgate/executor_select_test.go b/go/vt/vtgate/executor_select_test.go index d92ca58ff3a..011dd9d21aa 100644 --- a/go/vt/vtgate/executor_select_test.go +++ b/go/vt/vtgate/executor_select_test.go @@ -2815,13 +2815,13 @@ func TestCrossShardSubquery(t *testing.T) { result, err := executorExec(ctx, executor, session, "select id1 from (select u1.id id1, u2.id from user u1 join user u2 on u2.id = u1.col where u1.id = 1) as t", nil) require.NoError(t, err) wantQueries := []*querypb.BoundQuery{{ - Sql: "select id1, t.`u1.col` from (select u1.id as id1, u1.col as `u1.col` from `user` as u1 where u1.id = 1) as t", + Sql: "select id1, t.`u1.col` from (select u1.id as id1, u1.col as `u1.col` from `user` as u1 where u1.id = 1) as t(`u1.col`)", BindVariables: map[string]*querypb.BindVariable{}, }} utils.MustMatch(t, wantQueries, sbc1.Queries) wantQueries = []*querypb.BoundQuery{{ - Sql: "select 1 from (select u2.id from `user` as u2 where u2.id = :u1_col) as t", + Sql: "select 1 from (select u2.id as id from `user` as u2 where u2.id = :u1_col) as t", BindVariables: map[string]*querypb.BindVariable{"u1_col": sqltypes.Int32BindVariable(3)}, }} utils.MustMatch(t, wantQueries, sbc2.Queries) @@ -2891,12 +2891,12 @@ func TestCrossShardSubqueryStream(t *testing.T) { result, err := executorStream(ctx, executor, "select id1 from (select u1.id id1, u2.id from user u1 join user u2 on u2.id = u1.col where u1.id = 1) as t") require.NoError(t, err) wantQueries := []*querypb.BoundQuery{{ - Sql: "select id1, t.`u1.col` from (select u1.id as id1, u1.col as `u1.col` from `user` as u1 where u1.id = 1) as t", + Sql: "select id1, t.`u1.col` from (select u1.id as id1, u1.col as `u1.col` from `user` as u1 where u1.id = 1) as t(`u1.col`)", BindVariables: map[string]*querypb.BindVariable{}, }} utils.MustMatch(t, wantQueries, sbc1.Queries) wantQueries = []*querypb.BoundQuery{{ - Sql: "select 1 from (select u2.id from `user` as u2 where u2.id = :u1_col) as t", + Sql: "select 1 from (select u2.id as id from `user` as u2 where u2.id = :u1_col) as t", BindVariables: map[string]*querypb.BindVariable{"u1_col": sqltypes.Int32BindVariable(3)}, }} utils.MustMatch(t, wantQueries, sbc2.Queries) @@ -2934,10 +2934,10 @@ func TestCrossShardSubqueryGetFields(t *testing.T) { result, err := executorExec(ctx, executor, session, "select main1.col, t.id1 from main1 join (select u1.id id1, u2.id from user u1 join user u2 on u2.id = u1.col where u1.id = 1) as t", nil) require.NoError(t, err) wantQueries := []*querypb.BoundQuery{{ - Sql: "select t.id1, t.`u1.col` from (select u1.id as id1, u1.col as `u1.col` from `user` as u1 where 1 != 1) as t where 1 != 1", + Sql: "select t.id1, t.`u1.col` from (select u1.id as id1, u1.col as `u1.col` from `user` as u1 where 1 != 1) as t(`u1.col`) where 1 != 1", BindVariables: map[string]*querypb.BindVariable{}, }, { - Sql: "select 1 from (select u2.id from `user` as u2 where 1 != 1) as t where 1 != 1", + Sql: "select 1 from (select u2.id as id from `user` as u2 where 1 != 1) as t where 1 != 1", BindVariables: map[string]*querypb.BindVariable{ "u1_col": sqltypes.NullBindVariable, }, @@ -3847,14 +3847,14 @@ func TestSelectAggregationNoData(t *testing.T) { { sql: `select count(*) from (select col1, col2 from user limit 2) x`, sandboxRes: sqltypes.MakeTestResult(sqltypes.MakeTestFields("col1|col2|1", "int64|int64|int64")), - expSandboxQ: "select col1, col2, 1 from (select col1, col2 from `user`) as x limit :__upper_limit", + expSandboxQ: "select x.col1, x.col2, 1 from (select col1, col2 from `user`) as x limit :__upper_limit", expField: `[name:"count(*)" type:INT64]`, expRow: `[[INT64(0)]]`, }, { sql: `select col2, count(*) from (select col1, col2 from user limit 2) x group by col2`, sandboxRes: sqltypes.MakeTestResult(sqltypes.MakeTestFields("col1|col2|1|weight_string(col2)", "int64|int64|int64|varbinary")), - expSandboxQ: "select col1, col2, 1, weight_string(col2) from (select col1, col2 from `user`) as x limit :__upper_limit", + expSandboxQ: "select x.col1, x.col2, 1, weight_string(x.col2) from (select col1, col2 from `user`) as x limit :__upper_limit", expField: `[name:"col2" type:INT64 name:"count(*)" type:INT64]`, expRow: `[]`, }, @@ -3939,70 +3939,70 @@ func TestSelectAggregationData(t *testing.T) { { sql: `select count(*) from (select col1, col2 from user limit 2) x`, sandboxRes: sqltypes.MakeTestResult(sqltypes.MakeTestFields("col1|col2|1", "int64|int64|int64"), "100|200|1", "200|300|1"), - expSandboxQ: "select col1, col2, 1 from (select col1, col2 from `user`) as x limit :__upper_limit", + expSandboxQ: "select x.col1, x.col2, 1 from (select col1, col2 from `user`) as x limit :__upper_limit", expField: `[name:"count(*)" type:INT64]`, expRow: `[[INT64(2)]]`, }, { sql: `select col2, count(*) from (select col1, col2 from user limit 9) x group by col2`, sandboxRes: sqltypes.MakeTestResult(sqltypes.MakeTestFields("col1|col2|1|weight_string(col2)", "int64|int64|int64|varbinary"), "100|3|1|NULL", "200|2|1|NULL"), - expSandboxQ: "select col1, col2, 1, weight_string(col2) from (select col1, col2 from `user`) as x limit :__upper_limit", + expSandboxQ: "select x.col1, x.col2, 1, weight_string(x.col2) from (select col1, col2 from `user`) as x limit :__upper_limit", expField: `[name:"col2" type:INT64 name:"count(*)" type:INT64]`, expRow: `[[INT64(2) INT64(4)] [INT64(3) INT64(5)]]`, }, { sql: `select count(col1) from (select id, col1 from user limit 2) x`, sandboxRes: sqltypes.MakeTestResult(sqltypes.MakeTestFields("id|col1", "int64|varchar"), "1|a", "2|b"), - expSandboxQ: "select id, col1 from (select id, col1 from `user`) as x limit :__upper_limit", + expSandboxQ: "select x.id, x.col1 from (select id, col1 from `user`) as x limit :__upper_limit", expField: `[name:"count(col1)" type:INT64]`, expRow: `[[INT64(2)]]`, }, { sql: `select count(col1), col2 from (select col2, col1 from user limit 9) x group by col2`, sandboxRes: sqltypes.MakeTestResult(sqltypes.MakeTestFields("col2|col1|weight_string(col2)", "int64|varchar|varbinary"), "3|a|NULL", "2|b|NULL"), - expSandboxQ: "select col2, col1, weight_string(col2) from (select col2, col1 from `user`) as x limit :__upper_limit", + expSandboxQ: "select x.col2, x.col1, weight_string(x.col2) from (select col2, col1 from `user`) as x limit :__upper_limit", expField: `[name:"count(col1)" type:INT64 name:"col2" type:INT64]`, expRow: `[[INT64(4) INT64(2)] [INT64(5) INT64(3)]]`, }, { sql: `select col1, count(col2) from (select col1, col2 from user limit 9) x group by col1`, sandboxRes: sqltypes.MakeTestResult(sqltypes.MakeTestFields("col1|col2|weight_string(col1)", "varchar|int64|varbinary"), "a|1|a", "b|null|b"), - expSandboxQ: "select col1, col2, weight_string(col1) from (select col1, col2 from `user`) as x limit :__upper_limit", + expSandboxQ: "select x.col1, x.col2, weight_string(x.col1) from (select col1, col2 from `user`) as x limit :__upper_limit", expField: `[name:"col1" type:VARCHAR name:"count(col2)" type:INT64]`, expRow: `[[VARCHAR("a") INT64(5)] [VARCHAR("b") INT64(0)]]`, }, { sql: `select col1, count(col2) from (select col1, col2 from user limit 32) x group by col1`, sandboxRes: sqltypes.MakeTestResult(sqltypes.MakeTestFields("col1|col2|weight_string(col1)", "varchar|int64|varbinary"), "null|1|null", "null|null|null", "a|1|a", "b|null|b"), - expSandboxQ: "select col1, col2, weight_string(col1) from (select col1, col2 from `user`) as x limit :__upper_limit", + expSandboxQ: "select x.col1, x.col2, weight_string(x.col1) from (select col1, col2 from `user`) as x limit :__upper_limit", expField: `[name:"col1" type:VARCHAR name:"count(col2)" type:INT64]`, expRow: `[[NULL INT64(8)] [VARCHAR("a") INT64(8)] [VARCHAR("b") INT64(0)]]`, }, { sql: `select col1, sum(col2) from (select col1, col2 from user limit 4) x group by col1`, sandboxRes: sqltypes.MakeTestResult(sqltypes.MakeTestFields("col1|col2|weight_string(col1)", "varchar|int64|varbinary"), "a|3|a"), - expSandboxQ: "select col1, col2, weight_string(col1) from (select col1, col2 from `user`) as x limit :__upper_limit", + expSandboxQ: "select x.col1, x.col2, weight_string(x.col1) from (select col1, col2 from `user`) as x limit :__upper_limit", expField: `[name:"col1" type:VARCHAR name:"sum(col2)" type:DECIMAL]`, expRow: `[[VARCHAR("a") DECIMAL(12)]]`, }, { sql: `select col1, sum(col2) from (select col1, col2 from user limit 4) x group by col1`, sandboxRes: sqltypes.MakeTestResult(sqltypes.MakeTestFields("col1|col2|weight_string(col1)", "varchar|varchar|varbinary"), "a|2|a"), - expSandboxQ: "select col1, col2, weight_string(col1) from (select col1, col2 from `user`) as x limit :__upper_limit", + expSandboxQ: "select x.col1, x.col2, weight_string(x.col1) from (select col1, col2 from `user`) as x limit :__upper_limit", expField: `[name:"col1" type:VARCHAR name:"sum(col2)" type:FLOAT64]`, expRow: `[[VARCHAR("a") FLOAT64(8)]]`, }, { sql: `select col1, sum(col2) from (select col1, col2 from user limit 4) x group by col1`, sandboxRes: sqltypes.MakeTestResult(sqltypes.MakeTestFields("col1|col2|weight_string(col1)", "varchar|varchar|varbinary"), "a|x|a"), - expSandboxQ: "select col1, col2, weight_string(col1) from (select col1, col2 from `user`) as x limit :__upper_limit", + expSandboxQ: "select x.col1, x.col2, weight_string(x.col1) from (select col1, col2 from `user`) as x limit :__upper_limit", expField: `[name:"col1" type:VARCHAR name:"sum(col2)" type:FLOAT64]`, expRow: `[[VARCHAR("a") FLOAT64(0)]]`, }, { sql: `select col1, sum(col2) from (select col1, col2 from user limit 4) x group by col1`, sandboxRes: sqltypes.MakeTestResult(sqltypes.MakeTestFields("col1|col2|weight_string(col1)", "varchar|varchar|varbinary"), "a|null|a"), - expSandboxQ: "select col1, col2, weight_string(col1) from (select col1, col2 from `user`) as x limit :__upper_limit", + expSandboxQ: "select x.col1, x.col2, weight_string(x.col1) from (select col1, col2 from `user`) as x limit :__upper_limit", expField: `[name:"col1" type:VARCHAR name:"sum(col2)" type:FLOAT64]`, expRow: `[[VARCHAR("a") NULL]]`, }, From d884fec8ba5ba016ebaf03b983fe63d2605ccf1c Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Wed, 16 Oct 2024 09:39:33 +0200 Subject: [PATCH 3/5] Empty commit to trigger CI Signed-off-by: Andres Taylor From 7be439eff960a99779511379841f75f1a23e6391 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Wed, 16 Oct 2024 16:01:31 +0200 Subject: [PATCH 4/5] bugfix: don't add columns to derived table definition Signed-off-by: Andres Taylor --- .../planbuilder/operators/query_planning.go | 5 ---- .../planbuilder/testdata/from_cases.json | 12 +++++----- .../planbuilder/testdata/tpch_cases.json | 24 +++++++++---------- 3 files changed, 18 insertions(+), 23 deletions(-) diff --git a/go/vt/vtgate/planbuilder/operators/query_planning.go b/go/vt/vtgate/planbuilder/operators/query_planning.go index adff05e55cb..b531c23b877 100644 --- a/go/vt/vtgate/planbuilder/operators/query_planning.go +++ b/go/vt/vtgate/planbuilder/operators/query_planning.go @@ -22,8 +22,6 @@ import ( "slices" "strconv" - "vitess.io/vitess/go/slice" - "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vtgate/planbuilder/operators/ops" "vitess.io/vitess/go/vt/vtgate/planbuilder/operators/rewrite" @@ -560,9 +558,6 @@ func createProjectionWithTheseColumns( proj.Columns = AliasedProjections(p.columns) if dt != nil { kopy := *dt - kopy.Columns = slice.Map(p.columnAliases, func(s string) sqlparser.IdentifierCI { - return sqlparser.NewIdentifierCI(s) - }) proj.DT = &kopy } diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.json b/go/vt/vtgate/planbuilder/testdata/from_cases.json index b5efdb18eae..9e668bd68a2 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.json @@ -1989,8 +1989,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select t.id, t.`user.col` from (select `user`.id, `user`.col1, `user`.col as `user.col` from `user` where 1 != 1) as t(`user.col`) where 1 != 1", - "Query": "select t.id, t.`user.col` from (select `user`.id, `user`.col1, `user`.col as `user.col` from `user`) as t(`user.col`)", + "FieldQuery": "select t.id, t.`user.col` from (select `user`.id, `user`.col1, `user`.col as `user.col` from `user` where 1 != 1) as t where 1 != 1", + "Query": "select t.id, t.`user.col` from (select `user`.id, `user`.col1, `user`.col as `user.col` from `user`) as t", "Table": "`user`" }, { @@ -2000,8 +2000,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select 1 from (select user_extra.col as `user_extra.col` from user_extra where 1 != 1) as t(`user_extra.col`) where 1 != 1", - "Query": "select 1 from (select user_extra.col as `user_extra.col` from user_extra where user_extra.col = :user_col) as t(`user_extra.col`)", + "FieldQuery": "select 1 from (select user_extra.col as `user_extra.col` from user_extra where 1 != 1) as t where 1 != 1", + "Query": "select 1 from (select user_extra.col as `user_extra.col` from user_extra where user_extra.col = :user_col) as t", "Table": "user_extra" } ] @@ -3187,8 +3187,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select i + 1 from (select `user`.id from `user` where 1 != 1) as t where 1 != 1", - "Query": "select i + 1 from (select `user`.id from `user`) as t", + "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`" }, { diff --git a/go/vt/vtgate/planbuilder/testdata/tpch_cases.json b/go/vt/vtgate/planbuilder/testdata/tpch_cases.json index 107b26cc055..f1d0ba7dfcc 100644 --- a/go/vt/vtgate/planbuilder/testdata/tpch_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/tpch_cases.json @@ -588,9 +588,9 @@ "Name": "main", "Sharded": true }, - "FieldQuery": "select sum(volume) as revenue, l_year, shipping.l_suppkey, shipping.l_orderkey, weight_string(l_year), shipping.supp_nation, weight_string(shipping.supp_nation), shipping.cust_nation, weight_string(shipping.cust_nation) from (select extract(year from l_shipdate) as l_year, l_extendedprice * (1 - l_discount) as volume, l_suppkey as l_suppkey, l_orderkey as l_orderkey from lineitem where 1 != 1) as shipping(l_orderkey) where 1 != 1 group by l_year, shipping.l_suppkey, shipping.l_orderkey, weight_string(l_year)", + "FieldQuery": "select sum(volume) as revenue, l_year, shipping.l_suppkey, shipping.l_orderkey, weight_string(l_year), shipping.supp_nation, weight_string(shipping.supp_nation), shipping.cust_nation, weight_string(shipping.cust_nation) from (select extract(year from l_shipdate) as l_year, l_extendedprice * (1 - l_discount) as volume, l_suppkey as l_suppkey, l_orderkey as l_orderkey from lineitem where 1 != 1) as shipping where 1 != 1 group by l_year, shipping.l_suppkey, shipping.l_orderkey, weight_string(l_year)", "OrderBy": "(5|6) ASC, (7|8) ASC, (1|4) ASC", - "Query": "select sum(volume) as revenue, l_year, shipping.l_suppkey, shipping.l_orderkey, weight_string(l_year), shipping.supp_nation, weight_string(shipping.supp_nation), shipping.cust_nation, weight_string(shipping.cust_nation) from (select extract(year from l_shipdate) as l_year, l_extendedprice * (1 - l_discount) as volume, l_suppkey as l_suppkey, l_orderkey as l_orderkey from lineitem where l_shipdate between date('1995-01-01') and date('1996-12-31')) as shipping(l_orderkey) group by l_year, shipping.l_suppkey, shipping.l_orderkey, weight_string(l_year) order by shipping.supp_nation asc, shipping.cust_nation asc, shipping.l_year asc", + "Query": "select sum(volume) as revenue, l_year, shipping.l_suppkey, shipping.l_orderkey, weight_string(l_year), shipping.supp_nation, weight_string(shipping.supp_nation), shipping.cust_nation, weight_string(shipping.cust_nation) from (select extract(year from l_shipdate) as l_year, l_extendedprice * (1 - l_discount) as volume, l_suppkey as l_suppkey, l_orderkey as l_orderkey from lineitem where l_shipdate between date('1995-01-01') and date('1996-12-31')) as shipping group by l_year, shipping.l_suppkey, shipping.l_orderkey, weight_string(l_year) order by shipping.supp_nation asc, shipping.cust_nation asc, shipping.l_year asc", "Table": "lineitem" }, { @@ -600,8 +600,8 @@ "Name": "main", "Sharded": true }, - "FieldQuery": "select count(*), shipping.o_custkey from (select o_custkey as o_custkey, o_orderkey as o_orderkey from orders where 1 != 1) as shipping(o_orderkey) where 1 != 1 group by shipping.o_custkey", - "Query": "select count(*), shipping.o_custkey from (select o_custkey as o_custkey, o_orderkey as o_orderkey from orders where o_orderkey = :l_orderkey) as shipping(o_orderkey) group by shipping.o_custkey", + "FieldQuery": "select count(*), shipping.o_custkey from (select o_custkey as o_custkey, o_orderkey as o_orderkey from orders where 1 != 1) as shipping where 1 != 1 group by shipping.o_custkey", + "Query": "select count(*), shipping.o_custkey from (select o_custkey as o_custkey, o_orderkey as o_orderkey from orders where o_orderkey = :l_orderkey) as shipping group by shipping.o_custkey", "Table": "orders", "Values": [ ":l_orderkey" @@ -636,8 +636,8 @@ "Name": "main", "Sharded": true }, - "FieldQuery": "select count(*), shipping.s_nationkey from (select s_suppkey as s_suppkey, s_nationkey as s_nationkey from supplier where 1 != 1) as shipping(s_nationkey) where 1 != 1 group by shipping.s_nationkey", - "Query": "select count(*), shipping.s_nationkey from (select s_suppkey as s_suppkey, s_nationkey as s_nationkey from supplier where s_suppkey = :l_suppkey) as shipping(s_nationkey) group by shipping.s_nationkey", + "FieldQuery": "select count(*), shipping.s_nationkey from (select s_suppkey as s_suppkey, s_nationkey as s_nationkey from supplier where 1 != 1) as shipping where 1 != 1 group by shipping.s_nationkey", + "Query": "select count(*), shipping.s_nationkey from (select s_suppkey as s_suppkey, s_nationkey as s_nationkey from supplier where s_suppkey = :l_suppkey) as shipping group by shipping.s_nationkey", "Table": "supplier", "Values": [ ":l_suppkey" @@ -651,8 +651,8 @@ "Name": "main", "Sharded": true }, - "FieldQuery": "select count(*), supp_nation, weight_string(supp_nation), n1.n_nationkey as `n1.n_nationkey`, shipping.supp_nation as `shipping.supp_nation`, shipping.`n1.n_name = 'FRANCE'` as `shipping.n1.n_name = 'FRANCE'`, shipping.`n1.n_name = 'GERMANY'` as `shipping.n1.n_name = 'GERMANY'` from (select n1.n_name as supp_nation, n1.n_name = 'FRANCE' as `n1.n_name = 'FRANCE'`, n1.n_name = 'GERMANY' as `n1.n_name = 'GERMANY'`, n1.n_nationkey as `n1.n_nationkey`, shipping.supp_nation as `shipping.supp_nation`, shipping.`n1.n_name = 'FRANCE'` as `shipping.n1.n_name = 'FRANCE'`, shipping.`n1.n_name = 'GERMANY'` as `shipping.n1.n_name = 'GERMANY'` from nation as n1 where 1 != 1) as shipping(`n1.n_nationkey`, `shipping.supp_nation`, `shipping.n1.n_name = 'FRANCE'`, `shipping.n1.n_name = 'GERMANY'`) where 1 != 1 group by supp_nation, weight_string(supp_nation)", - "Query": "select count(*), supp_nation, weight_string(supp_nation), n1.n_nationkey as `n1.n_nationkey`, shipping.supp_nation as `shipping.supp_nation`, shipping.`n1.n_name = 'FRANCE'` as `shipping.n1.n_name = 'FRANCE'`, shipping.`n1.n_name = 'GERMANY'` as `shipping.n1.n_name = 'GERMANY'` from (select n1.n_name as supp_nation, n1.n_name = 'FRANCE' as `n1.n_name = 'FRANCE'`, n1.n_name = 'GERMANY' as `n1.n_name = 'GERMANY'`, n1.n_nationkey as `n1.n_nationkey`, shipping.supp_nation as `shipping.supp_nation`, shipping.`n1.n_name = 'FRANCE'` as `shipping.n1.n_name = 'FRANCE'`, shipping.`n1.n_name = 'GERMANY'` as `shipping.n1.n_name = 'GERMANY'` from nation as n1 where n1.n_nationkey = :s_nationkey) as shipping(`n1.n_nationkey`, `shipping.supp_nation`, `shipping.n1.n_name = 'FRANCE'`, `shipping.n1.n_name = 'GERMANY'`) group by supp_nation, weight_string(supp_nation)", + "FieldQuery": "select count(*), supp_nation, weight_string(supp_nation), n1.n_nationkey as `n1.n_nationkey`, shipping.supp_nation as `shipping.supp_nation`, shipping.`n1.n_name = 'FRANCE'` as `shipping.n1.n_name = 'FRANCE'`, shipping.`n1.n_name = 'GERMANY'` as `shipping.n1.n_name = 'GERMANY'` from (select n1.n_name as supp_nation, n1.n_name = 'FRANCE' as `n1.n_name = 'FRANCE'`, n1.n_name = 'GERMANY' as `n1.n_name = 'GERMANY'`, n1.n_nationkey as `n1.n_nationkey`, shipping.supp_nation as `shipping.supp_nation`, shipping.`n1.n_name = 'FRANCE'` as `shipping.n1.n_name = 'FRANCE'`, shipping.`n1.n_name = 'GERMANY'` as `shipping.n1.n_name = 'GERMANY'` from nation as n1 where 1 != 1) as shipping where 1 != 1 group by supp_nation, weight_string(supp_nation)", + "Query": "select count(*), supp_nation, weight_string(supp_nation), n1.n_nationkey as `n1.n_nationkey`, shipping.supp_nation as `shipping.supp_nation`, shipping.`n1.n_name = 'FRANCE'` as `shipping.n1.n_name = 'FRANCE'`, shipping.`n1.n_name = 'GERMANY'` as `shipping.n1.n_name = 'GERMANY'` from (select n1.n_name as supp_nation, n1.n_name = 'FRANCE' as `n1.n_name = 'FRANCE'`, n1.n_name = 'GERMANY' as `n1.n_name = 'GERMANY'`, n1.n_nationkey as `n1.n_nationkey`, shipping.supp_nation as `shipping.supp_nation`, shipping.`n1.n_name = 'FRANCE'` as `shipping.n1.n_name = 'FRANCE'`, shipping.`n1.n_name = 'GERMANY'` as `shipping.n1.n_name = 'GERMANY'` from nation as n1 where n1.n_nationkey = :s_nationkey) as shipping group by supp_nation, weight_string(supp_nation)", "Table": "nation", "Values": [ ":s_nationkey" @@ -691,8 +691,8 @@ "Name": "main", "Sharded": true }, - "FieldQuery": "select count(*), shipping.c_nationkey from (select c_custkey as c_custkey, c_nationkey as c_nationkey from customer where 1 != 1) as shipping(c_nationkey) where 1 != 1 group by shipping.c_nationkey", - "Query": "select count(*), shipping.c_nationkey from (select c_custkey as c_custkey, c_nationkey as c_nationkey from customer where c_custkey = :o_custkey) as shipping(c_nationkey) group by shipping.c_nationkey", + "FieldQuery": "select count(*), shipping.c_nationkey from (select c_custkey as c_custkey, c_nationkey as c_nationkey from customer where 1 != 1) as shipping where 1 != 1 group by shipping.c_nationkey", + "Query": "select count(*), shipping.c_nationkey from (select c_custkey as c_custkey, c_nationkey as c_nationkey from customer where c_custkey = :o_custkey) as shipping group by shipping.c_nationkey", "Table": "customer", "Values": [ ":o_custkey" @@ -706,8 +706,8 @@ "Name": "main", "Sharded": true }, - "FieldQuery": "select count(*), cust_nation, weight_string(cust_nation), n2.n_nationkey as `n2.n_nationkey`, shipping.cust_nation as `shipping.cust_nation` from (select n2.n_name as cust_nation, n2.n_name = 'GERMANY' as `n2.n_name = 'GERMANY'`, n2.n_name = 'FRANCE' as `n2.n_name = 'FRANCE'`, n2.n_nationkey as `n2.n_nationkey`, shipping.cust_nation as `shipping.cust_nation` from nation as n2 where 1 != 1) as shipping(`n2.n_nationkey`, `shipping.cust_nation`) where 1 != 1 group by cust_nation, weight_string(cust_nation)", - "Query": "select count(*), cust_nation, weight_string(cust_nation), n2.n_nationkey as `n2.n_nationkey`, shipping.cust_nation as `shipping.cust_nation` from (select n2.n_name as cust_nation, n2.n_name = 'GERMANY' as `n2.n_name = 'GERMANY'`, n2.n_name = 'FRANCE' as `n2.n_name = 'FRANCE'`, n2.n_nationkey as `n2.n_nationkey`, shipping.cust_nation as `shipping.cust_nation` from nation as n2 where (:n1_n_name = 'FRANCE' and n2.n_name = 'GERMANY' or :n1_n_name = 'GERMANY' and n2.n_name = 'FRANCE') and n2.n_nationkey = :c_nationkey) as shipping(`n2.n_nationkey`, `shipping.cust_nation`) group by cust_nation, weight_string(cust_nation)", + "FieldQuery": "select count(*), cust_nation, weight_string(cust_nation), n2.n_nationkey as `n2.n_nationkey`, shipping.cust_nation as `shipping.cust_nation` from (select n2.n_name as cust_nation, n2.n_name = 'GERMANY' as `n2.n_name = 'GERMANY'`, n2.n_name = 'FRANCE' as `n2.n_name = 'FRANCE'`, n2.n_nationkey as `n2.n_nationkey`, shipping.cust_nation as `shipping.cust_nation` from nation as n2 where 1 != 1) as shipping where 1 != 1 group by cust_nation, weight_string(cust_nation)", + "Query": "select count(*), cust_nation, weight_string(cust_nation), n2.n_nationkey as `n2.n_nationkey`, shipping.cust_nation as `shipping.cust_nation` from (select n2.n_name as cust_nation, n2.n_name = 'GERMANY' as `n2.n_name = 'GERMANY'`, n2.n_name = 'FRANCE' as `n2.n_name = 'FRANCE'`, n2.n_nationkey as `n2.n_nationkey`, shipping.cust_nation as `shipping.cust_nation` from nation as n2 where (:n1_n_name = 'FRANCE' and n2.n_name = 'GERMANY' or :n1_n_name = 'GERMANY' and n2.n_name = 'FRANCE') and n2.n_nationkey = :c_nationkey) as shipping group by cust_nation, weight_string(cust_nation)", "Table": "nation", "Values": [ ":c_nationkey" From 9ec7b0a13d52fd9b58343428c3e0b20ba4b87c68 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Wed, 16 Oct 2024 17:53:07 +0200 Subject: [PATCH 5/5] test: update test expectations Signed-off-by: Andres Taylor --- go/vt/vtgate/executor_select_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/go/vt/vtgate/executor_select_test.go b/go/vt/vtgate/executor_select_test.go index 011dd9d21aa..0130843cbb1 100644 --- a/go/vt/vtgate/executor_select_test.go +++ b/go/vt/vtgate/executor_select_test.go @@ -2815,7 +2815,7 @@ func TestCrossShardSubquery(t *testing.T) { result, err := executorExec(ctx, executor, session, "select id1 from (select u1.id id1, u2.id from user u1 join user u2 on u2.id = u1.col where u1.id = 1) as t", nil) require.NoError(t, err) wantQueries := []*querypb.BoundQuery{{ - Sql: "select id1, t.`u1.col` from (select u1.id as id1, u1.col as `u1.col` from `user` as u1 where u1.id = 1) as t(`u1.col`)", + Sql: "select id1, t.`u1.col` from (select u1.id as id1, u1.col as `u1.col` from `user` as u1 where u1.id = 1) as t", BindVariables: map[string]*querypb.BindVariable{}, }} utils.MustMatch(t, wantQueries, sbc1.Queries) @@ -2891,7 +2891,7 @@ func TestCrossShardSubqueryStream(t *testing.T) { result, err := executorStream(ctx, executor, "select id1 from (select u1.id id1, u2.id from user u1 join user u2 on u2.id = u1.col where u1.id = 1) as t") require.NoError(t, err) wantQueries := []*querypb.BoundQuery{{ - Sql: "select id1, t.`u1.col` from (select u1.id as id1, u1.col as `u1.col` from `user` as u1 where u1.id = 1) as t(`u1.col`)", + Sql: "select id1, t.`u1.col` from (select u1.id as id1, u1.col as `u1.col` from `user` as u1 where u1.id = 1) as t", BindVariables: map[string]*querypb.BindVariable{}, }} utils.MustMatch(t, wantQueries, sbc1.Queries) @@ -2934,7 +2934,7 @@ func TestCrossShardSubqueryGetFields(t *testing.T) { result, err := executorExec(ctx, executor, session, "select main1.col, t.id1 from main1 join (select u1.id id1, u2.id from user u1 join user u2 on u2.id = u1.col where u1.id = 1) as t", nil) require.NoError(t, err) wantQueries := []*querypb.BoundQuery{{ - Sql: "select t.id1, t.`u1.col` from (select u1.id as id1, u1.col as `u1.col` from `user` as u1 where 1 != 1) as t(`u1.col`) where 1 != 1", + Sql: "select t.id1, t.`u1.col` from (select u1.id as id1, u1.col as `u1.col` from `user` as u1 where 1 != 1) as t where 1 != 1", BindVariables: map[string]*querypb.BindVariable{}, }, { Sql: "select 1 from (select u2.id as id from `user` as u2 where 1 != 1) as t where 1 != 1",