Skip to content

Commit

Permalink
planner: unify the argument of range functions to use SessionCtx in…
Browse files Browse the repository at this point in the history
…stead of `StatementContext` (#29999)
  • Loading branch information
qw4990 authored Nov 23, 2021
1 parent eb4bc5d commit d15ca2a
Show file tree
Hide file tree
Showing 15 changed files with 83 additions and 106 deletions.
4 changes: 2 additions & 2 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -4052,7 +4052,7 @@ func buildRangesForIndexJoin(ctx sessionctx.Context, lookUpContents []*indexJoin
return retRanges, nil
}

return ranger.UnionRanges(ctx.GetSessionVars().StmtCtx, tmpDatumRanges, true)
return ranger.UnionRanges(ctx, tmpDatumRanges, true)
}

// buildKvRangesForIndexJoin builds kv ranges for index join when the inner plan is index scan plan.
Expand Down Expand Up @@ -4106,7 +4106,7 @@ func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, l
return kvRanges, nil
}

tmpDatumRanges, err = ranger.UnionRanges(ctx.GetSessionVars().StmtCtx, tmpDatumRanges, true)
tmpDatumRanges, err = ranger.UnionRanges(ctx, tmpDatumRanges, true)
if err != nil {
return nil, err
}
Expand Down
10 changes: 5 additions & 5 deletions planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -621,7 +621,7 @@ func (e *Execute) rebuildRange(p Plan) error {
}
}
if pkCol != nil {
ranges, err := ranger.BuildTableRange(x.AccessConditions, x.ctx.GetSessionVars().StmtCtx, pkCol.RetType)
ranges, err := ranger.BuildTableRange(x.AccessConditions, x.ctx, pkCol.RetType)
if err != nil {
return err
}
Expand Down Expand Up @@ -671,7 +671,7 @@ func (e *Execute) rebuildRange(p Plan) error {
}
}
if pkCol != nil {
ranges, err := ranger.BuildTableRange(x.AccessConditions, x.ctx.GetSessionVars().StmtCtx, pkCol.RetType)
ranges, err := ranger.BuildTableRange(x.AccessConditions, x.ctx, pkCol.RetType)
if err != nil {
return err
}
Expand Down Expand Up @@ -774,7 +774,7 @@ func (e *Execute) buildRangeForTableScan(sctx sessionctx.Context, ts *PhysicalTa
}
}
if pkCol != nil {
ts.Ranges, err = ranger.BuildTableRange(ts.AccessCondition, sctx.GetSessionVars().StmtCtx, pkCol.RetType)
ts.Ranges, err = ranger.BuildTableRange(ts.AccessCondition, sctx, pkCol.RetType)
if err != nil {
return err
}
Expand Down Expand Up @@ -1484,10 +1484,10 @@ func IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx sessionctx.Context, p Plan) (bo
switch v := p.(type) {
case *PhysicalIndexReader:
indexScan := v.IndexPlans[0].(*PhysicalIndexScan)
return indexScan.IsPointGetByUniqueKey(ctx.GetSessionVars().StmtCtx), nil
return indexScan.IsPointGetByUniqueKey(ctx), nil
case *PhysicalTableReader:
tableScan := v.TablePlans[0].(*PhysicalTableScan)
isPointRange := len(tableScan.Ranges) == 1 && tableScan.Ranges[0].IsPoint(ctx.GetSessionVars().StmtCtx)
isPointRange := len(tableScan.Ranges) == 1 && tableScan.Ranges[0].IsPoint(ctx)
if !isPointRange {
return false, nil
}
Expand Down
7 changes: 3 additions & 4 deletions planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -1206,7 +1206,7 @@ func (cwc *ColWithCmpFuncManager) BuildRangesByRow(ctx sessionctx.Context, row c
}
exprs = append(exprs, newExpr) // nozero
}
ranges, err := ranger.BuildColumnRange(exprs, ctx.GetSessionVars().StmtCtx, cwc.TargetCol.RetType, cwc.colLength)
ranges, err := ranger.BuildColumnRange(exprs, ctx, cwc.TargetCol.RetType, cwc.colLength)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -1453,7 +1453,7 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *util.AccessPath
var ranges, nextColRange []*ranger.Range
var err error
if len(colAccesses) > 0 {
nextColRange, err = ranger.BuildColumnRange(colAccesses, ijHelper.join.ctx.GetSessionVars().StmtCtx, lastPossibleCol.RetType, path.IdxColLens[lastColPos])
nextColRange, err = ranger.BuildColumnRange(colAccesses, ijHelper.join.ctx, lastPossibleCol.RetType, path.IdxColLens[lastColPos])
if err != nil {
return false, err
}
Expand Down Expand Up @@ -1555,14 +1555,13 @@ func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAn
HighVal: make([]types.Datum, pointLength),
})
}
sc := ijHelper.join.ctx.GetSessionVars().StmtCtx
for i, j := 0, 0; j < len(eqAndInFuncs); i++ {
// This position is occupied by join key.
if ijHelper.curIdxOff2KeyOff[i] != -1 {
continue
}
exprs := []expression.Expression{eqAndInFuncs[j]}
oneColumnRan, err := ranger.BuildColumnRange(exprs, sc, ijHelper.curNotUsedIndexCols[j].RetType, ijHelper.curNotUsedColLens[j])
oneColumnRan, err := ranger.BuildColumnRange(exprs, ijHelper.join.ctx, ijHelper.curNotUsedIndexCols[j].RetType, ijHelper.curNotUsedColLens[j])
if err != nil {
return nil, false, err
}
Expand Down
4 changes: 2 additions & 2 deletions planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -860,7 +860,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter
if canConvertPointGet {
allRangeIsPoint := true
for _, ran := range path.Ranges {
if !ran.IsPoint(ds.ctx.GetSessionVars().StmtCtx) {
if !ran.IsPoint(ds.ctx) {
allRangeIsPoint = false
break
}
Expand Down Expand Up @@ -1581,7 +1581,7 @@ func (ds *DataSource) crossEstimateRowCount(path *util.AccessPath, conds []expre
return 0, false, corr
}
sc := ds.ctx.GetSessionVars().StmtCtx
ranges, err := ranger.BuildColumnRange(accessConds, sc, col.RetType, types.UnspecifiedLength)
ranges, err := ranger.BuildColumnRange(accessConds, ds.ctx, col.RetType, types.UnspecifiedLength)
if len(ranges) == 0 || err != nil {
return 0, err == nil, corr
}
Expand Down
2 changes: 1 addition & 1 deletion planner/core/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -844,7 +844,7 @@ func (ds *DataSource) deriveTablePathStats(path *util.AccessPath, conds []expres
path.CountAfterAccess = 1
return nil
}
path.Ranges, err = ranger.BuildTableRange(path.AccessConds, sc, pkCol.RetType)
path.Ranges, err = ranger.BuildTableRange(path.AccessConds, ds.ctx, pkCol.RetType)
if err != nil {
return err
}
Expand Down
7 changes: 3 additions & 4 deletions planner/core/physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@ import (
"github.com/pingcap/tidb/planner/property"
"github.com/pingcap/tidb/planner/util"
"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/table/tables"
Expand Down Expand Up @@ -548,7 +547,7 @@ func (ts *PhysicalTableScan) ResolveCorrelatedColumns() ([]*ranger.Range, error)
} else {
var err error
pkTP := ts.Table.GetPkColInfo().FieldType
ts.Ranges, err = ranger.BuildTableRange(access, ts.SCtx().GetSessionVars().StmtCtx, &pkTP)
ts.Ranges, err = ranger.BuildTableRange(access, ts.SCtx(), &pkTP)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -1198,11 +1197,11 @@ func (p *PhysicalIndexScan) IsPartition() (bool, int64) {
}

// IsPointGetByUniqueKey checks whether is a point get by unique key.
func (p *PhysicalIndexScan) IsPointGetByUniqueKey(sc *stmtctx.StatementContext) bool {
func (p *PhysicalIndexScan) IsPointGetByUniqueKey(sctx sessionctx.Context) bool {
return len(p.Ranges) == 1 &&
p.Index.Unique &&
len(p.Ranges[0].LowVal) == len(p.Index.Columns) &&
p.Ranges[0].IsPoint(sc)
p.Ranges[0].IsPoint(sctx)
}

// PhysicalSelection represents a filter.
Expand Down
6 changes: 3 additions & 3 deletions planner/core/rule_partition_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ func (s *partitionProcessor) findUsedPartitions(ctx sessionctx.Context, tbl tabl
ranges := detachedResult.Ranges
used := make([]int, 0, len(ranges))
for _, r := range ranges {
if r.IsPointNullable(ctx.GetSessionVars().StmtCtx) {
if r.IsPointNullable(ctx) {
if !r.HighVal[0].IsNull() {
if len(r.HighVal) != len(partIdx) {
used = []int{-1}
Expand Down Expand Up @@ -473,7 +473,7 @@ func (l *listPartitionPruner) locateColumnPartitionsByCondition(cond expression.
return nil, true, nil
}
var locations []tables.ListPartitionLocation
if r.IsPointNullable(sc) {
if r.IsPointNullable(l.ctx) {
location, err := colPrune.LocatePartition(sc, r.HighVal[0])
if types.ErrOverflow.Equal(err) {
return nil, true, nil // return full-scan if over-flow
Expand Down Expand Up @@ -555,7 +555,7 @@ func (l *listPartitionPruner) findUsedListPartitions(conds []expression.Expressi
}
used := make(map[int]struct{}, len(ranges))
for _, r := range ranges {
if r.IsPointNullable(l.ctx.GetSessionVars().StmtCtx) {
if r.IsPointNullable(l.ctx) {
if len(r.HighVal) != len(exprCols) {
return l.fullRange, nil
}
Expand Down
9 changes: 4 additions & 5 deletions planner/core/stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -284,7 +284,7 @@ func (ds *DataSource) derivePathStatsAndTryHeuristics() error {
selected = path
break
}
if path.OnlyPointRange(ds.SCtx().GetSessionVars().StmtCtx) {
if path.OnlyPointRange(ds.SCtx()) {
if path.IsTablePath() || path.Index.Unique {
if path.IsSingleScan {
selected = path
Expand Down Expand Up @@ -477,11 +477,10 @@ func (ts *LogicalTableScan) DeriveStats(childStats []*property.StatsInfo, selfSc
ts.AccessConds[i] = expression.PushDownNot(ts.ctx, expr)
}
ts.stats = ts.Source.deriveStatsByFilter(ts.AccessConds, nil)
sc := ts.SCtx().GetSessionVars().StmtCtx
// ts.Handle could be nil if PK is Handle, and PK column has been pruned.
// TODO: support clustered index.
if ts.HandleCols != nil {
ts.Ranges, err = ranger.BuildTableRange(ts.AccessConds, sc, ts.HandleCols.GetCol(0).RetType)
ts.Ranges, err = ranger.BuildTableRange(ts.AccessConds, ts.ctx, ts.HandleCols.GetCol(0).RetType)
} else {
isUnsigned := false
if ts.Source.tableInfo.PKIsHandle {
Expand Down Expand Up @@ -633,7 +632,7 @@ func (ds *DataSource) accessPathsForConds(conditions []expression.Expression, us
continue
}
// If we have point or empty range, just remove other possible paths.
if len(path.Ranges) == 0 || path.OnlyPointRange(ds.SCtx().GetSessionVars().StmtCtx) {
if len(path.Ranges) == 0 || path.OnlyPointRange(ds.SCtx()) {
if len(results) == 0 {
results = append(results, path)
} else {
Expand All @@ -658,7 +657,7 @@ func (ds *DataSource) accessPathsForConds(conditions []expression.Expression, us
continue
}
// If we have empty range, or point range on unique index, just remove other possible paths.
if len(path.Ranges) == 0 || (path.OnlyPointRange(ds.SCtx().GetSessionVars().StmtCtx) && path.Index.Unique) {
if len(path.Ranges) == 0 || (path.OnlyPointRange(ds.SCtx()) && path.Index.Unique) {
if len(results) == 0 {
results = append(results, path)
} else {
Expand Down
7 changes: 3 additions & 4 deletions planner/util/path.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/ranger"
Expand Down Expand Up @@ -151,11 +150,11 @@ func isColEqCorColOrConstant(ctx sessionctx.Context, filter expression.Expressio
}

// OnlyPointRange checks whether each range is a point(no interval range exists).
func (path *AccessPath) OnlyPointRange(sc *stmtctx.StatementContext) bool {
func (path *AccessPath) OnlyPointRange(sctx sessionctx.Context) bool {
noIntervalRange := true
if path.IsIntHandlePath {
for _, ran := range path.Ranges {
if !ran.IsPoint(sc) {
if !ran.IsPoint(sctx) {
noIntervalRange = false
break
}
Expand All @@ -165,7 +164,7 @@ func (path *AccessPath) OnlyPointRange(sc *stmtctx.StatementContext) bool {
haveNullVal := false
for _, ran := range path.Ranges {
// Not point or the not full matched.
if !ran.IsPoint(sc) || len(ran.HighVal) != len(path.Index.Columns) {
if !ran.IsPoint(sctx) || len(ran.HighVal) != len(path.Index.Columns) {
noIntervalRange = false
break
}
Expand Down
3 changes: 1 addition & 2 deletions statistics/selectivity.go
Original file line number Diff line number Diff line change
Expand Up @@ -362,13 +362,12 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp
}

func getMaskAndRanges(ctx sessionctx.Context, exprs []expression.Expression, rangeType ranger.RangeType, lengths []int, cachedPath *planutil.AccessPath, cols ...*expression.Column) (mask int64, ranges []*ranger.Range, partCover bool, err error) {
sc := ctx.GetSessionVars().StmtCtx
isDNF := false
var accessConds, remainedConds []expression.Expression
switch rangeType {
case ranger.ColumnRangeType:
accessConds = ranger.ExtractAccessConditionsForColumn(exprs, cols[0])
ranges, err = ranger.BuildColumnRange(accessConds, sc, cols[0].RetType, types.UnspecifiedLength)
ranges, err = ranger.BuildColumnRange(accessConds, ctx, cols[0].RetType, types.UnspecifiedLength)
case ranger.IndexRangeType:
if cachedPath != nil {
ranges, accessConds, remainedConds, isDNF = cachedPath.Ranges, cachedPath.AccessConds, cachedPath.TableFilters, cachedPath.IsDNFCond
Expand Down
9 changes: 4 additions & 5 deletions util/ranger/detacher.go
Original file line number Diff line number Diff line change
Expand Up @@ -216,7 +216,7 @@ func extractIndexPointRangesForCNF(sctx sessionctx.Context, conds []expression.E
sameLens, allPoints := true, true
numCols := int(0)
for j, ran := range res.Ranges {
if !ran.IsPoint(sctx.GetSessionVars().StmtCtx) {
if !ran.IsPoint(sctx) {
allPoints = false
break
}
Expand Down Expand Up @@ -610,13 +610,12 @@ func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Ex
// detachDNFCondAndBuildRangeForIndex will detach the index filters from table filters when it's a DNF.
// We will detach the conditions of every DNF items, then compose them to a DNF.
func (d *rangeDetacher) detachDNFCondAndBuildRangeForIndex(condition *expression.ScalarFunction, newTpSlice []*types.FieldType) ([]*Range, []expression.Expression, []*valueInfo, bool, error) {
sc := d.sctx.GetSessionVars().StmtCtx
firstColumnChecker := &conditionChecker{
checkerCol: d.cols[0],
shouldReserve: d.lengths[0] != types.UnspecifiedLength,
length: d.lengths[0],
}
rb := builder{sc: sc}
rb := builder{sc: d.sctx.GetSessionVars().StmtCtx}
dnfItems := expression.FlattenDNFConditions(condition)
newAccessItems := make([]expression.Expression, 0, len(dnfItems))
var totalRanges []*Range
Expand Down Expand Up @@ -666,7 +665,7 @@ func (d *rangeDetacher) detachDNFCondAndBuildRangeForIndex(condition *expression
firstColumnChecker.shouldReserve = d.lengths[0] != types.UnspecifiedLength
}
points := rb.build(item)
ranges, err := points2Ranges(sc, points, newTpSlice[0])
ranges, err := points2Ranges(d.sctx, points, newTpSlice[0])
if err != nil {
return nil, nil, nil, false, errors.Trace(err)
}
Expand All @@ -693,7 +692,7 @@ func (d *rangeDetacher) detachDNFCondAndBuildRangeForIndex(condition *expression
if hasPrefix(d.lengths) {
fixPrefixColRange(totalRanges, d.lengths, newTpSlice)
}
totalRanges, err := UnionRanges(sc, totalRanges, d.mergeConsecutive)
totalRanges, err := UnionRanges(d.sctx, totalRanges, d.mergeConsecutive)
if err != nil {
return nil, nil, nil, false, errors.Trace(err)
}
Expand Down
Loading

0 comments on commit d15ca2a

Please sign in to comment.