|
| 1 | +// Copyright 2023 PingCAP, Inc. |
| 2 | +// |
| 3 | +// Licensed under the Apache License, Version 2.0 (the "License"); |
| 4 | +// you may not use this file except in compliance with the License. |
| 5 | +// You may obtain a copy of the License at |
| 6 | +// |
| 7 | +// http://www.apache.org/licenses/LICENSE-2.0 |
| 8 | +// |
| 9 | +// Unless required by applicable law or agreed to in writing, software |
| 10 | +// distributed under the License is distributed on an "AS IS" BASIS, |
| 11 | +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 12 | +// See the License for the specific language governing permissions and |
| 13 | +// limitations under the License. |
| 14 | + |
| 15 | +package cardinality |
| 16 | + |
| 17 | +import ( |
| 18 | + "math" |
| 19 | + |
| 20 | + "github.com/pingcap/tidb/pkg/expression" |
| 21 | + "github.com/pingcap/tidb/pkg/planner/context" |
| 22 | + "github.com/pingcap/tidb/pkg/planner/property" |
| 23 | + "github.com/pingcap/tidb/pkg/planner/util" |
| 24 | + "github.com/pingcap/tidb/pkg/statistics" |
| 25 | + "github.com/pingcap/tidb/pkg/types" |
| 26 | + "github.com/pingcap/tidb/pkg/util/ranger" |
| 27 | + "github.com/pingcap/tidb/pkg/util/set" |
| 28 | +) |
| 29 | + |
| 30 | +// SelectionFactor is the factor which is used to estimate the row count of selection. |
| 31 | +const SelectionFactor = 0.8 |
| 32 | + |
| 33 | +// AdjustRowCountForTableScanByLimit will adjust the row count for table scan by limit. |
| 34 | +// For a query like `select pk from t using index(primary) where pk > 10 limit 1`, the row count of the table scan |
| 35 | +// should be adjusted by the limit number 1, because only one row is returned. |
| 36 | +func AdjustRowCountForTableScanByLimit(sctx context.PlanContext, |
| 37 | + dsStatsInfo, dsTableStats *property.StatsInfo, dsStatisticTable *statistics.Table, |
| 38 | + path *util.AccessPath, expectedCnt float64, desc bool) float64 { |
| 39 | + rowCount := path.CountAfterAccess |
| 40 | + if expectedCnt < dsStatsInfo.RowCount { |
| 41 | + selectivity := dsStatsInfo.RowCount / path.CountAfterAccess |
| 42 | + uniformEst := min(path.CountAfterAccess, expectedCnt/selectivity) |
| 43 | + |
| 44 | + corrEst, ok, corr := crossEstimateTableRowCount(sctx, |
| 45 | + dsStatsInfo, dsTableStats, dsStatisticTable, path, expectedCnt, desc) |
| 46 | + if ok { |
| 47 | + // TODO: actually, before using this count as the estimated row count of table scan, we need additionally |
| 48 | + // check if count < row_count(first_region | last_region), and use the larger one since we build one copTask |
| 49 | + // for one region now, so even if it is `limit 1`, we have to scan at least one region in table scan. |
| 50 | + // Currently, we can use `tikvrpc.CmdDebugGetRegionProperties` interface as `getSampRegionsRowCount()` does |
| 51 | + // to get the row count in a region, but that result contains MVCC old version rows, so it is not that accurate. |
| 52 | + // Considering that when this scenario happens, the execution time is close between IndexScan and TableScan, |
| 53 | + // we do not add this check temporarily. |
| 54 | + |
| 55 | + // to reduce risks of correlation adjustment, use the maximum between uniformEst and corrEst |
| 56 | + rowCount = max(uniformEst, corrEst) |
| 57 | + } else if abs := math.Abs(corr); abs < 1 { |
| 58 | + correlationFactor := math.Pow(1-abs, float64(sctx.GetSessionVars().CorrelationExpFactor)) |
| 59 | + rowCount = min(path.CountAfterAccess, uniformEst/correlationFactor) |
| 60 | + } |
| 61 | + } |
| 62 | + return rowCount |
| 63 | +} |
| 64 | + |
| 65 | +// crossEstimateTableRowCount estimates row count of table scan using histogram of another column which is in TableFilters |
| 66 | +// and has high order correlation with handle column. For example, if the query is like: |
| 67 | +// `select * from tbl where a = 1 order by pk limit 1` |
| 68 | +// if order of column `a` is strictly correlated with column `pk`, the row count of table scan should be: |
| 69 | +// `1 + row_count(a < 1 or a is null)` |
| 70 | +func crossEstimateTableRowCount(sctx context.PlanContext, |
| 71 | + dsStatsInfo, dsTableStats *property.StatsInfo, dsStatisticTable *statistics.Table, |
| 72 | + path *util.AccessPath, expectedCnt float64, desc bool) (float64, bool, float64) { |
| 73 | + if dsStatisticTable.Pseudo || len(path.TableFilters) == 0 || !sctx.GetSessionVars().EnableCorrelationAdjustment { |
| 74 | + return 0, false, 0 |
| 75 | + } |
| 76 | + col, corr := getMostCorrCol4Handle(path.TableFilters, dsStatisticTable, sctx.GetSessionVars().CorrelationThreshold) |
| 77 | + return crossEstimateRowCount(sctx, dsStatsInfo, dsTableStats, path, path.TableFilters, col, corr, expectedCnt, desc) |
| 78 | +} |
| 79 | + |
| 80 | +// AdjustRowCountForIndexScanByLimit will adjust the row count for table scan by limit. |
| 81 | +// For a query like `select k from t using index(k) where k > 10 limit 1`, the row count of the index scan |
| 82 | +// should be adjusted by the limit number 1, because only one row is returned. |
| 83 | +func AdjustRowCountForIndexScanByLimit(sctx context.PlanContext, |
| 84 | + dsStatsInfo, dsTableStats *property.StatsInfo, dsStatisticTable *statistics.Table, |
| 85 | + path *util.AccessPath, expectedCnt float64, desc bool) float64 { |
| 86 | + rowCount := path.CountAfterAccess |
| 87 | + count, ok, corr := crossEstimateIndexRowCount(sctx, |
| 88 | + dsStatsInfo, dsTableStats, dsStatisticTable, path, expectedCnt, desc) |
| 89 | + if ok { |
| 90 | + rowCount = count |
| 91 | + } else if abs := math.Abs(corr); abs < 1 { |
| 92 | + // If OptOrderingIdxSelRatio is enabled - estimate the difference between index and table filtering, as this represents |
| 93 | + // the possible scan range when LIMIT rows will be found. orderRatio is the estimated percentage of that range when the first |
| 94 | + // row is expected to be found. Index filtering applies orderRatio twice. Once found - rows are estimated to be clustered (expectedCnt). |
| 95 | + // This formula is to bias away from non-filtering (or poorly filtering) indexes that provide order due, where filtering exists |
| 96 | + // outside of that index. Such plans have high risk since we cannot estimate when rows will be found. |
| 97 | + orderRatio := sctx.GetSessionVars().OptOrderingIdxSelRatio |
| 98 | + if dsStatsInfo.RowCount < path.CountAfterAccess && orderRatio >= 0 { |
| 99 | + rowsToMeetFirst := (((path.CountAfterAccess - path.CountAfterIndex) * orderRatio) + (path.CountAfterIndex - dsStatsInfo.RowCount)) * orderRatio |
| 100 | + rowCount = rowsToMeetFirst + expectedCnt |
| 101 | + } else { |
| 102 | + // Assume rows are linearly distributed throughout the range - for example: selectivity 0.1 assumes that a |
| 103 | + // qualified row is found every 10th row. |
| 104 | + correlationFactor := math.Pow(1-abs, float64(sctx.GetSessionVars().CorrelationExpFactor)) |
| 105 | + selectivity := dsStatsInfo.RowCount / rowCount |
| 106 | + rowCount = min(expectedCnt/selectivity/correlationFactor, rowCount) |
| 107 | + } |
| 108 | + } |
| 109 | + return rowCount |
| 110 | +} |
| 111 | + |
| 112 | +// crossEstimateIndexRowCount estimates row count of index scan using histogram of another column which is in TableFilters/IndexFilters |
| 113 | +// and has high order correlation with the first index column. For example, if the query is like: |
| 114 | +// `select * from tbl where a = 1 order by b limit 1` |
| 115 | +// if order of column `a` is strictly correlated with column `b`, the row count of IndexScan(b) should be: |
| 116 | +// `1 + row_count(a < 1 or a is null)` |
| 117 | +func crossEstimateIndexRowCount(sctx context.PlanContext, |
| 118 | + dsStatsInfo, dsTableStats *property.StatsInfo, dsStatisticTable *statistics.Table, |
| 119 | + path *util.AccessPath, expectedCnt float64, desc bool) (float64, bool, float64) { |
| 120 | + filtersLen := len(path.TableFilters) + len(path.IndexFilters) |
| 121 | + sessVars := sctx.GetSessionVars() |
| 122 | + if dsStatisticTable.Pseudo || filtersLen == 0 || !sessVars.EnableExtendedStats || !sctx.GetSessionVars().EnableCorrelationAdjustment { |
| 123 | + return 0, false, 0 |
| 124 | + } |
| 125 | + col, corr := getMostCorrCol4Index(path, dsStatisticTable, sessVars.CorrelationThreshold) |
| 126 | + filters := make([]expression.Expression, 0, filtersLen) |
| 127 | + filters = append(filters, path.TableFilters...) |
| 128 | + filters = append(filters, path.IndexFilters...) |
| 129 | + return crossEstimateRowCount(sctx, dsStatsInfo, dsTableStats, path, filters, col, corr, expectedCnt, desc) |
| 130 | +} |
| 131 | + |
| 132 | +// crossEstimateRowCount is the common logic of crossEstimateTableRowCount and crossEstimateIndexRowCount. |
| 133 | +func crossEstimateRowCount(sctx context.PlanContext, |
| 134 | + dsStatsInfo, dsTableStats *property.StatsInfo, |
| 135 | + path *util.AccessPath, conds []expression.Expression, col *expression.Column, |
| 136 | + corr, expectedCnt float64, desc bool) (float64, bool, float64) { |
| 137 | + // If the scan is not full range scan, we cannot use histogram of other columns for estimation, because |
| 138 | + // the histogram reflects value distribution in the whole table level. |
| 139 | + if col == nil || len(path.AccessConds) > 0 { |
| 140 | + return 0, false, corr |
| 141 | + } |
| 142 | + colUniqueID := col.UniqueID |
| 143 | + if corr < 0 { |
| 144 | + desc = !desc |
| 145 | + } |
| 146 | + accessConds, remained := ranger.DetachCondsForColumn(sctx, conds, col) |
| 147 | + if len(accessConds) == 0 { |
| 148 | + return 0, false, corr |
| 149 | + } |
| 150 | + ranges, accessConds, _, err := ranger.BuildColumnRange(accessConds, sctx, col.RetType, types.UnspecifiedLength, sctx.GetSessionVars().RangeMaxSize) |
| 151 | + if len(ranges) == 0 || len(accessConds) == 0 || err != nil { |
| 152 | + return 0, err == nil, corr |
| 153 | + } |
| 154 | + idxID := int64(-1) |
| 155 | + idxIDs, idxExists := dsStatsInfo.HistColl.ColUniqueID2IdxIDs[colUniqueID] |
| 156 | + if idxExists && len(idxIDs) > 0 { |
| 157 | + idxID = idxIDs[0] |
| 158 | + } |
| 159 | + rangeCounts, ok := getColumnRangeCounts(sctx, colUniqueID, ranges, dsTableStats.HistColl, idxID) |
| 160 | + if !ok { |
| 161 | + return 0, false, corr |
| 162 | + } |
| 163 | + convertedRanges, count, isFull := convertRangeFromExpectedCnt(ranges, rangeCounts, expectedCnt, desc) |
| 164 | + if isFull { |
| 165 | + return path.CountAfterAccess, true, 0 |
| 166 | + } |
| 167 | + var rangeCount float64 |
| 168 | + if idxExists { |
| 169 | + rangeCount, err = GetRowCountByIndexRanges(sctx, dsTableStats.HistColl, idxID, convertedRanges) |
| 170 | + } else { |
| 171 | + rangeCount, err = GetRowCountByColumnRanges(sctx, dsTableStats.HistColl, colUniqueID, convertedRanges) |
| 172 | + } |
| 173 | + if err != nil { |
| 174 | + return 0, false, corr |
| 175 | + } |
| 176 | + scanCount := rangeCount + expectedCnt - count |
| 177 | + if len(remained) > 0 { |
| 178 | + scanCount = scanCount / SelectionFactor |
| 179 | + } |
| 180 | + scanCount = min(scanCount, path.CountAfterAccess) |
| 181 | + return scanCount, true, 0 |
| 182 | +} |
| 183 | + |
| 184 | +// getColumnRangeCounts estimates row count for each range respectively. |
| 185 | +func getColumnRangeCounts(sctx context.PlanContext, colID int64, ranges []*ranger.Range, histColl *statistics.HistColl, idxID int64) ([]float64, bool) { |
| 186 | + var err error |
| 187 | + var count float64 |
| 188 | + rangeCounts := make([]float64, len(ranges)) |
| 189 | + for i, ran := range ranges { |
| 190 | + if idxID >= 0 { |
| 191 | + idxHist := histColl.Indices[idxID] |
| 192 | + if statistics.IndexStatsIsInvalid(sctx, idxHist, histColl, idxID) { |
| 193 | + return nil, false |
| 194 | + } |
| 195 | + count, err = GetRowCountByIndexRanges(sctx, histColl, idxID, []*ranger.Range{ran}) |
| 196 | + } else { |
| 197 | + colHist := histColl.Columns[colID] |
| 198 | + if statistics.ColumnStatsIsInvalid(colHist, sctx, histColl, colID) { |
| 199 | + return nil, false |
| 200 | + } |
| 201 | + count, err = GetRowCountByColumnRanges(sctx, histColl, colID, []*ranger.Range{ran}) |
| 202 | + } |
| 203 | + if err != nil { |
| 204 | + return nil, false |
| 205 | + } |
| 206 | + rangeCounts[i] = count |
| 207 | + } |
| 208 | + return rangeCounts, true |
| 209 | +} |
| 210 | + |
| 211 | +// convertRangeFromExpectedCnt builds new ranges used to estimate row count we need to scan in table scan before finding specified |
| 212 | +// number of tuples which fall into input ranges. |
| 213 | +func convertRangeFromExpectedCnt(ranges []*ranger.Range, rangeCounts []float64, expectedCnt float64, desc bool) ([]*ranger.Range, float64, bool) { |
| 214 | + var i int |
| 215 | + var count float64 |
| 216 | + var convertedRanges []*ranger.Range |
| 217 | + if desc { |
| 218 | + for i = len(ranges) - 1; i >= 0; i-- { |
| 219 | + if count+rangeCounts[i] >= expectedCnt { |
| 220 | + break |
| 221 | + } |
| 222 | + count += rangeCounts[i] |
| 223 | + } |
| 224 | + if i < 0 { |
| 225 | + return nil, 0, true |
| 226 | + } |
| 227 | + convertedRanges = []*ranger.Range{{LowVal: ranges[i].HighVal, HighVal: []types.Datum{types.MaxValueDatum()}, LowExclude: !ranges[i].HighExclude, Collators: ranges[i].Collators}} |
| 228 | + } else { |
| 229 | + for i = 0; i < len(ranges); i++ { |
| 230 | + if count+rangeCounts[i] >= expectedCnt { |
| 231 | + break |
| 232 | + } |
| 233 | + count += rangeCounts[i] |
| 234 | + } |
| 235 | + if i == len(ranges) { |
| 236 | + return nil, 0, true |
| 237 | + } |
| 238 | + convertedRanges = []*ranger.Range{{LowVal: []types.Datum{{}}, HighVal: ranges[i].LowVal, HighExclude: !ranges[i].LowExclude, Collators: ranges[i].Collators}} |
| 239 | + } |
| 240 | + return convertedRanges, count, false |
| 241 | +} |
| 242 | + |
| 243 | +// getMostCorrCol4Index checks if column in the condition is correlated enough with the first index column. If the condition |
| 244 | +// contains multiple columns, return nil and get the max correlation, which would be used in the heuristic estimation. |
| 245 | +func getMostCorrCol4Index(path *util.AccessPath, histColl *statistics.Table, threshold float64) (*expression.Column, float64) { |
| 246 | + if histColl.ExtendedStats == nil || len(histColl.ExtendedStats.Stats) == 0 { |
| 247 | + return nil, 0 |
| 248 | + } |
| 249 | + var cols []*expression.Column |
| 250 | + cols = expression.ExtractColumnsFromExpressions(cols, path.TableFilters, nil) |
| 251 | + cols = expression.ExtractColumnsFromExpressions(cols, path.IndexFilters, nil) |
| 252 | + if len(cols) == 0 { |
| 253 | + return nil, 0 |
| 254 | + } |
| 255 | + colSet := set.NewInt64Set() |
| 256 | + var corr float64 |
| 257 | + var corrCol *expression.Column |
| 258 | + for _, col := range cols { |
| 259 | + if colSet.Exist(col.UniqueID) { |
| 260 | + continue |
| 261 | + } |
| 262 | + colSet.Insert(col.UniqueID) |
| 263 | + curCorr := float64(0) |
| 264 | + for _, item := range histColl.ExtendedStats.Stats { |
| 265 | + if (col.ID == item.ColIDs[0] && path.FullIdxCols[0].ID == item.ColIDs[1]) || |
| 266 | + (col.ID == item.ColIDs[1] && path.FullIdxCols[0].ID == item.ColIDs[0]) { |
| 267 | + curCorr = item.ScalarVals |
| 268 | + break |
| 269 | + } |
| 270 | + } |
| 271 | + if corrCol == nil || math.Abs(corr) < math.Abs(curCorr) { |
| 272 | + corrCol = col |
| 273 | + corr = curCorr |
| 274 | + } |
| 275 | + } |
| 276 | + if len(colSet) == 1 && math.Abs(corr) >= threshold { |
| 277 | + return corrCol, corr |
| 278 | + } |
| 279 | + return nil, corr |
| 280 | +} |
| 281 | + |
| 282 | +// getMostCorrCol4Handle checks if column in the condition is correlated enough with handle. If the condition |
| 283 | +// contains multiple columns, return nil and get the max correlation, which would be used in the heuristic estimation. |
| 284 | +func getMostCorrCol4Handle(exprs []expression.Expression, histColl *statistics.Table, threshold float64) (*expression.Column, float64) { |
| 285 | + var cols []*expression.Column |
| 286 | + cols = expression.ExtractColumnsFromExpressions(cols, exprs, nil) |
| 287 | + if len(cols) == 0 { |
| 288 | + return nil, 0 |
| 289 | + } |
| 290 | + colSet := set.NewInt64Set() |
| 291 | + var corr float64 |
| 292 | + var corrCol *expression.Column |
| 293 | + for _, col := range cols { |
| 294 | + if colSet.Exist(col.UniqueID) { |
| 295 | + continue |
| 296 | + } |
| 297 | + colSet.Insert(col.UniqueID) |
| 298 | + hist, ok := histColl.Columns[col.ID] |
| 299 | + if !ok { |
| 300 | + continue |
| 301 | + } |
| 302 | + curCorr := hist.Correlation |
| 303 | + if corrCol == nil || math.Abs(corr) < math.Abs(curCorr) { |
| 304 | + corrCol = col |
| 305 | + corr = curCorr |
| 306 | + } |
| 307 | + } |
| 308 | + if len(colSet) == 1 && math.Abs(corr) >= threshold { |
| 309 | + return corrCol, corr |
| 310 | + } |
| 311 | + return nil, corr |
| 312 | +} |
0 commit comments