You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
 
 

474 lines
18 KiB

// Copyright 2016 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
// // Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package core
import (
"context"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/expression/aggregation"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/types"
)
type aggregationPushDownSolver struct {
aggregationEliminateChecker
}
// isDecomposable checks if an aggregate function is decomposable. An aggregation function $F$ is decomposable
// if there exist aggregation functions F_1 and F_2 such that F(S_1 union all S_2) = F_2(F_1(S_1),F_1(S_2)),
// where S_1 and S_2 are two sets of values. We call S_1 and S_2 partial groups.
// It's easy to see that max, min, first row is decomposable, no matter whether it's distinct, but sum(distinct) and
// count(distinct) is not.
// Currently we don't support avg and concat.
func (a *aggregationPushDownSolver) isDecomposableWithJoin(fun *aggregation.AggFuncDesc) bool {
if len(fun.OrderByItems) > 0 {
return false
}
switch fun.Name {
case ast.AggFuncAvg, ast.AggFuncGroupConcat, ast.AggFuncVarPop, ast.AggFuncJsonObjectAgg, ast.AggFuncStddevPop, ast.AggFuncVarSamp, ast.AggFuncStddevSamp, ast.AggFuncApproxPercentile:
// TODO: Support avg push down.
return false
case ast.AggFuncMax, ast.AggFuncMin, ast.AggFuncFirstRow:
return true
case ast.AggFuncSum, ast.AggFuncCount:
return !fun.HasDistinct
default:
return false
}
}
func (a *aggregationPushDownSolver) isDecomposableWithUnion(fun *aggregation.AggFuncDesc) bool {
if len(fun.OrderByItems) > 0 {
return false
}
switch fun.Name {
case ast.AggFuncGroupConcat, ast.AggFuncVarPop, ast.AggFuncJsonObjectAgg, ast.AggFuncApproxPercentile:
return false
case ast.AggFuncMax, ast.AggFuncMin, ast.AggFuncFirstRow:
return true
case ast.AggFuncSum, ast.AggFuncCount, ast.AggFuncAvg, ast.AggFuncApproxCountDistinct:
return true
default:
return false
}
}
// getAggFuncChildIdx gets which children it belongs to, 0 stands for left, 1 stands for right, -1 stands for both.
func (a *aggregationPushDownSolver) getAggFuncChildIdx(aggFunc *aggregation.AggFuncDesc, schema *expression.Schema) int {
fromLeft, fromRight := false, false
var cols []*expression.Column
cols = expression.ExtractColumnsFromExpressions(cols, aggFunc.Args, nil)
for _, col := range cols {
if schema.Contains(col) {
fromLeft = true
} else {
fromRight = true
}
}
if fromLeft && fromRight {
return -1
} else if fromLeft {
return 0
}
return 1
}
// collectAggFuncs collects all aggregate functions and splits them into two parts: "leftAggFuncs" and "rightAggFuncs" whose
// arguments are all from left child or right child separately. If some aggregate functions have the arguments that have
// columns both from left and right children, the whole aggregation is forbidden to push down.
func (a *aggregationPushDownSolver) collectAggFuncs(agg *LogicalAggregation, join *LogicalJoin) (valid bool, leftAggFuncs, rightAggFuncs []*aggregation.AggFuncDesc) {
valid = true
leftChild := join.children[0]
for _, aggFunc := range agg.AggFuncs {
if !a.isDecomposableWithJoin(aggFunc) {
return false, nil, nil
}
index := a.getAggFuncChildIdx(aggFunc, leftChild.Schema())
switch index {
case 0:
leftAggFuncs = append(leftAggFuncs, aggFunc)
case 1:
rightAggFuncs = append(rightAggFuncs, aggFunc)
default:
return false, nil, nil
}
}
return
}
// collectGbyCols collects all columns from gby-items and join-conditions and splits them into two parts: "leftGbyCols" and
// "rightGbyCols". e.g. For query "SELECT SUM(B.id) FROM A, B WHERE A.c1 = B.c1 AND A.c2 != B.c2 GROUP BY B.c3" , the optimized
// query should be "SELECT SUM(B.agg) FROM A, (SELECT SUM(id) as agg, c1, c2, c3 FROM B GROUP BY id, c1, c2, c3) as B
// WHERE A.c1 = B.c1 AND A.c2 != B.c2 GROUP BY B.c3". As you see, all the columns appearing in join-conditions should be
// treated as group by columns in join subquery.
func (a *aggregationPushDownSolver) collectGbyCols(agg *LogicalAggregation, join *LogicalJoin) (leftGbyCols, rightGbyCols []*expression.Column) {
leftChild := join.children[0]
ctx := agg.ctx
for _, gbyExpr := range agg.GroupByItems {
cols := expression.ExtractColumns(gbyExpr)
for _, col := range cols {
if leftChild.Schema().Contains(col) {
leftGbyCols = append(leftGbyCols, col)
} else {
rightGbyCols = append(rightGbyCols, col)
}
}
}
// extract equal conditions
for _, eqFunc := range join.EqualConditions {
leftGbyCols = a.addGbyCol(ctx, leftGbyCols, eqFunc.GetArgs()[0].(*expression.Column))
rightGbyCols = a.addGbyCol(ctx, rightGbyCols, eqFunc.GetArgs()[1].(*expression.Column))
}
for _, leftCond := range join.LeftConditions {
cols := expression.ExtractColumns(leftCond)
leftGbyCols = a.addGbyCol(ctx, leftGbyCols, cols...)
}
for _, rightCond := range join.RightConditions {
cols := expression.ExtractColumns(rightCond)
rightGbyCols = a.addGbyCol(ctx, rightGbyCols, cols...)
}
for _, otherCond := range join.OtherConditions {
cols := expression.ExtractColumns(otherCond)
for _, col := range cols {
if leftChild.Schema().Contains(col) {
leftGbyCols = a.addGbyCol(ctx, leftGbyCols, col)
} else {
rightGbyCols = a.addGbyCol(ctx, rightGbyCols, col)
}
}
}
return
}
func (a *aggregationPushDownSolver) splitAggFuncsAndGbyCols(agg *LogicalAggregation, join *LogicalJoin) (valid bool,
leftAggFuncs, rightAggFuncs []*aggregation.AggFuncDesc,
leftGbyCols, rightGbyCols []*expression.Column) {
valid, leftAggFuncs, rightAggFuncs = a.collectAggFuncs(agg, join)
if !valid {
return
}
leftGbyCols, rightGbyCols = a.collectGbyCols(agg, join)
return
}
// addGbyCol adds a column to gbyCols. If a group by column has existed, it will not be added repeatedly.
func (a *aggregationPushDownSolver) addGbyCol(ctx sessionctx.Context, gbyCols []*expression.Column, cols ...*expression.Column) []*expression.Column {
for _, c := range cols {
duplicate := false
for _, gbyCol := range gbyCols {
if c.Equal(ctx, gbyCol) {
duplicate = true
break
}
}
if !duplicate {
gbyCols = append(gbyCols, c)
}
}
return gbyCols
}
// checkValidJoin checks if this join should be pushed across.
func (a *aggregationPushDownSolver) checkValidJoin(join *LogicalJoin) bool {
return join.JoinType == InnerJoin || join.JoinType == LeftOuterJoin || join.JoinType == RightOuterJoin
}
// decompose splits an aggregate function to two parts: a final mode function and a partial mode function. Currently
// there are no differences between partial mode and complete mode, so we can confuse them.
func (a *aggregationPushDownSolver) decompose(ctx sessionctx.Context, aggFunc *aggregation.AggFuncDesc, schema *expression.Schema) ([]*aggregation.AggFuncDesc, *expression.Schema) {
// Result is a slice because avg should be decomposed to sum and count. Currently we don't process this case.
result := []*aggregation.AggFuncDesc{aggFunc.Clone()}
for _, aggFunc := range result {
schema.Append(&expression.Column{
UniqueID: ctx.GetSessionVars().AllocPlanColumnID(),
RetType: aggFunc.RetTp,
})
}
aggFunc.Args = expression.Column2Exprs(schema.Columns[schema.Len()-len(result):])
aggFunc.Mode = aggregation.FinalMode
return result, schema
}
// tryToPushDownAgg tries to push down an aggregate function into a join path. If all aggFuncs are first row, we won't
// process it temporarily. If not, We will add additional group by columns and first row functions. We make a new aggregation operator.
// If the pushed aggregation is grouped by unique key, it's no need to push it down.
func (a *aggregationPushDownSolver) tryToPushDownAgg(aggFuncs []*aggregation.AggFuncDesc, gbyCols []*expression.Column, join *LogicalJoin, childIdx int, aggHints aggHintInfo, blockOffset int) (_ LogicalPlan, err error) {
child := join.children[childIdx]
if aggregation.IsAllFirstRow(aggFuncs) {
return child, nil
}
// If the join is multiway-join, we forbid pushing down.
if _, ok := join.children[childIdx].(*LogicalJoin); ok {
return child, nil
}
tmpSchema := expression.NewSchema(gbyCols...)
for _, key := range child.Schema().Keys {
if tmpSchema.ColumnsIndices(key) != nil {
return child, nil
}
}
agg, err := a.makeNewAgg(join.ctx, aggFuncs, gbyCols, aggHints, blockOffset)
if err != nil {
return nil, err
}
agg.SetChildren(child)
// If agg has no group-by item, it will return a default value, which may cause some bugs.
// So here we add a group-by item forcely.
if len(agg.GroupByItems) == 0 {
agg.GroupByItems = []expression.Expression{&expression.Constant{
Value: types.NewDatum(0),
RetType: types.NewFieldType(mysql.TypeLong)}}
}
if (childIdx == 0 && join.JoinType == RightOuterJoin) || (childIdx == 1 && join.JoinType == LeftOuterJoin) {
var existsDefaultValues bool
join.DefaultValues, existsDefaultValues = a.getDefaultValues(agg)
if !existsDefaultValues {
return child, nil
}
}
return agg, nil
}
func (a *aggregationPushDownSolver) getDefaultValues(agg *LogicalAggregation) ([]types.Datum, bool) {
defaultValues := make([]types.Datum, 0, agg.Schema().Len())
for _, aggFunc := range agg.AggFuncs {
value, existsDefaultValue := aggFunc.EvalNullValueInOuterJoin(agg.ctx, agg.children[0].Schema())
if !existsDefaultValue {
return nil, false
}
defaultValues = append(defaultValues, value)
}
return defaultValues, true
}
func (a *aggregationPushDownSolver) checkAnyCountAndSum(aggFuncs []*aggregation.AggFuncDesc) bool {
for _, fun := range aggFuncs {
if fun.Name == ast.AggFuncSum || fun.Name == ast.AggFuncCount {
return true
}
}
return false
}
// TODO:
// 1. https://github.com/pingcap/tidb/issues/16355, push avg & distinct functions across join
// 2. remove this method and use splitPartialAgg instead for clean code.
func (a *aggregationPushDownSolver) makeNewAgg(ctx sessionctx.Context, aggFuncs []*aggregation.AggFuncDesc, gbyCols []*expression.Column, aggHints aggHintInfo, blockOffset int) (*LogicalAggregation, error) {
agg := LogicalAggregation{
GroupByItems: expression.Column2Exprs(gbyCols),
groupByCols: gbyCols,
aggHints: aggHints,
}.Init(ctx, blockOffset)
aggLen := len(aggFuncs) + len(gbyCols)
newAggFuncDescs := make([]*aggregation.AggFuncDesc, 0, aggLen)
schema := expression.NewSchema(make([]*expression.Column, 0, aggLen)...)
for _, aggFunc := range aggFuncs {
var newFuncs []*aggregation.AggFuncDesc
newFuncs, schema = a.decompose(ctx, aggFunc, schema)
newAggFuncDescs = append(newAggFuncDescs, newFuncs...)
}
for _, gbyCol := range gbyCols {
firstRow, err := aggregation.NewAggFuncDesc(agg.ctx, ast.AggFuncFirstRow, []expression.Expression{gbyCol}, false)
if err != nil {
return nil, err
}
newCol, _ := gbyCol.Clone().(*expression.Column)
newCol.RetType = firstRow.RetTp
newAggFuncDescs = append(newAggFuncDescs, firstRow)
schema.Append(newCol)
}
agg.AggFuncs = newAggFuncDescs
agg.SetSchema(schema)
// TODO: Add a Projection if any argument of aggregate funcs or group by items are scalar functions.
// agg.buildProjectionIfNecessary()
return agg, nil
}
func (a *aggregationPushDownSolver) splitPartialAgg(agg *LogicalAggregation) (pushedAgg *LogicalAggregation) {
partial, final, _ := BuildFinalModeAggregation(agg.ctx, &AggInfo{
AggFuncs: agg.AggFuncs,
GroupByItems: agg.GroupByItems,
Schema: agg.schema,
}, false)
agg.SetSchema(final.Schema)
agg.AggFuncs = final.AggFuncs
agg.GroupByItems = final.GroupByItems
agg.collectGroupByColumns()
pushedAgg = LogicalAggregation{
AggFuncs: partial.AggFuncs,
GroupByItems: partial.GroupByItems,
aggHints: agg.aggHints,
}.Init(agg.ctx, agg.blockOffset)
pushedAgg.SetSchema(partial.Schema)
pushedAgg.collectGroupByColumns()
return
}
// pushAggCrossUnion will try to push the agg down to the union. If the new aggregation's group-by columns doesn't contain unique key.
// We will return the new aggregation. Otherwise we will transform the aggregation to projection.
func (a *aggregationPushDownSolver) pushAggCrossUnion(agg *LogicalAggregation, unionSchema *expression.Schema, unionChild LogicalPlan) (LogicalPlan, error) {
ctx := agg.ctx
newAgg := LogicalAggregation{
AggFuncs: make([]*aggregation.AggFuncDesc, 0, len(agg.AggFuncs)),
GroupByItems: make([]expression.Expression, 0, len(agg.GroupByItems)),
aggHints: agg.aggHints,
}.Init(ctx, agg.blockOffset)
newAgg.SetSchema(agg.schema.Clone())
for _, aggFunc := range agg.AggFuncs {
newAggFunc := aggFunc.Clone()
newArgs := make([]expression.Expression, 0, len(newAggFunc.Args))
for _, arg := range newAggFunc.Args {
newArgs = append(newArgs, expression.ColumnSubstitute(arg, unionSchema, expression.Column2Exprs(unionChild.Schema().Columns)))
}
newAggFunc.Args = newArgs
newAgg.AggFuncs = append(newAgg.AggFuncs, newAggFunc)
}
for _, gbyExpr := range agg.GroupByItems {
newExpr := expression.ColumnSubstitute(gbyExpr, unionSchema, expression.Column2Exprs(unionChild.Schema().Columns))
newAgg.GroupByItems = append(newAgg.GroupByItems, newExpr)
// TODO: if there is a duplicated first_row function, we can delete it.
firstRow, err := aggregation.NewAggFuncDesc(agg.ctx, ast.AggFuncFirstRow, []expression.Expression{gbyExpr}, false)
if err != nil {
return nil, err
}
newAgg.AggFuncs = append(newAgg.AggFuncs, firstRow)
}
newAgg.collectGroupByColumns()
tmpSchema := expression.NewSchema(newAgg.groupByCols...)
// e.g. Union distinct will add a aggregation like `select join_agg_0, join_agg_1, join_agg_2 from t group by a, b, c` above UnionAll.
// And the pushed agg will be something like `select a, b, c, a, b, c from t group by a, b, c`. So if we just return child as join does,
// this will cause error during executor phase.
for _, key := range unionChild.Schema().Keys {
if tmpSchema.ColumnsIndices(key) != nil {
if ok, proj := ConvertAggToProj(newAgg, newAgg.schema); ok {
proj.SetChildren(unionChild)
return proj, nil
}
break
}
}
newAgg.SetChildren(unionChild)
return newAgg, nil
}
func (a *aggregationPushDownSolver) optimize(ctx context.Context, p LogicalPlan) (LogicalPlan, error) {
return a.aggPushDown(p)
}
func (a *aggregationPushDownSolver) tryAggPushDownForUnion(union *LogicalUnionAll, agg *LogicalAggregation) error {
for _, aggFunc := range agg.AggFuncs {
if !a.isDecomposableWithUnion(aggFunc) {
return nil
}
}
pushedAgg := a.splitPartialAgg(agg)
newChildren := make([]LogicalPlan, 0, len(union.Children()))
for _, child := range union.Children() {
newChild, err := a.pushAggCrossUnion(pushedAgg, union.Schema(), child)
if err != nil {
return err
}
newChildren = append(newChildren, newChild)
}
union.SetSchema(expression.NewSchema(newChildren[0].Schema().Columns...))
union.SetChildren(newChildren...)
return nil
}
// aggPushDown tries to push down aggregate functions to join paths.
func (a *aggregationPushDownSolver) aggPushDown(p LogicalPlan) (_ LogicalPlan, err error) {
if agg, ok := p.(*LogicalAggregation); ok {
proj := a.tryToEliminateAggregation(agg)
if proj != nil {
p = proj
} else {
child := agg.children[0]
if join, ok1 := child.(*LogicalJoin); ok1 && a.checkValidJoin(join) && p.SCtx().GetSessionVars().AllowAggPushDown {
if valid, leftAggFuncs, rightAggFuncs, leftGbyCols, rightGbyCols := a.splitAggFuncsAndGbyCols(agg, join); valid {
var lChild, rChild LogicalPlan
// If there exist count or sum functions in left join path, we can't push any
// aggregate function into right join path.
rightInvalid := a.checkAnyCountAndSum(leftAggFuncs)
leftInvalid := a.checkAnyCountAndSum(rightAggFuncs)
if rightInvalid {
rChild = join.children[1]
} else {
rChild, err = a.tryToPushDownAgg(rightAggFuncs, rightGbyCols, join, 1, agg.aggHints, agg.blockOffset)
if err != nil {
return nil, err
}
}
if leftInvalid {
lChild = join.children[0]
} else {
lChild, err = a.tryToPushDownAgg(leftAggFuncs, leftGbyCols, join, 0, agg.aggHints, agg.blockOffset)
if err != nil {
return nil, err
}
}
join.SetChildren(lChild, rChild)
join.SetSchema(expression.MergeSchema(lChild.Schema(), rChild.Schema()))
buildKeyInfo(join)
proj := a.tryToEliminateAggregation(agg)
if proj != nil {
p = proj
}
}
} else if proj, ok1 := child.(*LogicalProjection); ok1 && p.SCtx().GetSessionVars().AllowAggPushDown {
// TODO: This optimization is not always reasonable. We have not supported pushing projection to kv layer yet,
// so we must do this optimization.
for i, gbyItem := range agg.GroupByItems {
agg.GroupByItems[i] = expression.ColumnSubstitute(gbyItem, proj.schema, proj.Exprs)
}
agg.collectGroupByColumns()
for _, aggFunc := range agg.AggFuncs {
newArgs := make([]expression.Expression, 0, len(aggFunc.Args))
for _, arg := range aggFunc.Args {
newArgs = append(newArgs, expression.ColumnSubstitute(arg, proj.schema, proj.Exprs))
}
aggFunc.Args = newArgs
}
projChild := proj.children[0]
agg.SetChildren(projChild)
} else if union, ok1 := child.(*LogicalUnionAll); ok1 && p.SCtx().GetSessionVars().AllowAggPushDown {
err := a.tryAggPushDownForUnion(union, agg)
if err != nil {
return nil, err
}
} else if union, ok1 := child.(*LogicalPartitionUnionAll); ok1 {
err := a.tryAggPushDownForUnion(&union.LogicalUnionAll, agg)
if err != nil {
return nil, err
}
}
}
}
newChildren := make([]LogicalPlan, 0, len(p.Children()))
for _, child := range p.Children() {
newChild, err := a.aggPushDown(child)
if err != nil {
return nil, err
}
newChildren = append(newChildren, newChild)
}
p.SetChildren(newChildren...)
return p, nil
}
func (*aggregationPushDownSolver) name() string {
return "aggregation_push_down"
}