Skip to content

Commit

Permalink
[release-18.0] bugfix: Columns alias expanding (#14935) (#14955)
Browse files Browse the repository at this point in the history
Signed-off-by: Andres Taylor <[email protected]>
Signed-off-by: Florent Poinsard <[email protected]>
Co-authored-by: Andres Taylor <[email protected]>
Co-authored-by: Florent Poinsard <[email protected]>
  • Loading branch information
3 people authored Jan 27, 2024
1 parent cc346e2 commit f4d1487
Show file tree
Hide file tree
Showing 20 changed files with 489 additions and 447 deletions.
8 changes: 4 additions & 4 deletions go/vt/vtgate/executor_select_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1026,7 +1026,7 @@ func TestLastInsertIDInVirtualTable(t *testing.T) {
_, err := executorExec(ctx, executor, session, "select * from (select last_insert_id()) as t", nil)
require.NoError(t, err)
wantQueries := []*querypb.BoundQuery{{
Sql: "select t.`last_insert_id()` from (select :__lastInsertId as `last_insert_id()` from dual) as t",
Sql: "select `last_insert_id()` from (select :__lastInsertId as `last_insert_id()` from dual) as t",
BindVariables: map[string]*querypb.BindVariable{"__lastInsertId": sqltypes.Uint64BindVariable(0)},
}}

Expand Down Expand Up @@ -4121,7 +4121,7 @@ func TestSelectView(t *testing.T) {
_, err = executor.Execute(context.Background(), nil, "TestSelectView", session, "select * from user_details_view", nil)
require.NoError(t, err)
wantQueries := []*querypb.BoundQuery{{
Sql: "select user_details_view.id, user_details_view.col from (select `user`.id, user_extra.col from `user`, user_extra where `user`.id = user_extra.user_id) as user_details_view",
Sql: "select id, col from (select `user`.id, user_extra.col from `user`, user_extra where `user`.id = user_extra.user_id) as user_details_view",
BindVariables: map[string]*querypb.BindVariable{},
}}
utils.MustMatch(t, wantQueries, sbc.Queries)
Expand All @@ -4130,7 +4130,7 @@ func TestSelectView(t *testing.T) {
_, err = executor.Execute(context.Background(), nil, "TestSelectView", session, "select * from user_details_view where id = 2", nil)
require.NoError(t, err)
wantQueries = []*querypb.BoundQuery{{
Sql: "select user_details_view.id, user_details_view.col from (select `user`.id, user_extra.col from `user`, user_extra where `user`.id = :id /* INT64 */ and `user`.id = user_extra.user_id) as user_details_view",
Sql: "select id, col from (select `user`.id, user_extra.col from `user`, user_extra where `user`.id = :id /* INT64 */ and `user`.id = user_extra.user_id) as user_details_view",
BindVariables: map[string]*querypb.BindVariable{
"id": sqltypes.Int64BindVariable(2),
},
Expand All @@ -4143,7 +4143,7 @@ func TestSelectView(t *testing.T) {
bvtg1, _ := sqltypes.BuildBindVariable([]int64{1, 2, 3, 4, 5})
bvals, _ := sqltypes.BuildBindVariable([]int64{1, 2})
wantQueries = []*querypb.BoundQuery{{
Sql: "select user_details_view.id, user_details_view.col from (select `user`.id, user_extra.col from `user`, user_extra where `user`.id in ::__vals and `user`.id = user_extra.user_id) as user_details_view",
Sql: "select id, col from (select `user`.id, user_extra.col from `user`, user_extra where `user`.id in ::__vals and `user`.id = user_extra.user_id) as user_details_view",
BindVariables: map[string]*querypb.BindVariable{
"vtg1": bvtg1,
"__vals": bvals,
Expand Down
15 changes: 5 additions & 10 deletions go/vt/vtgate/planbuilder/operators/aggregation_pushing.go
Original file line number Diff line number Diff line change
Expand Up @@ -432,17 +432,12 @@ var errAbortAggrPushing = fmt.Errorf("abort aggregation pushing")
func addColumnsFromLHSInJoinPredicates(ctx *plancontext.PlanningContext, rootAggr *Aggregator, join *ApplyJoin, lhs *joinPusher) error {
for _, pred := range join.JoinPredicates {
for _, bve := range pred.LHSExprs {
expr := bve.Expr
wexpr, err := rootAggr.QP.GetSimplifiedExpr(ctx, expr)
if err != nil {
return err
}
idx, found := canReuseColumn(ctx, lhs.pushed.Columns, expr, extractExpr)
idx, found := canReuseColumn(ctx, lhs.pushed.Columns, bve.Expr, extractExpr)
if !found {
idx = len(lhs.pushed.Columns)
lhs.pushed.Columns = append(lhs.pushed.Columns, aeWrap(expr))
lhs.pushed.Columns = append(lhs.pushed.Columns, aeWrap(bve.Expr))
}
_, found = canReuseColumn(ctx, lhs.pushed.Grouping, wexpr, func(by GroupBy) sqlparser.Expr {
_, found = canReuseColumn(ctx, lhs.pushed.Grouping, bve.Expr, func(by GroupBy) sqlparser.Expr {
return by.SimplifiedExpr
})

Expand All @@ -451,8 +446,8 @@ func addColumnsFromLHSInJoinPredicates(ctx *plancontext.PlanningContext, rootAgg
}

lhs.pushed.Grouping = append(lhs.pushed.Grouping, GroupBy{
Inner: expr,
SimplifiedExpr: wexpr,
Inner: bve.Expr,
SimplifiedExpr: bve.Expr,
ColOffset: idx,
WSOffset: -1,
})
Expand Down
11 changes: 3 additions & 8 deletions go/vt/vtgate/planbuilder/operators/distinct.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,16 +52,11 @@ func (d *Distinct) planOffsets(ctx *plancontext.PlanningContext) error {
return err
}
for idx, col := range columns {
e, err := d.QP.GetSimplifiedExpr(ctx, col.Expr)
if err != nil {
// ambiguous columns are not a problem for DISTINCT
e = col.Expr
}
var wsCol *int
typ, coll, _ := ctx.SemTable.TypeForExpr(e)
typ, coll, _ := ctx.SemTable.TypeForExpr(col.Expr)

if ctx.SemTable.NeedsWeightString(e) {
offset, err := d.Source.AddColumn(ctx, true, false, aeWrap(weightStringFor(e)))
if ctx.SemTable.NeedsWeightString(col.Expr) {
offset, err := d.Source.AddColumn(ctx, true, false, aeWrap(weightStringFor(col.Expr)))
if err != nil {
return err
}
Expand Down
147 changes: 12 additions & 135 deletions go/vt/vtgate/planbuilder/operators/queryprojection.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"fmt"
"io"
"slices"
"sort"
"strings"

"vitess.io/vitess/go/mysql/collations"
Expand Down Expand Up @@ -216,9 +215,7 @@ func createQPFromSelect(ctx *plancontext.PlanningContext, sel *sqlparser.Select)
if err := qp.addGroupBy(ctx, sel.GroupBy); err != nil {
return nil, err
}
if err := qp.addOrderBy(ctx, sel.OrderBy); err != nil {
return nil, err
}
qp.addOrderBy(ctx, sel.OrderBy)
if !qp.HasAggr && sel.Having != nil {
qp.HasAggr = containsAggr(sel.Having.Expr)
}
Expand Down Expand Up @@ -340,10 +337,7 @@ func createQPFromUnion(ctx *plancontext.PlanningContext, union *sqlparser.Union)
return nil, err
}

err = qp.addOrderBy(ctx, union.OrderBy)
if err != nil {
return nil, err
}
qp.addOrderBy(ctx, union.OrderBy)

return qp, nil
}
Expand All @@ -365,29 +359,24 @@ func (es *expressionSet) add(ctx *plancontext.PlanningContext, e sqlparser.Expr)
return true
}

func (qp *QueryProjection) addOrderBy(ctx *plancontext.PlanningContext, orderBy sqlparser.OrderBy) error {
func (qp *QueryProjection) addOrderBy(ctx *plancontext.PlanningContext, orderBy sqlparser.OrderBy) {
canPushSorting := true
es := &expressionSet{}
for _, order := range orderBy {
simpleExpr, err := qp.GetSimplifiedExpr(ctx, order.Expr)
if err != nil {
return err
}
if sqlparser.IsNull(simpleExpr) {
if sqlparser.IsNull(order.Expr) {
// ORDER BY null can safely be ignored
continue
}
if !es.add(ctx, simpleExpr) {
if !es.add(ctx, order.Expr) {
continue
}
qp.OrderExprs = append(qp.OrderExprs, ops.OrderBy{
Inner: sqlparser.CloneRefOfOrder(order),
SimplifiedExpr: simpleExpr,
SimplifiedExpr: order.Expr,
})
canPushSorting = canPushSorting && !containsAggr(simpleExpr)
canPushSorting = canPushSorting && !containsAggr(order.Expr)
}
qp.CanPushSorting = canPushSorting
return nil
}

func (qp *QueryProjection) calculateDistinct(ctx *plancontext.PlanningContext) error {
Expand Down Expand Up @@ -436,20 +425,16 @@ func (qp *QueryProjection) addGroupBy(ctx *plancontext.PlanningContext, groupBy
es := &expressionSet{}
for _, group := range groupBy {
selectExprIdx, aliasExpr := qp.FindSelectExprIndexForExpr(ctx, group)
simpleExpr, err := qp.GetSimplifiedExpr(ctx, group)
if err != nil {
return err
}

if err = checkForInvalidGroupingExpressions(simpleExpr); err != nil {
if err := checkForInvalidGroupingExpressions(group); err != nil {
return err
}

if !es.add(ctx, simpleExpr) {
if !es.add(ctx, group) {
continue
}

groupBy := NewGroupBy(group, simpleExpr, aliasExpr)
groupBy := NewGroupBy(group, group, aliasExpr)
groupBy.InnerIndex = selectExprIdx

qp.groupByExprs = append(qp.groupByExprs, groupBy)
Expand All @@ -471,64 +456,6 @@ func (qp *QueryProjection) isExprInGroupByExprs(ctx *plancontext.PlanningContext
return false
}

// GetSimplifiedExpr takes an expression used in ORDER BY or GROUP BY, and returns an expression that is simpler to evaluate
func (qp *QueryProjection) GetSimplifiedExpr(ctx *plancontext.PlanningContext, e sqlparser.Expr) (found sqlparser.Expr, err error) {
if qp == nil {
return e, nil
}
// If the ORDER BY is against a column alias, we need to remember the expression
// behind the alias. The weightstring(.) calls needs to be done against that expression and not the alias.
// Eg - select music.foo as bar, weightstring(music.foo) from music order by bar

in, isColName := e.(*sqlparser.ColName)
if !(isColName && in.Qualifier.IsEmpty()) {
// we are only interested in unqualified column names. if it's not a column name and not unqualified, we're done
return e, nil
}

check := func(e sqlparser.Expr) error {
if found != nil && !ctx.SemTable.EqualsExprWithDeps(found, e) {
return &semantics.AmbiguousColumnError{Column: sqlparser.String(in)}
}
found = e
return nil
}

for _, selectExpr := range qp.SelectExprs {
ae, ok := selectExpr.Col.(*sqlparser.AliasedExpr)
if !ok {
continue
}
aliased := !ae.As.IsEmpty()
if aliased {
if in.Name.Equal(ae.As) {
err = check(ae.Expr)
if err != nil {
return nil, err
}
}
} else {
seCol, ok := ae.Expr.(*sqlparser.ColName)
if !ok {
continue
}
if seCol.Name.Equal(in.Name) {
// If the column name matches, we have a match, even if the table name is not listed
err = check(ae.Expr)
if err != nil {
return nil, err
}
}
}
}

if found == nil {
found = e
}

return found, nil
}

// toString should only be used for tests
func (qp *QueryProjection) toString() string {
type output struct {
Expand Down Expand Up @@ -794,46 +721,6 @@ func (qp *QueryProjection) FindSelectExprIndexForExpr(ctx *plancontext.PlanningC
return nil, nil
}

// OldAlignGroupByAndOrderBy TODO Remove once all of horizon planning is done on the operators
func (qp *QueryProjection) OldAlignGroupByAndOrderBy(ctx *plancontext.PlanningContext) {
// The ORDER BY can be performed before the OA

var newGrouping []GroupBy
if len(qp.OrderExprs) == 0 {
// The query didn't ask for any particular order, so we are free to add arbitrary ordering.
// We'll align the grouping and ordering by the output columns
newGrouping = qp.GetGrouping()
SortGrouping(newGrouping)
for _, groupBy := range newGrouping {
qp.OrderExprs = append(qp.OrderExprs, groupBy.AsOrderBy())
}
} else {
// Here we align the GROUP BY and ORDER BY.
// First step is to make sure that the GROUP BY is in the same order as the ORDER BY
used := make([]bool, len(qp.groupByExprs))
for _, orderExpr := range qp.OrderExprs {
for i, groupingExpr := range qp.groupByExprs {
if !used[i] && ctx.SemTable.EqualsExpr(groupingExpr.SimplifiedExpr, orderExpr.SimplifiedExpr) {
newGrouping = append(newGrouping, groupingExpr)
used[i] = true
}
}
}
if len(newGrouping) != len(qp.groupByExprs) {
// we are missing some groupings. We need to add them both to the new groupings list, but also to the ORDER BY
for i, added := range used {
if !added {
groupBy := qp.groupByExprs[i]
newGrouping = append(newGrouping, groupBy)
qp.OrderExprs = append(qp.OrderExprs, groupBy.AsOrderBy())
}
}
}
}

qp.groupByExprs = newGrouping
}

// AlignGroupByAndOrderBy aligns the group by and order by columns, so they are in the same order
// The GROUP BY clause is a set - the order between the elements does not make any difference,
// so we can simply re-arrange the column order
Expand Down Expand Up @@ -903,18 +790,14 @@ func (qp *QueryProjection) useGroupingOverDistinct(ctx *plancontext.PlanningCont
// not an alias Expr, cannot continue forward.
return false, nil
}
sExpr, err := qp.GetSimplifiedExpr(ctx, ae.Expr)
if err != nil {
return false, err
}
// check if the grouping already exists on that column.
found := slices.IndexFunc(qp.groupByExprs, func(gb GroupBy) bool {
return ctx.SemTable.EqualsExprWithDeps(gb.SimplifiedExpr, sExpr)
return ctx.SemTable.EqualsExprWithDeps(gb.SimplifiedExpr, ae.Expr)
})
if found != -1 {
continue
}
groupBy := NewGroupBy(ae.Expr, sExpr, ae)
groupBy := NewGroupBy(ae.Expr, ae.Expr, ae)
selectExprIdx := idx
groupBy.InnerIndex = &selectExprIdx

Expand All @@ -938,12 +821,6 @@ func checkForInvalidGroupingExpressions(expr sqlparser.Expr) error {
}, expr)
}

func SortGrouping(a []GroupBy) {
sort.Slice(a, func(i, j int) bool {
return CompareRefInt(a[i].InnerIndex, a[j].InnerIndex)
})
}

// CompareRefInt compares two references of integers.
// In case either one is nil, it is considered to be smaller
func CompareRefInt(a *int, b *int) bool {
Expand Down
11 changes: 9 additions & 2 deletions go/vt/vtgate/planbuilder/operators/queryprojection_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ func TestQP(t *testing.T) {
{
sql: "select 1, count(1) from user order by 1",
expOrder: []ops.OrderBy{
{Inner: &sqlparser.Order{Expr: sqlparser.NewIntLiteral("1")}, SimplifiedExpr: sqlparser.NewIntLiteral("1")},
{Inner: &sqlparser.Order{Expr: sqlparser.NewStrLiteral("")}, SimplifiedExpr: sqlparser.NewStrLiteral("")},
},
},
{
Expand All @@ -61,7 +61,14 @@ func TestQP(t *testing.T) {
sql: "SELECT CONCAT(last_name,', ',first_name) AS full_name FROM mytable ORDER BY full_name", // alias in order not supported
expOrder: []ops.OrderBy{
{
Inner: &sqlparser.Order{Expr: sqlparser.NewColName("full_name")},
Inner: &sqlparser.Order{Expr: &sqlparser.FuncExpr{
Name: sqlparser.NewIdentifierCI("CONCAT"),
Exprs: sqlparser.SelectExprs{
&sqlparser.AliasedExpr{Expr: sqlparser.NewColName("last_name")},
&sqlparser.AliasedExpr{Expr: sqlparser.NewStrLiteral(", ")},
&sqlparser.AliasedExpr{Expr: sqlparser.NewColName("first_name")},
},
}},
SimplifiedExpr: &sqlparser.FuncExpr{
Name: sqlparser.NewIdentifierCI("CONCAT"),
Exprs: sqlparser.SelectExprs{
Expand Down
Loading

0 comments on commit f4d1487

Please sign in to comment.