Skip to content

Commit

Permalink
executor: fix wrong plan type for dataReaderBuilder... (#17028) (#17036)
Browse files Browse the repository at this point in the history
  • Loading branch information
sre-bot authored May 9, 2020
1 parent 1c6b36d commit 75b3d09
Show file tree
Hide file tree
Showing 3 changed files with 46 additions and 1 deletion.
20 changes: 19 additions & 1 deletion executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -2610,7 +2610,12 @@ type mockPhysicalIndexReader struct {

func (builder *dataReaderBuilder) buildExecutorForIndexJoin(ctx context.Context, lookUpContents []*indexJoinLookUpContent,
IndexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) {
switch v := builder.Plan.(type) {
return builder.buildExecutorForIndexJoinInternal(ctx, builder.Plan, lookUpContents, IndexRanges, keyOff2IdxOff, cwc)
}

func (builder *dataReaderBuilder) buildExecutorForIndexJoinInternal(ctx context.Context, plan plannercore.Plan, lookUpContents []*indexJoinLookUpContent,
IndexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) {
switch v := plan.(type) {
case *plannercore.PhysicalTableReader:
return builder.buildTableReaderForIndexJoin(ctx, v, lookUpContents)
case *plannercore.PhysicalIndexReader:
Expand All @@ -2627,6 +2632,19 @@ func (builder *dataReaderBuilder) buildExecutorForIndexJoin(ctx context.Context,
// Then we need a Projection upon IndexLookupReader to prune the redundant column.
case *plannercore.PhysicalProjection:
return builder.buildProjectionForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc)
// Need to support physical selection because after PR 16389, TiDB will push down all the expr supported by TiKV or TiFlash
// in predicate push down stage, so if there is an expr which only supported by TiFlash, a physical selection will be added after index read
case *plannercore.PhysicalSelection:
childExec, err := builder.buildExecutorForIndexJoinInternal(ctx, v.Children()[0], lookUpContents, IndexRanges, keyOff2IdxOff, cwc)
if err != nil {
return nil, err
}
exec := &SelectionExec{
baseExecutor: newBaseExecutor(builder.ctx, v.Schema(), v.ExplainID(), childExec),
filters: v.Conditions,
}
err = exec.open(ctx)
return exec, err
case *mockPhysicalIndexReader:
return v.e, nil
}
Expand Down
4 changes: 4 additions & 0 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1169,6 +1169,10 @@ func (e *SelectionExec) Open(ctx context.Context) error {
if err := e.baseExecutor.Open(ctx); err != nil {
return err
}
return e.open(ctx)
}

func (e *SelectionExec) open(ctx context.Context) error {
e.memTracker = memory.NewTracker(e.id, -1)
e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker)
e.childResult = newFirstChunk(e.children[0])
Expand Down
23 changes: 23 additions & 0 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4976,6 +4976,29 @@ func (s *testIntegrationSuite) TestExprPushdown(c *C) {
tk.MustQuery("select id from t use index(key2) where substr(col2, 1, 1) = '5' and substr(col1, 1, 1) = '4'").Check(testkit.Rows("3"))
tk.MustQuery("select count(id) from t use index(key2) where substr(col2, 1, 1) = '5' and substr(col1, 1, 1) = '4'").Check(testkit.Rows("1"))
}
func (s *testIntegrationSuite) TestIssue16973(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1(id varchar(36) not null primary key, org_id varchar(36) not null, " +
"status tinyint default 1 not null, ns varchar(36) default '' not null);")
tk.MustExec("create table t2(id varchar(36) not null primary key, order_id varchar(36) not null, " +
"begin_time timestamp(3) default CURRENT_TIMESTAMP(3) not null);")
tk.MustExec("create index idx_oid on t2(order_id);")
tk.MustExec("insert into t1 value (1,1,1,'a');")
tk.MustExec("insert into t1 value (2,1,2,'a');")
tk.MustExec("insert into t1 value (3,1,3,'a');")
tk.MustExec("insert into t2 value (1,2,date'2020-05-08');")

rows := tk.MustQuery("explain SELECT /*+ INL_MERGE_JOIN(t1,t2) */ COUNT(*) FROM t1 LEFT JOIN t2 ON t1.id = t2.order_id WHERE t1.ns = 'a' AND t1.org_id IN (1) " +
"AND t1.status IN (2,6,10) AND timestampdiff(month, t2.begin_time, date'2020-05-06') = 0;").Rows()
c.Assert(fmt.Sprintf("%v", rows[1][0]), Matches, ".*IndexMergeJoin.*")
c.Assert(fmt.Sprintf("%v", rows[4][3]), Equals, "table:t1")
c.Assert(fmt.Sprintf("%v", rows[5][0]), Matches, ".*Selection.*")
c.Assert(fmt.Sprintf("%v", rows[9][3]), Equals, "table:t2")
tk.MustQuery("SELECT /*+ INL_MERGE_JOIN(t1,t2) */ COUNT(*) FROM t1 LEFT JOIN t2 ON t1.id = t2.order_id WHERE t1.ns = 'a' AND t1.org_id IN (1) " +
"AND t1.status IN (2,6,10) AND timestampdiff(month, t2.begin_time, date'2020-05-06') = 0;").Check(testkit.Rows("1"))
}

func (s *testIntegrationSuite) TestExprPushdownBlacklist(c *C) {
tk := testkit.NewTestKit(c, s.store)
Expand Down

0 comments on commit 75b3d09

Please sign in to comment.