From 15081b3b0361f9c06d419f90e594a782382a18aa Mon Sep 17 00:00:00 2001 From: time-and-fate <25057648+time-and-fate@users.noreply.github.com> Date: Fri, 29 Dec 2023 20:51:19 +0800 Subject: [PATCH] add --- pkg/executor/test/analyzetest/analyze_test.go | 25 ++-- pkg/planner/core/indexmerge_path.go | 121 ++++++++++++------ pkg/planner/core/logical_plans.go | 15 ++- pkg/planner/core/planbuilder.go | 10 +- .../r/planner/core/indexmerge_path.result | 4 - 5 files changed, 111 insertions(+), 64 deletions(-) diff --git a/pkg/executor/test/analyzetest/analyze_test.go b/pkg/executor/test/analyzetest/analyze_test.go index 6afcc5f260b03..b7e4694f2cedf 100644 --- a/pkg/executor/test/analyzetest/analyze_test.go +++ b/pkg/executor/test/analyzetest/analyze_test.go @@ -2774,10 +2774,12 @@ func TestAnalyzeColumnsSkipMVIndexJsonCol(t *testing.T) { tk.MustExec("analyze table t columns a") tk.MustQuery("show warnings").Sort().Check(testkit.Rows(""+ "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", - "Warning 1105 Columns b are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", - "Warning 1105 analyzing multi-valued indexes is not supported, skip idx_c")) - tk.MustQuery("select job_info from mysql.analyze_jobs where table_schema = 'test' and table_name = 't'").Check(testkit.Rows( - "analyze table columns a, b with 256 buckets, 500 topn, 1 samplerate")) + "Warning 1105 Columns b are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats")) + tk.MustQuery("select job_info from mysql.analyze_jobs where table_schema = 'test' and table_name = 't'").Sort().Check( + testkit.Rows( + "analyze index idx_c", + "analyze table columns a, b with 256 buckets, 500 topn, 1 samplerate", + )) is := dom.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) @@ -2788,13 +2790,12 @@ func TestAnalyzeColumnsSkipMVIndexJsonCol(t *testing.T) { require.True(t, stats.Columns[tblInfo.Columns[1].ID].IsStatsInitialized()) require.False(t, stats.Columns[tblInfo.Columns[2].ID].IsStatsInitialized()) require.True(t, stats.Indices[tblInfo.Indices[0].ID].IsStatsInitialized()) - require.False(t, stats.Indices[tblInfo.Indices[1].ID].IsStatsInitialized()) + require.True(t, stats.Indices[tblInfo.Indices[1].ID].IsStatsInitialized()) } // TestAnalyzeMVIndex tests analyzing the mv index use some real data in the table. // It checks the analyze jobs, async loading and the stats content in the memory. func TestAnalyzeMVIndex(t *testing.T) { - t.Skip() require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/DebugAnalyzeJobOperations", "return(true)")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/statistics/handle/DebugAnalyzeJobOperations", "return(true)")) defer func() { @@ -2944,12 +2945,12 @@ func TestAnalyzeMVIndex(t *testing.T) { )) tk.MustQuery("explain format = brief select * from t where '1' member of (j->'$.bin')").Check(testkit.Rows( "IndexMerge 0.03 root type: union", - "├─IndexRangeScan(Build) 0.03 cop[tikv] table:t, index:ij_binary(cast(json_extract(`j`, _utf8mb4'$.bin') as binary(50) array)) range:[0x31,0x31], keep order:false, stats:partial[ia:allEvicted, ij_binary:allEvicted, j:unInitialized]", + "├─IndexRangeScan(Build) 0.03 cop[tikv] table:t, index:ij_binary(cast(json_extract(`j`, _utf8mb4'$.bin') as binary(50) array)) range:[\"1\",\"1\"], keep order:false, stats:partial[ia:allEvicted, ij_binary:allEvicted, j:unInitialized]", "└─TableRowIDScan(Probe) 0.03 cop[tikv] table:t keep order:false, stats:partial[ia:allEvicted, ij_binary:allEvicted, j:unInitialized]", )) tk.MustQuery("explain format = brief select * from t where '1' member of (j->'$.char')").Check(testkit.Rows( "IndexMerge 0.03 root type: union", - "├─IndexRangeScan(Build) 0.03 cop[tikv] table:t, index:ij_char(cast(json_extract(`j`, _utf8mb4'$.char') as char(50) array)) range:[0x31,0x31], keep order:false, stats:partial[ia:allEvicted, ij_char:allEvicted, j:unInitialized]", + "├─IndexRangeScan(Build) 0.03 cop[tikv] table:t, index:ij_char(cast(json_extract(`j`, _utf8mb4'$.char') as char(50) array)) range:[\"1\",\"1\"], keep order:false, stats:partial[ia:allEvicted, ij_char:allEvicted, j:unInitialized]", "└─TableRowIDScan(Probe) 0.03 cop[tikv] table:t keep order:false, stats:partial[ia:allEvicted, ij_char:allEvicted, j:unInitialized]", )) // 3.2. emulate the background async loading @@ -2972,12 +2973,12 @@ func TestAnalyzeMVIndex(t *testing.T) { )) tk.MustQuery("explain format = brief select * from t where '1' member of (j->'$.bin')").Check(testkit.Rows( "IndexMerge 0.03 root type: union", - "├─IndexRangeScan(Build) 0.03 cop[tikv] table:t, index:ij_binary(cast(json_extract(`j`, _utf8mb4'$.bin') as binary(50) array)) range:[0x31,0x31], keep order:false, stats:partial[j:unInitialized]", + "├─IndexRangeScan(Build) 0.03 cop[tikv] table:t, index:ij_binary(cast(json_extract(`j`, _utf8mb4'$.bin') as binary(50) array)) range:[\"1\",\"1\"], keep order:false, stats:partial[j:unInitialized]", "└─TableRowIDScan(Probe) 0.03 cop[tikv] table:t keep order:false, stats:partial[j:unInitialized]", )) tk.MustQuery("explain format = brief select * from t where '1' member of (j->'$.char')").Check(testkit.Rows( "IndexMerge 0.03 root type: union", - "├─IndexRangeScan(Build) 0.03 cop[tikv] table:t, index:ij_char(cast(json_extract(`j`, _utf8mb4'$.char') as char(50) array)) range:[0x31,0x31], keep order:false, stats:partial[j:unInitialized]", + "├─IndexRangeScan(Build) 0.03 cop[tikv] table:t, index:ij_char(cast(json_extract(`j`, _utf8mb4'$.char') as char(50) array)) range:[\"1\",\"1\"], keep order:false, stats:partial[j:unInitialized]", "└─TableRowIDScan(Probe) 0.03 cop[tikv] table:t keep order:false, stats:partial[j:unInitialized]", )) @@ -2998,12 +2999,12 @@ func TestAnalyzeMVIndex(t *testing.T) { )) tk.MustQuery("explain format = brief select * from t where '1' member of (j->'$.bin')").Check(testkit.Rows( "IndexMerge 0.03 root type: union", - "├─IndexRangeScan(Build) 0.03 cop[tikv] table:t, index:ij_binary(cast(json_extract(`j`, _utf8mb4'$.bin') as binary(50) array)) range:[0x31,0x31], keep order:false, stats:partial[ia:allEvicted, j:unInitialized]", + "├─IndexRangeScan(Build) 0.03 cop[tikv] table:t, index:ij_binary(cast(json_extract(`j`, _utf8mb4'$.bin') as binary(50) array)) range:[\"1\",\"1\"], keep order:false, stats:partial[ia:allEvicted, j:unInitialized]", "└─TableRowIDScan(Probe) 0.03 cop[tikv] table:t keep order:false, stats:partial[ia:allEvicted, j:unInitialized]", )) tk.MustQuery("explain format = brief select * from t where '1' member of (j->'$.char')").Check(testkit.Rows( "IndexMerge 0.03 root type: union", - "├─IndexRangeScan(Build) 0.03 cop[tikv] table:t, index:ij_char(cast(json_extract(`j`, _utf8mb4'$.char') as char(50) array)) range:[0x31,0x31], keep order:false, stats:partial[ia:allEvicted, j:unInitialized]", + "├─IndexRangeScan(Build) 0.03 cop[tikv] table:t, index:ij_char(cast(json_extract(`j`, _utf8mb4'$.char') as char(50) array)) range:[\"1\",\"1\"], keep order:false, stats:partial[ia:allEvicted, j:unInitialized]", "└─TableRowIDScan(Probe) 0.03 cop[tikv] table:t keep order:false, stats:partial[ia:allEvicted, j:unInitialized]", )) diff --git a/pkg/planner/core/indexmerge_path.go b/pkg/planner/core/indexmerge_path.go index 672c5ca73a9e8..f7c306ecfc2b1 100644 --- a/pkg/planner/core/indexmerge_path.go +++ b/pkg/planner/core/indexmerge_path.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/debugtrace" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/logutil" @@ -680,17 +681,17 @@ func (ds *DataSource) generateMVIndexPartialPath4Or(normalPathCnt int, indexMerg bestNeedSelection bool ) for _, onePossibleMVIndexPath := range possibleMVIndexPaths { - idxCols, ok := ds.prepareCols4MVIndex(onePossibleMVIndexPath.Index) + idxCols, ok := prepareCols4MVIndex(ds.table.Meta(), onePossibleMVIndexPath.Index, ds.TblCols) if !ok { continue } // for every cnfCond, try to map it into possible mv index path. // remainingFilters is not cared here, because it will be all suspended on the table side. - accessFilters, remainingFilters := ds.collectFilters4MVIndex(cnfConds, idxCols) + accessFilters, remainingFilters := collectFilters4MVIndex(ds.SCtx(), cnfConds, idxCols) if len(accessFilters) == 0 { continue } - paths, isIntersection, ok, err := ds.buildPartialPaths4MVIndex(accessFilters, idxCols, onePossibleMVIndexPath.Index) + paths, isIntersection, ok, err := buildPartialPaths4MVIndex(ds.SCtx(), accessFilters, idxCols, onePossibleMVIndexPath.Index, ds.tableStats.HistColl) if err != nil { logutil.BgLogger().Debug("build index merge partial mv index paths failed", zap.Error(err)) return nil, nil, false, err @@ -748,7 +749,7 @@ func (ds *DataSource) generateMVIndexMergePartialPaths4And(normalPathCnt int, in mvAndPartialPath := make([]*util.AccessPath, 0, len(possibleMVIndexPaths)) usedAccessCondsMap := make(map[string]expression.Expression, len(indexMergeConds)) for idx := 0; idx < len(possibleMVIndexPaths); idx++ { - idxCols, ok := ds.prepareCols4MVIndex(possibleMVIndexPaths[idx].Index) + idxCols, ok := prepareCols4MVIndex(ds.table.Meta(), possibleMVIndexPaths[idx].Index, ds.TblCols) if !ok { continue } @@ -762,7 +763,7 @@ func (ds *DataSource) generateMVIndexMergePartialPaths4And(normalPathCnt int, in // derive each mutation access filters accessFilters[mvColOffset] = mvFilterMu - partialPaths, isIntersection, ok, err := ds.buildPartialPaths4MVIndex(accessFilters, idxCols, possibleMVIndexPaths[idx].Index) + partialPaths, isIntersection, ok, err := buildPartialPaths4MVIndex(ds.SCtx(), accessFilters, idxCols, possibleMVIndexPaths[idx].Index, ds.tableStats.HistColl) if err != nil { logutil.BgLogger().Debug("build index merge partial mv index paths failed", zap.Error(err)) return nil, nil, err @@ -855,7 +856,7 @@ func (ds *DataSource) generateIndexMergeOnDNF4MVIndex(normalPathCnt int, filters continue // not a MVIndex path } - idxCols, ok := ds.prepareCols4MVIndex(ds.possibleAccessPaths[idx].Index) + idxCols, ok := prepareCols4MVIndex(ds.table.Meta(), ds.possibleAccessPaths[idx].Index, ds.TblCols) if !ok { continue } @@ -876,12 +877,12 @@ func (ds *DataSource) generateIndexMergeOnDNF4MVIndex(normalPathCnt int, filters mvIndexFilters = expression.FlattenCNFConditions(sf) // (1 member of (a) and b=1) --> [(1 member of (a)), b=1] } - accessFilters, remainingFilters := ds.collectFilters4MVIndex(mvIndexFilters, idxCols) + accessFilters, remainingFilters := collectFilters4MVIndex(ds.SCtx(), mvIndexFilters, idxCols) if len(accessFilters) == 0 || len(remainingFilters) > 0 { // limitation 1 cannotFit = true break } - paths, isIntersection, ok, err := ds.buildPartialPaths4MVIndex(accessFilters, idxCols, ds.possibleAccessPaths[idx].Index) + paths, isIntersection, ok, err := buildPartialPaths4MVIndex(ds.SCtx(), accessFilters, idxCols, ds.possibleAccessPaths[idx].Index, ds.tableStats.HistColl) if err != nil { return nil, err } @@ -899,7 +900,12 @@ func (ds *DataSource) generateIndexMergeOnDNF4MVIndex(normalPathCnt int, filters remainingFilters = append(remainingFilters, filters[:current]...) remainingFilters = append(remainingFilters, filters[current+1:]...) - indexMergePath := ds.buildPartialPathUp4MVIndex(partialPaths, false, remainingFilters) + indexMergePath := ds.buildPartialPathUp4MVIndex( + partialPaths, + false, + remainingFilters, + ds.tableStats.HistColl, + ) mvIndexPaths = append(mvIndexPaths, indexMergePath) } } @@ -1022,7 +1028,12 @@ func (ds *DataSource) generateIndexMerge4ComposedIndex(normalPathCnt int, indexM if needSelection4MVIndex || needSelection4NormalIndex { indexMergeTableFilters = indexMergeConds } - mvp := ds.buildPartialPathUp4MVIndex(combinedPartialPaths, false, indexMergeTableFilters) + mvp := ds.buildPartialPathUp4MVIndex( + combinedPartialPaths, + false, + indexMergeTableFilters, + ds.tableStats.HistColl, + ) ds.possibleAccessPaths = append(ds.possibleAccessPaths, mvp) return nil } @@ -1069,7 +1080,7 @@ func (ds *DataSource) generateIndexMerge4ComposedIndex(normalPathCnt int, indexM remainedCNFs = append(remainedCNFs, CNFItem) } } - mvp := ds.buildPartialPathUp4MVIndex(combinedPartialPaths, true, remainedCNFs) + mvp := ds.buildPartialPathUp4MVIndex(combinedPartialPaths, true, remainedCNFs, ds.tableStats.HistColl) ds.possibleAccessPaths = append(ds.possibleAccessPaths, mvp) return nil @@ -1106,17 +1117,17 @@ func (ds *DataSource) generateIndexMerge4MVIndex(normalPathCnt int, filters []ex continue // not a MVIndex path } - idxCols, ok := ds.prepareCols4MVIndex(ds.possibleAccessPaths[idx].Index) + idxCols, ok := prepareCols4MVIndex(ds.table.Meta(), ds.possibleAccessPaths[idx].Index, ds.TblCols) if !ok { continue } - accessFilters, remainingFilters := ds.collectFilters4MVIndex(filters, idxCols) + accessFilters, remainingFilters := collectFilters4MVIndex(ds.SCtx(), filters, idxCols) if len(accessFilters) == 0 { // cannot use any filter on this MVIndex continue } - partialPaths, isIntersection, ok, err := ds.buildPartialPaths4MVIndex(accessFilters, idxCols, ds.possibleAccessPaths[idx].Index) + partialPaths, isIntersection, ok, err := buildPartialPaths4MVIndex(ds.SCtx(), accessFilters, idxCols, ds.possibleAccessPaths[idx].Index, ds.tableStats.HistColl) if err != nil { return err } @@ -1124,13 +1135,24 @@ func (ds *DataSource) generateIndexMerge4MVIndex(normalPathCnt int, filters []ex continue } - ds.possibleAccessPaths = append(ds.possibleAccessPaths, ds.buildPartialPathUp4MVIndex(partialPaths, isIntersection, remainingFilters)) + ds.possibleAccessPaths = append(ds.possibleAccessPaths, ds.buildPartialPathUp4MVIndex( + partialPaths, + isIntersection, + remainingFilters, + ds.tableStats.HistColl, + ), + ) } return nil } // buildPartialPathUp4MVIndex builds these partial paths up to a complete index merge path. -func (*DataSource) buildPartialPathUp4MVIndex(partialPaths []*util.AccessPath, isIntersection bool, remainingFilters []expression.Expression) *util.AccessPath { +func (*DataSource) buildPartialPathUp4MVIndex( + partialPaths []*util.AccessPath, + isIntersection bool, + remainingFilters []expression.Expression, + histColl *statistics.HistColl, +) *util.AccessPath { indexMergePath := &util.AccessPath{PartialIndexPaths: partialPaths, IndexMergeAccessMVIndex: true} indexMergePath.IndexMergeIsIntersection = isIntersection indexMergePath.TableFilters = remainingFilters @@ -1152,9 +1174,18 @@ func (*DataSource) buildPartialPathUp4MVIndex(partialPaths []*util.AccessPath, i // buildPartialPaths4MVIndex builds partial paths by using these accessFilters upon this MVIndex. // The accessFilters must be corresponding to these idxCols. // OK indicates whether it builds successfully. These partial paths should be ignored if ok==false. -func (ds *DataSource) buildPartialPaths4MVIndex(accessFilters []expression.Expression, - idxCols []*expression.Column, mvIndex *model.IndexInfo) ( - partialPaths []*util.AccessPath, isIntersection bool, ok bool, err error) { +func buildPartialPaths4MVIndex( + sctx sessionctx.Context, + accessFilters []expression.Expression, + idxCols []*expression.Column, + mvIndex *model.IndexInfo, + histColl *statistics.HistColl, +) ( + partialPaths []*util.AccessPath, + isIntersection bool, + ok bool, + err error, +) { var virColID = -1 for i := range idxCols { // index column may contain other virtual column. @@ -1167,7 +1198,7 @@ func (ds *DataSource) buildPartialPaths4MVIndex(accessFilters []expression.Expre return nil, false, false, nil } if len(accessFilters) <= virColID { // no filter related to the vir-col, build a partial path directly. - partialPath, ok, err := ds.buildPartialPath4MVIndex(accessFilters, idxCols, mvIndex) + partialPath, ok, err := buildPartialPath4MVIndex(sctx, accessFilters, idxCols, mvIndex, histColl) return []*util.AccessPath{partialPath}, false, ok, err } @@ -1193,21 +1224,21 @@ func (ds *DataSource) buildPartialPaths4MVIndex(accessFilters []expression.Expre virColVals = append(virColVals, v) case ast.JSONContains: // (json_contains(a->'$.zip', '[1, 2, 3]') isIntersection = true - virColVals, ok = jsonArrayExpr2Exprs(ds.SCtx(), sf.GetArgs()[1], jsonType) + virColVals, ok = jsonArrayExpr2Exprs(sctx, sf.GetArgs()[1], jsonType) if !ok || len(virColVals) == 0 { // json_contains(JSON, '[]') is TRUE return nil, false, false, nil } case ast.JSONOverlaps: // (json_overlaps(a->'$.zip', '[1, 2, 3]') var jsonPathIdx int - if sf.GetArgs()[0].Equal(ds.SCtx(), targetJSONPath) { + if sf.GetArgs()[0].Equal(sctx, targetJSONPath) { jsonPathIdx = 0 // (json_overlaps(a->'$.zip', '[1, 2, 3]') - } else if sf.GetArgs()[1].Equal(ds.SCtx(), targetJSONPath) { + } else if sf.GetArgs()[1].Equal(sctx, targetJSONPath) { jsonPathIdx = 1 // (json_overlaps('[1, 2, 3]', a->'$.zip') } else { return nil, false, false, nil } var ok bool - virColVals, ok = jsonArrayExpr2Exprs(ds.SCtx(), sf.GetArgs()[1-jsonPathIdx], jsonType) + virColVals, ok = jsonArrayExpr2Exprs(sctx, sf.GetArgs()[1-jsonPathIdx], jsonType) if !ok || len(virColVals) == 0 { // forbid empty array for safety return nil, false, false, nil } @@ -1223,7 +1254,7 @@ func (ds *DataSource) buildPartialPaths4MVIndex(accessFilters []expression.Expre for _, v := range virColVals { // rewrite json functions to EQ to calculate range, `(1 member of j)` -> `j=1`. - eq, err := expression.NewFunction(ds.SCtx(), ast.EQ, types.NewFieldType(mysql.TypeTiny), virCol, v) + eq, err := expression.NewFunction(sctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), virCol, v) if err != nil { return nil, false, false, err } @@ -1231,7 +1262,7 @@ func (ds *DataSource) buildPartialPaths4MVIndex(accessFilters []expression.Expre copy(newAccessFilters, accessFilters) newAccessFilters[virColID] = eq - partialPath, ok, err := ds.buildPartialPath4MVIndex(newAccessFilters, idxCols, mvIndex) + partialPath, ok, err := buildPartialPath4MVIndex(sctx, newAccessFilters, idxCols, mvIndex, histColl) if !ok || err != nil { return nil, false, ok, err } @@ -1249,7 +1280,13 @@ func isSafeTypeConversion4MVIndexRange(valType, mvIndexType *types.FieldType) (s } // buildPartialPath4MVIndex builds a partial path on this MVIndex with these accessFilters. -func (ds *DataSource) buildPartialPath4MVIndex(accessFilters []expression.Expression, idxCols []*expression.Column, mvIndex *model.IndexInfo) (*util.AccessPath, bool, error) { +func buildPartialPath4MVIndex( + sctx sessionctx.Context, + accessFilters []expression.Expression, + idxCols []*expression.Column, + mvIndex *model.IndexInfo, + histColl *statistics.HistColl, +) (*util.AccessPath, bool, error) { partialPath := &util.AccessPath{Index: mvIndex} partialPath.Ranges = ranger.FullRange() for i := 0; i < len(idxCols); i++ { @@ -1258,7 +1295,7 @@ func (ds *DataSource) buildPartialPath4MVIndex(accessFilters []expression.Expres partialPath.FullIdxCols = append(partialPath.FullIdxCols, idxCols[i]) partialPath.FullIdxColLens = append(partialPath.FullIdxColLens, mvIndex.Columns[i].Length) } - if err := ds.detachCondAndBuildRangeForPath(partialPath, accessFilters); err != nil { + if err := detachCondAndBuildRangeForPath(sctx, partialPath, accessFilters, histColl); err != nil { return nil, false, err } if len(partialPath.AccessConds) != len(accessFilters) || len(partialPath.TableFilters) > 0 { @@ -1268,13 +1305,17 @@ func (ds *DataSource) buildPartialPath4MVIndex(accessFilters []expression.Expres return partialPath, true, nil } -func (ds *DataSource) prepareCols4MVIndex(mvIndex *model.IndexInfo) (idxCols []*expression.Column, ok bool) { +func prepareCols4MVIndex( + tableInfo *model.TableInfo, + mvIndex *model.IndexInfo, + tblCols []*expression.Column, +) (idxCols []*expression.Column, ok bool) { var virColNum = 0 for i := range mvIndex.Columns { colOffset := mvIndex.Columns[i].Offset - colMeta := ds.table.Meta().Cols()[colOffset] + colMeta := tableInfo.Cols()[colOffset] var col *expression.Column - for _, c := range ds.TblCols { + for _, c := range tblCols { if c.ID == colMeta.ID { col = c break @@ -1301,7 +1342,7 @@ func (ds *DataSource) prepareCols4MVIndex(mvIndex *model.IndexInfo) (idxCols []* // collectFilters4MVIndex splits these filters into 2 parts where accessFilters can be used to access this index directly. // For idx(x, cast(a as array), z), `x=1 and (2 member of a) and z=1 and x+z>0` is split to: // accessFilters: `x=1 and (2 member of a) and z=1`, remaining: `x+z>0`. -func (ds *DataSource) collectFilters4MVIndex(filters []expression.Expression, idxCols []*expression.Column) (accessFilters, remainingFilters []expression.Expression) { +func collectFilters4MVIndex(sctx sessionctx.Context, filters []expression.Expression, idxCols []*expression.Column) (accessFilters, remainingFilters []expression.Expression) { usedAsAccess := make([]bool, len(filters)) for _, col := range idxCols { found := false @@ -1309,7 +1350,7 @@ func (ds *DataSource) collectFilters4MVIndex(filters []expression.Expression, id if usedAsAccess[i] { continue } - if ds.checkFilter4MVIndexColumn(f, col) { + if checkFilter4MVIndexColumn(sctx, f, col) { accessFilters = append(accessFilters, f) usedAsAccess[i] = true found = true @@ -1378,7 +1419,7 @@ func (ds *DataSource) collectFilters4MVIndexMutations(filters []expression.Expre if usedAsAccess[i] { continue } - if ds.checkFilter4MVIndexColumn(f, col) { + if checkFilter4MVIndexColumn(ds.SCtx(), f, col) { if col.VirtualExpr != nil && col.VirtualExpr.GetType().IsArray() { // assert jsonColOffset should always be the same. // if the filter is from virtual expression, it means it is about the mv json col. @@ -1404,7 +1445,7 @@ func (ds *DataSource) collectFilters4MVIndexMutations(filters []expression.Expre } // checkFilter4MVIndexColumn checks whether this filter can be used as an accessFilter to access the MVIndex column. -func (ds *DataSource) checkFilter4MVIndexColumn(filter expression.Expression, idxCol *expression.Column) bool { +func checkFilter4MVIndexColumn(sctx sessionctx.Context, filter expression.Expression, idxCol *expression.Column) bool { sf, ok := filter.(*expression.ScalarFunction) if !ok { return false @@ -1416,12 +1457,12 @@ func (ds *DataSource) checkFilter4MVIndexColumn(filter expression.Expression, id } switch sf.FuncName.L { case ast.JSONMemberOf: // (1 member of a) - return targetJSONPath.Equal(ds.SCtx(), sf.GetArgs()[1]) + return targetJSONPath.Equal(sctx, sf.GetArgs()[1]) case ast.JSONContains: // json_contains(a, '1') - return targetJSONPath.Equal(ds.SCtx(), sf.GetArgs()[0]) + return targetJSONPath.Equal(sctx, sf.GetArgs()[0]) case ast.JSONOverlaps: // json_overlaps(a, '1') or json_overlaps('1', a) - return targetJSONPath.Equal(ds.SCtx(), sf.GetArgs()[0]) || - targetJSONPath.Equal(ds.SCtx(), sf.GetArgs()[1]) + return targetJSONPath.Equal(sctx, sf.GetArgs()[0]) || + targetJSONPath.Equal(sctx, sf.GetArgs()[1]) default: return false } @@ -1444,7 +1485,7 @@ func (ds *DataSource) checkFilter4MVIndexColumn(filter expression.Expression, id if argCol == nil || argConst == nil { return false } - if argCol.Equal(ds.SCtx(), idxCol) { + if argCol.Equal(sctx, idxCol) { return true } } diff --git a/pkg/planner/core/logical_plans.go b/pkg/planner/core/logical_plans.go index 729a45b1df812..c00cc962bc61e 100644 --- a/pkg/planner/core/logical_plans.go +++ b/pkg/planner/core/logical_plans.go @@ -1608,12 +1608,17 @@ func (ds *DataSource) Convert2Gathers() (gathers []LogicalPlan) { return gathers } -func (ds *DataSource) detachCondAndBuildRangeForPath(path *util.AccessPath, conds []expression.Expression) error { +func detachCondAndBuildRangeForPath( + sctx sessionctx.Context, + path *util.AccessPath, + conds []expression.Expression, + histColl *statistics.HistColl, +) error { if len(path.IdxCols) == 0 { path.TableFilters = conds return nil } - res, err := ranger.DetachCondAndBuildRangeForIndex(ds.SCtx(), conds, path.IdxCols, path.IdxColLens, ds.SCtx().GetSessionVars().RangeMaxSize) + res, err := ranger.DetachCondAndBuildRangeForIndex(sctx, conds, path.IdxCols, path.IdxColLens, sctx.GetSessionVars().RangeMaxSize) if err != nil { return err } @@ -1629,7 +1634,7 @@ func (ds *DataSource) detachCondAndBuildRangeForPath(path *util.AccessPath, cond path.ConstCols[i] = res.ColumnValues[i] != nil } } - path.CountAfterAccess, err = cardinality.GetRowCountByIndexRanges(ds.SCtx(), ds.tableStats.HistColl, path.Index.ID, path.Ranges) + path.CountAfterAccess, err = cardinality.GetRowCountByIndexRanges(sctx, histColl, path.Index.ID, path.Ranges) return err } @@ -1641,7 +1646,7 @@ func (ds *DataSource) deriveCommonHandleTablePathStats(path *util.AccessPath, co if len(conds) == 0 { return nil } - if err := ds.detachCondAndBuildRangeForPath(path, conds); err != nil { + if err := detachCondAndBuildRangeForPath(ds.SCtx(), path, conds, ds.tableStats.HistColl); err != nil { return err } if path.EqOrInCondCount == len(path.AccessConds) { @@ -1786,7 +1791,7 @@ func (ds *DataSource) fillIndexPath(path *util.AccessPath, conds []expression.Ex } } } - err := ds.detachCondAndBuildRangeForPath(path, conds) + err := detachCondAndBuildRangeForPath(ds.SCtx(), path, conds, ds.tableStats.HistColl) return err } diff --git a/pkg/planner/core/planbuilder.go b/pkg/planner/core/planbuilder.go index 51fd8b9bb78da..869dc183bbd73 100644 --- a/pkg/planner/core/planbuilder.go +++ b/pkg/planner/core/planbuilder.go @@ -2150,7 +2150,11 @@ func getColOffsetForAnalyze(colsInfo []*model.ColumnInfo, colID int64) int { // TODO: find a better way to find indexed columns in ANALYZE rather than use IndexColumn.Offset // For multi-valued index, we need to collect it separately here and analyze it as independent index analyze task. // See comments for AnalyzeResults.ForMVIndex for more details. -func getModifiedIndexesInfoForAnalyze(sctx sessionctx.Context, tblInfo *model.TableInfo, allColumns bool, colsInfo []*model.ColumnInfo) ([]*model.IndexInfo, []*model.IndexInfo) { +func getModifiedIndexesInfoForAnalyze( + tblInfo *model.TableInfo, + allColumns bool, + colsInfo []*model.ColumnInfo, +) ([]*model.IndexInfo, []*model.IndexInfo) { idxsInfo := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) independentIdxsInfo := make([]*model.IndexInfo, 0) for _, originIdx := range tblInfo.Indices { @@ -2158,7 +2162,7 @@ func getModifiedIndexesInfoForAnalyze(sctx sessionctx.Context, tblInfo *model.Ta continue } if originIdx.MVIndex { - sctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("analyzing multi-valued indexes is not supported, skip %s", originIdx.Name.L)) + independentIdxsInfo = append(independentIdxsInfo, originIdx) continue } if allColumns { @@ -2277,7 +2281,7 @@ func (b *PlanBuilder) buildAnalyzeFullSamplingTask( } execColsInfo = b.filterSkipColumnTypes(execColsInfo, tbl, &mustAnalyzedCols) allColumns := len(tbl.TableInfo.Columns) == len(execColsInfo) - indexes, independentIndexes := getModifiedIndexesInfoForAnalyze(b.ctx, tbl.TableInfo, allColumns, execColsInfo) + indexes, independentIndexes := getModifiedIndexesInfoForAnalyze(tbl.TableInfo, allColumns, execColsInfo) handleCols := BuildHandleColsForAnalyze(b.ctx, tbl.TableInfo, allColumns, execColsInfo) newTask := AnalyzeColumnsTask{ HandleCols: handleCols, diff --git a/tests/integrationtest/r/planner/core/indexmerge_path.result b/tests/integrationtest/r/planner/core/indexmerge_path.result index 2c15dcaea6556..d89e1f168c9d1 100644 --- a/tests/integrationtest/r/planner/core/indexmerge_path.result +++ b/tests/integrationtest/r/planner/core/indexmerge_path.result @@ -7,14 +7,10 @@ set tidb_analyze_version=2; analyze table t; Level Code Message Note 1105 Analyze use auto adjusted sample rate 1.000000 for table planner__core__indexmerge_path.t, reason to use this rate is "use min(1, 110000/10000) as the sample-rate=1" -Warning 1105 analyzing multi-valued indexes is not supported, skip idx -Warning 1105 analyzing multi-valued indexes is not supported, skip idx2 analyze table t index idx; Level Code Message Note 1105 Analyze use auto adjusted sample rate 1.000000 for table planner__core__indexmerge_path.t, reason to use this rate is "TiDB assumes that the table is empty, use sample-rate=1" Warning 1105 The version 2 would collect all statistics not only the selected indexes -Warning 1105 analyzing multi-valued indexes is not supported, skip idx -Warning 1105 analyzing multi-valued indexes is not supported, skip idx2 set tidb_analyze_version=1; analyze table t; Level Code Message