Skip to content

Commit

Permalink
planner, util/ranger: recognize the best matching index stats more ac…
Browse files Browse the repository at this point in the history
…curately for OR list in `Selectivity()` (#56001)

close #54323
  • Loading branch information
time-and-fate authored Sep 23, 2024
1 parent d6ad92f commit fd41e60
Show file tree
Hide file tree
Showing 7 changed files with 276 additions and 63 deletions.
147 changes: 110 additions & 37 deletions pkg/planner/cardinality/selectivity.go
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,8 @@ func Selectivity(
id := col.UniqueID
colStats := coll.GetCol(id)
if colStats != nil {
maskCovered, ranges, _, err := getMaskAndRanges(ctx, remainedExprs, ranger.ColumnRangeType, nil, nil, col)
maskCovered, ranges, _, _, err :=
getMaskAndRanges(ctx, remainedExprs, ranger.ColumnRangeType, nil, nil, col)
if err != nil {
return 0, nil, errors.Trace(err)
}
Expand Down Expand Up @@ -196,8 +197,8 @@ func Selectivity(
if len(idxCols) > len(idxStats.Info.Columns) {
lengths = append(lengths, types.UnspecifiedLength)
}
maskCovered, ranges, partCover, err := getMaskAndRanges(ctx, remainedExprs,
ranger.IndexRangeType, lengths, id2Paths[idxStats.ID], idxCols...)
maskCovered, ranges, partCover, minAccessCondsForDNFCond, err :=
getMaskAndRanges(ctx, remainedExprs, ranger.IndexRangeType, lengths, id2Paths[idxStats.ID], idxCols...)
if err != nil {
return 0, nil, errors.Trace(err)
}
Expand All @@ -207,13 +208,14 @@ func Selectivity(
}
selectivity := cnt / float64(coll.RealtimeCount)
nodes = append(nodes, &StatsNode{
Tp: IndexType,
ID: id,
mask: maskCovered,
Ranges: ranges,
numCols: len(idxStats.Info.Columns),
Selectivity: selectivity,
partCover: partCover,
Tp: IndexType,
ID: id,
mask: maskCovered,
Ranges: ranges,
numCols: len(idxStats.Info.Columns),
Selectivity: selectivity,
partCover: partCover,
minAccessCondsForDNFCond: minAccessCondsForDNFCond,
})
}
}
Expand Down Expand Up @@ -546,6 +548,8 @@ type StatsNode struct {
// partCover indicates whether the bit in the mask is for a full cover or partial cover. It is only true
// when the condition is a DNF expression on index, and the expression is not totally extracted as access condition.
partCover bool
// Please see comments of planner/util.AccessPath.MinAccessCondsForDNFCond for more details.
minAccessCondsForDNFCond int
}

// The type of the StatsNode.
Expand Down Expand Up @@ -604,7 +608,18 @@ func GetUsableSetsByGreedy(nodes []*StatsNode) (newBlocks []*StatsNode) {
mask := int64(math.MaxInt64)
for {
// Choose the index that covers most.
bestID, bestCount, bestTp, bestNumCols, bestMask, bestSel := -1, 0, ColType, 0, int64(0), float64(0)
bestMask := int64(0)
best := &statsNodeForGreedyChoice{
StatsNode: &StatsNode{
Tp: ColType,
Selectivity: 0,
numCols: 0,
partCover: true,
minAccessCondsForDNFCond: -1,
},
idx: -1,
coverCount: 0,
}
for i, set := range nodes {
if marked[i] {
continue
Expand All @@ -620,35 +635,86 @@ func GetUsableSetsByGreedy(nodes []*StatsNode) (newBlocks []*StatsNode) {
marked[i] = true
continue
}
// We greedy select the stats info based on:
// (1): The stats type, always prefer the primary key or index.
// (2): The number of expression that it covers, the more the better.
// (3): The number of columns that it contains, the less the better.
// (4): The selectivity of the covered conditions, the less the better.
// The rationale behind is that lower selectivity tends to reflect more functional dependencies
// between columns. It's hard to decide the priority of this rule against rule 2 and 3, in order
// to avoid massive plan changes between tidb-server versions, I adopt this conservative strategy
// to impose this rule after rule 2 and 3.
if (bestTp == ColType && set.Tp != ColType) ||
bestCount < bits ||
(bestCount == bits && bestNumCols > set.numCols) ||
(bestCount == bits && bestNumCols == set.numCols && bestSel > set.Selectivity) {
bestID, bestCount, bestTp, bestNumCols, bestMask, bestSel = i, bits, set.Tp, set.numCols, curMask, set.Selectivity
current := &statsNodeForGreedyChoice{
StatsNode: set,
idx: i,
coverCount: bits,
}
if current.isBetterThan(best) {
best = current
bestMask = curMask
}
}
if bestCount == 0 {
if best.coverCount == 0 {
break
}

// Update the mask, remove the bit that nodes[bestID].mask has.
// Update the mask, remove the bit that nodes[best.idx].mask has.
mask &^= bestMask

newBlocks = append(newBlocks, nodes[bestID])
marked[bestID] = true
newBlocks = append(newBlocks, nodes[best.idx])
marked[best.idx] = true
}
return
}

type statsNodeForGreedyChoice struct {
*StatsNode
idx int
coverCount int
}

func (s *statsNodeForGreedyChoice) isBetterThan(other *statsNodeForGreedyChoice) bool {
// none of them should be nil
if s == nil || other == nil {
return false
}
// 1. The stats type, always prefer the primary key or index.
if s.Tp != ColType && other.Tp == ColType {
return true
}
// 2. The number of expression that it covers, the more, the better.
if s.coverCount > other.coverCount {
return true
}
// Worse or equal. We return false for both cases. The same for the following rules.
if s.coverCount != other.coverCount {
return false
}
// 3. It's only for DNF. Full cover is better than partial cover
if !s.partCover && other.partCover {
return true
}
if s.partCover != other.partCover {
return false
}
// 4. It's only for DNF. The minimum number of access conditions among all DNF items, the more, the better.
// s.coverCount is not enough for DNF, so we use this field to make the judgment more accurate.
if s.minAccessCondsForDNFCond > other.minAccessCondsForDNFCond {
return true
}
if s.minAccessCondsForDNFCond != other.minAccessCondsForDNFCond {
return false
}

// 5. The number of columns that it contains, the less, the better.
if s.numCols < other.numCols {
return true
}
if s.numCols != other.numCols {
return false
}
// 6. The selectivity of the covered conditions, the less, the better.
// The rationale behind is that lower selectivity tends to reflect more functional dependencies
// between columns. It's hard to decide the priority of this rule against rules above, in order
// to avoid massive plan changes between tidb-server versions, I adopt this conservative strategy
// to impose this rule after rules above.
if s.Selectivity < other.Selectivity {
return true
}
return false
}

// isColEqCorCol checks if the expression is an eq function that one side is correlated column and another is column.
// If so, it will return the column's reference. Otherwise, return nil instead.
func isColEqCorCol(filter expression.Expression) *expression.Column {
Expand Down Expand Up @@ -695,7 +761,7 @@ func findPrefixOfIndexByCol(ctx planctx.PlanContext, cols []*expression.Column,

func getMaskAndRanges(ctx planctx.PlanContext, exprs []expression.Expression, rangeType ranger.RangeType,
lengths []int, cachedPath *planutil.AccessPath, cols ...*expression.Column) (
mask int64, ranges []*ranger.Range, partCover bool, err error) {
mask int64, ranges []*ranger.Range, partCover bool, minAccessCondsForDNFCond int, err error) {
isDNF := false
var accessConds, remainedConds []expression.Expression
switch rangeType {
Expand All @@ -705,25 +771,32 @@ func getMaskAndRanges(ctx planctx.PlanContext, exprs []expression.Expression, ra
types.UnspecifiedLength, ctx.GetSessionVars().RangeMaxSize)
case ranger.IndexRangeType:
if cachedPath != nil {
ranges, accessConds, remainedConds, isDNF = cachedPath.Ranges,
cachedPath.AccessConds, cachedPath.TableFilters, cachedPath.IsDNFCond
ranges = cachedPath.Ranges
accessConds = cachedPath.AccessConds
remainedConds = cachedPath.TableFilters
isDNF = cachedPath.IsDNFCond
minAccessCondsForDNFCond = cachedPath.MinAccessCondsForDNFCond
break
}
var res *ranger.DetachRangeResult
res, err = ranger.DetachCondAndBuildRangeForIndex(ctx.GetRangerCtx(), exprs, cols, lengths, ctx.GetSessionVars().RangeMaxSize)
if err != nil {
return 0, nil, false, err
return 0, nil, false, 0, err
}
ranges, accessConds, remainedConds, isDNF = res.Ranges, res.AccessConds, res.RemainedConds, res.IsDNFCond
ranges = res.Ranges
accessConds = res.AccessConds
remainedConds = res.RemainedConds
isDNF = res.IsDNFCond
minAccessCondsForDNFCond = res.MinAccessCondsForDNFCond
default:
panic("should never be here")
}
if err != nil {
return 0, nil, false, err
return 0, nil, false, 0, err
}
if isDNF && len(accessConds) > 0 {
mask |= 1
return mask, ranges, len(remainedConds) > 0, nil
return mask, ranges, len(remainedConds) > 0, minAccessCondsForDNFCond, nil
}
for i := range exprs {
for j := range accessConds {
Expand All @@ -733,7 +806,7 @@ func getMaskAndRanges(ctx planctx.PlanContext, exprs []expression.Expression, ra
}
}
}
return mask, ranges, false, nil
return mask, ranges, false, 0, nil
}

func getMaskAndSelectivityForMVIndex(
Expand Down
1 change: 1 addition & 0 deletions pkg/planner/core/logical_datasource.go
Original file line number Diff line number Diff line change
Expand Up @@ -609,6 +609,7 @@ func detachCondAndBuildRangeForPath(
path.EqCondCount = res.EqCondCount
path.EqOrInCondCount = res.EqOrInCount
path.IsDNFCond = res.IsDNFCond
path.MinAccessCondsForDNFCond = res.MinAccessCondsForDNFCond
path.ConstCols = make([]bool, len(path.IdxCols))
if res.ColumnValues != nil {
for i := range path.ConstCols {
Expand Down
9 changes: 8 additions & 1 deletion pkg/planner/util/path.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,13 @@ type AccessPath struct {

StoreType kv.StoreType

IsDNFCond bool
// If the top level of the filters is an OR list, IsDNFCond is true.
// In this case, MinAccessCondsForDNFCond will record the minimum number of access conditions among all DNF items.
// For example, if the filter is (a=1 and b=2) or (a=3 and b=4) or (a=5 and b=6 and c=7),
// for index (a) or index (b), MinAccessCondsForDNFCond will be 1;
// for index (a, b, c), MinAccessCondsForDNFCond will be 2.
IsDNFCond bool
MinAccessCondsForDNFCond int

// IsIntHandlePath indicates whether this path is table path.
IsIntHandlePath bool
Expand Down Expand Up @@ -112,6 +118,7 @@ func (path *AccessPath) Clone() *AccessPath {
PartialIndexPaths: nil,
StoreType: path.StoreType,
IsDNFCond: path.IsDNFCond,
MinAccessCondsForDNFCond: path.MinAccessCondsForDNFCond,
IsIntHandlePath: path.IsIntHandlePath,
IsCommonHandlePath: path.IsCommonHandlePath,
Forced: path.Forced,
Expand Down
Loading

0 comments on commit fd41e60

Please sign in to comment.