Skip to content
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: propagate more filters in PropagateConstant #7276

Merged
merged 9 commits into from
Aug 30, 2018

Conversation

bb7133
Copy link
Member

@bb7133 bb7133 commented Aug 4, 2018

What have you changed? (mandatory)

This PR try to propagate more filters/constraints in PropagateConstant to archive the optimization mentioned in #7098. As long as an expression is deterministic, it can be propagated through the equality condition.

For example(the case in #7098), before this PR:

TiDB(localhost:4000) > desc select * from t1 join t2 on t1.a=t2.a and t1.a in (12, 13) and t2.a in (14, 15);
+-------------------------+----------+------+--------------------------------------------------------------------+
| id                      | count    | task | operator info                                                      |
+-------------------------+----------+------+--------------------------------------------------------------------+
| HashLeftJoin_6          | 25.00    | root | inner join, inner:TableReader_13, equal:[eq(test.t1.a, test.t2.a)] |
| ├─TableReader_10        | 20.00    | root | data:Selection_9                                                   |
| │ └─Selection_9         | 20.00    | cop  | in(test.t1.a, 12, 13)                                              |
| │   └─TableScan_8       | 10000.00 | cop  | table:t1, range:[-inf,+inf], keep order:false, stats:pseudo        |
| └─TableReader_13        | 20.00    | root | data:Selection_12                                                  |
|   └─Selection_12        | 20.00    | cop  | in(test.t2.a, 14, 15)                                              |
|     └─TableScan_11      | 10000.00 | cop  | table:t2, range:[-inf,+inf], keep order:false, stats:pseudo        |
+-------------------------+----------+------+--------------------------------------------------------------------+
7 rows in set (0.00 sec)

After this PR, we have:

TiDB(localhost:4000) > desc select * from t1 join t2 on t1.a=t2.a and t1.a in (12, 13) and t2.a in (14, 15);
+-------------------------+----------+------+--------------------------------------------------------------------+
| id                      | count    | task | operator info                                                      |
+-------------------------+----------+------+--------------------------------------------------------------------+
| HashLeftJoin_6          | 0.00     | root | inner join, inner:TableReader_13, equal:[eq(test.t1.a, test.t2.a)] |
| ├─TableReader_10        | 0.00     | root | data:Selection_9                                                   |
| │ └─Selection_9         | 0.00     | cop  | in(test.t1.a, 12, 13), in(test.t1.a, 14, 15)                       |
| │   └─TableScan_8       | 10000.00 | cop  | table:t1, range:[-inf,+inf], keep order:false, stats:pseudo        |
| └─TableReader_13        | 0.00     | root | data:Selection_12                                                  |
|   └─Selection_12        | 0.00     | cop  | in(test.t2.a, 14, 15), in(test.t2.a, 12, 13)                       |
|     └─TableScan_11      | 10000.00 | cop  | table:t2, range:[-inf,+inf], keep order:false, stats:pseudo        |
+-------------------------+----------+------+--------------------------------------------------------------------+
7 rows in set (0.00 sec)

However, propagation should be avoid for conditions with non-deterministic expression like rand(). A nonDeterministicFuncNameMap is added to make sure those expressions are excluded.

What is the type of the changes? (mandatory)

  • Improvement (non-breaking change which is an improvement to an existing feature)

How has this PR been tested? (mandatory)

  • Some unit test cases are added

Does this PR affect documentation (docs/docs-cn) update? (mandatory)

No

Does this PR affect tidb-ansible update? (mandatory)

No

Does this PR need to be added to the release notes? (mandatory)

No

Refer to a related PR or issue link (optional)

#7098

Add a few positive/negative examples (optional)

Since we try to do propagations as much as possible, some propagated filters may be un-necessary, or at least can be folded/combined, for example:

mysql> desc select * from t t1, t t2 where t1.a = t2.a and t1.a < t2.a + 1;
+--------------------------+----------+------+----------------------------------------------------------------------------------------------+
| id                       | count    | task | operator info                                                                                |
+--------------------------+----------+------+----------------------------------------------------------------------------------------------+
| HashLeftJoin_8           | 10000.00 | root | inner join, inner:TableReader_15, equal:[eq(t1.a, t2.a)], other cond:lt(t1.a, plus(t2.a, 1)) |
| ├─TableReader_12         | 8000.00  | root | data:Selection_11                                                                            |
| │ └─Selection_11         | 8000.00  | cop  | lt(t1.a, plus(t1.a, 1))                                                                      |
| │   └─TableScan_10       | 10000.00 | cop  | table:t1, range:[-inf,+inf], keep order:false, stats:pseudo                                  |
| └─TableReader_15         | 8000.00  | root | data:Selection_14                                                                            |
|   └─Selection_14         | 8000.00  | cop  | lt(t2.a, plus(t2.a, 1))                                                                      |
|     └─TableScan_13       | 10000.00 | cop  | table:t2, range:[-inf,+inf], keep order:false, stats:pseudo                                  |
+--------------------------+----------+------+----------------------------------------------------------------------------------------------+
7 rows in set (0.00 sec)

@zz-jason zz-jason self-assigned this Aug 4, 2018
@zz-jason
Copy link
Member

zz-jason commented Aug 4, 2018

/run-all-tests

@zz-jason zz-jason added contribution This PR is from a community contributor. sig/planner SIG: Planner type/enhancement The issue or PR belongs to an enhancement. labels Aug 4, 2018
ast.GE: true,
ast.NE: true,
// nonDeterministicFuncNameMap stores all the non-deterministic operators that cannot be propagated.
var nonDeterministicFuncNameMap = map[string]bool{
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe we can use unFoldableFunctions in "expression/function_traits.go" to replace this map?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK~


condsLen := len(s.conditions)
for j, colj := range s.columns {
for k, colk := range s.columns {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how about:

for j, colj := range s.columns {
	for k := j+1; k < len(s.columns); k++ {
		...
		replaced, _, newExpr := s.tryToReplaceCond(colj, colk, cond)
		replaced, _, newExpr := s.tryToReplaceCond(colk, colj, cond)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good idea! thank you!

func (s *propagateConstantSolver) tryToReplaceCond(src *Column, tgt *Column, cond Expression) (bool, bool, Expression) {
replaced := false
var r *ScalarFunction
if funct, ok := cond.(*ScalarFunction); ok {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/funct/sf/?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK~

@shenli
Copy link
Member

shenli commented Aug 5, 2018

@bb7133 Thanks! Please fix the CI.

@winoros
Copy link
Member

winoros commented Aug 6, 2018

/run-unit-test

1 similar comment
@winoros
Copy link
Member

winoros commented Aug 7, 2018

/run-unit-test

@winoros winoros changed the title expression/constant_propagation: propagate more filters in PropagateC… expression: propagate more filters in PropagateConstant Aug 7, 2018
if _, ok := unFoldableFunctions[sf.FuncName.L]; ok {
return false, true, cond
}
if _, isEq := eqFuncNameMap[sf.FuncName.L]; isEq {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

just sf.FuncName.L == ast.EQ is okay. And add comment why we skip it.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK~

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we still keep eqFuncNameMap? @winoros

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@bb7133 Seems that it can be removed. validPropagateCond now is only used to check eq cond. You can modify this method and remove the map.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks! @winoros

winoros
winoros previously requested changes Aug 7, 2018
if r == nil {
r = sf.Clone().(*ScalarFunction)
}
r.GetArgs()[idx] = tgt
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You'd better call NewFunction. Don't set scalar function's arg.

Copy link
Member Author

@bb7133 bb7133 Aug 7, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see, it's better to treat scalar function immutable

if r == nil {
r = sf.Clone().(*ScalarFunction)
}
r.GetArgs()[idx] = subExpr
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ditto

@winoros
Copy link
Member

winoros commented Aug 10, 2018

@bb7133
Please fix the test

FAIL: constant_test.go:56: testExpressionSuite.TestConstantPropagation

constant_test.go:158:
    c.Assert(strings.Join(result, ", "), Equals, tt.result, Commentf("different for expr %s", tt.conditions))
... obtained string = "eq(test.t.0, test.t.1), le(mul(test.t.0, test.t.0), 50), le(mul(test.t.1, test.t.0), 50)"
... expected string = "eq(test.t.0, test.t.1), le(mul(test.t.0, test.t.0), 50), le(mul(test.t.1, test.t.1), 50)"
... different for expr [eq(test.t.0, test.t.1) le(mul(test.t.0, test.t.0), 50)]

You can run go test locally to test it.

@bb7133
Copy link
Member Author

bb7133 commented Aug 11, 2018

Fixed, PTAL @winoros, thanks!

@zz-jason
Copy link
Member

/run-all-tests

@zz-jason
Copy link
Member

LGTM

@zz-jason zz-jason added the status/LGT1 Indicates that a PR has LGTM 1. label Aug 11, 2018
if eq, ok := cond.(*ScalarFunction); ok {
if _, ok := funNameMap[eq.FuncName.L]; !ok {
if eq.FuncName.L != ast.EQ {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. update the comment of this function since we do not consider non-equal ops in this function.
  2. s/ validPropagateCond/ validEqualCond?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

All right

// bool: if 'cond' contains non-deterministic expression
// Expression: the replaced expression, or original 'cond' if the replacement is not happened
func (s *propagateConstantSolver) tryToReplaceCond(src *Column, tgt *Column, cond Expression) (bool, bool, Expression) {
if sf, ok := cond.(*ScalarFunction); ok {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sf, ok := cond.(*ScalarFunction); !ok {
return false, false, cond
}
......

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK

if _, ok := unFoldableFunctions[sf.FuncName.L]; ok {
return false, true, cond
}
// Equality is handled in propagateEQ already
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. propagateEQ only handles equal functions like column = constant but not handle the case like
    a = b and b = funcname(), should we consider it here?
  2. Or may be we do not need propagateEQ now?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There are many optimizations we can consider in constant propagation and now I think it may be a good idea to have a doc so we can:

  1. Make further discussions and details.
  2. Make a list of subtasks.

I will try to give this initial doc soon. Your comment is appreciated, thanks

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've updated the code to support b = func() case, thanks

continue
}
colk := s.columns[k]
for i := 0; i < condsLen; i++ {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/ condsLen/ len(s.conditions)
and remove line 107
since we only use it here.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it's better to keep condsLen because we're adding elements to conditions while iterating it. Iteration on new elements are not needed.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We need to comment for the behavior of the 3-layer loop.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

addressed, thanks

// tryToReplaceCond aims to replace all occurances of column 'src' and try to replace it with 'tgt' in 'cond'
// It returns
// bool: if a replacement happened
// bool: if 'cond' contains non-deterministic expression
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems that we can remove this return value?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This boolean value is used to avoid some bad case like: a = b and cast(b, int) < rand(), for which we cannot propagate a cast(a, int) < rand().

There may be more sophisticated solution, do you have any advise? thanks!

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok, never mind

@@ -158,6 +146,51 @@ func (s *propagateConstantSolver) validPropagateCond(cond Expression, funNameMap
return nil, nil
}

// tryToReplaceCond aims to replace all occurances of column 'src' and try to replace it with 'tgt' in 'cond'
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We need a more detailed comment here,
an example may help.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah,I will try to state more details

@winoros
Copy link
Member

winoros commented Aug 13, 2018

/unit-test

@zz-jason
Copy link
Member

@bb7133 any update?

@bb7133
Copy link
Member Author

bb7133 commented Aug 29, 2018

code updated, sorry it costed a long time @zz-jason

1. add propagation for EQ condition like 'a eq func'
2. refined some method/variable names
3. added more comments
4. added 1 more unit test

Signed-off-by: bb7133 <[email protected]>
@shenli
Copy link
Member

shenli commented Aug 30, 2018

/run-all-tests

// e.g. For expression a = b and b = c and c = d and c < 1 , we can get extra a < 1 and b < 1 and d < 1.
// However, for a = b and a < rand(), we cannot propagate a < rand() to b < rand() because rand() is non-deterministic
//
// This propagation may bring redundancies that we need to resolve later, for example:
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In the present planner, we can do this by adding a post-process after the logical optimization, before deriving the statistics and build index/table ranges.

In the new cascades planner, we can do this by adding a rule which only works on a Filter operator and remove the duplicated filters.

@bb7133 What's your opinion?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Current ranger can spot these redundant filters and merge them?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@eurekaka maybe we can do the enhancement in ranger?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems like ranger would only simplify and merge these filters when they are considered to be index filters, if there is no index and no int primary key, these filters would be in PhysicalSelection as what they are originally; maybe we should extract the range simplification logic out of ranger, and this should be like the 'post-process' you mentioned above.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@zz-jason Sounds interesting! And looks @tiancaiamao is trying to solve (at least part of) the redundant filters in his PR, which we can expect.

// for a = b and a < 3 and b < 4, we get new a < 4 and b < 3 but should expect a < 3 and b < 3
// for a = b and a in (3) and b in (4), we get b in (3) and a in (4) but should expect 'false'
//
// TODO: remove redundancies later
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's better to add a github issue for this TODO item.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK.

@zz-jason
Copy link
Member

@winoros @XuHuaiyu PTAL

@tiancaiamao
Copy link
Contributor

FYI: I'm rewriting a much more aggressive constant propagate... @bb7133 @zz-jason
Hope I could to pull the request today.

@zz-jason
Copy link
Member

@tiancaiamao What's the idea of the "aggressive constant propagate"?

@tiancaiamao
Copy link
Contributor

Current constant propagate doesn't infer much on InEQ condition:

"a > 3" && "a > 5" => "a > 5"
"a > 5" && "a < 3" => false

And it doesn't handle function:

to_days(t) > date '2018-03-05' && to_days(t) < '2018-03-01' => false

I'm writing an "aggressive" one to solve this issue #7516

Copy link
Contributor

@XuHuaiyu XuHuaiyu left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@eurekaka
Copy link
Contributor

LGTM

@eurekaka eurekaka added status/LGT2 Indicates that a PR has LGTM 2. and removed status/LGT1 Indicates that a PR has LGTM 1. labels Aug 30, 2018
@zz-jason
Copy link
Member

/run-all-tests

@bb7133
Copy link
Member Author

bb7133 commented Aug 30, 2018

Well, @tiancaiamao your idea is mentioned in my TODO list in the comment, it would be great if you can solve it. BTW, I wrote a doc about my opinions on related optimizations: https://docs.google.com/document/d/1G3wVBaiza9GI5q9nwHLCB2I7r1HUbg6RpTNFpQWTFhQ/edit?usp=sharing

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
contribution This PR is from a community contributor. sig/planner SIG: Planner status/LGT2 Indicates that a PR has LGTM 2. type/enhancement The issue or PR belongs to an enhancement.
Projects
None yet
Development

Successfully merging this pull request may close these issues.

7 participants