forked from pingcap/tidb
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathrule_aggregation_elimination.go
167 lines (153 loc) · 6.31 KB
/
rule_aggregation_elimination.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
// Copyright 2018 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"
"math"
"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 aggregationEliminator struct {
aggregationEliminateChecker
}
type aggregationEliminateChecker struct {
}
// tryToEliminateAggregation will eliminate aggregation grouped by unique key.
// e.g. select min(b) from t group by a. If a is a unique key, then this sql is equal to `select b from t group by a`.
// For count(expr), sum(expr), avg(expr), count(distinct expr, [expr...]) we may need to rewrite the expr. Details are shown below.
// If we can eliminate agg successful, we return a projection. Else we return a nil pointer.
func (a *aggregationEliminateChecker) tryToEliminateAggregation(agg *LogicalAggregation) *LogicalProjection {
for _, af := range agg.AggFuncs {
// TODO(issue #9968): Actually, we can rewrite GROUP_CONCAT when all the
// arguments it accepts are promised to be NOT-NULL.
// When it accepts only 1 argument, we can extract this argument into a
// projection.
// When it accepts multiple arguments, we can wrap the arguments with a
// function CONCAT_WS and extract this function into a projection.
// BUT, GROUP_CONCAT should truncate the final result according to the
// system variable `group_concat_max_len`. To ensure the correctness of
// the result, we close the elimination of GROUP_CONCAT here.
if af.Name == ast.AggFuncGroupConcat {
return nil
}
}
schemaByGroupby := expression.NewSchema(agg.GetGroupByCols()...)
coveredByUniqueKey := false
for _, key := range agg.children[0].Schema().Keys {
if schemaByGroupby.ColumnsIndices(key) != nil {
coveredByUniqueKey = true
break
}
}
if coveredByUniqueKey {
// GroupByCols has unique key, so this aggregation can be removed.
if ok, proj := ConvertAggToProj(agg, agg.schema); ok {
proj.SetChildren(agg.children[0])
return proj
}
}
return nil
}
// ConvertAggToProj convert aggregation to projection.
func ConvertAggToProj(agg *LogicalAggregation, schema *expression.Schema) (bool, *LogicalProjection) {
proj := LogicalProjection{
Exprs: make([]expression.Expression, 0, len(agg.AggFuncs)),
}.Init(agg.ctx, agg.blockOffset)
for _, fun := range agg.AggFuncs {
ok, expr := rewriteExpr(agg.ctx, fun)
if !ok {
return false, nil
}
proj.Exprs = append(proj.Exprs, expr)
}
proj.SetSchema(schema.Clone())
return true, proj
}
// rewriteExpr will rewrite the aggregate function to expression doesn't contain aggregate function.
func rewriteExpr(ctx sessionctx.Context, aggFunc *aggregation.AggFuncDesc) (bool, expression.Expression) {
switch aggFunc.Name {
case ast.AggFuncCount:
if aggFunc.Mode == aggregation.FinalMode {
return true, wrapCastFunction(ctx, aggFunc.Args[0], aggFunc.RetTp)
}
return true, rewriteCount(ctx, aggFunc.Args, aggFunc.RetTp)
case ast.AggFuncSum, ast.AggFuncAvg, ast.AggFuncFirstRow, ast.AggFuncMax, ast.AggFuncMin, ast.AggFuncGroupConcat:
return true, wrapCastFunction(ctx, aggFunc.Args[0], aggFunc.RetTp)
case ast.AggFuncBitAnd, ast.AggFuncBitOr, ast.AggFuncBitXor:
return true, rewriteBitFunc(ctx, aggFunc.Name, aggFunc.Args[0], aggFunc.RetTp)
default:
return false, nil
}
}
func rewriteCount(ctx sessionctx.Context, exprs []expression.Expression, targetTp *types.FieldType) expression.Expression {
// If is count(expr), we will change it to if(isnull(expr), 0, 1).
// If is count(distinct x, y, z) we will change it to if(isnull(x) or isnull(y) or isnull(z), 0, 1).
// If is count(expr not null), we will change it to constant 1.
isNullExprs := make([]expression.Expression, 0, len(exprs))
for _, expr := range exprs {
if mysql.HasNotNullFlag(expr.GetType().Flag) {
isNullExprs = append(isNullExprs, expression.NewZero())
} else {
isNullExpr := expression.NewFunctionInternal(ctx, ast.IsNull, types.NewFieldType(mysql.TypeTiny), expr)
isNullExprs = append(isNullExprs, isNullExpr)
}
}
innerExpr := expression.ComposeDNFCondition(ctx, isNullExprs...)
newExpr := expression.NewFunctionInternal(ctx, ast.If, targetTp, innerExpr, expression.NewZero(), expression.NewOne())
return newExpr
}
func rewriteBitFunc(ctx sessionctx.Context, funcType string, arg expression.Expression, targetTp *types.FieldType) expression.Expression {
// For not integer type. We need to cast(cast(arg as signed) as unsigned) to make the bit function work.
innerCast := expression.WrapWithCastAsInt(ctx, arg)
outerCast := wrapCastFunction(ctx, innerCast, targetTp)
var finalExpr expression.Expression
if funcType != ast.AggFuncBitAnd {
finalExpr = expression.NewFunctionInternal(ctx, ast.Ifnull, targetTp, outerCast, expression.NewZero())
} else {
finalExpr = expression.NewFunctionInternal(ctx, ast.Ifnull, outerCast.GetType(), outerCast, &expression.Constant{Value: types.NewUintDatum(math.MaxUint64), RetType: targetTp})
}
return finalExpr
}
// wrapCastFunction will wrap a cast if the targetTp is not equal to the arg's.
func wrapCastFunction(ctx sessionctx.Context, arg expression.Expression, targetTp *types.FieldType) expression.Expression {
if arg.GetType() == targetTp {
return arg
}
return expression.BuildCastFunction(ctx, arg, targetTp)
}
func (a *aggregationEliminator) optimize(ctx context.Context, p LogicalPlan) (LogicalPlan, error) {
newChildren := make([]LogicalPlan, 0, len(p.Children()))
for _, child := range p.Children() {
newChild, err := a.optimize(ctx, child)
if err != nil {
return nil, err
}
newChildren = append(newChildren, newChild)
}
p.SetChildren(newChildren...)
agg, ok := p.(*LogicalAggregation)
if !ok {
return p, nil
}
if proj := a.tryToEliminateAggregation(agg); proj != nil {
return proj, nil
}
return p, nil
}
func (*aggregationEliminator) name() string {
return "aggregation_eliminate"
}