-
Notifications
You must be signed in to change notification settings - Fork 5.8k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
expression: constraint propagate for '>' and monotonous function #8640
Changes from 11 commits
6cbf528
f966b75
e11f0ed
12d9212
3376571
9a3659b
b3287fc
0f08c3e
bdcf799
3cf386b
ba71802
c2e6940
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -16,6 +16,7 @@ package expression | |
import ( | ||
"bytes" | ||
|
||
"github.com/pingcap/parser/ast" | ||
"github.com/pingcap/parser/mysql" | ||
"github.com/pingcap/tidb/sessionctx" | ||
"github.com/pingcap/tidb/types" | ||
|
@@ -78,24 +79,35 @@ func newExprSet(conditions []Expression) *exprSet { | |
return &exprs | ||
} | ||
|
||
type constraintSolver []constraintPropagateRule | ||
|
||
func newConstraintSolver(rules ...constraintPropagateRule) constraintSolver { | ||
return constraintSolver(rules) | ||
} | ||
|
||
type pgSolver2 struct{} | ||
|
||
// PropagateConstant propagate constant values of deterministic predicates in a condition. | ||
func (s pgSolver2) PropagateConstant(ctx sessionctx.Context, conditions []Expression) []Expression { | ||
solver := newConstraintSolver(ruleConstantFalse, ruleColumnEQConst) | ||
return solver.Solve(ctx, conditions) | ||
} | ||
|
||
// Solve propagate constraint according to the rules in the constraintSolver. | ||
func (s constraintSolver) Solve(ctx sessionctx.Context, conditions []Expression) []Expression { | ||
exprs := newExprSet(conditions) | ||
s.fixPoint(ctx, exprs) | ||
return exprs.Slice() | ||
} | ||
|
||
// fixPoint is the core of the constant propagation algorithm. | ||
// fixPoint is the core of the constraint propagation algorithm. | ||
// It will iterate the expression set over and over again, pick two expressions, | ||
// apply one to another. | ||
// If new conditions can be inferred, they will be append into the expression set. | ||
// Until no more conditions can be inferred from the set, the algorithm finish. | ||
func (s pgSolver2) fixPoint(ctx sessionctx.Context, exprs *exprSet) { | ||
func (s constraintSolver) fixPoint(ctx sessionctx.Context, exprs *exprSet) { | ||
for { | ||
saveLen := len(exprs.data) | ||
iterOnce(ctx, exprs) | ||
s.iterOnce(ctx, exprs) | ||
if saveLen == len(exprs.data) { | ||
break | ||
} | ||
|
@@ -104,7 +116,7 @@ func (s pgSolver2) fixPoint(ctx sessionctx.Context, exprs *exprSet) { | |
} | ||
|
||
// iterOnce picks two expressions from the set, try to propagate new conditions from them. | ||
func iterOnce(ctx sessionctx.Context, exprs *exprSet) { | ||
func (s constraintSolver) iterOnce(ctx sessionctx.Context, exprs *exprSet) { | ||
for i := 0; i < len(exprs.data); i++ { | ||
if exprs.tombstone[i] { | ||
continue | ||
|
@@ -116,24 +128,19 @@ func iterOnce(ctx sessionctx.Context, exprs *exprSet) { | |
if i == j { | ||
continue | ||
} | ||
solve(ctx, i, j, exprs) | ||
s.solve(ctx, i, j, exprs) | ||
} | ||
} | ||
} | ||
|
||
// solve uses exprs[i] exprs[j] to propagate new conditions. | ||
func solve(ctx sessionctx.Context, i, j int, exprs *exprSet) { | ||
for _, rule := range rules { | ||
func (s constraintSolver) solve(ctx sessionctx.Context, i, j int, exprs *exprSet) { | ||
for _, rule := range s { | ||
rule(ctx, i, j, exprs) | ||
} | ||
} | ||
|
||
type constantPropagateRule func(ctx sessionctx.Context, i, j int, exprs *exprSet) | ||
|
||
var rules = []constantPropagateRule{ | ||
ruleConstantFalse, | ||
ruleColumnEQConst, | ||
} | ||
type constraintPropagateRule func(ctx sessionctx.Context, i, j int, exprs *exprSet) | ||
|
||
// ruleConstantFalse propagates from CNF condition that false plus anything returns false. | ||
// false, a = 1, b = c ... => false | ||
|
@@ -164,3 +171,134 @@ func ruleColumnEQConst(ctx sessionctx.Context, i, j int, exprs *exprSet) { | |
} | ||
} | ||
} | ||
|
||
// ruleColumnOPConst propagates the "column OP const" condition. | ||
func ruleColumnOPConst(ctx sessionctx.Context, i, j int, exprs *exprSet) { | ||
cond := exprs.data[i] | ||
f1, ok := cond.(*ScalarFunction) | ||
if !ok { | ||
return | ||
} | ||
if f1.FuncName.L != ast.GE && f1.FuncName.L != ast.GT && | ||
f1.FuncName.L != ast.LE && f1.FuncName.L != ast.LT { | ||
return | ||
} | ||
OP1 := f1.FuncName.L | ||
|
||
var col1 *Column | ||
var con1 *Constant | ||
col1, ok = f1.GetArgs()[0].(*Column) | ||
if !ok { | ||
return | ||
} | ||
con1, ok = f1.GetArgs()[1].(*Constant) | ||
if !ok { | ||
return | ||
} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We don't consider There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There are only two rules that really useful to partition pruning:
Why?
Let's take a look at
The final conditions would be:
So the problem become that whether we can propagate:
You'll find the PR can't handle the Then you'll find the rule order matters. Not a serious problem, we can control the order by I'll make pruning for There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Thant would be great. |
||
|
||
expr := exprs.data[j] | ||
f2, ok := expr.(*ScalarFunction) | ||
if !ok { | ||
return | ||
} | ||
|
||
// The simple case: | ||
// col >= c1, col < c2, c1 >= c2 => false | ||
// col >= c1, col <= c2, c1 > c2 => false | ||
// col >= c1, col OP c2, c1 ^OP c2, where OP in [< , <=] => false | ||
// col OP1 c1 where OP1 in [>= , <], col OP2 c2 where OP1 opsite OP2, c1 ^OP2 c2 => false | ||
// | ||
// The extended case: | ||
// col >= c1, f(col) < c2, f is monotonous, f(c1) >= c2 => false | ||
// | ||
// Proof: | ||
// col > c1, f is monotonous => f(col) > f(c1) | ||
// f(col) > f(c1), f(col) < c2, f(c1) >= c2 => false | ||
OP2 := f2.FuncName.L | ||
if !opsiteOP(OP1, OP2) { | ||
return | ||
} | ||
|
||
con2, ok := f2.GetArgs()[1].(*Constant) | ||
if !ok { | ||
return | ||
} | ||
arg0 := f2.GetArgs()[0] | ||
// The simple case. | ||
var fc1 Expression | ||
col2, ok := arg0.(*Column) | ||
if ok { | ||
fc1 = con1 | ||
} else { | ||
// The extended case. | ||
scalarFunc, ok := arg0.(*ScalarFunction) | ||
if !ok { | ||
return | ||
} | ||
_, ok = monotoneIncFuncs[scalarFunc.FuncName.L] | ||
if !ok { | ||
return | ||
} | ||
col2, ok = scalarFunc.GetArgs()[0].(*Column) | ||
if !ok { | ||
return | ||
} | ||
var err error | ||
fc1, err = NewFunction(ctx, scalarFunc.FuncName.L, scalarFunc.RetType, con1) | ||
if err != nil { | ||
log.Warn(err) | ||
return | ||
} | ||
} | ||
if !col1.Equal(ctx, col2) { | ||
return | ||
} | ||
v, isNull, err := compareConstant(ctx, negOP(OP2), fc1, con2) | ||
if err != nil { | ||
log.Warn(err) | ||
return | ||
} | ||
if !isNull && v > 0 { | ||
exprs.SetConstFalse() | ||
} | ||
return | ||
} | ||
|
||
// opsiteOP the opsite direction of a compare operation, used in ruleColumnOPConst. | ||
func opsiteOP(op1, op2 string) bool { | ||
switch { | ||
case op1 == ast.GE || op1 == ast.GT: | ||
return op2 == ast.LT || op2 == ast.LE | ||
case op1 == ast.LE || op1 == ast.LT: | ||
return op2 == ast.GT || op2 == ast.GE | ||
} | ||
return false | ||
} | ||
|
||
func negOP(cmp string) string { | ||
switch cmp { | ||
case ast.LT: | ||
return ast.GE | ||
case ast.LE: | ||
return ast.GT | ||
case ast.GT: | ||
return ast.LE | ||
case ast.GE: | ||
return ast.LT | ||
} | ||
return "" | ||
} | ||
|
||
// monotoneIncFuncs are those functions that for any x y, if x > y => f(x) > f(y) | ||
var monotoneIncFuncs = map[string]struct{}{ | ||
ast.ToDays: {}, | ||
} | ||
|
||
// compareConstant compares two expressions. c1 and c2 should be constant with the same type. | ||
func compareConstant(ctx sessionctx.Context, fn string, c1, c2 Expression) (int64, bool, error) { | ||
winoros marked this conversation as resolved.
Show resolved
Hide resolved
|
||
cmp, err := NewFunction(ctx, fn, types.NewFieldType(mysql.TypeTiny), c1, c2) | ||
if err != nil { | ||
return 0, false, err | ||
} | ||
return cmp.EvalInt(ctx, chunk.Row{}) | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
how about:
s/newColumn/newInt64Column/
s/newColumnWithType/newColumn/
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There are so many
newColumn
in this file https://github.com/pingcap/tidb/blob/3cf386b5ca2300a0e0cebeae64d58eac9fcbff6c/expression/constant_test.goIf the function signature changes, it bring about ~50 (lines) irrelevant changes to this PR.
And this is
_test.go
, in this filenewColumn
indicatesnewInt64Column