Skip to content

Commit

Permalink
planner: simplify the indexJoinBuildHelper structure (pingcap#55079)
Browse files Browse the repository at this point in the history
  • Loading branch information
qw4990 committed Jul 31, 2024
1 parent e494895 commit 383643f
Show file tree
Hide file tree
Showing 6 changed files with 90 additions and 68 deletions.
117 changes: 67 additions & 50 deletions pkg/planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/planner/cardinality"
"github.com/pingcap/tidb/pkg/planner/context"
"github.com/pingcap/tidb/pkg/planner/core/base"
"github.com/pingcap/tidb/pkg/planner/core/cost"
"github.com/pingcap/tidb/pkg/planner/core/operator/logicalop"
Expand Down Expand Up @@ -553,7 +554,7 @@ func constructIndexJoin(

join := PhysicalIndexJoin{
basePhysicalJoin: baseJoin,
innerTask: innerTask,
innerPlan: innerTask.Plan(),
KeyOff2IdxOff: newKeyOff,
Ranges: ranges,
CompareFilters: compareFilters,
Expand Down Expand Up @@ -790,12 +791,17 @@ childLoop:

func getIndexJoinBuildHelper(p *LogicalJoin, ds *DataSource, innerJoinKeys []*expression.Column, checkPathValid func(path *util.AccessPath) bool, outerJoinKeys []*expression.Column) (*indexJoinBuildHelper, []int) {
helper := &indexJoinBuildHelper{
join: p,
innerPlan: ds,
sctx: p.SCtx(),
joinOtherConditions: p.OtherConditions,
outerJoinKeys: outerJoinKeys,
innerJoinKeys: innerJoinKeys,
innerPushedConditions: ds.PushedDownConds,
innerSchema: ds.Schema(),
innerStats: ds.StatsInfo(),
}
for _, path := range ds.PossibleAccessPaths {
if checkPathValid(path) {
emptyRange, err := helper.analyzeLookUpFilters(path, ds, innerJoinKeys, outerJoinKeys, false)
emptyRange, err := helper.analyzeLookUpFilters(path, false)
if emptyRange {
return nil, nil
}
Expand Down Expand Up @@ -849,7 +855,7 @@ func buildIndexJoinInner2TableScan(
if helper == nil {
return nil
}
rangeInfo := helper.buildRangeDecidedByInformation(helper.chosenPath.IdxCols, outerJoinKeys)
rangeInfo := helper.buildRangeDecidedByInformation(helper.chosenPath.IdxCols)
innerTask = constructInnerTableScanTask(p, wrapper, helper.chosenRanges.Range(), outerJoinKeys, rangeInfo, false, false, avgInnerRowCnt)
// The index merge join's inner plan is different from index join, so we
// should construct another inner plan for it.
Expand Down Expand Up @@ -947,7 +953,7 @@ func buildIndexJoinInner2IndexScan(
return nil
}
joins = make([]base.PhysicalPlan, 0, 3)
rangeInfo := helper.buildRangeDecidedByInformation(helper.chosenPath.IdxCols, outerJoinKeys)
rangeInfo := helper.buildRangeDecidedByInformation(helper.chosenPath.IdxCols)
maxOneRow := false
if helper.chosenPath.Index.Unique && helper.usedColsLen == len(helper.chosenPath.FullIdxCols) {
l := len(helper.chosenAccess)
Expand Down Expand Up @@ -984,9 +990,16 @@ func buildIndexJoinInner2IndexScan(
}

type indexJoinBuildHelper struct {
join *LogicalJoin
innerPlan *DataSource

// read-only fields, information of the outer child
sctx context.PlanContext
joinOtherConditions []expression.Expression
outerJoinKeys []*expression.Column
innerJoinKeys []*expression.Column
innerSchema *expression.Schema
innerPushedConditions []expression.Expression
innerStats *property.StatsInfo

// below is mutable fields
usedColsLen int
usedColsNDV float64
chosenAccess []expression.Expression
Expand All @@ -1003,7 +1016,7 @@ type indexJoinBuildHelper struct {
curIdxOff2KeyOff []int
}

func (ijHelper *indexJoinBuildHelper) buildRangeDecidedByInformation(idxCols []*expression.Column, outerJoinKeys []*expression.Column) string {
func (ijHelper *indexJoinBuildHelper) buildRangeDecidedByInformation(idxCols []*expression.Column) string {
buffer := bytes.NewBufferString("[")
isFirst := true
for idxOff, keyOff := range ijHelper.idxOff2KeyOff {
Expand All @@ -1015,9 +1028,9 @@ func (ijHelper *indexJoinBuildHelper) buildRangeDecidedByInformation(idxCols []*
} else {
isFirst = false
}
fmt.Fprintf(buffer, "eq(%v, %v)", idxCols[idxOff], outerJoinKeys[keyOff])
fmt.Fprintf(buffer, "eq(%v, %v)", idxCols[idxOff], ijHelper.outerJoinKeys[keyOff])
}
ectx := ijHelper.join.SCtx().GetExprCtx().GetEvalCtx()
ectx := ijHelper.sctx.GetExprCtx().GetEvalCtx()
// It is to build the range info which is used in explain. It is necessary to redact the range info.
redact := ectx.GetTiDBRedactLog()
for _, access := range ijHelper.chosenAccess {
Expand Down Expand Up @@ -1648,17 +1661,17 @@ For each idxCols,
*/
// For example, innerKeys[t1.a, t1.sum_b, t1.c], idxCols [a, b, c]
// 'curIdxOff2KeyOff' = [0, -1, 2]
func (ijHelper *indexJoinBuildHelper) resetContextForIndex(innerKeys []*expression.Column, idxCols []*expression.Column, colLens []int, outerKeys []*expression.Column) {
tmpSchema := expression.NewSchema(innerKeys...)
func (ijHelper *indexJoinBuildHelper) resetContextForIndex(idxCols []*expression.Column, colLens []int) {
tmpSchema := expression.NewSchema(ijHelper.innerJoinKeys...)
ijHelper.curIdxOff2KeyOff = make([]int, len(idxCols))
ijHelper.curNotUsedIndexCols = make([]*expression.Column, 0, len(idxCols))
ijHelper.curNotUsedColLens = make([]int, 0, len(idxCols))
for i, idxCol := range idxCols {
ijHelper.curIdxOff2KeyOff[i] = tmpSchema.ColumnIndex(idxCol)
if ijHelper.curIdxOff2KeyOff[i] >= 0 {
// Don't use the join columns if their collations are unmatched and the new collation is enabled.
if collate.NewCollationEnabled() && types.IsString(idxCol.RetType.GetType()) && types.IsString(outerKeys[ijHelper.curIdxOff2KeyOff[i]].RetType.GetType()) {
et, err := expression.CheckAndDeriveCollationFromExprs(ijHelper.innerPlan.SCtx().GetExprCtx(), "equal", types.ETInt, idxCol, outerKeys[ijHelper.curIdxOff2KeyOff[i]])
if collate.NewCollationEnabled() && types.IsString(idxCol.RetType.GetType()) && types.IsString(ijHelper.outerJoinKeys[ijHelper.curIdxOff2KeyOff[i]].RetType.GetType()) {
et, err := expression.CheckAndDeriveCollationFromExprs(ijHelper.sctx.GetExprCtx(), "equal", types.ETInt, idxCol, ijHelper.outerJoinKeys[ijHelper.curIdxOff2KeyOff[i]])
if err != nil {
logutil.BgLogger().Error("Unexpected error happened during constructing index join", zap.Stack("stack"))
}
Expand All @@ -1677,11 +1690,12 @@ func (ijHelper *indexJoinBuildHelper) resetContextForIndex(innerKeys []*expressi
// usefulEqOrInFilters is the continuous eq/in conditions on current unused index columns.
// remainedEqOrIn is part of usefulEqOrInFilters, which needs to be evaluated again in selection.
// remainingRangeCandidates is the other conditions for future use.
func (ijHelper *indexJoinBuildHelper) findUsefulEqAndInFilters(innerPlan *DataSource) (usefulEqOrInFilters, remainedEqOrIn, remainingRangeCandidates []expression.Expression, emptyRange bool) {
func (ijHelper *indexJoinBuildHelper) findUsefulEqAndInFilters() (usefulEqOrInFilters, remainedEqOrIn, remainingRangeCandidates []expression.Expression, emptyRange bool) {
// 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, _, emptyRange = ranger.ExtractEqAndInCondition(
innerPlan.SCtx().GetRangerCtx(), innerPlan.PushedDownConds,
ijHelper.sctx.GetRangerCtx(),
ijHelper.innerPushedConditions,
ijHelper.curNotUsedIndexCols,
ijHelper.curNotUsedColLens,
)
Expand All @@ -1691,10 +1705,10 @@ func (ijHelper *indexJoinBuildHelper) findUsefulEqAndInFilters(innerPlan *DataSo
// 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) buildLastColManager(nextCol *expression.Column,
innerPlan *DataSource, cwc *ColWithCmpFuncManager) []expression.Expression {
cwc *ColWithCmpFuncManager) []expression.Expression {
var lastColAccesses []expression.Expression
loopOtherConds:
for _, filter := range ijHelper.join.OtherConditions {
for _, filter := range ijHelper.joinOtherConditions {
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
Expand All @@ -1717,7 +1731,7 @@ loopOtherConds:
continue
}
for _, col := range affectedCols {
if innerPlan.Schema().Contains(col) {
if ijHelper.innerSchema.Contains(col) {
continue loopOtherConds
}
}
Expand Down Expand Up @@ -1759,18 +1773,16 @@ func (ijHelper *indexJoinBuildHelper) removeUselessEqAndInFunc(idxCols []*expres
type mutableIndexJoinRange struct {
ranges ranger.Ranges

buildHelper *indexJoinBuildHelper
path *util.AccessPath
innerJoinKeys []*expression.Column
outerJoinKeys []*expression.Column
buildHelper *indexJoinBuildHelper
path *util.AccessPath
}

func (mr *mutableIndexJoinRange) Range() ranger.Ranges {
return mr.ranges
}

func (mr *mutableIndexJoinRange) Rebuild() error {
empty, err := mr.buildHelper.analyzeLookUpFilters(mr.path, mr.buildHelper.innerPlan, mr.innerJoinKeys, mr.outerJoinKeys, true)
empty, err := mr.buildHelper.analyzeLookUpFilters(mr.path, true)
if err != nil {
return err
}
Expand All @@ -1786,16 +1798,21 @@ func (mr *mutableIndexJoinRange) Rebuild() error {
return nil
}

func (ijHelper *indexJoinBuildHelper) createMutableIndexJoinRange(relatedExprs []expression.Expression, ranges []*ranger.Range, path *util.AccessPath, innerKeys, outerKeys []*expression.Column) ranger.MutableRanges {
func (ijHelper *indexJoinBuildHelper) createMutableIndexJoinRange(relatedExprs []expression.Expression, ranges []*ranger.Range, path *util.AccessPath) ranger.MutableRanges {
// if the plan-cache is enabled and these ranges depend on some parameters, we have to rebuild these ranges after changing parameters
if expression.MaybeOverOptimized4PlanCache(ijHelper.join.SCtx().GetExprCtx(), relatedExprs) {
if expression.MaybeOverOptimized4PlanCache(ijHelper.sctx.GetExprCtx(), relatedExprs) {
// assume that path, innerKeys and outerKeys will not be modified in the follow-up process
return &mutableIndexJoinRange{
ranges: ranges,
buildHelper: &indexJoinBuildHelper{innerPlan: ijHelper.innerPlan, join: ijHelper.join},
path: path,
innerJoinKeys: innerKeys,
outerJoinKeys: outerKeys,
ranges: ranges,
buildHelper: &indexJoinBuildHelper{
sctx: ijHelper.sctx,
innerSchema: ijHelper.innerSchema,
innerPushedConditions: ijHelper.innerPushedConditions,
innerStats: ijHelper.innerStats,
innerJoinKeys: ijHelper.innerJoinKeys,
outerJoinKeys: ijHelper.outerJoinKeys,
joinOtherConditions: ijHelper.joinOtherConditions},
path: path,
}
}
return ranger.Ranges(ranges)
Expand All @@ -1809,37 +1826,37 @@ func (ijHelper *indexJoinBuildHelper) updateByTemplateRangeResult(tempRangeRes *
ijHelper.curIdxOff2KeyOff[i] = -1
}
newAccesses = accesses[:tempRangeRes.eqAndInCntInRange]
newRemained = ranger.AppendConditionsIfNotExist(ijHelper.innerPlan.SCtx().GetExprCtx().GetEvalCtx(),
newRemained = ranger.AppendConditionsIfNotExist(ijHelper.sctx.GetExprCtx().GetEvalCtx(),
remained, accesses[tempRangeRes.eqAndInCntInRange:])
return
}

func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *util.AccessPath, innerPlan *DataSource, innerJoinKeys []*expression.Column, outerJoinKeys []*expression.Column, rebuildMode bool) (emptyRange bool, err error) {
func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *util.AccessPath, rebuildMode bool) (emptyRange bool, err error) {
if len(path.IdxCols) == 0 {
return false, nil
}
accesses := make([]expression.Expression, 0, len(path.IdxCols))
ijHelper.resetContextForIndex(innerJoinKeys, path.IdxCols, path.IdxColLens, outerJoinKeys)
notKeyEqAndIn, remained, rangeFilterCandidates, emptyRange := ijHelper.findUsefulEqAndInFilters(innerPlan)
ijHelper.resetContextForIndex(path.IdxCols, path.IdxColLens)
notKeyEqAndIn, remained, rangeFilterCandidates, emptyRange := ijHelper.findUsefulEqAndInFilters()
if emptyRange {
return true, nil
}
var remainedEqAndIn []expression.Expression
notKeyEqAndIn, remainedEqAndIn = ijHelper.removeUselessEqAndInFunc(path.IdxCols, notKeyEqAndIn)
matchedKeyCnt := len(ijHelper.curPossibleUsedKeys)
// If no join key is matched while join keys actually are not empty. We don't choose index join for now.
if matchedKeyCnt <= 0 && len(innerJoinKeys) > 0 {
if matchedKeyCnt <= 0 && len(ijHelper.innerJoinKeys) > 0 {
return false, nil
}
accesses = append(accesses, notKeyEqAndIn...)
remained = ranger.AppendConditionsIfNotExist(innerPlan.SCtx().GetExprCtx().GetEvalCtx(), remained, remainedEqAndIn)
remained = ranger.AppendConditionsIfNotExist(ijHelper.sctx.GetExprCtx().GetEvalCtx(), remained, remainedEqAndIn)
lastColPos := matchedKeyCnt + len(notKeyEqAndIn)
// There should be some equal conditions. But we don't need that there must be some join key in accesses here.
// A more strict check is applied later.
if lastColPos <= 0 {
return false, nil
}
rangeMaxSize := ijHelper.join.SCtx().GetSessionVars().RangeMaxSize
rangeMaxSize := ijHelper.sctx.GetSessionVars().RangeMaxSize
if rebuildMode {
// When rebuilding ranges for plan cache, we don't restrict range mem limit.
rangeMaxSize = 0
Expand All @@ -1858,7 +1875,7 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *util.AccessPath
return tempRangeRes.emptyRange, tempRangeRes.err
}
lastColPos, accesses, remained = ijHelper.updateByTemplateRangeResult(tempRangeRes, accesses, remained)
mutableRange := ijHelper.createMutableIndexJoinRange(accesses, tempRangeRes.ranges, path, innerJoinKeys, outerJoinKeys)
mutableRange := ijHelper.createMutableIndexJoinRange(accesses, tempRangeRes.ranges, path)
ijHelper.updateBestChoice(mutableRange, path, accesses, remained, nil, lastColPos, rebuildMode)
return false, nil
}
Expand All @@ -1868,7 +1885,7 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *util.AccessPath
colLength: path.IdxColLens[lastColPos],
affectedColSchema: expression.NewSchema(),
}
lastColAccess := ijHelper.buildLastColManager(lastPossibleCol, innerPlan, lastColManager)
lastColAccess := ijHelper.buildLastColManager(lastPossibleCol, lastColManager)
// If the column manager holds no expression, then we fallback to find whether there're useful normal filters
if len(lastColAccess) == 0 {
// If there's no join key matching index column, then choosing hash join is always a better idea.
Expand All @@ -1877,12 +1894,12 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *util.AccessPath
if matchedKeyCnt <= 0 {
return false, nil
}
colAccesses, colRemained := ranger.DetachCondsForColumn(ijHelper.join.SCtx().GetRangerCtx(), rangeFilterCandidates, lastPossibleCol)
colAccesses, colRemained := ranger.DetachCondsForColumn(ijHelper.sctx.GetRangerCtx(), rangeFilterCandidates, lastPossibleCol)
var nextColRange []*ranger.Range
var err error
if len(colAccesses) > 0 {
var colRemained2 []expression.Expression
nextColRange, colAccesses, colRemained2, err = ranger.BuildColumnRange(colAccesses, ijHelper.join.SCtx().GetRangerCtx(), lastPossibleCol.RetType, path.IdxColLens[lastColPos], rangeMaxSize)
nextColRange, colAccesses, colRemained2, err = ranger.BuildColumnRange(colAccesses, ijHelper.sctx.GetRangerCtx(), lastPossibleCol.RetType, path.IdxColLens[lastColPos], rangeMaxSize)
if err != nil {
return false, err
}
Expand All @@ -1907,7 +1924,7 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *util.AccessPath
} else {
remained = append(remained, colAccesses...)
}
mutableRange := ijHelper.createMutableIndexJoinRange(accesses, tempRangeRes.ranges, path, innerJoinKeys, outerJoinKeys)
mutableRange := ijHelper.createMutableIndexJoinRange(accesses, tempRangeRes.ranges, path)
ijHelper.updateBestChoice(mutableRange, path, accesses, remained, nil, lastColPos, rebuildMode)
return false, nil
}
Expand All @@ -1927,7 +1944,7 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *util.AccessPath
}
lastColManager = nil
}
mutableRange := ijHelper.createMutableIndexJoinRange(accesses, tempRangeRes.ranges, path, innerJoinKeys, outerJoinKeys)
mutableRange := ijHelper.createMutableIndexJoinRange(accesses, tempRangeRes.ranges, path)
ijHelper.updateBestChoice(mutableRange, path, accesses, remained, lastColManager, lastColPos, rebuildMode)
return false, nil
}
Expand All @@ -1948,8 +1965,8 @@ func (ijHelper *indexJoinBuildHelper) updateBestChoice(ranges ranger.MutableRang
return
}
var innerNDV float64
if stats := ijHelper.innerPlan.StatsInfo(); stats != nil && stats.StatsVersion != statistics.PseudoVersion {
innerNDV, _ = cardinality.EstimateColsNDVWithMatchedLen(path.IdxCols[:usedColsLen], ijHelper.innerPlan.Schema(), stats)
if stats := ijHelper.innerStats; stats != nil && stats.StatsVersion != statistics.PseudoVersion {
innerNDV, _ = cardinality.EstimateColsNDVWithMatchedLen(path.IdxCols[:usedColsLen], ijHelper.innerSchema, stats)
}
// We choose the index by the NDV of the used columns, the larger the better.
// If NDVs are same, we choose index which uses more columns.
Expand Down Expand Up @@ -1999,7 +2016,7 @@ func appendTailTemplateRange(originRanges ranger.Ranges, rangeMaxSize int64) (ra
func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAndInFuncs []expression.Expression, nextColRange []*ranger.Range,
haveExtraCol bool, rangeMaxSize int64) (res *templateRangeResult) {
res = &templateRangeResult{}
ctx := ijHelper.join.SCtx()
ctx := ijHelper.sctx
sc := ctx.GetSessionVars().StmtCtx
defer func() {
if sc.MemTracker != nil && res != nil && len(res.ranges) > 0 {
Expand All @@ -2023,7 +2040,7 @@ func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAn
i++
} else {
exprs := []expression.Expression{eqAndInFuncs[j]}
oneColumnRan, _, remained, err := ranger.BuildColumnRange(exprs, ijHelper.join.SCtx().GetRangerCtx(), ijHelper.curNotUsedIndexCols[j].RetType, ijHelper.curNotUsedColLens[j], rangeMaxSize)
oneColumnRan, _, remained, err := ranger.BuildColumnRange(exprs, ijHelper.sctx.GetRangerCtx(), ijHelper.curNotUsedIndexCols[j].RetType, ijHelper.curNotUsedColLens[j], rangeMaxSize)
if err != nil {
return &templateRangeResult{err: err}
}
Expand Down
12 changes: 10 additions & 2 deletions pkg/planner/core/exhaust_physical_plans_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -195,8 +195,16 @@ func testAnalyzeLookUpFilters(t *testing.T, testCtx *indexJoinContext, testCase
others, err := rewriteSimpleExpr(ctx.GetExprCtx(), testCase.otherConds, joinNode.Schema(), testCtx.joinColNames)
require.NoError(t, err)
joinNode.OtherConditions = others
helper := &indexJoinBuildHelper{join: joinNode, lastColManager: nil, innerPlan: dataSourceNode}
_, err = helper.analyzeLookUpFilters(testCtx.path, dataSourceNode, testCase.innerKeys, testCase.innerKeys, testCase.rebuildMode)
helper := &indexJoinBuildHelper{
sctx: ctx,
joinOtherConditions: others,
lastColManager: nil,
outerJoinKeys: testCase.innerKeys,
innerJoinKeys: testCase.innerKeys,
innerStats: dataSourceNode.StatsInfo(),
innerSchema: dataSourceNode.Schema(),
innerPushedConditions: dataSourceNode.PushedDownConds}
_, err = helper.analyzeLookUpFilters(testCtx.path, testCase.rebuildMode)
if helper.chosenRanges == nil {
helper.chosenRanges = ranger.Ranges{}
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -467,13 +467,13 @@ func appendCandidate4PhysicalOptimizeOp(pop *optimizetrace.PhysicalOptimizeOp, l
switch join := pp.(type) {
case *PhysicalIndexMergeJoin:
index = join.InnerChildIdx
plan = join.innerTask.Plan()
plan = join.innerPlan
case *PhysicalIndexHashJoin:
index = join.InnerChildIdx
plan = join.innerTask.Plan()
plan = join.innerPlan
case *PhysicalIndexJoin:
index = join.InnerChildIdx
plan = join.innerTask.Plan()
plan = join.innerPlan
}
if index != -1 {
child := lp.(*logicalop.BaseLogicalPlan).Children()[index]
Expand Down
Loading

0 comments on commit 383643f

Please sign in to comment.