diff --git a/pkg/planner/core/indexmerge_path.go b/pkg/planner/core/indexmerge_path.go index f0ce254dae567..4364be98a3429 100644 --- a/pkg/planner/core/indexmerge_path.go +++ b/pkg/planner/core/indexmerge_path.go @@ -44,7 +44,7 @@ import ( ) // generateIndexMergePath generates IndexMerge AccessPaths on this DataSource. -func (ds *DataSource) generateIndexMergePath() error { +func generateIndexMergePath(ds *DataSource) error { if ds.SCtx().GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { debugtrace.EnterContextCommon(ds.SCtx()) defer debugtrace.LeaveContextCommon(ds.SCtx()) @@ -81,14 +81,14 @@ func (ds *DataSource) generateIndexMergePath() error { regularPathCount := len(ds.PossibleAccessPaths) var err error - if warningMsg, err = ds.generateIndexMerge4NormalIndex(regularPathCount, indexMergeConds); err != nil { + if warningMsg, err = generateIndexMerge4NormalIndex(ds, regularPathCount, indexMergeConds); err != nil { return err } - if err := ds.generateIndexMerge4MVIndex(regularPathCount, indexMergeConds); err != nil { + if err := generateIndexMerge4MVIndex(ds, regularPathCount, indexMergeConds); err != nil { return err } oldIndexMergeCount := len(ds.PossibleAccessPaths) - if err := ds.generateIndexMerge4ComposedIndex(regularPathCount, indexMergeConds); err != nil { + if err := generateIndexMerge4ComposedIndex(ds, regularPathCount, indexMergeConds); err != nil { return err } @@ -127,12 +127,13 @@ func (ds *DataSource) generateIndexMergePath() error { } // If there is a multi-valued index hint, remove all paths which don't use the specified index. - ds.cleanAccessPathForMVIndexHint() + cleanAccessPathForMVIndexHint(ds) return nil } -func (ds *DataSource) generateNormalIndexPartialPaths4DNF( +func generateNormalIndexPartialPaths4DNF( + ds *DataSource, dnfItems []expression.Expression, candidatePaths []*util.AccessPath, ) (paths []*util.AccessPath, needSelection bool, usedMap []bool) { @@ -149,7 +150,7 @@ func (ds *DataSource) generateNormalIndexPartialPaths4DNF( needSelection = true } } - itemPaths := ds.accessPathsForConds(pushedDownCNFItems, candidatePaths) + itemPaths := accessPathsForConds(ds, pushedDownCNFItems, candidatePaths) if len(itemPaths) == 0 { // for this dnf item, we couldn't generate an index merge partial path. // (1 member of (a)) or (3 member of (b)) or d=1; if one dnf item like d=1 here could walk index path, @@ -205,7 +206,7 @@ func (ds *DataSource) generateNormalIndexPartialPaths4DNF( // PartialIndexPaths: empty // 1D array here, currently is not decided yet. // PartialAlternativeIndexPaths: [[a, ac], [b, bc]] // 2D array here, each for one DNF item choices. // } -func (ds *DataSource) generateIndexMergeOrPaths(filters []expression.Expression) error { +func generateIndexMergeOrPaths(ds *DataSource, filters []expression.Expression) error { usedIndexCount := len(ds.PossibleAccessPaths) pushDownCtx := util.GetPushDownCtx(ds.SCtx()) for k, cond := range filters { @@ -230,7 +231,7 @@ func (ds *DataSource) generateIndexMergeOrPaths(filters []expression.Expression) } } - itemPaths := ds.accessPathsForConds(pushedDownCNFItems, ds.PossibleAccessPaths[:usedIndexCount]) + itemPaths := accessPathsForConds(ds, pushedDownCNFItems, ds.PossibleAccessPaths[:usedIndexCount]) if len(itemPaths) == 0 { partialAlternativePaths = nil break @@ -291,7 +292,7 @@ func (ds *DataSource) generateIndexMergeOrPaths(filters []expression.Expression) // isInIndexMergeHints returns true if the input index name is not excluded by the IndexMerge hints, which means either // (1) there's no IndexMerge hint, (2) there's IndexMerge hint but no specified index names, or (3) the input index // name is specified in the IndexMerge hints. -func (ds *DataSource) isInIndexMergeHints(name string) bool { +func isInIndexMergeHints(ds *DataSource, name string) bool { // if no index merge hints, all mv index is accessible if len(ds.IndexMergeHints) == 0 { return true @@ -310,7 +311,7 @@ func (ds *DataSource) isInIndexMergeHints(name string) bool { } // indexMergeHintsHasSpecifiedIdx returns true if there's IndexMerge hint, and it has specified index names. -func (ds *DataSource) indexMergeHintsHasSpecifiedIdx() bool { +func indexMergeHintsHasSpecifiedIdx(ds *DataSource) bool { for _, hint := range ds.IndexMergeHints { if hint.IndexHint == nil || len(hint.IndexHint.IndexNames) == 0 { continue @@ -323,7 +324,7 @@ func (ds *DataSource) indexMergeHintsHasSpecifiedIdx() bool { } // indexMergeHintsHasSpecifiedIdx return true if the input index name is specified in the IndexMerge hint. -func (ds *DataSource) isSpecifiedInIndexMergeHints(name string) bool { +func isSpecifiedInIndexMergeHints(ds *DataSource, name string) bool { for _, hint := range ds.IndexMergeHints { if hint.IndexHint == nil || len(hint.IndexHint.IndexNames) == 0 { continue @@ -338,7 +339,8 @@ func (ds *DataSource) isSpecifiedInIndexMergeHints(name string) bool { } // accessPathsForConds generates all possible index paths for conditions. -func (ds *DataSource) accessPathsForConds( +func accessPathsForConds( + ds *DataSource, conditions []expression.Expression, candidatePaths []*util.AccessPath, ) []*util.AccessPath { @@ -346,7 +348,7 @@ func (ds *DataSource) accessPathsForConds( for _, path := range candidatePaths { newPath := &util.AccessPath{} if path.IsTablePath() { - if !ds.isInIndexMergeHints("primary") { + if !isInIndexMergeHints(ds, "primary") { continue } if ds.TableInfo.IsCommonHandle { @@ -382,7 +384,7 @@ func (ds *DataSource) accessPathsForConds( } } else { newPath.Index = path.Index - if !ds.isInIndexMergeHints(newPath.Index.Name.L) { + if !isInIndexMergeHints(ds, newPath.Index.Name.L) { continue } err := ds.fillIndexPath(newPath, conditions) @@ -450,17 +452,17 @@ func buildIndexMergeOrPath( return indexMergePath } -func (ds *DataSource) generateNormalIndexPartialPath4And(normalPathCnt int, usedAccessMap map[string]expression.Expression) []*util.AccessPath { - if res := ds.generateIndexMergeAndPaths(normalPathCnt, usedAccessMap); res != nil { +func generateNormalIndexPartialPath4And(ds *DataSource, normalPathCnt int, usedAccessMap map[string]expression.Expression) []*util.AccessPath { + if res := generateIndexMergeAndPaths(ds, normalPathCnt, usedAccessMap); res != nil { return res.PartialIndexPaths } return nil } // generateIndexMergeAndPaths generates IndexMerge paths for `AND` (a.k.a. intersection type IndexMerge) -func (ds *DataSource) generateIndexMergeAndPaths(normalPathCnt int, usedAccessMap map[string]expression.Expression) *util.AccessPath { +func generateIndexMergeAndPaths(ds *DataSource, normalPathCnt int, usedAccessMap map[string]expression.Expression) *util.AccessPath { // For now, we only consider intersection type IndexMerge when the index names are specified in the hints. - if !ds.indexMergeHintsHasSpecifiedIdx() { + if !indexMergeHintsHasSpecifiedIdx(ds) { return nil } composedWithMvIndex := len(usedAccessMap) != 0 @@ -477,7 +479,7 @@ func (ds *DataSource) generateIndexMergeAndPaths(normalPathCnt int, usedAccessMa if ds.PossibleAccessPaths[i].Index.MVIndex { continue } - if !ds.isSpecifiedInIndexMergeHints(originalPath.Index.Name.L) { + if !isSpecifiedInIndexMergeHints(ds, originalPath.Index.Name.L) { continue } // If the path contains a full range, ignore it. @@ -595,14 +597,14 @@ func (ds *DataSource) generateIndexMergeAndPaths(normalPathCnt int, usedAccessMa } // generateMVIndexMergePartialPaths4And try to find mv index merge partial path from a collection of cnf conditions. -func (ds *DataSource) generateMVIndexMergePartialPaths4And(normalPathCnt int, indexMergeConds []expression.Expression, histColl *statistics.HistColl) ([]*util.AccessPath, map[string]expression.Expression, error) { +func generateMVIndexMergePartialPaths4And(ds *DataSource, normalPathCnt int, indexMergeConds []expression.Expression, histColl *statistics.HistColl) ([]*util.AccessPath, map[string]expression.Expression, error) { // step1: collect all mv index paths possibleMVIndexPaths := make([]*util.AccessPath, 0, len(ds.PossibleAccessPaths)) for idx := 0; idx < normalPathCnt; idx++ { if !isMVIndexPath(ds.PossibleAccessPaths[idx]) { continue // not a MVIndex path } - if !ds.isInIndexMergeHints(ds.PossibleAccessPaths[idx].Index.Name.L) { + if !isInIndexMergeHints(ds, ds.PossibleAccessPaths[idx].Index.Name.L) { continue } possibleMVIndexPaths = append(possibleMVIndexPaths, ds.PossibleAccessPaths[idx]) @@ -699,7 +701,7 @@ func (ds *DataSource) generateMVIndexMergePartialPaths4And(normalPathCnt int, in return mvAndPartialPath, usedAccessCondsMap, nil } -func (ds *DataSource) generateIndexMerge4NormalIndex(regularPathCount int, indexMergeConds []expression.Expression) (string, error) { +func generateIndexMerge4NormalIndex(ds *DataSource, regularPathCount int, indexMergeConds []expression.Expression) (string, error) { isPossibleIdxMerge := len(indexMergeConds) > 0 && // have corresponding access conditions, and len(ds.PossibleAccessPaths) > 1 // have multiple index paths if !isPossibleIdxMerge { @@ -744,12 +746,12 @@ func (ds *DataSource) generateIndexMerge4NormalIndex(regularPathCount int, index } // 1. Generate possible IndexMerge paths for `OR`. - err := ds.generateIndexMergeOrPaths(indexMergeConds) + err := generateIndexMergeOrPaths(ds, indexMergeConds) if err != nil { return "", err } // 2. Generate possible IndexMerge paths for `AND`. - indexMergeAndPath := ds.generateIndexMergeAndPaths(regularPathCount, nil) + indexMergeAndPath := generateIndexMergeAndPaths(ds, regularPathCount, nil) if indexMergeAndPath != nil { ds.PossibleAccessPaths = append(ds.PossibleAccessPaths, indexMergeAndPath) } @@ -768,14 +770,14 @@ func (ds *DataSource) generateIndexMerge4NormalIndex(regularPathCount int, index 1). all filters in the DNF have to be used as access-filters: ((1 member of (a)) or (2 member of (a)) or b > 10) cannot be used to access the MVIndex. 2). cannot support json_contains: (json_contains(a, '[1, 2]') or json_contains(a, '[3, 4]')) is not supported since a single IndexMerge cannot represent this SQL. */ -func (ds *DataSource) generateIndexMergeOnDNF4MVIndex(normalPathCnt int, filters []expression.Expression) (mvIndexPaths []*util.AccessPath, err error) { +func generateIndexMergeOnDNF4MVIndex(ds *DataSource, normalPathCnt int, filters []expression.Expression) (mvIndexPaths []*util.AccessPath, err error) { for idx := 0; idx < normalPathCnt; idx++ { if !isMVIndexPath(ds.PossibleAccessPaths[idx]) { continue // not a MVIndex path } // for single MV index usage, if specified use the specified one, if not, all can be access and chosen by cost model. - if !ds.isInIndexMergeHints(ds.PossibleAccessPaths[idx].Index.Name.L) { + if !isInIndexMergeHints(ds, ds.PossibleAccessPaths[idx].Index.Name.L) { continue } @@ -871,7 +873,7 @@ DNF path IndexRangeScan(non-mv-index-if-any)(?) --- COP TableRowIdScan(t) --- COP */ -func (ds *DataSource) generateIndexMerge4ComposedIndex(normalPathCnt int, indexMergeConds []expression.Expression) error { +func generateIndexMerge4ComposedIndex(ds *DataSource, normalPathCnt int, indexMergeConds []expression.Expression) error { isPossibleIdxMerge := len(indexMergeConds) > 0 && // have corresponding access conditions, and len(ds.PossibleAccessPaths) > 1 // have multiple index paths if !isPossibleIdxMerge { @@ -883,9 +885,9 @@ func (ds *DataSource) generateIndexMerge4ComposedIndex(normalPathCnt int, indexM candidateAccessPaths := make([]*util.AccessPath, 0, len(ds.PossibleAccessPaths)) for idx := 0; idx < normalPathCnt; idx++ { if (ds.PossibleAccessPaths[idx].IsTablePath() && - !ds.isInIndexMergeHints("primary")) || + !isInIndexMergeHints(ds, "primary")) || (!ds.PossibleAccessPaths[idx].IsTablePath() && - !ds.isInIndexMergeHints(ds.PossibleAccessPaths[idx].Index.Name.L)) { + !isInIndexMergeHints(ds, ds.PossibleAccessPaths[idx].Index.Name.L)) { continue } if isMVIndexPath(ds.PossibleAccessPaths[idx]) { @@ -955,14 +957,14 @@ func (ds *DataSource) generateIndexMerge4ComposedIndex(normalPathCnt int, indexM // step1: firstly collect all the potential normal index partial paths. // step2: secondly collect all the potential mv index partial path, and merge them into one if possible. // step3: thirdly merge normal index paths and mv index paths together to compose a bigger index merge path. - mvIndexPartialPaths, usedAccessMap, err := ds.generateMVIndexMergePartialPaths4And(normalPathCnt, indexMergeConds, ds.TableStats.HistColl) + mvIndexPartialPaths, usedAccessMap, err := generateMVIndexMergePartialPaths4And(ds, normalPathCnt, indexMergeConds, ds.TableStats.HistColl) if err != nil { return err } if len(mvIndexPartialPaths) == 0 { return nil } - normalIndexPartialPaths := ds.generateNormalIndexPartialPath4And(normalPathCnt, usedAccessMap) + normalIndexPartialPaths := generateNormalIndexPartialPath4And(ds, normalPathCnt, usedAccessMap) // since multi normal index merge path is handled before, here focus on multi mv index merge, or mv and normal mixed index merge composed := (len(mvIndexPartialPaths) > 1) || (len(mvIndexPartialPaths) == 1 && len(normalIndexPartialPaths) >= 1) if !composed { @@ -1028,8 +1030,8 @@ func (ds *DataSource) generateIndexMerge4ComposedIndex(normalPathCnt int, indexM IndexRangeScan(a, [3,3]) TableRowIdScan(t) */ -func (ds *DataSource) generateIndexMerge4MVIndex(normalPathCnt int, filters []expression.Expression) error { - dnfMVIndexPaths, err := ds.generateIndexMergeOnDNF4MVIndex(normalPathCnt, filters) +func generateIndexMerge4MVIndex(ds *DataSource, normalPathCnt int, filters []expression.Expression) error { + dnfMVIndexPaths, err := generateIndexMergeOnDNF4MVIndex(ds, normalPathCnt, filters) if err != nil { return err } @@ -1041,7 +1043,7 @@ func (ds *DataSource) generateIndexMerge4MVIndex(normalPathCnt int, filters []ex } // for single MV index usage, if specified use the specified one, if not, all can be access and chosen by cost model. - if !ds.isInIndexMergeHints(ds.PossibleAccessPaths[idx].Index.Name.L) { + if !isInIndexMergeHints(ds, ds.PossibleAccessPaths[idx].Index.Name.L) { continue } @@ -1442,7 +1444,7 @@ func CollectFilters4MVIndexMutations(sctx base.PlanContext, filters []expression // cleanAccessPathForMVIndexHint removes all other access path if there is a multi-valued index hint, and this hint // has a valid path -func (ds *DataSource) cleanAccessPathForMVIndexHint() { +func cleanAccessPathForMVIndexHint(ds *DataSource) { forcedMultiValuedIndex := make(map[int64]struct{}, len(ds.PossibleAccessPaths)) for _, p := range ds.PossibleAccessPaths { if !isMVIndexPath(p) || !p.Forced { diff --git a/pkg/planner/core/indexmerge_unfinished_path.go b/pkg/planner/core/indexmerge_unfinished_path.go index 5fa3e967038f8..c22ea11e4d3b7 100644 --- a/pkg/planner/core/indexmerge_unfinished_path.go +++ b/pkg/planner/core/indexmerge_unfinished_path.go @@ -142,7 +142,8 @@ func initUnfinishedPathsFromExpr( // generateNormalIndexPartialPaths4DNF is introduced for handle a slice of DNF items and a slice of // candidate AccessPaths before, now we reuse it to handle single filter and single candidate AccessPath, // so we need to wrap them in a slice here. - paths, needSelection, usedMap := ds.generateNormalIndexPartialPaths4DNF( + paths, needSelection, usedMap := generateNormalIndexPartialPaths4DNF( + ds, []expression.Expression{expr}, []*util.AccessPath{path}, ) @@ -370,7 +371,8 @@ func buildIntoAccessPath( // case 2: non-mv index var usedMap []bool // Reuse the previous implementation. The same usage as in initUnfinishedPathsFromExpr(). - paths, needSelection, usedMap = ds.generateNormalIndexPartialPaths4DNF( + paths, needSelection, usedMap = generateNormalIndexPartialPaths4DNF( + ds, []expression.Expression{ expression.ComposeCNFCondition( ds.SCtx().GetExprCtx(), diff --git a/pkg/planner/core/logical_datasource.go b/pkg/planner/core/logical_datasource.go index e831210d5a2f2..5433f098cee48 100644 --- a/pkg/planner/core/logical_datasource.go +++ b/pkg/planner/core/logical_datasource.go @@ -334,7 +334,7 @@ func (ds *DataSource) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema, return nil, err } - if err := ds.generateIndexMergePath(); err != nil { + if err := generateIndexMergePath(ds); err != nil { return nil, err } diff --git a/pkg/server/testdata/optimizer_suite_out.json b/pkg/server/testdata/optimizer_suite_out.json index d538e402e73ae..d0bcde3006534 100644 --- a/pkg/server/testdata/optimizer_suite_out.json +++ b/pkg/server/testdata/optimizer_suite_out.json @@ -197,7 +197,7 @@ ] }, { - "github.com/pingcap/tidb/pkg/planner/core.(*DataSource).generateIndexMergePath": null + "github.com/pingcap/tidb/pkg/planner/core.generateIndexMergePath": null }, { "Access paths": [ @@ -403,7 +403,7 @@ } }, { - "github.com/pingcap/tidb/pkg/planner/core.(*DataSource).generateIndexMergePath": null + "github.com/pingcap/tidb/pkg/planner/core.generateIndexMergePath": null }, { "Access paths": [ @@ -625,7 +625,7 @@ ] }, { - "github.com/pingcap/tidb/pkg/planner/core.(*DataSource).generateIndexMergePath": null + "github.com/pingcap/tidb/pkg/planner/core.generateIndexMergePath": null }, { "Access paths": [