Skip to content

Commit

Permalink
executor: �do not reorder handles when building requests for `IndexMe…
Browse files Browse the repository at this point in the history
…rgeJoin` (#20138) (#20172)
  • Loading branch information
qw4990 authored Oct 3, 2020
1 parent 8edaa34 commit a605570
Show file tree
Hide file tree
Showing 6 changed files with 29 additions and 14 deletions.
22 changes: 12 additions & 10 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -2638,15 +2638,15 @@ type mockPhysicalIndexReader struct {
}

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

func (builder *dataReaderBuilder) buildExecutorForIndexJoinInternal(ctx context.Context, plan plannercore.Plan, lookUpContents []*indexJoinLookUpContent,
IndexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) {
IndexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager, canReorderHandles bool) (Executor, error) {
switch v := plan.(type) {
case *plannercore.PhysicalTableReader:
return builder.buildTableReaderForIndexJoin(ctx, v, lookUpContents)
return builder.buildTableReaderForIndexJoin(ctx, v, lookUpContents, canReorderHandles)
case *plannercore.PhysicalIndexReader:
return builder.buildIndexReaderForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc)
case *plannercore.PhysicalIndexLookUpReader:
Expand All @@ -2664,7 +2664,7 @@ func (builder *dataReaderBuilder) buildExecutorForIndexJoinInternal(ctx context.
// 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)
childExec, err := builder.buildExecutorForIndexJoinInternal(ctx, v.Children()[0], lookUpContents, IndexRanges, keyOff2IdxOff, cwc, canReorderHandles)
if err != nil {
return nil, err
}
Expand All @@ -2683,7 +2683,7 @@ func (builder *dataReaderBuilder) buildExecutorForIndexJoinInternal(ctx context.
func (builder *dataReaderBuilder) buildUnionScanForIndexJoin(ctx context.Context, v *plannercore.PhysicalUnionScan,
values []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) {
childBuilder := &dataReaderBuilder{Plan: v.Children()[0], executorBuilder: builder.executorBuilder}
reader, err := childBuilder.buildExecutorForIndexJoin(ctx, values, indexRanges, keyOff2IdxOff, cwc)
reader, err := childBuilder.buildExecutorForIndexJoin(ctx, values, indexRanges, keyOff2IdxOff, cwc, true)
if err != nil {
return nil, err
}
Expand All @@ -2692,7 +2692,7 @@ func (builder *dataReaderBuilder) buildUnionScanForIndexJoin(ctx context.Context
return us, err
}

func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalTableReader, lookUpContents []*indexJoinLookUpContent) (Executor, error) {
func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalTableReader, lookUpContents []*indexJoinLookUpContent, canReorderHandles bool) (Executor, error) {
e, err := buildNoRangeTableReader(builder.executorBuilder, v)
if err != nil {
return nil, err
Expand All @@ -2712,16 +2712,18 @@ func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Conte
handles = append(handles, handle)
}
}
return builder.buildTableReaderFromHandles(ctx, e, handles)
return builder.buildTableReaderFromHandles(ctx, e, handles, canReorderHandles)
}

func (builder *dataReaderBuilder) buildTableReaderFromHandles(ctx context.Context, e *TableReaderExecutor, handles []int64) (Executor, error) {
func (builder *dataReaderBuilder) buildTableReaderFromHandles(ctx context.Context, e *TableReaderExecutor, handles []int64, canReorderHandles bool) (Executor, error) {
startTS, err := builder.getSnapshotTS()
if err != nil {
return nil, err
}

sort.Sort(sortutil.Int64Slice(handles))
if canReorderHandles {
sort.Sort(sortutil.Int64Slice(handles))
}
var b distsql.RequestBuilder
kvReq, err := b.SetTableHandles(getPhysicalTableID(e.table), handles).
SetDAGRequest(e.dagPB).
Expand Down
2 changes: 1 addition & 1 deletion executor/distsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -533,7 +533,7 @@ func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, handles []in
plans: e.tblPlans,
}
tableReaderExec.buildVirtualColumnInfo()
tableReader, err := e.dataReaderBuilder.buildTableReaderFromHandles(ctx, tableReaderExec, handles)
tableReader, err := e.dataReaderBuilder.buildTableReaderFromHandles(ctx, tableReaderExec, handles, true)
if err != nil {
logutil.Logger(ctx).Error("build table reader from handles failed", zap.Error(err))
return nil, err
Expand Down
2 changes: 1 addition & 1 deletion executor/index_lookup_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -655,7 +655,7 @@ func (iw *innerWorker) fetchInnerResults(ctx context.Context, task *lookUpJoinTa
atomic.AddInt64(&iw.stats.fetch, int64(time.Since(start)))
}()
}
innerExec, err := iw.readerBuilder.buildExecutorForIndexJoin(ctx, lookUpContent, iw.indexRanges, iw.keyOff2IdxOff, iw.nextColCompareFilters)
innerExec, err := iw.readerBuilder.buildExecutorForIndexJoin(ctx, lookUpContent, iw.indexRanges, iw.keyOff2IdxOff, iw.nextColCompareFilters, true)
if err != nil {
return err
}
Expand Down
2 changes: 1 addition & 1 deletion executor/index_lookup_merge_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -496,7 +496,7 @@ func (imw *innerMergeWorker) handleTask(ctx context.Context, task *lookUpMergeJo
dLookUpKeys[i], dLookUpKeys[lenKeys-i-1] = dLookUpKeys[lenKeys-i-1], dLookUpKeys[i]
}
}
imw.innerExec, err = imw.readerBuilder.buildExecutorForIndexJoin(ctx, dLookUpKeys, imw.indexRanges, imw.keyOff2IdxOff, imw.nextColCompareFilters)
imw.innerExec, err = imw.readerBuilder.buildExecutorForIndexJoin(ctx, dLookUpKeys, imw.indexRanges, imw.keyOff2IdxOff, imw.nextColCompareFilters, false)
if err != nil {
return err
}
Expand Down
13 changes: 13 additions & 0 deletions executor/index_lookup_merge_join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,19 @@ func (s *testSuite9) TestIssue18631(c *C) {
"1 1 1 1 1 1 1 1"))
}

func (s *testSuite9) TestIssue20137(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1 (id bigint(20) unsigned, primary key(id))")
tk.MustExec("create table t2 (id bigint(20) unsigned)")
tk.MustExec("insert into t1 values (8738875760185212610)")
tk.MustExec("insert into t1 values (9814441339970117597)")
tk.MustExec("insert into t2 values (8738875760185212610)")
tk.MustExec("insert into t2 values (9814441339970117597)")
tk.MustQuery("select /*+ INL_MERGE_JOIN(t1, t2) */ * from t2 left join t1 on t1.id = t2.id order by t1.id").Check(
testkit.Rows("8738875760185212610 8738875760185212610", "9814441339970117597 9814441339970117597"))
}

func (s *testSuiteAgg) TestIndexJoinOnSinglePartitionTable(c *C) {
// For issue 19145
tk := testkit.NewTestKitWithInit(c, s.store)
Expand Down
2 changes: 1 addition & 1 deletion executor/index_merge_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -424,7 +424,7 @@ func (e *IndexMergeReaderExecutor) buildFinalTableReader(ctx context.Context, ha
plans: e.tblPlans,
}
tableReaderExec.buildVirtualColumnInfo()
tableReader, err := e.dataReaderBuilder.buildTableReaderFromHandles(ctx, tableReaderExec, handles)
tableReader, err := e.dataReaderBuilder.buildTableReaderFromHandles(ctx, tableReaderExec, handles, true)
if err != nil {
logutil.Logger(ctx).Error("build table reader from handles failed", zap.Error(err))
return nil, err
Expand Down

0 comments on commit a605570

Please sign in to comment.