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

planner: fix wrong behavior for != any() (#20058) #20062

Merged
merged 6 commits into from
Sep 21, 2020
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion cmd/explaintest/r/select.result
Original file line number Diff line number Diff line change
Expand Up @@ -384,7 +384,7 @@ explain select a != any (select a from t t2) from t t1;
id estRows task access object operator info
Projection_8 10000.00 root and(or(or(gt(Column#8, 1), ne(test.t.a, Column#7)), if(ne(Column#9, 0), <nil>, 0)), and(ne(Column#10, 0), if(isnull(test.t.a), <nil>, 1)))->Column#11
└─HashJoin_9 10000.00 root CARTESIAN inner join
├─StreamAgg_14(Build) 1.00 root funcs:firstrow(Column#13)->Column#7, funcs:count(distinct Column#14)->Column#8, funcs:sum(Column#15)->Column#9, funcs:count(1)->Column#10
├─StreamAgg_14(Build) 1.00 root funcs:max(Column#13)->Column#7, funcs:count(distinct Column#14)->Column#8, funcs:sum(Column#15)->Column#9, funcs:count(1)->Column#10
│ └─Projection_19 10000.00 root test.t.a, test.t.a, cast(isnull(test.t.a), decimal(65,0) BINARY)->Column#15
│ └─TableReader_18 10000.00 root data:TableFullScan_17
│ └─TableFullScan_17 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo
Expand Down
14 changes: 8 additions & 6 deletions planner/core/expression_rewriter.go
Original file line number Diff line number Diff line change
Expand Up @@ -639,7 +639,9 @@ func (er *expressionRewriter) buildQuantifierPlan(plan4Agg *LogicalAggregation,
// t.id != s.id or count(distinct s.id) > 1 or [any checker]. If there are two different values in s.id ,
// there must exist a s.id that doesn't equal to t.id.
func (er *expressionRewriter) handleNEAny(lexpr, rexpr expression.Expression, np LogicalPlan) {
firstRowFunc, err := aggregation.NewAggFuncDesc(er.sctx, ast.AggFuncFirstRow, []expression.Expression{rexpr}, false)
// If there is NULL in s.id column, s.id should be the value that isn't null in condition t.id != s.id.
// So use function max to filter NULL.
maxFunc, err := aggregation.NewAggFuncDesc(er.sctx, ast.AggFuncMax, []expression.Expression{rexpr}, false)
if err != nil {
er.err = err
return
Expand All @@ -650,24 +652,24 @@ func (er *expressionRewriter) handleNEAny(lexpr, rexpr expression.Expression, np
return
}
plan4Agg := LogicalAggregation{
AggFuncs: []*aggregation.AggFuncDesc{firstRowFunc, countFunc},
AggFuncs: []*aggregation.AggFuncDesc{maxFunc, countFunc},
}.Init(er.sctx, er.b.getSelectOffset())
if hint := er.b.TableHints(); hint != nil {
plan4Agg.aggHints = hint.aggHints
}
plan4Agg.SetChildren(np)
firstRowResultCol := &expression.Column{
maxResultCol := &expression.Column{
UniqueID: er.sctx.GetSessionVars().AllocPlanColumnID(),
RetType: firstRowFunc.RetTp,
RetType: maxFunc.RetTp,
}
count := &expression.Column{
UniqueID: er.sctx.GetSessionVars().AllocPlanColumnID(),
RetType: countFunc.RetTp,
}
plan4Agg.names = append(plan4Agg.names, types.EmptyName, types.EmptyName)
plan4Agg.SetSchema(expression.NewSchema(firstRowResultCol, count))
plan4Agg.SetSchema(expression.NewSchema(maxResultCol, count))
gtFunc := expression.NewFunctionInternal(er.sctx, ast.GT, types.NewFieldType(mysql.TypeTiny), count, expression.NewOne())
neCond := expression.NewFunctionInternal(er.sctx, ast.NE, types.NewFieldType(mysql.TypeTiny), lexpr, firstRowResultCol)
neCond := expression.NewFunctionInternal(er.sctx, ast.NE, types.NewFieldType(mysql.TypeTiny), lexpr, maxResultCol)
cond := expression.ComposeDNFCondition(er.sctx, gtFunc, neCond)
er.buildQuantifierPlan(plan4Agg, cond, lexpr, rexpr, false)
}
Expand Down
23 changes: 23 additions & 0 deletions planner/core/expression_rewriter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -260,3 +260,26 @@ func (s *testExpressionRewriterSuite) TestPatternLikeToExpression(c *C) {
tk.MustQuery("select 0 like '0';").Check(testkit.Rows("1"))
tk.MustQuery("select 0.00 like '0.00';").Check(testkit.Rows("1"))
}

func (s *testExpressionRewriterSuite) TestIssue20007(c *C) {
defer testleak.AfterTest(c)()
store, dom, err := newStoreWithBootstrap()
c.Assert(err, IsNil)
tk := testkit.NewTestKit(c, store)
defer func() {
dom.Close()
store.Close()
}()

tk.MustExec("use test;")
tk.MustExec("drop table if exists t1, t2;")
tk.MustExec("create table t1 (c_int int, c_str varchar(40), c_datetime datetime, primary key(c_int));")
tk.MustExec("create table t2 (c_int int, c_str varchar(40), c_datetime datetime, primary key (c_datetime)) partition by range (to_days(c_datetime)) ( partition p0 values less than (to_days('2020-02-01')), partition p1 values less than (to_days('2020-04-01')), partition p2 values less than (to_days('2020-06-01')), partition p3 values less than maxvalue);")
tk.MustExec("insert into t1 (c_int, c_str, c_datetime) values (1, 'xenodochial bassi', '2020-04-29 03:22:51'), (2, 'epic wiles', '2020-01-02 23:29:51'), (3, 'silly burnell', '2020-02-25 07:43:07');")
tk.MustExec("insert into t2 (c_int, c_str, c_datetime) values (1, 'trusting matsumoto', '2020-01-07 00:57:18'), (2, 'pedantic boyd', '2020-06-08 23:12:16'), (null, 'strange hypatia', '2020-05-23 17:45:27');")
// Test 10 times.
for i := 0; i < 10; i++ {
tk.MustQuery("select * from t1 where c_int != any (select c_int from t2 where t1.c_str <= t2.c_str); ").Check(
testkit.Rows("2 epic wiles 2020-01-02 23:29:51", "3 silly burnell 2020-02-25 07:43:07"))
}
}