From b327024c5a09a5dd5b8be78ffa1f98391e940143 Mon Sep 17 00:00:00 2001 From: AilinKid <314806019@qq.com> Date: Fri, 16 Aug 2024 17:10:44 +0800 Subject: [PATCH] . Signed-off-by: AilinKid <314806019@qq.com> --- pkg/planner/core/exhaust_physical_plans.go | 2 +- pkg/planner/core/find_best_task.go | 134 ++++++++++----------- pkg/planner/core/indexmerge_path.go | 5 +- pkg/planner/core/logical_datasource.go | 6 +- pkg/planner/core/logical_plans_test.go | 2 +- pkg/planner/core/stats.go | 2 +- 6 files changed, 76 insertions(+), 75 deletions(-) diff --git a/pkg/planner/core/exhaust_physical_plans.go b/pkg/planner/core/exhaust_physical_plans.go index d6ee33080748f..a498ddd7dc339 100644 --- a/pkg/planner/core/exhaust_physical_plans.go +++ b/pkg/planner/core/exhaust_physical_plans.go @@ -1239,7 +1239,7 @@ func constructInnerIndexScanTask( cop.commonHandleCols = ds.CommonHandleCols } is.initSchema(append(path.FullIdxCols, ds.CommonHandleCols...), cop.tablePlan != nil) - indexConds, tblConds := ds.splitIndexFilterConditions(filterConds, path.FullIdxCols, path.FullIdxColLens) + indexConds, tblConds := splitIndexFilterConditions(ds, filterConds, path.FullIdxCols, path.FullIdxColLens) // Note: due to a regression in JOB workload, we use the optimizer fix control to enable this for now. // diff --git a/pkg/planner/core/find_best_task.go b/pkg/planner/core/find_best_task.go index 6b0e5a854eb0a..2264bf55ba35b 100644 --- a/pkg/planner/core/find_best_task.go +++ b/pkg/planner/core/find_best_task.go @@ -653,7 +653,7 @@ func findBestTask4LogicalMemTable(lp base.LogicalPlan, prop *property.PhysicalPr } // tryToGetDualTask will check if the push down predicate has false constant. If so, it will return table dual. -func (ds *DataSource) tryToGetDualTask() (base.Task, error) { +func tryToGetDualTask(ds *DataSource) (base.Task, error) { for _, cond := range ds.PushedDownConds { if con, ok := cond.(*expression.Constant); ok && con.DeferredExpr == nil && con.ParamMarker == nil { result, _, err := expression.EvalBool(ds.SCtx().GetExprCtx().GetEvalCtx(), []expression.Expression{cond}, chunk.Row{}) @@ -750,7 +750,7 @@ func compareCandidates(sctx base.PlanContext, prop *property.PhysicalProperty, l return 0 } -func (ds *DataSource) isMatchProp(path *util.AccessPath, prop *property.PhysicalProperty) bool { +func isMatchProp(ds *DataSource, path *util.AccessPath, prop *property.PhysicalProperty) bool { var isMatchProp bool if path.IsIntHandlePath { pkCol := ds.getPKIsHandleCol() @@ -856,7 +856,7 @@ func (ds *DataSource) isMatchProp(path *util.AccessPath, prop *property.Physical // // at last, according to determinedIndexPartialPaths to rewrite their real countAfterAccess, this part is move from deriveStats to // here. -func (ds *DataSource) matchPropForIndexMergeAlternatives(path *util.AccessPath, prop *property.PhysicalProperty) (*util.AccessPath, bool) { +func matchPropForIndexMergeAlternatives(ds *DataSource, path *util.AccessPath, prop *property.PhysicalProperty) (*util.AccessPath, bool) { // target: // 1: index merge case, try to match the every alternative partial path to the order property as long as // possible, and generate that property-matched index merge path out if any. @@ -899,7 +899,7 @@ func (ds *DataSource) matchPropForIndexMergeAlternatives(path *util.AccessPath, matchIdxes := make([]int, 0, 1) for i, oneIndexAlternativePath := range oneItemAlternatives { // if there is some sort items and this path doesn't match this prop, continue. - if !noSortItem && !ds.isMatchProp(oneIndexAlternativePath, prop) { + if !noSortItem && !isMatchProp(ds, oneIndexAlternativePath, prop) { continue } // two possibility here: @@ -1031,7 +1031,7 @@ func (ds *DataSource) matchPropForIndexMergeAlternatives(path *util.AccessPath, return indexMergePath, true } -func (ds *DataSource) isMatchPropForIndexMerge(path *util.AccessPath, prop *property.PhysicalProperty) bool { +func isMatchPropForIndexMerge(ds *DataSource, path *util.AccessPath, prop *property.PhysicalProperty) bool { // Execution part doesn't support the merge operation for intersection case yet. if path.IndexMergeIsIntersection { return false @@ -1041,31 +1041,31 @@ func (ds *DataSource) isMatchPropForIndexMerge(path *util.AccessPath, prop *prop return false } for _, partialPath := range path.PartialIndexPaths { - if !ds.isMatchProp(partialPath, prop) { + if !isMatchProp(ds, partialPath, prop) { return false } } return true } -func (ds *DataSource) getTableCandidate(path *util.AccessPath, prop *property.PhysicalProperty) *candidatePath { +func getTableCandidate(ds *DataSource, path *util.AccessPath, prop *property.PhysicalProperty) *candidatePath { candidate := &candidatePath{path: path} - candidate.isMatchProp = ds.isMatchProp(path, prop) + candidate.isMatchProp = isMatchProp(ds, path, prop) candidate.accessCondsColMap = util.ExtractCol2Len(ds.SCtx().GetExprCtx().GetEvalCtx(), path.AccessConds, nil, nil) return candidate } -func (ds *DataSource) getIndexCandidate(path *util.AccessPath, prop *property.PhysicalProperty) *candidatePath { +func getIndexCandidate(ds *DataSource, path *util.AccessPath, prop *property.PhysicalProperty) *candidatePath { candidate := &candidatePath{path: path} - candidate.isMatchProp = ds.isMatchProp(path, prop) + candidate.isMatchProp = isMatchProp(ds, path, prop) candidate.accessCondsColMap = util.ExtractCol2Len(ds.SCtx().GetExprCtx().GetEvalCtx(), path.AccessConds, path.IdxCols, path.IdxColLens) candidate.indexCondsColMap = util.ExtractCol2Len(ds.SCtx().GetExprCtx().GetEvalCtx(), append(path.AccessConds, path.IndexFilters...), path.FullIdxCols, path.FullIdxColLens) return candidate } -func (ds *DataSource) convergeIndexMergeCandidate(path *util.AccessPath, prop *property.PhysicalProperty) *candidatePath { +func convergeIndexMergeCandidate(ds *DataSource, path *util.AccessPath, prop *property.PhysicalProperty) *candidatePath { // since the all index path alternative paths is collected and undetermined, and we should determine a possible and concrete path for this prop. - possiblePath, match := ds.matchPropForIndexMergeAlternatives(path, prop) + possiblePath, match := matchPropForIndexMergeAlternatives(ds, path, prop) if possiblePath == nil { return nil } @@ -1073,15 +1073,15 @@ func (ds *DataSource) convergeIndexMergeCandidate(path *util.AccessPath, prop *p return candidate } -func (ds *DataSource) getIndexMergeCandidate(path *util.AccessPath, prop *property.PhysicalProperty) *candidatePath { +func getIndexMergeCandidate(ds *DataSource, path *util.AccessPath, prop *property.PhysicalProperty) *candidatePath { candidate := &candidatePath{path: path} - candidate.isMatchProp = ds.isMatchPropForIndexMerge(path, prop) + candidate.isMatchProp = isMatchPropForIndexMerge(ds, path, prop) return candidate } // skylinePruning prunes access paths according to different factors. An access path can be pruned only if // there exists a path that is not worse than it at all factors and there is at least one better factor. -func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candidatePath { +func skylinePruning(ds *DataSource, prop *property.PhysicalProperty) []*candidatePath { candidates := make([]*candidatePath, 0, 4) for _, path := range ds.PossibleAccessPaths { // We should check whether the possible access path is valid first. @@ -1090,14 +1090,14 @@ func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candida } if len(path.PartialAlternativeIndexPaths) > 0 { // OR normal index merge path, try to determine every index partial path for this property. - candidate := ds.convergeIndexMergeCandidate(path, prop) + candidate := convergeIndexMergeCandidate(ds, path, prop) if candidate != nil { candidates = append(candidates, candidate) } continue } if path.PartialIndexPaths != nil { - candidates = append(candidates, ds.getIndexMergeCandidate(path, prop)) + candidates = append(candidates, getIndexMergeCandidate(ds, path, prop)) continue } // if we already know the range of the scan is empty, just return a TableDual @@ -1106,7 +1106,7 @@ func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candida } var currentCandidate *candidatePath if path.IsTablePath() { - currentCandidate = ds.getTableCandidate(path, prop) + currentCandidate = getTableCandidate(ds, path, prop) } else { if !(len(path.AccessConds) > 0 || !prop.IsSortItemEmpty() || path.Forced || path.IsSingleScan) { continue @@ -1116,7 +1116,7 @@ func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candida // 2. We have a non-empty prop to match. // 3. This index is forced to choose. // 4. The needed columns are all covered by index columns(and handleCol). - currentCandidate = ds.getIndexCandidate(path, prop) + currentCandidate = getIndexCandidate(ds, path, prop) } pruned := false for i := len(candidates) - 1; i >= 0; i-- { @@ -1166,7 +1166,7 @@ func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candida return candidates } -func (ds *DataSource) getPruningInfo(candidates []*candidatePath, prop *property.PhysicalProperty) string { +func getPruningInfo(ds *DataSource, candidates []*candidatePath, prop *property.PhysicalProperty) string { if len(candidates) == len(ds.PossibleAccessPaths) { return "" } @@ -1209,7 +1209,7 @@ func (ds *DataSource) getPruningInfo(candidates []*candidatePath, prop *property strings.Join(names, ","), tableName, strings.Join(items, " "), prop.TaskTp) } -func (ds *DataSource) isPointGetConvertableSchema() bool { +func isPointGetConvertableSchema(ds *DataSource) bool { for _, col := range ds.Columns { if col.Name.L == model.ExtraHandleName.L { continue @@ -1225,7 +1225,7 @@ func (ds *DataSource) isPointGetConvertableSchema() bool { // exploreEnforcedPlan determines whether to explore enforced plans for this DataSource if it has already found an unenforced plan. // See #46177 for more information. -func (ds *DataSource) exploreEnforcedPlan() bool { +func exploreEnforcedPlan(ds *DataSource) bool { // default value is false to keep it compatible with previous versions. return fixcontrol.GetBoolWithDefault(ds.SCtx().GetSessionVars().GetOptimizerFixControlMap(), fixcontrol.Fix46177, false) } @@ -1240,7 +1240,7 @@ func findBestTask4DS(ds *DataSource, prop *property.PhysicalProperty, planCounte if ds.IsForUpdateRead && ds.SCtx().GetSessionVars().TxnCtx.IsExplicit { hasPointGetPath := false for _, path := range ds.PossibleAccessPaths { - if ds.isPointGetPath(path) { + if isPointGetPath(ds, path) { hasPointGetPath = true break } @@ -1279,7 +1279,7 @@ func findBestTask4DS(ds *DataSource, prop *property.PhysicalProperty, planCounte if err != nil { return nil, 0, err } - if !unenforcedTask.Invalid() && !ds.exploreEnforcedPlan() { + if !unenforcedTask.Invalid() && !exploreEnforcedPlan(ds) { ds.StoreTask(prop, unenforcedTask) return unenforcedTask, cnt, nil } @@ -1317,7 +1317,7 @@ func findBestTask4DS(ds *DataSource, prop *property.PhysicalProperty, planCounte err = validateTableSamplePlan(ds, t, err) }() - t, err = ds.tryToGetDualTask() + t, err = tryToGetDualTask(ds) if err != nil || t != nil { planCounter.Dec(1) if t != nil { @@ -1327,8 +1327,8 @@ func findBestTask4DS(ds *DataSource, prop *property.PhysicalProperty, planCounte } t = base.InvalidTask - candidates := ds.skylinePruning(prop) - pruningInfo := ds.getPruningInfo(candidates, prop) + candidates := skylinePruning(ds, prop) + pruningInfo := getPruningInfo(ds, candidates, prop) defer func() { if err == nil && t != nil && !t.Invalid() && pruningInfo != "" { warnErr := errors.NewNoStackError(pruningInfo) @@ -1344,7 +1344,7 @@ func findBestTask4DS(ds *DataSource, prop *property.PhysicalProperty, planCounte for _, candidate := range candidates { path := candidate.path if path.PartialIndexPaths != nil { - idxMergeTask, err := ds.convertToIndexMergeScan(prop, candidate, opt) + idxMergeTask, err := convertToIndexMergeScan(ds, prop, candidate, opt) if err != nil { return nil, 0, err } @@ -1382,7 +1382,7 @@ func findBestTask4DS(ds *DataSource, prop *property.PhysicalProperty, planCounte return t, cntPlan, nil } - canConvertPointGet := len(path.Ranges) > 0 && path.StoreType == kv.TiKV && ds.isPointGetConvertableSchema() + canConvertPointGet := len(path.Ranges) > 0 && path.StoreType == kv.TiKV && isPointGetConvertableSchema(ds) if canConvertPointGet && path.Index != nil && path.Index.MVIndex { canConvertPointGet = false // cannot use PointGet upon MVIndex @@ -1449,9 +1449,9 @@ func findBestTask4DS(ds *DataSource, prop *property.PhysicalProperty, planCounte if allRangeIsPoint { var pointGetTask base.Task if len(path.Ranges) == 1 { - pointGetTask = ds.convertToPointGet(prop, candidate) + pointGetTask = convertToPointGet(ds, prop, candidate) } else { - pointGetTask = ds.convertToBatchPointGet(prop, candidate) + pointGetTask = convertToBatchPointGet(ds, prop, candidate) } // Batch/PointGet plans may be over-optimized, like `a>=1(?) and a<=1(?)` --> `a=1` --> PointGet(a=1). @@ -1487,9 +1487,9 @@ func findBestTask4DS(ds *DataSource, prop *property.PhysicalProperty, planCounte } var tblTask base.Task if ds.SampleInfo != nil { - tblTask, err = ds.convertToSampleTable(prop, candidate, opt) + tblTask, err = convertToSampleTable(ds, prop, candidate, opt) } else { - tblTask, err = ds.convertToTableScan(prop, candidate, opt) + tblTask, err = convertToTableScan(ds, prop, candidate, opt) } if err != nil { return nil, 0, err @@ -1519,7 +1519,7 @@ func findBestTask4DS(ds *DataSource, prop *property.PhysicalProperty, planCounte if ds.SampleInfo != nil { continue } - idxTask, err := ds.convertToIndexScan(prop, candidate, opt) + idxTask, err := convertToIndexScan(ds, prop, candidate, opt) if err != nil { return nil, 0, err } @@ -1544,7 +1544,7 @@ func findBestTask4DS(ds *DataSource, prop *property.PhysicalProperty, planCounte } // convertToIndexMergeScan builds the index merge scan for intersection or union cases. -func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, candidate *candidatePath, _ *optimizetrace.PhysicalOptimizeOp) (task base.Task, err error) { +func convertToIndexMergeScan(ds *DataSource, prop *property.PhysicalProperty, candidate *candidatePath, _ *optimizetrace.PhysicalOptimizeOp) (task base.Task, err error) { if prop.IsFlashProp() || prop.TaskTp == property.CopSingleReadTaskType { return base.InvalidTask, nil } @@ -1589,10 +1589,10 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c for _, partPath := range path.PartialIndexPaths { var scan base.PhysicalPlan if partPath.IsTablePath() { - scan = ds.convertToPartialTableScan(prop, partPath, candidate.isMatchProp, byItems) + scan = convertToPartialTableScan(ds, prop, partPath, candidate.isMatchProp, byItems) } else { var remainingFilters []expression.Expression - scan, remainingFilters, err = ds.convertToPartialIndexScan(cop.physPlanPartInfo, prop, partPath, candidate.isMatchProp, byItems) + scan, remainingFilters, err = convertToPartialIndexScan(ds, cop.physPlanPartInfo, prop, partPath, candidate.isMatchProp, byItems) if err != nil { return base.InvalidTask, err } @@ -1607,7 +1607,7 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c if prop.ExpectedCnt < ds.StatsInfo().RowCount { totalRowCount *= prop.ExpectedCnt / ds.StatsInfo().RowCount } - ts, remainingFilters2, moreColumn, err := ds.buildIndexMergeTableScan(path.TableFilters, totalRowCount, candidate.isMatchProp) + ts, remainingFilters2, moreColumn, err := buildIndexMergeTableScan(ds, path.TableFilters, totalRowCount, candidate.isMatchProp) if err != nil { return base.InvalidTask, err } @@ -1645,8 +1645,8 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c return task, nil } -func (ds *DataSource) convertToPartialIndexScan(physPlanPartInfo *PhysPlanPartInfo, prop *property.PhysicalProperty, path *util.AccessPath, matchProp bool, byItems []*util.ByItems) (base.PhysicalPlan, []expression.Expression, error) { - is := ds.getOriginalPhysicalIndexScan(prop, path, matchProp, false) +func convertToPartialIndexScan(ds *DataSource, physPlanPartInfo *PhysPlanPartInfo, prop *property.PhysicalProperty, path *util.AccessPath, matchProp bool, byItems []*util.ByItems) (base.PhysicalPlan, []expression.Expression, error) { + is := getOriginalPhysicalIndexScan(ds, prop, path, matchProp, false) // TODO: Consider using isIndexCoveringColumns() to avoid another TableRead indexConds := path.IndexFilters if matchProp { @@ -1692,8 +1692,8 @@ func checkColinSchema(cols []*expression.Column, schema *expression.Schema) bool return true } -func (ds *DataSource) convertToPartialTableScan(prop *property.PhysicalProperty, path *util.AccessPath, matchProp bool, byItems []*util.ByItems) (tablePlan base.PhysicalPlan) { - ts, rowCount := ds.getOriginalPhysicalTableScan(prop, path, matchProp) +func convertToPartialTableScan(ds *DataSource, prop *property.PhysicalProperty, path *util.AccessPath, matchProp bool, byItems []*util.ByItems) (tablePlan base.PhysicalPlan) { + ts, rowCount := getOriginalPhysicalTableScan(ds, prop, path, matchProp) overwritePartialTableScanSchema(ds, ts) // remove ineffetive filter condition after overwriting physicalscan schema newFilterConds := make([]expression.Expression, 0, len(path.TableFilters)) @@ -1764,7 +1764,7 @@ func setIndexMergeTableScanHandleCols(ds *DataSource, ts *PhysicalTableScan) (er // buildIndexMergeTableScan() returns Selection that will be pushed to TiKV. // Filters that cannot be pushed to TiKV are also returned, and an extra Selection above IndexMergeReader will be constructed later. -func (ds *DataSource) buildIndexMergeTableScan(tableFilters []expression.Expression, +func buildIndexMergeTableScan(ds *DataSource, tableFilters []expression.Expression, totalRowCount float64, matchProp bool) (base.PhysicalPlan, []expression.Expression, bool, error) { ts := PhysicalTableScan{ Table: ds.TableInfo, @@ -1883,7 +1883,7 @@ func isIndexColsCoveringCol(sctx expression.EvalContext, col *expression.Column, return false } -func (ds *DataSource) indexCoveringColumn(column *expression.Column, indexColumns []*expression.Column, idxColLens []int, ignoreLen bool) bool { +func indexCoveringColumn(ds *DataSource, column *expression.Column, indexColumns []*expression.Column, idxColLens []int, ignoreLen bool) bool { if ds.TableInfo.PKIsHandle && mysql.HasPriKeyFlag(column.RetType.GetFlag()) { return true } @@ -1905,28 +1905,28 @@ func (ds *DataSource) indexCoveringColumn(column *expression.Column, indexColumn return true } -func (ds *DataSource) isIndexCoveringColumns(columns, indexColumns []*expression.Column, idxColLens []int) bool { +func isIndexCoveringColumns(ds *DataSource, columns, indexColumns []*expression.Column, idxColLens []int) bool { for _, col := range columns { - if !ds.indexCoveringColumn(col, indexColumns, idxColLens, false) { + if !indexCoveringColumn(ds, col, indexColumns, idxColLens, false) { return false } } return true } -func (ds *DataSource) isIndexCoveringCondition(condition expression.Expression, indexColumns []*expression.Column, idxColLens []int) bool { +func isIndexCoveringCondition(ds *DataSource, condition expression.Expression, indexColumns []*expression.Column, idxColLens []int) bool { switch v := condition.(type) { case *expression.Column: - return ds.indexCoveringColumn(v, indexColumns, idxColLens, false) + return indexCoveringColumn(ds, v, indexColumns, idxColLens, false) case *expression.ScalarFunction: // Even if the index only contains prefix `col`, the index can cover `col is null`. if v.FuncName.L == ast.IsNull { if col, ok := v.GetArgs()[0].(*expression.Column); ok { - return ds.indexCoveringColumn(col, indexColumns, idxColLens, true) + return indexCoveringColumn(ds, col, indexColumns, idxColLens, true) } } for _, arg := range v.GetArgs() { - if !ds.isIndexCoveringCondition(arg, indexColumns, idxColLens) { + if !isIndexCoveringCondition(ds, arg, indexColumns, idxColLens) { return false } } @@ -1935,17 +1935,17 @@ func (ds *DataSource) isIndexCoveringCondition(condition expression.Expression, return true } -func (ds *DataSource) isSingleScan(indexColumns []*expression.Column, idxColLens []int) bool { +func isSingleScan(ds *DataSource, indexColumns []*expression.Column, idxColLens []int) bool { if !ds.SCtx().GetSessionVars().OptPrefixIndexSingleScan || ds.ColsRequiringFullLen == nil { // ds.ColsRequiringFullLen is set at (*DataSource).PruneColumns. In some cases we don't reach (*DataSource).PruneColumns // and ds.ColsRequiringFullLen is nil, so we fall back to ds.isIndexCoveringColumns(ds.schema.Columns, indexColumns, idxColLens). - return ds.isIndexCoveringColumns(ds.Schema().Columns, indexColumns, idxColLens) + return isIndexCoveringColumns(ds, ds.Schema().Columns, indexColumns, idxColLens) } - if !ds.isIndexCoveringColumns(ds.ColsRequiringFullLen, indexColumns, idxColLens) { + if !isIndexCoveringColumns(ds, ds.ColsRequiringFullLen, indexColumns, idxColLens) { return false } for _, cond := range ds.AllConds { - if !ds.isIndexCoveringCondition(cond, indexColumns, idxColLens) { + if !isIndexCoveringCondition(ds, cond, indexColumns, idxColLens) { return false } } @@ -1965,7 +1965,7 @@ func (ts *PhysicalTableScan) appendExtraHandleCol(ds *DataSource) (*expression.C } // convertToIndexScan converts the DataSource to index scan with idx. -func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, +func convertToIndexScan(ds *DataSource, prop *property.PhysicalProperty, candidate *candidatePath, _ *optimizetrace.PhysicalOptimizeOp) (task base.Task, err error) { if candidate.path.Index.MVIndex { // MVIndex is special since different index rows may return the same _row_id and this can break some assumptions of IndexReader. @@ -1994,7 +1994,7 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, return base.InvalidTask, nil } path := candidate.path - is := ds.getOriginalPhysicalIndexScan(prop, path, candidate.isMatchProp, candidate.path.IsSingleScan) + is := getOriginalPhysicalIndexScan(ds, prop, path, candidate.isMatchProp, candidate.path.IsSingleScan) cop := &CopTask{ indexPlan: is, tblColHists: ds.TblColHists, @@ -2342,15 +2342,15 @@ func matchIndicesProp(sctx base.PlanContext, idxCols []*expression.Column, colLe return true } -func (ds *DataSource) splitIndexFilterConditions(conditions []expression.Expression, indexColumns []*expression.Column, +func splitIndexFilterConditions(ds *DataSource, conditions []expression.Expression, indexColumns []*expression.Column, idxColLens []int) (indexConds, tableConds []expression.Expression) { var indexConditions, tableConditions []expression.Expression for _, cond := range conditions { var covered bool if ds.SCtx().GetSessionVars().OptPrefixIndexSingleScan { - covered = ds.isIndexCoveringCondition(cond, indexColumns, idxColLens) + covered = isIndexCoveringCondition(ds, cond, indexColumns, idxColLens) } else { - covered = ds.isIndexCoveringColumns(expression.ExtractColumns(cond), indexColumns, idxColLens) + covered = isIndexCoveringColumns(ds, expression.ExtractColumns(cond), indexColumns, idxColLens) } if covered { indexConditions = append(indexConditions, cond) @@ -2409,7 +2409,7 @@ func GetPhysicalIndexScan4LogicalIndexScan(s *LogicalIndexScan, _ *expression.Sc // eg: create table t(a int, b int,c int unique, primary (a,b)) // select * from t where a = 1 and b = 1 and c =1; // the datasource can access by primary key(a,b) or unique key c which are both point-get-able -func (ds *DataSource) isPointGetPath(path *util.AccessPath) bool { +func isPointGetPath(ds *DataSource, path *util.AccessPath) bool { if len(path.Ranges) < 1 { return false } @@ -2437,7 +2437,7 @@ func (ds *DataSource) isPointGetPath(path *util.AccessPath) bool { } // convertToTableScan converts the DataSource to table scan. -func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candidate *candidatePath, _ *optimizetrace.PhysicalOptimizeOp) (base.Task, error) { +func convertToTableScan(ds *DataSource, prop *property.PhysicalProperty, candidate *candidatePath, _ *optimizetrace.PhysicalOptimizeOp) (base.Task, error) { // It will be handled in convertToIndexScan. if prop.TaskTp == property.CopMultiReadTaskType { return base.InvalidTask, nil @@ -2453,7 +2453,7 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid if !prop.IsSortItemEmpty() && candidate.path.ForceNoKeepOrder { return base.InvalidTask, nil } - ts, _ := ds.getOriginalPhysicalTableScan(prop, candidate.path, candidate.isMatchProp) + ts, _ := getOriginalPhysicalTableScan(ds, prop, candidate.path, candidate.isMatchProp) if ts.KeepOrder && ts.StoreType == kv.TiFlash && (ts.Desc || ds.SCtx().GetSessionVars().TiFlashFastScan) { // TiFlash fast mode(https://github.com/pingcap/tidb/pull/35851) does not keep order in TableScan return base.InvalidTask, nil @@ -2577,7 +2577,7 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid return task, nil } -func (ds *DataSource) convertToSampleTable(prop *property.PhysicalProperty, +func convertToSampleTable(ds *DataSource, prop *property.PhysicalProperty, candidate *candidatePath, _ *optimizetrace.PhysicalOptimizeOp) (base.Task, error) { if prop.TaskTp == property.CopMultiReadTaskType { return base.InvalidTask, nil @@ -2601,7 +2601,7 @@ func (ds *DataSource) convertToSampleTable(prop *property.PhysicalProperty, return rt, nil } -func (ds *DataSource) convertToPointGet(prop *property.PhysicalProperty, candidate *candidatePath) base.Task { +func convertToPointGet(ds *DataSource, prop *property.PhysicalProperty, candidate *candidatePath) base.Task { if !prop.IsSortItemEmpty() && !candidate.isMatchProp { return base.InvalidTask } @@ -2677,7 +2677,7 @@ func (ds *DataSource) convertToPointGet(prop *property.PhysicalProperty, candida return rTsk } -func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, candidate *candidatePath) base.Task { +func convertToBatchPointGet(ds *DataSource, prop *property.PhysicalProperty, candidate *candidatePath) base.Task { if !prop.IsSortItemEmpty() && !candidate.isMatchProp { return base.InvalidTask } @@ -2811,7 +2811,7 @@ func (ts *PhysicalTableScan) getScanRowSize() float64 { return cardinality.GetTableAvgRowSize(ts.SCtx(), ts.tblColHists, ts.Schema().Columns, ts.StoreType, ts.HandleCols != nil) } -func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProperty, path *util.AccessPath, isMatchProp bool) (*PhysicalTableScan, float64) { +func getOriginalPhysicalTableScan(ds *DataSource, prop *property.PhysicalProperty, path *util.AccessPath, isMatchProp bool) (*PhysicalTableScan, float64) { ts := PhysicalTableScan{ Table: ds.TableInfo, Columns: slices.Clone(ds.Columns), @@ -2853,7 +2853,7 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper return ts, rowCount } -func (ds *DataSource) getOriginalPhysicalIndexScan(prop *property.PhysicalProperty, path *util.AccessPath, isMatchProp bool, isSingleScan bool) *PhysicalIndexScan { +func getOriginalPhysicalIndexScan(ds *DataSource, prop *property.PhysicalProperty, path *util.AccessPath, isMatchProp bool, isSingleScan bool) *PhysicalIndexScan { idx := path.Index is := PhysicalIndexScan{ Table: ds.TableInfo, diff --git a/pkg/planner/core/indexmerge_path.go b/pkg/planner/core/indexmerge_path.go index ce49e872016bf..f0ce254dae567 100644 --- a/pkg/planner/core/indexmerge_path.go +++ b/pkg/planner/core/indexmerge_path.go @@ -985,7 +985,7 @@ func (ds *DataSource) generateIndexMerge4ComposedIndex(normalPathCnt int, indexM condInIdxFilter := make(map[string]struct{}, len(remainedCNFs)) // try to derive index filters for each path for _, path := range combinedPartialPaths { - idxFilters, _ := ds.splitIndexFilterConditions(remainedCNFs, path.FullIdxCols, path.FullIdxColLens) + idxFilters, _ := splitIndexFilterConditions(ds, remainedCNFs, path.FullIdxCols, path.FullIdxColLens) idxFilters = util.CloneExprs(idxFilters) path.IndexFilters = append(path.IndexFilters, idxFilters...) for _, idxFilter := range idxFilters { @@ -1072,7 +1072,8 @@ func (ds *DataSource) generateIndexMerge4MVIndex(normalPathCnt int, filters []ex // metadata. // And according to buildPartialPaths4MVIndex, there must be at least one partial path if it returns ok. firstPath := partialPaths[0] - idxFilters, tableFilters := ds.splitIndexFilterConditions( + idxFilters, tableFilters := splitIndexFilterConditions( + ds, remainingFilters, firstPath.FullIdxCols, firstPath.FullIdxColLens, diff --git a/pkg/planner/core/logical_datasource.go b/pkg/planner/core/logical_datasource.go index 77dbbcaeda94a..e831210d5a2f2 100644 --- a/pkg/planner/core/logical_datasource.go +++ b/pkg/planner/core/logical_datasource.go @@ -578,8 +578,8 @@ func (ds *DataSource) Convert2Gathers() (gathers []base.LogicalPlan) { if !path.IsIntHandlePath { path.FullIdxCols, path.FullIdxColLens = expression.IndexInfo2Cols(ds.Columns, ds.Schema().Columns, path.Index) path.IdxCols, path.IdxColLens = expression.IndexInfo2PrefixCols(ds.Columns, ds.Schema().Columns, path.Index) - // If index columns can cover all of the needed columns, we can use a IndexGather + IndexScan. - if ds.isSingleScan(path.FullIdxCols, path.FullIdxColLens) { + // If index columns can cover all the needed columns, we can use a IndexGather + IndexScan. + if isSingleScan(ds, path.FullIdxCols, path.FullIdxColLens) { gathers = append(gathers, ds.buildIndexGather(path)) } // TODO: If index columns can not cover the schema, use IndexLookUpGather. @@ -803,7 +803,7 @@ func (ds *DataSource) deriveIndexPathStats(path *util.AccessPath, _ []expression } } var indexFilters []expression.Expression - indexFilters, path.TableFilters = ds.splitIndexFilterConditions(path.TableFilters, path.FullIdxCols, path.FullIdxColLens) + indexFilters, path.TableFilters = splitIndexFilterConditions(ds, path.TableFilters, path.FullIdxCols, path.FullIdxColLens) path.IndexFilters = append(path.IndexFilters, indexFilters...) // If the `CountAfterAccess` is less than `stats.RowCount`, there must be some inconsistent stats info. // We prefer the `stats.RowCount` because it could use more stats info to calculate the selectivity. diff --git a/pkg/planner/core/logical_plans_test.go b/pkg/planner/core/logical_plans_test.go index fbb05bf3974ee..89f8e812dcb78 100644 --- a/pkg/planner/core/logical_plans_test.go +++ b/pkg/planner/core/logical_plans_test.go @@ -2013,7 +2013,7 @@ func TestSkylinePruning(t *testing.T) { lp = lp.Children()[0] } } - paths := ds.skylinePruning(byItemsToProperty(byItems)) + paths := skylinePruning(ds, byItemsToProperty(byItems)) require.Equal(t, tt.result, pathsName(paths), comment) domain.GetDomain(sctx).StatsHandle().Close() } diff --git a/pkg/planner/core/stats.go b/pkg/planner/core/stats.go index cc8d83d6ee216..c80ae25151d18 100644 --- a/pkg/planner/core/stats.go +++ b/pkg/planner/core/stats.go @@ -229,7 +229,7 @@ func (ds *DataSource) derivePathStatsAndTryHeuristics() error { path.IsSingleScan = true } else { ds.deriveIndexPathStats(path, ds.PushedDownConds, false) - path.IsSingleScan = ds.isSingleScan(path.FullIdxCols, path.FullIdxColLens) + path.IsSingleScan = isSingleScan(ds, path.FullIdxCols, path.FullIdxColLens) } // step: 3 // Try some heuristic rules to select access path.