Skip to content

Commit

Permalink
plan: merge continuous selections and delete surely true expressions (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
fzhedu authored May 10, 2021
1 parent c6c8265 commit f135c53
Show file tree
Hide file tree
Showing 13 changed files with 195 additions and 69 deletions.
6 changes: 3 additions & 3 deletions cmd/explaintest/r/explain_easy.result
Original file line number Diff line number Diff line change
Expand Up @@ -108,9 +108,9 @@ HashJoin 9990.00 root inner join, equal:[eq(test.t1.c1, test.t2.c2)]
└─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo
explain format = 'brief' select (select count(1) k from t1 s where s.c1 = t1.c1 having k != 0) from t1;
id estRows task access object operator info
Projection 10000.00 root ifnull(Column#10, 0)->Column#10
└─MergeJoin 10000.00 root left outer join, left key:test.t1.c1, right key:test.t1.c1
├─Projection(Build) 8000.00 root 1->Column#10, test.t1.c1
Projection 12500.00 root ifnull(Column#10, 0)->Column#10
└─MergeJoin 12500.00 root left outer join, left key:test.t1.c1, right key:test.t1.c1
├─Projection(Build) 10000.00 root 1->Column#10, test.t1.c1
│ └─TableReader 10000.00 root data:TableFullScan
│ └─TableFullScan 10000.00 cop[tikv] table:s keep order:true, stats:pseudo
└─TableReader(Probe) 10000.00 root data:TableFullScan
Expand Down
2 changes: 1 addition & 1 deletion executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8128,7 +8128,7 @@ func (s *testSerialSuite) TestIssue24210(c *C) {

// for SelectionExec
c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockSelectionExecBaseExecutorOpenReturnedError", `return(true)`), IsNil)
_, err = tk.Exec("select * from (select 1 as a) t where a > 0")
_, err = tk.Exec("select * from (select rand() as a) t where a > 0")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "mock SelectionExec.baseExecutor.Open returned error")
err = failpoint.Disable("github.com/pingcap/tidb/executor/mockSelectionExecBaseExecutorOpenReturnedError")
Expand Down
10 changes: 4 additions & 6 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5474,16 +5474,14 @@ func (s *testIntegrationSuite) TestExprPushdownBlacklist(c *C) {
// > pushed to both TiKV and TiFlash
rows := tk.MustQuery("explain format = 'brief' select * from test.t where b > date'1988-01-01' and b < date'1994-01-01' " +
"and cast(a as decimal(10,2)) > 10.10 and date_format(b,'%m') = '11'").Rows()
c.Assert(fmt.Sprintf("%v", rows[0][4]), Equals, "lt(test.t.b, 1994-01-01)")
c.Assert(fmt.Sprintf("%v", rows[1][4]), Equals, "gt(cast(test.t.a, decimal(10,2) BINARY), 10.10)")
c.Assert(fmt.Sprintf("%v", rows[3][4]), Equals, "eq(date_format(test.t.b, \"%m\"), \"11\"), gt(test.t.b, 1988-01-01)")
c.Assert(fmt.Sprintf("%v", rows[0][4]), Equals, "gt(cast(test.t.a, decimal(10,2) BINARY), 10.10), lt(test.t.b, 1994-01-01)")
c.Assert(fmt.Sprintf("%v", rows[2][4]), Equals, "eq(date_format(test.t.b, \"%m\"), \"11\"), gt(test.t.b, 1988-01-01)")

tk.MustExec("set @@session.tidb_isolation_read_engines = 'tikv'")
rows = tk.MustQuery("explain format = 'brief' select * from test.t where b > date'1988-01-01' and b < date'1994-01-01' " +
"and cast(a as decimal(10,2)) > 10.10 and date_format(b,'%m') = '11'").Rows()
c.Assert(fmt.Sprintf("%v", rows[0][4]), Equals, "lt(test.t.b, 1994-01-01)")
c.Assert(fmt.Sprintf("%v", rows[1][4]), Equals, "eq(date_format(test.t.b, \"%m\"), \"11\")")
c.Assert(fmt.Sprintf("%v", rows[3][4]), Equals, "gt(cast(test.t.a, decimal(10,2) BINARY), 10.10), gt(test.t.b, 1988-01-01)")
c.Assert(fmt.Sprintf("%v", rows[0][4]), Equals, "eq(date_format(test.t.b, \"%m\"), \"11\"), lt(test.t.b, 1994-01-01)")
c.Assert(fmt.Sprintf("%v", rows[2][4]), Equals, "gt(cast(test.t.a, decimal(10,2) BINARY), 10.10), gt(test.t.b, 1988-01-01)")

tk.MustExec("delete from mysql.expr_pushdown_blacklist where name = '<' and store_type = 'tikv,tiflash,tidb' and reason = 'for test'")
tk.MustExec("delete from mysql.expr_pushdown_blacklist where name = 'date_format' and store_type = 'tikv' and reason = 'for test'")
Expand Down
4 changes: 2 additions & 2 deletions expression/testdata/expression_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -186,7 +186,7 @@
{
"SQL": "explain format = 'brief' select * from t1 left join t2 on true where t1.a = 1 and t1.a = 1",
"Result": [
"HashJoin 80000.00 root CARTESIAN left outer join",
"HashJoin 100000.00 root CARTESIAN left outer join",
"├─TableReader(Build) 10.00 root data:Selection",
"│ └─Selection 10.00 cop[tikv] eq(test.t1.a, 1)",
"│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo",
Expand Down Expand Up @@ -253,7 +253,7 @@
{
"SQL": "explain format = 'brief' select * from t1 left join t2 on true where t1.a = 1 or (t1.a = 2 and t1.a = 3)",
"Result": [
"HashJoin 80000.00 root CARTESIAN left outer join",
"HashJoin 100000.00 root CARTESIAN left outer join",
"├─TableReader(Build) 10.00 root data:Selection",
"│ └─Selection 10.00 cop[tikv] or(eq(test.t1.a, 1), 0)",
"│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo",
Expand Down
1 change: 1 addition & 0 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -500,6 +500,7 @@ github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQD
github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10=
github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0=
github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo=
github.com/sergi/go-diff v1.1.0 h1:we8PVUC3FE2uYfodKH/nBHMSetSfHDR6scGdBi+erh0=
github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM=
github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA=
github.com/shirou/gopsutil v3.21.2+incompatible h1:U+YvJfjCh6MslYlIAXvPtzhW3YZEtc9uncueUNpD/0A=
Expand Down
38 changes: 38 additions & 0 deletions planner/core/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3651,3 +3651,41 @@ func (s *testIntegrationSuite) TestSequenceAsDataSource(c *C) {
tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...))
}
}

func (s *testIntegrationSerialSuite) TestMergeContinuousSelections(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists ts")
tk.MustExec("create table ts (col_char_64 char(64), col_varchar_64_not_null varchar(64) not null, col_varchar_key varchar(1), id int primary key, col_varchar_64 varchar(64),col_char_64_not_null char(64) not null);")

// Create virtual tiflash replica info.
dom := domain.GetDomain(tk.Se)
is := dom.InfoSchema()
db, exists := is.SchemaByName(model.NewCIStr("test"))
c.Assert(exists, IsTrue)
for _, tblInfo := range db.Tables {
if tblInfo.Name.L == "ts" {
tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{
Count: 1,
Available: true,
}
}
}

tk.MustExec(" set @@tidb_allow_mpp=1;")

var input []string
var output []struct {
SQL string
Plan []string
}
s.testData.GetTestCases(c, &input, &output)
for i, tt := range input {
s.testData.OnRecord(func() {
output[i].SQL = tt
output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows())
})
res := tk.MustQuery(tt)
res.Check(testkit.Rows(output[i].Plan...))
}
}
26 changes: 26 additions & 0 deletions planner/core/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/lock"
"github.com/pingcap/tidb/planner/property"
"github.com/pingcap/tidb/privilege"
Expand Down Expand Up @@ -156,9 +157,34 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic
return finalPlan, cost, nil
}

// mergeContinuousSelections merge continuous selections which may occur after changing plans.
func mergeContinuousSelections(p PhysicalPlan) {
if sel, ok := p.(*PhysicalSelection); ok {
for {
childSel := sel.children[0]
if tmp, ok := childSel.(*PhysicalSelection); ok {
sel.Conditions = append(sel.Conditions, tmp.Conditions...)
sel.SetChild(0, tmp.children[0])
} else {
break
}
}
}
for _, child := range p.Children() {
mergeContinuousSelections(child)
}
// merge continuous selections in a coprocessor task of tiflash
tableReader, isTableReader := p.(*PhysicalTableReader)
if isTableReader && tableReader.StoreType == kv.TiFlash {
mergeContinuousSelections(tableReader.tablePlan)
tableReader.TablePlans = flattenPushDownPlan(tableReader.tablePlan)
}
}

func postOptimize(sctx sessionctx.Context, plan PhysicalPlan) PhysicalPlan {
plan = eliminatePhysicalProjection(plan)
plan = InjectExtraProjection(plan)
mergeContinuousSelections(plan)
plan = eliminateUnionScanAndLock(sctx, plan)
plan = enableParallelApply(sctx, plan)
return plan
Expand Down
31 changes: 31 additions & 0 deletions planner/core/rule_predicate_push_down.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,12 @@ func addSelection(p LogicalPlan, child LogicalPlan, conditions []expression.Expr
p.Children()[chIdx] = dual
return
}

conditions = DeleteTrueExprs(p, conditions)
if len(conditions) == 0 {
p.Children()[chIdx] = child
return
}
selection := LogicalSelection{Conditions: conditions}.Init(p.SCtx(), p.SelectBlockOffset())
selection.SetChildren(child)
p.Children()[chIdx] = selection
Expand Down Expand Up @@ -73,6 +79,8 @@ func splitSetGetVarFunc(filters []expression.Expression) ([]expression.Expressio

// PredicatePushDown implements LogicalPlan PredicatePushDown interface.
func (p *LogicalSelection) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan) {
predicates = DeleteTrueExprs(p, predicates)
p.Conditions = DeleteTrueExprs(p, p.Conditions)
canBePushDown, canNotBePushDown := splitSetGetVarFunc(p.Conditions)
retConditions, child := p.children[0].PredicatePushDown(append(canBePushDown, predicates...))
retConditions = append(retConditions, canNotBePushDown...)
Expand Down Expand Up @@ -100,6 +108,7 @@ func (p *LogicalUnionScan) PredicatePushDown(predicates []expression.Expression)
// PredicatePushDown implements LogicalPlan PredicatePushDown interface.
func (ds *DataSource) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan) {
predicates = expression.PropagateConstant(ds.ctx, predicates)
predicates = DeleteTrueExprs(ds, predicates)
ds.allConds = predicates
ds.pushedDownConds, predicates = expression.PushDownExprs(ds.ctx.GetSessionVars().StmtCtx, predicates, ds.ctx.GetClient(), kv.UnSpecified)
return predicates, ds
Expand Down Expand Up @@ -532,6 +541,28 @@ func Conds2TableDual(p LogicalPlan, conds []expression.Expression) LogicalPlan {
return nil
}

// DeleteTrueExprs deletes the surely true expressions
func DeleteTrueExprs(p LogicalPlan, conds []expression.Expression) []expression.Expression {
newConds := make([]expression.Expression, 0, len(conds))
for _, cond := range conds {
con, ok := cond.(*expression.Constant)
if !ok {
newConds = append(newConds, cond)
continue
}
if expression.ContainMutableConst(p.SCtx(), []expression.Expression{con}) {
newConds = append(newConds, cond)
continue
}
sc := p.SCtx().GetSessionVars().StmtCtx
if isTrue, err := con.Value.ToBool(sc); err == nil && isTrue == 1 {
continue
}
newConds = append(newConds, cond)
}
return newConds
}

// outerJoinPropConst propagates constant equal and column equal conditions over outer join.
func (p *LogicalJoin) outerJoinPropConst(predicates []expression.Expression) []expression.Expression {
outerTable := p.children[0]
Expand Down
6 changes: 6 additions & 0 deletions planner/core/testdata/integration_serial_suite_in.json
Original file line number Diff line number Diff line change
Expand Up @@ -300,5 +300,11 @@
"cases": [
"select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t"
]
},
{
"name": "TestMergeContinuousSelections",
"cases": [
"desc format = 'brief' SELECT table2 . `col_char_64` AS field1 FROM `ts` AS table2 INNER JOIN (SELECT DISTINCT SUBQUERY3_t1 . * FROM `ts` AS SUBQUERY3_t1 LEFT OUTER JOIN `ts` AS SUBQUERY3_t2 ON SUBQUERY3_t2 . `col_varchar_64_not_null` = SUBQUERY3_t1 . `col_varchar_key`) AS table3 ON (table3 . `col_varchar_key` = table2 . `col_varchar_64`) WHERE table3 . `col_char_64_not_null` >= SOME (SELECT SUBQUERY4_t1 . `col_varchar_64` AS SUBQUERY4_field1 FROM `ts` AS SUBQUERY4_t1) GROUP BY field1 ;"
]
}
]
28 changes: 28 additions & 0 deletions planner/core/testdata/integration_serial_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -2608,5 +2608,33 @@
]
}
]
},
{
"Name": "TestMergeContinuousSelections",
"Cases": [
{
"SQL": "desc format = 'brief' SELECT table2 . `col_char_64` AS field1 FROM `ts` AS table2 INNER JOIN (SELECT DISTINCT SUBQUERY3_t1 . * FROM `ts` AS SUBQUERY3_t1 LEFT OUTER JOIN `ts` AS SUBQUERY3_t2 ON SUBQUERY3_t2 . `col_varchar_64_not_null` = SUBQUERY3_t1 . `col_varchar_key`) AS table3 ON (table3 . `col_varchar_key` = table2 . `col_varchar_64`) WHERE table3 . `col_char_64_not_null` >= SOME (SELECT SUBQUERY4_t1 . `col_varchar_64` AS SUBQUERY4_field1 FROM `ts` AS SUBQUERY4_t1) GROUP BY field1 ;",
"Plan": [
"HashAgg 7992.00 root group by:test.ts.col_char_64, funcs:firstrow(test.ts.col_char_64)->test.ts.col_char_64",
"└─HashJoin 9990.00 root CARTESIAN inner join, other cond:or(ge(test.ts.col_char_64_not_null, Column#25), if(ne(Column#26, 0), NULL, 0))",
" ├─Selection(Build) 0.80 root ne(Column#27, 0)",
" │ └─HashAgg 1.00 root funcs:min(Column#33)->Column#25, funcs:sum(Column#34)->Column#26, funcs:count(Column#35)->Column#27",
" │ └─TableReader 1.00 root data:ExchangeSender",
" │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough",
" │ └─HashAgg 1.00 batchCop[tiflash] funcs:min(Column#39)->Column#33, funcs:sum(Column#40)->Column#34, funcs:count(1)->Column#35",
" │ └─Projection 10000.00 batchCop[tiflash] test.ts.col_varchar_64, cast(isnull(test.ts.col_varchar_64), decimal(22,0) BINARY)->Column#40",
" │ └─TableFullScan 10000.00 batchCop[tiflash] table:SUBQUERY4_t1 keep order:false, stats:pseudo",
" └─TableReader(Probe) 12487.50 root data:ExchangeSender",
" └─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough",
" └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.ts.col_varchar_64, test.ts.col_varchar_key)]",
" ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ",
" │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast",
" │ └─Selection 9990.00 cop[tiflash] not(isnull(test.ts.col_varchar_64))",
" │ └─TableFullScan 10000.00 cop[tiflash] table:table2 keep order:false, stats:pseudo",
" └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.ts.col_varchar_key))",
" └─TableFullScan 10000.00 cop[tiflash] table:SUBQUERY3_t1 keep order:false, stats:pseudo"
]
}
]
}
]
Loading

0 comments on commit f135c53

Please sign in to comment.