diff --git a/go/vt/vtgate/planbuilder/operators/aggregator.go b/go/vt/vtgate/planbuilder/operators/aggregator.go index 5c9ba167171..53525a025a4 100644 --- a/go/vt/vtgate/planbuilder/operators/aggregator.go +++ b/go/vt/vtgate/planbuilder/operators/aggregator.go @@ -190,6 +190,64 @@ func (a *Aggregator) AddColumn(ctx *plancontext.PlanningContext, reuse bool, gro return offset } +func (a *Aggregator) AddWSColumn(ctx *plancontext.PlanningContext, offset int, underRoute bool) int { + if len(a.Columns) <= offset { + panic(vterrors.VT13001("offset out of range")) + } + + var expr sqlparser.Expr + // first search for the offset among the groupings + for i, by := range a.Grouping { + if by.ColOffset != offset { + continue + } + if by.WSOffset >= 0 { + // ah, we already have a weigh_string for this column. let's return it as is + return by.WSOffset + } + + // we need to add a WS column + a.Grouping[i].WSOffset = len(a.Columns) + expr = a.Columns[offset].Expr + break + } + + if expr == nil { + for _, aggr := range a.Aggregations { + if aggr.ColOffset != offset { + continue + } + if aggr.WSOffset >= 0 { + // ah, we already have a weigh_string for this column. let's return it as is + return aggr.WSOffset + } + + panic(vterrors.VT13001("expected to find a weight string for aggregation")) + } + + panic(vterrors.VT13001("could not find expression at offset")) + } + + wsExpr := weightStringFor(expr) + wsAe := aeWrap(wsExpr) + + wsOffset := len(a.Columns) + a.Columns = append(a.Columns, wsAe) + if underRoute { + // if we are under a route, we are done here. + // the column will be use when creating the query to send to the tablet, and that is all we need + return wsOffset + } + + incomingOffset := a.Source.AddWSColumn(ctx, offset, false) + + if wsOffset != incomingOffset { + // TODO: we could handle this case by adding a projection on under the aggregator to make the columns line up + panic(errFailedToPlan(wsAe)) + } + return wsOffset +} + func (a *Aggregator) findColInternal(ctx *plancontext.PlanningContext, ae *sqlparser.AliasedExpr, addToGroupBy bool) int { expr := ae.Expr offset := a.FindCol(ctx, expr, false) @@ -211,8 +269,19 @@ func (a *Aggregator) findColInternal(ctx *plancontext.PlanningContext, ae *sqlpa return -1 } +func isDerived(op Operator) bool { + switch op := op.(type) { + case *Horizon: + return op.IsDerived() + case selectExpressions: + return op.derivedName() != "" + default: + return false + } +} + func (a *Aggregator) GetColumns(ctx *plancontext.PlanningContext) []*sqlparser.AliasedExpr { - if _, isSourceDerived := a.Source.(*Horizon); isSourceDerived { + if isDerived(a.Source) { return a.Columns } @@ -278,6 +347,7 @@ func (a *Aggregator) planOffsets(ctx *plancontext.PlanningContext) Operator { if gb.ColOffset == -1 { offset := a.internalAddColumn(ctx, aeWrap(gb.Inner), false) a.Grouping[idx].ColOffset = offset + gb.ColOffset = offset } if gb.WSOffset != -1 || !ctx.SemTable.NeedsWeightString(gb.Inner) { continue diff --git a/go/vt/vtgate/planbuilder/operators/apply_join.go b/go/vt/vtgate/planbuilder/operators/apply_join.go index 043e76b5e15..95e806c4695 100644 --- a/go/vt/vtgate/planbuilder/operators/apply_join.go +++ b/go/vt/vtgate/planbuilder/operators/apply_join.go @@ -237,22 +237,52 @@ func (aj *ApplyJoin) AddColumn( return offset } +func (aj *ApplyJoin) AddWSColumn(ctx *plancontext.PlanningContext, offset int, underRoute bool) int { + if len(aj.Columns) == 0 { + aj.planOffsets(ctx) + } + + if len(aj.Columns) <= offset { + panic(vterrors.VT13001("offset out of range")) + } + + wsExpr := weightStringFor(aj.JoinColumns.columns[offset].Original) + if index := aj.FindCol(ctx, wsExpr, false); index != -1 { + // nice, we already have this column. no need to add anything + return index + } + + i := aj.Columns[offset] + out := 0 + if i < 0 { + out = aj.LHS.AddWSColumn(ctx, FromLeftOffset(i), underRoute) + out = ToLeftOffset(out) + aj.JoinColumns.addLeft(wsExpr) + } else { + out = aj.RHS.AddWSColumn(ctx, FromRightOffset(i), underRoute) + out = ToRightOffset(out) + aj.JoinColumns.addRight(wsExpr) + } + + if out >= 0 { + aj.addOffset(out) + } else { + col := aj.getJoinColumnFor(ctx, aeWrap(wsExpr), wsExpr, !ContainsAggr(ctx, wsExpr)) + aj.JoinColumns.add(col) + aj.planOffsetFor(ctx, col) + } + + return len(aj.Columns) - 1 +} + func (aj *ApplyJoin) planOffsets(ctx *plancontext.PlanningContext) Operator { + if len(aj.Columns) > 0 { + // we've already done offset planning + return aj + } for _, col := range aj.JoinColumns.columns { // Read the type description for applyJoinColumn to understand the following code - for _, lhsExpr := range col.LHSExprs { - offset := aj.LHS.AddColumn(ctx, true, col.GroupBy, aeWrap(lhsExpr.Expr)) - if col.RHSExpr == nil { - // if we don't have an RHS expr, it means that this is a pure LHS expression - aj.addOffset(-offset - 1) - } else { - aj.Vars[lhsExpr.Name] = offset - } - } - if col.RHSExpr != nil { - offset := aj.RHS.AddColumn(ctx, true, col.GroupBy, aeWrap(col.RHSExpr)) - aj.addOffset(offset + 1) - } + aj.planOffsetFor(ctx, col) } for _, col := range aj.JoinPredicates.columns { @@ -270,6 +300,22 @@ func (aj *ApplyJoin) planOffsets(ctx *plancontext.PlanningContext) Operator { return nil } +func (aj *ApplyJoin) planOffsetFor(ctx *plancontext.PlanningContext, col applyJoinColumn) { + for _, lhsExpr := range col.LHSExprs { + offset := aj.LHS.AddColumn(ctx, true, col.GroupBy, aeWrap(lhsExpr.Expr)) + if col.RHSExpr == nil { + // if we don't have an RHS expr, it means that this is a pure LHS expression + aj.addOffset(ToLeftOffset(offset)) + } else { + aj.Vars[lhsExpr.Name] = offset + } + } + if col.RHSExpr != nil { + offset := aj.RHS.AddColumn(ctx, true, col.GroupBy, aeWrap(col.RHSExpr)) + aj.addOffset(ToRightOffset(offset)) + } +} + func (aj *ApplyJoin) addOffset(offset int) { aj.Columns = append(aj.Columns, offset) } diff --git a/go/vt/vtgate/planbuilder/operators/comments.go b/go/vt/vtgate/planbuilder/operators/comments.go index 912fa4138d9..7e7749a61b5 100644 --- a/go/vt/vtgate/planbuilder/operators/comments.go +++ b/go/vt/vtgate/planbuilder/operators/comments.go @@ -54,6 +54,10 @@ func (l *LockAndComment) AddColumn(ctx *plancontext.PlanningContext, reuseExisti return l.Source.AddColumn(ctx, reuseExisting, addToGroupBy, expr) } +func (l *LockAndComment) AddWSColumn(ctx *plancontext.PlanningContext, offset int, underRoute bool) int { + return l.Source.AddWSColumn(ctx, offset, underRoute) +} + func (l *LockAndComment) FindCol(ctx *plancontext.PlanningContext, expr sqlparser.Expr, underRoute bool) int { return l.Source.FindCol(ctx, expr, underRoute) } diff --git a/go/vt/vtgate/planbuilder/operators/distinct.go b/go/vt/vtgate/planbuilder/operators/distinct.go index eeddd928f66..9c893a878cd 100644 --- a/go/vt/vtgate/planbuilder/operators/distinct.go +++ b/go/vt/vtgate/planbuilder/operators/distinct.go @@ -50,13 +50,11 @@ func (d *Distinct) planOffsets(ctx *plancontext.PlanningContext) Operator { for idx, col := range columns { e := col.Expr var wsCol *int - typ, _ := ctx.SemTable.TypeForExpr(e) - if ctx.SemTable.NeedsWeightString(e) { - offset := d.Source.AddColumn(ctx, true, false, aeWrap(weightStringFor(e))) + offset := d.Source.AddWSColumn(ctx, idx, false) wsCol = &offset } - + typ, _ := ctx.SemTable.TypeForExpr(e) d.Columns = append(d.Columns, engine.CheckCol{ Col: idx, WsCol: wsCol, @@ -94,6 +92,9 @@ func (d *Distinct) AddPredicate(ctx *plancontext.PlanningContext, expr sqlparser func (d *Distinct) AddColumn(ctx *plancontext.PlanningContext, reuse bool, gb bool, expr *sqlparser.AliasedExpr) int { return d.Source.AddColumn(ctx, reuse, gb, expr) } +func (d *Distinct) AddWSColumn(ctx *plancontext.PlanningContext, offset int, underRoute bool) int { + return d.Source.AddWSColumn(ctx, offset, underRoute) +} func (d *Distinct) FindCol(ctx *plancontext.PlanningContext, expr sqlparser.Expr, underRoute bool) int { return d.Source.FindCol(ctx, expr, underRoute) diff --git a/go/vt/vtgate/planbuilder/operators/filter.go b/go/vt/vtgate/planbuilder/operators/filter.go index c2432a40da9..babc309db72 100644 --- a/go/vt/vtgate/planbuilder/operators/filter.go +++ b/go/vt/vtgate/planbuilder/operators/filter.go @@ -95,6 +95,10 @@ func (f *Filter) FindCol(ctx *plancontext.PlanningContext, expr sqlparser.Expr, return f.Source.FindCol(ctx, expr, underRoute) } +func (f *Filter) AddWSColumn(ctx *plancontext.PlanningContext, offset int, underRoute bool) int { + return f.Source.AddWSColumn(ctx, offset, underRoute) +} + func (f *Filter) GetColumns(ctx *plancontext.PlanningContext) []*sqlparser.AliasedExpr { return f.Source.GetColumns(ctx) } diff --git a/go/vt/vtgate/planbuilder/operators/hash_join.go b/go/vt/vtgate/planbuilder/operators/hash_join.go index da6a63200db..d2ba6522691 100644 --- a/go/vt/vtgate/planbuilder/operators/hash_join.go +++ b/go/vt/vtgate/planbuilder/operators/hash_join.go @@ -121,6 +121,32 @@ func (hj *HashJoin) AddColumn(ctx *plancontext.PlanningContext, reuseExisting bo return len(hj.columns.columns) - 1 } +func (hj *HashJoin) AddWSColumn(ctx *plancontext.PlanningContext, offset int, underRoute bool) int { + hj.planOffsets(ctx) + + if len(hj.ColumnOffsets) <= offset { + panic(vterrors.VT13001("offset out of range")) + } + + // check if it already exists + wsExpr := weightStringFor(hj.columns.columns[offset].expr) + if index := hj.FindCol(ctx, wsExpr, false); index != -1 { + return index + } + + i := hj.ColumnOffsets[offset] + out := 0 + if i < 0 { + out = hj.LHS.AddWSColumn(ctx, FromLeftOffset(i), underRoute) + out = ToLeftOffset(out) + } else { + out = hj.RHS.AddWSColumn(ctx, FromRightOffset(i), underRoute) + out = ToRightOffset(out) + } + hj.ColumnOffsets = append(hj.ColumnOffsets, out) + return len(hj.ColumnOffsets) - 1 +} + func (hj *HashJoin) planOffsets(ctx *plancontext.PlanningContext) Operator { if hj.offset { return nil @@ -450,3 +476,19 @@ func (hj *HashJoin) addSingleSidedColumn( Info: &EvalEngine{EExpr: eexpr}, }, isPureOffset } + +func FromLeftOffset(i int) int { + return -i - 1 +} + +func ToLeftOffset(i int) int { + return -i - 1 +} + +func FromRightOffset(i int) int { + return i - 1 +} + +func ToRightOffset(i int) int { + return i + 1 +} diff --git a/go/vt/vtgate/planbuilder/operators/horizon.go b/go/vt/vtgate/planbuilder/operators/horizon.go index 4a4c990b1ed..30cce2617f8 100644 --- a/go/vt/vtgate/planbuilder/operators/horizon.go +++ b/go/vt/vtgate/planbuilder/operators/horizon.go @@ -122,6 +122,10 @@ func (h *Horizon) AddColumn(ctx *plancontext.PlanningContext, reuse bool, _ bool return offset } +func (h *Horizon) AddWSColumn(ctx *plancontext.PlanningContext, offset int, underRoute bool) int { + panic(errNoNewColumns) +} + var errNoNewColumns = vterrors.VT13001("can't add new columns to Horizon") // canReuseColumn is generic, so it can be used with slices of different types. diff --git a/go/vt/vtgate/planbuilder/operators/limit.go b/go/vt/vtgate/planbuilder/operators/limit.go index 152872ff8ff..9d0710d99ae 100644 --- a/go/vt/vtgate/planbuilder/operators/limit.go +++ b/go/vt/vtgate/planbuilder/operators/limit.go @@ -58,6 +58,10 @@ func (l *Limit) AddColumn(ctx *plancontext.PlanningContext, reuse bool, gb bool, return l.Source.AddColumn(ctx, reuse, gb, expr) } +func (l *Limit) AddWSColumn(ctx *plancontext.PlanningContext, offset int, underRoute bool) int { + return l.Source.AddWSColumn(ctx, offset, underRoute) +} + func (l *Limit) FindCol(ctx *plancontext.PlanningContext, expr sqlparser.Expr, underRoute bool) int { return l.Source.FindCol(ctx, expr, underRoute) } diff --git a/go/vt/vtgate/planbuilder/operators/offset_planning.go b/go/vt/vtgate/planbuilder/operators/offset_planning.go index 4204b6e0420..eb92cdf0920 100644 --- a/go/vt/vtgate/planbuilder/operators/offset_planning.go +++ b/go/vt/vtgate/planbuilder/operators/offset_planning.go @@ -19,10 +19,9 @@ package operators import ( "fmt" - "vitess.io/vitess/go/vt/vtgate/engine/opcode" - "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" + "vitess.io/vitess/go/vt/vtgate/engine/opcode" "vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext" "vitess.io/vitess/go/vt/vtgate/semantics" ) diff --git a/go/vt/vtgate/planbuilder/operators/operator.go b/go/vt/vtgate/planbuilder/operators/operator.go index d639643dda1..f1a38974c93 100644 --- a/go/vt/vtgate/planbuilder/operators/operator.go +++ b/go/vt/vtgate/planbuilder/operators/operator.go @@ -65,6 +65,9 @@ type ( AddColumn(ctx *plancontext.PlanningContext, reuseExisting bool, addToGroupBy bool, expr *sqlparser.AliasedExpr) int + // AddWSColumn is used to add a weight_string column to the operator + AddWSColumn(ctx *plancontext.PlanningContext, offset int, underRoute bool) int + FindCol(ctx *plancontext.PlanningContext, expr sqlparser.Expr, underRoute bool) int GetColumns(ctx *plancontext.PlanningContext) []*sqlparser.AliasedExpr diff --git a/go/vt/vtgate/planbuilder/operators/ordering.go b/go/vt/vtgate/planbuilder/operators/ordering.go index bc088ca2220..f8008022511 100644 --- a/go/vt/vtgate/planbuilder/operators/ordering.go +++ b/go/vt/vtgate/planbuilder/operators/ordering.go @@ -61,6 +61,10 @@ func (o *Ordering) AddColumn(ctx *plancontext.PlanningContext, reuse bool, gb bo return o.Source.AddColumn(ctx, reuse, gb, expr) } +func (o *Ordering) AddWSColumn(ctx *plancontext.PlanningContext, offset int, underRoute bool) int { + return o.Source.AddWSColumn(ctx, offset, underRoute) +} + func (o *Ordering) FindCol(ctx *plancontext.PlanningContext, expr sqlparser.Expr, underRoute bool) int { return o.Source.FindCol(ctx, expr, underRoute) } diff --git a/go/vt/vtgate/planbuilder/operators/phases.go b/go/vt/vtgate/planbuilder/operators/phases.go index 60e937a5b92..7bf414e9d16 100644 --- a/go/vt/vtgate/planbuilder/operators/phases.go +++ b/go/vt/vtgate/planbuilder/operators/phases.go @@ -50,7 +50,7 @@ func (p Phase) String() string { case initialPlanning: return "initial horizon planning optimization" case pullDistinctFromUnion: - return "pull distinct from UNION1" + return "pull distinct from UNION" case delegateAggregation: return "split aggregation between vtgate and mysql" case addAggrOrdering: diff --git a/go/vt/vtgate/planbuilder/operators/plan_query.go b/go/vt/vtgate/planbuilder/operators/plan_query.go index d5794a1bcf1..ea6b88f752d 100644 --- a/go/vt/vtgate/planbuilder/operators/plan_query.go +++ b/go/vt/vtgate/planbuilder/operators/plan_query.go @@ -110,6 +110,10 @@ func (noColumns) AddColumn(*plancontext.PlanningContext, bool, bool, *sqlparser. panic(vterrors.VT13001("noColumns operators have no column")) } +func (noColumns) AddWSColumn(ctx *plancontext.PlanningContext, offset int, underRoute bool) int { + panic(vterrors.VT13001("noColumns operators have no column")) +} + func (noColumns) GetColumns(*plancontext.PlanningContext) []*sqlparser.AliasedExpr { panic(vterrors.VT13001("noColumns operators have no column")) } diff --git a/go/vt/vtgate/planbuilder/operators/projection.go b/go/vt/vtgate/planbuilder/operators/projection.go index 2de282571df..ee333f65f25 100644 --- a/go/vt/vtgate/planbuilder/operators/projection.go +++ b/go/vt/vtgate/planbuilder/operators/projection.go @@ -293,6 +293,40 @@ func (p *Projection) addColumnsWithoutPushing(ctx *plancontext.PlanningContext, return offsets } +func (p *Projection) AddWSColumn(ctx *plancontext.PlanningContext, offset int, underRoute bool) int { + cols, aliased := p.Columns.(AliasedProjections) + if !aliased { + panic(vterrors.VT09015()) + } + + if offset >= len(cols) || offset < 0 { + panic(vterrors.VT13001(fmt.Sprintf("offset [%d] out of range [%d]", offset, len(cols)))) + } + + expr := cols[offset].EvalExpr + ws := weightStringFor(expr) + if offset := p.FindCol(ctx, ws, underRoute); offset >= 0 { + // if we already have this column, we can just return the offset + return offset + } + + aeWs := aeWrap(ws) + pe := newProjExprWithInner(aeWs, ws) + if underRoute { + return p.addProjExpr(pe) + } + + // we need to push down this column to our input + offsetOnInput := p.Source.FindCol(ctx, expr, false) + if offsetOnInput >= 0 { + // if we are not getting this from the source, we can solve this at offset planning time + inputOffset := p.Source.AddWSColumn(ctx, offsetOnInput, false) + pe.Info = Offset(inputOffset) + } + + return p.addProjExpr(pe) +} + func (p *Projection) AddColumn(ctx *plancontext.PlanningContext, reuse bool, addToGroupBy bool, ae *sqlparser.AliasedExpr) int { return p.addColumn(ctx, reuse, addToGroupBy, ae, true) } diff --git a/go/vt/vtgate/planbuilder/operators/projection_pushing.go b/go/vt/vtgate/planbuilder/operators/projection_pushing.go index c1118184e06..5999635a314 100644 --- a/go/vt/vtgate/planbuilder/operators/projection_pushing.go +++ b/go/vt/vtgate/planbuilder/operators/projection_pushing.go @@ -17,11 +17,10 @@ limitations under the License. package operators import ( + "fmt" "slices" - "strconv" "vitess.io/vitess/go/slice" - "vitess.io/vitess/go/test/dbg" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext" @@ -50,8 +49,7 @@ func (p *projector) add(pe *ProjExpr, alias string) { 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 := sqlparser.NewColName(column.Original.ColumnName()) out.Qualifier = p.tableName ctx.SemTable.CopySemanticInfo(expr, out) @@ -73,16 +71,6 @@ func (p *projector) get(ctx *plancontext.PlanningContext, expr sqlparser.Expr) s 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 -} - // tryPushProjection attempts to optimize a projection by pushing it down in the query plan func tryPushProjection( ctx *plancontext.PlanningContext, @@ -204,10 +192,27 @@ func pushProjectionToOuterContainer(ctx *plancontext.PlanningContext, p *Project return src, Rewrote("push projection into outer side of subquery container") } -// pushProjectionInApplyJoin pushes down a projection operation into an ApplyJoin operation. -// It processes each input column and creates new JoinPredicates for the ApplyJoin operation based on -// the input column's expression. It also creates new Projection operators for the left and right -// children of the ApplyJoin operation, if needed. +// pushProjectionInApplyJoin optimizes the ApplyJoin operation by pushing down the projection operation into it. This function works as follows: +// +// 1. It traverses each input column of the projection operation. +// 2. For each column, it generates new JoinPredicates for the ApplyJoin operation. These predicates are derived from the column's expression. +/* +Here's an ASCII representation of the transformation: + Before: + Projection[L.colX, R.colY] + | + ApplyJoin + / \ + LHS RHS + After: + ApplyJoin + / \ + Projection[L.colX] Projection[R.colY] + | | + LHS RHS +*/ +// In the transformed state, if necessary, new Projection operators are created for the left and right children of the ApplyJoin operation. +// These Projections can then hopefully be pushed down under a Route or Limit operation. func pushProjectionInApplyJoin( ctx *plancontext.PlanningContext, p *Projection, @@ -233,7 +238,7 @@ func pushProjectionInApplyJoin( } alias = p.DT.Columns[idx].String() } - splitProjectionAcrossJoin(ctx, src, lhs, rhs, pe, alias) + splitProjectionAcrossJoin(ctx, src, lhs, rhs, pe, alias, p.DT) } if p.isDerived() { @@ -255,23 +260,16 @@ func splitProjectionAcrossJoin( lhs, rhs *projector, pe *ProjExpr, colAlias string, + dt *DerivedTable, ) { - - // Check if the current expression can reuse an existing column in the ApplyJoin. - if _, found := canReuseColumn(ctx, join.JoinColumns.columns, pe.EvalExpr, joinColumnToExpr); found { - return - } - switch pe.Info.(type) { - case nil: - join.JoinColumns.add(splitUnexploredExpression(ctx, join, lhs, rhs, pe, colAlias)) - case Offset: + case Offset, nil: // for offsets, we'll just treat the expression as unexplored, and later stages will handle the new offset - join.JoinColumns.add(splitUnexploredExpression(ctx, join, lhs, rhs, pe, colAlias)) + join.JoinColumns.add(splitUnexploredExpression(ctx, join, lhs, rhs, pe, colAlias, dt)) case SubQueryExpression: join.JoinColumns.add(splitSubqueryExpression(ctx, join, lhs, rhs, pe, colAlias)) default: - panic(dbg.S(pe.Info)) + panic(vterrors.VT13001(fmt.Sprintf("unknown projection type %T", pe.Info))) } } @@ -292,9 +290,24 @@ func splitUnexploredExpression( lhs, rhs *projector, pe *ProjExpr, alias string, + dt *DerivedTable, ) applyJoinColumn { + original := sqlparser.CloneRefOfAliasedExpr(pe.Original) + expr := pe.ColExpr + + if dt != nil { + if !pe.isSameInAndOut(ctx) { + panic(vterrors.VT13001("derived table columns must be the same in and out")) + } + colName := pe.Original.ColumnName() + newExpr := sqlparser.NewColNameWithQualifier(colName, sqlparser.NewTableName(dt.Alias)) + ctx.SemTable.CopySemanticInfo(expr, newExpr) + original.Expr = newExpr + expr = newExpr + } + // Get a applyJoinColumn for the current expression. - col := join.getJoinColumnFor(ctx, pe.Original, pe.ColExpr, false) + col := join.getJoinColumnFor(ctx, original, expr, false) return pushDownSplitJoinCol(col, lhs, pe, alias, rhs) } @@ -349,8 +362,7 @@ func exposeColumnsThroughDerivedTable(ctx *plancontext.PlanningContext, p *Proje lhsIDs := TableID(src.LHS) rhsIDs := TableID(src.RHS) - rewriteColumnsForJoin(ctx, src.JoinPredicates.columns, lhsIDs, rhsIDs, lhs, rhs, false) - rewriteColumnsForJoin(ctx, src.JoinColumns.columns, lhsIDs, rhsIDs, lhs, rhs, true) + rewriteColumnsForJoin(ctx, src.JoinPredicates.columns, lhsIDs, rhsIDs, lhs, rhs) } func rewriteColumnsForJoin( @@ -358,8 +370,6 @@ func rewriteColumnsForJoin( columns []applyJoinColumn, 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 { @@ -370,30 +380,32 @@ func rewriteColumnsForJoin( continue } - // now we need to go over the predicate and find + // The RHSExprs are the expressions on the RHS of the join, and these have already been pushed down on the RHS + // of the ApplyJoin. These expressions don't need to be exposed through the derived table, they are just + // receiving the expressions from the LHS of the join using parameters. + var rewriteTo sqlparser.Expr pre := func(node, _ sqlparser.SQLNode) bool { - _, isSQ := node.(*sqlparser.Subquery) - if isSQ { + // We are looking for ColNames that belong to either the RHS or LHS of the join + // We'll replace these with columns being passed through the derived table + var col *sqlparser.ColName + switch node := node.(type) { + case *sqlparser.ColName: + col = node + case *sqlparser.Subquery: return false - } - expr, ok := node.(sqlparser.Expr) - if !ok { + default: return true } - deps := ctx.SemTable.RecursiveDeps(expr) + + deps := ctx.SemTable.RecursiveDeps(col) switch { - case deps.IsEmpty(): - return true case deps.IsSolvedBy(lhsIDs): - rewriteTo = lhs.get(ctx, expr) + rewriteTo = lhs.get(ctx, col) return false case deps.IsSolvedBy(rhsIDs): - if exposeRHS { - rewriteTo = rhs.get(ctx, expr) - } return false default: return true diff --git a/go/vt/vtgate/planbuilder/operators/query_planning.go b/go/vt/vtgate/planbuilder/operators/query_planning.go index d2eb1c37ccd..79e46030e58 100644 --- a/go/vt/vtgate/planbuilder/operators/query_planning.go +++ b/go/vt/vtgate/planbuilder/operators/query_planning.go @@ -404,7 +404,7 @@ func pushOrderingUnderAggr(ctx *plancontext.PlanningContext, order *Ordering, ag func canPushLeft(ctx *plancontext.PlanningContext, aj *ApplyJoin, order []OrderBy) bool { lhs := TableID(aj.LHS) for _, order := range order { - deps := ctx.SemTable.DirectDeps(order.Inner.Expr) + deps := ctx.SemTable.RecursiveDeps(order.Inner.Expr) if !deps.IsSolvedBy(lhs) { return false } diff --git a/go/vt/vtgate/planbuilder/operators/route.go b/go/vt/vtgate/planbuilder/operators/route.go index 82aeacb20b6..feeb091a725 100644 --- a/go/vt/vtgate/planbuilder/operators/route.go +++ b/go/vt/vtgate/planbuilder/operators/route.go @@ -672,23 +672,78 @@ func addMultipleColumnsToInput( return "", op, true, offset case *Union: - tableID := semantics.SingleTableSet(len(ctx.SemTable.Tables)) - ctx.SemTable.Tables = append(ctx.SemTable.Tables, nil) - unionColumns := op.GetColumns(ctx) - proj := &Projection{ - Source: op, - Columns: AliasedProjections(slice.Map(unionColumns, newProjExpr)), - DT: &DerivedTable{ - TableID: tableID, - Alias: "dt", - }, - } + proj := addDerivedProj(ctx, op) return addMultipleColumnsToInput(ctx, proj, reuse, addToGroupBy, exprs) default: return "", op, false, nil } } +func (r *Route) AddWSColumn(ctx *plancontext.PlanningContext, offset int, _ bool) int { + columns := r.GetColumns(ctx) + if offset > len(columns) { + panic(vterrors.VT13001(fmt.Sprintf("column %d not found", offset))) + } + col := columns[offset] + if offset := r.FindCol(ctx, weightStringFor(col.Expr), true); offset >= 0 { + return offset + } + + ok, foundOffset := addWSColumnToInput(ctx, r.Source, offset) + if !ok { + src := addDerivedProj(ctx, r.Source) + r.Source = src + return src.AddWSColumn(ctx, offset, true) + } + return foundOffset +} + +func addWSColumnToInput(ctx *plancontext.PlanningContext, source Operator, offset int) (bool, int) { + switch op := source.(type) { + case *SubQuery: + return addWSColumnToInput(ctx, op.Outer, offset) + case *Distinct: + return addWSColumnToInput(ctx, op.Source, offset) + case *Filter: + return addWSColumnToInput(ctx, op.Source, offset) + case *Projection: + return true, op.AddWSColumn(ctx, offset, true) + case *Aggregator: + return true, op.AddWSColumn(ctx, offset, true) + } + return false, -1 +} + +func addDerivedProj( + ctx *plancontext.PlanningContext, + op Operator, +) (projection *Projection) { + unionColumns := op.GetColumns(ctx) + columns := make(sqlparser.Columns, 0, len(unionColumns)) + for i := range unionColumns { + columns = append(columns, sqlparser.NewIdentifierCI(fmt.Sprintf("c%d", i))) + } + derivedProj := &Projection{ + Source: op, + Columns: AliasedProjections(slice.Map(unionColumns, newProjExpr)), + DT: &DerivedTable{ + TableID: ctx.SemTable.NewTableId(), + Alias: "dt", + Columns: columns, + }, + } + + proj := newAliasedProjection(derivedProj) + tbl := sqlparser.NewTableName("dt") + for i, col := range unionColumns { + projExpr := newProjExpr(col) + projExpr.EvalExpr = sqlparser.NewColNameWithQualifier(fmt.Sprintf("c%d", i), tbl) + proj.addProjExpr(projExpr) + } + + return proj +} + func (r *Route) FindCol(ctx *plancontext.PlanningContext, expr sqlparser.Expr, _ bool) int { return r.Source.FindCol(ctx, expr, true) } diff --git a/go/vt/vtgate/planbuilder/operators/subquery.go b/go/vt/vtgate/planbuilder/operators/subquery.go index 537737363c8..0597cbe0f18 100644 --- a/go/vt/vtgate/planbuilder/operators/subquery.go +++ b/go/vt/vtgate/planbuilder/operators/subquery.go @@ -179,6 +179,10 @@ func (sq *SubQuery) AddColumn(ctx *plancontext.PlanningContext, reuseExisting bo return sq.Outer.AddColumn(ctx, reuseExisting, addToGroupBy, exprs) } +func (sq *SubQuery) AddWSColumn(ctx *plancontext.PlanningContext, offset int, underRoute bool) int { + return sq.Outer.AddWSColumn(ctx, offset, underRoute) +} + func (sq *SubQuery) FindCol(ctx *plancontext.PlanningContext, expr sqlparser.Expr, underRoute bool) int { return sq.Outer.FindCol(ctx, expr, underRoute) } diff --git a/go/vt/vtgate/planbuilder/operators/subquery_container.go b/go/vt/vtgate/planbuilder/operators/subquery_container.go index e4feeab49d8..edbbec1125e 100644 --- a/go/vt/vtgate/planbuilder/operators/subquery_container.go +++ b/go/vt/vtgate/planbuilder/operators/subquery_container.go @@ -79,6 +79,10 @@ func (sqc *SubQueryContainer) AddColumn(ctx *plancontext.PlanningContext, reuseE return sqc.Outer.AddColumn(ctx, reuseExisting, addToGroupBy, exprs) } +func (sqc *SubQueryContainer) AddWSColumn(ctx *plancontext.PlanningContext, offset int, underRoute bool) int { + return sqc.Outer.AddWSColumn(ctx, offset, underRoute) +} + func (sqc *SubQueryContainer) FindCol(ctx *plancontext.PlanningContext, expr sqlparser.Expr, underRoute bool) int { return sqc.Outer.FindCol(ctx, expr, underRoute) } diff --git a/go/vt/vtgate/planbuilder/operators/table.go b/go/vt/vtgate/planbuilder/operators/table.go index 4bdf6f75c8b..14207fe3b3e 100644 --- a/go/vt/vtgate/planbuilder/operators/table.go +++ b/go/vt/vtgate/planbuilder/operators/table.go @@ -68,6 +68,10 @@ func (to *Table) AddColumn(*plancontext.PlanningContext, bool, bool, *sqlparser. panic(vterrors.VT13001("did not expect this method to be called")) } +func (*Table) AddWSColumn(*plancontext.PlanningContext, int, bool) int { + panic(vterrors.VT13001("did not expect this method to be called")) +} + func (to *Table) FindCol(ctx *plancontext.PlanningContext, expr sqlparser.Expr, underRoute bool) int { colToFind, ok := expr.(*sqlparser.ColName) if !ok { diff --git a/go/vt/vtgate/planbuilder/operators/union.go b/go/vt/vtgate/planbuilder/operators/union.go index 1d739c9f01c..d46ee609812 100644 --- a/go/vt/vtgate/planbuilder/operators/union.go +++ b/go/vt/vtgate/planbuilder/operators/union.go @@ -159,6 +159,10 @@ func (u *Union) GetSelectFor(source int) *sqlparser.Select { } } +func (u *Union) AddWSColumn(ctx *plancontext.PlanningContext, offset int, underRoute bool) int { + return u.addWeightStringToOffset(ctx, offset) +} + func (u *Union) AddColumn(ctx *plancontext.PlanningContext, reuse bool, gb bool, expr *sqlparser.AliasedExpr) int { if reuse { offset := u.FindCol(ctx, expr.Expr, false) @@ -188,21 +192,15 @@ func (u *Union) AddColumn(ctx *plancontext.PlanningContext, reuse bool, gb bool, panic(vterrors.VT13001(fmt.Sprintf("could not find the argument to the weight_string function: %s", sqlparser.String(wsArg)))) } - return u.addWeightStringToOffset(ctx, argIdx, gb) + return u.addWeightStringToOffset(ctx, argIdx) default: panic(vterrors.VT13001(fmt.Sprintf("only weight_string function is expected - got %s", sqlparser.String(expr)))) } } -func (u *Union) addWeightStringToOffset(ctx *plancontext.PlanningContext, argIdx int, addToGroupBy bool) (outputOffset int) { +func (u *Union) addWeightStringToOffset(ctx *plancontext.PlanningContext, argIdx int) (outputOffset int) { for i, src := range u.Sources { - exprs := u.Selects[i] - selectExpr := exprs[argIdx] - ae, ok := selectExpr.(*sqlparser.AliasedExpr) - if !ok { - panic(vterrors.VT09015()) - } - thisOffset := src.AddColumn(ctx, false, addToGroupBy, aeWrap(weightStringFor(ae.Expr))) + thisOffset := src.AddWSColumn(ctx, argIdx, false) // all offsets for the newly added ws need to line up if i == 0 { diff --git a/go/vt/vtgate/planbuilder/operators/utils_test.go b/go/vt/vtgate/planbuilder/operators/utils_test.go index 596489150da..035a273e964 100644 --- a/go/vt/vtgate/planbuilder/operators/utils_test.go +++ b/go/vt/vtgate/planbuilder/operators/utils_test.go @@ -58,6 +58,10 @@ func (f *fakeOp) AddColumn(ctx *plancontext.PlanningContext, reuseExisting bool, return len(f.cols) - 1 } +func (*fakeOp) AddWSColumn(*plancontext.PlanningContext, int, bool) int { + panic("implement me") +} + func (f *fakeOp) FindCol(ctx *plancontext.PlanningContext, a sqlparser.Expr, underRoute bool) int { return slices.IndexFunc(f.cols, func(b *sqlparser.AliasedExpr) bool { return a == b.Expr diff --git a/go/vt/vtgate/planbuilder/operators/vindex.go b/go/vt/vtgate/planbuilder/operators/vindex.go index c16944e7c49..fd907fdad27 100644 --- a/go/vt/vtgate/planbuilder/operators/vindex.go +++ b/go/vt/vtgate/planbuilder/operators/vindex.go @@ -75,6 +75,10 @@ func (v *Vindex) AddColumn(ctx *plancontext.PlanningContext, reuse bool, gb bool return addColumn(ctx, v, ae.Expr) } +func (*Vindex) AddWSColumn(*plancontext.PlanningContext, int, bool) int { + panic(vterrors.VT13001("did not expect this method to be called")) +} + func colNameToExpr(c *sqlparser.ColName) *sqlparser.AliasedExpr { return &sqlparser.AliasedExpr{ Expr: c, diff --git a/go/vt/vtgate/planbuilder/testdata/aggr_cases.json b/go/vt/vtgate/planbuilder/testdata/aggr_cases.json index 311f0f874cc..5706a4fb294 100644 --- a/go/vt/vtgate/planbuilder/testdata/aggr_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/aggr_cases.json @@ -2853,8 +2853,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select sum(col), 32 from (select `user`.col as col, 32 from `user` where 1 != 1) as t where 1 != 1", - "Query": "select sum(col), 32 from (select `user`.col as col, 32 from `user`) as t", + "FieldQuery": "select sum(col) from (select `user`.col as col, 32 from `user` where 1 != 1) as t where 1 != 1", + "Query": "select sum(col) from (select `user`.col as col, 32 from `user`) as t", "Table": "`user`" }, { diff --git a/go/vt/vtgate/planbuilder/testdata/cte_cases.json b/go/vt/vtgate/planbuilder/testdata/cte_cases.json index 0d7d9020ac2..337773faacb 100644 --- a/go/vt/vtgate/planbuilder/testdata/cte_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/cte_cases.json @@ -143,8 +143,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select sum(col), 32 from (select `user`.col as col, 32 from `user` where 1 != 1) as t where 1 != 1", - "Query": "select sum(col), 32 from (select `user`.col as col, 32 from `user`) as t", + "FieldQuery": "select sum(col) from (select `user`.col as col, 32 from `user` where 1 != 1) as t where 1 != 1", + "Query": "select sum(col) from (select `user`.col as col, 32 from `user`) as t", "Table": "`user`" }, { @@ -1283,41 +1283,32 @@ "QueryType": "SELECT", "Original": "with t as (select user.id from user join user_extra) select id, t.id from t", "Instructions": { - "OperatorType": "SimpleProjection", - "Columns": [ - 0, - 0 - ], + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,L:0", + "TableName": "`user`_user_extra", "Inputs": [ { - "OperatorType": "Join", - "Variant": "Join", - "JoinColumnIndexes": "L:0", - "TableName": "`user`_user_extra", - "Inputs": [ - { - "OperatorType": "Route", - "Variant": "Scatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select id from (select `user`.id from `user` where 1 != 1) as t where 1 != 1", - "Query": "select id from (select `user`.id from `user`) as t", - "Table": "`user`" - }, - { - "OperatorType": "Route", - "Variant": "Scatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select 1 from user_extra where 1 != 1", - "Query": "select 1 from user_extra", - "Table": "user_extra" - } - ] + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, t.id from (select `user`.id from `user` where 1 != 1) as t where 1 != 1", + "Query": "select id, t.id from (select `user`.id from `user`) as t", + "Table": "`user`" + }, + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user_extra where 1 != 1", + "Query": "select 1 from user_extra", + "Table": "user_extra" } ] }, @@ -1830,8 +1821,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select dt.id, dt.foo, weight_string(dt.id), weight_string(dt.foo) from (select id, foo from (select id, foo from `user` where 1 != 1) as x where 1 != 1 union select id, foo from (select id, foo from `user` where 1 != 1) as x where 1 != 1) as dt where 1 != 1", - "Query": "select dt.id, dt.foo, weight_string(dt.id), weight_string(dt.foo) from (select id, foo from (select id, foo from `user`) as x union select id, foo from (select id, foo from `user`) as x) as dt", + "FieldQuery": "select dt.c0 as id, dt.c1 as foo, weight_string(dt.c0), weight_string(dt.c1) from (select id, foo from (select id, foo from `user` where 1 != 1) as x where 1 != 1 union select id, foo from (select id, foo from `user` where 1 != 1) as x where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as id, dt.c1 as foo, weight_string(dt.c0), weight_string(dt.c1) from (select id, foo from (select id, foo from `user`) as x union select id, foo from (select id, foo from `user`) as x) as dt(c0, c1)", "Table": "`user`" } ] diff --git a/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json b/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json index b4a0ad63206..47f10cd273b 100644 --- a/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json @@ -526,7 +526,7 @@ { "OperatorType": "Join", "Variant": "LeftJoin", - "JoinColumnIndexes": "R:0,R:0", + "JoinColumnIndexes": "R:0", "TableName": "tbl10_tbl3", "Inputs": [ { @@ -1344,7 +1344,7 @@ { "OperatorType": "Join", "Variant": "LeftJoin", - "JoinColumnIndexes": "R:0,R:0", + "JoinColumnIndexes": "R:0", "JoinVars": { "tbl3_colx": 0 }, @@ -1427,7 +1427,7 @@ { "OperatorType": "Join", "Variant": "LeftJoin", - "JoinColumnIndexes": "R:0,R:0", + "JoinColumnIndexes": "R:0", "TableName": "tbl3_tbl1", "Inputs": [ { diff --git a/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_on_cases.json b/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_on_cases.json index d6829962f64..7b525b2dcc9 100644 --- a/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_on_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_on_cases.json @@ -526,7 +526,7 @@ { "OperatorType": "Join", "Variant": "LeftJoin", - "JoinColumnIndexes": "R:0,R:0", + "JoinColumnIndexes": "R:0", "TableName": "tbl10_tbl3", "Inputs": [ { @@ -1421,7 +1421,7 @@ { "OperatorType": "Join", "Variant": "LeftJoin", - "JoinColumnIndexes": "R:0,R:0", + "JoinColumnIndexes": "R:0", "JoinVars": { "tbl3_colx": 0 }, @@ -1504,7 +1504,7 @@ { "OperatorType": "Join", "Variant": "LeftJoin", - "JoinColumnIndexes": "R:0,R:0", + "JoinColumnIndexes": "R:0", "TableName": "tbl3_tbl1", "Inputs": [ { diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.json b/go/vt/vtgate/planbuilder/testdata/from_cases.json index aa0f1699db1..7d7963eaa45 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.json @@ -2465,41 +2465,32 @@ "QueryType": "SELECT", "Original": "select id, t.id from (select user.id from user join user_extra) as t", "Instructions": { - "OperatorType": "SimpleProjection", - "Columns": [ - 0, - 0 - ], + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,L:0", + "TableName": "`user`_user_extra", "Inputs": [ { - "OperatorType": "Join", - "Variant": "Join", - "JoinColumnIndexes": "L:0", - "TableName": "`user`_user_extra", - "Inputs": [ - { - "OperatorType": "Route", - "Variant": "Scatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select id from (select `user`.id from `user` where 1 != 1) as t where 1 != 1", - "Query": "select id from (select `user`.id from `user`) as t", - "Table": "`user`" - }, - { - "OperatorType": "Route", - "Variant": "Scatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select 1 from user_extra where 1 != 1", - "Query": "select 1 from user_extra", - "Table": "user_extra" - } - ] + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, t.id from (select `user`.id from `user` where 1 != 1) as t where 1 != 1", + "Query": "select id, t.id from (select `user`.id from `user`) as t", + "Table": "`user`" + }, + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user_extra where 1 != 1", + "Query": "select 1 from user_extra", + "Table": "user_extra" } ] }, @@ -4220,14 +4211,13 @@ "(0:2)", "(1:3)" ], - "ResultColumns": 2, "Inputs": [ { "OperatorType": "Join", "Variant": "HashLeftJoin", "Collation": "binary", "ComparisonType": "INT16", - "JoinColumnIndexes": "-1,2", + "JoinColumnIndexes": "-1,2,-3,3", "Predicate": "u.col = ue.col", "TableName": "`user`_user_extra", "Inputs": [ @@ -4238,8 +4228,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select u.id, u.col from (select id, col from `user` where 1 != 1) as u where 1 != 1", - "Query": "select distinct u.id, u.col from (select id, col from `user`) as u", + "FieldQuery": "select u.id, u.col, weight_string(u.id) from (select id, col from `user` where 1 != 1) as u where 1 != 1", + "Query": "select distinct u.id, u.col, weight_string(u.id) from (select id, col from `user`) as u", "Table": "`user`" }, { @@ -4253,8 +4243,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select ue.col, ue.user_id from (select col, user_id from user_extra where 1 != 1) as ue where 1 != 1", - "Query": "select ue.col, ue.user_id from (select col, user_id from user_extra) as ue limit :__upper_limit", + "FieldQuery": "select ue.col, ue.user_id, weight_string(ue.user_id) from (select col, user_id from user_extra where 1 != 1) as ue where 1 != 1", + "Query": "select ue.col, ue.user_id, weight_string(ue.user_id) from (select col, user_id from user_extra) as ue limit :__upper_limit", "Table": "user_extra" } ] diff --git a/go/vt/vtgate/planbuilder/testdata/large_union_cases.json b/go/vt/vtgate/planbuilder/testdata/large_union_cases.json index 89adb07335a..ac39682be4c 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 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", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from ((select content, user_id from music where 1 != 1) union (select content, user_id from music where 1 != 1)) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) 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(c0, c1)", "Table": "music", "Values": [ "1270698330" @@ -38,8 +38,8 @@ "Name": "user", "Sharded": true }, - "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", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from ((select content, user_id from music where 1 != 1) union (select content, user_id from music where 1 != 1)) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) 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(c0, c1)", "Table": "music", "Values": [ "1270699497" @@ -53,8 +53,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270703806 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270703806 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270703806" @@ -68,8 +68,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270707364 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270707364 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270707364" @@ -83,8 +83,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270714657 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270714657 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270714657" @@ -98,8 +98,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270721330 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270721330 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270721330" @@ -113,8 +113,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270812079 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270812079 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270812079" @@ -128,8 +128,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271011532 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271011532 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271011532" @@ -143,8 +143,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271034164 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271034164 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271034164" @@ -158,8 +158,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271034177 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271034177 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271034177" @@ -173,8 +173,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271066849 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271066849 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271066849" @@ -188,8 +188,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271098740 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271098740 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271098740" @@ -203,8 +203,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271355000 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271355000 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271355000" @@ -218,8 +218,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271639345 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271639345 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271639345" @@ -233,8 +233,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271914117 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271914117 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271914117" @@ -248,8 +248,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271924504 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271924504 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271924504" @@ -263,8 +263,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1272086055 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1272086055 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1272086055" @@ -278,8 +278,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1272127855 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1272127855 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1272127855" @@ -293,8 +293,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1272191137 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1272191137 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1272191137" @@ -308,8 +308,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1272468271 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1272468271 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1272468271" @@ -323,8 +323,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270637436 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270637436 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270637436" @@ -338,8 +338,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270644941 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270644941 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270644941" @@ -353,8 +353,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270650576 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270650576 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270650576" @@ -368,8 +368,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270652906 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270652906 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270652906" @@ -383,8 +383,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270660650 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270660650 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270660650" @@ -398,8 +398,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270670201 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270670201 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270670201" @@ -413,8 +413,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270707364 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270707364 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270707364" @@ -428,8 +428,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271365691 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271365691 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271365691" @@ -443,8 +443,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271799956 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271799956 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271799956" @@ -458,8 +458,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271914117 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271914117 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271914117" @@ -473,8 +473,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270637436 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270637436 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270637436" @@ -488,8 +488,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271799956 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271799956 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271799956" @@ -503,8 +503,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270637436 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270637436 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270637436" @@ -518,8 +518,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271639345 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271639345 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271639345" @@ -533,8 +533,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270644941 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270644941 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270644941" @@ -548,8 +548,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270649256 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270649256 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270649256" @@ -563,8 +563,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270653671 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270653671 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270653671" @@ -578,8 +578,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270670201 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270670201 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270670201" @@ -593,8 +593,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270717223 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270717223 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270717223" @@ -608,8 +608,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270720898 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270720898 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270720898" @@ -623,8 +623,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270982590 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270982590 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270982590" @@ -638,8 +638,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271346411 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271346411 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271346411" @@ -653,8 +653,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271352121 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271352121 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271352121" @@ -668,8 +668,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271354908 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271354908 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271354908" @@ -683,8 +683,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271365691 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271365691 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271365691" @@ -698,8 +698,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271367516 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271367516 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271367516" @@ -713,8 +713,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271472522 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271472522 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271472522" @@ -728,8 +728,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271607757 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271607757 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271607757" @@ -743,8 +743,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271639345 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271639345 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271639345" @@ -758,8 +758,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271821733 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271821733 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271821733" @@ -773,8 +773,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271914117 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271914117 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271914117" @@ -788,8 +788,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1272068709 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1272068709 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1272068709" @@ -803,8 +803,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1272127855 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1272127855 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1272127855" @@ -818,8 +818,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1272191137 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1272191137 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1272191137" @@ -833,8 +833,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1272244005 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1272244005 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1272244005" @@ -848,8 +848,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1272468271 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1272468271 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1272468271" @@ -863,8 +863,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270982590 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270982590 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270982590" @@ -878,8 +878,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271365691 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271365691 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271365691" @@ -893,8 +893,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271607757 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271607757 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271607757" @@ -908,8 +908,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1270982590 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1270982590 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1270982590" @@ -923,8 +923,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271365691 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271365691 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271365691" @@ -938,8 +938,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1271607757 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1271607757 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1271607757" @@ -953,8 +953,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select content, user_id, weight_string(content), weight_string(user_id) from music where 1 != 1", - "Query": "select distinct content, user_id, weight_string(content), weight_string(user_id) from music where user_id = 1272244005 order by created_at asc, id asc limit 11", + "FieldQuery": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select content, user_id from music where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as content, dt.c1 as user_id, weight_string(dt.c0), weight_string(dt.c1) from (select distinct content, user_id from music where user_id = 1272244005 order by created_at asc, id asc limit 11) as dt(c0, c1)", "Table": "music", "Values": [ "1272244005" diff --git a/go/vt/vtgate/planbuilder/testdata/memory_sort_cases.json b/go/vt/vtgate/planbuilder/testdata/memory_sort_cases.json index 4a879997925..951c9337ddf 100644 --- a/go/vt/vtgate/planbuilder/testdata/memory_sort_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/memory_sort_cases.json @@ -655,5 +655,55 @@ "user.user" ] } + }, + { + "comment": "Derived table split across two shards, and ordered by both", + "query": "select * from (select u.foo, ue.bar from user u, user_extra ue) tbl order by tbl.bar, tbl.foo", + "plan": { + "QueryType": "SELECT", + "Original": "select * from (select u.foo, ue.bar from user u, user_extra ue) tbl order by tbl.bar, tbl.foo", + "Instructions": { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "(1|2) ASC, (0|3) ASC", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,R:0,R:1,L:1", + "TableName": "`user`_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select foo, weight_string(foo) from (select u.foo from `user` as u where 1 != 1) as tbl where 1 != 1", + "Query": "select foo, weight_string(foo) from (select u.foo from `user` as u) as tbl", + "Table": "`user`" + }, + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select bar, weight_string(bar) from (select ue.bar from user_extra as ue where 1 != 1) as tbl where 1 != 1", + "Query": "select bar, weight_string(bar) from (select ue.bar from user_extra as ue) as tbl", + "Table": "user_extra" + } + ] + } + ] + }, + "TablesUsed": [ + "user.user", + "user.user_extra" + ] + } } ] diff --git a/go/vt/vtgate/planbuilder/testdata/postprocess_cases.json b/go/vt/vtgate/planbuilder/testdata/postprocess_cases.json index 53d9a136b23..35c0b78a91f 100644 --- a/go/vt/vtgate/planbuilder/testdata/postprocess_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/postprocess_cases.json @@ -1585,42 +1585,33 @@ "QueryType": "SELECT", "Original": "select name, name from user, music order by name", "Instructions": { - "OperatorType": "SimpleProjection", - "Columns": [ - 0, - 0 - ], + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,L:0", + "TableName": "`user`_music", "Inputs": [ { - "OperatorType": "Join", - "Variant": "Join", - "JoinColumnIndexes": "L:0", - "TableName": "`user`_music", - "Inputs": [ - { - "OperatorType": "Route", - "Variant": "Scatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select `name`, weight_string(`name`) from `user` where 1 != 1", - "OrderBy": "(0|1) ASC", - "Query": "select `name`, weight_string(`name`) from `user` order by `user`.`name` asc", - "Table": "`user`" - }, - { - "OperatorType": "Route", - "Variant": "Scatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select 1 from music where 1 != 1", - "Query": "select 1 from music", - "Table": "music" - } - ] + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `name`, `name`, weight_string(`name`) from `user` where 1 != 1", + "OrderBy": "(0|2) ASC", + "Query": "select `name`, `name`, weight_string(`name`) from `user` order by `user`.`name` asc", + "Table": "`user`" + }, + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from music where 1 != 1", + "Query": "select 1 from music", + "Table": "music" } ] }, @@ -2288,8 +2279,8 @@ "Name": "main", "Sharded": false }, - "FieldQuery": "select 1, 2, weight_string(1) from dual where 1 != 1", - "Query": "select 1, 2, weight_string(1) from dual", + "FieldQuery": "select dt.c0 as `1`, dt.c1 as `2`, weight_string(dt.c0) from (select 1, 2 from dual where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as `1`, dt.c1 as `2`, weight_string(dt.c0) from (select 1, 2 from dual) as dt(c0, c1)", "Table": "dual" } ] @@ -2355,8 +2346,8 @@ "Name": "main", "Sharded": false }, - "FieldQuery": "select 1, 2, weight_string(2) from dual where 1 != 1", - "Query": "select 1, 2, weight_string(2) from dual", + "FieldQuery": "select dt.c0 as `1`, dt.c1 as `2`, weight_string(dt.c1) from (select 1, 2 from dual where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as `1`, dt.c1 as `2`, weight_string(dt.c1) from (select 1, 2 from dual) as dt(c0, c1)", "Table": "dual" } ] diff --git a/go/vt/vtgate/planbuilder/testdata/tpch_cases.json b/go/vt/vtgate/planbuilder/testdata/tpch_cases.json index 609285c4bfe..119205e3015 100644 --- a/go/vt/vtgate/planbuilder/testdata/tpch_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/tpch_cases.json @@ -564,65 +564,225 @@ "ResultColumns": 4, "Inputs": [ { - "OperatorType": "Projection", - "Expressions": [ - ":2 as supp_nation", - ":3 as cust_nation", - ":4 as l_year", - "sum(volume) * count(*) as revenue", - ":5 as weight_string(supp_nation)", - ":6 as weight_string(cust_nation)", - ":7 as weight_string(l_year)" - ], + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "(0|4) ASC, (1|5) ASC, (2|6) ASC", "Inputs": [ { "OperatorType": "Join", "Variant": "Join", - "JoinColumnIndexes": "L:0,R:0,L:1,R:1,L:2,L:4,R:2,L:5", + "JoinColumnIndexes": "L:0,R:0,L:1,L:2,L:4,R:1,L:5", "JoinVars": { - "n1_n_name": 1, + "n1_n_name": 0, "o_custkey": 3 }, "TableName": "lineitem_orders_supplier_nation_customer_nation", "Inputs": [ { - "OperatorType": "Projection", - "Expressions": [ - "sum(volume) * count(*) as revenue", - ":2 as supp_nation", - ":3 as l_year", - ":4 as o_custkey", - ":5 as weight_string(supp_nation)", - ":6 as weight_string(l_year)" - ], + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "R:0,L:0,L:1,L:2,R:1,L:4", + "JoinVars": { + "l_suppkey": 3 + }, + "TableName": "lineitem_orders_supplier_nation", "Inputs": [ { "OperatorType": "Join", "Variant": "Join", - "JoinColumnIndexes": "L:0,R:0,R:1,L:1,L:2,R:2,L:4", + "JoinColumnIndexes": "L:0,L:1,R:0,L:2,L:4", "JoinVars": { - "l_suppkey": 3 + "l_orderkey": 3 }, - "TableName": "lineitem_orders_supplier_nation", + "TableName": "lineitem_orders", "Inputs": [ { - "OperatorType": "Projection", - "Expressions": [ - "sum(volume) * count(*) as revenue", - ":2 as l_year", - ":3 as o_custkey", - ":4 as l_suppkey", - ":5 as weight_string(l_year)" + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select shipping.l_year, shipping.volume, shipping.l_suppkey, shipping.l_orderkey, weight_string(shipping.l_year) 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", + "Query": "select shipping.l_year, shipping.volume, shipping.l_suppkey, shipping.l_orderkey, weight_string(shipping.l_year) 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", + "Table": "lineitem" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select shipping.o_custkey from (select o_custkey as o_custkey from orders where 1 != 1) as shipping where 1 != 1", + "Query": "select shipping.o_custkey from (select o_custkey as o_custkey from orders where o_orderkey = :l_orderkey) as shipping", + "Table": "orders", + "Values": [ + ":l_orderkey" + ], + "Vindex": "hash" + } + ] + }, + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "R:0,R:1", + "JoinVars": { + "s_nationkey": 0 + }, + "TableName": "supplier_nation", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select shipping.s_nationkey from (select s_nationkey as s_nationkey from supplier where 1 != 1) as shipping where 1 != 1", + "Query": "select shipping.s_nationkey from (select s_nationkey as s_nationkey from supplier where s_suppkey = :l_suppkey) as shipping", + "Table": "supplier", + "Values": [ + ":l_suppkey" + ], + "Vindex": "hash" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select shipping.supp_nation, weight_string(shipping.supp_nation) from (select n1.n_name as supp_nation from nation as n1 where 1 != 1) as shipping where 1 != 1", + "Query": "select shipping.supp_nation, weight_string(shipping.supp_nation) from (select n1.n_name as supp_nation from nation as n1 where n1.n_nationkey = :s_nationkey) as shipping", + "Table": "nation", + "Values": [ + ":s_nationkey" ], + "Vindex": "hash" + } + ] + } + ] + }, + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "R:0,R:1", + "JoinVars": { + "c_nationkey": 0 + }, + "TableName": "customer_nation", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select shipping.c_nationkey from (select c_nationkey as c_nationkey from customer where 1 != 1) as shipping where 1 != 1", + "Query": "select shipping.c_nationkey from (select c_nationkey as c_nationkey from customer where c_custkey = :o_custkey) as shipping", + "Table": "customer", + "Values": [ + ":o_custkey" + ], + "Vindex": "hash" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select shipping.cust_nation, weight_string(shipping.cust_nation) from (select n2.n_name as cust_nation from nation as n2 where 1 != 1) as shipping where 1 != 1", + "Query": "select shipping.cust_nation, weight_string(shipping.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", + "Table": "nation", + "Values": [ + ":c_nationkey" + ], + "Vindex": "hash" + } + ] + } + ] + } + ] + } + ] + }, + "TablesUsed": [ + "main.customer", + "main.lineitem", + "main.nation", + "main.orders", + "main.supplier" + ] + } + }, + { + "comment": "TPC-H query 8", + "query": "select o_year, sum(case when nation = 'BRAZIL' then volume else 0 end) / sum(volume) as mkt_share from ( select extract(year from o_orderdate) as o_year, l_extendedprice * (1 - l_discount) as volume, n2.n_name as nation from part, supplier, lineitem, orders, customer, nation n1, nation n2, region where p_partkey = l_partkey and s_suppkey = l_suppkey and l_orderkey = o_orderkey and o_custkey = c_custkey and c_nationkey = n1.n_nationkey and n1.n_regionkey = r_regionkey and r_name = 'AMERICA' and s_nationkey = n2.n_nationkey and o_orderdate between date '1995-01-01' and date('1996-12-31') and p_type = 'ECONOMY ANODIZED STEEL' ) as all_nations group by o_year order by o_year", + "plan": { + "QueryType": "SELECT", + "Original": "select o_year, sum(case when nation = 'BRAZIL' then volume else 0 end) / sum(volume) as mkt_share from ( select extract(year from o_orderdate) as o_year, l_extendedprice * (1 - l_discount) as volume, n2.n_name as nation from part, supplier, lineitem, orders, customer, nation n1, nation n2, region where p_partkey = l_partkey and s_suppkey = l_suppkey and l_orderkey = o_orderkey and o_custkey = c_custkey and c_nationkey = n1.n_nationkey and n1.n_regionkey = r_regionkey and r_name = 'AMERICA' and s_nationkey = n2.n_nationkey and o_orderdate between date '1995-01-01' and date('1996-12-31') and p_type = 'ECONOMY ANODIZED STEEL' ) as all_nations group by o_year order by o_year", + "Instructions": { + "OperatorType": "Projection", + "Expressions": [ + ":0 as o_year", + "sum(case when nation = 'BRAZIL' then volume else 0 end) / sum(volume) as mkt_share" + ], + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum(1) AS sum(case when nation = 'BRAZIL' then volume else 0 end), sum(2) AS sum(volume)", + "GroupBy": "(0|3)", + "Inputs": [ + { + "OperatorType": "SimpleProjection", + "Columns": [ + 0, + 3, + 1, + 4 + ], + "Inputs": [ + { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "(0|4) ASC", + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "R:0,L:0,L:1,L:3,R:1", + "JoinVars": { + "l_orderkey": 2 + }, + "TableName": "lineitem_part_supplier_nation_orders_customer_nation_region", + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,R:0,L:1,R:1", + "JoinVars": { + "l_suppkey": 2, + "volume": 0 + }, + "TableName": "lineitem_part_supplier_nation", "Inputs": [ { "OperatorType": "Join", "Variant": "Join", - "JoinColumnIndexes": "L:0,R:0,L:1,R:1,L:2,L:4", + "JoinColumnIndexes": "L:0,L:1,L:2", "JoinVars": { - "l_orderkey": 3 + "l_partkey": 3 }, - "TableName": "lineitem_orders", + "TableName": "lineitem_part", "Inputs": [ { "OperatorType": "Route", @@ -631,9 +791,8 @@ "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 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 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", + "FieldQuery": "select all_nations.volume, all_nations.l_orderkey, all_nations.l_suppkey, all_nations.l_partkey from (select l_extendedprice * (1 - l_discount) as volume, l_orderkey as l_orderkey, l_suppkey as l_suppkey, l_partkey as l_partkey from lineitem where 1 != 1) as all_nations where 1 != 1", + "Query": "select all_nations.volume, all_nations.l_orderkey, all_nations.l_suppkey, all_nations.l_partkey from (select l_extendedprice * (1 - l_discount) as volume, l_orderkey as l_orderkey, l_suppkey as l_suppkey, l_partkey as l_partkey from lineitem) as all_nations", "Table": "lineitem" }, { @@ -643,11 +802,52 @@ "Name": "main", "Sharded": true }, - "FieldQuery": "select count(*), shipping.o_custkey from (select o_custkey as o_custkey 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 from orders where o_orderkey = :l_orderkey) as shipping group by shipping.o_custkey", - "Table": "orders", + "FieldQuery": "select 1 from part where 1 != 1", + "Query": "select 1 from part where p_type = 'ECONOMY ANODIZED STEEL' and p_partkey = :l_partkey", + "Table": "part", "Values": [ - ":l_orderkey" + ":l_partkey" + ], + "Vindex": "hash" + } + ] + }, + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "R:0,R:1", + "JoinVars": { + "s_nationkey": 0 + }, + "TableName": "supplier_nation", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select all_nations.s_nationkey from (select s_nationkey as s_nationkey from supplier where 1 != 1) as all_nations where 1 != 1", + "Query": "select all_nations.s_nationkey from (select s_nationkey as s_nationkey from supplier where s_suppkey = :l_suppkey) as all_nations", + "Table": "supplier", + "Values": [ + ":l_suppkey" + ], + "Vindex": "hash" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select all_nations.nation, case when nation = 'BRAZIL' then :volume else 0 end from (select n2.n_name as nation from nation as n2 where 1 != 1) as all_nations where 1 != 1", + "Query": "select all_nations.nation, case when nation = 'BRAZIL' then :volume else 0 end from (select n2.n_name as nation from nation as n2 where n2.n_nationkey = :s_nationkey) as all_nations", + "Table": "nation", + "Values": [ + ":s_nationkey" ], "Vindex": "hash" } @@ -656,21 +856,22 @@ ] }, { - "OperatorType": "Projection", - "Expressions": [ - "count(*) * count(*) as count(*)", - ":2 as supp_nation", - ":3 as weight_string(supp_nation)" - ], + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,L:2", + "JoinVars": { + "c_nationkey": 1 + }, + "TableName": "orders_customer_nation_region", "Inputs": [ { "OperatorType": "Join", "Variant": "Join", - "JoinColumnIndexes": "L:0,R:0,R:1,R:2", + "JoinColumnIndexes": "L:0,R:0,L:2", "JoinVars": { - "s_nationkey": 1 + "o_custkey": 1 }, - "TableName": "supplier_nation", + "TableName": "orders_customer", "Inputs": [ { "OperatorType": "Route", @@ -679,11 +880,11 @@ "Name": "main", "Sharded": true }, - "FieldQuery": "select count(*), shipping.s_nationkey from (select 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_nationkey as s_nationkey from supplier where s_suppkey = :l_suppkey) as shipping group by shipping.s_nationkey", - "Table": "supplier", + "FieldQuery": "select all_nations.o_year, all_nations.o_custkey, weight_string(all_nations.o_year) from (select extract(year from o_orderdate) as o_year, o_custkey as o_custkey from orders where 1 != 1) as all_nations where 1 != 1", + "Query": "select all_nations.o_year, all_nations.o_custkey, weight_string(all_nations.o_year) from (select extract(year from o_orderdate) as o_year, o_custkey as o_custkey from orders where o_orderdate between date'1995-01-01' and date('1996-12-31') and o_orderkey = :l_orderkey) as all_nations", + "Table": "orders", "Values": [ - ":l_suppkey" + ":l_orderkey" ], "Vindex": "hash" }, @@ -694,11 +895,51 @@ "Name": "main", "Sharded": true }, - "FieldQuery": "select count(*), supp_nation, weight_string(supp_nation) 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'` 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, n1.n_name = 'FRANCE' as `n1.n_name = 'FRANCE'`, n1.n_name = 'GERMANY' as `n1.n_name = 'GERMANY'` from nation as n1 where n1.n_nationkey = :s_nationkey) as shipping group by supp_nation, weight_string(supp_nation)", + "FieldQuery": "select all_nations.c_nationkey from (select c_nationkey as c_nationkey from customer where 1 != 1) as all_nations where 1 != 1", + "Query": "select all_nations.c_nationkey from (select c_nationkey as c_nationkey from customer where c_custkey = :o_custkey) as all_nations", + "Table": "customer", + "Values": [ + ":o_custkey" + ], + "Vindex": "hash" + } + ] + }, + { + "OperatorType": "Join", + "Variant": "Join", + "JoinVars": { + "n1_n_regionkey": 0 + }, + "TableName": "nation_region", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select n1.n_regionkey from nation as n1 where 1 != 1", + "Query": "select n1.n_regionkey from nation as n1 where n1.n_nationkey = :c_nationkey", "Table": "nation", "Values": [ - ":s_nationkey" + ":c_nationkey" + ], + "Vindex": "hash" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select 1 from region where 1 != 1", + "Query": "select 1 from region where r_name = 'AMERICA' and r_regionkey = :n1_n_regionkey", + "Table": "region", + "Values": [ + ":n1_n_regionkey" ], "Vindex": "hash" } @@ -709,57 +950,231 @@ ] } ] - }, + } + ] + } + ] + } + ] + }, + "TablesUsed": [ + "main.customer", + "main.lineitem", + "main.nation", + "main.orders", + "main.part", + "main.region", + "main.supplier" + ] + } + }, + { + "comment": "TPC-H query 9", + "query": "select nation, o_year, sum(amount) as sum_profit from ( select n_name as nation, extract(year from o_orderdate) as o_year, l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount from part, supplier, lineitem, partsupp, orders, nation where s_suppkey = l_suppkey and ps_suppkey = l_suppkey and ps_partkey = l_partkey and p_partkey = l_partkey and o_orderkey = l_orderkey and s_nationkey = n_nationkey and p_name like '%green%' ) as profit group by nation, o_year order by nation, o_year desc", + "plan": { + "QueryType": "SELECT", + "Original": "select nation, o_year, sum(amount) as sum_profit from ( select n_name as nation, extract(year from o_orderdate) as o_year, l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount from part, supplier, lineitem, partsupp, orders, nation where s_suppkey = l_suppkey and ps_suppkey = l_suppkey and ps_partkey = l_partkey and p_partkey = l_partkey and o_orderkey = l_orderkey and s_nationkey = n_nationkey and p_name like '%green%' ) as profit group by nation, o_year order by nation, o_year desc", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum(2) AS sum_profit", + "GroupBy": "(0|3), (1|4)", + "ResultColumns": 3, + "Inputs": [ + { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "(0|3) ASC, (1|4) DESC", + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "R:0,L:0,L:1,R:1,L:3", + "JoinVars": { + "l_suppkey": 2 + }, + "TableName": "orders_lineitem_part_partsupp_supplier_nation", + "Inputs": [ { - "OperatorType": "Projection", - "Expressions": [ - "count(*) * count(*) as count(*)", - ":2 as cust_nation", - ":3 as weight_string(cust_nation)" - ], + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,R:0,L:1,L:3", + "JoinVars": { + "l_partkey": 2, + "l_suppkey": 1 + }, + "TableName": "orders_lineitem_part_partsupp", "Inputs": [ { "OperatorType": "Join", "Variant": "Join", - "JoinColumnIndexes": "L:0,R:0,R:1,R:2", + "JoinColumnIndexes": "L:0,R:0,R:1,L:2", "JoinVars": { - "c_nationkey": 1 + "o_orderkey": 1 }, - "TableName": "customer_nation", + "TableName": "orders_lineitem_part", "Inputs": [ { "OperatorType": "Route", - "Variant": "EqualUnique", + "Variant": "Scatter", "Keyspace": { "Name": "main", "Sharded": true }, - "FieldQuery": "select count(*), shipping.c_nationkey from (select 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_nationkey as c_nationkey from customer where c_custkey = :o_custkey) as shipping group by shipping.c_nationkey", - "Table": "customer", - "Values": [ - ":o_custkey" - ], - "Vindex": "hash" + "FieldQuery": "select profit.o_year, profit.o_orderkey, weight_string(profit.o_year) from (select extract(year from o_orderdate) as o_year, o_orderkey as o_orderkey from orders where 1 != 1) as profit where 1 != 1", + "Query": "select profit.o_year, profit.o_orderkey, weight_string(profit.o_year) from (select extract(year from o_orderdate) as o_year, o_orderkey as o_orderkey from orders) as profit", + "Table": "orders" }, + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,L:1", + "JoinVars": { + "l_partkey": 1 + }, + "TableName": "lineitem_part", + "Inputs": [ + { + "OperatorType": "VindexLookup", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "Values": [ + ":o_orderkey" + ], + "Vindex": "lineitem_map", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "IN", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select l_orderkey, l_linenumber from lineitem_map where 1 != 1", + "Query": "select l_orderkey, l_linenumber from lineitem_map where l_orderkey in ::__vals", + "Table": "lineitem_map", + "Values": [ + "::l_orderkey" + ], + "Vindex": "md5" + }, + { + "OperatorType": "Route", + "Variant": "ByDestination", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select profit.l_suppkey, profit.l_partkey from (select l_suppkey as l_suppkey, l_partkey as l_partkey from lineitem where 1 != 1) as profit where 1 != 1", + "Query": "select profit.l_suppkey, profit.l_partkey from (select l_suppkey as l_suppkey, l_partkey as l_partkey from lineitem where l_orderkey = :o_orderkey) as profit", + "Table": "lineitem" + } + ] + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select 1 from part where 1 != 1", + "Query": "select 1 from part where p_name like '%green%' and p_partkey = :l_partkey", + "Table": "part", + "Values": [ + ":l_partkey" + ], + "Vindex": "hash" + } + ] + } + ] + }, + { + "OperatorType": "VindexLookup", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "Values": [ + ":l_partkey" + ], + "Vindex": "partsupp_map", + "Inputs": [ { "OperatorType": "Route", - "Variant": "EqualUnique", + "Variant": "IN", "Keyspace": { "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)", - "Table": "nation", + "FieldQuery": "select ps_partkey, ps_suppkey from partsupp_map where 1 != 1", + "Query": "select ps_partkey, ps_suppkey from partsupp_map where ps_partkey in ::__vals", + "Table": "partsupp_map", "Values": [ - ":c_nationkey" + "::ps_partkey" ], - "Vindex": "hash" + "Vindex": "md5" + }, + { + "OperatorType": "Route", + "Variant": "ByDestination", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select profit.amount from (select l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount from partsupp where 1 != 1) as profit where 1 != 1", + "Query": "select profit.amount from (select l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount from partsupp where ps_partkey = :l_partkey and ps_suppkey = :l_suppkey) as profit", + "Table": "partsupp" } ] } ] + }, + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "R:0,R:1", + "JoinVars": { + "s_nationkey": 0 + }, + "TableName": "supplier_nation", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select profit.s_nationkey from (select s_nationkey as s_nationkey from supplier where 1 != 1) as profit where 1 != 1", + "Query": "select profit.s_nationkey from (select s_nationkey as s_nationkey from supplier where s_suppkey = :l_suppkey) as profit", + "Table": "supplier", + "Values": [ + ":l_suppkey" + ], + "Vindex": "hash" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select profit.nation, weight_string(profit.nation) from (select n_name as nation from nation where 1 != 1) as profit where 1 != 1", + "Query": "select profit.nation, weight_string(profit.nation) from (select n_name as nation from nation where n_nationkey = :s_nationkey) as profit", + "Table": "nation", + "Values": [ + ":s_nationkey" + ], + "Vindex": "hash" + } + ] } ] } @@ -768,24 +1183,15 @@ ] }, "TablesUsed": [ - "main.customer", "main.lineitem", "main.nation", "main.orders", + "main.part", + "main.partsupp", "main.supplier" ] } }, - { - "comment": "TPC-H query 8", - "query": "select o_year, sum(case when nation = 'BRAZIL' then volume else 0 end) / sum(volume) as mkt_share from ( select extract(year from o_orderdate) as o_year, l_extendedprice * (1 - l_discount) as volume, n2.n_name as nation from part, supplier, lineitem, orders, customer, nation n1, nation n2, region where p_partkey = l_partkey and s_suppkey = l_suppkey and l_orderkey = o_orderkey and o_custkey = c_custkey and c_nationkey = n1.n_nationkey and n1.n_regionkey = r_regionkey and r_name = 'AMERICA' and s_nationkey = n2.n_nationkey and o_orderdate between date '1995-01-01' and date('1996-12-31') and p_type = 'ECONOMY ANODIZED STEEL' ) as all_nations group by o_year order by o_year", - "plan": "VT13002: unexpected AST struct for query: o_year" - }, - { - "comment": "TPC-H query 9", - "query": "select nation, o_year, sum(amount) as sum_profit from ( select n_name as nation, extract(year from o_orderdate) as o_year, l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount from part, supplier, lineitem, partsupp, orders, nation where s_suppkey = l_suppkey and ps_suppkey = l_suppkey and ps_partkey = l_partkey and p_partkey = l_partkey and o_orderkey = l_orderkey and s_nationkey = n_nationkey and p_name like '%green%' ) as profit group by nation, o_year order by nation, o_year desc", - "plan": "VT13002: unexpected AST struct for query: nation" - }, { "comment": "TPC-H query 10", "query": "select c_custkey, c_name, sum(l_extendedprice * (1 - l_discount)) as revenue, c_acctbal, n_name, c_address, c_phone, c_comment from customer, orders, lineitem, nation where c_custkey = o_custkey and l_orderkey = o_orderkey and o_orderdate >= date('1993-10-01') and o_orderdate < date('1993-10-01') + interval '3' month and l_returnflag = 'R' and c_nationkey = n_nationkey group by c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment order by revenue desc limit 20", diff --git a/go/vt/vtgate/planbuilder/testdata/union_cases.json b/go/vt/vtgate/planbuilder/testdata/union_cases.json index cdbd368478f..cb9b5113b67 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 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", + "FieldQuery": "select dt.c0 as id, weight_string(dt.c0) from (select id from `user` where 1 != 1 union select id from music where 1 != 1) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as id, weight_string(dt.c0) from (select id from `user` union select id from music) as dt(c0)", "Table": "`user`, music" } ] @@ -336,8 +336,8 @@ "Name": "main", "Sharded": false }, - "FieldQuery": "select CHARACTER_SET_NAME, weight_string(CHARACTER_SET_NAME) from information_schema.CHARACTER_SETS where 1 != 1", - "Query": "select distinct CHARACTER_SET_NAME, weight_string(CHARACTER_SET_NAME) from information_schema.CHARACTER_SETS", + "FieldQuery": "select dt.c0 as CHARACTER_SET_NAME, weight_string(dt.c0) from (select CHARACTER_SET_NAME from information_schema.CHARACTER_SETS where 1 != 1) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as CHARACTER_SET_NAME, weight_string(dt.c0) from (select distinct CHARACTER_SET_NAME from information_schema.CHARACTER_SETS) as dt(c0)", "Table": "information_schema.CHARACTER_SETS" }, { @@ -347,8 +347,8 @@ "Name": "main", "Sharded": false }, - "FieldQuery": "select user_name, weight_string(user_name) from unsharded where 1 != 1", - "Query": "select distinct user_name, weight_string(user_name) from unsharded", + "FieldQuery": "select dt.c0 as user_name, weight_string(dt.c0) from (select user_name from unsharded where 1 != 1) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as user_name, weight_string(dt.c0) from (select distinct user_name from unsharded) as dt(c0)", "Table": "unsharded" } ] @@ -385,8 +385,8 @@ "Name": "user", "Sharded": true }, - "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", + "FieldQuery": "select dt.c0 as id, weight_string(dt.c0) 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(c0) where 1 != 1", + "Query": "select dt.c0 as id, weight_string(dt.c0) from (select id from `user` union select id from music union select 1 from dual) as dt(c0)", "Table": "`user`, dual, music" } ] @@ -504,8 +504,8 @@ "Name": "user", "Sharded": true }, - "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", + "FieldQuery": "select dt.c0 as id, weight_string(dt.c0) from ((select id from `user` where 1 != 1) union (select id from `user` where 1 != 1)) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as id, weight_string(dt.c0) from ((select id from `user` order by id desc) union (select id from `user` order by id asc)) as dt(c0)", "Table": "`user`" } ] @@ -535,8 +535,8 @@ "Name": "user", "Sharded": true }, - "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", + "FieldQuery": "select dt.c0 as `1`, weight_string(dt.c0) 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(c0) where 1 != 1", + "Query": "select dt.c0 as `1`, weight_string(dt.c0) 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(c0)", "Table": "`user`, dual" } ] @@ -601,8 +601,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select 'b', 'c', weight_string('b'), weight_string('c') from `user` where 1 != 1", - "Query": "select distinct 'b', 'c', weight_string('b'), weight_string('c') from `user`", + "FieldQuery": "select dt.c0 as `'b'`, dt.c1 as `'c'`, weight_string(dt.c0), weight_string(dt.c1) from (select 'b', 'c' from `user` where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as `'b'`, dt.c1 as `'c'`, weight_string(dt.c0), weight_string(dt.c1) from (select distinct 'b', 'c' from `user`) as dt(c0, c1)", "Table": "`user`" } ] @@ -639,8 +639,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select 'b', 'c', weight_string('b'), weight_string('c') from `user` where 1 != 1", - "Query": "select distinct 'b', 'c', weight_string('b'), weight_string('c') from `user`", + "FieldQuery": "select dt.c0 as `'b'`, dt.c1 as `'c'`, weight_string(dt.c0), weight_string(dt.c1) from (select 'b', 'c' from `user` where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as `'b'`, dt.c1 as `'c'`, weight_string(dt.c0), weight_string(dt.c1) from (select distinct 'b', 'c' from `user`) as dt(c0, c1)", "Table": "`user`" }, { @@ -764,8 +764,8 @@ "Name": "user", "Sharded": true }, - "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", + "FieldQuery": "select dt.c0 as id, weight_string(dt.c0) 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(c0) where 1 != 1", + "Query": "select dt.c0 as id, weight_string(dt.c0) from (select id from `user` union select id + 1 from `user` union select user_id from user_extra) as dt(c0)", "Table": "`user`, user_extra" } ] @@ -799,8 +799,8 @@ "Name": "user", "Sharded": true }, - "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", + "FieldQuery": "select dt.c0 as id, weight_string(dt.c0) from (select id from `user` where 1 != 1 union select id from music where 1 != 1) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as id, weight_string(dt.c0) from (select id from `user` union select id from music) as dt(c0)", "Table": "`user`, music" }, { @@ -810,8 +810,8 @@ "Name": "main", "Sharded": false }, - "FieldQuery": "select 1, weight_string(1) from unsharded where 1 != 1", - "Query": "select distinct 1, weight_string(1) from unsharded", + "FieldQuery": "select dt.c0 as `1`, weight_string(dt.c0) from (select 1 from unsharded where 1 != 1) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as `1`, weight_string(dt.c0) from (select distinct 1 from unsharded) as dt(c0)", "Table": "unsharded" } ] @@ -849,8 +849,8 @@ "Name": "user", "Sharded": true }, - "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", + "FieldQuery": "select dt.c0 as id, weight_string(dt.c0) from (select id from `user` where 1 != 1 union select 3 from dual where 1 != 1) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as id, weight_string(dt.c0) from (select id from `user` union select 3 from dual limit :__upper_limit) as dt(c0)", "Table": "`user`, dual" } ] @@ -951,8 +951,8 @@ "Name": "main", "Sharded": false }, - "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", + "FieldQuery": "select dt.c0 as col, weight_string(dt.c0) from (select col from unsharded where 1 != 1 union select col2 from unsharded where 1 != 1) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as col, weight_string(dt.c0) from (select col from unsharded union select col2 from unsharded) as dt(c0)", "Table": "unsharded" }, { @@ -962,8 +962,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select id, weight_string(id) from (select id from `user` where 1 != 1 union select col from user_extra where 1 != 1) as dt where 1 != 1", - "Query": "select id, weight_string(id) from (select id from `user` union select col from user_extra) as dt", + "FieldQuery": "select dt.c0 as id, weight_string(dt.c0) from (select id from `user` where 1 != 1 union select col from user_extra where 1 != 1) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as id, weight_string(dt.c0) from (select id from `user` union select col from user_extra) as dt(c0)", "Table": "`user`, user_extra" } ] @@ -1101,8 +1101,8 @@ "Name": "user", "Sharded": true }, - "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", + "FieldQuery": "select dt.c0 as id, weight_string(dt.c0) from (select id from `user` where 1 != 1 union select 3 from dual where 1 != 1) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as id, weight_string(dt.c0) from (select id from `user` union select 3 from dual) as dt(c0)", "Table": "`user`, dual" } ] @@ -1115,6 +1115,47 @@ ] } }, + { + "comment": "Conflicting column names in union", + "query": "select id, id from user union select id, bar from user_extra order by 1", + "plan": { + "QueryType": "SELECT", + "Original": "select id, id from user union select id, bar from user_extra order by 1", + "Instructions": { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "(0|2) ASC", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Distinct", + "Collations": [ + "(0:2)", + "(1:3)", + "2" + ], + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select dt.c0 as id, dt.c1 as id, weight_string(dt.c0), weight_string(dt.c1) from (select id, id from `user` where 1 != 1 union select id, bar from user_extra where 1 != 1) as dt(c0, c1) where 1 != 1", + "Query": "select dt.c0 as id, dt.c1 as id, weight_string(dt.c0), weight_string(dt.c1) from (select id, id from `user` union select id, bar from user_extra) as dt(c0, c1)", + "Table": "`user`, user_extra" + } + ] + } + ] + }, + "TablesUsed": [ + "user.user", + "user.user_extra" + ] + } + }, { "comment": "select 1 from (select id+42 as foo from user union select 1+id as foo from unsharded) as t", "query": "select 1 from (select id+42 as foo from user union select 1+id as foo from unsharded) as t", @@ -1143,8 +1184,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select id + 42 as foo, weight_string(id + 42) from `user` where 1 != 1", - "Query": "select distinct id + 42 as foo, weight_string(id + 42) from `user`", + "FieldQuery": "select dt.c0 as foo, weight_string(dt.c0) from (select id + 42 as foo from `user` where 1 != 1) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as foo, weight_string(dt.c0) from (select distinct id + 42 as foo from `user`) as dt(c0)", "Table": "`user`" }, { @@ -1154,8 +1195,8 @@ "Name": "main", "Sharded": false }, - "FieldQuery": "select 1 + id as foo, weight_string(1 + id) from unsharded where 1 != 1", - "Query": "select distinct 1 + id as foo, weight_string(1 + id) from unsharded", + "FieldQuery": "select dt.c0 as foo, weight_string(dt.c0) from (select 1 + id as foo from unsharded where 1 != 1) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as foo, weight_string(dt.c0) from (select distinct 1 + id as foo from unsharded) as dt(c0)", "Table": "unsharded" } ] @@ -1361,8 +1402,8 @@ "Name": "main", "Sharded": false }, - "FieldQuery": "select id, foo, bar, weight_string(id), weight_string(foo), weight_string(bar) from unsharded where 1 != 1", - "Query": "select distinct id, foo, bar, weight_string(id), weight_string(foo), weight_string(bar) from unsharded", + "FieldQuery": "select dt.c0 as id, dt.c1 as foo, dt.c2 as bar, weight_string(dt.c0), weight_string(dt.c1), weight_string(dt.c2) from (select id, foo, bar from unsharded where 1 != 1) as dt(c0, c1, c2) where 1 != 1", + "Query": "select dt.c0 as id, dt.c1 as foo, dt.c2 as bar, weight_string(dt.c0), weight_string(dt.c1), weight_string(dt.c2) from (select distinct id, foo, bar from unsharded) as dt(c0, c1, c2)", "Table": "unsharded" }, { @@ -1474,8 +1515,8 @@ "Name": "user", "Sharded": true }, - "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", + "FieldQuery": "select dt.c0 as bar, dt.c1 as baz, dt.c2 as toto, weight_string(dt.c0), weight_string(dt.c1), weight_string(dt.c2) from (select bar, baz, toto from music where 1 != 1 union select foo, foo, foo from `user` where 1 != 1) as dt(c0, c1, c2) where 1 != 1", + "Query": "select dt.c0 as bar, dt.c1 as baz, dt.c2 as toto, weight_string(dt.c0), weight_string(dt.c1), weight_string(dt.c2) from (select bar, baz, toto from music union select foo, foo, foo from `user`) as dt(c0, c1, c2)", "Table": "`user`, music" } ] @@ -1496,8 +1537,8 @@ "OperatorType": "Distinct", "Collations": [ "(0:3)", - "(1:3)", - "(2:3)" + "(1:4)", + "(2:5)" ], "ResultColumns": 3, "Inputs": [ @@ -1508,8 +1549,8 @@ "Name": "user", "Sharded": true }, - "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", + "FieldQuery": "select dt.c0 as foo, dt.c1 as foo, dt.c2 as foo, weight_string(dt.c0), weight_string(dt.c1), weight_string(dt.c2) from (select foo, foo, foo from `user` where 1 != 1 union select bar, baz, toto from music where 1 != 1) as dt(c0, c1, c2) where 1 != 1", + "Query": "select dt.c0 as foo, dt.c1 as foo, dt.c2 as foo, weight_string(dt.c0), weight_string(dt.c1), weight_string(dt.c2) from (select foo, foo, foo from `user` union select bar, baz, toto from music) as dt(c0, c1, c2)", "Table": "`user`, music" } ] @@ -1548,8 +1589,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select dt.foo, weight_string(dt.foo) from (select foo from `user` where 1 != 1 union select foo from `user` where 1 != 1) as dt where 1 != 1", - "Query": "select dt.foo, weight_string(dt.foo) from (select foo from `user` where bar = 12 union select foo from `user` where bar = 134) as dt", + "FieldQuery": "select dt.c0 as foo, weight_string(dt.c0) from (select foo from `user` where 1 != 1 union select foo from `user` where 1 != 1) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as foo, weight_string(dt.c0) from (select foo from `user` where bar = 12 union select foo from `user` where bar = 134) as dt(c0)", "Table": "`user`" } ] @@ -1567,8 +1608,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select dt.bar, weight_string(dt.bar) from (select bar from music where 1 != 1 union select bar from music where 1 != 1) as dt where 1 != 1", - "Query": "select dt.bar, weight_string(dt.bar) from (select bar from music where foo = 12 and bar = :t1_foo union select bar from music where foo = 1234 and bar = :t1_foo) as dt", + "FieldQuery": "select dt.c0 as bar, weight_string(dt.c0) from (select bar from music where 1 != 1 union select bar from music where 1 != 1) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as bar, weight_string(dt.c0) from (select bar from music where foo = 12 and bar = :t1_foo union select bar from music where foo = 1234 and bar = :t1_foo) as dt(c0)", "Table": "music" } ] @@ -1631,8 +1672,8 @@ "Name": "user", "Sharded": true }, - "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", + "FieldQuery": "select dt.c0 as col1, weight_string(dt.c0) from (select col1 from `user` where 1 != 1 union select 3 from `user` where 1 != 1) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as col1, weight_string(dt.c0) from (select col1 from `user` union select 3 from `user`) as dt(c0)", "Table": "`user`" } ] @@ -1662,8 +1703,8 @@ "Name": "user", "Sharded": true }, - "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", + "FieldQuery": "select dt.c0 as `3`, weight_string(dt.c0) from (select 3 from `user` where 1 != 1 union select col1 from `user` where 1 != 1) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as `3`, weight_string(dt.c0) from (select 3 from `user` union select col1 from `user`) as dt(c0)", "Table": "`user`" } ] @@ -1693,8 +1734,8 @@ "Name": "user", "Sharded": true }, - "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", + "FieldQuery": "select dt.c0 as `3`, weight_string(dt.c0) from (select 3 from `user` where 1 != 1 union select now() from `user` where 1 != 1) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as `3`, weight_string(dt.c0) from (select 3 from `user` union select now() from `user`) as dt(c0)", "Table": "`user`" } ] @@ -1724,8 +1765,8 @@ "Name": "user", "Sharded": true }, - "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", + "FieldQuery": "select dt.c0 as `now()`, weight_string(dt.c0) from (select now() from `user` where 1 != 1 union select 3 from `user` where 1 != 1) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as `now()`, weight_string(dt.c0) from (select now() from `user` union select 3 from `user`) as dt(c0)", "Table": "`user`" } ] @@ -1755,8 +1796,8 @@ "Name": "user", "Sharded": true }, - "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", + "FieldQuery": "select dt.c0 as `now()`, weight_string(dt.c0) from (select now() from `user` where 1 != 1 union select id from `user` where 1 != 1) as dt(c0) where 1 != 1", + "Query": "select dt.c0 as `now()`, weight_string(dt.c0) from (select now() from `user` union select id from `user`) as dt(c0)", "Table": "`user`" } ] diff --git a/go/vt/vtgate/semantics/early_rewriter.go b/go/vt/vtgate/semantics/early_rewriter.go index 10466234798..a75e9de9c25 100644 --- a/go/vt/vtgate/semantics/early_rewriter.go +++ b/go/vt/vtgate/semantics/early_rewriter.go @@ -681,7 +681,15 @@ func (r *earlyRewriter) fillInQualifiers(cursor *sqlparser.CopyOnWriteCursor) { if !found { panic("uh oh") } - tbl := r.tables.Tables[ts.TableOffset()] + offset := ts.TableOffset() + if offset < 0 { + // this is a column that is not coming from a table - it's an alias introduced in a SELECT expression + // Example: select (1+1) as foo from bar order by foo + // we don't want to add a qualifier to foo here + cursor.Replace(sqlparser.NewColName(col.Name.String())) + return + } + tbl := r.tables.Tables[offset] tblName, err := tbl.Name() if err != nil { panic(err) diff --git a/go/vt/vtgate/semantics/scoper.go b/go/vt/vtgate/semantics/scoper.go index 5901df51af7..3983032e069 100644 --- a/go/vt/vtgate/semantics/scoper.go +++ b/go/vt/vtgate/semantics/scoper.go @@ -288,6 +288,7 @@ func (s *scoper) createSpecialScopePostProjection(parent sqlparser.SQLNode) erro nScope.stmt = sel tableInfo = createVTableInfoForExpressions(sel.SelectExprs, nil /*needed for star expressions*/, s.org) nScope.tables = append(nScope.tables, tableInfo) + continue } thisTableInfo := createVTableInfoForExpressions(sel.SelectExprs, nil /*needed for star expressions*/, s.org) if len(tableInfo.cols) != len(thisTableInfo.cols) { @@ -297,7 +298,10 @@ func (s *scoper) createSpecialScopePostProjection(parent sqlparser.SQLNode) erro // at this stage, we don't store the actual dependencies, we only store the expressions. // only later will we walk the expression tree and figure out the deps. so, we need to create a // composite expression that contains all the expressions in the SELECTs that this UNION consists of - tableInfo.cols[i] = sqlparser.AndExpressions(col, thisTableInfo.cols[i]) + tableInfo.cols[i] = &sqlparser.AndExpr{ + Left: col, + Right: thisTableInfo.cols[i], + } } } diff --git a/go/vt/vtgate/semantics/semantic_state.go b/go/vt/vtgate/semantics/semantic_state.go index 66b0f99035a..f6f62a3eba5 100644 --- a/go/vt/vtgate/semantics/semantic_state.go +++ b/go/vt/vtgate/semantics/semantic_state.go @@ -994,3 +994,10 @@ func (st *SemTable) GetTargetTableSetForTableName(name sqlparser.TableName) (Tab } return "", vterrors.Errorf(vtrpcpb.Code_INTERNAL, "target table '%s' not found", sqlparser.String(name)) } + +// NewTableId creates a new table id and returns it. +func (st *SemTable) NewTableId() TableSet { + tableID := SingleTableSet(len(st.Tables)) + st.Tables = append(st.Tables, nil) + return tableID +}