Skip to content

Commit

Permalink
planner: adjust estimated rows to account for modified rows (#50970)
Browse files Browse the repository at this point in the history
close #47523
  • Loading branch information
terry1purcell committed Mar 13, 2024
1 parent 893104a commit 47a990e
Show file tree
Hide file tree
Showing 4 changed files with 53 additions and 45 deletions.
14 changes: 8 additions & 6 deletions pkg/planner/cardinality/row_count_column.go
Expand Up @@ -282,15 +282,17 @@ func GetColumnRowCount(sctx context.PlanContext, c *statistics.Column, ranges []
cnt = mathutil.Clamp(cnt, 0, c.TotalRowCount())

// If the current table row count has changed, we should scale the row count accordingly.
cnt *= c.GetIncreaseFactor(realtimeRowCount)
increaseFactor := c.GetIncreaseFactor(realtimeRowCount)
cnt *= increaseFactor

histNDV := c.NDV
if c.StatsVer == statistics.Version2 {
histNDV = histNDV - int64(c.TopN.Num())
}
// handling the out-of-range part
if (c.OutOfRange(lowVal) && !lowVal.IsNull()) || c.OutOfRange(highVal) {
cnt += c.Histogram.OutOfRangeRowCount(sctx, &lowVal, &highVal, modifyCount, histNDV)
histNDV := c.NDV
// Exclude the TopN
if c.StatsVer == statistics.Version2 {
histNDV -= int64(c.TopN.Num())
}
cnt += c.Histogram.OutOfRangeRowCount(sctx, &lowVal, &highVal, modifyCount, histNDV, increaseFactor)
}

if debugTrace {
Expand Down
27 changes: 18 additions & 9 deletions pkg/planner/cardinality/row_count_index.go
Expand Up @@ -222,7 +222,7 @@ func getIndexRowCountForStatsV2(sctx context.PlanContext, idx *statistics.Index,
defer debugtrace.LeaveContextCommon(sctx)
}
totalCount := float64(0)
isSingleCol := len(idx.Info.Columns) == 1
isSingleColIdx := len(idx.Info.Columns) == 1
for _, indexRange := range indexRanges {
var count float64
lb, err := codec.EncodeKey(sc.TimeZone(), nil, indexRange.LowVal...)
Expand Down Expand Up @@ -278,7 +278,7 @@ func getIndexRowCountForStatsV2(sctx context.PlanContext, idx *statistics.Index,
l := types.NewBytesDatum(lb)
r := types.NewBytesDatum(rb)
lowIsNull := bytes.Equal(lb, nullKeyBytes)
if isSingleCol && lowIsNull {
if isSingleColIdx && lowIsNull {
count += float64(idx.Histogram.NullCount)
}
expBackoffSuccess := false
Expand Down Expand Up @@ -325,15 +325,24 @@ func getIndexRowCountForStatsV2(sctx context.PlanContext, idx *statistics.Index,
}

// If the current table row count has changed, we should scale the row count accordingly.
count *= idx.GetIncreaseFactor(realtimeRowCount)
increaseFactor := idx.GetIncreaseFactor(realtimeRowCount)
count *= increaseFactor

histNDV := idx.NDV
if idx.StatsVer == statistics.Version2 {
histNDV = histNDV - int64(idx.TopN.Num())
}
// handling the out-of-range part
if (outOfRangeOnIndex(idx, l) && !(isSingleCol && lowIsNull)) || outOfRangeOnIndex(idx, r) {
count += idx.Histogram.OutOfRangeRowCount(sctx, &l, &r, modifyCount, histNDV)
if (outOfRangeOnIndex(idx, l) && !(isSingleColIdx && lowIsNull)) || outOfRangeOnIndex(idx, r) {
histNDV := idx.NDV
// Exclude the TopN in Stats Version 2
if idx.StatsVer == statistics.Version2 {
c, ok := coll.Columns[idx.Histogram.ID]
// If this is single column of a multi-column index - use the column's NDV rather than index NDV
isSingleColRange := len(indexRange.LowVal) == len(indexRange.HighVal) && len(indexRange.LowVal) == 1
if isSingleColRange && !isSingleColIdx && ok && c != nil && c.Histogram.NDV > 0 {
histNDV = c.Histogram.NDV - int64(c.TopN.Num())
} else {
histNDV -= int64(idx.TopN.Num())
}
}
count += idx.Histogram.OutOfRangeRowCount(sctx, &l, &r, modifyCount, histNDV, increaseFactor)
}

if debugTrace {
Expand Down
6 changes: 3 additions & 3 deletions pkg/planner/cardinality/testdata/cardinality_suite_out.json
Expand Up @@ -24,7 +24,7 @@
{
"Start": 800,
"End": 900,
"Count": 723.504166655054
"Count": 735.504166655054
},
{
"Start": 900,
Expand Down Expand Up @@ -79,7 +79,7 @@
{
"Start": 800,
"End": 1000,
"Count": 1181.696869573942
"Count": 1193.696869573942
},
{
"Start": 900,
Expand All @@ -104,7 +104,7 @@
{
"Start": 200,
"End": 400,
"Count": 1190.2788209899081
"Count": 1237.5288209899081
},
{
"Start": 200,
Expand Down
51 changes: 24 additions & 27 deletions pkg/statistics/histogram.go
Expand Up @@ -937,7 +937,7 @@ func (hg *Histogram) OutOfRange(val types.Datum) bool {
func (hg *Histogram) OutOfRangeRowCount(
sctx context.PlanContext,
lDatum, rDatum *types.Datum,
modifyCount, histNDV int64,
modifyCount, histNDV int64, increaseFactor float64,
) (result float64) {
debugTrace := sctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace
if debugTrace {
Expand Down Expand Up @@ -1052,38 +1052,35 @@ func (hg *Histogram) OutOfRangeRowCount(
rightPercent = (math.Pow(boundR-actualL, 2) - math.Pow(boundR-actualR, 2)) / math.Pow(histWidth, 2)
}

totalPercent := leftPercent*0.5 + rightPercent*0.5
if totalPercent > 1 {
totalPercent = 1
}
totalPercent := min(leftPercent*0.5+rightPercent*0.5, 1.0)
rowCount = totalPercent * hg.NotNullCount()

// Upper bound logic
// Upper & lower bound logic.
upperBound := rowCount
if histNDV > 0 {
upperBound = hg.NotNullCount() / float64(histNDV)
}

allowUseModifyCount := sctx.GetSessionVars().GetOptObjective() != variable.OptObjectiveDeterminate
// Use the modifyCount as the upper bound. Note that modifyCount contains insert, delete and update. So this is
// a rather loose upper bound.
// There are some scenarios where we need to handle out-of-range estimation after both insert and delete happen.
// But we don't know how many increases are in the modifyCount. So we have to use this loose bound to ensure it
// can produce a reasonable results in this scenario.
if rowCount > float64(modifyCount) && allowUseModifyCount {
return float64(modifyCount)
}

// In OptObjectiveDeterminate mode, we can't rely on the modify count anymore.
// An upper bound is necessary to make the estimation make sense for predicates with bound on only one end, like a > 1.
// But it's impossible to have a reliable upper bound in all cases.
// We use 1/NDV here (only the Histogram part is considered) and it seems reasonable and good enough for now.

if !allowUseModifyCount {
var upperBound float64
if histNDV > 0 {
upperBound = hg.NotNullCount() / float64(histNDV)
}
if rowCount > upperBound {
return upperBound
}
// In OptObjectiveDeterminate mode, we can't rely on the modify count anymore.
// An upper bound is necessary to make the estimation make sense for predicates with bound on only one end, like a > 1.
// We use 1/NDV here (only the Histogram part is considered) and it seems reasonable and good enough for now.
return min(rowCount, upperBound)
}

// If the modifyCount is large (compared to original table rows), then any out of range estimate is unreliable.
// Assume at least 1/NDV is returned
if float64(modifyCount) > hg.NotNullCount() && rowCount < upperBound {
rowCount = upperBound
} else if rowCount < upperBound {
// Adjust by increaseFactor if our estimate is low
rowCount *= increaseFactor
}
return rowCount

// Use modifyCount as a final bound
return min(rowCount, float64(modifyCount))
}

// Copy deep copies the histogram.
Expand Down

0 comments on commit 47a990e

Please sign in to comment.