From 5bdb46f47531a8284fd934a1208a795b8ae8d8de Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 27 Nov 2018 17:19:27 +0800 Subject: [PATCH 01/27] planner, executor: index join enhancement --- executor/builder.go | 78 ++++-- executor/executor_pkg_test.go | 21 +- executor/index_lookup_join.go | 56 +++-- planner/core/exhaust_physical_plans.go | 313 ++++++++++++++++++++----- planner/core/physical_plans.go | 3 +- planner/core/resolve_indices.go | 6 + util/ranger/detacher.go | 14 +- util/ranger/ranger.go | 2 +- 8 files changed, 366 insertions(+), 127 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 47663e20e5125..9224dfdd1dd03 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -38,7 +38,6 @@ import ( "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" @@ -1086,8 +1085,12 @@ func (b *executorBuilder) buildHashAgg(v *plannercore.PhysicalHashAgg) Executor e.FinalAggFuncs = append(e.FinalAggFuncs, finalAggFunc) if partialAggDesc.Name == ast.AggFuncGroupConcat { // For group_concat, finalAggFunc and partialAggFunc need shared `truncate` flag to do duplicate. - finalAggFunc.(interface{ SetTruncated(t *int32) }).SetTruncated( - partialAggFunc.(interface{ GetTruncated() *int32 }).GetTruncated(), + finalAggFunc.(interface { + SetTruncated(t *int32) + }).SetTruncated( + partialAggFunc.(interface { + GetTruncated() *int32 + }).GetTruncated(), ) } } @@ -1848,25 +1851,25 @@ type dataReaderBuilder struct { *executorBuilder } -func (builder *dataReaderBuilder) buildExecutorForIndexJoin(ctx context.Context, datums [][]types.Datum, - IndexRanges []*ranger.Range, keyOff2IdxOff []int) (Executor, error) { +func (builder *dataReaderBuilder) buildExecutorForIndexJoin(ctx context.Context, lookUpContents []*indexJoinLookUpContent, + IndexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCompareOps) (Executor, error) { switch v := builder.Plan.(type) { case *plannercore.PhysicalTableReader: - return builder.buildTableReaderForIndexJoin(ctx, v, datums) + return builder.buildTableReaderForIndexJoin(ctx, v, lookUpContents) case *plannercore.PhysicalIndexReader: - return builder.buildIndexReaderForIndexJoin(ctx, v, datums, IndexRanges, keyOff2IdxOff) + return builder.buildIndexReaderForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc) case *plannercore.PhysicalIndexLookUpReader: - return builder.buildIndexLookUpReaderForIndexJoin(ctx, v, datums, IndexRanges, keyOff2IdxOff) + return builder.buildIndexLookUpReaderForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc) case *plannercore.PhysicalUnionScan: - return builder.buildUnionScanForIndexJoin(ctx, v, datums, IndexRanges, keyOff2IdxOff) + return builder.buildUnionScanForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc) } return nil, errors.New("Wrong plan type for dataReaderBuilder") } func (builder *dataReaderBuilder) buildUnionScanForIndexJoin(ctx context.Context, v *plannercore.PhysicalUnionScan, - values [][]types.Datum, indexRanges []*ranger.Range, keyOff2IdxOff []int) (Executor, error) { + values []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCompareOps) (Executor, error) { childBuilder := &dataReaderBuilder{v.Children()[0], builder.executorBuilder} - reader, err := childBuilder.buildExecutorForIndexJoin(ctx, values, indexRanges, keyOff2IdxOff) + reader, err := childBuilder.buildExecutorForIndexJoin(ctx, values, indexRanges, keyOff2IdxOff, cwc) if err != nil { return nil, err } @@ -1879,14 +1882,14 @@ func (builder *dataReaderBuilder) buildUnionScanForIndexJoin(ctx context.Context return us, nil } -func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalTableReader, datums [][]types.Datum) (Executor, error) { +func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalTableReader, lookUpContents []*indexJoinLookUpContent) (Executor, error) { e, err := buildNoRangeTableReader(builder.executorBuilder, v) if err != nil { return nil, errors.Trace(err) } - handles := make([]int64, 0, len(datums)) - for _, datum := range datums { - handles = append(handles, datum[0].GetInt64()) + handles := make([]int64, 0, len(lookUpContents)) + for _, content := range lookUpContents { + handles = append(handles, content.keys[0].GetInt64()) } return builder.buildTableReaderFromHandles(ctx, e, handles) } @@ -1915,12 +1918,12 @@ func (builder *dataReaderBuilder) buildTableReaderFromHandles(ctx context.Contex } func (builder *dataReaderBuilder) buildIndexReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalIndexReader, - values [][]types.Datum, indexRanges []*ranger.Range, keyOff2IdxOff []int) (Executor, error) { + lookUpContents []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCompareOps) (Executor, error) { e, err := buildNoRangeIndexReader(builder.executorBuilder, v) if err != nil { return nil, errors.Trace(err) } - kvRanges, err := buildKvRangesForIndexJoin(e.ctx.GetSessionVars().StmtCtx, e.physicalTableID, e.index.ID, values, indexRanges, keyOff2IdxOff) + kvRanges, err := buildKvRangesForIndexJoin(e.ctx, e.physicalTableID, e.index.ID, lookUpContents, indexRanges, keyOff2IdxOff, cwc) if err != nil { return nil, errors.Trace(err) } @@ -1929,12 +1932,12 @@ func (builder *dataReaderBuilder) buildIndexReaderForIndexJoin(ctx context.Conte } func (builder *dataReaderBuilder) buildIndexLookUpReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalIndexLookUpReader, - values [][]types.Datum, indexRanges []*ranger.Range, keyOff2IdxOff []int) (Executor, error) { + lookUpContents []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCompareOps) (Executor, error) { e, err := buildNoRangeIndexLookUpReader(builder.executorBuilder, v) if err != nil { return nil, errors.Trace(err) } - kvRanges, err := buildKvRangesForIndexJoin(e.ctx.GetSessionVars().StmtCtx, e.physicalTableID, e.index.ID, values, indexRanges, keyOff2IdxOff) + kvRanges, err := buildKvRangesForIndexJoin(e.ctx, e.physicalTableID, e.index.ID, lookUpContents, indexRanges, keyOff2IdxOff, cwc) if err != nil { return nil, errors.Trace(err) } @@ -1943,17 +1946,40 @@ func (builder *dataReaderBuilder) buildIndexLookUpReaderForIndexJoin(ctx context } // buildKvRangesForIndexJoin builds kv ranges for index join when the inner plan is index scan plan. -func buildKvRangesForIndexJoin(sc *stmtctx.StatementContext, tableID, indexID int64, keyDatums [][]types.Datum, indexRanges []*ranger.Range, keyOff2IdxOff []int) ([]kv.KeyRange, error) { - kvRanges := make([]kv.KeyRange, 0, len(indexRanges)*len(keyDatums)) - for _, val := range keyDatums { - for _, ran := range indexRanges { +func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, lookUpContents []*indexJoinLookUpContent, + ranges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCompareOps) ([]kv.KeyRange, error) { + kvRanges := make([]kv.KeyRange, 0, len(ranges)*len(lookUpContents)) + lastPos := len(ranges[0].LowVal) - 1 + sc := ctx.GetSessionVars().StmtCtx + for _, content := range lookUpContents { + for _, ran := range ranges { for keyOff, idxOff := range keyOff2IdxOff { - ran.LowVal[idxOff] = val[keyOff] - ran.HighVal[idxOff] = val[keyOff] + ran.LowVal[idxOff] = content.keys[keyOff] + ran.HighVal[idxOff] = content.keys[keyOff] } } + if cwc != nil { + nextColRanges, err := cwc.BuildRangesByRow(ctx, content.row) + if err != nil { + return nil, err + } + for _, nextColRan := range nextColRanges { + for _, ran := range ranges { + ran.LowVal[lastPos] = nextColRan.LowVal[0] + ran.LowVal[lastPos] = nextColRan.HighVal[0] + ran.LowExclude = nextColRan.LowExclude + ran.HighExclude = nextColRan.HighExclude + } + tmpKvRanges, err := distsql.IndexRangesToKVRanges(sc, tableID, indexID, ranges, nil) + if err != nil { + return nil, errors.Trace(err) + } + kvRanges = append(kvRanges, tmpKvRanges...) + } + continue + } - tmpKvRanges, err := distsql.IndexRangesToKVRanges(sc, tableID, indexID, indexRanges, nil) + tmpKvRanges, err := distsql.IndexRangesToKVRanges(sc, tableID, indexID, ranges, nil) if err != nil { return nil, errors.Trace(err) } diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index b699cee269ca3..1520bb30feff8 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -14,15 +14,12 @@ package executor import ( - "time" - . "github.com/pingcap/check" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/auth" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" @@ -128,7 +125,7 @@ func buildSchema(names []string, ftypes []byte) *expression.Schema { return schema } -func (s *testExecSuite) TestBuildKvRangesForIndexJoin(c *C) { +func (s *testExecSuite) TestBuildKvRangesForIndexJoinWithoutCwc(c *C) { indexRanges := make([]*ranger.Range, 0, 6) indexRanges = append(indexRanges, generateIndexRange(1, 1, 1, 1, 1)) indexRanges = append(indexRanges, generateIndexRange(1, 1, 2, 1, 1)) @@ -137,16 +134,16 @@ func (s *testExecSuite) TestBuildKvRangesForIndexJoin(c *C) { indexRanges = append(indexRanges, generateIndexRange(2, 1, 1, 1, 1)) indexRanges = append(indexRanges, generateIndexRange(2, 1, 2, 1, 1)) - joinKeyRows := make([][]types.Datum, 0, 5) - joinKeyRows = append(joinKeyRows, generateDatumSlice(1, 1)) - joinKeyRows = append(joinKeyRows, generateDatumSlice(1, 2)) - joinKeyRows = append(joinKeyRows, generateDatumSlice(2, 1)) - joinKeyRows = append(joinKeyRows, generateDatumSlice(2, 2)) - joinKeyRows = append(joinKeyRows, generateDatumSlice(2, 3)) + joinKeyRows := make([]*indexJoinLookUpContent, 0, 5) + joinKeyRows = append(joinKeyRows, &indexJoinLookUpContent{keys: generateDatumSlice(1, 1)}) + joinKeyRows = append(joinKeyRows, &indexJoinLookUpContent{keys: generateDatumSlice(1, 2)}) + joinKeyRows = append(joinKeyRows, &indexJoinLookUpContent{keys: generateDatumSlice(2, 1)}) + joinKeyRows = append(joinKeyRows, &indexJoinLookUpContent{keys: generateDatumSlice(2, 2)}) + joinKeyRows = append(joinKeyRows, &indexJoinLookUpContent{keys: generateDatumSlice(2, 3)}) keyOff2IdxOff := []int{1, 3} - sc := &stmtctx.StatementContext{TimeZone: time.Local} - kvRanges, err := buildKvRangesForIndexJoin(sc, 0, 0, joinKeyRows, indexRanges, keyOff2IdxOff) + ctx := mock.NewContext() + kvRanges, err := buildKvRangesForIndexJoin(ctx, 0, 0, joinKeyRows, indexRanges, keyOff2IdxOff, nil) c.Assert(err, IsNil) // Check the kvRanges is in order. for i, kvRange := range kvRanges { diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 0a13aff72fe06..7ff3a926ad1ed 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/expression" + plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" @@ -67,6 +68,8 @@ type IndexLookUpJoin struct { keyOff2IdxOff []int innerPtrBytes [][]byte + nextColCompareFilters *plannercore.ColWithCompareOps + memTracker *memory.Tracker // track memory usage. } @@ -123,8 +126,9 @@ type innerWorker struct { ctx sessionctx.Context executorChk *chunk.Chunk - indexRanges []*ranger.Range - keyOff2IdxOff []int + indexRanges []*ranger.Range + nextColCompareFilters *plannercore.ColWithCompareOps + keyOff2IdxOff []int } // Open implements the Executor interface. @@ -423,13 +427,18 @@ func (iw *innerWorker) run(ctx context.Context, wg *sync.WaitGroup) { } } +type indexJoinLookUpContent struct { + keys []types.Datum + row chunk.Row +} + func (iw *innerWorker) handleTask(ctx context.Context, task *lookUpJoinTask) error { - dLookUpKeys, err := iw.constructDatumLookupKeys(task) + lookUpContents, err := iw.constructDatumLookupKeys(task) if err != nil { return errors.Trace(err) } - dLookUpKeys = iw.sortAndDedupDatumLookUpKeys(dLookUpKeys) - err = iw.fetchInnerResults(ctx, task, dLookUpKeys) + lookUpContents = iw.sortAndDedupLookUpContents(lookUpContents) + err = iw.fetchInnerResults(ctx, task, lookUpContents) if err != nil { return errors.Trace(err) } @@ -440,8 +449,8 @@ func (iw *innerWorker) handleTask(ctx context.Context, task *lookUpJoinTask) err return nil } -func (iw *innerWorker) constructDatumLookupKeys(task *lookUpJoinTask) ([][]types.Datum, error) { - dLookUpKeys := make([][]types.Datum, 0, task.outerResult.NumRows()) +func (iw *innerWorker) constructDatumLookupKeys(task *lookUpJoinTask) ([]*indexJoinLookUpContent, error) { + lookUpContents := make([]*indexJoinLookUpContent, 0, task.outerResult.NumRows()) keyBuf := make([]byte, 0, 64) for i := 0; i < task.outerResult.NumRows(); i++ { dLookUpKey, err := iw.constructDatumLookupKey(task, i) @@ -460,11 +469,11 @@ func (iw *innerWorker) constructDatumLookupKeys(task *lookUpJoinTask) ([][]types } // Store the encoded lookup key in chunk, so we can use it to lookup the matched inners directly. task.encodedLookUpKeys.AppendBytes(0, keyBuf) - dLookUpKeys = append(dLookUpKeys, dLookUpKey) + lookUpContents = append(lookUpContents, &indexJoinLookUpContent{keys: dLookUpKey, row: task.outerResult.GetRow(i)}) } task.memTracker.Consume(task.encodedLookUpKeys.MemoryUsage()) - return dLookUpKeys, nil + return lookUpContents, nil } func (iw *innerWorker) constructDatumLookupKey(task *lookUpJoinTask, rowIdx int) ([]types.Datum, error) { @@ -496,20 +505,23 @@ func (iw *innerWorker) constructDatumLookupKey(task *lookUpJoinTask, rowIdx int) return dLookupKey, nil } -func (iw *innerWorker) sortAndDedupDatumLookUpKeys(dLookUpKeys [][]types.Datum) [][]types.Datum { - if len(dLookUpKeys) < 2 { - return dLookUpKeys +func (iw *innerWorker) sortAndDedupLookUpContents(lookUpContents []*indexJoinLookUpContent) []*indexJoinLookUpContent { + if len(lookUpContents) < 2 { + return lookUpContents } sc := iw.ctx.GetSessionVars().StmtCtx - sort.Slice(dLookUpKeys, func(i, j int) bool { - cmp := compareRow(sc, dLookUpKeys[i], dLookUpKeys[j]) - return cmp < 0 + sort.Slice(lookUpContents, func(i, j int) bool { + cmp := compareRow(sc, lookUpContents[i].keys, lookUpContents[j].keys) + if cmp != 0 || iw.nextColCompareFilters == nil { + return cmp < 0 + } + return iw.nextColCompareFilters.CompareRow(lookUpContents[i].row, lookUpContents[j].row) < 0 }) - deDupedLookupKeys := dLookUpKeys[:1] - for i := 1; i < len(dLookUpKeys); i++ { - cmp := compareRow(sc, dLookUpKeys[i], dLookUpKeys[i-1]) - if cmp != 0 { - deDupedLookupKeys = append(deDupedLookupKeys, dLookUpKeys[i]) + deDupedLookupKeys := lookUpContents[:1] + for i := 1; i < len(lookUpContents); i++ { + cmp := compareRow(sc, lookUpContents[i].keys, lookUpContents[i-1].keys) + if cmp != 0 || (iw.nextColCompareFilters != nil && iw.nextColCompareFilters.CompareRow(lookUpContents[i].row, lookUpContents[i-1].row) != 0) { + deDupedLookupKeys = append(deDupedLookupKeys, lookUpContents[i]) } } return deDupedLookupKeys @@ -529,8 +541,8 @@ func compareRow(sc *stmtctx.StatementContext, left, right []types.Datum) int { return 0 } -func (iw *innerWorker) fetchInnerResults(ctx context.Context, task *lookUpJoinTask, dLookUpKeys [][]types.Datum) error { - innerExec, err := iw.readerBuilder.buildExecutorForIndexJoin(ctx, dLookUpKeys, iw.indexRanges, iw.keyOff2IdxOff) +func (iw *innerWorker) fetchInnerResults(ctx context.Context, task *lookUpJoinTask, lookUpContent []*indexJoinLookUpContent) error { + innerExec, err := iw.readerBuilder.buildExecutorForIndexJoin(ctx, lookUpContent, iw.indexRanges, iw.keyOff2IdxOff, iw.nextColCompareFilters) if err != nil { return errors.Trace(err) } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 67afd825f923b..b0bd9861d5eb1 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -16,16 +16,17 @@ package core import ( "math" - "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/ranger" + log "github.com/sirupsen/logrus" ) func (p *LogicalUnionScan) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { @@ -310,7 +311,7 @@ func joinKeysMatchIndex(keys, indexCols []*expression.Column, colLengths []int) // When inner plan is TableReader, the parameter `ranges` will be nil. Because pk only have one column. So all of its range // is generated during execution time. func (p *LogicalJoin) constructIndexJoin(prop *property.PhysicalProperty, innerJoinKeys, outerJoinKeys []*expression.Column, outerIdx int, - innerPlan PhysicalPlan, ranges []*ranger.Range, keyOff2IdxOff []int) []PhysicalPlan { + innerPlan PhysicalPlan, ranges []*ranger.Range, keyOff2IdxOff []int, compareFilters *ColWithCompareOps) []PhysicalPlan { joinType := p.JoinType outerSchema := p.children[outerIdx].Schema() // If the order by columns are not all from outer child, index join cannot promise the order. @@ -345,6 +346,7 @@ func (p *LogicalJoin) constructIndexJoin(prop *property.PhysicalProperty, innerJ innerPlan: innerPlan, KeyOff2IdxOff: newKeyOff, Ranges: ranges, + compareFilters: compareFilters, }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), chReqProps...) join.SetSchema(p.schema) return []PhysicalPlan{join} @@ -397,7 +399,7 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou innerPlan := p.constructInnerTableScan(ds, pkCol, outerJoinKeys, us) // Since the primary key means one value corresponding to exact one row, this will always be a no worse one // comparing to other index. - return p.constructIndexJoin(prop, innerJoinKeys, outerJoinKeys, outerIdx, innerPlan, nil, keyOff2IdxOff) + return p.constructIndexJoin(prop, innerJoinKeys, outerJoinKeys, outerIdx, innerPlan, nil, keyOff2IdxOff, nil) } } var ( @@ -405,14 +407,19 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou rangesOfBest []*ranger.Range maxUsedCols int remainedOfBest []expression.Expression - keyOff2IdxOff []int + idxOff2KeyOff []int + comparesOfBest *ColWithCompareOps ) for _, path := range ds.possibleAccessPaths { if path.isTablePath { continue } indexInfo := path.index - ranges, remained, tmpKeyOff2IdxOff := p.buildRangeForIndexJoin(indexInfo, ds, innerJoinKeys) + ranges, tmpIdxOff2KeyOff, remained, compareFilters, err := p.analyzeLookUpFilters(indexInfo, ds, innerJoinKeys) + if err != nil { + log.Warnf("[planner]: error happened when build index join: %v", err) + continue + } // We choose the index by the number of used columns of the range, the much the better. // Notice that there may be the cases like `t1.a=t2.a and b > 2 and b < 1`. So ranges can be nil though the conditions are valid. // But obviously when the range is nil, we don't need index join. @@ -421,12 +428,22 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou maxUsedCols = len(ranges[0].LowVal) rangesOfBest = ranges remainedOfBest = remained - keyOff2IdxOff = tmpKeyOff2IdxOff + idxOff2KeyOff = tmpIdxOff2KeyOff + comparesOfBest = compareFilters } } if bestIndexInfo != nil { + keyOff2IdxOff := make([]int, len(innerJoinKeys)) + for i := range keyOff2IdxOff { + keyOff2IdxOff[i] = -1 + } + for idxOff, keyOff := range idxOff2KeyOff { + if keyOff != -1 { + keyOff2IdxOff[keyOff] = idxOff + } + } innerPlan := p.constructInnerIndexScan(ds, bestIndexInfo, remainedOfBest, outerJoinKeys, us) - return p.constructIndexJoin(prop, innerJoinKeys, outerJoinKeys, outerIdx, innerPlan, rangesOfBest, keyOff2IdxOff) + return p.constructIndexJoin(prop, innerJoinKeys, outerJoinKeys, outerIdx, innerPlan, rangesOfBest, keyOff2IdxOff, comparesOfBest) } return nil } @@ -521,78 +538,252 @@ func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexIn return p.constructInnerUnionScan(us, reader) } -// buildRangeForIndexJoin checks whether this index can be used for building index join and return the range if this index is ok. -// If this index is invalid, just return nil range. -func (p *LogicalJoin) buildRangeForIndexJoin(indexInfo *model.IndexInfo, innerPlan *DataSource, innerJoinKeys []*expression.Column) ( - []*ranger.Range, []expression.Expression, []int) { - idxCols, colLengths := expression.IndexInfo2Cols(innerPlan.Schema().Columns, indexInfo) - if len(idxCols) == 0 { - return nil, nil, nil - } +var symmetricOp = map[string]string{ + ast.LT: ast.GT, + ast.GE: ast.LE, + ast.GT: ast.LT, + ast.LE: ast.GE, +} - // Extract the filter to calculate access and the filters that must be remained ones. - access, eqConds, remained, keyOff2IdxOff := p.buildFakeEqCondsForIndexJoin(innerJoinKeys, idxCols, colLengths, innerPlan.pushedDownConds) +type ColWithCompareOps struct { + targetCol *expression.Column + OpType []string + opArg []expression.Expression + tmpConstant []*expression.Constant + affectedColSchema *expression.Schema + compareFuncs []chunk.CompareFunc +} - if len(keyOff2IdxOff) == 0 { - return nil, nil, nil +func (cwc *ColWithCompareOps) appendNewExpr(opName string, arg expression.Expression, affectedCols []*expression.Column) { + cwc.OpType = append(cwc.OpType, opName) + cwc.opArg = append(cwc.opArg, arg) + cwc.tmpConstant = append(cwc.tmpConstant, &expression.Constant{RetType: cwc.targetCol.RetType}) + for _, col := range affectedCols { + if cwc.affectedColSchema.Contains(col) { + continue + } + cwc.compareFuncs = append(cwc.compareFuncs, chunk.GetCompareFunc(col.RetType)) + cwc.affectedColSchema.Append(col) } +} - // In `buildFakeEqCondsForIndexJoin`, we construct the equal conditions for join keys and remove filters that contain the join keys' column. - // When t1.a = t2.a and t1.a > 1, we can also guarantee that t1.a > 1 won't be chosen as the access condition. - // So the equal conditions we built can be successfully used to build a range if they can be used. They won't be affected by the existing filters. - ranges, accesses, moreRemained, _, err := ranger.DetachCondAndBuildRangeForIndex(p.ctx, access, idxCols, colLengths) - if err != nil { - terror.Log(errors.Trace(err)) - return nil, nil, nil +func (cwc *ColWithCompareOps) CompareRow(lhs, rhs chunk.Row) int { + for i, col := range cwc.affectedColSchema.Columns { + ret := cwc.compareFuncs[i](lhs, col.Index, rhs, col.Index) + if ret != 0 { + return ret + } } + return 0 +} - // We should guarantee that all the join's equal condition is used. - for _, eqCond := range eqConds { - if !expression.Contains(accesses, eqCond) { - return nil, nil, nil +func (cwc *ColWithCompareOps) BuildRangesByRow(ctx sessionctx.Context, row chunk.Row) ([]*ranger.Range, error) { + exprs := make([]expression.Expression, len(cwc.OpType)) + for i, opType := range cwc.OpType { + constantArg, err := cwc.opArg[i].Eval(row) + if err != nil { + return nil, err } + cwc.tmpConstant[i].Value = constantArg + newExpr, err := expression.NewFunction(ctx, opType, types.NewFieldType(mysql.TypeTiny), cwc.targetCol, cwc.tmpConstant[i]) + if err != nil { + return nil, err + } + exprs = append(exprs, newExpr) } - - return ranges, append(remained, moreRemained...), keyOff2IdxOff + ranges, err := ranger.BuildColumnRange(exprs, ctx.GetSessionVars().StmtCtx, cwc.targetCol.RetType) + if err != nil { + return nil, err + } + return ranges, nil } -func (p *LogicalJoin) buildFakeEqCondsForIndexJoin(keys, idxCols []*expression.Column, colLengths []int, - innerFilters []expression.Expression) (accesses, eqConds, remained []expression.Expression, keyOff2IdxOff []int) { - // Check whether all join keys match one column from index. - keyOff2IdxOff = joinKeysMatchIndex(keys, idxCols, colLengths) - if keyOff2IdxOff == nil { - return nil, nil, nil, nil +func (cwc *ColWithCompareOps) resolveIndices(schema *expression.Schema) { + for i := range cwc.opArg { + cwc.opArg[i] = cwc.opArg[i].ResolveIndices(schema) } +} - usableKeys := make([]*expression.Column, 0, len(keys)) - - conds := make([]expression.Expression, 0, len(keys)+len(innerFilters)) - eqConds = make([]expression.Expression, 0, len(keys)) - // Construct a fake equal expression for every join key for calculating the range. - for i, key := range keys { - if keyOff2IdxOff[i] < 0 { +func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan *DataSource, innerJoinKeys []*expression.Column) ([]*ranger.Range, []int, []expression.Expression, *ColWithCompareOps, error) { + idxCols, colLengths := expression.IndexInfo2Cols(innerPlan.schema.Columns, indexInfo) + if len(idxCols) == 0 { + return nil, nil, nil, nil, nil + } + tmpSchema := expression.NewSchema(innerJoinKeys...) + idxOff2keyOff := make([]int, len(idxCols)) + possibleUsedKeys := make([]*expression.Column, 0, len(idxCols)) + notKeyIdxCols := make([]*expression.Column, 0, len(idxCols)) + notKeyIdxColsLen := make([]int, 0, len(idxCols)) + matchedKeyCnt := 0 + for i, idxCol := range idxCols { + idxOff2keyOff[i] = tmpSchema.ColumnIndex(idxCol) + if idxOff2keyOff[i] >= 0 { + matchedKeyCnt++ + possibleUsedKeys = append(possibleUsedKeys, idxCol) continue } - usableKeys = append(usableKeys, key) - // Int datum 1 can convert to all column's type(numeric type, string type, json, time type, enum, set) safely. - fakeConstant := &expression.Constant{Value: types.NewIntDatum(1), RetType: key.GetType()} - eqFunc := expression.NewFunctionInternal(p.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), key, fakeConstant) - conds = append(conds, eqFunc) - eqConds = append(eqConds, eqFunc) + notKeyIdxCols = append(notKeyIdxCols, idxCol) + notKeyIdxColsLen = append(notKeyIdxColsLen, colLengths[i]) } - - // Look into every `innerFilter`, if it contains join keys' column, put this filter into `remained` part directly. - remained = make([]expression.Expression, 0, len(innerFilters)) - for _, filter := range innerFilters { - affectedCols := expression.ExtractColumns(filter) - if expression.ColumnSliceIsIntersect(affectedCols, usableKeys) { - remained = append(remained, filter) + if matchedKeyCnt <= 0 { + return nil, nil, nil, nil, nil + } + keyMatchedLen := len(idxCols) - 1 + for ; keyMatchedLen > 0; keyMatchedLen-- { + if idxOff2keyOff[keyMatchedLen] == -1 { + continue + } + } + remained := make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) + rangeFilterCandidates := make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) + for _, innerFilter := range innerPlan.pushedDownConds { + affectedCols := expression.ExtractColumns(innerFilter) + if expression.ColumnSliceIsIntersect(affectedCols, possibleUsedKeys) { + remained = append(remained, innerFilter) continue } - conds = append(conds, filter) + rangeFilterCandidates = append(rangeFilterCandidates, innerFilter) + } + notKeyEqAndIn, remainedEqAndIn, rangeFilterCandidates, _ := ranger.ExtractEqAndInCondition(p.ctx, rangeFilterCandidates, notKeyIdxCols, notKeyIdxColsLen) + // We hope that the index cols appeared in the join keys can all be used to build range. If it cannot be satisfied, + // we'll mark this index as cannot be used for index join. + if len(notKeyEqAndIn) < keyMatchedLen-matchedKeyCnt { + return nil, nil, nil, nil, nil + } + remained = append(remained, remainedEqAndIn...) + nextColPos := matchedKeyCnt + len(notKeyEqAndIn) + // If all cols have been considered, we can return the current result. + if nextColPos == len(idxCols) { + ranges, err := p.buildTemplateRange(idxOff2keyOff, matchedKeyCnt, notKeyEqAndIn, nil, false) + if err != nil { + return nil, nil, nil, nil, err + } + return ranges, idxOff2keyOff, remained, nil, nil + } + nextCol := idxCols[nextColPos] + nextColCmpFilterManager := &ColWithCompareOps{ + targetCol: nextCol, + affectedColSchema: expression.NewSchema(), + } +loopCandidates: + for _, filter := range rangeFilterCandidates { + sf, ok := filter.(*expression.ScalarFunction) + if !ok || !(sf.FuncName.L == ast.LE || sf.FuncName.L == ast.LT || sf.FuncName.L == ast.GE || sf.FuncName.L == ast.GT) { + continue + } + if lCol, ok := sf.GetArgs()[0].(*expression.Column); ok && lCol.Equal(nil, nextCol) { + affectedCols := expression.ExtractColumns(sf.GetArgs()[1]) + if len(affectedCols) == 0 { + continue + } + for _, col := range affectedCols { + if innerPlan.schema.Contains(col) { + continue loopCandidates + } + } + nextColCmpFilterManager.appendNewExpr(sf.FuncName.L, sf.GetArgs()[1], affectedCols) + } else if rCol, ok := sf.GetArgs()[1].(*expression.Column); ok && rCol.Equal(nil, nextCol) { + affectedCols := expression.ExtractColumns(sf.GetArgs()[0]) + if len(affectedCols) == 0 { + continue + } + for _, col := range affectedCols { + if innerPlan.schema.Contains(col) { + continue loopCandidates + } + } + nextColCmpFilterManager.appendNewExpr(symmetricOp[sf.FuncName.L], sf.GetArgs()[0], affectedCols) + } + } + if len(nextColCmpFilterManager.OpType) == 0 { + colAccesses, colRemained := ranger.DetachCondsForTableRange(p.ctx, rangeFilterCandidates, nextCol) + remained = append(remained, colRemained...) + if colLengths[nextColPos] != types.UnspecifiedLength { + remained = append(remained, colAccesses...) + } + nextColRange, err := ranger.BuildColumnRange(colAccesses, p.ctx.GetSessionVars().StmtCtx, nextCol.RetType) + ranges, err := p.buildTemplateRange(idxOff2keyOff, matchedKeyCnt, notKeyEqAndIn, nextColRange, false) + if err != nil { + return nil, nil, nil, nil, err + } + return ranges, idxOff2keyOff, remained, nil, nil + } + ranges, err := p.buildTemplateRange(idxOff2keyOff, matchedKeyCnt, notKeyEqAndIn, nil, true) + if err != nil { + return nil, nil, nil, nil, err } - return conds, eqConds, remained, keyOff2IdxOff + return ranges, idxOff2keyOff, remained, nextColCmpFilterManager, nil +} + +func (p *LogicalJoin) buildTemplateRange(idxOff2KeyOff []int, matchedKeyCnt int, eqAndInFuncs []expression.Expression, nextColRange []*ranger.Range, haveExtraCol bool) (ranges []*ranger.Range, err error) { + pointLength := matchedKeyCnt + len(eqAndInFuncs) + if nextColRange != nil { + for _, colRan := range nextColRange { + // The range's exclude status is the same with last col's. + ran := &ranger.Range{ + LowVal: make([]types.Datum, pointLength, pointLength+1), + HighVal: make([]types.Datum, pointLength, pointLength+1), + LowExclude: colRan.LowExclude, + HighExclude: colRan.HighExclude, + } + ran.LowVal = append(ran.LowVal, colRan.LowVal[0]) + ran.HighVal = append(ran.HighVal, colRan.HighVal[0]) + ranges = append(ranges, ran) + } + } else if haveExtraCol { + ranges = append(ranges, &ranger.Range{ + LowVal: make([]types.Datum, pointLength+1, pointLength+1), + HighVal: make([]types.Datum, pointLength+1, pointLength+1), + }) + } else { + ranges = append(ranges, &ranger.Range{ + LowVal: make([]types.Datum, pointLength, pointLength), + HighVal: make([]types.Datum, pointLength, pointLength), + }) + } + emptyRow := chunk.Row{} + for i, j := 0, 0; j < len(eqAndInFuncs); i++ { + // This position is occupied by join key. + if idxOff2KeyOff[i] != -1 { + continue + } + sf := eqAndInFuncs[j].(*expression.ScalarFunction) + // Deal with the first two args. + if _, ok := sf.GetArgs()[0].(*expression.Column); ok { + for _, ran := range ranges { + ran.LowVal[i], err = sf.GetArgs()[1].Eval(emptyRow) + if err != nil { + return nil, err + } + ran.HighVal[i] = ran.LowVal[i] + } + } else { + for _, ran := range ranges { + ran.LowVal[i], err = sf.GetArgs()[0].Eval(emptyRow) + if err != nil { + return nil, err + } + ran.HighVal[i] = ran.LowVal[i] + } + } + // If the length of in function's constant list is more than one, we will expand ranges. + curRangeLen := len(ranges) + for argIdx := 2; argIdx < len(sf.GetArgs()); argIdx++ { + newRanges := make([]*ranger.Range, 0, curRangeLen) + for oldRangeIdx := 0; oldRangeIdx < curRangeLen; oldRangeIdx++ { + newRange := ranges[oldRangeIdx].Clone() + newRange.LowVal[i], err = sf.GetArgs()[argIdx].Eval(emptyRow) + if err != nil { + return nil, err + } + newRanges = append(newRanges, newRange) + } + ranges = append(ranges, newRanges...) + } + j++ + } + return ranges, nil } // tryToGetIndexJoin will get index join by hints. If we can generate a valid index join by hint, the second return value diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 4c66330fabdfe..a87c7153facae 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -232,7 +232,8 @@ type PhysicalIndexJoin struct { // Ranges stores the IndexRanges when the inner plan is index scan. Ranges []*ranger.Range // KeyOff2IdxOff maps the offsets in join key to the offsets in the index. - KeyOff2IdxOff []int + KeyOff2IdxOff []int + compareFilters *ColWithCompareOps } // PhysicalMergeJoin represents merge join for inner/ outer join. diff --git a/planner/core/resolve_indices.go b/planner/core/resolve_indices.go index 27562bc21838b..92e4558940abc 100644 --- a/planner/core/resolve_indices.go +++ b/planner/core/resolve_indices.go @@ -125,6 +125,12 @@ func (p *PhysicalIndexJoin) ResolveIndices() { for i, expr := range p.OtherConditions { p.OtherConditions[i] = expr.ResolveIndices(expression.MergeSchema(lSchema, rSchema)) } + if p.compareFilters != nil { + p.compareFilters.resolveIndices(p.children[p.OuterIndex].Schema()) + for i := range p.compareFilters.affectedColSchema.Columns { + p.compareFilters.affectedColSchema.Columns[i] = p.compareFilters.affectedColSchema.Columns[i].ResolveIndices(p.children[p.OuterIndex].Schema()).(*expression.Column) + } + } } // ResolveIndices implements Plan interface. diff --git a/util/ranger/detacher.go b/util/ranger/detacher.go index f020a24b0e446..dbc0557b1e42a 100644 --- a/util/ranger/detacher.go +++ b/util/ranger/detacher.go @@ -145,7 +145,7 @@ func detachCNFCondAndBuildRangeForIndex(sctx sessionctx.Context, conditions []ex err error ) - accessConds, filterConds, newConditions, emptyRange := extractEqAndInCondition(sctx, conditions, cols, lengths) + accessConds, filterConds, newConditions, emptyRange := ExtractEqAndInCondition(sctx, conditions, cols, lengths) if emptyRange { return ranges, nil, nil, 0, nil } @@ -155,8 +155,6 @@ func detachCNFCondAndBuildRangeForIndex(sctx sessionctx.Context, conditions []ex break } } - // We should remove all accessConds, so that they will not be added to filter conditions. - newConditions = removeAccessConditions(newConditions, accessConds) eqOrInCount := len(accessConds) if eqOrInCount == len(cols) { // If curIndex equals to len of index columns, it means the rest conditions haven't been appended to filter conditions. @@ -189,7 +187,13 @@ func detachCNFCondAndBuildRangeForIndex(sctx sessionctx.Context, conditions []ex return ranges, accessConds, filterConds, eqCount, errors.Trace(err) } -func extractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Expression, +// ExtractEqAndInCondition will split the given condition into three parts by the information of index columns and their lengths. +// accesses: The condition will be used to build range. +// filters: filters is the part that some access conditions need to be evaluate again since it's only the prefix part of char column. +// newConditions: We'll simplify the given conditions if there're multiple in conditions or eq conditions on the same column. +// e.g. if there're a in (1, 2, 3) and a in (2, 3, 4). This two will be combined to a in (2, 3) and pushed to newConditions. +// bool: indicate whether there's nil range when merging eq and in conditions. +func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Expression, cols []*expression.Column, lengths []int) ([]expression.Expression, []expression.Expression, []expression.Expression, bool) { var filters []expression.Expression rb := builder{sc: sctx.GetSessionVars().StmtCtx} @@ -238,6 +242,8 @@ func extractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Ex filters = append(filters, cond) } } + // We should remove all accessConds, so that they will not be added to filter conditions. + newConditions = removeAccessConditions(newConditions, accesses) return accesses, filters, newConditions, false } diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index 0402209e25bb7..a797a5b0db182 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -474,7 +474,7 @@ func newFieldType(tp *types.FieldType) *types.FieldType { // 1. 'expr' must be either 'EQUAL' or 'IN' function. // 2. 'points' should not be empty. func points2EqOrInCond(ctx sessionctx.Context, points []point, expr expression.Expression) expression.Expression { - // len(points) cannot be 0 here, since we impose early termination in extractEqAndInCondition + // len(points) cannot be 0 here, since we impose early termination in ExtractEqAndInCondition sf, _ := expr.(*expression.ScalarFunction) // Constant and Column args should have same RetType, simply get from first arg retType := sf.GetArgs()[0].GetType() From 029c6fd80731014ae8c1ca7925cbdcd9b45c9afe Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 27 Nov 2018 19:01:51 +0800 Subject: [PATCH 02/27] tiny fix --- executor/builder.go | 11 ++++++----- executor/index_lookup_join.go | 15 ++++++++------- planner/core/exhaust_physical_plans.go | 7 +++++-- planner/core/physical_plans.go | 5 +++-- planner/core/resolve_indices.go | 8 ++++---- 5 files changed, 26 insertions(+), 20 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 9224dfdd1dd03..176de1d254497 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1623,10 +1623,11 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plannercore.PhysicalIndexJoin) readerBuilder: &dataReaderBuilder{innerPlan, b}, rowTypes: innerTypes, }, - workerWg: new(sync.WaitGroup), - joiner: newJoiner(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), - indexRanges: v.Ranges, - keyOff2IdxOff: v.KeyOff2IdxOff, + workerWg: new(sync.WaitGroup), + joiner: newJoiner(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), + indexRanges: v.Ranges, + keyOff2IdxOff: v.KeyOff2IdxOff, + nextColCompareFilters: v.CompareFilters, } outerKeyCols := make([]int, len(v.OuterJoinKeys)) for i := 0; i < len(v.OuterJoinKeys); i++ { @@ -1966,7 +1967,7 @@ func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, l for _, nextColRan := range nextColRanges { for _, ran := range ranges { ran.LowVal[lastPos] = nextColRan.LowVal[0] - ran.LowVal[lastPos] = nextColRan.HighVal[0] + ran.HighVal[lastPos] = nextColRan.HighVal[0] ran.LowExclude = nextColRan.LowExclude ran.HighExclude = nextColRan.HighExclude } diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 7ff3a926ad1ed..257aceeaa40d8 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -200,13 +200,14 @@ func (e *IndexLookUpJoin) newInnerWorker(taskCh chan *lookUpJoinTask) *innerWork copiedRanges = append(copiedRanges, ran.Clone()) } iw := &innerWorker{ - innerCtx: e.innerCtx, - outerCtx: e.outerCtx, - taskCh: taskCh, - ctx: e.ctx, - executorChk: chunk.NewChunkWithCapacity(e.innerCtx.rowTypes, e.maxChunkSize), - indexRanges: copiedRanges, - keyOff2IdxOff: e.keyOff2IdxOff, + innerCtx: e.innerCtx, + outerCtx: e.outerCtx, + taskCh: taskCh, + ctx: e.ctx, + executorChk: chunk.NewChunkWithCapacity(e.innerCtx.rowTypes, e.maxChunkSize), + indexRanges: copiedRanges, + keyOff2IdxOff: e.keyOff2IdxOff, + nextColCompareFilters: e.nextColCompareFilters, } return iw } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index b0bd9861d5eb1..7005b075162c4 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -346,7 +346,7 @@ func (p *LogicalJoin) constructIndexJoin(prop *property.PhysicalProperty, innerJ innerPlan: innerPlan, KeyOff2IdxOff: newKeyOff, Ranges: ranges, - compareFilters: compareFilters, + CompareFilters: compareFilters, }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), chReqProps...) join.SetSchema(p.schema) return []PhysicalPlan{join} @@ -606,6 +606,7 @@ func (cwc *ColWithCompareOps) resolveIndices(schema *expression.Schema) { func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan *DataSource, innerJoinKeys []*expression.Column) ([]*ranger.Range, []int, []expression.Expression, *ColWithCompareOps, error) { idxCols, colLengths := expression.IndexInfo2Cols(innerPlan.schema.Columns, indexInfo) + log.Warnf("index cols: %v", idxCols) if len(idxCols) == 0 { return nil, nil, nil, nil, nil } @@ -661,12 +662,13 @@ func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan return ranges, idxOff2keyOff, remained, nil, nil } nextCol := idxCols[nextColPos] + log.Warnf("next col: %v", nextCol) nextColCmpFilterManager := &ColWithCompareOps{ targetCol: nextCol, affectedColSchema: expression.NewSchema(), } loopCandidates: - for _, filter := range rangeFilterCandidates { + for _, filter := range p.OtherConditions { sf, ok := filter.(*expression.ScalarFunction) if !ok || !(sf.FuncName.L == ast.LE || sf.FuncName.L == ast.LT || sf.FuncName.L == ast.GE || sf.FuncName.L == ast.GT) { continue @@ -695,6 +697,7 @@ loopCandidates: nextColCmpFilterManager.appendNewExpr(symmetricOp[sf.FuncName.L], sf.GetArgs()[0], affectedCols) } } + log.Warnf("next col filters: %v", nextColCmpFilterManager.opArg) if len(nextColCmpFilterManager.OpType) == 0 { colAccesses, colRemained := ranger.DetachCondsForTableRange(p.ctx, rangeFilterCandidates, nextCol) remained = append(remained, colRemained...) diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index a87c7153facae..8a17a7869efaf 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -232,8 +232,9 @@ type PhysicalIndexJoin struct { // Ranges stores the IndexRanges when the inner plan is index scan. Ranges []*ranger.Range // KeyOff2IdxOff maps the offsets in join key to the offsets in the index. - KeyOff2IdxOff []int - compareFilters *ColWithCompareOps + KeyOff2IdxOff []int + // CompareFilters ... (will finish in later commit) + CompareFilters *ColWithCompareOps } // PhysicalMergeJoin represents merge join for inner/ outer join. diff --git a/planner/core/resolve_indices.go b/planner/core/resolve_indices.go index 92e4558940abc..9cae3abf5971d 100644 --- a/planner/core/resolve_indices.go +++ b/planner/core/resolve_indices.go @@ -125,10 +125,10 @@ func (p *PhysicalIndexJoin) ResolveIndices() { for i, expr := range p.OtherConditions { p.OtherConditions[i] = expr.ResolveIndices(expression.MergeSchema(lSchema, rSchema)) } - if p.compareFilters != nil { - p.compareFilters.resolveIndices(p.children[p.OuterIndex].Schema()) - for i := range p.compareFilters.affectedColSchema.Columns { - p.compareFilters.affectedColSchema.Columns[i] = p.compareFilters.affectedColSchema.Columns[i].ResolveIndices(p.children[p.OuterIndex].Schema()).(*expression.Column) + if p.CompareFilters != nil { + p.CompareFilters.resolveIndices(p.children[p.OuterIndex].Schema()) + for i := range p.CompareFilters.affectedColSchema.Columns { + p.CompareFilters.affectedColSchema.Columns[i] = p.CompareFilters.affectedColSchema.Columns[i].ResolveIndices(p.children[p.OuterIndex].Schema()).(*expression.Column) } } } From 92570d6d49faa327caf4acb7d9e8e9d73c48e8bd Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 27 Nov 2018 21:03:06 +0800 Subject: [PATCH 03/27] fix sqllogic-test --- go.mod | 1 + planner/core/exhaust_physical_plans.go | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/go.mod b/go.mod index 064a19cafa4fe..18f6b377804ae 100644 --- a/go.mod +++ b/go.mod @@ -14,6 +14,7 @@ require ( github.com/golang/protobuf v1.2.0 github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db // indirect github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c + github.com/gorilla/context v1.1.1 // indirect github.com/gorilla/mux v1.6.2 github.com/grpc-ecosystem/go-grpc-middleware v1.0.0 github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 7005b075162c4..327ed2e253f3c 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -631,8 +631,8 @@ func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan } keyMatchedLen := len(idxCols) - 1 for ; keyMatchedLen > 0; keyMatchedLen-- { - if idxOff2keyOff[keyMatchedLen] == -1 { - continue + if idxOff2keyOff[keyMatchedLen] != -1 { + break } } remained := make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) From 3f4c9b9173af75871c2d8d16e453ffc204d987e3 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 28 Nov 2018 11:26:52 +0800 Subject: [PATCH 04/27] fix sqllogic test again --- planner/core/exhaust_physical_plans.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 327ed2e253f3c..1fe61f8f5d406 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -606,7 +606,6 @@ func (cwc *ColWithCompareOps) resolveIndices(schema *expression.Schema) { func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan *DataSource, innerJoinKeys []*expression.Column) ([]*ranger.Range, []int, []expression.Expression, *ColWithCompareOps, error) { idxCols, colLengths := expression.IndexInfo2Cols(innerPlan.schema.Columns, indexInfo) - log.Warnf("index cols: %v", idxCols) if len(idxCols) == 0 { return nil, nil, nil, nil, nil } @@ -629,9 +628,9 @@ func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan if matchedKeyCnt <= 0 { return nil, nil, nil, nil, nil } - keyMatchedLen := len(idxCols) - 1 + keyMatchedLen := len(idxCols) for ; keyMatchedLen > 0; keyMatchedLen-- { - if idxOff2keyOff[keyMatchedLen] != -1 { + if idxOff2keyOff[keyMatchedLen-1] != -1 { break } } @@ -655,6 +654,7 @@ func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan nextColPos := matchedKeyCnt + len(notKeyEqAndIn) // If all cols have been considered, we can return the current result. if nextColPos == len(idxCols) { + remained = append(remained, rangeFilterCandidates...) ranges, err := p.buildTemplateRange(idxOff2keyOff, matchedKeyCnt, notKeyEqAndIn, nil, false) if err != nil { return nil, nil, nil, nil, err @@ -662,7 +662,6 @@ func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan return ranges, idxOff2keyOff, remained, nil, nil } nextCol := idxCols[nextColPos] - log.Warnf("next col: %v", nextCol) nextColCmpFilterManager := &ColWithCompareOps{ targetCol: nextCol, affectedColSchema: expression.NewSchema(), @@ -697,7 +696,6 @@ loopCandidates: nextColCmpFilterManager.appendNewExpr(symmetricOp[sf.FuncName.L], sf.GetArgs()[0], affectedCols) } } - log.Warnf("next col filters: %v", nextColCmpFilterManager.opArg) if len(nextColCmpFilterManager.OpType) == 0 { colAccesses, colRemained := ranger.DetachCondsForTableRange(p.ctx, rangeFilterCandidates, nextCol) remained = append(remained, colRemained...) @@ -711,6 +709,7 @@ loopCandidates: } return ranges, idxOff2keyOff, remained, nil, nil } + remained = append(remained, rangeFilterCandidates...) ranges, err := p.buildTemplateRange(idxOff2keyOff, matchedKeyCnt, notKeyEqAndIn, nil, true) if err != nil { return nil, nil, nil, nil, err @@ -780,6 +779,7 @@ func (p *LogicalJoin) buildTemplateRange(idxOff2KeyOff []int, matchedKeyCnt int, if err != nil { return nil, err } + newRange.HighVal[i] = newRange.LowVal[i] newRanges = append(newRanges, newRange) } ranges = append(ranges, newRanges...) From dbe54c9c8ea48298d6305fb805178d0ce158a5f4 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 28 Nov 2018 16:22:15 +0800 Subject: [PATCH 05/27] add comment --- planner/core/exhaust_physical_plans.go | 21 ++++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 1fe61f8f5d406..ed9892e2b1848 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -545,6 +545,8 @@ var symmetricOp = map[string]string{ ast.LE: ast.GE, } +// ColWithCompareOps is used in index join to handle the column with compare functions(>=, >, <, <=). +// It stores the compare functions and build ranges in execution phase. type ColWithCompareOps struct { targetCol *expression.Column OpType []string @@ -567,6 +569,7 @@ func (cwc *ColWithCompareOps) appendNewExpr(opName string, arg expression.Expres } } +// CompareRow sorts the row for deduplicate. func (cwc *ColWithCompareOps) CompareRow(lhs, rhs chunk.Row) int { for i, col := range cwc.affectedColSchema.Columns { ret := cwc.compareFuncs[i](lhs, col.Index, rhs, col.Index) @@ -577,6 +580,7 @@ func (cwc *ColWithCompareOps) CompareRow(lhs, rhs chunk.Row) int { return 0 } +// BuildRangesByRow will build range of the given row. It will eval each function's arg then call BuildRange. func (cwc *ColWithCompareOps) BuildRangesByRow(ctx sessionctx.Context, row chunk.Row) ([]*ranger.Range, error) { exprs := make([]expression.Expression, len(cwc.OpType)) for i, opType := range cwc.OpType { @@ -615,6 +619,7 @@ func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan notKeyIdxCols := make([]*expression.Column, 0, len(idxCols)) notKeyIdxColsLen := make([]int, 0, len(idxCols)) matchedKeyCnt := 0 + // This loop finds out what index column appears in join key, and what is not. for i, idxCol := range idxCols { idxOff2keyOff[i] = tmpSchema.ColumnIndex(idxCol) if idxOff2keyOff[i] >= 0 { @@ -625,6 +630,11 @@ func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan notKeyIdxCols = append(notKeyIdxCols, idxCol) notKeyIdxColsLen = append(notKeyIdxColsLen, colLengths[i]) } + // If no index column appears in join key, we just break. + // TODO: It may meet this case: There's no join key condition, but have compare filters. + // e.g. select * from t1, t2 on t1.a=t2.a and t2.b > t1.b-10 and t2.b < t1.b where t1.a=1 and t2.a=1. + // After constant propagation. The t1.a=t2.a is removed. And if we have index (t2.a, t2.b). It can apply index join + // to speed up. if matchedKeyCnt <= 0 { return nil, nil, nil, nil, nil } @@ -636,6 +646,7 @@ func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan } remained := make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) rangeFilterCandidates := make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) + // This loop deal first filter out the expressions that contains columns not in index. for _, innerFilter := range innerPlan.pushedDownConds { affectedCols := expression.ExtractColumns(innerFilter) if expression.ColumnSliceIsIntersect(affectedCols, possibleUsedKeys) { @@ -644,9 +655,11 @@ func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan } rangeFilterCandidates = append(rangeFilterCandidates, innerFilter) } + // Extract the eq/in functions of possible join key. This returned list keeps the same order with index column. notKeyEqAndIn, remainedEqAndIn, rangeFilterCandidates, _ := ranger.ExtractEqAndInCondition(p.ctx, rangeFilterCandidates, notKeyIdxCols, notKeyIdxColsLen) // We hope that the index cols appeared in the join keys can all be used to build range. If it cannot be satisfied, // we'll mark this index as cannot be used for index join. + // So we should make sure that all columns before the keyMatchedLen is join key or has eq/in function. if len(notKeyEqAndIn) < keyMatchedLen-matchedKeyCnt { return nil, nil, nil, nil, nil } @@ -666,7 +679,8 @@ func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan targetCol: nextCol, affectedColSchema: expression.NewSchema(), } -loopCandidates: + // We first loop the other conds to see whether there's conditions can be used to build range. +loopOtherConds: for _, filter := range p.OtherConditions { sf, ok := filter.(*expression.ScalarFunction) if !ok || !(sf.FuncName.L == ast.LE || sf.FuncName.L == ast.LT || sf.FuncName.L == ast.GE || sf.FuncName.L == ast.GT) { @@ -679,7 +693,7 @@ loopCandidates: } for _, col := range affectedCols { if innerPlan.schema.Contains(col) { - continue loopCandidates + continue loopOtherConds } } nextColCmpFilterManager.appendNewExpr(sf.FuncName.L, sf.GetArgs()[1], affectedCols) @@ -690,7 +704,7 @@ loopCandidates: } for _, col := range affectedCols { if innerPlan.schema.Contains(col) { - continue loopCandidates + continue loopOtherConds } } nextColCmpFilterManager.appendNewExpr(symmetricOp[sf.FuncName.L], sf.GetArgs()[0], affectedCols) @@ -734,6 +748,7 @@ func (p *LogicalJoin) buildTemplateRange(idxOff2KeyOff []int, matchedKeyCnt int, ranges = append(ranges, ran) } } else if haveExtraCol { + // Reserve a position for the last col. ranges = append(ranges, &ranger.Range{ LowVal: make([]types.Datum, pointLength+1, pointLength+1), HighVal: make([]types.Datum, pointLength+1, pointLength+1), From 519f562958c0fe5afa8f620096c31c2d91fbcdd2 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 10 Dec 2018 17:35:52 +0800 Subject: [PATCH 06/27] fix explain result --- planner/core/exhaust_physical_plans.go | 69 ++++++++++++++++++++------ planner/core/physical_plans.go | 2 +- 2 files changed, 56 insertions(+), 15 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index ed9892e2b1848..5dd6bf601dd04 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -14,6 +14,8 @@ package core import ( + "bytes" + "fmt" "math" "github.com/pingcap/parser/ast" @@ -406,6 +408,7 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou bestIndexInfo *model.IndexInfo rangesOfBest []*ranger.Range maxUsedCols int + accessesOfBest []expression.Expression remainedOfBest []expression.Expression idxOff2KeyOff []int comparesOfBest *ColWithCompareOps @@ -415,7 +418,7 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou continue } indexInfo := path.index - ranges, tmpIdxOff2KeyOff, remained, compareFilters, err := p.analyzeLookUpFilters(indexInfo, ds, innerJoinKeys) + ranges, tmpIdxOff2KeyOff, accesses, remained, compareFilters, err := p.analyzeLookUpFilters(indexInfo, ds, innerJoinKeys) if err != nil { log.Warnf("[planner]: error happened when build index join: %v", err) continue @@ -427,6 +430,7 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou bestIndexInfo = indexInfo maxUsedCols = len(ranges[0].LowVal) rangesOfBest = ranges + accessesOfBest = accesses remainedOfBest = remained idxOff2KeyOff = tmpIdxOff2KeyOff comparesOfBest = compareFilters @@ -442,12 +446,31 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou keyOff2IdxOff[keyOff] = idxOff } } - innerPlan := p.constructInnerIndexScan(ds, bestIndexInfo, remainedOfBest, outerJoinKeys, us) + idxCols, _ := expression.IndexInfo2Cols(ds.schema.Columns, bestIndexInfo) + rangeInfo := p.buildRangeDecidedByInformation(idxCols, idxOff2KeyOff, outerJoinKeys, accessesOfBest) + innerPlan := p.constructInnerIndexScan(ds, bestIndexInfo, remainedOfBest, outerJoinKeys, us, rangeInfo) return p.constructIndexJoin(prop, innerJoinKeys, outerJoinKeys, outerIdx, innerPlan, rangesOfBest, keyOff2IdxOff, comparesOfBest) } return nil } +func (p *LogicalJoin) buildRangeDecidedByInformation(idxCols []*expression.Column, idxOff2KeyOff []int, + outerJoinKeys []*expression.Column, accesses []expression.Expression) string { + buffer := bytes.NewBufferString("") + buffer.WriteString("range decided by:[") + for idxOff, keyOff := range idxOff2KeyOff { + if keyOff == -1 { + continue + } + buffer.WriteString(fmt.Sprintf(" eq(%v, %v)", idxCols[idxOff], outerJoinKeys[keyOff])) + } + for _, access := range accesses { + buffer.WriteString(fmt.Sprintf(" %v", access)) + } + buffer.WriteString(" ]") + return buffer.String() +} + // constructInnerTableScan is specially used to construct the inner plan for PhysicalIndexJoin. func (p *LogicalJoin) constructInnerTableScan(ds *DataSource, pk *expression.Column, outerJoinKeys []*expression.Column, us *LogicalUnionScan) PhysicalPlan { ranges := ranger.FullIntRange(mysql.HasUnsignedFlag(pk.RetType.Flag)) @@ -495,7 +518,8 @@ func (p *LogicalJoin) constructInnerUnionScan(us *LogicalUnionScan, reader Physi } // constructInnerIndexScan is specially used to construct the inner plan for PhysicalIndexJoin. -func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexInfo, remainedConds []expression.Expression, outerJoinKeys []*expression.Column, us *LogicalUnionScan) PhysicalPlan { +func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexInfo, remainedConds []expression.Expression, + outerJoinKeys []*expression.Column, us *LogicalUnionScan, rangeInfo string) PhysicalPlan { is := PhysicalIndexScan{ Table: ds.tableInfo, TableAsName: ds.TableAsName, @@ -505,7 +529,7 @@ func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexIn dataSourceSchema: ds.schema, KeepOrder: false, Ranges: ranger.FullRange(), - rangeDecidedBy: outerJoinKeys, + rangeDecidedBy: rangeInfo, }.Init(ds.ctx) is.filterCondition = remainedConds @@ -608,11 +632,24 @@ func (cwc *ColWithCompareOps) resolveIndices(schema *expression.Schema) { } } -func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan *DataSource, innerJoinKeys []*expression.Column) ([]*ranger.Range, []int, []expression.Expression, *ColWithCompareOps, error) { +func (cwc *ColWithCompareOps) String() { + buffer := bytes.NewBufferString("") + log.Warnf("%v, %v, %v", cwc.targetCol, cwc.OpType, cwc.opArg) + for i := range cwc.OpType { + buffer.WriteString(fmt.Sprintf("%v(%v, %v)", cwc.OpType[i], cwc.targetCol, cwc.opArg[i])) + if i < len(cwc.OpType)-1 { + buffer.WriteString(" ") + } + } +} + +func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan *DataSource, + innerJoinKeys []*expression.Column) ([]*ranger.Range, []int, []expression.Expression, []expression.Expression, *ColWithCompareOps, error) { idxCols, colLengths := expression.IndexInfo2Cols(innerPlan.schema.Columns, indexInfo) if len(idxCols) == 0 { - return nil, nil, nil, nil, nil + return nil, nil, nil, nil, nil, nil } + accesses := make([]expression.Expression, 0, len(idxCols)) tmpSchema := expression.NewSchema(innerJoinKeys...) idxOff2keyOff := make([]int, len(idxCols)) possibleUsedKeys := make([]*expression.Column, 0, len(idxCols)) @@ -636,7 +673,7 @@ func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan // After constant propagation. The t1.a=t2.a is removed. And if we have index (t2.a, t2.b). It can apply index join // to speed up. if matchedKeyCnt <= 0 { - return nil, nil, nil, nil, nil + return nil, nil, nil, nil, nil, nil } keyMatchedLen := len(idxCols) for ; keyMatchedLen > 0; keyMatchedLen-- { @@ -661,8 +698,9 @@ func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan // we'll mark this index as cannot be used for index join. // So we should make sure that all columns before the keyMatchedLen is join key or has eq/in function. if len(notKeyEqAndIn) < keyMatchedLen-matchedKeyCnt { - return nil, nil, nil, nil, nil + return nil, nil, nil, nil, nil, nil } + accesses = append(accesses, notKeyEqAndIn...) remained = append(remained, remainedEqAndIn...) nextColPos := matchedKeyCnt + len(notKeyEqAndIn) // If all cols have been considered, we can return the current result. @@ -670,9 +708,9 @@ func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan remained = append(remained, rangeFilterCandidates...) ranges, err := p.buildTemplateRange(idxOff2keyOff, matchedKeyCnt, notKeyEqAndIn, nil, false) if err != nil { - return nil, nil, nil, nil, err + return nil, nil, nil, nil, nil, err } - return ranges, idxOff2keyOff, remained, nil, nil + return ranges, idxOff2keyOff, accesses, remained, nil, nil } nextCol := idxCols[nextColPos] nextColCmpFilterManager := &ColWithCompareOps{ @@ -696,6 +734,7 @@ loopOtherConds: continue loopOtherConds } } + accesses = append(accesses, sf) nextColCmpFilterManager.appendNewExpr(sf.FuncName.L, sf.GetArgs()[1], affectedCols) } else if rCol, ok := sf.GetArgs()[1].(*expression.Column); ok && rCol.Equal(nil, nextCol) { affectedCols := expression.ExtractColumns(sf.GetArgs()[0]) @@ -707,6 +746,7 @@ loopOtherConds: continue loopOtherConds } } + accesses = append(accesses, sf) nextColCmpFilterManager.appendNewExpr(symmetricOp[sf.FuncName.L], sf.GetArgs()[0], affectedCols) } } @@ -716,20 +756,21 @@ loopOtherConds: if colLengths[nextColPos] != types.UnspecifiedLength { remained = append(remained, colAccesses...) } + accesses = append(accesses, colAccesses...) nextColRange, err := ranger.BuildColumnRange(colAccesses, p.ctx.GetSessionVars().StmtCtx, nextCol.RetType) ranges, err := p.buildTemplateRange(idxOff2keyOff, matchedKeyCnt, notKeyEqAndIn, nextColRange, false) if err != nil { - return nil, nil, nil, nil, err + return nil, nil, nil, nil, nil, err } - return ranges, idxOff2keyOff, remained, nil, nil + return ranges, idxOff2keyOff, accesses, remained, nil, nil } remained = append(remained, rangeFilterCandidates...) ranges, err := p.buildTemplateRange(idxOff2keyOff, matchedKeyCnt, notKeyEqAndIn, nil, true) if err != nil { - return nil, nil, nil, nil, err + return nil, nil, nil, nil, nil, err } - return ranges, idxOff2keyOff, remained, nextColCmpFilterManager, nil + return ranges, idxOff2keyOff, accesses, remained, nextColCmpFilterManager, nil } func (p *LogicalJoin) buildTemplateRange(idxOff2KeyOff []int, matchedKeyCnt int, eqAndInFuncs []expression.Expression, nextColRange []*ranger.Range, haveExtraCol bool) (ranges []*ranger.Range, err error) { diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 8a17a7869efaf..680678fe16c96 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -113,7 +113,7 @@ type PhysicalIndexScan struct { // It is used for query feedback. Hist *statistics.Histogram - rangeDecidedBy []*expression.Column + rangeDecidedBy string // The index scan may be on a partition. isPartition bool From 3a0b74c28f7c5987a4d420585552ed8c7fb45526 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 13 Dec 2018 15:58:35 +0800 Subject: [PATCH 07/27] fix string method --- planner/core/exhaust_physical_plans.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 5dd6bf601dd04..665858cb829ce 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -632,7 +632,7 @@ func (cwc *ColWithCompareOps) resolveIndices(schema *expression.Schema) { } } -func (cwc *ColWithCompareOps) String() { +func (cwc *ColWithCompareOps) String() string { buffer := bytes.NewBufferString("") log.Warnf("%v, %v, %v", cwc.targetCol, cwc.OpType, cwc.opArg) for i := range cwc.OpType { @@ -641,6 +641,7 @@ func (cwc *ColWithCompareOps) String() { buffer.WriteString(" ") } } + return buffer.String() } func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan *DataSource, From 7d5c4df46858b1e6625928509e887a946b99b630 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 13 Dec 2018 17:30:48 +0800 Subject: [PATCH 08/27] add unit test and fix things found in it --- planner/core/exhaust_physical_plans.go | 6 +- planner/core/exhaust_physical_plans_test.go | 239 ++++++++++++++++++++ planner/core/rule_partition_processor.go | 3 +- statistics/selectivity.go | 3 +- util/ranger/ranger.go | 10 +- util/ranger/ranger_test.go | 3 +- 6 files changed, 258 insertions(+), 6 deletions(-) create mode 100644 planner/core/exhaust_physical_plans_test.go diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index a86a7a549fe5c..a1bb8aa8b5e06 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -574,6 +574,7 @@ var symmetricOp = map[string]string{ // It stores the compare functions and build ranges in execution phase. type ColWithCompareOps struct { targetCol *expression.Column + colLength int OpType []string opArg []expression.Expression tmpConstant []*expression.Constant @@ -620,7 +621,7 @@ func (cwc *ColWithCompareOps) BuildRangesByRow(ctx sessionctx.Context, row chunk } exprs = append(exprs, newExpr) } - ranges, err := ranger.BuildColumnRange(exprs, ctx.GetSessionVars().StmtCtx, cwc.targetCol.RetType) + ranges, err := ranger.BuildColumnRange(exprs, ctx.GetSessionVars().StmtCtx, cwc.targetCol.RetType, cwc.colLength) if err != nil { return nil, err } @@ -717,6 +718,7 @@ func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan nextCol := idxCols[nextColPos] nextColCmpFilterManager := &ColWithCompareOps{ targetCol: nextCol, + colLength: colLengths[nextColPos], affectedColSchema: expression.NewSchema(), } // We first loop the other conds to see whether there's conditions can be used to build range. @@ -759,7 +761,7 @@ loopOtherConds: remained = append(remained, colAccesses...) } accesses = append(accesses, colAccesses...) - nextColRange, err := ranger.BuildColumnRange(colAccesses, p.ctx.GetSessionVars().StmtCtx, nextCol.RetType) + nextColRange, err := ranger.BuildColumnRange(colAccesses, p.ctx.GetSessionVars().StmtCtx, nextCol.RetType, colLengths[nextColPos]) ranges, err := p.buildTemplateRange(idxOff2keyOff, matchedKeyCnt, notKeyEqAndIn, nextColRange, false) if err != nil { return nil, nil, nil, nil, nil, err diff --git a/planner/core/exhaust_physical_plans_test.go b/planner/core/exhaust_physical_plans_test.go new file mode 100644 index 0000000000000..2fd66cdc937e7 --- /dev/null +++ b/planner/core/exhaust_physical_plans_test.go @@ -0,0 +1,239 @@ +// Copyright 2017 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package core + +import ( + "fmt" + + . "github.com/pingcap/check" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/types" + "github.com/sirupsen/logrus" +) + +func (s *testUnitTestSuit) rewriteSimpleExpr(str string, schema *expression.Schema) ([]expression.Expression, error) { + if str == "" { + return nil, nil + } + filters, err := expression.ParseSimpleExprsWithSchema(s.ctx, str, schema) + if err != nil { + return nil, err + } + if sf, ok := filters[0].(*expression.ScalarFunction); ok && sf.FuncName.L == ast.LogicAnd { + filters = expression.FlattenCNFConditions(sf) + } + return filters, nil +} + +func (s *testUnitTestSuit) TestIndexJoinAnalyzeLookUpFilters(c *C) { + s.ctx.GetSessionVars().PlanID = -1 + joinNode := LogicalJoin{}.Init(s.ctx) + dataSourceNode := DataSource{}.Init(s.ctx) + dsSchema := expression.NewSchema() + dsSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("a"), + TblName: model.NewCIStr("t"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + dsSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("b"), + TblName: model.NewCIStr("t"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + dsSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("c"), + TblName: model.NewCIStr("t"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeVarchar), + }) + dsSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("d"), + TblName: model.NewCIStr("t"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + dataSourceNode.schema = dsSchema + outerChildSchema := expression.NewSchema() + outerChildSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("e"), + TblName: model.NewCIStr("t1"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + outerChildSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("f"), + TblName: model.NewCIStr("t1"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + outerChildSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("g"), + TblName: model.NewCIStr("t1"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeVarchar), + }) + outerChildSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("h"), + TblName: model.NewCIStr("t1"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + joinNode.SetSchema(expression.MergeSchema(dsSchema, outerChildSchema)) + var idxCols []*model.IndexColumn + idxCols = append(idxCols, &model.IndexColumn{Name: model.NewCIStr("a"), Length: types.UnspecifiedLength}) + idxCols = append(idxCols, &model.IndexColumn{Name: model.NewCIStr("b"), Length: types.UnspecifiedLength}) + idxCols = append(idxCols, &model.IndexColumn{Name: model.NewCIStr("c"), Length: 2}) + idxCols = append(idxCols, &model.IndexColumn{Name: model.NewCIStr("d"), Length: types.UnspecifiedLength}) + idxInfo := &model.IndexInfo{Columns: idxCols} + + tests := []struct { + innerKeys []*expression.Column + pushedDownConds string + otherConds string + ranges string + idxOff2KeyOff string + accesses string + remained string + compareFilters string + }{ + // Join key not continuous and no pushed filter to match. + { + innerKeys: []*expression.Column{dsSchema.Columns[0], dsSchema.Columns[2]}, + pushedDownConds: "", + otherConds: "", + ranges: "[]", + idxOff2KeyOff: "[]", + accesses: "[]", + remained: "[]", + compareFilters: "", + }, + // Join key and pushed eq filter not continuous. + { + innerKeys: []*expression.Column{dsSchema.Columns[2]}, + pushedDownConds: "a = 1", + otherConds: "", + ranges: "[]", + idxOff2KeyOff: "[]", + accesses: "[]", + remained: "[]", + compareFilters: "", + }, + // Keys are continuous. + { + innerKeys: []*expression.Column{dsSchema.Columns[1]}, + pushedDownConds: "a = 1", + otherConds: "", + ranges: "[[1 NULL NULL,1 NULL +inf]]", + idxOff2KeyOff: "[-1 0 -1 -1]", + accesses: "[eq(test.t.a, 1)]", + remained: "[]", + compareFilters: "", + }, + // Keys are continuous and there're correlated filters. + { + innerKeys: []*expression.Column{dsSchema.Columns[1]}, + pushedDownConds: "a = 1", + otherConds: "c > g and c < concat(g, \"ab\")", + ranges: "[[1 NULL NULL,1 NULL NULL]]", + idxOff2KeyOff: "[-1 0 -1 -1]", + accesses: "[eq(test.t.a, 1) gt(test.t.c, test.t1.g) lt(test.t.c, concat(test.t1.g, ab))]", + remained: "[]", + compareFilters: "gt(test.t.c, test.t1.g) lt(test.t.c, concat(test.t1.g, ab))", + }, + // cast function won't be involved. + { + innerKeys: []*expression.Column{dsSchema.Columns[1]}, + pushedDownConds: "a = 1", + otherConds: "c > g and c < g + 10", + ranges: "[[1 NULL NULL,1 NULL NULL]]", + idxOff2KeyOff: "[-1 0 -1 -1]", + accesses: "[eq(test.t.a, 1) gt(test.t.c, test.t1.g)]", + remained: "[]", + compareFilters: "gt(test.t.c, test.t1.g)", + }, + // Can deal with prefix index correctly. + { + innerKeys: []*expression.Column{dsSchema.Columns[1]}, + pushedDownConds: "a = 1 and c > 'a' and c < 'aaaaaa'", + otherConds: "", + ranges: "[[1 NULL \"a\",1 NULL \"[97 97]\"]]", + idxOff2KeyOff: "[-1 0 -1 -1]", + accesses: "[eq(test.t.a, 1) gt(test.t.c, a) lt(test.t.c, aaaaaa)]", + remained: "[gt(test.t.c, a) lt(test.t.c, aaaaaa)]", + compareFilters: "", + }, + // Can generate correct ranges for in functions. + { + innerKeys: []*expression.Column{dsSchema.Columns[1]}, + pushedDownConds: "a in (1, 2, 3) and c in ('a', 'b', 'c')", + otherConds: "", + ranges: "[[1 NULL \"a\" NULL,1 NULL \"a\" +inf] [2 NULL \"a\" NULL,2 NULL \"a\" +inf] [3 NULL \"a\" NULL,3 NULL \"a\" +inf] [1 NULL \"b\" NULL,1 NULL \"b\" +inf] [2 NULL \"b\" NULL,2 NULL \"b\" +inf] [3 NULL \"b\" NULL,3 NULL \"b\" +inf] [1 NULL \"c\" NULL,1 NULL \"c\" +inf] [2 NULL \"c\" NULL,2 NULL \"c\" +inf] [3 NULL \"c\" NULL,3 NULL \"c\" +inf]]", + idxOff2KeyOff: "[-1 0 -1 -1]", + accesses: "[in(test.t.a, 1, 2, 3) in(test.t.c, a, b, c)]", + remained: "[in(test.t.c, a, b, c)]", + compareFilters: "", + }, + // Can generate correct ranges for in functions with correlated filters.. + { + innerKeys: []*expression.Column{dsSchema.Columns[1]}, + pushedDownConds: "a in (1, 2, 3) and c in ('a', 'b', 'c')", + otherConds: "d > h and d < h + 100", + ranges: "[[1 NULL \"a\" NULL,1 NULL \"a\" NULL] [2 NULL \"a\" NULL,2 NULL \"a\" NULL] [3 NULL \"a\" NULL,3 NULL \"a\" NULL] [1 NULL \"b\" NULL,1 NULL \"b\" NULL] [2 NULL \"b\" NULL,2 NULL \"b\" NULL] [3 NULL \"b\" NULL,3 NULL \"b\" NULL] [1 NULL \"c\" NULL,1 NULL \"c\" NULL] [2 NULL \"c\" NULL,2 NULL \"c\" NULL] [3 NULL \"c\" NULL,3 NULL \"c\" NULL]]", + idxOff2KeyOff: "[-1 0 -1 -1]", + accesses: "[in(test.t.a, 1, 2, 3) in(test.t.c, a, b, c) gt(test.t.d, test.t1.h) lt(test.t.d, plus(test.t1.h, 100))]", + remained: "[in(test.t.c, a, b, c)]", + compareFilters: "gt(test.t.d, test.t1.h) lt(test.t.d, plus(test.t1.h, 100))", + }, + // Join keys are not continuous and the pushed key connect the key but not eq/in functions. + { + innerKeys: []*expression.Column{dsSchema.Columns[0], dsSchema.Columns[2]}, + pushedDownConds: "b > 1", + otherConds: "", + ranges: "[]", + idxOff2KeyOff: "[]", + accesses: "[]", + remained: "[]", + compareFilters: "", + }, + } + for _, tt := range tests { + pushed, err := s.rewriteSimpleExpr(tt.pushedDownConds, dsSchema) + c.Assert(err, IsNil) + dataSourceNode.pushedDownConds = pushed + others, err := s.rewriteSimpleExpr(tt.otherConds, joinNode.schema) + c.Assert(err, IsNil) + joinNode.OtherConditions = others + logrus.Warnf("others: %v", joinNode.OtherConditions) + ranges, idxOff2KeyOff, accesses, remained, compareFilters, err := joinNode.analyzeLookUpFilters(idxInfo, dataSourceNode, tt.innerKeys) + c.Assert(err, IsNil) + c.Assert(fmt.Sprintf("%v", ranges), Equals, tt.ranges) + c.Assert(fmt.Sprintf("%v", idxOff2KeyOff), Equals, tt.idxOff2KeyOff) + c.Assert(fmt.Sprintf("%v", accesses), Equals, tt.accesses) + c.Assert(fmt.Sprintf("%v", remained), Equals, tt.remained) + c.Assert(fmt.Sprintf("%v", compareFilters), Equals, tt.compareFilters) + } +} diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index c97392446a2b5..bff8c5edaa9ee 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -17,6 +17,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table/tables" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/ranger" ) @@ -136,7 +137,7 @@ func (s *partitionProcessor) canBePrune(ctx sessionctx.Context, col *expression. // Calculate the column range to prune. accessConds := ranger.ExtractAccessConditionsForColumn(conds, col.UniqueID) - r, err := ranger.BuildColumnRange(accessConds, ctx.GetSessionVars().StmtCtx, col.RetType) + r, err := ranger.BuildColumnRange(accessConds, ctx.GetSessionVars().StmtCtx, col.RetType, types.UnspecifiedLength) if err != nil { return false, errors.Trace(err) } diff --git a/statistics/selectivity.go b/statistics/selectivity.go index c54ad550b9793..fbaeb9f5b3982 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/ranger" ) @@ -235,7 +236,7 @@ func getMaskAndRanges(ctx sessionctx.Context, exprs []expression.Expression, ran switch rangeType { case ranger.ColumnRangeType: accessConds = ranger.ExtractAccessConditionsForColumn(exprs, cols[0].UniqueID) - ranges, err = ranger.BuildColumnRange(accessConds, sc, cols[0].RetType) + ranges, err = ranger.BuildColumnRange(accessConds, sc, cols[0].RetType, types.UnspecifiedLength) case ranger.IndexRangeType: ranges, accessConds, err = ranger.DetachSimpleCondAndBuildRangeForIndex(ctx, exprs, cols, lengths) default: diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index 1087b7a1cb1af..f7e07cbc30bb0 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -261,7 +261,7 @@ func BuildTableRange(accessConditions []expression.Expression, sc *stmtctx.State } // BuildColumnRange builds the range for sampling histogram to calculate the row count. -func BuildColumnRange(conds []expression.Expression, sc *stmtctx.StatementContext, tp *types.FieldType) ([]*Range, error) { +func BuildColumnRange(conds []expression.Expression, sc *stmtctx.StatementContext, tp *types.FieldType, colLen int) ([]*Range, error) { if len(conds) == 0 { return []*Range{{LowVal: []types.Datum{{}}, HighVal: []types.Datum{types.MaxValueDatum()}}}, nil } @@ -279,6 +279,14 @@ func BuildColumnRange(conds []expression.Expression, sc *stmtctx.StatementContex if err != nil { return nil, errors.Trace(err) } + if colLen != types.UnspecifiedLength { + for _, ran := range ranges { + fixRangeDatum(&ran.LowVal[0], colLen, tp) + ran.LowExclude = false + fixRangeDatum(&ran.HighVal[0], colLen, tp) + ran.HighExclude = false + } + } return ranges, nil } diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 8d8ae3be3a825..0bc1a1a6b79ba 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/mocktikv" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" @@ -928,7 +929,7 @@ func (s *testRangerSuite) TestColumnRange(c *C) { c.Assert(col, NotNil) conds = ranger.ExtractAccessConditionsForColumn(conds, col.UniqueID) c.Assert(fmt.Sprintf("%s", conds), Equals, tt.accessConds, Commentf("wrong access conditions for expr: %s", tt.exprStr)) - result, err := ranger.BuildColumnRange(conds, new(stmtctx.StatementContext), col.RetType) + result, err := ranger.BuildColumnRange(conds, new(stmtctx.StatementContext), col.RetType, types.UnspecifiedLength) c.Assert(err, IsNil) got := fmt.Sprintf("%v", result) c.Assert(got, Equals, tt.resultStr, Commentf("different for expr %s, col: %v", tt.exprStr, col)) From 16196a18c981212ffe2be7fe128aedc959e70902 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 13 Dec 2018 18:47:40 +0800 Subject: [PATCH 09/27] more clear fix and remove unused changes --- executor/builder.go | 8 ++------ planner/core/exhaust_physical_plans.go | 14 +++++++++++--- planner/core/exhaust_physical_plans_test.go | 6 ++---- 3 files changed, 15 insertions(+), 13 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 79f4ad520b52c..753f2e987058f 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1086,12 +1086,8 @@ func (b *executorBuilder) buildHashAgg(v *plannercore.PhysicalHashAgg) Executor e.FinalAggFuncs = append(e.FinalAggFuncs, finalAggFunc) if partialAggDesc.Name == ast.AggFuncGroupConcat { // For group_concat, finalAggFunc and partialAggFunc need shared `truncate` flag to do duplicate. - finalAggFunc.(interface { - SetTruncated(t *int32) - }).SetTruncated( - partialAggFunc.(interface { - GetTruncated() *int32 - }).GetTruncated(), + finalAggFunc.(interface{ SetTruncated(t *int32) }).SetTruncated( + partialAggFunc.(interface{ GetTruncated() *int32 }).GetTruncated(), ) } } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index a1bb8aa8b5e06..4747a2436d191 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -636,7 +636,6 @@ func (cwc *ColWithCompareOps) resolveIndices(schema *expression.Schema) { func (cwc *ColWithCompareOps) String() string { buffer := bytes.NewBufferString("") - log.Warnf("%v, %v, %v", cwc.targetCol, cwc.OpType, cwc.opArg) for i := range cwc.OpType { buffer.WriteString(fmt.Sprintf("%v(%v, %v)", cwc.OpType[i], cwc.targetCol, cwc.opArg[i])) if i < len(cwc.OpType)-1 { @@ -761,8 +760,17 @@ loopOtherConds: remained = append(remained, colAccesses...) } accesses = append(accesses, colAccesses...) - nextColRange, err := ranger.BuildColumnRange(colAccesses, p.ctx.GetSessionVars().StmtCtx, nextCol.RetType, colLengths[nextColPos]) - ranges, err := p.buildTemplateRange(idxOff2keyOff, matchedKeyCnt, notKeyEqAndIn, nextColRange, false) + var ranges, nextColRange []*ranger.Range + var err error + if len(colAccesses) > 0 { + nextColRange, err = ranger.BuildColumnRange(colAccesses, p.ctx.GetSessionVars().StmtCtx, nextCol.RetType, colLengths[nextColPos]) + if err != nil { + return nil, nil, nil, nil, nil, err + } + ranges, err = p.buildTemplateRange(idxOff2keyOff, matchedKeyCnt, notKeyEqAndIn, nextColRange, false) + } else { + ranges, err = p.buildTemplateRange(idxOff2keyOff, matchedKeyCnt, notKeyEqAndIn, nil, false) + } if err != nil { return nil, nil, nil, nil, nil, err } diff --git a/planner/core/exhaust_physical_plans_test.go b/planner/core/exhaust_physical_plans_test.go index 2fd66cdc937e7..952900a006be5 100644 --- a/planner/core/exhaust_physical_plans_test.go +++ b/planner/core/exhaust_physical_plans_test.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/types" - "github.com/sirupsen/logrus" ) func (s *testUnitTestSuit) rewriteSimpleExpr(str string, schema *expression.Schema) ([]expression.Expression, error) { @@ -147,7 +146,7 @@ func (s *testUnitTestSuit) TestIndexJoinAnalyzeLookUpFilters(c *C) { innerKeys: []*expression.Column{dsSchema.Columns[1]}, pushedDownConds: "a = 1", otherConds: "", - ranges: "[[1 NULL NULL,1 NULL +inf]]", + ranges: "[[1 NULL,1 NULL]]", idxOff2KeyOff: "[-1 0 -1 -1]", accesses: "[eq(test.t.a, 1)]", remained: "[]", @@ -191,7 +190,7 @@ func (s *testUnitTestSuit) TestIndexJoinAnalyzeLookUpFilters(c *C) { innerKeys: []*expression.Column{dsSchema.Columns[1]}, pushedDownConds: "a in (1, 2, 3) and c in ('a', 'b', 'c')", otherConds: "", - ranges: "[[1 NULL \"a\" NULL,1 NULL \"a\" +inf] [2 NULL \"a\" NULL,2 NULL \"a\" +inf] [3 NULL \"a\" NULL,3 NULL \"a\" +inf] [1 NULL \"b\" NULL,1 NULL \"b\" +inf] [2 NULL \"b\" NULL,2 NULL \"b\" +inf] [3 NULL \"b\" NULL,3 NULL \"b\" +inf] [1 NULL \"c\" NULL,1 NULL \"c\" +inf] [2 NULL \"c\" NULL,2 NULL \"c\" +inf] [3 NULL \"c\" NULL,3 NULL \"c\" +inf]]", + ranges: "[[1 NULL \"a\",1 NULL \"a\"] [2 NULL \"a\",2 NULL \"a\"] [3 NULL \"a\",3 NULL \"a\"] [1 NULL \"b\",1 NULL \"b\"] [2 NULL \"b\",2 NULL \"b\"] [3 NULL \"b\",3 NULL \"b\"] [1 NULL \"c\",1 NULL \"c\"] [2 NULL \"c\",2 NULL \"c\"] [3 NULL \"c\",3 NULL \"c\"]]", idxOff2KeyOff: "[-1 0 -1 -1]", accesses: "[in(test.t.a, 1, 2, 3) in(test.t.c, a, b, c)]", remained: "[in(test.t.c, a, b, c)]", @@ -227,7 +226,6 @@ func (s *testUnitTestSuit) TestIndexJoinAnalyzeLookUpFilters(c *C) { others, err := s.rewriteSimpleExpr(tt.otherConds, joinNode.schema) c.Assert(err, IsNil) joinNode.OtherConditions = others - logrus.Warnf("others: %v", joinNode.OtherConditions) ranges, idxOff2KeyOff, accesses, remained, compareFilters, err := joinNode.analyzeLookUpFilters(idxInfo, dataSourceNode, tt.innerKeys) c.Assert(err, IsNil) c.Assert(fmt.Sprintf("%v", ranges), Equals, tt.ranges) From d9d51d85555918dc1b77c897db8b198f3554db52 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 13 Dec 2018 19:23:18 +0800 Subject: [PATCH 10/27] fix unit-test and explain test --- cmd/explaintest/r/explain_complex.result | 6 +++--- .../r/explain_complex_stats.result | 6 +++--- cmd/explaintest/r/explain_easy.result | 2 +- cmd/explaintest/r/topn_push_down.result | 4 ++-- cmd/explaintest/r/tpch.result | 20 +++++++++---------- executor/index_lookup_join_test.go | 6 +++--- planner/core/exhaust_physical_plans.go | 3 +-- 7 files changed, 23 insertions(+), 24 deletions(-) diff --git a/cmd/explaintest/r/explain_complex.result b/cmd/explaintest/r/explain_complex.result index 36de2d89a32be..b2bd21fdb5586 100644 --- a/cmd/explaintest/r/explain_complex.result +++ b/cmd/explaintest/r/explain_complex.result @@ -124,7 +124,7 @@ Projection_13 1.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test.d │ └─Selection_32 0.00 cop eq(gad.bm, 0), eq(gad.pt, "android") │ └─TableScan_31 3333.33 cop table:st, keep order:false, stats:pseudo └─IndexLookUp_23 0.00 root - ├─IndexScan_20 10.00 cop table:dd, index:aid, dic, range: decided by [gad.aid gad.ip], keep order:false, stats:pseudo + ├─IndexScan_20 10.00 cop table:dd, index:aid, dic, range: decided by [ eq(test.dd.aid, gad.aid) ], keep order:false, stats:pseudo └─Selection_22 0.00 cop eq(test.dd.bm, 0), eq(test.dd.pt, "android"), gt(test.dd.t, 1478143908) └─TableScan_21 10.00 cop table:dd, keep order:false, stats:pseudo explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext from st gad join dd sdk on gad.aid = sdk.aid and gad.dic = sdk.mac and gad.t < sdk.t where gad.t > 1477971479 and gad.bm = 0 and gad.pt = 'ios' and gad.dit = 'mac' and sdk.t > 1477971479 and sdk.bm = 0 and sdk.pt = 'ios' limit 3000; @@ -137,7 +137,7 @@ Projection_10 0.00 root gad.id, sdk.id, gad.aid, gad.cm, sdk.dic, sdk.ip, sdk.t, │ └─Selection_26 0.00 cop eq(gad.bm, 0), eq(gad.dit, "mac"), eq(gad.pt, "ios") │ └─TableScan_25 3333.33 cop table:st, keep order:false, stats:pseudo └─IndexLookUp_17 0.00 root - ├─IndexScan_14 10.00 cop table:sdk, index:aid, dic, range: decided by [gad.aid gad.dic], keep order:false, stats:pseudo + ├─IndexScan_14 10.00 cop table:sdk, index:aid, dic, range: decided by [ eq(sdk.aid, gad.aid) ], keep order:false, stats:pseudo └─Selection_16 0.00 cop eq(sdk.bm, 0), eq(sdk.pt, "ios"), gt(sdk.t, 1477971479) └─TableScan_15 10.00 cop table:dd, keep order:false, stats:pseudo explain SELECT cm, p1, p2, p3, p4, p5, p6_md5, p7_md5, count(1) as click_pv, count(DISTINCT ip) as click_ip FROM st WHERE (t between 1478188800 and 1478275200) and aid='cn.sbkcq' and pt='android' GROUP BY cm, p1, p2, p3, p4, p5, p6_md5, p7_md5; @@ -157,7 +157,7 @@ Projection_10 0.00 root dt.id, dt.aid, dt.pt, dt.dic, dt.cm, rr.gid, rr.acd, rr. │ └─Selection_42 0.00 cop eq(dt.bm, 0), eq(dt.pt, "ios"), gt(dt.t, 1478185592) │ └─TableScan_41 10000.00 cop table:dt, range:[0,+inf], keep order:false, stats:pseudo └─IndexLookUp_18 3.33 root - ├─IndexScan_15 10.00 cop table:rr, index:aid, dic, range: decided by [dt.aid dt.dic], keep order:false, stats:pseudo + ├─IndexScan_15 10.00 cop table:rr, index:aid, dic, range: decided by [ eq(rr.aid, dt.aid) eq(rr.dic, dt.dic) ], keep order:false, stats:pseudo └─Selection_17 3.33 cop eq(rr.pt, "ios"), gt(rr.t, 1478185592) └─TableScan_16 10.00 cop table:rr, keep order:false, stats:pseudo explain select pc,cr,count(DISTINCT uid) as pay_users,count(oid) as pay_times,sum(am) as am from pp where ps=2 and ppt>=1478188800 and ppt<1478275200 and pi in ('510017','520017') and uid in ('18089709','18090780') group by pc,cr; diff --git a/cmd/explaintest/r/explain_complex_stats.result b/cmd/explaintest/r/explain_complex_stats.result index 1569a2d3a850c..9223c3a525e63 100644 --- a/cmd/explaintest/r/explain_complex_stats.result +++ b/cmd/explaintest/r/explain_complex_stats.result @@ -133,7 +133,7 @@ Projection_13 424.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test │ └─Selection_28 424.00 cop eq(gad.bm, 0), eq(gad.pt, "android"), gt(gad.t, 1478143908) │ └─TableScan_27 1999.00 cop table:gad, range:[0,+inf], keep order:false └─IndexLookUp_23 455.80 root - ├─IndexScan_20 1.00 cop table:dd, index:aid, dic, range: decided by [gad.aid gad.ip], keep order:false + ├─IndexScan_20 1.00 cop table:dd, index:aid, dic, range: decided by [ eq(test.dd.aid, gad.aid) ], keep order:false └─Selection_22 455.80 cop eq(test.dd.bm, 0), eq(test.dd.pt, "android"), gt(test.dd.t, 1478143908) └─TableScan_21 1.00 cop table:dd, keep order:false explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext from st gad join dd sdk on gad.aid = sdk.aid and gad.dic = sdk.mac and gad.t < sdk.t where gad.t > 1477971479 and gad.bm = 0 and gad.pt = 'ios' and gad.dit = 'mac' and sdk.t > 1477971479 and sdk.bm = 0 and sdk.pt = 'ios' limit 3000; @@ -145,7 +145,7 @@ Projection_10 170.34 root gad.id, sdk.id, gad.aid, gad.cm, sdk.dic, sdk.ip, sdk. │ └─Selection_22 170.34 cop eq(gad.bm, 0), eq(gad.dit, "mac"), eq(gad.pt, "ios"), gt(gad.t, 1477971479) │ └─TableScan_21 1999.00 cop table:gad, range:[0,+inf], keep order:false └─IndexLookUp_17 509.04 root - ├─IndexScan_14 1.00 cop table:sdk, index:aid, dic, range: decided by [gad.aid gad.dic], keep order:false + ├─IndexScan_14 1.00 cop table:sdk, index:aid, dic, range: decided by [ eq(sdk.aid, gad.aid) ], keep order:false └─Selection_16 509.04 cop eq(sdk.bm, 0), eq(sdk.pt, "ios"), gt(sdk.t, 1477971479) └─TableScan_15 1.00 cop table:dd, keep order:false explain SELECT cm, p1, p2, p3, p4, p5, p6_md5, p7_md5, count(1) as click_pv, count(DISTINCT ip) as click_ip FROM st WHERE (t between 1478188800 and 1478275200) and aid='cn.sbkcq' and pt='android' GROUP BY cm, p1, p2, p3, p4, p5, p6_md5, p7_md5; @@ -165,7 +165,7 @@ Projection_10 428.32 root dt.id, dt.aid, dt.pt, dt.dic, dt.cm, rr.gid, rr.acd, r │ └─Selection_42 428.32 cop eq(dt.bm, 0), eq(dt.pt, "ios"), gt(dt.t, 1478185592) │ └─TableScan_41 2000.00 cop table:dt, range:[0,+inf], keep order:false └─IndexLookUp_18 970.00 root - ├─IndexScan_15 1.00 cop table:rr, index:aid, dic, range: decided by [dt.aid dt.dic], keep order:false + ├─IndexScan_15 1.00 cop table:rr, index:aid, dic, range: decided by [ eq(rr.aid, dt.aid) eq(rr.dic, dt.dic) ], keep order:false └─Selection_17 970.00 cop eq(rr.pt, "ios"), gt(rr.t, 1478185592) └─TableScan_16 1.00 cop table:rr, keep order:false explain select pc,cr,count(DISTINCT uid) as pay_users,count(oid) as pay_times,sum(am) as am from pp where ps=2 and ppt>=1478188800 and ppt<1478275200 and pi in ('510017','520017') and uid in ('18089709','18090780') group by pc,cr; diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index f6631ebd4056b..1d5e9784adff5 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -46,7 +46,7 @@ IndexJoin_11 4166.67 root left outer join, inner:IndexLookUp_10, outer key:test. ├─TableReader_23 3333.33 root data:TableScan_22 │ └─TableScan_22 3333.33 cop table:t1, range:(1,+inf], keep order:false, stats:pseudo └─IndexLookUp_10 10.00 root - ├─IndexScan_8 10.00 cop table:t2, index:c1, range: decided by [test.t1.c2], keep order:false, stats:pseudo + ├─IndexScan_8 10.00 cop table:t2, index:c1, range: decided by [ eq(test.t2.c1, test.t1.c2) ], keep order:false, stats:pseudo └─TableScan_9 10.00 cop table:t2, keep order:false, stats:pseudo explain update t1 set t1.c2 = 2 where t1.c1 = 1; id count task operator info diff --git a/cmd/explaintest/r/topn_push_down.result b/cmd/explaintest/r/topn_push_down.result index 4651bb89ac6b2..b1d4075be16ca 100644 --- a/cmd/explaintest/r/topn_push_down.result +++ b/cmd/explaintest/r/topn_push_down.result @@ -178,8 +178,8 @@ Projection_13 0.00 root te.expect_time │ │ └─Selection_104 0.00 cop eq(tr.brand_identy, 32314), eq(tr.domain_type, 2) │ │ └─TableScan_102 0.00 cop table:tr, keep order:false │ └─IndexLookUp_34 250.00 root - │ ├─IndexScan_31 10.00 cop table:te, index:trade_id, range: decided by [tr.id], keep order:false, stats:pseudo + │ ├─IndexScan_31 10.00 cop table:te, index:trade_id, range: decided by [ eq(te.trade_id, tr.id) ], keep order:false, stats:pseudo │ └─Selection_33 250.00 cop ge(te.expect_time, 2018-04-23 00:00:00.000000), le(te.expect_time, 2018-04-23 23:59:59.000000) │ └─TableScan_32 10.00 cop table:te, keep order:false, stats:pseudo └─IndexReader_136 10.00 root index:IndexScan_135 - └─IndexScan_135 10.00 cop table:p, index:relate_id, range: decided by [tr.id], keep order:false, stats:pseudo + └─IndexScan_135 10.00 cop table:p, index:relate_id, range: decided by [ eq(p.relate_id, tr.id) ], keep order:false, stats:pseudo diff --git a/cmd/explaintest/r/tpch.result b/cmd/explaintest/r/tpch.result index 1a9320261c2b0..236a3ac211b04 100644 --- a/cmd/explaintest/r/tpch.result +++ b/cmd/explaintest/r/tpch.result @@ -260,7 +260,7 @@ Projection_14 10.00 root tpch.lineitem.l_orderkey, 7_col_0, tpch.orders.o_orderd │ └─Selection_48 36870000.00 cop lt(tpch.orders.o_orderdate, 1995-03-13 00:00:00.000000) │ └─TableScan_47 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false └─IndexLookUp_25 162945114.27 root - ├─IndexScan_22 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + ├─IndexScan_22 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [ eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey) ], keep order:false └─Selection_24 162945114.27 cop gt(tpch.lineitem.l_shipdate, 1995-03-13 00:00:00.000000) └─TableScan_23 1.00 cop table:lineitem, keep order:false /* @@ -301,7 +301,7 @@ Sort_10 1.00 root tpch.orders.o_orderpriority:asc │ └─Selection_32 2925937.50 cop ge(tpch.orders.o_orderdate, 1995-01-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1995-04-01) │ └─TableScan_31 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false └─IndexLookUp_20 240004648.80 root - ├─IndexScan_17 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + ├─IndexScan_17 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [ eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey) ], keep order:false └─Selection_19 240004648.80 cop lt(tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate) └─TableScan_18 1.00 cop table:lineitem, keep order:false /* @@ -535,7 +535,7 @@ Sort_29 718.01 root all_nations.o_year:asc │ │ │ │ └─Selection_77 22382008.93 cop ge(tpch.orders.o_orderdate, 1995-01-01 00:00:00.000000), le(tpch.orders.o_orderdate, 1996-12-31 00:00:00.000000) │ │ │ │ └─TableScan_76 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false │ │ │ └─IndexLookUp_55 1.00 root - │ │ │ ├─IndexScan_53 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + │ │ │ ├─IndexScan_53 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [ eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey) ], keep order:false │ │ │ └─TableScan_54 1.00 cop table:lineitem, keep order:false │ │ └─TableReader_83 61674.00 root data:Selection_82 │ │ └─Selection_82 61674.00 cop eq(tpch.part.p_type, "SMALL PLATED COPPER") @@ -609,7 +609,7 @@ Sort_25 2406.00 root profit.nation:asc, profit.o_year:desc │ └─TableReader_39 1.00 root data:TableScan_38 │ └─TableScan_38 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false └─IndexLookUp_33 1.00 root - ├─IndexScan_31 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.lineitem.l_suppkey tpch.lineitem.l_partkey], keep order:false + ├─IndexScan_31 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [ eq(tpch.partsupp.ps_partkey, tpch.lineitem.l_partkey) eq(tpch.partsupp.ps_suppkey, tpch.lineitem.l_suppkey) ], keep order:false └─TableScan_32 1.00 cop table:partsupp, keep order:false /* Q10 Returned Item Reporting Query @@ -669,7 +669,7 @@ Projection_17 20.00 root tpch.customer.c_custkey, tpch.customer.c_name, 9_col_0, │ └─Selection_47 3017307.69 cop ge(tpch.orders.o_orderdate, 1993-08-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1993-11-01) │ └─TableScan_46 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false └─IndexLookUp_31 73916005.00 root - ├─IndexScan_28 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + ├─IndexScan_28 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [ eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey) ], keep order:false └─Selection_30 73916005.00 cop eq(tpch.lineitem.l_returnflag, "R") └─TableScan_29 1.00 cop table:lineitem, keep order:false /* @@ -929,7 +929,7 @@ Sort_13 3863988.24 root supplier_cnt:desc, tpch.part.p_brand:asc, tpch.part.p_ty │ │ └─Selection_40 1200618.43 cop in(tpch.part.p_size, 48, 19, 12, 4, 41, 7, 21, 39), ne(tpch.part.p_brand, "Brand#34"), not(like(tpch.part.p_type, "LARGE BRUSHED%", 92)) │ │ └─TableScan_39 10000000.00 cop table:part, range:[-inf,+inf], keep order:false │ └─IndexReader_25 1.00 root index:IndexScan_24 - │ └─IndexScan_24 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false + │ └─IndexScan_24 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [ eq(tpch.partsupp.ps_partkey, tpch.part.p_partkey) ], keep order:false └─TableReader_46 400000.00 root data:Selection_45 └─Selection_45 400000.00 cop like(tpch.supplier.s_comment, "%Customer%Complaints%", 92) └─TableScan_44 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false @@ -1035,7 +1035,7 @@ Projection_24 100.00 root tpch.customer.c_name, tpch.customer.c_custkey, tpch.or │ └─HashAgg_50 74063872.00 cop group by:tpch.lineitem.l_orderkey, funcs:sum(tpch.lineitem.l_quantity), firstrow(tpch.lineitem.l_orderkey) │ └─TableScan_55 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false └─IndexLookUp_34 1.00 root - ├─IndexScan_32 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + ├─IndexScan_32 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [ eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey) ], keep order:false └─TableScan_33 1.00 cop table:lineitem, keep order:false /* Q19 Discounted Revenue Query @@ -1157,7 +1157,7 @@ Sort_28 20000.00 root tpch.supplier.s_name:asc │ │ └─Selection_74 80007.93 cop like(tpch.part.p_name, "green%", 92) │ │ └─TableScan_73 10000000.00 cop table:part, range:[-inf,+inf], keep order:false │ └─IndexLookUp_58 1.00 root - │ ├─IndexScan_56 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false + │ ├─IndexScan_56 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [ eq(tpch.partsupp.ps_partkey, tpch.part.p_partkey) ], keep order:false │ └─TableScan_57 1.00 cop table:partsupp, keep order:false └─TableReader_80 44189356.65 root data:Selection_79 └─Selection_79 44189356.65 cop ge(tpch.lineitem.l_shipdate, 1993-01-01 00:00:00.000000), lt(tpch.lineitem.l_shipdate, 1994-01-01) @@ -1231,10 +1231,10 @@ Projection_25 1.00 root tpch.supplier.s_name, 17_col_0 │ │ └─Selection_57 36517371.00 cop eq(tpch.orders.o_orderstatus, "F") │ │ └─TableScan_56 1.00 cop table:orders, range: decided by [l1.l_orderkey], keep order:false │ └─IndexLookUp_52 1.00 root - │ ├─IndexScan_50 1.00 cop table:l2, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false + │ ├─IndexScan_50 1.00 cop table:l2, index:L_ORDERKEY, L_LINENUMBER, range: decided by [ eq(l2.l_orderkey, l1.l_orderkey) ], keep order:false │ └─TableScan_51 1.00 cop table:lineitem, keep order:false └─IndexLookUp_36 240004648.80 root - ├─IndexScan_33 1.00 cop table:l3, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false + ├─IndexScan_33 1.00 cop table:l3, index:L_ORDERKEY, L_LINENUMBER, range: decided by [ eq(l3.l_orderkey, l1.l_orderkey) ], keep order:false └─Selection_35 240004648.80 cop gt(l3.l_receiptdate, l3.l_commitdate) └─TableScan_34 1.00 cop table:lineitem, keep order:false /* diff --git a/executor/index_lookup_join_test.go b/executor/index_lookup_join_test.go index 3da1ede553a4f..fb60f397ea952 100644 --- a/executor/index_lookup_join_test.go +++ b/executor/index_lookup_join_test.go @@ -67,7 +67,7 @@ func (s *testSuite) TestIndexJoinUnionScan(c *C) { "│ └─TableScan_14 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", "└─UnionScan_11 10.00 root ", " └─IndexLookUp_10 10.00 root ", - " ├─IndexScan_8 10.00 cop table:t2, index:a, range: decided by [test.t1.a], keep order:false, stats:pseudo", + " ├─IndexScan_8 10.00 cop table:t2, index:a, range: decided by [ eq(test.t2.a, test.t1.a) ], keep order:false, stats:pseudo", " └─TableScan_9 10.00 cop table:t2, keep order:false, stats:pseudo", )) tk.MustQuery("select /*+ TIDB_INLJ(t1, t2)*/ * from t1 join t2 on t1.a = t2.a").Check(testkit.Rows( @@ -83,7 +83,7 @@ func (s *testSuite) TestIndexJoinUnionScan(c *C) { " │ └─TableScan_13 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", " └─UnionScan_10 10.00 root ", " └─IndexReader_9 10.00 root index:IndexScan_8", - " └─IndexScan_8 10.00 cop table:t2, index:a, range: decided by [test.t1.a], keep order:false, stats:pseudo", + " └─IndexScan_8 10.00 cop table:t2, index:a, range: decided by [ eq(test.t2.a, test.t1.a) ], keep order:false, stats:pseudo", )) tk.MustQuery("select /*+ TIDB_INLJ(t1, t2)*/ t1.a, t2.a from t1 join t2 on t1.a = t2.a").Check(testkit.Rows( "2 2", @@ -110,7 +110,7 @@ func (s *testSuite) TestBatchIndexJoinUnionScan(c *C) { " │ └─TableScan_26 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", " └─UnionScan_23 10.00 root ", " └─IndexReader_22 10.00 root index:IndexScan_21", - " └─IndexScan_21 10.00 cop table:t2, index:a, range: decided by [test.t1.a], keep order:false, stats:pseudo", + " └─IndexScan_21 10.00 cop table:t2, index:a, range: decided by [ eq(test.t2.a, test.t1.a) ], keep order:false, stats:pseudo", )) tk.MustQuery("select /*+ TIDB_INLJ(t1, t2)*/ count(*) from t1 join t2 on t1.a = t2.id").Check(testkit.Rows( "4", diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 4747a2436d191..784c9d14e4d9a 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -457,8 +457,7 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou func (p *LogicalJoin) buildRangeDecidedByInformation(idxCols []*expression.Column, idxOff2KeyOff []int, outerJoinKeys []*expression.Column, accesses []expression.Expression) string { - buffer := bytes.NewBufferString("") - buffer.WriteString("range decided by:[") + buffer := bytes.NewBufferString("[") for idxOff, keyOff := range idxOff2KeyOff { if keyOff == -1 { continue From 1d576a515a15e8dcb9210368ec4cb84df724ad48 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 19 Feb 2019 19:52:49 +0800 Subject: [PATCH 11/27] address comments --- executor/index_lookup_join.go | 4 ++-- planner/core/physical_plans.go | 6 +++++- util/ranger/ranger.go | 10 ++++++---- 3 files changed, 13 insertions(+), 7 deletions(-) diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index a1d7c7c80341a..3018785a6c0ec 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -437,7 +437,7 @@ type indexJoinLookUpContent struct { } func (iw *innerWorker) handleTask(ctx context.Context, task *lookUpJoinTask) error { - lookUpContents, err := iw.constructDatumLookupKeys(task) + lookUpContents, err := iw.constructLookupContent(task) if err != nil { return errors.Trace(err) } @@ -453,7 +453,7 @@ func (iw *innerWorker) handleTask(ctx context.Context, task *lookUpJoinTask) err return nil } -func (iw *innerWorker) constructDatumLookupKeys(task *lookUpJoinTask) ([]*indexJoinLookUpContent, error) { +func (iw *innerWorker) constructLookupContent(task *lookUpJoinTask) ([]*indexJoinLookUpContent, error) { lookUpContents := make([]*indexJoinLookUpContent, 0, task.outerResult.NumRows()) keyBuf := make([]byte, 0, 64) for i := 0; i < task.outerResult.NumRows(); i++ { diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 4177566116092..3d274941e9343 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -235,7 +235,11 @@ type PhysicalIndexJoin struct { Ranges []*ranger.Range // KeyOff2IdxOff maps the offsets in join key to the offsets in the index. KeyOff2IdxOff []int - // CompareFilters ... (will finish in later commit) + // CompareFilters stores the filters for last column if those filters need to be evaluated during execution. + // e.g. select * from t where t.a = t1.a and t.b > t1.b and t.b < t1.b+10 + // If there's index(t.a, t.b). All the filters can be used to construct index range but t.b > t1.b and t.b < t1.b=10 + // need to be evaluated after we fetch the data of t1. + // This struct stores them and evaluate them to ranges. CompareFilters *ColWithCompareOps } diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index b14c97d606f8f..6518bf3cdeb62 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -281,10 +281,12 @@ func BuildColumnRange(conds []expression.Expression, sc *stmtctx.StatementContex } if colLen != types.UnspecifiedLength { for _, ran := range ranges { - fixRangeDatum(&ran.LowVal[0], colLen, tp) - ran.LowExclude = false - fixRangeDatum(&ran.HighVal[0], colLen, tp) - ran.HighExclude = false + if fixRangeDatum(&ran.LowVal[0], colLen, tp) { + ran.LowExclude = false + } + if fixRangeDatum(&ran.HighVal[0], colLen, tp) { + ran.HighExclude = false + } } } return ranges, nil From b68fde36f53663c7c05c2c94c9aeec870db202ed Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 20 Feb 2019 20:53:39 +0800 Subject: [PATCH 12/27] split big method --- planner/core/exhaust_physical_plans.go | 251 +++++++++++--------- planner/core/exhaust_physical_plans_test.go | 15 +- 2 files changed, 149 insertions(+), 117 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 1d4e704ac5e47..377f44affb2c0 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -393,66 +393,62 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou return p.constructIndexJoin(prop, innerJoinKeys, outerJoinKeys, outerIdx, innerPlan, nil, keyOff2IdxOff, nil) } } - var ( - bestIndexInfo *model.IndexInfo - rangesOfBest []*ranger.Range - maxUsedCols int - accessesOfBest []expression.Expression - remainedOfBest []expression.Expression - idxOff2KeyOff []int - comparesOfBest *ColWithCompareOps - ) + helper := &indexJoinBuildHelper{join: p} for _, path := range ds.possibleAccessPaths { if path.isTablePath { continue } indexInfo := path.index - ranges, tmpIdxOff2KeyOff, accesses, remained, compareFilters, err := p.analyzeLookUpFilters(indexInfo, ds, innerJoinKeys) + err := helper.analyzeLookUpFilters(indexInfo, ds, innerJoinKeys) if err != nil { log.Warnf("[planner]: error happened when build index join: %v", err) continue } - // We choose the index by the number of used columns of the range, the much the better. - // Notice that there may be the cases like `t1.a=t2.a and b > 2 and b < 1`. So ranges can be nil though the conditions are valid. - // But obviously when the range is nil, we don't need index join. - if len(ranges) > 0 && len(ranges[0].LowVal) > maxUsedCols { - bestIndexInfo = indexInfo - maxUsedCols = len(ranges[0].LowVal) - rangesOfBest = ranges - accessesOfBest = accesses - remainedOfBest = remained - idxOff2KeyOff = tmpIdxOff2KeyOff - comparesOfBest = compareFilters - } } - if bestIndexInfo != nil { + if helper.chosenIndexInfo != nil { keyOff2IdxOff := make([]int, len(innerJoinKeys)) for i := range keyOff2IdxOff { keyOff2IdxOff[i] = -1 } - for idxOff, keyOff := range idxOff2KeyOff { + for idxOff, keyOff := range helper.idxOff2KeyOff { if keyOff != -1 { keyOff2IdxOff[keyOff] = idxOff } } - idxCols, _ := expression.IndexInfo2Cols(ds.schema.Columns, bestIndexInfo) - rangeInfo := p.buildRangeDecidedByInformation(idxCols, idxOff2KeyOff, outerJoinKeys, accessesOfBest) - innerPlan := p.constructInnerIndexScan(ds, bestIndexInfo, remainedOfBest, outerJoinKeys, us, rangeInfo) - return p.constructIndexJoin(prop, innerJoinKeys, outerJoinKeys, outerIdx, innerPlan, rangesOfBest, keyOff2IdxOff, comparesOfBest) + idxCols, _ := expression.IndexInfo2Cols(ds.schema.Columns, helper.chosenIndexInfo) + rangeInfo := helper.buildRangeDecidedByInformation(idxCols, outerJoinKeys) + innerPlan := p.constructInnerIndexScan(ds, helper.chosenIndexInfo, helper.chosenRemained, outerJoinKeys, us, rangeInfo) + return p.constructIndexJoin(prop, innerJoinKeys, outerJoinKeys, outerIdx, innerPlan, helper.chosenRanges, keyOff2IdxOff, helper.lastColManager) } return nil } -func (p *LogicalJoin) buildRangeDecidedByInformation(idxCols []*expression.Column, idxOff2KeyOff []int, - outerJoinKeys []*expression.Column, accesses []expression.Expression) string { +type indexJoinBuildHelper struct { + join *LogicalJoin + + chosenIndexInfo *model.IndexInfo + maxUsedCols int + chosenAccess []expression.Expression + chosenRemained []expression.Expression + idxOff2KeyOff []int + lastColManager *ColWithCompareOps + chosenRanges []*ranger.Range + + curPossibleUsedKeys []*expression.Column + curNotUsedIndexCols []*expression.Column + curNotUsedColLens []int + curIdxOff2KeyOff []int +} + +func (ijHelper *indexJoinBuildHelper) buildRangeDecidedByInformation(idxCols []*expression.Column, outerJoinKeys []*expression.Column) string { buffer := bytes.NewBufferString("[") - for idxOff, keyOff := range idxOff2KeyOff { + for idxOff, keyOff := range ijHelper.idxOff2KeyOff { if keyOff == -1 { continue } buffer.WriteString(fmt.Sprintf(" eq(%v, %v)", idxCols[idxOff], outerJoinKeys[keyOff])) } - for _, access := range accesses { + for _, access := range ijHelper.chosenAccess { buffer.WriteString(fmt.Sprintf(" %v", access)) } buffer.WriteString(" ]") @@ -636,84 +632,51 @@ func (cwc *ColWithCompareOps) String() string { return buffer.String() } -func (p *LogicalJoin) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan *DataSource, - innerJoinKeys []*expression.Column) ([]*ranger.Range, []int, []expression.Expression, []expression.Expression, *ColWithCompareOps, error) { - idxCols, colLengths := expression.IndexInfo2Cols(innerPlan.schema.Columns, indexInfo) - if len(idxCols) == 0 { - return nil, nil, nil, nil, nil, nil - } - accesses := make([]expression.Expression, 0, len(idxCols)) - tmpSchema := expression.NewSchema(innerJoinKeys...) - idxOff2keyOff := make([]int, len(idxCols)) - possibleUsedKeys := make([]*expression.Column, 0, len(idxCols)) - notKeyIdxCols := make([]*expression.Column, 0, len(idxCols)) - notKeyIdxColsLen := make([]int, 0, len(idxCols)) - matchedKeyCnt := 0 - // This loop finds out what index column appears in join key, and what is not. +func (ijHelper *indexJoinBuildHelper) checkIndex(tmpSchema *expression.Schema, idxCols []*expression.Column, colLens []int) { + ijHelper.curIdxOff2KeyOff = make([]int, len(idxCols)) + ijHelper.curPossibleUsedKeys = make([]*expression.Column, 0, len(idxCols)) + ijHelper.curNotUsedIndexCols = make([]*expression.Column, 0, len(idxCols)) + ijHelper.curNotUsedColLens = make([]int, 0, len(idxCols)) for i, idxCol := range idxCols { - idxOff2keyOff[i] = tmpSchema.ColumnIndex(idxCol) - if idxOff2keyOff[i] >= 0 { - matchedKeyCnt++ - possibleUsedKeys = append(possibleUsedKeys, idxCol) + ijHelper.curIdxOff2KeyOff[i] = tmpSchema.ColumnIndex(idxCol) + if ijHelper.curIdxOff2KeyOff[i] >= 0 { + ijHelper.curPossibleUsedKeys = append(ijHelper.curPossibleUsedKeys, idxCol) continue } - notKeyIdxCols = append(notKeyIdxCols, idxCol) - notKeyIdxColsLen = append(notKeyIdxColsLen, colLengths[i]) - } - // If no index column appears in join key, we just break. - // TODO: It may meet this case: There's no join key condition, but have compare filters. - // e.g. select * from t1, t2 on t1.a=t2.a and t2.b > t1.b-10 and t2.b < t1.b where t1.a=1 and t2.a=1. - // After constant propagation. The t1.a=t2.a is removed. And if we have index (t2.a, t2.b). It can apply index join - // to speed up. - if matchedKeyCnt <= 0 { - return nil, nil, nil, nil, nil, nil - } - keyMatchedLen := len(idxCols) - for ; keyMatchedLen > 0; keyMatchedLen-- { - if idxOff2keyOff[keyMatchedLen-1] != -1 { - break - } + ijHelper.curNotUsedIndexCols = append(ijHelper.curNotUsedIndexCols, idxCol) + ijHelper.curNotUsedColLens = append(ijHelper.curNotUsedColLens, colLens[i]) } +} + +func (ijHelper *indexJoinBuildHelper) analyzeSimpleFilters(innerPlan *DataSource, keyMatchedLen, matchedKeyCnt int) ([]expression.Expression, []expression.Expression, []expression.Expression) { remained := make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) rangeFilterCandidates := make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) // This loop deal first filter out the expressions that contains columns not in index. for _, innerFilter := range innerPlan.pushedDownConds { affectedCols := expression.ExtractColumns(innerFilter) - if expression.ColumnSliceIsIntersect(affectedCols, possibleUsedKeys) { + if expression.ColumnSliceIsIntersect(affectedCols, ijHelper.curPossibleUsedKeys) { remained = append(remained, innerFilter) continue } rangeFilterCandidates = append(rangeFilterCandidates, innerFilter) } // Extract the eq/in functions of possible join key. This returned list keeps the same order with index column. - notKeyEqAndIn, remainedEqAndIn, rangeFilterCandidates, _ := ranger.ExtractEqAndInCondition(p.ctx, rangeFilterCandidates, notKeyIdxCols, notKeyIdxColsLen) + notKeyEqAndIn, remainedEqAndIn, rangeFilterCandidates, _ := ranger.ExtractEqAndInCondition(innerPlan.ctx, rangeFilterCandidates, ijHelper.curNotUsedIndexCols, ijHelper.curNotUsedColLens) // We hope that the index cols appeared in the join keys can all be used to build range. If it cannot be satisfied, // we'll mark this index as cannot be used for index join. // So we should make sure that all columns before the keyMatchedLen is join key or has eq/in function. if len(notKeyEqAndIn) < keyMatchedLen-matchedKeyCnt { - return nil, nil, nil, nil, nil, nil + return nil, nil, nil } - accesses = append(accesses, notKeyEqAndIn...) remained = append(remained, remainedEqAndIn...) - nextColPos := matchedKeyCnt + len(notKeyEqAndIn) - // If all cols have been considered, we can return the current result. - if nextColPos == len(idxCols) { - remained = append(remained, rangeFilterCandidates...) - ranges, err := p.buildTemplateRange(idxOff2keyOff, matchedKeyCnt, notKeyEqAndIn, nil, false) - if err != nil { - return nil, nil, nil, nil, nil, err - } - return ranges, idxOff2keyOff, accesses, remained, nil, nil - } - nextCol := idxCols[nextColPos] - nextColCmpFilterManager := &ColWithCompareOps{ - targetCol: nextCol, - colLength: colLengths[nextColPos], - affectedColSchema: expression.NewSchema(), - } - // We first loop the other conds to see whether there's conditions can be used to build range. + return notKeyEqAndIn, remained, rangeFilterCandidates +} + +func (ijHelper *indexJoinBuildHelper) analyzeOtherFilters(nextCol *expression.Column, + innerPlan *DataSource, cwc *ColWithCompareOps) []expression.Expression { + var lastColAccesses []expression.Expression loopOtherConds: - for _, filter := range p.OtherConditions { + for _, filter := range ijHelper.join.OtherConditions { sf, ok := filter.(*expression.ScalarFunction) if !ok || !(sf.FuncName.L == ast.LE || sf.FuncName.L == ast.LT || sf.FuncName.L == ast.GE || sf.FuncName.L == ast.GT) { continue @@ -728,8 +691,8 @@ loopOtherConds: continue loopOtherConds } } - accesses = append(accesses, sf) - nextColCmpFilterManager.appendNewExpr(sf.FuncName.L, sf.GetArgs()[1], affectedCols) + lastColAccesses = append(lastColAccesses, sf) + cwc.appendNewExpr(sf.FuncName.L, sf.GetArgs()[1], affectedCols) } else if rCol, ok := sf.GetArgs()[1].(*expression.Column); ok && rCol.Equal(nil, nextCol) { affectedCols := expression.ExtractColumns(sf.GetArgs()[0]) if len(affectedCols) == 0 { @@ -740,43 +703,111 @@ loopOtherConds: continue loopOtherConds } } - accesses = append(accesses, sf) - nextColCmpFilterManager.appendNewExpr(symmetricOp[sf.FuncName.L], sf.GetArgs()[0], affectedCols) + lastColAccesses = append(lastColAccesses, sf) + cwc.appendNewExpr(symmetricOp[sf.FuncName.L], sf.GetArgs()[0], affectedCols) } } - if len(nextColCmpFilterManager.OpType) == 0 { - colAccesses, colRemained := ranger.DetachCondsForTableRange(p.ctx, rangeFilterCandidates, nextCol) - remained = append(remained, colRemained...) - if colLengths[nextColPos] != types.UnspecifiedLength { - remained = append(remained, colAccesses...) + return lastColAccesses +} + +func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan *DataSource, innerJoinKeys []*expression.Column) error { + idxCols, colLengths := expression.IndexInfo2Cols(innerPlan.schema.Columns, indexInfo) + if len(idxCols) == 0 { + return nil + } + accesses := make([]expression.Expression, 0, len(idxCols)) + tmpSchema := expression.NewSchema(innerJoinKeys...) + ijHelper.checkIndex(tmpSchema, idxCols, colLengths) + matchedKeyCnt := len(ijHelper.curPossibleUsedKeys) + // If no index column appears in join key, we just break. + // TODO: It may meet this case: There's no join key condition, but have compare filters. + // e.g. select * from t1, t2 on t1.a=t2.a and t2.b > t1.b-10 and t2.b < t1.b where t1.a=1 and t2.a=1. + // After constant propagation. The t1.a=t2.a is removed. And if we have index (t2.a, t2.b). It can apply index join + // to speed up. + if matchedKeyCnt <= 0 { + return nil + } + keyMatchedLen := len(idxCols) + for ; keyMatchedLen > 0; keyMatchedLen-- { + if ijHelper.curIdxOff2KeyOff[keyMatchedLen-1] != -1 { + break } - accesses = append(accesses, colAccesses...) + } + notKeyEqAndIn, remained, rangeFilterCandidates := ijHelper.analyzeSimpleFilters(innerPlan, keyMatchedLen, matchedKeyCnt) + accesses = append(accesses, notKeyEqAndIn...) + // We hope that the index cols appeared in the join keys can all be used to build range. If it cannot be satisfied, + // we'll mark this index as cannot be used for index join. + // So we should make sure that all columns before the keyMatchedLen is join key or has eq/in function. + if len(notKeyEqAndIn) < keyMatchedLen-matchedKeyCnt { + return nil + } + lastColPos := matchedKeyCnt + len(notKeyEqAndIn) + // If all cols have been considered, we can return the current result. + if lastColPos == len(idxCols) { + remained = append(remained, rangeFilterCandidates...) + ranges, err := ijHelper.buildTemplateRange(matchedKeyCnt, notKeyEqAndIn, nil, false) + if err != nil { + return err + } + ijHelper.updateBestChoice(ranges, indexInfo, accesses, remained, nil) + return nil + } + lastPossibleCol := idxCols[lastColPos] + lastColManager := &ColWithCompareOps{ + targetCol: lastPossibleCol, + colLength: colLengths[lastColPos], + affectedColSchema: expression.NewSchema(), + } + lastColAccess := ijHelper.analyzeOtherFilters(lastPossibleCol, innerPlan, lastColManager) + if len(lastColAccess) == 0 { + colAccesses, colRemained := ranger.DetachCondsForTableRange(ijHelper.join.ctx, rangeFilterCandidates, lastPossibleCol) var ranges, nextColRange []*ranger.Range var err error if len(colAccesses) > 0 { - nextColRange, err = ranger.BuildColumnRange(colAccesses, p.ctx.GetSessionVars().StmtCtx, nextCol.RetType, colLengths[nextColPos]) + nextColRange, err = ranger.BuildColumnRange(colAccesses, ijHelper.join.ctx.GetSessionVars().StmtCtx, lastPossibleCol.RetType, colLengths[lastColPos]) if err != nil { - return nil, nil, nil, nil, nil, err + return err } - ranges, err = p.buildTemplateRange(idxOff2keyOff, matchedKeyCnt, notKeyEqAndIn, nextColRange, false) - } else { - ranges, err = p.buildTemplateRange(idxOff2keyOff, matchedKeyCnt, notKeyEqAndIn, nil, false) } + ranges, err = ijHelper.buildTemplateRange(matchedKeyCnt, notKeyEqAndIn, nextColRange, false) if err != nil { - return nil, nil, nil, nil, nil, err + return err + } + remained = append(remained, colRemained...) + if colLengths[lastColPos] != types.UnspecifiedLength { + remained = append(remained, colAccesses...) } - return ranges, idxOff2keyOff, accesses, remained, nil, nil + accesses = append(accesses, colAccesses...) + ijHelper.updateBestChoice(ranges, indexInfo, accesses, remained, nil) + return nil } + accesses = append(accesses, lastColAccess...) remained = append(remained, rangeFilterCandidates...) - ranges, err := p.buildTemplateRange(idxOff2keyOff, matchedKeyCnt, notKeyEqAndIn, nil, true) + ranges, err := ijHelper.buildTemplateRange(matchedKeyCnt, notKeyEqAndIn, nil, true) if err != nil { - return nil, nil, nil, nil, nil, err + return err } + ijHelper.updateBestChoice(ranges, indexInfo, accesses, remained, lastColManager) + return nil +} - return ranges, idxOff2keyOff, accesses, remained, nextColCmpFilterManager, nil +func (ijHelper *indexJoinBuildHelper) updateBestChoice(ranges []*ranger.Range, idxInfo *model.IndexInfo, accesses, + remained []expression.Expression, lastColManager *ColWithCompareOps) { + // We choose the index by the number of used columns of the range, the much the better. + // Notice that there may be the cases like `t1.a=t2.a and b > 2 and b < 1`. So ranges can be nil though the conditions are valid. + // But obviously when the range is nil, we don't need index join. + if len(ranges) > 0 && len(ranges[0].LowVal) > ijHelper.maxUsedCols { + ijHelper.chosenIndexInfo = idxInfo + ijHelper.maxUsedCols = len(ranges[0].LowVal) + ijHelper.chosenRanges = ranges + ijHelper.chosenAccess = accesses + ijHelper.chosenRemained = remained + ijHelper.idxOff2KeyOff = ijHelper.curIdxOff2KeyOff + ijHelper.lastColManager = lastColManager + } } -func (p *LogicalJoin) buildTemplateRange(idxOff2KeyOff []int, matchedKeyCnt int, eqAndInFuncs []expression.Expression, nextColRange []*ranger.Range, haveExtraCol bool) (ranges []*ranger.Range, err error) { +func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAndInFuncs []expression.Expression, nextColRange []*ranger.Range, haveExtraCol bool) (ranges []*ranger.Range, err error) { pointLength := matchedKeyCnt + len(eqAndInFuncs) if nextColRange != nil { for _, colRan := range nextColRange { @@ -806,7 +837,7 @@ func (p *LogicalJoin) buildTemplateRange(idxOff2KeyOff []int, matchedKeyCnt int, emptyRow := chunk.Row{} for i, j := 0, 0; j < len(eqAndInFuncs); i++ { // This position is occupied by join key. - if idxOff2KeyOff[i] != -1 { + if ijHelper.curIdxOff2KeyOff[i] != -1 { continue } sf := eqAndInFuncs[j].(*expression.ScalarFunction) diff --git a/planner/core/exhaust_physical_plans_test.go b/planner/core/exhaust_physical_plans_test.go index 952900a006be5..36212cadc74da 100644 --- a/planner/core/exhaust_physical_plans_test.go +++ b/planner/core/exhaust_physical_plans_test.go @@ -179,7 +179,7 @@ func (s *testUnitTestSuit) TestIndexJoinAnalyzeLookUpFilters(c *C) { innerKeys: []*expression.Column{dsSchema.Columns[1]}, pushedDownConds: "a = 1 and c > 'a' and c < 'aaaaaa'", otherConds: "", - ranges: "[[1 NULL \"a\",1 NULL \"[97 97]\"]]", + ranges: "[(1 NULL \"a\",1 NULL \"[97 97]\"]]", idxOff2KeyOff: "[-1 0 -1 -1]", accesses: "[eq(test.t.a, 1) gt(test.t.c, a) lt(test.t.c, aaaaaa)]", remained: "[gt(test.t.c, a) lt(test.t.c, aaaaaa)]", @@ -226,12 +226,13 @@ func (s *testUnitTestSuit) TestIndexJoinAnalyzeLookUpFilters(c *C) { others, err := s.rewriteSimpleExpr(tt.otherConds, joinNode.schema) c.Assert(err, IsNil) joinNode.OtherConditions = others - ranges, idxOff2KeyOff, accesses, remained, compareFilters, err := joinNode.analyzeLookUpFilters(idxInfo, dataSourceNode, tt.innerKeys) + helper := &indexJoinBuildHelper{join: joinNode, lastColManager: nil} + err = helper.analyzeLookUpFilters(idxInfo, dataSourceNode, tt.innerKeys) c.Assert(err, IsNil) - c.Assert(fmt.Sprintf("%v", ranges), Equals, tt.ranges) - c.Assert(fmt.Sprintf("%v", idxOff2KeyOff), Equals, tt.idxOff2KeyOff) - c.Assert(fmt.Sprintf("%v", accesses), Equals, tt.accesses) - c.Assert(fmt.Sprintf("%v", remained), Equals, tt.remained) - c.Assert(fmt.Sprintf("%v", compareFilters), Equals, tt.compareFilters) + c.Assert(fmt.Sprintf("%v", helper.chosenRanges), Equals, tt.ranges) + c.Assert(fmt.Sprintf("%v", helper.idxOff2KeyOff), Equals, tt.idxOff2KeyOff) + c.Assert(fmt.Sprintf("%v", helper.chosenAccess), Equals, tt.accesses) + c.Assert(fmt.Sprintf("%v", helper.chosenRemained), Equals, tt.remained) + c.Assert(fmt.Sprintf("%v", helper.lastColManager), Equals, tt.compareFilters) } } From 28299511d9cf60f29b01db7f86dc01711b03097d Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 27 Feb 2019 13:30:20 +0800 Subject: [PATCH 13/27] address comments --- cmd/explaintest/r/explain_complex.result | 6 +++--- .../r/explain_complex_stats.result | 6 +++--- cmd/explaintest/r/explain_easy.result | 2 +- cmd/explaintest/r/topn_push_down.result | 4 ++-- cmd/explaintest/r/tpch.result | 20 +++++++++---------- executor/index_lookup_join_test.go | 6 +++--- planner/core/exhaust_physical_plans.go | 14 +++++++++---- 7 files changed, 32 insertions(+), 26 deletions(-) diff --git a/cmd/explaintest/r/explain_complex.result b/cmd/explaintest/r/explain_complex.result index 9cd940340b29f..371121283ac96 100644 --- a/cmd/explaintest/r/explain_complex.result +++ b/cmd/explaintest/r/explain_complex.result @@ -120,7 +120,7 @@ Projection_13 1.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test.d └─HashAgg_19 1.00 root group by:gad.aid, test.dd.dic, funcs:firstrow(gad.id), firstrow(gad.aid), firstrow(gad.cm), firstrow(gad.p1), firstrow(gad.p2), firstrow(gad.p3), firstrow(gad.p4), firstrow(gad.p5), firstrow(gad.p6_md5), firstrow(gad.p7_md5), firstrow(gad.ext), firstrow(gad.t), firstrow(test.dd.id), firstrow(test.dd.dic), firstrow(test.dd.ip), firstrow(test.dd.t) └─IndexJoin_24 0.00 root inner join, inner:IndexLookUp_23, outer key:gad.aid, inner key:test.dd.aid, other cond:eq(test.dd.ip, gad.ip), gt(test.dd.t, gad.t) ├─IndexLookUp_23 0.00 root - │ ├─IndexScan_20 10.00 cop table:dd, index:aid, dic, range: decided by [ eq(test.dd.aid, gad.aid) ], keep order:false, stats:pseudo + │ ├─IndexScan_20 10.00 cop table:dd, index:aid, dic, range: decided by [eq(test.dd.aid, gad.aid)], keep order:false, stats:pseudo │ └─Selection_22 0.00 cop eq(test.dd.bm, 0), eq(test.dd.pt, "android"), gt(test.dd.t, 1478143908), not(isnull(test.dd.ip)), not(isnull(test.dd.t)) │ └─TableScan_21 10.00 cop table:dd, keep order:false, stats:pseudo └─IndexLookUp_33 3.33 root @@ -137,7 +137,7 @@ Projection_10 0.00 root gad.id, sdk.id, gad.aid, gad.cm, sdk.dic, sdk.ip, sdk.t, │ └─Selection_26 0.00 cop eq(gad.bm, 0), eq(gad.dit, "mac"), eq(gad.pt, "ios"), not(isnull(gad.dic)) │ └─TableScan_25 3333.33 cop table:st, keep order:false, stats:pseudo └─IndexLookUp_17 0.00 root - ├─IndexScan_14 10.00 cop table:sdk, index:aid, dic, range: decided by [ eq(sdk.aid, gad.aid) ], keep order:false, stats:pseudo + ├─IndexScan_14 10.00 cop table:sdk, index:aid, dic, range: decided by [eq(sdk.aid, gad.aid)], keep order:false, stats:pseudo └─Selection_16 0.00 cop eq(sdk.bm, 0), eq(sdk.pt, "ios"), gt(sdk.t, 1477971479), not(isnull(sdk.mac)), not(isnull(sdk.t)) └─TableScan_15 10.00 cop table:dd, keep order:false, stats:pseudo explain SELECT cm, p1, p2, p3, p4, p5, p6_md5, p7_md5, count(1) as click_pv, count(DISTINCT ip) as click_ip FROM st WHERE (t between 1478188800 and 1478275200) and aid='cn.sbkcq' and pt='android' GROUP BY cm, p1, p2, p3, p4, p5, p6_md5, p7_md5; @@ -157,7 +157,7 @@ Projection_10 0.00 root dt.id, dt.aid, dt.pt, dt.dic, dt.cm, rr.gid, rr.acd, rr. │ └─Selection_42 0.00 cop eq(dt.bm, 0), eq(dt.pt, "ios"), gt(dt.t, 1478185592), not(isnull(dt.dic)) │ └─TableScan_41 10000.00 cop table:dt, range:[0,+inf], keep order:false, stats:pseudo └─IndexLookUp_18 3.33 root - ├─IndexScan_15 10.00 cop table:rr, index:aid, dic, range: decided by [ eq(rr.aid, dt.aid) eq(rr.dic, dt.dic) ], keep order:false, stats:pseudo + ├─IndexScan_15 10.00 cop table:rr, index:aid, dic, range: decided by [eq(rr.aid, dt.aid) eq(rr.dic, dt.dic)], keep order:false, stats:pseudo └─Selection_17 3.33 cop eq(rr.pt, "ios"), gt(rr.t, 1478185592) └─TableScan_16 10.00 cop table:rr, keep order:false, stats:pseudo explain select pc,cr,count(DISTINCT uid) as pay_users,count(oid) as pay_times,sum(am) as am from pp where ps=2 and ppt>=1478188800 and ppt<1478275200 and pi in ('510017','520017') and uid in ('18089709','18090780') group by pc,cr; diff --git a/cmd/explaintest/r/explain_complex_stats.result b/cmd/explaintest/r/explain_complex_stats.result index df3f0ed8c11d6..58d4768082ce7 100644 --- a/cmd/explaintest/r/explain_complex_stats.result +++ b/cmd/explaintest/r/explain_complex_stats.result @@ -133,7 +133,7 @@ Projection_13 424.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test │ └─Selection_28 424.00 cop eq(gad.bm, 0), eq(gad.pt, "android"), gt(gad.t, 1478143908), not(isnull(gad.ip)) │ └─TableScan_27 1999.00 cop table:gad, range:[0,+inf], keep order:false └─IndexLookUp_23 455.80 root - ├─IndexScan_20 1.00 cop table:dd, index:aid, dic, range: decided by [ eq(test.dd.aid, gad.aid) ], keep order:false + ├─IndexScan_20 1.00 cop table:dd, index:aid, dic, range: decided by [eq(test.dd.aid, gad.aid)], keep order:false └─Selection_22 455.80 cop eq(test.dd.bm, 0), eq(test.dd.pt, "android"), gt(test.dd.t, 1478143908), not(isnull(test.dd.ip)), not(isnull(test.dd.t)) └─TableScan_21 1.00 cop table:dd, keep order:false explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext from st gad join dd sdk on gad.aid = sdk.aid and gad.dic = sdk.mac and gad.t < sdk.t where gad.t > 1477971479 and gad.bm = 0 and gad.pt = 'ios' and gad.dit = 'mac' and sdk.t > 1477971479 and sdk.bm = 0 and sdk.pt = 'ios' limit 3000; @@ -145,7 +145,7 @@ Projection_10 170.34 root gad.id, sdk.id, gad.aid, gad.cm, sdk.dic, sdk.ip, sdk. │ └─Selection_22 170.34 cop eq(gad.bm, 0), eq(gad.dit, "mac"), eq(gad.pt, "ios"), gt(gad.t, 1477971479), not(isnull(gad.dic)) │ └─TableScan_21 1999.00 cop table:gad, range:[0,+inf], keep order:false └─IndexLookUp_17 509.04 root - ├─IndexScan_14 1.00 cop table:sdk, index:aid, dic, range: decided by [ eq(sdk.aid, gad.aid) ], keep order:false + ├─IndexScan_14 1.00 cop table:sdk, index:aid, dic, range: decided by [eq(sdk.aid, gad.aid)], keep order:false └─Selection_16 509.04 cop eq(sdk.bm, 0), eq(sdk.pt, "ios"), gt(sdk.t, 1477971479), not(isnull(sdk.mac)), not(isnull(sdk.t)) └─TableScan_15 1.00 cop table:dd, keep order:false explain SELECT cm, p1, p2, p3, p4, p5, p6_md5, p7_md5, count(1) as click_pv, count(DISTINCT ip) as click_ip FROM st WHERE (t between 1478188800 and 1478275200) and aid='cn.sbkcq' and pt='android' GROUP BY cm, p1, p2, p3, p4, p5, p6_md5, p7_md5; @@ -165,7 +165,7 @@ Projection_10 428.32 root dt.id, dt.aid, dt.pt, dt.dic, dt.cm, rr.gid, rr.acd, r │ └─Selection_42 428.32 cop eq(dt.bm, 0), eq(dt.pt, "ios"), gt(dt.t, 1478185592), not(isnull(dt.dic)) │ └─TableScan_41 2000.00 cop table:dt, range:[0,+inf], keep order:false └─IndexLookUp_18 970.00 root - ├─IndexScan_15 1.00 cop table:rr, index:aid, dic, range: decided by [ eq(rr.aid, dt.aid) eq(rr.dic, dt.dic) ], keep order:false + ├─IndexScan_15 1.00 cop table:rr, index:aid, dic, range: decided by [eq(rr.aid, dt.aid) eq(rr.dic, dt.dic)], keep order:false └─Selection_17 970.00 cop eq(rr.pt, "ios"), gt(rr.t, 1478185592) └─TableScan_16 1.00 cop table:rr, keep order:false explain select pc,cr,count(DISTINCT uid) as pay_users,count(oid) as pay_times,sum(am) as am from pp where ps=2 and ppt>=1478188800 and ppt<1478275200 and pi in ('510017','520017') and uid in ('18089709','18090780') group by pc,cr; diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 0ad2ad0ffdcfb..30b6299829a03 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -47,7 +47,7 @@ IndexJoin_12 4166.67 root left outer join, inner:IndexLookUp_11, outer key:test. │ └─TableScan_25 3333.33 cop table:t1, range:(1,+inf], keep order:false, stats:pseudo └─IndexLookUp_11 0.00 root ├─Selection_10 0.00 cop not(isnull(test.t2.c1)) - │ └─IndexScan_8 10.00 cop table:t2, index:c1, range: decided by [ eq(test.t2.c1, test.t1.c2) ], keep order:false, stats:pseudo + │ └─IndexScan_8 10.00 cop table:t2, index:c1, range: decided by [eq(test.t2.c1, test.t1.c2)], keep order:false, stats:pseudo └─TableScan_9 0.00 cop table:t2, keep order:false explain update t1 set t1.c2 = 2 where t1.c1 = 1; id count task operator info diff --git a/cmd/explaintest/r/topn_push_down.result b/cmd/explaintest/r/topn_push_down.result index 5332e3d4265ab..f23cc678505ab 100644 --- a/cmd/explaintest/r/topn_push_down.result +++ b/cmd/explaintest/r/topn_push_down.result @@ -178,12 +178,12 @@ Projection_13 0.00 root te.expect_time │ │ └─Selection_84 0.00 cop eq(tr.brand_identy, 32314), eq(tr.domain_type, 2) │ │ └─TableScan_82 0.00 cop table:tr, keep order:false │ └─IndexLookUp_35 250.00 root - │ ├─IndexScan_32 10.00 cop table:te, index:trade_id, range: decided by [ eq(te.trade_id, tr.id) ], keep order:false, stats:pseudo + │ ├─IndexScan_32 10.00 cop table:te, index:trade_id, range: decided by [eq(te.trade_id, tr.id)], keep order:false, stats:pseudo │ └─Selection_34 250.00 cop ge(te.expect_time, 2018-04-23 00:00:00.000000), le(te.expect_time, 2018-04-23 23:59:59.000000) │ └─TableScan_33 10.00 cop table:te, keep order:false, stats:pseudo └─IndexReader_103 0.00 root index:Selection_102 └─Selection_102 0.00 cop not(isnull(p.relate_id)) - └─IndexScan_101 10.00 cop table:p, index:relate_id, range: decided by [ eq(p.relate_id, tr.id) ], keep order:false, stats:pseudo + └─IndexScan_101 10.00 cop table:p, index:relate_id, range: decided by [eq(p.relate_id, tr.id)], keep order:false, stats:pseudo desc select 1 as a from dual order by a limit 1; id count task operator info Projection_7 1.00 root 1 diff --git a/cmd/explaintest/r/tpch.result b/cmd/explaintest/r/tpch.result index 7d8e533fa73f6..47d4ff66cfd6c 100644 --- a/cmd/explaintest/r/tpch.result +++ b/cmd/explaintest/r/tpch.result @@ -261,7 +261,7 @@ Projection_14 10.00 root tpch.lineitem.l_orderkey, 7_col_0, tpch.orders.o_orderd │ └─Selection_52 36870000.00 cop lt(tpch.orders.o_orderdate, 1995-03-13 00:00:00.000000) │ └─TableScan_51 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false └─IndexLookUp_28 162945114.27 root - ├─IndexScan_25 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [ eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey) ], keep order:false + ├─IndexScan_25 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false └─Selection_27 162945114.27 cop gt(tpch.lineitem.l_shipdate, 1995-03-13 00:00:00.000000) └─TableScan_26 1.00 cop table:lineitem, keep order:false /* @@ -302,7 +302,7 @@ Sort_10 1.00 root tpch.orders.o_orderpriority:asc │ └─Selection_33 2925937.50 cop ge(tpch.orders.o_orderdate, 1995-01-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1995-04-01) │ └─TableScan_32 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false └─IndexLookUp_20 240004648.80 root - ├─IndexScan_17 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [ eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey) ], keep order:false + ├─IndexScan_17 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false └─Selection_19 240004648.80 cop lt(tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate) └─TableScan_18 1.00 cop table:lineitem, keep order:false /* @@ -538,7 +538,7 @@ Sort_29 718.01 root all_nations.o_year:asc │ │ │ │ └─Selection_77 22382008.93 cop ge(tpch.orders.o_orderdate, 1995-01-01 00:00:00.000000), le(tpch.orders.o_orderdate, 1996-12-31 00:00:00.000000) │ │ │ │ └─TableScan_76 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false │ │ │ └─IndexLookUp_55 1.00 root - │ │ │ ├─IndexScan_53 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [ eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey) ], keep order:false + │ │ │ ├─IndexScan_53 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false │ │ │ └─TableScan_54 1.00 cop table:lineitem, keep order:false │ │ └─TableReader_83 61674.00 root data:Selection_82 │ │ └─Selection_82 61674.00 cop eq(tpch.part.p_type, "SMALL PLATED COPPER") @@ -612,7 +612,7 @@ Sort_25 2406.00 root profit.nation:asc, profit.o_year:desc │ └─TableReader_40 1.00 root data:TableScan_39 │ └─TableScan_39 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false └─IndexLookUp_34 1.00 root - ├─IndexScan_32 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [ eq(tpch.partsupp.ps_partkey, tpch.lineitem.l_partkey) eq(tpch.partsupp.ps_suppkey, tpch.lineitem.l_suppkey) ], keep order:false + ├─IndexScan_32 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [eq(tpch.partsupp.ps_partkey, tpch.lineitem.l_partkey) eq(tpch.partsupp.ps_suppkey, tpch.lineitem.l_suppkey)], keep order:false └─TableScan_33 1.00 cop table:partsupp, keep order:false /* Q10 Returned Item Reporting Query @@ -673,7 +673,7 @@ Projection_17 20.00 root tpch.customer.c_custkey, tpch.customer.c_name, 9_col_0, │ └─Selection_47 3017307.69 cop ge(tpch.orders.o_orderdate, 1993-08-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1993-11-01) │ └─TableScan_46 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false └─IndexLookUp_31 73916005.00 root - ├─IndexScan_28 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [ eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey) ], keep order:false + ├─IndexScan_28 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false └─Selection_30 73916005.00 cop eq(tpch.lineitem.l_returnflag, "R") └─TableScan_29 1.00 cop table:lineitem, keep order:false /* @@ -936,7 +936,7 @@ Sort_13 3863988.24 root supplier_cnt:desc, tpch.part.p_brand:asc, tpch.part.p_ty │ │ └─Selection_40 1200618.43 cop in(tpch.part.p_size, 48, 19, 12, 4, 41, 7, 21, 39), ne(tpch.part.p_brand, "Brand#34"), not(like(tpch.part.p_type, "LARGE BRUSHED%", 92)) │ │ └─TableScan_39 10000000.00 cop table:part, range:[-inf,+inf], keep order:false │ └─IndexReader_26 1.00 root index:IndexScan_25 - │ └─IndexScan_25 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [ eq(tpch.partsupp.ps_partkey, tpch.part.p_partkey) ], keep order:false + │ └─IndexScan_25 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [eq(tpch.partsupp.ps_partkey, tpch.part.p_partkey)], keep order:false └─TableReader_46 400000.00 root data:Selection_45 └─Selection_45 400000.00 cop like(tpch.supplier.s_comment, "%Customer%Complaints%", 92) └─TableScan_44 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false @@ -1042,7 +1042,7 @@ Projection_24 100.00 root tpch.customer.c_name, tpch.customer.c_custkey, tpch.or │ └─HashAgg_53 74063872.00 cop group by:tpch.lineitem.l_orderkey, funcs:sum(tpch.lineitem.l_quantity), firstrow(tpch.lineitem.l_orderkey) │ └─TableScan_58 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false └─IndexLookUp_37 1.00 root - ├─IndexScan_35 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [ eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey) ], keep order:false + ├─IndexScan_35 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false └─TableScan_36 1.00 cop table:lineitem, keep order:false /* Q19 Discounted Revenue Query @@ -1165,7 +1165,7 @@ Sort_28 20000.00 root tpch.supplier.s_name:asc │ │ └─Selection_75 80007.93 cop like(tpch.part.p_name, "green%", 92) │ │ └─TableScan_74 10000000.00 cop table:part, range:[-inf,+inf], keep order:false │ └─IndexLookUp_58 1.00 root - │ ├─IndexScan_56 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [ eq(tpch.partsupp.ps_partkey, tpch.part.p_partkey) ], keep order:false + │ ├─IndexScan_56 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [eq(tpch.partsupp.ps_partkey, tpch.part.p_partkey)], keep order:false │ └─TableScan_57 1.00 cop table:partsupp, keep order:false └─TableReader_81 44189356.65 root data:Selection_80 └─Selection_80 44189356.65 cop ge(tpch.lineitem.l_shipdate, 1993-01-01 00:00:00.000000), lt(tpch.lineitem.l_shipdate, 1994-01-01) @@ -1239,10 +1239,10 @@ Projection_25 1.00 root tpch.supplier.s_name, 17_col_0 │ │ └─Selection_60 36517371.00 cop eq(tpch.orders.o_orderstatus, "F") │ │ └─TableScan_59 1.00 cop table:orders, range: decided by [l1.l_orderkey], keep order:false │ └─IndexLookUp_55 1.00 root - │ ├─IndexScan_53 1.00 cop table:l2, index:L_ORDERKEY, L_LINENUMBER, range: decided by [ eq(l2.l_orderkey, l1.l_orderkey) ], keep order:false + │ ├─IndexScan_53 1.00 cop table:l2, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(l2.l_orderkey, l1.l_orderkey)], keep order:false │ └─TableScan_54 1.00 cop table:lineitem, keep order:false └─IndexLookUp_39 240004648.80 root - ├─IndexScan_36 1.00 cop table:l3, index:L_ORDERKEY, L_LINENUMBER, range: decided by [ eq(l3.l_orderkey, l1.l_orderkey) ], keep order:false + ├─IndexScan_36 1.00 cop table:l3, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(l3.l_orderkey, l1.l_orderkey)], keep order:false └─Selection_38 240004648.80 cop gt(l3.l_receiptdate, l3.l_commitdate) └─TableScan_37 1.00 cop table:lineitem, keep order:false /* diff --git a/executor/index_lookup_join_test.go b/executor/index_lookup_join_test.go index 3210123b61913..f9e76f253e94d 100644 --- a/executor/index_lookup_join_test.go +++ b/executor/index_lookup_join_test.go @@ -70,7 +70,7 @@ func (s *testSuite1) TestIndexJoinUnionScan(c *C) { "└─UnionScan_12 0.00 root not(isnull(test.t2.a))", " └─IndexLookUp_11 0.00 root ", " ├─Selection_10 0.00 cop not(isnull(test.t2.a))", - " │ └─IndexScan_8 10.00 cop table:t2, index:a, range: decided by [ eq(test.t2.a, test.t1.a) ], keep order:false, stats:pseudo", + " │ └─IndexScan_8 10.00 cop table:t2, index:a, range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", " └─TableScan_9 0.00 cop table:t2, keep order:false", )) tk.MustQuery("select /*+ TIDB_INLJ(t1, t2)*/ * from t1 join t2 on t1.a = t2.a").Check(testkit.Rows( @@ -88,7 +88,7 @@ func (s *testSuite1) TestIndexJoinUnionScan(c *C) { " └─UnionScan_11 0.00 root not(isnull(test.t2.a))", " └─IndexReader_10 0.00 root index:Selection_9", " └─Selection_9 0.00 cop not(isnull(test.t2.a))", - " └─IndexScan_8 10.00 cop table:t2, index:a, range: decided by [ eq(test.t2.a, test.t1.a) ], keep order:false, stats:pseudo", + " └─IndexScan_8 10.00 cop table:t2, index:a, range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", )) tk.MustQuery("select /*+ TIDB_INLJ(t1, t2)*/ t1.a, t2.a from t1 join t2 on t1.a = t2.a").Check(testkit.Rows( "2 2", @@ -117,7 +117,7 @@ func (s *testSuite1) TestBatchIndexJoinUnionScan(c *C) { " └─UnionScan_26 0.00 root not(isnull(test.t2.a))", " └─IndexReader_25 0.00 root index:Selection_24", " └─Selection_24 0.00 cop not(isnull(test.t2.a))", - " └─IndexScan_23 10.00 cop table:t2, index:a, range: decided by [ eq(test.t2.a, test.t1.a) ], keep order:false, stats:pseudo", + " └─IndexScan_23 10.00 cop table:t2, index:a, range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", )) tk.MustQuery("select /*+ TIDB_INLJ(t1, t2)*/ count(*) from t1 join t2 on t1.a = t2.id").Check(testkit.Rows( "4", diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index aae8ee27b9d9b..7d89faa0a6e31 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -430,7 +430,6 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou err := helper.analyzeLookUpFilters(indexInfo, ds, innerJoinKeys) if err != nil { log.Warnf("[planner]: error happened when build index join: %v", err) - continue } } if helper.chosenIndexInfo != nil { @@ -470,16 +469,23 @@ type indexJoinBuildHelper struct { func (ijHelper *indexJoinBuildHelper) buildRangeDecidedByInformation(idxCols []*expression.Column, outerJoinKeys []*expression.Column) string { buffer := bytes.NewBufferString("[") + isFirst := true for idxOff, keyOff := range ijHelper.idxOff2KeyOff { if keyOff == -1 { continue } - buffer.WriteString(fmt.Sprintf(" eq(%v, %v)", idxCols[idxOff], outerJoinKeys[keyOff])) + if !isFirst { + buffer.WriteString(" ") + } else { + isFirst = false + } + buffer.WriteString(fmt.Sprintf("eq(%v, %v)", idxCols[idxOff], outerJoinKeys[keyOff])) } for _, access := range ijHelper.chosenAccess { + // Since now there must be eq/in condition so here we can just append space directly. buffer.WriteString(fmt.Sprintf(" %v", access)) } - buffer.WriteString(" ]") + buffer.WriteString("]") return buffer.String() } @@ -606,7 +612,7 @@ func (cwc *ColWithCompareOps) appendNewExpr(opName string, arg expression.Expres } } -// CompareRow sorts the row for deduplicate. +// CompareRow compares the rows for deduplicate. func (cwc *ColWithCompareOps) CompareRow(lhs, rhs chunk.Row) int { for i, col := range cwc.affectedColSchema.Columns { ret := cwc.compareFuncs[i](lhs, col.Index, rhs, col.Index) From 6af174d92c68aaef39bec60ea2f06454f7b13350 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 4 Mar 2019 12:50:28 +0800 Subject: [PATCH 14/27] address comments --- executor/builder.go | 10 +++++----- executor/index_lookup_join.go | 5 +++-- planner/core/exhaust_physical_plans.go | 6 +++--- planner/core/exhaust_physical_plans_test.go | 2 +- 4 files changed, 12 insertions(+), 11 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 401a70e67d983..3736ae7d4f794 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1538,11 +1538,11 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plannercore.PhysicalIndexJoin) readerBuilder: &dataReaderBuilder{innerPlan, b}, rowTypes: innerTypes, }, - workerWg: new(sync.WaitGroup), - joiner: newJoiner(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), - indexRanges: v.Ranges, - keyOff2IdxOff: v.KeyOff2IdxOff, - nextColCompareFilters: v.CompareFilters, + workerWg: new(sync.WaitGroup), + joiner: newJoiner(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), + indexRanges: v.Ranges, + keyOff2IdxOff: v.KeyOff2IdxOff, + lastColhelper: v.CompareFilters, } outerKeyCols := make([]int, len(v.OuterJoinKeys)) for i := 0; i < len(v.OuterJoinKeys); i++ { diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 8188e74817c7a..04d2162339092 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -68,7 +68,8 @@ type IndexLookUpJoin struct { keyOff2IdxOff []int innerPtrBytes [][]byte - nextColCompareFilters *plannercore.ColWithCompareOps + // lastColhelper store the information for last col if there's complicated filter like col > x_col and col < x_col + 100. + lastColhelper *plannercore.ColWithCompareOps memTracker *memory.Tracker // track memory usage. } @@ -211,7 +212,7 @@ func (e *IndexLookUpJoin) newInnerWorker(taskCh chan *lookUpJoinTask) *innerWork executorChk: chunk.NewChunkWithCapacity(e.innerCtx.rowTypes, e.maxChunkSize), indexRanges: copiedRanges, keyOff2IdxOff: e.keyOff2IdxOff, - nextColCompareFilters: e.nextColCompareFilters, + nextColCompareFilters: e.lastColhelper, } return iw } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 7d89faa0a6e31..f40d12bffc456 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -666,7 +666,8 @@ func (cwc *ColWithCompareOps) String() string { return buffer.String() } -func (ijHelper *indexJoinBuildHelper) checkIndex(tmpSchema *expression.Schema, idxCols []*expression.Column, colLens []int) { +func (ijHelper *indexJoinBuildHelper) checkIndex(innerKeys []*expression.Column, idxCols []*expression.Column, colLens []int) { + tmpSchema := expression.NewSchema(innerKeys...) ijHelper.curIdxOff2KeyOff = make([]int, len(idxCols)) ijHelper.curPossibleUsedKeys = make([]*expression.Column, 0, len(idxCols)) ijHelper.curNotUsedIndexCols = make([]*expression.Column, 0, len(idxCols)) @@ -750,8 +751,7 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(indexInfo *model.Inde return nil } accesses := make([]expression.Expression, 0, len(idxCols)) - tmpSchema := expression.NewSchema(innerJoinKeys...) - ijHelper.checkIndex(tmpSchema, idxCols, colLengths) + ijHelper.checkIndex(innerJoinKeys, idxCols, colLengths) matchedKeyCnt := len(ijHelper.curPossibleUsedKeys) // If no index column appears in join key, we just break. // TODO: It may meet this case: There's no join key condition, but have compare filters. diff --git a/planner/core/exhaust_physical_plans_test.go b/planner/core/exhaust_physical_plans_test.go index 36212cadc74da..6ace57a2bcb41 100644 --- a/planner/core/exhaust_physical_plans_test.go +++ b/planner/core/exhaust_physical_plans_test.go @@ -1,4 +1,4 @@ -// Copyright 2017 PingCAP, Inc. +// Copyright 2018 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. From 209f4f36d93663e5bf281289c86387e477f98452 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 4 Mar 2019 12:59:19 +0800 Subject: [PATCH 15/27] address comments --- planner/core/exhaust_physical_plans.go | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index f40d12bffc456..fea24f03c84be 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -324,10 +324,21 @@ func joinKeysMatchIndex(keys, indexCols []*expression.Column, colLengths []int) // When inner plan is TableReader, the parameter `ranges` will be nil. Because pk only have one column. So all of its range // is generated during execution time. -func (p *LogicalJoin) constructIndexJoin(prop *property.PhysicalProperty, innerJoinKeys, outerJoinKeys []*expression.Column, outerIdx int, - innerPlan PhysicalPlan, ranges []*ranger.Range, keyOff2IdxOff []int, compareFilters *ColWithCompareOps) []PhysicalPlan { +func (p *LogicalJoin) constructIndexJoin(prop *property.PhysicalProperty, outerIdx int, innerPlan PhysicalPlan, + ranges []*ranger.Range, keyOff2IdxOff []int, compareFilters *ColWithCompareOps) []PhysicalPlan { joinType := p.JoinType outerSchema := p.children[outerIdx].Schema() + var ( + innerJoinKeys []*expression.Column + outerJoinKeys []*expression.Column + ) + if outerIdx == 0 { + outerJoinKeys = p.LeftJoinKeys + innerJoinKeys = p.RightJoinKeys + } else { + innerJoinKeys = p.LeftJoinKeys + outerJoinKeys = p.RightJoinKeys + } all, _ := prop.AllSameOrder() // If the order by columns are not all from outer child, index join cannot promise the order. if !prop.AllColsFromSchema(outerSchema) || !all { @@ -418,7 +429,7 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou innerPlan := p.constructInnerTableScan(ds, pkCol, outerJoinKeys, us) // Since the primary key means one value corresponding to exact one row, this will always be a no worse one // comparing to other index. - return p.constructIndexJoin(prop, innerJoinKeys, outerJoinKeys, outerIdx, innerPlan, nil, keyOff2IdxOff, nil) + return p.constructIndexJoin(prop, outerIdx, innerPlan, nil, keyOff2IdxOff, nil) } } helper := &indexJoinBuildHelper{join: p} @@ -445,7 +456,7 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou idxCols, _ := expression.IndexInfo2Cols(ds.schema.Columns, helper.chosenIndexInfo) rangeInfo := helper.buildRangeDecidedByInformation(idxCols, outerJoinKeys) innerPlan := p.constructInnerIndexScan(ds, helper.chosenIndexInfo, helper.chosenRemained, outerJoinKeys, us, rangeInfo) - return p.constructIndexJoin(prop, innerJoinKeys, outerJoinKeys, outerIdx, innerPlan, helper.chosenRanges, keyOff2IdxOff, helper.lastColManager) + return p.constructIndexJoin(prop, outerIdx, innerPlan, helper.chosenRanges, keyOff2IdxOff, helper.lastColManager) } return nil } From d738794fecd9023a7fd6c178bbfe09b112e37611 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 6 Mar 2019 16:32:50 +0800 Subject: [PATCH 16/27] address comments --- planner/core/exhaust_physical_plans.go | 39 +++++++++++++------------- planner/core/explain.go | 4 +-- planner/core/find_best_task.go | 1 - planner/core/physical_plans.go | 3 +- 4 files changed, 22 insertions(+), 25 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index b9f9701df06f8..ed76496aa80aa 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -532,7 +532,7 @@ func (p *LogicalJoin) constructInnerUnionScan(us *LogicalUnionScan, reader Physi } // constructInnerIndexScan is specially used to construct the inner plan for PhysicalIndexJoin. -func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexInfo, remainedConds []expression.Expression, +func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexInfo, filterConds []expression.Expression, outerJoinKeys []*expression.Column, us *LogicalUnionScan, rangeInfo string) PhysicalPlan { is := PhysicalIndexScan{ Table: ds.tableInfo, @@ -543,9 +543,8 @@ func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexIn dataSourceSchema: ds.schema, KeepOrder: false, Ranges: ranger.FullRange(), - rangeDecidedBy: rangeInfo, + rangeInfo: rangeInfo, }.Init(ds.ctx) - is.filterCondition = remainedConds var rowCount float64 idxHist, ok := ds.statisticTable.Indices[idx.ID] @@ -568,7 +567,7 @@ func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexIn } is.initSchema(ds.id, idx, cop.tablePlan != nil) - indexConds, tblConds := splitIndexFilterConditions(remainedConds, idx.Columns, ds.tableInfo) + indexConds, tblConds := splitIndexFilterConditions(filterConds, idx.Columns, ds.tableInfo) path := &accessPath{indexFilters: indexConds, tableFilters: tblConds, countAfterIndex: math.MaxFloat64} is.addPushedDownSelection(cop, ds, math.MaxFloat64, path) t := finishCopTask(ds.ctx, cop) @@ -679,28 +678,28 @@ func (ijHelper *indexJoinBuildHelper) checkIndex(innerKeys []*expression.Column, } } -func (ijHelper *indexJoinBuildHelper) analyzeSimpleFilters(innerPlan *DataSource, keyMatchedLen, matchedKeyCnt int) ([]expression.Expression, []expression.Expression, []expression.Expression) { - remained := make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) +// findUsefulEqAndInFilters analyzes the pushedDownConds held by inner child and split them to three parts. +// first part is the continuous eq/in conditions on current unused index columns. +// second part is the conditions which cannot be used for building ranges. +// third part is the other conditions for future use. +func (ijHelper *indexJoinBuildHelper) findUsefulEqAndInFilters(innerPlan *DataSource) ([]expression.Expression, []expression.Expression, []expression.Expression) { + uselessFilters := make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) rangeFilterCandidates := make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) - // This loop deal first filter out the expressions that contains columns not in index. + // This loop finds the possible filters which can be used to build ranges. + // If the filter contains index column covered by join keys, it will be useless since we always use join key to build range for that index column.. for _, innerFilter := range innerPlan.pushedDownConds { affectedCols := expression.ExtractColumns(innerFilter) if expression.ColumnSliceIsIntersect(affectedCols, ijHelper.curPossibleUsedKeys) { - remained = append(remained, innerFilter) + uselessFilters = append(uselessFilters, innerFilter) continue } rangeFilterCandidates = append(rangeFilterCandidates, innerFilter) } - // Extract the eq/in functions of possible join key. This returned list keeps the same order with index column. - notKeyEqAndIn, remainedEqAndIn, rangeFilterCandidates, _ := ranger.ExtractEqAndInCondition(innerPlan.ctx, rangeFilterCandidates, ijHelper.curNotUsedIndexCols, ijHelper.curNotUsedColLens) - // We hope that the index cols appeared in the join keys can all be used to build range. If it cannot be satisfied, - // we'll mark this index as cannot be used for index join. - // So we should make sure that all columns before the keyMatchedLen is join key or has eq/in function. - if len(notKeyEqAndIn) < keyMatchedLen-matchedKeyCnt { - return nil, nil, nil - } - remained = append(remained, remainedEqAndIn...) - return notKeyEqAndIn, remained, rangeFilterCandidates + // Extract the eq/in functions of possible join key. + // you can see the comment of ExtractEqAndInCondition to get the meaning of the second return value. + eqOrInOnUnusedIdxCols, remainedEqOrIn, rangeFilterCandidates, _ := ranger.ExtractEqAndInCondition(innerPlan.ctx, rangeFilterCandidates, ijHelper.curNotUsedIndexCols, ijHelper.curNotUsedColLens) + uselessFilters = append(uselessFilters, remainedEqOrIn...) + return eqOrInOnUnusedIdxCols, uselessFilters, rangeFilterCandidates } func (ijHelper *indexJoinBuildHelper) analyzeOtherFilters(nextCol *expression.Column, @@ -763,7 +762,7 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(indexInfo *model.Inde break } } - notKeyEqAndIn, remained, rangeFilterCandidates := ijHelper.analyzeSimpleFilters(innerPlan, keyMatchedLen, matchedKeyCnt) + notKeyEqAndIn, remained, rangeFilterCandidates := ijHelper.findUsefulEqAndInFilters(innerPlan) accesses = append(accesses, notKeyEqAndIn...) // We hope that the index cols appeared in the join keys can all be used to build range. If it cannot be satisfied, // we'll mark this index as cannot be used for index join. @@ -772,7 +771,7 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(indexInfo *model.Inde return nil } lastColPos := matchedKeyCnt + len(notKeyEqAndIn) - // If all cols have been considered, we can return the current result. + // If all the idnex columns are covered by eq/in conditions, we don't need to consider other conditions anymore if lastColPos == len(idxCols) { remained = append(remained, rangeFilterCandidates...) ranges, err := ijHelper.buildTemplateRange(matchedKeyCnt, notKeyEqAndIn, nil, false) diff --git a/planner/core/explain.go b/planner/core/explain.go index 80bfe1e6d2735..9e4ee48ce9b25 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -57,8 +57,8 @@ func (p *PhysicalIndexScan) ExplainInfo() string { break } } - if len(p.rangeDecidedBy) > 0 { - fmt.Fprintf(buffer, ", range: decided by %v", p.rangeDecidedBy) + if len(p.rangeInfo) > 0 { + fmt.Fprintf(buffer, ", range: decided by %v", p.rangeInfo) } else if haveCorCol { fmt.Fprintf(buffer, ", range: decided by %v", p.AccessCondition) } else if len(p.Ranges) > 0 { diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index a471d806f3bad..cfd11fc4a43b4 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -461,7 +461,6 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, candid IdxColLens: path.idxColLens, AccessCondition: path.accessConds, Ranges: path.ranges, - filterCondition: path.indexFilters, dataSourceSchema: ds.schema, isPartition: ds.isPartition, physicalTableID: ds.physicalTableID, diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index fc0503c476377..32cef06e882e9 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -90,7 +90,6 @@ type PhysicalIndexScan struct { // AccessCondition is used to calculate range. AccessCondition []expression.Expression - filterCondition []expression.Expression Table *model.TableInfo Index *model.IndexInfo @@ -115,7 +114,7 @@ type PhysicalIndexScan struct { // It is used for query feedback. Hist *statistics.Histogram - rangeDecidedBy string + rangeInfo string // The index scan may be on a partition. isPartition bool From 96ac2e726ada88d9f5c9fe8131ca1a3d0cadc2a3 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 7 Mar 2019 20:38:49 +0800 Subject: [PATCH 17/27] address comments --- executor/builder.go | 12 ++-- executor/index_lookup_join.go | 8 +-- planner/core/exhaust_physical_plans.go | 92 ++++++++++++++------------ planner/core/physical_plans.go | 2 +- 4 files changed, 61 insertions(+), 53 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 27323b8334d33..fbb4aefbd555d 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1542,7 +1542,7 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plannercore.PhysicalIndexJoin) joiner: newJoiner(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), indexRanges: v.Ranges, keyOff2IdxOff: v.KeyOff2IdxOff, - lastColhelper: v.CompareFilters, + lastColHelper: v.CompareFilters, } outerKeyCols := make([]int, len(v.OuterJoinKeys)) for i := 0; i < len(v.OuterJoinKeys); i++ { @@ -1780,7 +1780,7 @@ type dataReaderBuilder struct { } func (builder *dataReaderBuilder) buildExecutorForIndexJoin(ctx context.Context, lookUpContents []*indexJoinLookUpContent, - IndexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCompareOps) (Executor, error) { + IndexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) { switch v := builder.Plan.(type) { case *plannercore.PhysicalTableReader: return builder.buildTableReaderForIndexJoin(ctx, v, lookUpContents) @@ -1795,7 +1795,7 @@ func (builder *dataReaderBuilder) buildExecutorForIndexJoin(ctx context.Context, } func (builder *dataReaderBuilder) buildUnionScanForIndexJoin(ctx context.Context, v *plannercore.PhysicalUnionScan, - values []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCompareOps) (Executor, error) { + values []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) { childBuilder := &dataReaderBuilder{v.Children()[0], builder.executorBuilder} reader, err := childBuilder.buildExecutorForIndexJoin(ctx, values, indexRanges, keyOff2IdxOff, cwc) if err != nil { @@ -1851,7 +1851,7 @@ func (builder *dataReaderBuilder) buildTableReaderFromHandles(ctx context.Contex } func (builder *dataReaderBuilder) buildIndexReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalIndexReader, - lookUpContents []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCompareOps) (Executor, error) { + lookUpContents []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) { e, err := buildNoRangeIndexReader(builder.executorBuilder, v) if err != nil { return nil, errors.Trace(err) @@ -1865,7 +1865,7 @@ func (builder *dataReaderBuilder) buildIndexReaderForIndexJoin(ctx context.Conte } func (builder *dataReaderBuilder) buildIndexLookUpReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalIndexLookUpReader, - lookUpContents []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCompareOps) (Executor, error) { + lookUpContents []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) { e, err := buildNoRangeIndexLookUpReader(builder.executorBuilder, v) if err != nil { return nil, errors.Trace(err) @@ -1880,7 +1880,7 @@ func (builder *dataReaderBuilder) buildIndexLookUpReaderForIndexJoin(ctx context // buildKvRangesForIndexJoin builds kv ranges for index join when the inner plan is index scan plan. func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, lookUpContents []*indexJoinLookUpContent, - ranges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCompareOps) ([]kv.KeyRange, error) { + ranges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) ([]kv.KeyRange, error) { kvRanges := make([]kv.KeyRange, 0, len(ranges)*len(lookUpContents)) lastPos := len(ranges[0].LowVal) - 1 sc := ctx.GetSessionVars().StmtCtx diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 04d2162339092..a7cfc431876fe 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -68,8 +68,8 @@ type IndexLookUpJoin struct { keyOff2IdxOff []int innerPtrBytes [][]byte - // lastColhelper store the information for last col if there's complicated filter like col > x_col and col < x_col + 100. - lastColhelper *plannercore.ColWithCompareOps + // lastColHelper store the information for last col if there's complicated filter like col > x_col and col < x_col + 100. + lastColHelper *plannercore.ColWithCmpFuncManager memTracker *memory.Tracker // track memory usage. } @@ -129,7 +129,7 @@ type innerWorker struct { executorChk *chunk.Chunk indexRanges []*ranger.Range - nextColCompareFilters *plannercore.ColWithCompareOps + nextColCompareFilters *plannercore.ColWithCmpFuncManager keyOff2IdxOff []int } @@ -212,7 +212,7 @@ func (e *IndexLookUpJoin) newInnerWorker(taskCh chan *lookUpJoinTask) *innerWork executorChk: chunk.NewChunkWithCapacity(e.innerCtx.rowTypes, e.maxChunkSize), indexRanges: copiedRanges, keyOff2IdxOff: e.keyOff2IdxOff, - nextColCompareFilters: e.lastColhelper, + nextColCompareFilters: e.lastColHelper, } return iw } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index ed76496aa80aa..8098d2a7071f7 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -310,7 +310,7 @@ func joinKeysMatchIndex(keys, indexCols []*expression.Column, colLengths []int) // When inner plan is TableReader, the parameter `ranges` will be nil. Because pk only have one column. So all of its range // is generated during execution time. func (p *LogicalJoin) constructIndexJoin(prop *property.PhysicalProperty, outerIdx int, innerPlan PhysicalPlan, - ranges []*ranger.Range, keyOff2IdxOff []int, compareFilters *ColWithCompareOps) []PhysicalPlan { + ranges []*ranger.Range, keyOff2IdxOff []int, compareFilters *ColWithCmpFuncManager) []PhysicalPlan { joinType := p.JoinType outerSchema := p.children[outerIdx].Schema() var ( @@ -454,7 +454,7 @@ type indexJoinBuildHelper struct { chosenAccess []expression.Expression chosenRemained []expression.Expression idxOff2KeyOff []int - lastColManager *ColWithCompareOps + lastColManager *ColWithCmpFuncManager chosenRanges []*ranger.Range curPossibleUsedKeys []*expression.Column @@ -582,9 +582,9 @@ var symmetricOp = map[string]string{ ast.LE: ast.GE, } -// ColWithCompareOps is used in index join to handle the column with compare functions(>=, >, <, <=). +// ColWithCmpFuncManager is used in index join to handle the column with compare functions(>=, >, <, <=). // It stores the compare functions and build ranges in execution phase. -type ColWithCompareOps struct { +type ColWithCmpFuncManager struct { targetCol *expression.Column colLength int OpType []string @@ -594,7 +594,7 @@ type ColWithCompareOps struct { compareFuncs []chunk.CompareFunc } -func (cwc *ColWithCompareOps) appendNewExpr(opName string, arg expression.Expression, affectedCols []*expression.Column) { +func (cwc *ColWithCmpFuncManager) appendNewExpr(opName string, arg expression.Expression, affectedCols []*expression.Column) { cwc.OpType = append(cwc.OpType, opName) cwc.opArg = append(cwc.opArg, arg) cwc.tmpConstant = append(cwc.tmpConstant, &expression.Constant{RetType: cwc.targetCol.RetType}) @@ -608,7 +608,7 @@ func (cwc *ColWithCompareOps) appendNewExpr(opName string, arg expression.Expres } // CompareRow compares the rows for deduplicate. -func (cwc *ColWithCompareOps) CompareRow(lhs, rhs chunk.Row) int { +func (cwc *ColWithCmpFuncManager) CompareRow(lhs, rhs chunk.Row) int { for i, col := range cwc.affectedColSchema.Columns { ret := cwc.compareFuncs[i](lhs, col.Index, rhs, col.Index) if ret != 0 { @@ -619,7 +619,7 @@ func (cwc *ColWithCompareOps) CompareRow(lhs, rhs chunk.Row) int { } // BuildRangesByRow will build range of the given row. It will eval each function's arg then call BuildRange. -func (cwc *ColWithCompareOps) BuildRangesByRow(ctx sessionctx.Context, row chunk.Row) ([]*ranger.Range, error) { +func (cwc *ColWithCmpFuncManager) BuildRangesByRow(ctx sessionctx.Context, row chunk.Row) ([]*ranger.Range, error) { exprs := make([]expression.Expression, len(cwc.OpType)) for i, opType := range cwc.OpType { constantArg, err := cwc.opArg[i].Eval(row) @@ -640,7 +640,7 @@ func (cwc *ColWithCompareOps) BuildRangesByRow(ctx sessionctx.Context, row chunk return ranges, nil } -func (cwc *ColWithCompareOps) resolveIndices(schema *expression.Schema) (err error) { +func (cwc *ColWithCmpFuncManager) resolveIndices(schema *expression.Schema) (err error) { for i := range cwc.opArg { cwc.opArg[i], err = cwc.opArg[i].ResolveIndices(schema) if err != nil { @@ -650,7 +650,8 @@ func (cwc *ColWithCompareOps) resolveIndices(schema *expression.Schema) (err err return nil } -func (cwc *ColWithCompareOps) String() string { +// String implements Stringer interface. +func (cwc *ColWithCmpFuncManager) String() string { buffer := bytes.NewBufferString("") for i := range cwc.OpType { buffer.WriteString(fmt.Sprintf("%v(%v, %v)", cwc.OpType[i], cwc.targetCol, cwc.opArg[i])) @@ -679,12 +680,12 @@ func (ijHelper *indexJoinBuildHelper) checkIndex(innerKeys []*expression.Column, } // findUsefulEqAndInFilters analyzes the pushedDownConds held by inner child and split them to three parts. -// first part is the continuous eq/in conditions on current unused index columns. -// second part is the conditions which cannot be used for building ranges. -// third part is the other conditions for future use. -func (ijHelper *indexJoinBuildHelper) findUsefulEqAndInFilters(innerPlan *DataSource) ([]expression.Expression, []expression.Expression, []expression.Expression) { - uselessFilters := make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) - rangeFilterCandidates := make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) +// usefulEqOrInFilters is the continuous eq/in conditions on current unused index columns. +// uselessFilters is the conditions which cannot be used for building ranges. +// remainingRangeCandidates is the other conditions for future use. +func (ijHelper *indexJoinBuildHelper) findUsefulEqAndInFilters(innerPlan *DataSource) (usefulEqOrInFilters, uselessFilters, remainingRangeCandidates []expression.Expression) { + uselessFilters = make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) + remainingRangeCandidates = make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) // This loop finds the possible filters which can be used to build ranges. // If the filter contains index column covered by join keys, it will be useless since we always use join key to build range for that index column.. for _, innerFilter := range innerPlan.pushedDownConds { @@ -693,17 +694,24 @@ func (ijHelper *indexJoinBuildHelper) findUsefulEqAndInFilters(innerPlan *DataSo uselessFilters = append(uselessFilters, innerFilter) continue } - rangeFilterCandidates = append(rangeFilterCandidates, innerFilter) + remainingRangeCandidates = append(remainingRangeCandidates, innerFilter) } + var remainedEqOrIn []expression.Expression // Extract the eq/in functions of possible join key. // you can see the comment of ExtractEqAndInCondition to get the meaning of the second return value. - eqOrInOnUnusedIdxCols, remainedEqOrIn, rangeFilterCandidates, _ := ranger.ExtractEqAndInCondition(innerPlan.ctx, rangeFilterCandidates, ijHelper.curNotUsedIndexCols, ijHelper.curNotUsedColLens) + usefulEqOrInFilters, remainedEqOrIn, remainingRangeCandidates, _ = ranger.ExtractEqAndInCondition( + innerPlan.ctx, remainingRangeCandidates, + ijHelper.curNotUsedIndexCols, + ijHelper.curNotUsedColLens, + ) uselessFilters = append(uselessFilters, remainedEqOrIn...) - return eqOrInOnUnusedIdxCols, uselessFilters, rangeFilterCandidates + return usefulEqOrInFilters, uselessFilters, remainingRangeCandidates } +// analyzeOtherFilters analyze the `OtherConditions` of join to see whether there're some filters can be used in manager. +// The returned value is just for outputting explain information func (ijHelper *indexJoinBuildHelper) analyzeOtherFilters(nextCol *expression.Column, - innerPlan *DataSource, cwc *ColWithCompareOps) []expression.Expression { + innerPlan *DataSource, cwc *ColWithCmpFuncManager) []expression.Expression { var lastColAccesses []expression.Expression loopOtherConds: for _, filter := range ijHelper.join.OtherConditions { @@ -711,31 +719,30 @@ loopOtherConds: if !ok || !(sf.FuncName.L == ast.LE || sf.FuncName.L == ast.LT || sf.FuncName.L == ast.GE || sf.FuncName.L == ast.GT) { continue } + var funcName string + var anotherArg expression.Expression if lCol, ok := sf.GetArgs()[0].(*expression.Column); ok && lCol.Equal(nil, nextCol) { - affectedCols := expression.ExtractColumns(sf.GetArgs()[1]) - if len(affectedCols) == 0 { - continue - } - for _, col := range affectedCols { - if innerPlan.schema.Contains(col) { - continue loopOtherConds - } - } - lastColAccesses = append(lastColAccesses, sf) - cwc.appendNewExpr(sf.FuncName.L, sf.GetArgs()[1], affectedCols) + anotherArg = sf.GetArgs()[1] + funcName = sf.FuncName.L } else if rCol, ok := sf.GetArgs()[1].(*expression.Column); ok && rCol.Equal(nil, nextCol) { - affectedCols := expression.ExtractColumns(sf.GetArgs()[0]) - if len(affectedCols) == 0 { - continue - } - for _, col := range affectedCols { - if innerPlan.schema.Contains(col) { - continue loopOtherConds - } + anotherArg = sf.GetArgs()[0] + // The column manager always build expression in the form of col op arg1. + // So we need use the symmetric one of the current function. + funcName = symmetricOp[sf.FuncName.L] + } else { + continue + } + affectedCols := expression.ExtractColumns(anotherArg) + if len(affectedCols) == 0 { + continue + } + for _, col := range affectedCols { + if innerPlan.schema.Contains(col) { + continue loopOtherConds } - lastColAccesses = append(lastColAccesses, sf) - cwc.appendNewExpr(symmetricOp[sf.FuncName.L], sf.GetArgs()[0], affectedCols) } + lastColAccesses = append(lastColAccesses, sf) + cwc.appendNewExpr(funcName, anotherArg, affectedCols) } return lastColAccesses } @@ -782,12 +789,13 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(indexInfo *model.Inde return nil } lastPossibleCol := idxCols[lastColPos] - lastColManager := &ColWithCompareOps{ + lastColManager := &ColWithCmpFuncManager{ targetCol: lastPossibleCol, colLength: colLengths[lastColPos], affectedColSchema: expression.NewSchema(), } lastColAccess := ijHelper.analyzeOtherFilters(lastPossibleCol, innerPlan, lastColManager) + // If the column manager holds no expression, then we fallback to find whether there're useful normal filters if len(lastColAccess) == 0 { colAccesses, colRemained := ranger.DetachCondsForTableRange(ijHelper.join.ctx, rangeFilterCandidates, lastPossibleCol) var ranges, nextColRange []*ranger.Range @@ -821,7 +829,7 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(indexInfo *model.Inde } func (ijHelper *indexJoinBuildHelper) updateBestChoice(ranges []*ranger.Range, idxInfo *model.IndexInfo, accesses, - remained []expression.Expression, lastColManager *ColWithCompareOps) { + remained []expression.Expression, lastColManager *ColWithCmpFuncManager) { // We choose the index by the number of used columns of the range, the much the better. // Notice that there may be the cases like `t1.a=t2.a and b > 2 and b < 1`. So ranges can be nil though the conditions are valid. // But obviously when the range is nil, we don't need index join. diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 32cef06e882e9..c92e6c177b193 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -239,7 +239,7 @@ type PhysicalIndexJoin struct { // If there's index(t.a, t.b). All the filters can be used to construct index range but t.b > t1.b and t.b < t1.b=10 // need to be evaluated after we fetch the data of t1. // This struct stores them and evaluate them to ranges. - CompareFilters *ColWithCompareOps + CompareFilters *ColWithCmpFuncManager } // PhysicalMergeJoin represents merge join for inner/ outer join. From 12a1e603d2bb7c937322616ee9633259372b9978 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 7 Mar 2019 20:48:17 +0800 Subject: [PATCH 18/27] rename method --- planner/core/exhaust_physical_plans.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 8098d2a7071f7..6dfc67a118f1c 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -708,9 +708,9 @@ func (ijHelper *indexJoinBuildHelper) findUsefulEqAndInFilters(innerPlan *DataSo return usefulEqOrInFilters, uselessFilters, remainingRangeCandidates } -// analyzeOtherFilters analyze the `OtherConditions` of join to see whether there're some filters can be used in manager. +// buildLastColManager analyze the `OtherConditions` of join to see whether there're some filters can be used in manager. // The returned value is just for outputting explain information -func (ijHelper *indexJoinBuildHelper) analyzeOtherFilters(nextCol *expression.Column, +func (ijHelper *indexJoinBuildHelper) buildLastColManager(nextCol *expression.Column, innerPlan *DataSource, cwc *ColWithCmpFuncManager) []expression.Expression { var lastColAccesses []expression.Expression loopOtherConds: @@ -794,7 +794,7 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(indexInfo *model.Inde colLength: colLengths[lastColPos], affectedColSchema: expression.NewSchema(), } - lastColAccess := ijHelper.analyzeOtherFilters(lastPossibleCol, innerPlan, lastColManager) + lastColAccess := ijHelper.buildLastColManager(lastPossibleCol, innerPlan, lastColManager) // If the column manager holds no expression, then we fallback to find whether there're useful normal filters if len(lastColAccess) == 0 { colAccesses, colRemained := ranger.DetachCondsForTableRange(ijHelper.join.ctx, rangeFilterCandidates, lastPossibleCol) From 3dbf2227cd8b83b4949d977ea7862eb38d374a54 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 12 Mar 2019 17:06:05 +0800 Subject: [PATCH 19/27] index join can be chosen when there's no eq cond --- cmd/explaintest/r/index_join.result | 4 ++-- cmd/explaintest/r/topn_push_down.result | 4 ++-- planner/core/exhaust_physical_plans.go | 9 --------- 3 files changed, 4 insertions(+), 13 deletions(-) diff --git a/cmd/explaintest/r/index_join.result b/cmd/explaintest/r/index_join.result index 4f81db898cf7f..861b01bdfade7 100644 --- a/cmd/explaintest/r/index_join.result +++ b/cmd/explaintest/r/index_join.result @@ -9,7 +9,7 @@ id count task operator info IndexJoin_16 5.00 root inner join, inner:IndexLookUp_15, outer key:test.t2.a, inner key:test.t1.a ├─IndexLookUp_15 0.00 root │ ├─Selection_14 0.00 cop not(isnull(test.t1.a)) -│ │ └─IndexScan_12 5.00 cop table:t1, index:a, range: decided by [test.t2.a], keep order:false +│ │ └─IndexScan_12 5.00 cop table:t1, index:a, range: decided by [eq(test.t1.a, test.t2.a)], keep order:false │ └─TableScan_13 0.00 cop table:t1, keep order:false └─TableReader_19 1.00 root data:Selection_18 └─Selection_18 1.00 cop not(isnull(test.t2.a)) @@ -23,5 +23,5 @@ Projection_6 5.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b │ └─TableScan_30 1.00 cop table:t2, range:[-inf,+inf], keep order:false └─IndexLookUp_11 0.00 root ├─Selection_10 0.00 cop not(isnull(test.t1.a)) - │ └─IndexScan_8 5.00 cop table:t1, index:a, range: decided by [test.t2.a], keep order:false + │ └─IndexScan_8 5.00 cop table:t1, index:a, range: decided by [eq(test.t1.a, test.t2.a)], keep order:false └─TableScan_9 0.00 cop table:t1, keep order:false diff --git a/cmd/explaintest/r/topn_push_down.result b/cmd/explaintest/r/topn_push_down.result index 6d38ecf0a9c22..0bc8f51519b5d 100644 --- a/cmd/explaintest/r/topn_push_down.result +++ b/cmd/explaintest/r/topn_push_down.result @@ -227,7 +227,7 @@ Limit_11 5.00 root offset:0, count:5 ├─TableReader_17 4.00 root data:TableScan_16 │ └─TableScan_16 4.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─IndexReader_14 10.00 root index:IndexScan_13 - └─IndexScan_13 10.00 cop table:t2, index:a, range: decided by [t1.a], keep order:false, stats:pseudo + └─IndexScan_13 10.00 cop table:t2, index:a, range: decided by [eq(t2.a, t1.a)], keep order:false, stats:pseudo explain select /*+ TIDB_INLJ(t2) */ * from t t1 left join t t2 on t1.a = t2.a where t2.a is null limit 5; id count task operator info Limit_12 5.00 root offset:0, count:5 @@ -236,7 +236,7 @@ Limit_12 5.00 root offset:0, count:5 ├─TableReader_19 4.00 root data:TableScan_18 │ └─TableScan_18 4.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─IndexReader_16 10.00 root index:IndexScan_15 - └─IndexScan_15 10.00 cop table:t2, index:a, range: decided by [t1.a], keep order:false, stats:pseudo + └─IndexScan_15 10.00 cop table:t2, index:a, range: decided by [eq(t2.a, t1.a)], keep order:false, stats:pseudo explain select /*+ TIDB_SMJ(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a limit 5; id count task operator info Limit_11 5.00 root offset:0, count:5 diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index ed867fddff530..05c54dc2d8c38 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -945,21 +945,12 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ errMsg = fmt.Sprintf("Optimizer Hint %s is inapplicable", p.hintInfo.restore2IndexJoinHint()) } - // Append inapplicable reason. - if len(p.EqualConditions) == 0 { - errMsg += " without column equal ON condition" - } - // Generate warning message to client. warning := ErrInternal.GenWithStack(errMsg) p.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) } }() - if len(p.EqualConditions) == 0 { - return nil, false - } - switch p.JoinType { case SemiJoin, AntiSemiJoin, LeftOuterSemiJoin, AntiLeftOuterSemiJoin, LeftOuterJoin: join := p.getIndexJoinByOuterIdx(prop, 0) From 1667c2cf0c5cab4a81448b998062d4fea9a2e498 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 15 Apr 2019 19:59:49 +0800 Subject: [PATCH 20/27] typo --- planner/core/exhaust_physical_plans.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index cf61baaa4cd49..192da66bcd659 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -785,7 +785,7 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(indexInfo *model.Inde return nil } lastColPos := matchedKeyCnt + len(notKeyEqAndIn) - // If all the idnex columns are covered by eq/in conditions, we don't need to consider other conditions anymore + // If all the index columns are covered by eq/in conditions, we don't need to consider other conditions anymore. if lastColPos == len(idxCols) { remained = append(remained, rangeFilterCandidates...) ranges, err := ijHelper.buildTemplateRange(matchedKeyCnt, notKeyEqAndIn, nil, false) From c42ed2c91bdce63d7c289287c3d0ea3af2075abc Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 16 Apr 2019 15:02:19 +0800 Subject: [PATCH 21/27] update to be compatiable with issue#9562 --- planner/core/cbo_test.go | 2 +- planner/core/exhaust_physical_plans.go | 54 ++++++++++++++------- planner/core/exhaust_physical_plans_test.go | 14 +++--- 3 files changed, 45 insertions(+), 25 deletions(-) diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index caefa22d9f1eb..77ae0789d4b75 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -909,7 +909,7 @@ func (s *testAnalyzeSuite) TestIssue9562(c *C) { "│ └─TableScan_10 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", "└─IndexReader_8 0.00 root index:Selection_7", " └─Selection_7 0.00 cop not(isnull(test.t2.a)), not(isnull(test.t2.c))", - " └─IndexScan_6 10.00 cop table:t2, index:a, b, c, range: decided by [test.t1.a test.t1.c], keep order:false, stats:pseudo", + " └─IndexScan_6 10.00 cop table:t2, index:a, b, c, range: decided by [eq(test.t2.a, test.t1.a) gt(test.t2.b, minus(test.t1.b, 1)) lt(test.t2.b, plus(test.t1.b, 1))], keep order:false, stats:pseudo", )) tk.MustExec("create table t(a int, b int, index idx_ab(a, b))") diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 192da66bcd659..e44e5a1cc4904 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -669,21 +669,22 @@ func (cwc *ColWithCmpFuncManager) String() string { return buffer.String() } -func (ijHelper *indexJoinBuildHelper) checkIndex(innerKeys []*expression.Column, idxCols []*expression.Column, colLens []int) { +func (ijHelper *indexJoinBuildHelper) checkIndex(innerKeys []*expression.Column, idxCols []*expression.Column, colLens []int) bool { tmpSchema := expression.NewSchema(innerKeys...) ijHelper.curIdxOff2KeyOff = make([]int, len(idxCols)) - ijHelper.curPossibleUsedKeys = make([]*expression.Column, 0, len(idxCols)) ijHelper.curNotUsedIndexCols = make([]*expression.Column, 0, len(idxCols)) ijHelper.curNotUsedColLens = make([]int, 0, len(idxCols)) + keyMatched := false for i, idxCol := range idxCols { ijHelper.curIdxOff2KeyOff[i] = tmpSchema.ColumnIndex(idxCol) if ijHelper.curIdxOff2KeyOff[i] >= 0 { - ijHelper.curPossibleUsedKeys = append(ijHelper.curPossibleUsedKeys, idxCol) + keyMatched = true continue } ijHelper.curNotUsedIndexCols = append(ijHelper.curNotUsedIndexCols, idxCol) ijHelper.curNotUsedColLens = append(ijHelper.curNotUsedColLens, colLens[i]) } + return keyMatched } // findUsefulEqAndInFilters analyzes the pushedDownConds held by inner child and split them to three parts. @@ -754,36 +755,55 @@ loopOtherConds: return lastColAccesses } +func (ijHelper *indexJoinBuildHelper) removeUselessEqAndInFunc( + idxCols []*expression.Column, + notKeyEqAndIn []expression.Expression) ( + usefulEqAndIn, uselessOnes []expression.Expression, +) { + ijHelper.curPossibleUsedKeys = make([]*expression.Column, 0, len(idxCols)) + for idxColPos, notKeyColPos := 0, 0; idxColPos < len(idxCols); idxColPos++ { + if ijHelper.curIdxOff2KeyOff[idxColPos] != -1 { + ijHelper.curPossibleUsedKeys = append(ijHelper.curPossibleUsedKeys, idxCols[idxColPos]) + continue + } + if notKeyColPos < len(notKeyEqAndIn) && ijHelper.curNotUsedIndexCols[notKeyColPos].Equal(nil, idxCols[idxColPos]) { + notKeyColPos++ + continue + } + for i := idxColPos + 1; i < len(idxCols); i++ { + ijHelper.curIdxOff2KeyOff[i] = -1 + } + remained := make([]expression.Expression, 0, len(notKeyEqAndIn)-notKeyColPos) + remained = append(remained, notKeyEqAndIn[notKeyColPos:]...) + notKeyEqAndIn = notKeyEqAndIn[:notKeyColPos] + return notKeyEqAndIn, remained + } + return notKeyEqAndIn, nil +} + func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan *DataSource, innerJoinKeys []*expression.Column) error { idxCols, colLengths := expression.IndexInfo2Cols(innerPlan.schema.Columns, indexInfo) if len(idxCols) == 0 { return nil } accesses := make([]expression.Expression, 0, len(idxCols)) - ijHelper.checkIndex(innerJoinKeys, idxCols, colLengths) - matchedKeyCnt := len(ijHelper.curPossibleUsedKeys) // If no index column appears in join key, we just break. // TODO: It may meet this case: There's no join key condition, but have compare filters. // e.g. select * from t1, t2 on t1.a=t2.a and t2.b > t1.b-10 and t2.b < t1.b where t1.a=1 and t2.a=1. // After constant propagation. The t1.a=t2.a is removed. And if we have index (t2.a, t2.b). It can apply index join // to speed up. - if matchedKeyCnt <= 0 { + if !ijHelper.checkIndex(innerJoinKeys, idxCols, colLengths) { return nil } - keyMatchedLen := len(idxCols) - for ; keyMatchedLen > 0; keyMatchedLen-- { - if ijHelper.curIdxOff2KeyOff[keyMatchedLen-1] != -1 { - break - } - } notKeyEqAndIn, remained, rangeFilterCandidates := ijHelper.findUsefulEqAndInFilters(innerPlan) - accesses = append(accesses, notKeyEqAndIn...) - // We hope that the index cols appeared in the join keys can all be used to build range. If it cannot be satisfied, - // we'll mark this index as cannot be used for index join. - // So we should make sure that all columns before the keyMatchedLen is join key or has eq/in function. - if len(notKeyEqAndIn) < keyMatchedLen-matchedKeyCnt { + var remainedEqAndIn []expression.Expression + notKeyEqAndIn, remainedEqAndIn = ijHelper.removeUselessEqAndInFunc(idxCols, notKeyEqAndIn) + matchedKeyCnt := len(ijHelper.curPossibleUsedKeys) + if matchedKeyCnt <= 0 { return nil } + accesses = append(accesses, notKeyEqAndIn...) + remained = append(remained, remainedEqAndIn...) lastColPos := matchedKeyCnt + len(notKeyEqAndIn) // If all the index columns are covered by eq/in conditions, we don't need to consider other conditions anymore. if lastColPos == len(idxCols) { diff --git a/planner/core/exhaust_physical_plans_test.go b/planner/core/exhaust_physical_plans_test.go index 6ace57a2bcb41..a7cf2a21a9a1f 100644 --- a/planner/core/exhaust_physical_plans_test.go +++ b/planner/core/exhaust_physical_plans_test.go @@ -124,8 +124,8 @@ func (s *testUnitTestSuit) TestIndexJoinAnalyzeLookUpFilters(c *C) { innerKeys: []*expression.Column{dsSchema.Columns[0], dsSchema.Columns[2]}, pushedDownConds: "", otherConds: "", - ranges: "[]", - idxOff2KeyOff: "[]", + ranges: "[[NULL,NULL]]", + idxOff2KeyOff: "[0 -1 -1 -1]", accesses: "[]", remained: "[]", compareFilters: "", @@ -212,14 +212,14 @@ func (s *testUnitTestSuit) TestIndexJoinAnalyzeLookUpFilters(c *C) { innerKeys: []*expression.Column{dsSchema.Columns[0], dsSchema.Columns[2]}, pushedDownConds: "b > 1", otherConds: "", - ranges: "[]", - idxOff2KeyOff: "[]", - accesses: "[]", + ranges: "[(NULL 1,NULL +inf]]", + idxOff2KeyOff: "[0 -1 -1 -1]", + accesses: "[gt(test.t.b, 1)]", remained: "[]", compareFilters: "", }, } - for _, tt := range tests { + for i, tt := range tests { pushed, err := s.rewriteSimpleExpr(tt.pushedDownConds, dsSchema) c.Assert(err, IsNil) dataSourceNode.pushedDownConds = pushed @@ -229,7 +229,7 @@ func (s *testUnitTestSuit) TestIndexJoinAnalyzeLookUpFilters(c *C) { helper := &indexJoinBuildHelper{join: joinNode, lastColManager: nil} err = helper.analyzeLookUpFilters(idxInfo, dataSourceNode, tt.innerKeys) c.Assert(err, IsNil) - c.Assert(fmt.Sprintf("%v", helper.chosenRanges), Equals, tt.ranges) + c.Assert(fmt.Sprintf("%v", helper.chosenRanges), Equals, tt.ranges, Commentf("test case: #%v", i)) c.Assert(fmt.Sprintf("%v", helper.idxOff2KeyOff), Equals, tt.idxOff2KeyOff) c.Assert(fmt.Sprintf("%v", helper.chosenAccess), Equals, tt.accesses) c.Assert(fmt.Sprintf("%v", helper.chosenRemained), Equals, tt.remained) From ba8c208f7c17136f3e2dbccfdc3bf9efea0c7de7 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 16 Apr 2019 15:03:20 +0800 Subject: [PATCH 22/27] update logger --- planner/core/exhaust_physical_plans.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index e44e5a1cc4904..87eefc4b76b1a 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -15,6 +15,7 @@ package core import ( "bytes" + "context" "fmt" "math" @@ -27,9 +28,10 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/set" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) func (p *LogicalUnionScan) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { @@ -439,7 +441,7 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou indexInfo := path.index err := helper.analyzeLookUpFilters(indexInfo, ds, innerJoinKeys) if err != nil { - log.Warnf("[planner]: error happened when build index join: %v", err) + logutil.Logger(context.Background()).Warn("build index join failed", zap.Error(err)) } } if helper.chosenIndexInfo != nil { From 7631437f2ddb6736517a8aab7850384d52776b4d Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 16 Apr 2019 15:13:25 +0800 Subject: [PATCH 23/27] add some comment --- planner/core/exhaust_physical_plans.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 87eefc4b76b1a..b8bc03cc323f8 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -757,6 +757,12 @@ loopOtherConds: return lastColAccesses } +// removeUselessEqAndInFunc removes the useless eq/in conditions. It's designed for the following case: +// t1 join t2 on t1.a=t2.a and t1.c=t2.c where t1.b > t2.b-10 and t1.b < t2.b+10 there's index(a, b, c) on t1. +// In this case the curIdxOff2KeyOff is [0 -1 1] and the notKeyEqAndIn is []. +// It's clearly that the column c cannot be used to access data. So we need to remove it and reset the IdxOff2KeyOff to +// [0 -1 -1]. +// So that we can use t1.a=t2.a and t1.b > t2.b-10 and t1.b < t2.b+10 to build ranges then access data. func (ijHelper *indexJoinBuildHelper) removeUselessEqAndInFunc( idxCols []*expression.Column, notKeyEqAndIn []expression.Expression) ( From dca3ff493cfbf728a1eab3753a18bbb1844de28b Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 16 Apr 2019 15:26:36 +0800 Subject: [PATCH 24/27] fix unit test --- planner/core/exhaust_physical_plans.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index b8bc03cc323f8..f3c1105d2f129 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -966,6 +966,11 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ errMsg = fmt.Sprintf("Optimizer Hint %s is inapplicable", restore2JoinHint(TiDBIndexNestedLoopJoin, p.hintInfo.indexNestedLoopJoinTables)) } + // Append inapplicable reason. + if len(p.EqualConditions) == 0 { + errMsg += " without column equal ON condition" + } + // Generate warning message to client. warning := ErrInternal.GenWithStack(errMsg) p.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) From bfbd3b414cdbdc2669c9eb845fd6b2cfa6149d3a Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 22 Apr 2019 13:15:38 +0800 Subject: [PATCH 25/27] fix explaintest --- .../r/access_path_selection.result | 8 +- cmd/explaintest/r/explain.result | 14 +-- cmd/explaintest/r/explain_complex.result | 38 ++++---- .../r/explain_complex_stats.result | 38 ++++---- cmd/explaintest/r/explain_easy.result | 97 ++++++++++--------- cmd/explaintest/r/explain_easy_stats.result | 50 +++++----- cmd/explaintest/r/index_join.result | 14 +-- cmd/explaintest/r/topn_push_down.result | 26 ++--- cmd/explaintest/r/tpch.result | 76 +++++++-------- cmd/explaintest/r/window_function.result | 30 +++--- 10 files changed, 198 insertions(+), 193 deletions(-) diff --git a/cmd/explaintest/r/access_path_selection.result b/cmd/explaintest/r/access_path_selection.result index ec0309e75dee7..f9a146c8b8331 100644 --- a/cmd/explaintest/r/access_path_selection.result +++ b/cmd/explaintest/r/access_path_selection.result @@ -55,7 +55,7 @@ ANALYZE TABLE unknown_correlation; EXPLAIN SELECT * FROM unknown_correlation WHERE a = 2 ORDER BY id limit 1; id count task operator info Limit_11 1.00 root offset:0, count:1 -└─TableReader_24 1.00 root data:Limit_23 - └─Limit_23 1.00 cop offset:0, count:1 - └─Selection_21 1.00 cop eq(test.unknown_correlation.a, 2) - └─TableScan_20 4.17 cop table:unknown_correlation, range:[-inf,+inf], keep order:true +└─TableReader_22 1.00 root data:Limit_21 + └─Limit_21 1.00 cop offset:0, count:1 + └─Selection_20 1.00 cop eq(test.unknown_correlation.a, 2) + └─TableScan_19 4.17 cop table:unknown_correlation, range:[-inf,+inf], keep order:true diff --git a/cmd/explaintest/r/explain.result b/cmd/explaintest/r/explain.result index 1fbaf91769480..0055b8834a554 100644 --- a/cmd/explaintest/r/explain.result +++ b/cmd/explaintest/r/explain.result @@ -26,12 +26,14 @@ drop table if exists t; create table t(id int primary key, a int, b int); explain select group_concat(a) from t group by id; id count task operator info -Projection_6 10000.00 root cast(test.t.a) -└─TableReader_8 10000.00 root data:TableScan_7 - └─TableScan_7 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo +StreamAgg_8 8000.00 root group by:col_1, funcs:group_concat(col_0, ",") +└─Projection_18 10000.00 root cast(test.t.a), test.t.id + └─TableReader_15 10000.00 root data:TableScan_14 + └─TableScan_14 10000.00 cop table:t, range:[-inf,+inf], keep order:true, stats:pseudo explain select group_concat(a, b) from t group by id; id count task operator info -Projection_6 10000.00 root cast(test.t.a) -└─TableReader_8 10000.00 root data:TableScan_7 - └─TableScan_7 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo +StreamAgg_8 8000.00 root group by:col_2, funcs:group_concat(col_0, col_1, ",") +└─Projection_18 10000.00 root cast(test.t.a), cast(test.t.b), test.t.id + └─TableReader_15 10000.00 root data:TableScan_14 + └─TableScan_14 10000.00 cop table:t, range:[-inf,+inf], keep order:true, stats:pseudo drop table t; diff --git a/cmd/explaintest/r/explain_complex.result b/cmd/explaintest/r/explain_complex.result index 66d38378b0f33..e23a148310563 100644 --- a/cmd/explaintest/r/explain_complex.result +++ b/cmd/explaintest/r/explain_complex.result @@ -107,12 +107,12 @@ explain SELECT `ds`, `p1`, `p2`, `p3`, `p4`, `p5`, `p6_md5`, `p7_md5`, count(dic id count task operator info Projection_7 53.00 root test.dt.ds, test.dt.p1, test.dt.p2, test.dt.p3, test.dt.p4, test.dt.p5, test.dt.p6_md5, test.dt.p7_md5, install_device └─Sort_8 53.00 root test.dt.ds2:desc - └─HashAgg_17 53.00 root group by:col_10, col_11, col_12, col_13, col_14, col_15, col_16, col_17, funcs:count(col_0), firstrow(col_1), firstrow(col_2), firstrow(col_3), firstrow(col_4), firstrow(col_5), firstrow(col_6), firstrow(col_7), firstrow(col_8), firstrow(col_9) - └─IndexLookUp_18 53.00 root - ├─IndexScan_14 2650.00 cop table:dt, index:cm, range:[1062,1062], [1086,1086], [1423,1423], [1424,1424], [1425,1425], [1426,1426], [1427,1427], [1428,1428], [1429,1429], [1430,1430], [1431,1431], [1432,1432], [1433,1433], [1434,1434], [1435,1435], [1436,1436], [1437,1437], [1438,1438], [1439,1439], [1440,1440], [1441,1441], [1442,1442], [1443,1443], [1444,1444], [1445,1445], [1446,1446], [1447,1447], [1448,1448], [1449,1449], [1450,1450], [1451,1451], [1452,1452], [1488,1488], [1489,1489], [1490,1490], [1491,1491], [1492,1492], [1493,1493], [1494,1494], [1495,1495], [1496,1496], [1497,1497], [1550,1550], [1551,1551], [1552,1552], [1553,1553], [1554,1554], [1555,1555], [1556,1556], [1557,1557], [1558,1558], [1559,1559], [1597,1597], [1598,1598], [1599,1599], [1600,1600], [1601,1601], [1602,1602], [1603,1603], [1604,1604], [1605,1605], [1606,1606], [1607,1607], [1608,1608], [1609,1609], [1610,1610], [1611,1611], [1612,1612], [1613,1613], [1614,1614], [1615,1615], [1616,1616], [1623,1623], [1624,1624], [1625,1625], [1626,1626], [1627,1627], [1628,1628], [1629,1629], [1630,1630], [1631,1631], [1632,1632], [1709,1709], [1719,1719], [1720,1720], [1843,1843], [2813,2813], [2814,2814], [2815,2815], [2816,2816], [2817,2817], [2818,2818], [2819,2819], [2820,2820], [2821,2821], [2822,2822], [2823,2823], [2824,2824], [2825,2825], [2826,2826], [2827,2827], [2828,2828], [2829,2829], [2830,2830], [2831,2831], [2832,2832], [2833,2833], [2834,2834], [2835,2835], [2836,2836], [2837,2837], [2838,2838], [2839,2839], [2840,2840], [2841,2841], [2842,2842], [2843,2843], [2844,2844], [2845,2845], [2846,2846], [2847,2847], [2848,2848], [2849,2849], [2850,2850], [2851,2851], [2852,2852], [2853,2853], [2854,2854], [2855,2855], [2856,2856], [2857,2857], [2858,2858], [2859,2859], [2860,2860], [2861,2861], [2862,2862], [2863,2863], [2864,2864], [2865,2865], [2866,2866], [2867,2867], [2868,2868], [2869,2869], [2870,2870], [2871,2871], [2872,2872], [3139,3139], [3140,3140], [3141,3141], [3142,3142], [3143,3143], [3144,3144], [3145,3145], [3146,3146], [3147,3147], [3148,3148], [3149,3149], [3150,3150], [3151,3151], [3152,3152], [3153,3153], [3154,3154], [3155,3155], [3156,3156], [3157,3157], [3158,3158], [3386,3386], [3387,3387], [3388,3388], [3389,3389], [3390,3390], [3391,3391], [3392,3392], [3393,3393], [3394,3394], [3395,3395], [3664,3664], [3665,3665], [3666,3666], [3667,3667], [3668,3668], [3670,3670], [3671,3671], [3672,3672], [3673,3673], [3674,3674], [3676,3676], [3677,3677], [3678,3678], [3679,3679], [3680,3680], [3681,3681], [3682,3682], [3683,3683], [3684,3684], [3685,3685], [3686,3686], [3687,3687], [3688,3688], [3689,3689], [3690,3690], [3691,3691], [3692,3692], [3693,3693], [3694,3694], [3695,3695], [3696,3696], [3697,3697], [3698,3698], [3699,3699], [3700,3700], [3701,3701], [3702,3702], [3703,3703], [3704,3704], [3705,3705], [3706,3706], [3707,3707], [3708,3708], [3709,3709], [3710,3710], [3711,3711], [3712,3712], [3713,3713], [3714,3714], [3715,3715], [3960,3960], [3961,3961], [3962,3962], [3963,3963], [3964,3964], [3965,3965], [3966,3966], [3967,3967], [3968,3968], [3978,3978], [3979,3979], [3980,3980], [3981,3981], [3982,3982], [3983,3983], [3984,3984], [3985,3985], [3986,3986], [3987,3987], [4208,4208], [4209,4209], [4210,4210], [4211,4211], [4212,4212], [4304,4304], [4305,4305], [4306,4306], [4307,4307], [4308,4308], [4866,4866], [4867,4867], [4868,4868], [4869,4869], [4870,4870], [4871,4871], [4872,4872], [4873,4873], [4874,4874], [4875,4875], keep order:false, stats:pseudo + └─HashAgg_16 53.00 root group by:col_10, col_11, col_12, col_13, col_14, col_15, col_16, col_17, funcs:count(col_0), firstrow(col_1), firstrow(col_2), firstrow(col_3), firstrow(col_4), firstrow(col_5), firstrow(col_6), firstrow(col_7), firstrow(col_8), firstrow(col_9) + └─IndexLookUp_17 53.00 root + ├─IndexScan_13 2650.00 cop table:dt, index:cm, range:[1062,1062], [1086,1086], [1423,1423], [1424,1424], [1425,1425], [1426,1426], [1427,1427], [1428,1428], [1429,1429], [1430,1430], [1431,1431], [1432,1432], [1433,1433], [1434,1434], [1435,1435], [1436,1436], [1437,1437], [1438,1438], [1439,1439], [1440,1440], [1441,1441], [1442,1442], [1443,1443], [1444,1444], [1445,1445], [1446,1446], [1447,1447], [1448,1448], [1449,1449], [1450,1450], [1451,1451], [1452,1452], [1488,1488], [1489,1489], [1490,1490], [1491,1491], [1492,1492], [1493,1493], [1494,1494], [1495,1495], [1496,1496], [1497,1497], [1550,1550], [1551,1551], [1552,1552], [1553,1553], [1554,1554], [1555,1555], [1556,1556], [1557,1557], [1558,1558], [1559,1559], [1597,1597], [1598,1598], [1599,1599], [1600,1600], [1601,1601], [1602,1602], [1603,1603], [1604,1604], [1605,1605], [1606,1606], [1607,1607], [1608,1608], [1609,1609], [1610,1610], [1611,1611], [1612,1612], [1613,1613], [1614,1614], [1615,1615], [1616,1616], [1623,1623], [1624,1624], [1625,1625], [1626,1626], [1627,1627], [1628,1628], [1629,1629], [1630,1630], [1631,1631], [1632,1632], [1709,1709], [1719,1719], [1720,1720], [1843,1843], [2813,2813], [2814,2814], [2815,2815], [2816,2816], [2817,2817], [2818,2818], [2819,2819], [2820,2820], [2821,2821], [2822,2822], [2823,2823], [2824,2824], [2825,2825], [2826,2826], [2827,2827], [2828,2828], [2829,2829], [2830,2830], [2831,2831], [2832,2832], [2833,2833], [2834,2834], [2835,2835], [2836,2836], [2837,2837], [2838,2838], [2839,2839], [2840,2840], [2841,2841], [2842,2842], [2843,2843], [2844,2844], [2845,2845], [2846,2846], [2847,2847], [2848,2848], [2849,2849], [2850,2850], [2851,2851], [2852,2852], [2853,2853], [2854,2854], [2855,2855], [2856,2856], [2857,2857], [2858,2858], [2859,2859], [2860,2860], [2861,2861], [2862,2862], [2863,2863], [2864,2864], [2865,2865], [2866,2866], [2867,2867], [2868,2868], [2869,2869], [2870,2870], [2871,2871], [2872,2872], [3139,3139], [3140,3140], [3141,3141], [3142,3142], [3143,3143], [3144,3144], [3145,3145], [3146,3146], [3147,3147], [3148,3148], [3149,3149], [3150,3150], [3151,3151], [3152,3152], [3153,3153], [3154,3154], [3155,3155], [3156,3156], [3157,3157], [3158,3158], [3386,3386], [3387,3387], [3388,3388], [3389,3389], [3390,3390], [3391,3391], [3392,3392], [3393,3393], [3394,3394], [3395,3395], [3664,3664], [3665,3665], [3666,3666], [3667,3667], [3668,3668], [3670,3670], [3671,3671], [3672,3672], [3673,3673], [3674,3674], [3676,3676], [3677,3677], [3678,3678], [3679,3679], [3680,3680], [3681,3681], [3682,3682], [3683,3683], [3684,3684], [3685,3685], [3686,3686], [3687,3687], [3688,3688], [3689,3689], [3690,3690], [3691,3691], [3692,3692], [3693,3693], [3694,3694], [3695,3695], [3696,3696], [3697,3697], [3698,3698], [3699,3699], [3700,3700], [3701,3701], [3702,3702], [3703,3703], [3704,3704], [3705,3705], [3706,3706], [3707,3707], [3708,3708], [3709,3709], [3710,3710], [3711,3711], [3712,3712], [3713,3713], [3714,3714], [3715,3715], [3960,3960], [3961,3961], [3962,3962], [3963,3963], [3964,3964], [3965,3965], [3966,3966], [3967,3967], [3968,3968], [3978,3978], [3979,3979], [3980,3980], [3981,3981], [3982,3982], [3983,3983], [3984,3984], [3985,3985], [3986,3986], [3987,3987], [4208,4208], [4209,4209], [4210,4210], [4211,4211], [4212,4212], [4304,4304], [4305,4305], [4306,4306], [4307,4307], [4308,4308], [4866,4866], [4867,4867], [4868,4868], [4869,4869], [4870,4870], [4871,4871], [4872,4872], [4873,4873], [4874,4874], [4875,4875], keep order:false, stats:pseudo └─HashAgg_11 53.00 cop group by:test.dt.ds, test.dt.p1, test.dt.p2, test.dt.p3, test.dt.p4, test.dt.p5, test.dt.p6_md5, test.dt.p7_md5, funcs:count(test.dt.dic), firstrow(test.dt.ds), firstrow(test.dt.ds2), firstrow(test.dt.p1), firstrow(test.dt.p2), firstrow(test.dt.p3), firstrow(test.dt.p4), firstrow(test.dt.p5), firstrow(test.dt.p6_md5), firstrow(test.dt.p7_md5) - └─Selection_16 66.25 cop ge(test.dt.ds, 2016-09-01 00:00:00.000000), le(test.dt.ds, 2016-11-03 00:00:00.000000) - └─TableScan_15 2650.00 cop table:dt, keep order:false, stats:pseudo + └─Selection_15 66.25 cop ge(test.dt.ds, 2016-09-01 00:00:00.000000), le(test.dt.ds, 2016-11-03 00:00:00.000000) + └─TableScan_14 2650.00 cop table:dt, keep order:false, stats:pseudo explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext, gad.t as gtime from st gad join (select id, aid, pt, dic, ip, t from dd where pt = 'android' and bm = 0 and t > 1478143908) sdk on gad.aid = sdk.aid and gad.ip = sdk.ip and sdk.t > gad.t where gad.t > 1478143908 and gad.pt = 'android' group by gad.aid, sdk.dic limit 2500; id count task operator info Projection_13 1.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test.dd.ip, test.dd.t, gad.p1, gad.p2, gad.p3, gad.p4, gad.p5, gad.p6_md5, gad.p7_md5, gad.ext, gad.t @@ -120,7 +120,7 @@ Projection_13 1.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test.d └─HashAgg_19 1.00 root group by:gad.aid, test.dd.dic, funcs:firstrow(gad.id), firstrow(gad.aid), firstrow(gad.cm), firstrow(gad.p1), firstrow(gad.p2), firstrow(gad.p3), firstrow(gad.p4), firstrow(gad.p5), firstrow(gad.p6_md5), firstrow(gad.p7_md5), firstrow(gad.ext), firstrow(gad.t), firstrow(test.dd.id), firstrow(test.dd.dic), firstrow(test.dd.ip), firstrow(test.dd.t) └─IndexJoin_24 0.00 root inner join, inner:IndexLookUp_23, outer key:gad.aid, inner key:test.dd.aid, other cond:eq(test.dd.ip, gad.ip), gt(test.dd.t, gad.t) ├─IndexLookUp_23 0.00 root - │ ├─IndexScan_20 10.00 cop table:dd, index:aid, dic, range: decided by [gad.aid gad.ip], keep order:false, stats:pseudo + │ ├─IndexScan_20 10.00 cop table:dd, index:aid, dic, range: decided by [eq(test.dd.aid, gad.aid)], keep order:false, stats:pseudo │ └─Selection_22 0.00 cop eq(test.dd.bm, 0), eq(test.dd.pt, "android"), gt(test.dd.t, 1478143908), not(isnull(test.dd.ip)), not(isnull(test.dd.t)) │ └─TableScan_21 10.00 cop table:dd, keep order:false, stats:pseudo └─IndexLookUp_33 3.33 root @@ -137,37 +137,37 @@ Projection_10 0.00 root gad.id, sdk.id, gad.aid, gad.cm, sdk.dic, sdk.ip, sdk.t, │ └─Selection_26 0.00 cop eq(gad.bm, 0), eq(gad.dit, "mac"), eq(gad.pt, "ios"), not(isnull(gad.dic)) │ └─TableScan_25 3333.33 cop table:st, keep order:false, stats:pseudo └─IndexLookUp_17 0.00 root - ├─IndexScan_14 10.00 cop table:sdk, index:aid, dic, range: decided by [gad.aid gad.dic], keep order:false, stats:pseudo + ├─IndexScan_14 10.00 cop table:sdk, index:aid, dic, range: decided by [eq(sdk.aid, gad.aid)], keep order:false, stats:pseudo └─Selection_16 0.00 cop eq(sdk.bm, 0), eq(sdk.pt, "ios"), gt(sdk.t, 1477971479), not(isnull(sdk.mac)), not(isnull(sdk.t)) └─TableScan_15 10.00 cop table:dd, keep order:false, stats:pseudo explain SELECT cm, p1, p2, p3, p4, p5, p6_md5, p7_md5, count(1) as click_pv, count(DISTINCT ip) as click_ip FROM st WHERE (t between 1478188800 and 1478275200) and aid='cn.sbkcq' and pt='android' GROUP BY cm, p1, p2, p3, p4, p5, p6_md5, p7_md5; id count task operator info Projection_5 1.00 root test.st.cm, test.st.p1, test.st.p2, test.st.p3, test.st.p4, test.st.p5, test.st.p6_md5, test.st.p7_md5, 3_col_0, 3_col_1 └─HashAgg_7 1.00 root group by:test.st.cm, test.st.p1, test.st.p2, test.st.p3, test.st.p4, test.st.p5, test.st.p6_md5, test.st.p7_md5, funcs:count(1), count(distinct test.st.ip), firstrow(test.st.cm), firstrow(test.st.p1), firstrow(test.st.p2), firstrow(test.st.p3), firstrow(test.st.p4), firstrow(test.st.p5), firstrow(test.st.p6_md5), firstrow(test.st.p7_md5) - └─IndexLookUp_16 0.00 root - ├─IndexScan_13 250.00 cop table:st, index:t, range:[1478188800,1478275200], keep order:false, stats:pseudo - └─Selection_15 0.00 cop eq(test.st.aid, "cn.sbkcq"), eq(test.st.pt, "android") - └─TableScan_14 250.00 cop table:st, keep order:false, stats:pseudo + └─IndexLookUp_15 0.00 root + ├─IndexScan_12 250.00 cop table:st, index:t, range:[1478188800,1478275200], keep order:false, stats:pseudo + └─Selection_14 0.00 cop eq(test.st.aid, "cn.sbkcq"), eq(test.st.pt, "android") + └─TableScan_13 250.00 cop table:st, keep order:false, stats:pseudo explain select dt.id as id, dt.aid as aid, dt.pt as pt, dt.dic as dic, dt.cm as cm, rr.gid as gid, rr.acd as acd, rr.t as t,dt.p1 as p1, dt.p2 as p2, dt.p3 as p3, dt.p4 as p4, dt.p5 as p5, dt.p6_md5 as p6, dt.p7_md5 as p7 from dt dt join rr rr on (rr.pt = 'ios' and rr.t > 1478185592 and dt.aid = rr.aid and dt.dic = rr.dic) where dt.pt = 'ios' and dt.t > 1478185592 and dt.bm = 0 limit 2000; id count task operator info Projection_10 0.00 root dt.id, dt.aid, dt.pt, dt.dic, dt.cm, rr.gid, rr.acd, rr.t, dt.p1, dt.p2, dt.p3, dt.p4, dt.p5, dt.p6_md5, dt.p7_md5 └─Limit_13 0.00 root offset:0, count:2000 └─IndexJoin_19 0.00 root inner join, inner:IndexLookUp_18, outer key:dt.aid, dt.dic, inner key:rr.aid, rr.dic - ├─TableReader_43 0.00 root data:Selection_42 - │ └─Selection_42 0.00 cop eq(dt.bm, 0), eq(dt.pt, "ios"), gt(dt.t, 1478185592), not(isnull(dt.dic)) - │ └─TableScan_41 10000.00 cop table:dt, range:[0,+inf], keep order:false, stats:pseudo + ├─TableReader_41 0.00 root data:Selection_40 + │ └─Selection_40 0.00 cop eq(dt.bm, 0), eq(dt.pt, "ios"), gt(dt.t, 1478185592), not(isnull(dt.dic)) + │ └─TableScan_39 10000.00 cop table:dt, range:[0,+inf], keep order:false, stats:pseudo └─IndexLookUp_18 3.33 root - ├─IndexScan_15 10.00 cop table:rr, index:aid, dic, range: decided by [dt.aid dt.dic], keep order:false, stats:pseudo + ├─IndexScan_15 10.00 cop table:rr, index:aid, dic, range: decided by [eq(rr.aid, dt.aid) eq(rr.dic, dt.dic)], keep order:false, stats:pseudo └─Selection_17 3.33 cop eq(rr.pt, "ios"), gt(rr.t, 1478185592) └─TableScan_16 10.00 cop table:rr, keep order:false, stats:pseudo explain select pc,cr,count(DISTINCT uid) as pay_users,count(oid) as pay_times,sum(am) as am from pp where ps=2 and ppt>=1478188800 and ppt<1478275200 and pi in ('510017','520017') and uid in ('18089709','18090780') group by pc,cr; id count task operator info Projection_5 1.00 root test.pp.pc, test.pp.cr, 3_col_0, 3_col_1, 3_col_2 └─HashAgg_7 1.00 root group by:test.pp.cr, test.pp.pc, funcs:count(distinct test.pp.uid), count(test.pp.oid), sum(test.pp.am), firstrow(test.pp.pc), firstrow(test.pp.cr) - └─IndexLookUp_24 0.00 root - ├─IndexScan_21 0.40 cop table:pp, index:uid, pi, range:[18089709 510017,18089709 510017], [18089709 520017,18089709 520017], [18090780 510017,18090780 510017], [18090780 520017,18090780 520017], keep order:false, stats:pseudo - └─Selection_23 0.00 cop eq(test.pp.ps, 2), ge(test.pp.ppt, 1478188800), lt(test.pp.ppt, 1478275200) - └─TableScan_22 0.40 cop table:pp, keep order:false, stats:pseudo + └─IndexLookUp_21 0.00 root + ├─IndexScan_18 0.40 cop table:pp, index:uid, pi, range:[18089709 510017,18089709 510017], [18089709 520017,18089709 520017], [18090780 510017,18090780 510017], [18090780 520017,18090780 520017], keep order:false, stats:pseudo + └─Selection_20 0.00 cop eq(test.pp.ps, 2), ge(test.pp.ppt, 1478188800), lt(test.pp.ppt, 1478275200) + └─TableScan_19 0.40 cop table:pp, keep order:false, stats:pseudo CREATE TABLE `tbl_001` (`a` int, `b` int); CREATE TABLE `tbl_002` (`a` int, `b` int); CREATE TABLE `tbl_003` (`a` int, `b` int); diff --git a/cmd/explaintest/r/explain_complex_stats.result b/cmd/explaintest/r/explain_complex_stats.result index 805695e77132c..5db97abb7f63e 100644 --- a/cmd/explaintest/r/explain_complex_stats.result +++ b/cmd/explaintest/r/explain_complex_stats.result @@ -117,12 +117,12 @@ explain SELECT ds, p1, p2, p3, p4, p5, p6_md5, p7_md5, count(dic) as install_dev id count task operator info Projection_7 21.40 root test.dt.ds, test.dt.p1, test.dt.p2, test.dt.p3, test.dt.p4, test.dt.p5, test.dt.p6_md5, test.dt.p7_md5, install_device └─Sort_8 21.40 root test.dt.ds2:desc - └─HashAgg_17 21.40 root group by:col_10, col_11, col_12, col_13, col_14, col_15, col_16, col_17, funcs:count(col_0), firstrow(col_1), firstrow(col_2), firstrow(col_3), firstrow(col_4), firstrow(col_5), firstrow(col_6), firstrow(col_7), firstrow(col_8), firstrow(col_9) - └─IndexLookUp_18 21.40 root - ├─IndexScan_14 128.32 cop table:dt, index:cm, range:[1062,1062], [1086,1086], [1423,1423], [1424,1424], [1425,1425], [1426,1426], [1427,1427], [1428,1428], [1429,1429], [1430,1430], [1431,1431], [1432,1432], [1433,1433], [1434,1434], [1435,1435], [1436,1436], [1437,1437], [1438,1438], [1439,1439], [1440,1440], [1441,1441], [1442,1442], [1443,1443], [1444,1444], [1445,1445], [1446,1446], [1447,1447], [1448,1448], [1449,1449], [1450,1450], [1451,1451], [1452,1452], [1488,1488], [1489,1489], [1490,1490], [1491,1491], [1492,1492], [1493,1493], [1494,1494], [1495,1495], [1496,1496], [1497,1497], [1550,1550], [1551,1551], [1552,1552], [1553,1553], [1554,1554], [1555,1555], [1556,1556], [1557,1557], [1558,1558], [1559,1559], [1597,1597], [1598,1598], [1599,1599], [1600,1600], [1601,1601], [1602,1602], [1603,1603], [1604,1604], [1605,1605], [1606,1606], [1607,1607], [1608,1608], [1609,1609], [1610,1610], [1611,1611], [1612,1612], [1613,1613], [1614,1614], [1615,1615], [1616,1616], [1623,1623], [1624,1624], [1625,1625], [1626,1626], [1627,1627], [1628,1628], [1629,1629], [1630,1630], [1631,1631], [1632,1632], [1709,1709], [1719,1719], [1720,1720], [1843,1843], [2813,2813], [2814,2814], [2815,2815], [2816,2816], [2817,2817], [2818,2818], [2819,2819], [2820,2820], [2821,2821], [2822,2822], [2823,2823], [2824,2824], [2825,2825], [2826,2826], [2827,2827], [2828,2828], [2829,2829], [2830,2830], [2831,2831], [2832,2832], [2833,2833], [2834,2834], [2835,2835], [2836,2836], [2837,2837], [2838,2838], [2839,2839], [2840,2840], [2841,2841], [2842,2842], [2843,2843], [2844,2844], [2845,2845], [2846,2846], [2847,2847], [2848,2848], [2849,2849], [2850,2850], [2851,2851], [2852,2852], [2853,2853], [2854,2854], [2855,2855], [2856,2856], [2857,2857], [2858,2858], [2859,2859], [2860,2860], [2861,2861], [2862,2862], [2863,2863], [2864,2864], [2865,2865], [2866,2866], [2867,2867], [2868,2868], [2869,2869], [2870,2870], [2871,2871], [2872,2872], [3139,3139], [3140,3140], [3141,3141], [3142,3142], [3143,3143], [3144,3144], [3145,3145], [3146,3146], [3147,3147], [3148,3148], [3149,3149], [3150,3150], [3151,3151], [3152,3152], [3153,3153], [3154,3154], [3155,3155], [3156,3156], [3157,3157], [3158,3158], [3386,3386], [3387,3387], [3388,3388], [3389,3389], [3390,3390], [3391,3391], [3392,3392], [3393,3393], [3394,3394], [3395,3395], [3664,3664], [3665,3665], [3666,3666], [3667,3667], [3668,3668], [3670,3670], [3671,3671], [3672,3672], [3673,3673], [3674,3674], [3676,3676], [3677,3677], [3678,3678], [3679,3679], [3680,3680], [3681,3681], [3682,3682], [3683,3683], [3684,3684], [3685,3685], [3686,3686], [3687,3687], [3688,3688], [3689,3689], [3690,3690], [3691,3691], [3692,3692], [3693,3693], [3694,3694], [3695,3695], [3696,3696], [3697,3697], [3698,3698], [3699,3699], [3700,3700], [3701,3701], [3702,3702], [3703,3703], [3704,3704], [3705,3705], [3706,3706], [3707,3707], [3708,3708], [3709,3709], [3710,3710], [3711,3711], [3712,3712], [3713,3713], [3714,3714], [3715,3715], [3960,3960], [3961,3961], [3962,3962], [3963,3963], [3964,3964], [3965,3965], [3966,3966], [3967,3967], [3968,3968], [3978,3978], [3979,3979], [3980,3980], [3981,3981], [3982,3982], [3983,3983], [3984,3984], [3985,3985], [3986,3986], [3987,3987], [4208,4208], [4209,4209], [4210,4210], [4211,4211], [4212,4212], [4304,4304], [4305,4305], [4306,4306], [4307,4307], [4308,4308], [4866,4866], [4867,4867], [4868,4868], [4869,4869], [4870,4870], [4871,4871], [4872,4872], [4873,4873], [4874,4874], [4875,4875], keep order:false + └─HashAgg_16 21.40 root group by:col_10, col_11, col_12, col_13, col_14, col_15, col_16, col_17, funcs:count(col_0), firstrow(col_1), firstrow(col_2), firstrow(col_3), firstrow(col_4), firstrow(col_5), firstrow(col_6), firstrow(col_7), firstrow(col_8), firstrow(col_9) + └─IndexLookUp_17 21.40 root + ├─IndexScan_13 128.32 cop table:dt, index:cm, range:[1062,1062], [1086,1086], [1423,1423], [1424,1424], [1425,1425], [1426,1426], [1427,1427], [1428,1428], [1429,1429], [1430,1430], [1431,1431], [1432,1432], [1433,1433], [1434,1434], [1435,1435], [1436,1436], [1437,1437], [1438,1438], [1439,1439], [1440,1440], [1441,1441], [1442,1442], [1443,1443], [1444,1444], [1445,1445], [1446,1446], [1447,1447], [1448,1448], [1449,1449], [1450,1450], [1451,1451], [1452,1452], [1488,1488], [1489,1489], [1490,1490], [1491,1491], [1492,1492], [1493,1493], [1494,1494], [1495,1495], [1496,1496], [1497,1497], [1550,1550], [1551,1551], [1552,1552], [1553,1553], [1554,1554], [1555,1555], [1556,1556], [1557,1557], [1558,1558], [1559,1559], [1597,1597], [1598,1598], [1599,1599], [1600,1600], [1601,1601], [1602,1602], [1603,1603], [1604,1604], [1605,1605], [1606,1606], [1607,1607], [1608,1608], [1609,1609], [1610,1610], [1611,1611], [1612,1612], [1613,1613], [1614,1614], [1615,1615], [1616,1616], [1623,1623], [1624,1624], [1625,1625], [1626,1626], [1627,1627], [1628,1628], [1629,1629], [1630,1630], [1631,1631], [1632,1632], [1709,1709], [1719,1719], [1720,1720], [1843,1843], [2813,2813], [2814,2814], [2815,2815], [2816,2816], [2817,2817], [2818,2818], [2819,2819], [2820,2820], [2821,2821], [2822,2822], [2823,2823], [2824,2824], [2825,2825], [2826,2826], [2827,2827], [2828,2828], [2829,2829], [2830,2830], [2831,2831], [2832,2832], [2833,2833], [2834,2834], [2835,2835], [2836,2836], [2837,2837], [2838,2838], [2839,2839], [2840,2840], [2841,2841], [2842,2842], [2843,2843], [2844,2844], [2845,2845], [2846,2846], [2847,2847], [2848,2848], [2849,2849], [2850,2850], [2851,2851], [2852,2852], [2853,2853], [2854,2854], [2855,2855], [2856,2856], [2857,2857], [2858,2858], [2859,2859], [2860,2860], [2861,2861], [2862,2862], [2863,2863], [2864,2864], [2865,2865], [2866,2866], [2867,2867], [2868,2868], [2869,2869], [2870,2870], [2871,2871], [2872,2872], [3139,3139], [3140,3140], [3141,3141], [3142,3142], [3143,3143], [3144,3144], [3145,3145], [3146,3146], [3147,3147], [3148,3148], [3149,3149], [3150,3150], [3151,3151], [3152,3152], [3153,3153], [3154,3154], [3155,3155], [3156,3156], [3157,3157], [3158,3158], [3386,3386], [3387,3387], [3388,3388], [3389,3389], [3390,3390], [3391,3391], [3392,3392], [3393,3393], [3394,3394], [3395,3395], [3664,3664], [3665,3665], [3666,3666], [3667,3667], [3668,3668], [3670,3670], [3671,3671], [3672,3672], [3673,3673], [3674,3674], [3676,3676], [3677,3677], [3678,3678], [3679,3679], [3680,3680], [3681,3681], [3682,3682], [3683,3683], [3684,3684], [3685,3685], [3686,3686], [3687,3687], [3688,3688], [3689,3689], [3690,3690], [3691,3691], [3692,3692], [3693,3693], [3694,3694], [3695,3695], [3696,3696], [3697,3697], [3698,3698], [3699,3699], [3700,3700], [3701,3701], [3702,3702], [3703,3703], [3704,3704], [3705,3705], [3706,3706], [3707,3707], [3708,3708], [3709,3709], [3710,3710], [3711,3711], [3712,3712], [3713,3713], [3714,3714], [3715,3715], [3960,3960], [3961,3961], [3962,3962], [3963,3963], [3964,3964], [3965,3965], [3966,3966], [3967,3967], [3968,3968], [3978,3978], [3979,3979], [3980,3980], [3981,3981], [3982,3982], [3983,3983], [3984,3984], [3985,3985], [3986,3986], [3987,3987], [4208,4208], [4209,4209], [4210,4210], [4211,4211], [4212,4212], [4304,4304], [4305,4305], [4306,4306], [4307,4307], [4308,4308], [4866,4866], [4867,4867], [4868,4868], [4869,4869], [4870,4870], [4871,4871], [4872,4872], [4873,4873], [4874,4874], [4875,4875], keep order:false └─HashAgg_11 21.40 cop group by:test.dt.ds, test.dt.p1, test.dt.p2, test.dt.p3, test.dt.p4, test.dt.p5, test.dt.p6_md5, test.dt.p7_md5, funcs:count(test.dt.dic), firstrow(test.dt.ds), firstrow(test.dt.ds2), firstrow(test.dt.p1), firstrow(test.dt.p2), firstrow(test.dt.p3), firstrow(test.dt.p4), firstrow(test.dt.p5), firstrow(test.dt.p6_md5), firstrow(test.dt.p7_md5) - └─Selection_16 21.43 cop ge(test.dt.ds, 2016-09-01 00:00:00.000000), le(test.dt.ds, 2016-11-03 00:00:00.000000) - └─TableScan_15 128.32 cop table:dt, keep order:false + └─Selection_15 21.43 cop ge(test.dt.ds, 2016-09-01 00:00:00.000000), le(test.dt.ds, 2016-11-03 00:00:00.000000) + └─TableScan_14 128.32 cop table:dt, keep order:false explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext, gad.t as gtime from st gad join (select id, aid, pt, dic, ip, t from dd where pt = 'android' and bm = 0 and t > 1478143908) sdk on gad.aid = sdk.aid and gad.ip = sdk.ip and sdk.t > gad.t where gad.t > 1478143908 and gad.bm = 0 and gad.pt = 'android' group by gad.aid, sdk.dic limit 2500; id count task operator info Projection_13 424.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test.dd.ip, test.dd.t, gad.p1, gad.p2, gad.p3, gad.p4, gad.p5, gad.p6_md5, gad.p7_md5, gad.ext, gad.t @@ -133,7 +133,7 @@ Projection_13 424.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test │ └─Selection_28 424.00 cop eq(gad.bm, 0), eq(gad.pt, "android"), gt(gad.t, 1478143908), not(isnull(gad.ip)) │ └─TableScan_27 1999.00 cop table:gad, range:[0,+inf], keep order:false └─IndexLookUp_23 455.80 root - ├─IndexScan_20 1.00 cop table:dd, index:aid, dic, range: decided by [gad.aid gad.ip], keep order:false + ├─IndexScan_20 1.00 cop table:dd, index:aid, dic, range: decided by [eq(test.dd.aid, gad.aid)], keep order:false └─Selection_22 455.80 cop eq(test.dd.bm, 0), eq(test.dd.pt, "android"), gt(test.dd.t, 1478143908), not(isnull(test.dd.ip)), not(isnull(test.dd.t)) └─TableScan_21 1.00 cop table:dd, keep order:false explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext from st gad join dd sdk on gad.aid = sdk.aid and gad.dic = sdk.mac and gad.t < sdk.t where gad.t > 1477971479 and gad.bm = 0 and gad.pt = 'ios' and gad.dit = 'mac' and sdk.t > 1477971479 and sdk.bm = 0 and sdk.pt = 'ios' limit 3000; @@ -145,37 +145,37 @@ Projection_10 170.34 root gad.id, sdk.id, gad.aid, gad.cm, sdk.dic, sdk.ip, sdk. │ └─Selection_22 170.34 cop eq(gad.bm, 0), eq(gad.dit, "mac"), eq(gad.pt, "ios"), gt(gad.t, 1477971479), not(isnull(gad.dic)) │ └─TableScan_21 1999.00 cop table:gad, range:[0,+inf], keep order:false └─IndexLookUp_17 509.04 root - ├─IndexScan_14 1.00 cop table:sdk, index:aid, dic, range: decided by [gad.aid gad.dic], keep order:false + ├─IndexScan_14 1.00 cop table:sdk, index:aid, dic, range: decided by [eq(sdk.aid, gad.aid)], keep order:false └─Selection_16 509.04 cop eq(sdk.bm, 0), eq(sdk.pt, "ios"), gt(sdk.t, 1477971479), not(isnull(sdk.mac)), not(isnull(sdk.t)) └─TableScan_15 1.00 cop table:dd, keep order:false explain SELECT cm, p1, p2, p3, p4, p5, p6_md5, p7_md5, count(1) as click_pv, count(DISTINCT ip) as click_ip FROM st WHERE (t between 1478188800 and 1478275200) and aid='cn.sbkcq' and pt='android' GROUP BY cm, p1, p2, p3, p4, p5, p6_md5, p7_md5; id count task operator info Projection_5 39.28 root test.st.cm, test.st.p1, test.st.p2, test.st.p3, test.st.p4, test.st.p5, test.st.p6_md5, test.st.p7_md5, 3_col_0, 3_col_1 └─HashAgg_7 39.28 root group by:test.st.cm, test.st.p1, test.st.p2, test.st.p3, test.st.p4, test.st.p5, test.st.p6_md5, test.st.p7_md5, funcs:count(1), count(distinct test.st.ip), firstrow(test.st.cm), firstrow(test.st.p1), firstrow(test.st.p2), firstrow(test.st.p3), firstrow(test.st.p4), firstrow(test.st.p5), firstrow(test.st.p6_md5), firstrow(test.st.p7_md5) - └─IndexLookUp_16 39.38 root - ├─IndexScan_13 160.23 cop table:st, index:t, range:[1478188800,1478275200], keep order:false - └─Selection_15 39.38 cop eq(test.st.aid, "cn.sbkcq"), eq(test.st.pt, "android") - └─TableScan_14 160.23 cop table:st, keep order:false + └─IndexLookUp_15 39.38 root + ├─IndexScan_12 160.23 cop table:st, index:t, range:[1478188800,1478275200], keep order:false + └─Selection_14 39.38 cop eq(test.st.aid, "cn.sbkcq"), eq(test.st.pt, "android") + └─TableScan_13 160.23 cop table:st, keep order:false explain select dt.id as id, dt.aid as aid, dt.pt as pt, dt.dic as dic, dt.cm as cm, rr.gid as gid, rr.acd as acd, rr.t as t,dt.p1 as p1, dt.p2 as p2, dt.p3 as p3, dt.p4 as p4, dt.p5 as p5, dt.p6_md5 as p6, dt.p7_md5 as p7 from dt dt join rr rr on (rr.pt = 'ios' and rr.t > 1478185592 and dt.aid = rr.aid and dt.dic = rr.dic) where dt.pt = 'ios' and dt.t > 1478185592 and dt.bm = 0 limit 2000; id count task operator info Projection_10 428.32 root dt.id, dt.aid, dt.pt, dt.dic, dt.cm, rr.gid, rr.acd, rr.t, dt.p1, dt.p2, dt.p3, dt.p4, dt.p5, dt.p6_md5, dt.p7_md5 └─Limit_13 428.32 root offset:0, count:2000 └─IndexJoin_19 428.32 root inner join, inner:IndexLookUp_18, outer key:dt.aid, dt.dic, inner key:rr.aid, rr.dic - ├─TableReader_43 428.32 root data:Selection_42 - │ └─Selection_42 428.32 cop eq(dt.bm, 0), eq(dt.pt, "ios"), gt(dt.t, 1478185592), not(isnull(dt.dic)) - │ └─TableScan_41 2000.00 cop table:dt, range:[0,+inf], keep order:false + ├─TableReader_41 428.32 root data:Selection_40 + │ └─Selection_40 428.32 cop eq(dt.bm, 0), eq(dt.pt, "ios"), gt(dt.t, 1478185592), not(isnull(dt.dic)) + │ └─TableScan_39 2000.00 cop table:dt, range:[0,+inf], keep order:false └─IndexLookUp_18 970.00 root - ├─IndexScan_15 1.00 cop table:rr, index:aid, dic, range: decided by [dt.aid dt.dic], keep order:false + ├─IndexScan_15 1.00 cop table:rr, index:aid, dic, range: decided by [eq(rr.aid, dt.aid) eq(rr.dic, dt.dic)], keep order:false └─Selection_17 970.00 cop eq(rr.pt, "ios"), gt(rr.t, 1478185592) └─TableScan_16 1.00 cop table:rr, keep order:false explain select pc,cr,count(DISTINCT uid) as pay_users,count(oid) as pay_times,sum(am) as am from pp where ps=2 and ppt>=1478188800 and ppt<1478275200 and pi in ('510017','520017') and uid in ('18089709','18090780') group by pc,cr; id count task operator info Projection_5 207.86 root test.pp.pc, test.pp.cr, 3_col_0, 3_col_1, 3_col_2 └─HashAgg_7 207.86 root group by:test.pp.cr, test.pp.pc, funcs:count(distinct test.pp.uid), count(test.pp.oid), sum(test.pp.am), firstrow(test.pp.pc), firstrow(test.pp.cr) - └─IndexLookUp_24 207.86 root - ├─IndexScan_18 627.00 cop table:pp, index:ps, range:[2,2], keep order:false - └─Selection_20 207.86 cop ge(test.pp.ppt, 1478188800), in(test.pp.pi, 510017, 520017), in(test.pp.uid, 18089709, 18090780), lt(test.pp.ppt, 1478275200) - └─TableScan_19 627.00 cop table:pp, keep order:false + └─IndexLookUp_21 207.86 root + ├─IndexScan_15 627.00 cop table:pp, index:ps, range:[2,2], keep order:false + └─Selection_17 207.86 cop ge(test.pp.ppt, 1478188800), in(test.pp.pi, 510017, 520017), in(test.pp.uid, 18089709, 18090780), lt(test.pp.ppt, 1478275200) + └─TableScan_16 627.00 cop table:pp, keep order:false drop table if exists tbl_001; CREATE TABLE tbl_001 (a int, b int); load stats 's/explain_complex_stats_tbl_001.json'; diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 400fddda2e81e..1497fb03490de 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -24,9 +24,9 @@ TableReader_5 10000.00 root data:TableScan_4 └─TableScan_4 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo explain select * from t1 order by c2; id count task operator info -IndexLookUp_13 10000.00 root -├─IndexScan_11 10000.00 cop table:t1, index:c2, range:[NULL,+inf], keep order:true, stats:pseudo -└─TableScan_12 10000.00 cop table:t1, keep order:false, stats:pseudo +IndexLookUp_12 10000.00 root +├─IndexScan_10 10000.00 cop table:t1, index:c2, range:[NULL,+inf], keep order:true, stats:pseudo +└─TableScan_11 10000.00 cop table:t1, keep order:false, stats:pseudo explain select * from t2 order by c2; id count task operator info Sort_4 10000.00 root test.t2.c2:asc @@ -43,12 +43,12 @@ IndexReader_6 10.00 root index:IndexScan_5 explain select * from t1 left join t2 on t1.c2 = t2.c1 where t1.c1 > 1; id count task operator info IndexJoin_12 4166.67 root left outer join, inner:IndexLookUp_11, outer key:test.t1.c2, inner key:test.t2.c1 -├─TableReader_26 3333.33 root data:TableScan_25 -│ └─TableScan_25 3333.33 cop table:t1, range:(1,+inf], keep order:false, stats:pseudo +├─TableReader_24 3333.33 root data:TableScan_23 +│ └─TableScan_23 3333.33 cop table:t1, range:(1,+inf], keep order:false, stats:pseudo └─IndexLookUp_11 0.00 root ├─Selection_10 0.00 cop not(isnull(test.t2.c1)) - │ └─IndexScan_8 10.00 cop table:t2, index:c1, range: decided by [test.t1.c2], keep order:false, stats:pseudo - └─TableScan_9 0.00 cop table:t2, keep order:false + │ └─IndexScan_8 10.00 cop table:t2, index:c1, range: decided by [eq(test.t2.c1, test.t1.c2)], keep order:false, stats:pseudo + └─TableScan_9 0.00 cop table:t2, keep order:false, stats:pseudo explain update t1 set t1.c2 = 2 where t1.c1 = 1; id count task operator info Point_Get_1 1.00 root table:t1, handle:1 @@ -61,8 +61,8 @@ explain select count(b.c2) from t1 a, t2 b where a.c1 = b.c2 group by a.c1; id count task operator info Projection_11 9990.00 root cast(join_agg_0) └─IndexJoin_14 9990.00 root inner join, inner:TableReader_13, outer key:b.c2, inner key:a.c1 - ├─TableReader_13 10.00 root data:TableScan_12 - │ └─TableScan_12 10.00 cop table:a, range: decided by [b.c2], keep order:false, stats:pseudo + ├─TableReader_13 1.00 root data:TableScan_12 + │ └─TableScan_12 1.00 cop table:a, range: decided by [b.c2], keep order:false, stats:pseudo └─HashAgg_22 7992.00 root group by:col_2, funcs:count(col_0), firstrow(col_1) └─TableReader_23 7992.00 root data:HashAgg_17 └─HashAgg_17 7992.00 cop group by:b.c2, funcs:count(b.c2), firstrow(b.c2) @@ -80,7 +80,7 @@ IndexLookUp_12 1.11 root ├─Selection_10 3.33 cop gt(test.t1.c1, 1) │ └─IndexScan_8 10.00 cop table:t1, index:c2, range:[1,1], keep order:false, stats:pseudo └─Selection_11 1.11 cop lt(test.t1.c3, 1) - └─TableScan_9 3.33 cop table:t1, keep order:false + └─TableScan_9 3.33 cop table:t1, keep order:false, stats:pseudo explain select * from t1 where c1 = 1 and c2 > 1; id count task operator info TableReader_7 0.33 root data:Selection_6 @@ -99,8 +99,8 @@ explain select c1 from t1 where c1 in (select c2 from t2); id count task operator info Projection_9 9990.00 root test.t1.c1 └─IndexJoin_12 9990.00 root inner join, inner:TableReader_11, outer key:test.t2.c2, inner key:test.t1.c1 - ├─TableReader_11 10.00 root data:TableScan_10 - │ └─TableScan_10 10.00 cop table:t1, range: decided by [test.t2.c2], keep order:false, stats:pseudo + ├─TableReader_11 1.00 root data:TableScan_10 + │ └─TableScan_10 1.00 cop table:t1, range: decided by [test.t2.c2], keep order:false, stats:pseudo └─HashAgg_20 7992.00 root group by:col_1, funcs:firstrow(col_0) └─TableReader_21 7992.00 root data:HashAgg_15 └─HashAgg_15 7992.00 cop group by:test.t2.c2, funcs:firstrow(test.t2.c2) @@ -126,11 +126,11 @@ Projection_12 10000.00 root eq(test.t1.c2, test.t2.c2) ├─TableReader_16 10000.00 root data:TableScan_15 │ └─TableScan_15 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─Limit_21 1.00 root offset:0, count:1 - └─Projection_44 1.00 root test.t2.c1, test.t2.c2 - └─IndexLookUp_43 1.00 root - ├─Limit_42 1.00 cop offset:0, count:1 - │ └─IndexScan_40 1.00 cop table:t2, index:c1, range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true, stats:pseudo - └─TableScan_41 1.00 cop table:t2, keep order:false + └─Projection_41 1.00 root test.t2.c1, test.t2.c2 + └─IndexLookUp_40 1.00 root + ├─Limit_39 1.00 cop offset:0, count:1 + │ └─IndexScan_37 1.00 cop table:t2, index:c1, range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true, stats:pseudo + └─TableScan_38 1.00 cop table:t2, keep order:false, stats:pseudo explain select * from t1 order by c1 desc limit 1; id count task operator info Limit_10 1.00 root offset:0, count:1 @@ -140,12 +140,12 @@ Limit_10 1.00 root offset:0, count:1 explain select * from t4 use index(idx) where a > 1 and b > 1 and c > 1 limit 1; id count task operator info Limit_9 1.00 root offset:0, count:1 -└─IndexLookUp_17 1.00 root - ├─Selection_14 3.00 cop gt(test.t4.b, 1) - │ └─IndexScan_12 9.00 cop table:t4, index:a, b, range:(1,+inf], keep order:false, stats:pseudo - └─Limit_16 1.00 cop offset:0, count:1 - └─Selection_15 1.00 cop gt(test.t4.c, 1) - └─TableScan_13 3.00 cop table:t4, keep order:false +└─IndexLookUp_16 1.00 root + ├─Selection_13 3.00 cop gt(test.t4.b, 1) + │ └─IndexScan_11 9.00 cop table:t4, index:a, b, range:(1,+inf], keep order:false, stats:pseudo + └─Limit_15 1.00 cop offset:0, count:1 + └─Selection_14 1.00 cop gt(test.t4.c, 1) + └─TableScan_12 3.00 cop table:t4, keep order:false, stats:pseudo explain select * from t4 where a > 1 and c > 1 limit 1; id count task operator info Limit_8 1.00 root offset:0, count:1 @@ -291,9 +291,9 @@ Projection_11 10000.00 root 9_aux_0 └─IndexJoin_44 12.50 root inner join, inner:TableReader_43, outer key:s.a, inner key:t1.a ├─TableReader_37 1.00 root data:TableScan_36 │ └─TableScan_36 1.00 cop table:s, range: decided by [eq(s.a, test.t.a)], keep order:false, stats:pseudo - └─TableReader_43 10.00 root data:Selection_42 - └─Selection_42 10.00 cop eq(t1.a, test.t.a) - └─TableScan_41 10.00 cop table:t1, range: decided by [s.a], keep order:false, stats:pseudo + └─TableReader_43 0.80 root data:Selection_42 + └─Selection_42 0.80 cop eq(t1.a, test.t.a) + └─TableScan_41 1.00 cop table:t1, range: decided by [s.a], keep order:false, stats:pseudo explain select t.c in (select count(*) from t s use index(idx), t t1 where s.b = t.a and s.a = t1.a) from t; id count task operator info Projection_11 10000.00 root 9_aux_0 @@ -304,8 +304,8 @@ Projection_11 10000.00 root 9_aux_0 └─IndexJoin_32 12.50 root inner join, inner:TableReader_31, outer key:s.a, inner key:t1.a ├─IndexReader_27 10.00 root index:IndexScan_26 │ └─IndexScan_26 10.00 cop table:s, index:b, range: decided by [eq(s.b, test.t.a)], keep order:false, stats:pseudo - └─TableReader_31 10.00 root data:TableScan_30 - └─TableScan_30 10.00 cop table:t1, range: decided by [s.a], keep order:false, stats:pseudo + └─TableReader_31 1.00 root data:TableScan_30 + └─TableScan_30 1.00 cop table:t1, range: decided by [s.a], keep order:false, stats:pseudo explain select t.c in (select count(*) from t s use index(idx), t t1 where s.b = t.a and s.c = t1.a) from t; id count task operator info Projection_11 10000.00 root 9_aux_0 @@ -318,8 +318,8 @@ Projection_11 10000.00 root 9_aux_0 │ ├─IndexScan_26 10.00 cop table:s, index:b, range: decided by [eq(s.b, test.t.a)], keep order:false, stats:pseudo │ └─Selection_28 9.99 cop not(isnull(s.c)) │ └─TableScan_27 10.00 cop table:t, keep order:false, stats:pseudo - └─TableReader_33 10.00 root data:TableScan_32 - └─TableScan_32 10.00 cop table:t1, range: decided by [s.c], keep order:false, stats:pseudo + └─TableReader_33 1.00 root data:TableScan_32 + └─TableScan_32 1.00 cop table:t1, range: decided by [s.c], keep order:false, stats:pseudo insert into t values(1, 1, 1), (2, 2 ,2), (3, 3, 3), (4, 3, 4),(5,3,5); analyze table t; explain select t.c in (select count(*) from t s, t t1 where s.b = t.a and s.b = 3 and s.a = t1.a) from t; @@ -329,11 +329,12 @@ Projection_11 5.00 root 9_aux_0 ├─TableReader_15 5.00 root data:TableScan_14 │ └─TableScan_14 5.00 cop table:t, range:[-inf,+inf], keep order:false └─StreamAgg_20 1.00 root funcs:count(1) - └─IndexJoin_48 3.00 root inner join, inner:TableReader_47, outer key:s.a, inner key:t1.a - ├─IndexReader_40 3.00 root index:IndexScan_39 - │ └─IndexScan_39 3.00 cop table:s, index:b, range:[3,3], keep order:false - └─TableReader_47 4.00 root data:Selection_46 - └─Selection_46 4.00 cop eq(3, test.t.a) + └─IndexJoin_48 2.40 root inner join, inner:TableReader_47, outer key:s.a, inner key:t1.a + ├─IndexReader_40 2.40 root index:Selection_39 + │ └─Selection_39 2.40 cop eq(3, test.t.a) + │ └─IndexScan_38 3.00 cop table:s, index:b, range:[3,3], keep order:false + └─TableReader_47 0.80 root data:Selection_46 + └─Selection_46 0.80 cop eq(3, test.t.a) └─TableScan_45 1.00 cop table:t1, range: decided by [s.a], keep order:false explain select t.c in (select count(*) from t s left join t t1 on s.a = t1.a where 3 = t.a and s.b = 3) from t; id count task operator info @@ -342,12 +343,13 @@ Projection_10 5.00 root 9_aux_0 ├─TableReader_14 5.00 root data:TableScan_13 │ └─TableScan_13 5.00 cop table:t, range:[-inf,+inf], keep order:false └─StreamAgg_19 1.00 root funcs:count(1) - └─Selection_36 2.40 root eq(3, test.t.a) - └─IndexJoin_40 3.00 root left outer join, inner:TableReader_39, outer key:s.a, inner key:t1.a - ├─IndexReader_33 3.00 root index:IndexScan_32 - │ └─IndexScan_32 3.00 cop table:s, index:b, range:[3,3], keep order:false - └─TableReader_39 1.00 root data:TableScan_38 - └─TableScan_38 1.00 cop table:t1, range: decided by [s.a], keep order:false + └─IndexJoin_42 2.40 root left outer join, inner:TableReader_41, outer key:s.a, inner key:t1.a + ├─IndexReader_34 2.40 root index:Selection_33 + │ └─Selection_33 2.40 cop eq(3, test.t.a) + │ └─IndexScan_32 3.00 cop table:s, index:b, range:[3,3], keep order:false + └─TableReader_41 0.80 root data:Selection_40 + └─Selection_40 0.80 cop eq(3, test.t.a) + └─TableScan_39 1.00 cop table:t1, range: decided by [s.a], keep order:false explain select t.c in (select count(*) from t s right join t t1 on s.a = t1.a where 3 = t.a and t1.b = 3) from t; id count task operator info Projection_10 5.00 root 9_aux_0 @@ -355,12 +357,13 @@ Projection_10 5.00 root 9_aux_0 ├─TableReader_14 5.00 root data:TableScan_13 │ └─TableScan_13 5.00 cop table:t, range:[-inf,+inf], keep order:false └─StreamAgg_19 1.00 root funcs:count(1) - └─IndexJoin_39 2.40 root right outer join, inner:TableReader_38, outer key:t1.a, inner key:s.a - ├─TableReader_38 1.00 root data:TableScan_37 - │ └─TableScan_37 1.00 cop table:s, range: decided by [t1.a], keep order:false - └─IndexReader_33 2.40 root index:Selection_32 - └─Selection_32 2.40 cop eq(3, test.t.a) - └─IndexScan_31 3.00 cop table:t1, index:b, range:[3,3], keep order:false + └─IndexJoin_42 2.40 root right outer join, inner:TableReader_41, outer key:t1.a, inner key:s.a + ├─TableReader_41 0.80 root data:Selection_40 + │ └─Selection_40 0.80 cop eq(3, test.t.a) + │ └─TableScan_39 1.00 cop table:s, range: decided by [t1.a], keep order:false + └─IndexReader_34 2.40 root index:Selection_33 + └─Selection_33 2.40 cop eq(3, test.t.a) + └─IndexScan_32 3.00 cop table:t1, index:b, range:[3,3], keep order:false drop table if exists t; create table t(a int unsigned); explain select t.a = '123455' from t; diff --git a/cmd/explaintest/r/explain_easy_stats.result b/cmd/explaintest/r/explain_easy_stats.result index 6ec2cbcb6a57c..1f41bb9e626a7 100644 --- a/cmd/explaintest/r/explain_easy_stats.result +++ b/cmd/explaintest/r/explain_easy_stats.result @@ -27,9 +27,9 @@ TableReader_5 1999.00 root data:TableScan_4 └─TableScan_4 1999.00 cop table:t1, range:[-inf,+inf], keep order:false explain select * from t1 order by c2; id count task operator info -IndexLookUp_13 1999.00 root -├─IndexScan_11 1999.00 cop table:t1, index:c2, range:[NULL,+inf], keep order:true -└─TableScan_12 1999.00 cop table:t1, keep order:false +IndexLookUp_12 1999.00 root +├─IndexScan_10 1999.00 cop table:t1, index:c2, range:[NULL,+inf], keep order:true +└─TableScan_11 1999.00 cop table:t1, keep order:false explain select * from t2 order by c2; id count task operator info Sort_4 1985.00 root test.t2.c2:asc @@ -46,14 +46,14 @@ IndexReader_6 0.00 root index:IndexScan_5 explain select * from t1 left join t2 on t1.c2 = t2.c1 where t1.c1 > 1; id count task operator info MergeJoin_7 2481.25 root left outer join, left key:test.t1.c2, right key:test.t2.c1 -├─IndexLookUp_18 1998.00 root -│ ├─Selection_17 1998.00 cop gt(test.t1.c1, 1) -│ │ └─IndexScan_15 1999.00 cop table:t1, index:c2, range:[NULL,+inf], keep order:true -│ └─TableScan_16 1998.00 cop table:t1, keep order:false -└─Projection_24 1985.00 root test.t2.c1, test.t2.c2 - └─IndexLookUp_23 1985.00 root - ├─IndexScan_21 1985.00 cop table:t2, index:c1, range:[-inf,+inf], keep order:true - └─TableScan_22 1985.00 cop table:t2, keep order:false +├─IndexLookUp_17 1998.00 root +│ ├─Selection_16 1998.00 cop gt(test.t1.c1, 1) +│ │ └─IndexScan_14 1999.00 cop table:t1, index:c2, range:[NULL,+inf], keep order:true +│ └─TableScan_15 1998.00 cop table:t1, keep order:false, stats:pseudo +└─Projection_22 1985.00 root test.t2.c1, test.t2.c2 + └─IndexLookUp_21 1985.00 root + ├─IndexScan_19 1985.00 cop table:t2, index:c1, range:[-inf,+inf], keep order:true + └─TableScan_20 1985.00 cop table:t2, keep order:false explain update t1 set t1.c2 = 2 where t1.c1 = 1; id count task operator info Point_Get_1 1.00 root table:t1, handle:1 @@ -85,7 +85,7 @@ IndexLookUp_12 0.00 root ├─Selection_10 0.00 cop gt(test.t1.c1, 1) │ └─IndexScan_8 0.00 cop table:t1, index:c2, range:[1,1], keep order:false └─Selection_11 0.00 cop lt(test.t1.c3, 1) - └─TableScan_9 0.00 cop table:t1, keep order:false + └─TableScan_9 0.00 cop table:t1, keep order:false, stats:pseudo explain select * from t1 where c1 = 1 and c2 > 1; id count task operator info TableReader_7 0.50 root data:Selection_6 @@ -112,11 +112,11 @@ Projection_12 1999.00 root eq(test.t1.c2, test.t2.c2) ├─TableReader_16 1999.00 root data:TableScan_15 │ └─TableScan_15 1999.00 cop table:t1, range:[-inf,+inf], keep order:false └─Limit_21 1.00 root offset:0, count:1 - └─Projection_44 1.00 root test.t2.c1, test.t2.c2 - └─IndexLookUp_43 1.00 root - ├─Limit_42 1.00 cop offset:0, count:1 - │ └─IndexScan_40 1.25 cop table:t2, index:c1, range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true - └─TableScan_41 1.00 cop table:t2, keep order:false + └─Projection_41 1.00 root test.t2.c1, test.t2.c2 + └─IndexLookUp_40 1.00 root + ├─Limit_39 1.00 cop offset:0, count:1 + │ └─IndexScan_37 1.25 cop table:t2, index:c1, range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true + └─TableScan_38 1.00 cop table:t2, keep order:false, stats:pseudo explain select * from t1 order by c1 desc limit 1; id count task operator info Limit_10 1.00 root offset:0, count:1 @@ -170,17 +170,17 @@ TableDual_5 0.00 root rows:0 explain select * from index_prune WHERE a = 1010010404050976781 AND b = 26467085526790 LIMIT 1, 1; id count task operator info Limit_9 1.00 root offset:1, count:1 -└─IndexLookUp_15 1.00 root - ├─Limit_14 1.00 cop offset:0, count:2 - │ └─IndexScan_12 1.00 cop table:index_prune, index:a, b, range:[1010010404050976781 26467085526790,1010010404050976781 26467085526790], keep order:false - └─TableScan_13 1.00 cop table:index_prune, keep order:false +└─IndexLookUp_14 1.00 root + ├─Limit_13 1.00 cop offset:0, count:2 + │ └─IndexScan_11 1.00 cop table:index_prune, index:a, b, range:[1010010404050976781 26467085526790,1010010404050976781 26467085526790], keep order:false + └─TableScan_12 1.00 cop table:index_prune, keep order:false, stats:pseudo explain select * from index_prune WHERE a = 1010010404050976781 AND b = 26467085526790 LIMIT 1, 0; id count task operator info Limit_9 0.00 root offset:1, count:0 -└─IndexLookUp_15 0.00 root - ├─Limit_14 0.00 cop offset:0, count:1 - │ └─IndexScan_12 1.00 cop table:index_prune, index:a, b, range:[1010010404050976781 26467085526790,1010010404050976781 26467085526790], keep order:false - └─TableScan_13 0.00 cop table:index_prune, keep order:false +└─IndexLookUp_14 0.00 root + ├─Limit_13 0.00 cop offset:0, count:1 + │ └─IndexScan_11 1.00 cop table:index_prune, index:a, b, range:[1010010404050976781 26467085526790,1010010404050976781 26467085526790], keep order:false + └─TableScan_12 0.00 cop table:index_prune, keep order:false, stats:pseudo explain select * from index_prune WHERE a = 1010010404050976781 AND b = 26467085526790 LIMIT 0, 1; id count task operator info Point_Get_1 1.00 root table:index_prune, index:a b diff --git a/cmd/explaintest/r/index_join.result b/cmd/explaintest/r/index_join.result index 4f81db898cf7f..6d5555bc8993e 100644 --- a/cmd/explaintest/r/index_join.result +++ b/cmd/explaintest/r/index_join.result @@ -9,8 +9,8 @@ id count task operator info IndexJoin_16 5.00 root inner join, inner:IndexLookUp_15, outer key:test.t2.a, inner key:test.t1.a ├─IndexLookUp_15 0.00 root │ ├─Selection_14 0.00 cop not(isnull(test.t1.a)) -│ │ └─IndexScan_12 5.00 cop table:t1, index:a, range: decided by [test.t2.a], keep order:false -│ └─TableScan_13 0.00 cop table:t1, keep order:false +│ │ └─IndexScan_12 5.00 cop table:t1, index:a, range: decided by [eq(test.t1.a, test.t2.a)], keep order:false +│ └─TableScan_13 0.00 cop table:t1, keep order:false, stats:pseudo └─TableReader_19 1.00 root data:Selection_18 └─Selection_18 1.00 cop not(isnull(test.t2.a)) └─TableScan_17 1.00 cop table:t2, range:[-inf,+inf], keep order:false @@ -18,10 +18,10 @@ explain select * from t1 join t2 on t1.a=t2.a; id count task operator info Projection_6 5.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b └─IndexJoin_12 5.00 root inner join, inner:IndexLookUp_11, outer key:test.t2.a, inner key:test.t1.a - ├─TableReader_32 1.00 root data:Selection_31 - │ └─Selection_31 1.00 cop not(isnull(test.t2.a)) - │ └─TableScan_30 1.00 cop table:t2, range:[-inf,+inf], keep order:false + ├─TableReader_30 1.00 root data:Selection_29 + │ └─Selection_29 1.00 cop not(isnull(test.t2.a)) + │ └─TableScan_28 1.00 cop table:t2, range:[-inf,+inf], keep order:false └─IndexLookUp_11 0.00 root ├─Selection_10 0.00 cop not(isnull(test.t1.a)) - │ └─IndexScan_8 5.00 cop table:t1, index:a, range: decided by [test.t2.a], keep order:false - └─TableScan_9 0.00 cop table:t1, keep order:false + │ └─IndexScan_8 5.00 cop table:t1, index:a, range: decided by [eq(test.t1.a, test.t2.a)], keep order:false + └─TableScan_9 0.00 cop table:t1, keep order:false, stats:pseudo diff --git a/cmd/explaintest/r/topn_push_down.result b/cmd/explaintest/r/topn_push_down.result index a92ad36b9b19c..e8ba90678635a 100644 --- a/cmd/explaintest/r/topn_push_down.result +++ b/cmd/explaintest/r/topn_push_down.result @@ -169,21 +169,21 @@ LIMIT 0, 5; id count task operator info Projection_13 0.00 root te.expect_time └─Limit_19 0.00 root offset:0, count:5 - └─IndexJoin_104 0.00 root left outer join, inner:IndexReader_103, outer key:tr.id, inner key:p.relate_id - ├─TopN_107 0.00 root te.expect_time:asc, offset:0, count:5 + └─IndexJoin_92 0.00 root left outer join, inner:IndexReader_91, outer key:tr.id, inner key:p.relate_id + ├─TopN_95 0.00 root te.expect_time:asc, offset:0, count:5 │ └─IndexJoin_36 0.00 root inner join, inner:IndexLookUp_35, outer key:tr.id, inner key:te.trade_id - │ ├─IndexLookUp_85 0.00 root - │ │ ├─Selection_83 0.00 cop eq(tr.business_type, 18), in(tr.trade_type, 1) - │ │ │ └─IndexScan_81 10.00 cop table:tr, index:shop_identy, trade_status, business_type, trade_pay_status, trade_type, delivery_type, source, biz_date, range:[810094178,810094178], keep order:false, stats:pseudo - │ │ └─Selection_84 0.00 cop eq(tr.brand_identy, 32314), eq(tr.domain_type, 2) - │ │ └─TableScan_82 0.00 cop table:tr, keep order:false + │ ├─IndexLookUp_74 0.00 root + │ │ ├─Selection_72 0.00 cop eq(tr.business_type, 18), in(tr.trade_type, 1) + │ │ │ └─IndexScan_70 10.00 cop table:tr, index:shop_identy, trade_status, business_type, trade_pay_status, trade_type, delivery_type, source, biz_date, range:[810094178,810094178], keep order:false, stats:pseudo + │ │ └─Selection_73 0.00 cop eq(tr.brand_identy, 32314), eq(tr.domain_type, 2) + │ │ └─TableScan_71 0.00 cop table:tr, keep order:false, stats:pseudo │ └─IndexLookUp_35 250.00 root - │ ├─IndexScan_32 10.00 cop table:te, index:trade_id, range: decided by [tr.id], keep order:false, stats:pseudo + │ ├─IndexScan_32 10.00 cop table:te, index:trade_id, range: decided by [eq(te.trade_id, tr.id)], keep order:false, stats:pseudo │ └─Selection_34 250.00 cop ge(te.expect_time, 2018-04-23 00:00:00.000000), le(te.expect_time, 2018-04-23 23:59:59.000000) │ └─TableScan_33 10.00 cop table:te, keep order:false, stats:pseudo - └─IndexReader_103 0.00 root index:Selection_102 - └─Selection_102 0.00 cop not(isnull(p.relate_id)) - └─IndexScan_101 10.00 cop table:p, index:relate_id, range: decided by [tr.id], keep order:false, stats:pseudo + └─IndexReader_91 0.00 root index:Selection_90 + └─Selection_90 0.00 cop not(isnull(p.relate_id)) + └─IndexScan_89 10.00 cop table:p, index:relate_id, range: decided by [eq(p.relate_id, tr.id)], keep order:false, stats:pseudo desc select 1 as a from dual order by a limit 1; id count task operator info Projection_6 1.00 root 1 @@ -226,7 +226,7 @@ Limit_11 5.00 root offset:0, count:5 ├─TableReader_17 4.00 root data:TableScan_16 │ └─TableScan_16 4.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─IndexReader_14 10.00 root index:IndexScan_13 - └─IndexScan_13 10.00 cop table:t2, index:a, range: decided by [t1.a], keep order:false, stats:pseudo + └─IndexScan_13 10.00 cop table:t2, index:a, range: decided by [eq(t2.a, t1.a)], keep order:false, stats:pseudo explain select /*+ TIDB_INLJ(t2) */ * from t t1 left join t t2 on t1.a = t2.a where t2.a is null limit 5; id count task operator info Limit_12 5.00 root offset:0, count:5 @@ -235,7 +235,7 @@ Limit_12 5.00 root offset:0, count:5 ├─TableReader_19 4.00 root data:TableScan_18 │ └─TableScan_18 4.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─IndexReader_16 10.00 root index:IndexScan_15 - └─IndexScan_15 10.00 cop table:t2, index:a, range: decided by [t1.a], keep order:false, stats:pseudo + └─IndexScan_15 10.00 cop table:t2, index:a, range: decided by [eq(t2.a, t1.a)], keep order:false, stats:pseudo explain select /*+ TIDB_SMJ(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a limit 5; id count task operator info Limit_11 5.00 root offset:0, count:5 diff --git a/cmd/explaintest/r/tpch.result b/cmd/explaintest/r/tpch.result index 21a9c49500f38..a083977411ea6 100644 --- a/cmd/explaintest/r/tpch.result +++ b/cmd/explaintest/r/tpch.result @@ -251,17 +251,17 @@ id count task operator info Projection_14 10.00 root tpch.lineitem.l_orderkey, 7_col_0, tpch.orders.o_orderdate, tpch.orders.o_shippriority └─TopN_17 10.00 root 7_col_0:desc, tpch.orders.o_orderdate:asc, offset:0, count:10 └─HashAgg_23 40227041.09 root group by:col_4, col_5, col_6, funcs:sum(col_0), firstrow(col_1), firstrow(col_2), firstrow(col_3) - └─Projection_60 91515927.49 root mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), tpch.orders.o_orderdate, tpch.orders.o_shippriority, tpch.lineitem.l_orderkey, tpch.lineitem.l_orderkey, tpch.orders.o_orderdate, tpch.orders.o_shippriority + └─Projection_59 91515927.49 root mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), tpch.orders.o_orderdate, tpch.orders.o_shippriority, tpch.lineitem.l_orderkey, tpch.lineitem.l_orderkey, tpch.orders.o_orderdate, tpch.orders.o_shippriority └─IndexJoin_29 91515927.49 root inner join, inner:IndexLookUp_28, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey - ├─HashRightJoin_50 22592975.51 root inner join, inner:TableReader_56, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] - │ ├─TableReader_56 1498236.00 root data:Selection_55 - │ │ └─Selection_55 1498236.00 cop eq(tpch.customer.c_mktsegment, "AUTOMOBILE") - │ │ └─TableScan_54 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false - │ └─TableReader_53 36870000.00 root data:Selection_52 - │ └─Selection_52 36870000.00 cop lt(tpch.orders.o_orderdate, 1995-03-13 00:00:00.000000) - │ └─TableScan_51 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false + ├─HashRightJoin_49 22592975.51 root inner join, inner:TableReader_55, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] + │ ├─TableReader_55 1498236.00 root data:Selection_54 + │ │ └─Selection_54 1498236.00 cop eq(tpch.customer.c_mktsegment, "AUTOMOBILE") + │ │ └─TableScan_53 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false + │ └─TableReader_52 36870000.00 root data:Selection_51 + │ └─Selection_51 36870000.00 cop lt(tpch.orders.o_orderdate, 1995-03-13 00:00:00.000000) + │ └─TableScan_50 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false └─IndexLookUp_28 162945114.27 root - ├─IndexScan_25 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + ├─IndexScan_25 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false └─Selection_27 162945114.27 cop gt(tpch.lineitem.l_shipdate, 1995-03-13 00:00:00.000000) └─TableScan_26 1.00 cop table:lineitem, keep order:false /* @@ -298,11 +298,11 @@ Sort_10 1.00 root tpch.orders.o_orderpriority:asc └─Projection_12 1.00 root tpch.orders.o_orderpriority, 7_col_0 └─HashAgg_15 1.00 root group by:tpch.orders.o_orderpriority, funcs:count(1), firstrow(tpch.orders.o_orderpriority) └─IndexJoin_21 2340750.00 root semi join, inner:IndexLookUp_20, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey - ├─TableReader_34 2925937.50 root data:Selection_33 - │ └─Selection_33 2925937.50 cop ge(tpch.orders.o_orderdate, 1995-01-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1995-04-01) - │ └─TableScan_32 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false + ├─TableReader_33 2925937.50 root data:Selection_32 + │ └─Selection_32 2925937.50 cop ge(tpch.orders.o_orderdate, 1995-01-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1995-04-01) + │ └─TableScan_31 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false └─IndexLookUp_20 240004648.80 root - ├─IndexScan_17 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + ├─IndexScan_17 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false └─Selection_19 240004648.80 cop lt(tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate) └─TableScan_18 1.00 cop table:lineitem, keep order:false /* @@ -360,8 +360,8 @@ Sort_23 5.00 root revenue:desc │ │ │ └─TableScan_56 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false │ │ └─TableReader_59 300005811.00 root data:TableScan_58 │ │ └─TableScan_58 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false - │ └─TableReader_37 11822812.50 root data:Selection_36 - │ └─Selection_36 11822812.50 cop ge(tpch.orders.o_orderdate, 1994-01-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1995-01-01) + │ └─TableReader_37 0.80 root data:Selection_36 + │ └─Selection_36 0.80 cop ge(tpch.orders.o_orderdate, 1994-01-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1995-01-01) │ └─TableScan_35 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false └─TableReader_30 1.00 root data:TableScan_29 └─TableScan_29 1.00 cop table:customer, range: decided by [tpch.supplier.s_nationkey tpch.orders.o_custkey], keep order:false @@ -538,7 +538,7 @@ Sort_29 718.01 root all_nations.o_year:asc │ │ │ │ └─Selection_77 22382008.93 cop ge(tpch.orders.o_orderdate, 1995-01-01 00:00:00.000000), le(tpch.orders.o_orderdate, 1996-12-31 00:00:00.000000) │ │ │ │ └─TableScan_76 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false │ │ │ └─IndexLookUp_55 1.00 root - │ │ │ ├─IndexScan_53 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + │ │ │ ├─IndexScan_53 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false │ │ │ └─TableScan_54 1.00 cop table:lineitem, keep order:false │ │ └─TableReader_83 61674.00 root data:Selection_82 │ │ └─Selection_82 61674.00 cop eq(tpch.part.p_type, "SMALL PLATED COPPER") @@ -612,7 +612,7 @@ Sort_25 2406.00 root profit.nation:asc, profit.o_year:desc │ └─TableReader_40 1.00 root data:TableScan_39 │ └─TableScan_39 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false └─IndexLookUp_34 1.00 root - ├─IndexScan_32 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.lineitem.l_suppkey tpch.lineitem.l_partkey], keep order:false + ├─IndexScan_32 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [eq(tpch.partsupp.ps_partkey, tpch.lineitem.l_partkey) eq(tpch.partsupp.ps_suppkey, tpch.lineitem.l_suppkey)], keep order:false └─TableScan_33 1.00 cop table:partsupp, keep order:false /* Q10 Returned Item Reporting Query @@ -673,7 +673,7 @@ Projection_17 20.00 root tpch.customer.c_custkey, tpch.customer.c_name, 9_col_0, │ └─Selection_47 3017307.69 cop ge(tpch.orders.o_orderdate, 1993-08-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1993-11-01) │ └─TableScan_46 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false └─IndexLookUp_31 73916005.00 root - ├─IndexScan_28 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + ├─IndexScan_28 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false └─Selection_30 73916005.00 cop eq(tpch.lineitem.l_returnflag, "R") └─TableScan_29 1.00 cop table:lineitem, keep order:false /* @@ -769,11 +769,11 @@ id count task operator info Sort_9 1.00 root tpch.lineitem.l_shipmode:asc └─Projection_11 1.00 root tpch.lineitem.l_shipmode, 5_col_0, 5_col_1 └─HashAgg_14 1.00 root group by:col_3, funcs:sum(col_0), sum(col_1), firstrow(col_2) - └─Projection_40 10023369.01 root cast(case(or(eq(tpch.orders.o_orderpriority, "1-URGENT"), eq(tpch.orders.o_orderpriority, "2-HIGH")), 1, 0)), cast(case(and(ne(tpch.orders.o_orderpriority, "1-URGENT"), ne(tpch.orders.o_orderpriority, "2-HIGH")), 1, 0)), tpch.lineitem.l_shipmode, tpch.lineitem.l_shipmode + └─Projection_39 10023369.01 root cast(case(or(eq(tpch.orders.o_orderpriority, "1-URGENT"), eq(tpch.orders.o_orderpriority, "2-HIGH")), 1, 0)), cast(case(and(ne(tpch.orders.o_orderpriority, "1-URGENT"), ne(tpch.orders.o_orderpriority, "2-HIGH")), 1, 0)), tpch.lineitem.l_shipmode, tpch.lineitem.l_shipmode └─IndexJoin_18 10023369.01 root inner join, inner:TableReader_17, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey - ├─TableReader_36 10023369.01 root data:Selection_35 - │ └─Selection_35 10023369.01 cop ge(tpch.lineitem.l_receiptdate, 1997-01-01 00:00:00.000000), in(tpch.lineitem.l_shipmode, "RAIL", "FOB"), lt(tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate), lt(tpch.lineitem.l_receiptdate, 1998-01-01), lt(tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate) - │ └─TableScan_34 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false + ├─TableReader_35 10023369.01 root data:Selection_34 + │ └─Selection_34 10023369.01 cop ge(tpch.lineitem.l_receiptdate, 1997-01-01 00:00:00.000000), in(tpch.lineitem.l_shipmode, "RAIL", "FOB"), lt(tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate), lt(tpch.lineitem.l_receiptdate, 1998-01-01), lt(tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate) + │ └─TableScan_33 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false └─TableReader_17 1.00 root data:TableScan_16 └─TableScan_16 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false /* @@ -936,7 +936,7 @@ Sort_13 3863988.24 root supplier_cnt:desc, tpch.part.p_brand:asc, tpch.part.p_ty │ │ └─Selection_40 1200618.43 cop in(tpch.part.p_size, 48, 19, 12, 4, 41, 7, 21, 39), ne(tpch.part.p_brand, "Brand#34"), not(like(tpch.part.p_type, "LARGE BRUSHED%", 92)) │ │ └─TableScan_39 10000000.00 cop table:part, range:[-inf,+inf], keep order:false │ └─IndexReader_26 1.00 root index:IndexScan_25 - │ └─IndexScan_25 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false + │ └─IndexScan_25 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [eq(tpch.partsupp.ps_partkey, tpch.part.p_partkey)], keep order:false └─TableReader_46 400000.00 root data:Selection_45 └─Selection_45 400000.00 cop like(tpch.supplier.s_comment, "%Customer%Complaints%", 92) └─TableScan_44 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false @@ -1042,7 +1042,7 @@ Projection_24 100.00 root tpch.customer.c_name, tpch.customer.c_custkey, tpch.or │ └─HashAgg_53 74063872.00 cop group by:tpch.lineitem.l_orderkey, funcs:sum(tpch.lineitem.l_quantity), firstrow(tpch.lineitem.l_orderkey) │ └─TableScan_58 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false └─IndexLookUp_37 1.00 root - ├─IndexScan_35 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + ├─IndexScan_35 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false └─TableScan_36 1.00 cop table:lineitem, keep order:false /* Q19 Discounted Revenue Query @@ -1096,8 +1096,8 @@ StreamAgg_13 1.00 root funcs:sum(col_0) ├─TableReader_22 6286493.79 root data:Selection_21 │ └─Selection_21 6286493.79 cop eq(tpch.lineitem.l_shipinstruct, "DELIVER IN PERSON"), in(tpch.lineitem.l_shipmode, "AIR", "AIR REG"), or(and(ge(tpch.lineitem.l_quantity, 4), le(tpch.lineitem.l_quantity, 14)), or(and(ge(tpch.lineitem.l_quantity, 18), le(tpch.lineitem.l_quantity, 28)), and(ge(tpch.lineitem.l_quantity, 29), le(tpch.lineitem.l_quantity, 39)))) │ └─TableScan_20 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false - └─TableReader_28 8000000.00 root data:Selection_27 - └─Selection_27 8000000.00 cop ge(tpch.part.p_size, 1), or(and(eq(tpch.part.p_brand, "Brand#52"), and(in(tpch.part.p_container, "SM CASE", "SM BOX", "SM PACK", "SM PKG"), le(tpch.part.p_size, 5))), or(and(eq(tpch.part.p_brand, "Brand#11"), and(in(tpch.part.p_container, "MED BAG", "MED BOX", "MED PKG", "MED PACK"), le(tpch.part.p_size, 10))), and(eq(tpch.part.p_brand, "Brand#51"), and(in(tpch.part.p_container, "LG CASE", "LG BOX", "LG PACK", "LG PKG"), le(tpch.part.p_size, 15))))) + └─TableReader_28 0.80 root data:Selection_27 + └─Selection_27 0.80 cop ge(tpch.part.p_size, 1), or(and(eq(tpch.part.p_brand, "Brand#52"), and(in(tpch.part.p_container, "SM CASE", "SM BOX", "SM PACK", "SM PKG"), le(tpch.part.p_size, 5))), or(and(eq(tpch.part.p_brand, "Brand#11"), and(in(tpch.part.p_container, "MED BAG", "MED BOX", "MED PKG", "MED PACK"), le(tpch.part.p_size, 10))), and(eq(tpch.part.p_brand, "Brand#51"), and(in(tpch.part.p_container, "LG CASE", "LG BOX", "LG PACK", "LG PKG"), le(tpch.part.p_size, 15))))) └─TableScan_26 1.00 cop table:part, range: decided by [tpch.lineitem.l_partkey], keep order:false /* Q20 Potential Part Promotion Query @@ -1159,17 +1159,17 @@ Sort_28 20000.00 root tpch.supplier.s_name:asc └─Projection_47 64006.34 root tpch.partsupp.ps_partkey, tpch.partsupp.ps_suppkey, tpch.partsupp.ps_availqty, tpch.part.p_partkey, mul(0.5, 14_col_0) └─Selection_48 64006.34 root gt(cast(tpch.partsupp.ps_availqty), mul(0.5, 14_col_0)) └─HashAgg_51 80007.93 root group by:tpch.partsupp.ps_partkey, tpch.partsupp.ps_suppkey, funcs:firstrow(tpch.partsupp.ps_partkey), firstrow(tpch.partsupp.ps_suppkey), firstrow(tpch.partsupp.ps_availqty), firstrow(tpch.part.p_partkey), sum(tpch.lineitem.l_quantity) - └─HashLeftJoin_54 9711455.06 root left outer join, inner:TableReader_81, equal:[eq(tpch.partsupp.ps_partkey, tpch.lineitem.l_partkey) eq(tpch.partsupp.ps_suppkey, tpch.lineitem.l_suppkey)] + └─HashLeftJoin_54 9711455.06 root left outer join, inner:TableReader_80, equal:[eq(tpch.partsupp.ps_partkey, tpch.lineitem.l_partkey) eq(tpch.partsupp.ps_suppkey, tpch.lineitem.l_suppkey)] ├─IndexJoin_59 321865.05 root inner join, inner:IndexLookUp_58, outer key:tpch.part.p_partkey, inner key:tpch.partsupp.ps_partkey - │ ├─TableReader_76 80007.93 root data:Selection_75 - │ │ └─Selection_75 80007.93 cop like(tpch.part.p_name, "green%", 92) - │ │ └─TableScan_74 10000000.00 cop table:part, range:[-inf,+inf], keep order:false + │ ├─TableReader_75 80007.93 root data:Selection_74 + │ │ └─Selection_74 80007.93 cop like(tpch.part.p_name, "green%", 92) + │ │ └─TableScan_73 10000000.00 cop table:part, range:[-inf,+inf], keep order:false │ └─IndexLookUp_58 1.00 root - │ ├─IndexScan_56 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false + │ ├─IndexScan_56 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [eq(tpch.partsupp.ps_partkey, tpch.part.p_partkey)], keep order:false │ └─TableScan_57 1.00 cop table:partsupp, keep order:false - └─TableReader_81 44189356.65 root data:Selection_80 - └─Selection_80 44189356.65 cop ge(tpch.lineitem.l_shipdate, 1993-01-01 00:00:00.000000), lt(tpch.lineitem.l_shipdate, 1994-01-01) - └─TableScan_79 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false + └─TableReader_80 44189356.65 root data:Selection_79 + └─Selection_79 44189356.65 cop ge(tpch.lineitem.l_shipdate, 1993-01-01 00:00:00.000000), lt(tpch.lineitem.l_shipdate, 1994-01-01) + └─TableScan_78 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false /* Q21 Suppliers Who Kept Orders Waiting Query This query identifies certain suppliers who were not able to ship required parts in a timely manner. @@ -1235,14 +1235,14 @@ Projection_25 1.00 root tpch.supplier.s_name, 17_col_0 │ │ │ └─TableReader_80 240004648.80 root data:Selection_79 │ │ │ └─Selection_79 240004648.80 cop gt(l1.l_receiptdate, l1.l_commitdate) │ │ │ └─TableScan_78 300005811.00 cop table:l1, range:[-inf,+inf], keep order:false - │ │ └─TableReader_61 36517371.00 root data:Selection_60 - │ │ └─Selection_60 36517371.00 cop eq(tpch.orders.o_orderstatus, "F") + │ │ └─TableReader_61 0.80 root data:Selection_60 + │ │ └─Selection_60 0.80 cop eq(tpch.orders.o_orderstatus, "F") │ │ └─TableScan_59 1.00 cop table:orders, range: decided by [l1.l_orderkey], keep order:false │ └─IndexLookUp_55 1.00 root - │ ├─IndexScan_53 1.00 cop table:l2, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false + │ ├─IndexScan_53 1.00 cop table:l2, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(l2.l_orderkey, l1.l_orderkey)], keep order:false │ └─TableScan_54 1.00 cop table:lineitem, keep order:false └─IndexLookUp_39 240004648.80 root - ├─IndexScan_36 1.00 cop table:l3, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false + ├─IndexScan_36 1.00 cop table:l3, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(l3.l_orderkey, l1.l_orderkey)], keep order:false └─Selection_38 240004648.80 cop gt(l3.l_receiptdate, l3.l_commitdate) └─TableScan_37 1.00 cop table:lineitem, keep order:false /* diff --git a/cmd/explaintest/r/window_function.result b/cmd/explaintest/r/window_function.result index 7fc186eb98260..3f4e6132897fe 100644 --- a/cmd/explaintest/r/window_function.result +++ b/cmd/explaintest/r/window_function.result @@ -18,34 +18,34 @@ explain select sum(a) over(partition by a order by b) from t; id count task operator info Projection_7 10000.00 root sum(a) over(partition by a order by b) └─Window_8 10000.00 root sum(cast(test.t.a)) over(partition by test.t.a order by test.t.b asc range between unbounded preceding and current row) - └─Sort_12 10000.00 root test.t.a:asc, test.t.b:asc - └─TableReader_11 10000.00 root data:TableScan_10 - └─TableScan_10 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo + └─Sort_11 10000.00 root test.t.a:asc, test.t.b:asc + └─TableReader_10 10000.00 root data:TableScan_9 + └─TableScan_9 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo explain select sum(a) over(partition by a order by b rows unbounded preceding) from t; id count task operator info Projection_7 10000.00 root sum(a) over(partition by a order by b rows unbounded preceding) └─Window_8 10000.00 root sum(cast(test.t.a)) over(partition by test.t.a order by test.t.b asc rows between unbounded preceding and current row) - └─Sort_12 10000.00 root test.t.a:asc, test.t.b:asc - └─TableReader_11 10000.00 root data:TableScan_10 - └─TableScan_10 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo + └─Sort_11 10000.00 root test.t.a:asc, test.t.b:asc + └─TableReader_10 10000.00 root data:TableScan_9 + └─TableScan_9 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo explain select sum(a) over(partition by a order by b rows between 1 preceding and 1 following) from t; id count task operator info Projection_7 10000.00 root sum(a) over(partition by a order by b rows between 1 preceding and 1 following) └─Window_8 10000.00 root sum(cast(test.t.a)) over(partition by test.t.a order by test.t.b asc rows between 1 preceding and 1 following) - └─Sort_12 10000.00 root test.t.a:asc, test.t.b:asc - └─TableReader_11 10000.00 root data:TableScan_10 - └─TableScan_10 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo + └─Sort_11 10000.00 root test.t.a:asc, test.t.b:asc + └─TableReader_10 10000.00 root data:TableScan_9 + └─TableScan_9 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo explain select sum(a) over(partition by a order by b range between 1 preceding and 1 following) from t; id count task operator info Projection_7 10000.00 root sum(a) over(partition by a order by b range between 1 preceding and 1 following) └─Window_8 10000.00 root sum(cast(test.t.a)) over(partition by test.t.a order by test.t.b asc range between 1 preceding and 1 following) - └─Sort_12 10000.00 root test.t.a:asc, test.t.b:asc - └─TableReader_11 10000.00 root data:TableScan_10 - └─TableScan_10 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo + └─Sort_11 10000.00 root test.t.a:asc, test.t.b:asc + └─TableReader_10 10000.00 root data:TableScan_9 + └─TableScan_9 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo explain select sum(a) over(partition by a order by c range between interval '2:30' minute_second preceding and interval '2:30' minute_second following) from t; id count task operator info Projection_7 10000.00 root sum(a) over(partition by a order by c range between interval '2:30' minute_second preceding and interval '2:30' minute_second following) └─Window_8 10000.00 root sum(cast(test.t.a)) over(partition by test.t.a order by test.t.c asc range between interval "2:30" "MINUTE_SECOND" preceding and interval "2:30" "MINUTE_SECOND" following) - └─Sort_12 10000.00 root test.t.a:asc, test.t.c:asc - └─TableReader_11 10000.00 root data:TableScan_10 - └─TableScan_10 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo + └─Sort_11 10000.00 root test.t.a:asc, test.t.c:asc + └─TableReader_10 10000.00 root data:TableScan_9 + └─TableScan_9 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo From 662ef3d00037d549a28b6f8d410734994bd0bee9 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Sun, 28 Apr 2019 20:20:17 +0800 Subject: [PATCH 26/27] address comments --- planner/core/exhaust_physical_plans.go | 13 +------------ 1 file changed, 1 insertion(+), 12 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index cf042cfe3f08d..b03afade02151 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -702,22 +702,11 @@ func (ijHelper *indexJoinBuildHelper) checkIndex(innerKeys []*expression.Column, // remainingRangeCandidates is the other conditions for future use. func (ijHelper *indexJoinBuildHelper) findUsefulEqAndInFilters(innerPlan *DataSource) (usefulEqOrInFilters, uselessFilters, remainingRangeCandidates []expression.Expression) { uselessFilters = make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) - remainingRangeCandidates = make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) - // This loop finds the possible filters which can be used to build ranges. - // If the filter contains index column covered by join keys, it will be useless since we always use join key to build range for that index column.. - for _, innerFilter := range innerPlan.pushedDownConds { - affectedCols := expression.ExtractColumns(innerFilter) - if expression.ColumnSliceIsIntersect(affectedCols, ijHelper.curPossibleUsedKeys) { - uselessFilters = append(uselessFilters, innerFilter) - continue - } - remainingRangeCandidates = append(remainingRangeCandidates, innerFilter) - } var remainedEqOrIn []expression.Expression // Extract the eq/in functions of possible join key. // you can see the comment of ExtractEqAndInCondition to get the meaning of the second return value. usefulEqOrInFilters, remainedEqOrIn, remainingRangeCandidates, _ = ranger.ExtractEqAndInCondition( - innerPlan.ctx, remainingRangeCandidates, + innerPlan.ctx, innerPlan.pushedDownConds, ijHelper.curNotUsedIndexCols, ijHelper.curNotUsedColLens, ) From cba3da9ad048fa6bb1312b9aa499f6f062bb2f7a Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Sun, 28 Apr 2019 20:26:42 +0800 Subject: [PATCH 27/27] fix unit test --- planner/core/cbo_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index 0faa3bf72296c..3a739fb9028c5 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -1036,7 +1036,7 @@ func (s *testAnalyzeSuite) TestLimitCrossEstimation(c *C) { " │ └─TopN_30 1.00 cop t1.a:asc, offset:0, count:1", " │ └─IndexScan_29 6.00 cop table:t1, index:b, range:[-inf,6], keep order:false", " └─IndexReader_57 1.04 root index:IndexScan_56", - " └─IndexScan_56 1.04 cop table:t2, index:b, range: decided by [t1.a], keep order:false", + " └─IndexScan_56 1.04 cop table:t2, index:b, range: decided by [eq(t2.b, t1.a)], keep order:false", )) // Desc TableScan. tk.MustExec("truncate table t")