diff --git a/errors.toml b/errors.toml index 9fc992b0b0a34..aca0b30bfc8eb 100644 --- a/errors.toml +++ b/errors.toml @@ -1561,6 +1561,11 @@ error = ''' Global Index is needed for index '%-.192s', since the unique index is not including all partitioning columns, and GLOBAL is not given as IndexOption ''' +["ddl:8265"] +error = ''' +Auto analyze is not effective for index '%-.192s', need analyze manually +''' + ["domain:8027"] error = ''' Information schema is out of date: schema failed to update in 1 lease, please make sure TiDB can connect to TiKV diff --git a/pkg/ddl/create_table.go b/pkg/ddl/create_table.go index f13b6f1d5d643..c566496dd23f4 100644 --- a/pkg/ddl/create_table.go +++ b/pkg/ddl/create_table.go @@ -405,7 +405,7 @@ func buildTableInfoWithCheck(ctx sessionctx.Context, s *ast.CreateTableStmt, dbC if err = checkTableInfoValidWithStmt(ctx, tbInfo, s); err != nil { return nil, err } - if err = checkTableInfoValidExtra(tbInfo); err != nil { + if err = checkTableInfoValidExtra(ctx, tbInfo); err != nil { return nil, err } return tbInfo, nil @@ -507,7 +507,7 @@ func checkGeneratedColumn(ctx sessionctx.Context, schemaName pmodel.CIStr, table // name length and column count. // (checkTableInfoValid is also used in repairing objects which don't perform // these checks. Perhaps the two functions should be merged together regardless?) -func checkTableInfoValidExtra(tbInfo *model.TableInfo) error { +func checkTableInfoValidExtra(ctx sessionctx.Context, tbInfo *model.TableInfo) error { if err := checkTooLongTable(tbInfo.Name); err != nil { return err } @@ -527,7 +527,7 @@ func checkTableInfoValidExtra(tbInfo *model.TableInfo) error { if err := checkColumnsAttributes(tbInfo.Columns); err != nil { return errors.Trace(err) } - if err := checkGlobalIndexes(tbInfo); err != nil { + if err := checkGlobalIndexes(ctx, tbInfo); err != nil { return errors.Trace(err) } diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index c244590d6b2b2..189b1875f3f43 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -952,7 +952,7 @@ func checkInvisibleIndexOnPK(tblInfo *model.TableInfo) error { } // checkGlobalIndex check if the index is allowed to have global index -func checkGlobalIndex(tblInfo *model.TableInfo, indexInfo *model.IndexInfo) error { +func checkGlobalIndex(ctx sessionctx.Context, tblInfo *model.TableInfo, indexInfo *model.IndexInfo) error { pi := tblInfo.GetPartitionInfo() isPartitioned := pi != nil && pi.Type != pmodel.PartitionTypeNone if indexInfo.Global { @@ -974,14 +974,15 @@ func checkGlobalIndex(tblInfo *model.TableInfo, indexInfo *model.IndexInfo) erro if inAllPartitionColumns { return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global Index including all columns in the partitioning expression") } + validateGlobalIndexWithGeneratedColumns(ctx.GetSessionVars().StmtCtx.ErrCtx(), tblInfo, indexInfo.Name.O, indexInfo.Columns) } return nil } // checkGlobalIndexes check if global index is supported. -func checkGlobalIndexes(tblInfo *model.TableInfo) error { +func checkGlobalIndexes(ctx sessionctx.Context, tblInfo *model.TableInfo) error { for _, indexInfo := range tblInfo.Indices { - err := checkGlobalIndex(tblInfo, indexInfo) + err := checkGlobalIndex(ctx, tblInfo, indexInfo) if err != nil { return err } @@ -1090,7 +1091,7 @@ func (e *executor) createTableWithInfoJob( } } - if err := checkTableInfoValidExtra(tbInfo); err != nil { + if err := checkTableInfoValidExtra(ctx, tbInfo); err != nil { return nil, err } @@ -4557,6 +4558,7 @@ func (e *executor) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexN if indexOption == nil || !indexOption.Global { return dbterror.ErrGlobalIndexNotExplicitlySet.GenWithStackByArgs("PRIMARY") } + validateGlobalIndexWithGeneratedColumns(ctx.GetSessionVars().StmtCtx.ErrCtx(), tblInfo, indexName.O, indexColumns) } } @@ -4716,6 +4718,7 @@ func (e *executor) createIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast if !globalIndex { return dbterror.ErrGlobalIndexNotExplicitlySet.GenWithStackByArgs(indexName.O) } + validateGlobalIndexWithGeneratedColumns(ctx.GetSessionVars().StmtCtx.ErrCtx(), tblInfo, indexName.O, indexColumns) } else if globalIndex { // TODO: remove this restriction return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global IndexOption on index including all columns in the partitioning expression") @@ -5165,6 +5168,18 @@ func validateCommentLength(ec errctx.Context, sqlMode mysql.SQLMode, name string return *comment, nil } +func validateGlobalIndexWithGeneratedColumns(ec errctx.Context, tblInfo *model.TableInfo, indexName string, indexColumns []*model.IndexColumn) { + // Auto analyze is not effective when a global index contains prefix columns or virtual generated columns. + for _, col := range indexColumns { + colInfo := tblInfo.Columns[col.Offset] + isPrefixCol := col.Length != types.UnspecifiedLength + if colInfo.IsVirtualGenerated() || isPrefixCol { + ec.AppendWarning(dbterror.ErrWarnGlobalIndexNeedManuallyAnalyze.FastGenByArgs(indexName)) + return + } + } +} + // BuildAddedPartitionInfo build alter table add partition info func BuildAddedPartitionInfo(ctx expression.BuildContext, meta *model.TableInfo, spec *ast.AlterTableSpec) (*model.PartitionInfo, error) { numParts := uint64(0) diff --git a/pkg/errno/errcode.go b/pkg/errno/errcode.go index ed109587548cf..5fef55b943be5 100644 --- a/pkg/errno/errcode.go +++ b/pkg/errno/errcode.go @@ -1143,6 +1143,8 @@ const ( ErrGlobalIndexNotExplicitlySet = 8264 + ErrWarnGlobalIndexNeedManuallyAnalyze = 8265 + // Resource group errors. ErrResourceGroupExists = 8248 ErrResourceGroupNotExists = 8249 diff --git a/pkg/errno/errname.go b/pkg/errno/errname.go index 46da93a92bcbb..630ab6a6fa21d 100644 --- a/pkg/errno/errname.go +++ b/pkg/errno/errname.go @@ -1174,4 +1174,6 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrBDRRestrictedDDL: mysql.Message("The operation is not allowed while the bdr role of this cluster is set to %s.", nil), ErrGlobalIndexNotExplicitlySet: mysql.Message("Global Index is needed for index '%-.192s', since the unique index is not including all partitioning columns, and GLOBAL is not given as IndexOption", nil), + + ErrWarnGlobalIndexNeedManuallyAnalyze: mysql.Message("Auto analyze is not effective for index '%-.192s', need analyze manually", nil), } diff --git a/pkg/executor/analyze_col_v2.go b/pkg/executor/analyze_col_v2.go index dec8c15ce44a5..4fa9e9f9070e7 100644 --- a/pkg/executor/analyze_col_v2.go +++ b/pkg/executor/analyze_col_v2.go @@ -76,9 +76,8 @@ func (e *AnalyzeColumnsExecV2) analyzeColumnsPushDownV2(gp *gp.Pool) *statistics isSpecial := false for _, col := range idx.Columns { colInfo := e.colsInfo[col.Offset] - isVirtualCol := colInfo.IsGenerated() && !colInfo.GeneratedStored isPrefixCol := col.Length != types.UnspecifiedLength - if isVirtualCol || isPrefixCol { + if colInfo.IsVirtualGenerated() || isPrefixCol { isSpecial = true break } diff --git a/pkg/executor/analyze_idx.go b/pkg/executor/analyze_idx.go index 7dcb05cc33f16..45834a73bb6d7 100644 --- a/pkg/executor/analyze_idx.go +++ b/pkg/executor/analyze_idx.go @@ -89,8 +89,8 @@ func analyzeIndexPushdown(idxExec *AnalyzeIndexExec) *statistics.AnalyzeResults Count: cnt, Snapshot: idxExec.snapshot, } - if idxExec.idxInfo.MVIndex { - result.ForMVIndex = true + if idxExec.idxInfo.MVIndex || (idxExec.idxInfo.Global && statsVer == statistics.Version2) { + result.ForMVIndexOrGlobalIndex = true } return result } diff --git a/pkg/planner/core/planbuilder.go b/pkg/planner/core/planbuilder.go index 2aafe5a75aa62..9af111326fc4d 100644 --- a/pkg/planner/core/planbuilder.go +++ b/pkg/planner/core/planbuilder.go @@ -2304,18 +2304,24 @@ func getColOffsetForAnalyze(colsInfo []*model.ColumnInfo, colID int64) int { // in the execution phase of ANALYZE, we need to modify index.Columns[i].Offset according to colInfos. // 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. +// For a special global index, we also need to analyze it as independent index analyze task. // See comments for AnalyzeResults.ForMVIndex for more details. func getModifiedIndexesInfoForAnalyze( tblInfo *model.TableInfo, allColumns bool, colsInfo []*model.ColumnInfo, -) ([]*model.IndexInfo, []*model.IndexInfo) { +) ([]*model.IndexInfo, []*model.IndexInfo, []*model.IndexInfo) { idxsInfo := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) independentIdxsInfo := make([]*model.IndexInfo, 0) + specialGlobalIdxsInfo := make([]*model.IndexInfo, 0) for _, originIdx := range tblInfo.Indices { if originIdx.State != model.StatePublic { continue } + if handleutil.IsSpecialGlobalIndex(originIdx, tblInfo) { + specialGlobalIdxsInfo = append(specialGlobalIdxsInfo, originIdx) + continue + } if originIdx.MVIndex { independentIdxsInfo = append(independentIdxsInfo, originIdx) continue @@ -2333,7 +2339,7 @@ func getModifiedIndexesInfoForAnalyze( } idxsInfo = append(idxsInfo, idx) } - return idxsInfo, independentIdxsInfo + return idxsInfo, independentIdxsInfo, specialGlobalIdxsInfo } // filterSkipColumnTypes filters out columns whose types are in the skipTypes list. @@ -2370,6 +2376,43 @@ func (b *PlanBuilder) filterSkipColumnTypes(origin []*model.ColumnInfo, tbl *res return } +// This function is to check whether all indexes is special global index or not. +// A special global index is an index that is both a global index and an expression index or a prefix index. +func checkIsAllSpecialGlobalIndex(as *ast.AnalyzeTableStmt, tbl *resolve.TableNameW) (bool, error) { + isAnalyzeTable := len(as.PartitionNames) == 0 + + // For `Analyze table t index` + if as.IndexFlag && len(as.IndexNames) == 0 { + for _, idx := range tbl.TableInfo.Indices { + if idx.State != model.StatePublic { + continue + } + if !handleutil.IsSpecialGlobalIndex(idx, tbl.TableInfo) { + return false, nil + } + // For `Analyze table t partition p0 index` + if !isAnalyzeTable { + return false, errors.NewNoStackErrorf("Analyze global index '%s' can't work with analyze specified partitions", idx.Name.O) + } + } + } else { + for _, idxName := range as.IndexNames { + idx := tbl.TableInfo.FindIndexByName(idxName.L) + if idx == nil || idx.State != model.StatePublic { + return false, plannererrors.ErrAnalyzeMissIndex.GenWithStackByArgs(idxName.O, tbl.Name.O) + } + if !handleutil.IsSpecialGlobalIndex(idx, tbl.TableInfo) { + return false, nil + } + // For `Analyze table t partition p0 index idx0` + if !isAnalyzeTable { + return false, errors.NewNoStackErrorf("Analyze global index '%s' can't work with analyze specified partitions", idx.Name.O) + } + } + } + return true, nil +} + func (b *PlanBuilder) buildAnalyzeFullSamplingTask( as *ast.AnalyzeTableStmt, analyzePlan *Analyze, @@ -2385,6 +2428,13 @@ func (b *PlanBuilder) buildAnalyzeFullSamplingTask( b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackError("The version 2 stats would ignore the INCREMENTAL keyword and do full sampling")) } + isAnalyzeTable := len(as.PartitionNames) == 0 + + allSpecialGlobalIndex, err := checkIsAllSpecialGlobalIndex(as, tbl) + if err != nil { + return err + } + astOpts, err := handleAnalyzeOptionsV2(as.AnalyzeOpts) if err != nil { return err @@ -2405,7 +2455,7 @@ func (b *PlanBuilder) buildAnalyzeFullSamplingTask( if err != nil { return err } - isAnalyzeTable := len(as.PartitionNames) == 0 + optionsMap, colsInfoMap, err := b.genV2AnalyzeOptions(persistOpts, tbl, isAnalyzeTable, physicalIDs, astOpts, as.ColumnChoice, astColList, &predicateCols, &mustAnalyzedCols, mustAllColumns) if err != nil { return err @@ -2414,51 +2464,79 @@ func (b *PlanBuilder) buildAnalyzeFullSamplingTask( analyzePlan.OptionsMap[physicalID] = opts } - // Build tasks for each partition. - for i, id := range physicalIDs { - physicalID := id - if id == tbl.TableInfo.ID { - id = statistics.NonPartitionTableID - } - info := AnalyzeInfo{ - DBName: tbl.Schema.O, - TableName: tbl.Name.O, - PartitionName: partitionNames[i], - TableID: statistics.AnalyzeTableID{TableID: tbl.TableInfo.ID, PartitionID: id}, - StatsVersion: version, + var indexes, independentIndexes, specialGlobalIndexes []*model.IndexInfo + + needAnalyzeCols := !(as.IndexFlag && allSpecialGlobalIndex) + + if needAnalyzeCols { + if as.IndexFlag { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("The version 2 would collect all statistics not only the selected indexes")) } - if optsV2, ok := optionsMap[physicalID]; ok { - info.V2Options = &optsV2 - } - execColsInfo := astColsInfo - if colsInfo, ok := colsInfoMap[physicalID]; ok { - execColsInfo = colsInfo - } - execColsInfo = b.filterSkipColumnTypes(execColsInfo, tbl, &mustAnalyzedCols) - allColumns := len(tbl.TableInfo.Columns) == len(execColsInfo) - indexes, independentIndexes := getModifiedIndexesInfoForAnalyze(tbl.TableInfo, allColumns, execColsInfo) - handleCols := BuildHandleColsForAnalyze(b.ctx, tbl.TableInfo, allColumns, execColsInfo) - newTask := AnalyzeColumnsTask{ - HandleCols: handleCols, - ColsInfo: execColsInfo, - AnalyzeInfo: info, - TblInfo: tbl.TableInfo, - Indexes: indexes, - } - if newTask.HandleCols == nil { - extraCol := model.NewExtraHandleColInfo() - // Always place _tidb_rowid at the end of colsInfo, this is corresponding to logics in `analyzeColumnsPushdown`. - newTask.ColsInfo = append(newTask.ColsInfo, extraCol) - newTask.HandleCols = util.NewIntHandleCols(colInfoToColumn(extraCol, len(newTask.ColsInfo)-1)) - } - analyzePlan.ColTasks = append(analyzePlan.ColTasks, newTask) - for _, indexInfo := range independentIndexes { - newIdxTask := AnalyzeIndexTask{ - IndexInfo: indexInfo, - TblInfo: tbl.TableInfo, + // Build tasks for each partition. + for i, id := range physicalIDs { + physicalID := id + if id == tbl.TableInfo.ID { + id = statistics.NonPartitionTableID + } + info := AnalyzeInfo{ + DBName: tbl.Schema.O, + TableName: tbl.Name.O, + PartitionName: partitionNames[i], + TableID: statistics.AnalyzeTableID{TableID: tbl.TableInfo.ID, PartitionID: id}, + StatsVersion: version, + } + if optsV2, ok := optionsMap[physicalID]; ok { + info.V2Options = &optsV2 + } + execColsInfo := astColsInfo + if colsInfo, ok := colsInfoMap[physicalID]; ok { + execColsInfo = colsInfo + } + execColsInfo = b.filterSkipColumnTypes(execColsInfo, tbl, &mustAnalyzedCols) + allColumns := len(tbl.TableInfo.Columns) == len(execColsInfo) + indexes, independentIndexes, specialGlobalIndexes = getModifiedIndexesInfoForAnalyze(tbl.TableInfo, allColumns, execColsInfo) + handleCols := BuildHandleColsForAnalyze(b.ctx, tbl.TableInfo, allColumns, execColsInfo) + newTask := AnalyzeColumnsTask{ + HandleCols: handleCols, + ColsInfo: execColsInfo, AnalyzeInfo: info, + TblInfo: tbl.TableInfo, + Indexes: indexes, + } + if newTask.HandleCols == nil { + extraCol := model.NewExtraHandleColInfo() + // Always place _tidb_rowid at the end of colsInfo, this is corresponding to logics in `analyzeColumnsPushdown`. + newTask.ColsInfo = append(newTask.ColsInfo, extraCol) + newTask.HandleCols = util.NewIntHandleCols(colInfoToColumn(extraCol, len(newTask.ColsInfo)-1)) + } + analyzePlan.ColTasks = append(analyzePlan.ColTasks, newTask) + for _, indexInfo := range independentIndexes { + newIdxTask := AnalyzeIndexTask{ + IndexInfo: indexInfo, + TblInfo: tbl.TableInfo, + AnalyzeInfo: info, + } + analyzePlan.IdxTasks = append(analyzePlan.IdxTasks, newIdxTask) + } + } + } + + if isAnalyzeTable { + if needAnalyzeCols { + // When `needAnalyzeCols == true`, non-global indexes already covered by previous loop, + // deal with global index here. + for _, indexInfo := range specialGlobalIndexes { + analyzePlan.IdxTasks = append(analyzePlan.IdxTasks, generateIndexTasks(indexInfo, as, tbl.TableInfo, nil, nil, version)...) + } + } else { + // For `analyze table t index idx1[, idx2]` and all indexes are global index. + for _, idxName := range as.IndexNames { + idx := tbl.TableInfo.FindIndexByName(idxName.L) + if idx == nil || !handleutil.IsSpecialGlobalIndex(idx, tbl.TableInfo) { + continue + } + analyzePlan.IdxTasks = append(analyzePlan.IdxTasks, generateIndexTasks(idx, as, tbl.TableInfo, nil, nil, version)...) } - analyzePlan.IdxTasks = append(analyzePlan.IdxTasks, newIdxTask) } } @@ -2740,7 +2818,6 @@ func (b *PlanBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt, opts map[ast.A b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackError("The analyze version from the session is not compatible with the existing statistics of the table. Use the existing version instead")) } if version == statistics.Version2 { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackError("The version 2 would collect all statistics not only the selected indexes")) return b.buildAnalyzeTable(as, opts, version) } for _, idxName := range as.IndexNames { @@ -2793,7 +2870,6 @@ func (b *PlanBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt, opts map[as b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("The analyze version from the session is not compatible with the existing statistics of the table. Use the existing version instead")) } if version == statistics.Version2 { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("The version 2 would collect all statistics not only the selected indexes")) return b.buildAnalyzeTable(as, opts, version) } for _, idx := range tblInfo.Indices { diff --git a/pkg/statistics/analyze.go b/pkg/statistics/analyze.go index f4161a598a906..83338737effb9 100644 --- a/pkg/statistics/analyze.go +++ b/pkg/statistics/analyze.go @@ -109,7 +109,10 @@ type AnalyzeResults struct { // take care of those table-level fields. // In conclusion, when saving the analyze result for mv index, we need to store the index stats, as for the // table-level fields, we only need to update the version. - ForMVIndex bool + // + // The global index has only one key range, so an independent task is used to process it. + // Global index needs to update only the version at the table-level fields, just like mv index. + ForMVIndexOrGlobalIndex bool } // DestroyAndPutToPool destroys the result and put it to the pool. diff --git a/pkg/statistics/handle/autoanalyze/autoanalyze.go b/pkg/statistics/handle/autoanalyze/autoanalyze.go index 3cc3b084bdf9e..d654e1e67bfa8 100644 --- a/pkg/statistics/handle/autoanalyze/autoanalyze.go +++ b/pkg/statistics/handle/autoanalyze/autoanalyze.go @@ -687,7 +687,7 @@ func tryAutoAnalyzePartitionTableInDynamicMode( } // Check if any index of the table needs to analyze. for _, idx := range tblInfo.Indices { - if idx.State != model.StatePublic { + if idx.State != model.StatePublic || statsutil.IsSpecialGlobalIndex(idx, tblInfo) { continue } // Collect all the partition names that need to analyze. diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/analysis_job_factory.go b/pkg/statistics/handle/autoanalyze/priorityqueue/analysis_job_factory.go index 8e0dbaaf21837..566269b9fc55f 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/analysis_job_factory.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/analysis_job_factory.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/statistics" statstypes "github.com/pingcap/tidb/pkg/statistics/handle/types" + "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/util/intest" "github.com/pingcap/tidb/pkg/util/timeutil" "github.com/tikv/client-go/v2/oracle" @@ -302,7 +303,8 @@ func (*AnalysisJobFactory) CheckNewlyAddedIndexesNeedAnalyzeForPartitionedTable( for _, idx := range tblInfo.Indices { // No need to analyze the index if it's not public. - if idx.State != model.StatePublic { + // Special global index also no need to trigger by auto analyze. + if idx.State != model.StatePublic || util.IsSpecialGlobalIndex(idx, tblInfo) { continue } diff --git a/pkg/statistics/handle/storage/save.go b/pkg/statistics/handle/storage/save.go index 5492de3d1f0d1..3de7ea472f7a6 100644 --- a/pkg/statistics/handle/storage/save.go +++ b/pkg/statistics/handle/storage/save.go @@ -151,9 +151,9 @@ func SaveTableStatsToStorage(sctx sessionctx.Context, if len(rows) > 0 { snapshot := rows[0].GetUint64(0) // A newer version analyze result has been written, so skip this writing. - // For multi-valued index analyze, this check is not needed because we expect there's another normal v2 analyze + // For multi-valued index or global index analyze, this check is not needed because we expect there's another normal v2 analyze // table task that may update the snapshot in stats_meta table (that task may finish before or after this task). - if snapshot >= results.Snapshot && results.StatsVer == statistics.Version2 && !results.ForMVIndex { + if snapshot >= results.Snapshot && results.StatsVer == statistics.Version2 && !results.ForMVIndexOrGlobalIndex { return } curCnt = int64(rows[0].GetUint64(1)) @@ -167,7 +167,7 @@ func SaveTableStatsToStorage(sctx sessionctx.Context, // In these cases, we use REPLACE INTO to directly insert/update the version, count and snapshot. snapShot := results.Snapshot count := results.Count - if results.ForMVIndex { + if results.ForMVIndexOrGlobalIndex { snapShot = 0 count = 0 } @@ -181,7 +181,7 @@ func SaveTableStatsToStorage(sctx sessionctx.Context, return 0, err } statsVer = version - } else if results.ForMVIndex { + } else if results.ForMVIndexOrGlobalIndex { // 1-2. There's already an existing record for this table, and we are handling stats for mv index now. // In this case, we only update the version. See comments for AnalyzeResults.ForMVIndex for more details. if _, err = util.Exec(sctx, diff --git a/pkg/statistics/handle/util/BUILD.bazel b/pkg/statistics/handle/util/BUILD.bazel index 1d6a46b63c17b..6a256c47dcecf 100644 --- a/pkg/statistics/handle/util/BUILD.bazel +++ b/pkg/statistics/handle/util/BUILD.bazel @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "util", @@ -14,12 +14,14 @@ go_library( deps = [ "//pkg/infoschema", "//pkg/kv", + "//pkg/meta/model", "//pkg/parser/terror", "//pkg/planner/core/resolve", "//pkg/sessionctx", "//pkg/sessionctx/sysproctrack", "//pkg/sessionctx/variable", "//pkg/table", + "//pkg/types", "//pkg/util", "//pkg/util/chunk", "//pkg/util/intest", @@ -33,3 +35,15 @@ go_library( "@org_uber_go_atomic//:atomic", ], ) + +go_test( + name = "util_test", + timeout = "short", + srcs = ["util_test.go"], + flaky = True, + deps = [ + ":util", + "//pkg/testkit", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/statistics/handle/util/util.go b/pkg/statistics/handle/util/util.go index ef4840ceb8fbb..c75ad2b7d8e37 100644 --- a/pkg/statistics/handle/util/util.go +++ b/pkg/statistics/handle/util/util.go @@ -23,10 +23,12 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/intest" @@ -309,3 +311,19 @@ type JSONPredicateColumn struct { LastAnalyzedAt *string `json:"last_analyzed_at"` ID int64 `json:"id"` } + +// IsSpecialGlobalIndex checks a index is a special global index or not. +// A special global index is one that is a global index and has virtual generated columns or prefix columns. +func IsSpecialGlobalIndex(idx *model.IndexInfo, tblInfo *model.TableInfo) bool { + if !idx.Global { + return false + } + for _, col := range idx.Columns { + colInfo := tblInfo.Columns[col.Offset] + isPrefixCol := col.Length != types.UnspecifiedLength + if colInfo.IsVirtualGenerated() || isPrefixCol { + return true + } + } + return false +} diff --git a/pkg/statistics/handle/util/util_test.go b/pkg/statistics/handle/util/util_test.go new file mode 100644 index 0000000000000..ff309a990ab88 --- /dev/null +++ b/pkg/statistics/handle/util/util_test.go @@ -0,0 +1,57 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package util_test + +import ( + "testing" + + "github.com/pingcap/tidb/pkg/statistics/handle/util" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/stretchr/testify/require" +) + +func TestIsSpecialGlobalIndex(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec( + "create table t(" + + " a int," + + " b int," + + " c int," + + " d varchar(20)," + + " unique index b(b) global," + + " index c(c)," + + " unique index ub_s((b+1)) global," + + " unique index ud_s(d(3)) global," + + " index b_s((b+1))," + + " index d_s(d(3))" + + ") partition by hash(a) partitions 5") + + tblInfo := dom.MustGetTableInfo(t, "test", "t") + cnt := 0 + for _, idx := range tblInfo.Indices { + switch idx.Name.O { + case "b", "c", "b_s", "d_s": + cnt++ + require.False(t, util.IsSpecialGlobalIndex(idx, tblInfo)) + case "ub_s", "ud_s": + cnt++ + require.True(t, util.IsSpecialGlobalIndex(idx, tblInfo)) + } + } + require.Equal(t, cnt, len(tblInfo.Indices)) +} diff --git a/pkg/util/dbterror/ddl_terror.go b/pkg/util/dbterror/ddl_terror.go index bd483c4226672..931c99a7171a0 100644 --- a/pkg/util/dbterror/ddl_terror.go +++ b/pkg/util/dbterror/ddl_terror.go @@ -497,6 +497,9 @@ var ( "tidb_enable_dist_task setting. To utilize distributed task execution, please enable tidb_ddl_enable_fast_reorg first."), nil)) // ErrGlobalIndexNotExplicitlySet is for Global index when not explicitly said GLOBAL, including UPDATE INDEXES ErrGlobalIndexNotExplicitlySet = ClassDDL.NewStd(mysql.ErrGlobalIndexNotExplicitlySet) + // ErrWarnGlobalIndexNeedManuallyAnalyze is used for global indexes, + // which cannot trigger automatic analysis when it contains prefix columns or virtual generated columns. + ErrWarnGlobalIndexNeedManuallyAnalyze = ClassDDL.NewStd(mysql.ErrWarnGlobalIndexNeedManuallyAnalyze) ) // ReorgRetryableErrCodes is the error codes that are retryable for reorganization. diff --git a/tests/integrationtest/r/executor/analyze.result b/tests/integrationtest/r/executor/analyze.result index 76fd04acb7dbc..641f7c5525351 100644 --- a/tests/integrationtest/r/executor/analyze.result +++ b/tests/integrationtest/r/executor/analyze.result @@ -741,8 +741,8 @@ analyze table t partition p1 index idxb; analyze table t partition p2 index idxc; show warnings; Level Code Message -Warning 1105 The version 2 would collect all statistics not only the selected indexes Warning 1105 No predicate column has been collected yet for table executor__analyze.t, so only indexes and the columns composing the indexes will be analyzed +Warning 1105 The version 2 would collect all statistics not only the selected indexes Note 1105 Analyze use auto adjusted sample rate 1.000000 for table executor__analyze.t's partition p2, reason to use this rate is "use min(1, 110000/10000) as the sample-rate=1" analyze table t partition p0; show warnings; diff --git a/tests/integrationtest/r/globalindex/analyze.result b/tests/integrationtest/r/globalindex/analyze.result new file mode 100644 index 0000000000000..a1d95df19af5f --- /dev/null +++ b/tests/integrationtest/r/globalindex/analyze.result @@ -0,0 +1,152 @@ +set tidb_enable_global_index=true; +drop table if exists t; +CREATE TABLE `t` ( +`a` int(11) DEFAULT NULL, +`b` int(11) DEFAULT NULL, +`c` int(11) DEFAULT NULL, +UNIQUE KEY `idx` ((`c` + 1)) global, +UNIQUE KEY `idx1` (`c`) global +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH (`b`) PARTITIONS 4; +show warnings; +Level Code Message +Warning 8265 Auto analyze is not effective for index 'idx', need analyze manually +insert into t values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5); +insert into t(a) values (1), (2); +analyze table t; +#Test special global index +explain select c+1 from t where (c+1)>3; +id estRows task access object operator info +IndexReader_7 3.00 root partition:all index:IndexRangeScan_5 +└─IndexRangeScan_5 3.00 cop[tikv] table:t, index:idx(`c` + 1) range:(3,+inf], keep order:false +explain select c+1 from t where (c+1)>4; +id estRows task access object operator info +IndexReader_7 2.00 root partition:all index:IndexRangeScan_5 +└─IndexRangeScan_5 2.00 cop[tikv] table:t, index:idx(`c` + 1) range:(4,+inf], keep order:false +#Test normal global index +explain select c from t where c > 2; +id estRows task access object operator info +IndexReader_7 3.00 root partition:all index:IndexRangeScan_5 +└─IndexRangeScan_5 3.00 cop[tikv] table:t, index:idx1(c) range:(2,+inf], keep order:false +explain select c from t where c > 3; +id estRows task access object operator info +IndexReader_7 2.00 root partition:all index:IndexRangeScan_5 +└─IndexRangeScan_5 2.00 cop[tikv] table:t, index:idx1(c) range:(3,+inf], keep order:false +alter table t add unique index idx2((`c` + 2)) global; +show warnings; +Level Code Message +analyze table t index idx2; +#Test special global index after analyze index +explain select c+2 from t where (c+2)>3; +id estRows task access object operator info +IndexReader_7 4.00 root partition:all index:IndexRangeScan_5 +└─IndexRangeScan_5 4.00 cop[tikv] table:t, index:idx2(`c` + 2) range:(3,+inf], keep order:false +explain select c+2 from t where (c+2)>4; +id estRows task access object operator info +IndexReader_7 3.00 root partition:all index:IndexRangeScan_5 +└─IndexRangeScan_5 3.00 cop[tikv] table:t, index:idx2(`c` + 2) range:(4,+inf], keep order:false +#Test return error with analyze special global index +analyze table t partition p0, p1 index idx; +Error 1105 (HY000): Analyze global index 'idx' can't work with analyze specified partitions +analyze table t partition p0, p1 index; +Error 1105 (HY000): Analyze global index 'idx' can't work with analyze specified partitions +show stats_histograms where table_name='t' and Db_name='globalindex__analyze'; +Db_name Table_name Partition_name Column_name Is_index Update_time Distinct_count Null_count Avg_col_size Correlation Load_status Total_mem_usage Hist_mem_usage Topn_mem_usage Cms_mem_usage +globalindex__analyze t global c 0 5 2 0 allLoaded 237 0 237 0 +globalindex__analyze t global idx 1 5 2 0 allLoaded 237 0 237 0 +globalindex__analyze t global idx1 1 5 2 0 allLoaded 237 0 237 0 +globalindex__analyze t global idx2 1 5 2 0 allLoaded 237 0 237 0 +globalindex__analyze t p0 c 0 1 2 1 allLoaded 73 0 73 0 +globalindex__analyze t p0 idx1 1 1 2 0 allLoaded 73 0 73 0 +globalindex__analyze t p1 c 0 2 0 1 allLoaded 114 0 114 0 +globalindex__analyze t p1 idx1 1 2 0 0 allLoaded 114 0 114 0 +globalindex__analyze t p2 c 0 1 0 1 allLoaded 73 0 73 0 +globalindex__analyze t p2 idx1 1 1 0 0 allLoaded 73 0 73 0 +globalindex__analyze t p3 c 0 1 0 1 allLoaded 73 0 73 0 +globalindex__analyze t p3 idx1 1 1 0 0 allLoaded 73 0 73 0 +#Test global index + primary key + prefix index +drop table if exists t; +create table t ( +id int, +value int, +name varchar(20), +primary key(name(2), id) global +) partition by hash(value) partitions 4; +show warnings; +Level Code Message +Warning 8265 Auto analyze is not effective for index 'PRIMARY', need analyze manually +insert into t values (1, 1, 'abc'), (2, 2, 'abd'), (3, 3, 'axe'), (4, 4, 'axf'), (5, 5, 'azd'); +analyze table t; +explain select id from t use index(primary) where name like 'ab%'; +id estRows task access object operator info +Projection_4 2.00 root globalindex__analyze.t.id +└─IndexLookUp_10 2.00 root partition:all + ├─IndexRangeScan_6(Build) 2.00 cop[tikv] table:t, index:PRIMARY(name, id) range:["ab","ac"), keep order:false + └─Selection_9(Probe) 2.00 cop[tikv] like(globalindex__analyze.t.name, "ab%", 92) + └─TableRowIDScan_7 2.00 cop[tikv] table:t keep order:false +select id from t use index(primary) where name like 'ab%'; +id +1 +2 +#Test return error with analyze special global index +analyze table t partition p0 index primary; +Error 1105 (HY000): Analyze global index 'PRIMARY' can't work with analyze specified partitions +analyze table t partition p0 index; +Error 1105 (HY000): Analyze global index 'PRIMARY' can't work with analyze specified partitions +#Test analyze special global indexes and normal indexes mixed. +drop table if exists t; +create table t( +a int, +b int, +c int, +d varchar(20), +index b(b), +unique index b_s((b + 1)) global, +unique index d_s(d(3)) global +) partition by hash(a) partitions 5; +show warnings; +Level Code Message +Warning 8265 Auto analyze is not effective for index 'b_s', need analyze manually +Warning 8265 Auto analyze is not effective for index 'd_s', need analyze manually +insert into t values (1, 1, 1, 'abc'), (2, 2, 2, 'abd'), (3, 3, 3, 'axe'), (4, 4, 4, 'axf'), (5, 5, 5, 'azd'); +#Test `analyze table t index idx1[, idx2]` stmt +analyze table t index b, b_s, d_s; +explain select * from t use index(b_s) where b + 1 > 3; +id estRows task access object operator info +Projection_4 3.00 root globalindex__analyze.t.a, globalindex__analyze.t.b, globalindex__analyze.t.c, globalindex__analyze.t.d +└─IndexLookUp_9 3.00 root partition:all + ├─IndexRangeScan_6(Build) 3.00 cop[tikv] table:t, index:b_s(`b` + 1) range:(3,+inf], keep order:false + └─TableRowIDScan_7(Probe) 3.00 cop[tikv] table:t keep order:false +explain select * from t use index(b) where b > 3; +id estRows task access object operator info +IndexLookUp_7 2.00 root partition:all +├─IndexRangeScan_5(Build) 2.00 cop[tikv] table:t, index:b(b) range:(3,+inf], keep order:false +└─TableRowIDScan_6(Probe) 2.00 cop[tikv] table:t keep order:false +explain select * from t use index(d_s) where d like 'ab%'; +id estRows task access object operator info +IndexLookUp_9 2.00 root partition:all +├─IndexRangeScan_5(Build) 2.00 cop[tikv] table:t, index:d_s(d) range:["ab","ac"), keep order:false +└─Selection_8(Probe) 2.00 cop[tikv] like(globalindex__analyze.t.d, "ab%", 92) + └─TableRowIDScan_6 2.00 cop[tikv] table:t keep order:false +show stats_histograms where table_name='t' and Db_name='globalindex__analyze'; +Db_name Table_name Partition_name Column_name Is_index Update_time Distinct_count Null_count Avg_col_size Correlation Load_status Total_mem_usage Hist_mem_usage Topn_mem_usage Cms_mem_usage +globalindex__analyze t global b 0 5 0 0 allLoaded 237 0 237 0 +globalindex__analyze t global b 1 5 0 0 allLoaded 237 0 237 0 +globalindex__analyze t global b_s 1 5 0 0 allLoaded 237 0 237 0 +globalindex__analyze t global d 0 5 0 0 allLoaded 242 0 242 0 +globalindex__analyze t global d_s 1 5 0 0 allLoaded 242 0 242 0 +globalindex__analyze t p0 b 0 1 0 1 allLoaded 73 0 73 0 +globalindex__analyze t p0 b 1 1 0 0 allLoaded 73 0 73 0 +globalindex__analyze t p0 d 0 1 0 1 allLoaded 74 0 74 0 +globalindex__analyze t p1 b 0 1 0 1 allLoaded 73 0 73 0 +globalindex__analyze t p1 b 1 1 0 0 allLoaded 73 0 73 0 +globalindex__analyze t p1 d 0 1 0 1 allLoaded 74 0 74 0 +globalindex__analyze t p2 b 0 1 0 1 allLoaded 73 0 73 0 +globalindex__analyze t p2 b 1 1 0 0 allLoaded 73 0 73 0 +globalindex__analyze t p2 d 0 1 0 1 allLoaded 74 0 74 0 +globalindex__analyze t p3 b 0 1 0 1 allLoaded 73 0 73 0 +globalindex__analyze t p3 b 1 1 0 0 allLoaded 73 0 73 0 +globalindex__analyze t p3 d 0 1 0 1 allLoaded 74 0 74 0 +globalindex__analyze t p4 b 0 1 0 1 allLoaded 73 0 73 0 +globalindex__analyze t p4 b 1 1 0 0 allLoaded 73 0 73 0 +globalindex__analyze t p4 d 0 1 0 1 allLoaded 74 0 74 0 diff --git a/tests/integrationtest/r/globalindex/expression_index.result b/tests/integrationtest/r/globalindex/expression_index.result index 680328d5e9738..e55d24a4627f1 100644 --- a/tests/integrationtest/r/globalindex/expression_index.result +++ b/tests/integrationtest/r/globalindex/expression_index.result @@ -5,6 +5,9 @@ CREATE TABLE `t` ( UNIQUE KEY `idx` ((lower(`b`))) global ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin PARTITION BY HASH (`a`) PARTITIONS 5; +show warnings; +Level Code Message +Warning 8265 Auto analyze is not effective for index 'idx', need analyze manually insert into t values (1, 'a'), (2, 'b'), (3, 'C'), (4, 'd'), (5, 'x'); insert into t values (3, 'c'); Error 1062 (23000): Duplicate entry 'c' for key 't.idx' diff --git a/tests/integrationtest/t/globalindex/analyze.test b/tests/integrationtest/t/globalindex/analyze.test new file mode 100644 index 0000000000000..12401dec55cab --- /dev/null +++ b/tests/integrationtest/t/globalindex/analyze.test @@ -0,0 +1,98 @@ +set tidb_enable_global_index=true; + +drop table if exists t; +CREATE TABLE `t` ( + `a` int(11) DEFAULT NULL, + `b` int(11) DEFAULT NULL, + `c` int(11) DEFAULT NULL, + UNIQUE KEY `idx` ((`c` + 1)) global, + UNIQUE KEY `idx1` (`c`) global +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH (`b`) PARTITIONS 4; + +show warnings; + +insert into t values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5); +insert into t(a) values (1), (2); +analyze table t; + +--echo #Test special global index +explain select c+1 from t where (c+1)>3; +explain select c+1 from t where (c+1)>4; + +--echo #Test normal global index +explain select c from t where c > 2; +explain select c from t where c > 3; + +alter table t add unique index idx2((`c` + 2)) global; +show warnings; + +analyze table t index idx2; + +--echo #Test special global index after analyze index +explain select c+2 from t where (c+2)>3; +explain select c+2 from t where (c+2)>4; + +--echo #Test return error with analyze special global index +--error 1105 +analyze table t partition p0, p1 index idx; +--error 1105 +analyze table t partition p0, p1 index; + +--sorted_result +--replace_column 6 9 +show stats_histograms where table_name='t' and Db_name='globalindex__analyze'; + + +--echo #Test global index + primary key + prefix index +drop table if exists t; +create table t ( + id int, + value int, + name varchar(20), + primary key(name(2), id) global +) partition by hash(value) partitions 4; + +show warnings; + +insert into t values (1, 1, 'abc'), (2, 2, 'abd'), (3, 3, 'axe'), (4, 4, 'axf'), (5, 5, 'azd'); + +analyze table t; + +explain select id from t use index(primary) where name like 'ab%'; +select id from t use index(primary) where name like 'ab%'; + +--echo #Test return error with analyze special global index +--error 1105 +analyze table t partition p0 index primary; +--error 1105 +analyze table t partition p0 index; + + +--echo #Test analyze special global indexes and normal indexes mixed. +drop table if exists t; +create table t( + a int, + b int, + c int, + d varchar(20), + index b(b), + unique index b_s((b + 1)) global, + unique index d_s(d(3)) global +) partition by hash(a) partitions 5; + +show warnings; + +insert into t values (1, 1, 1, 'abc'), (2, 2, 2, 'abd'), (3, 3, 3, 'axe'), (4, 4, 4, 'axf'), (5, 5, 5, 'azd'); + +-- echo #Test `analyze table t index idx1[, idx2]` stmt +analyze table t index b, b_s, d_s; + +explain select * from t use index(b_s) where b + 1 > 3; +explain select * from t use index(b) where b > 3; +explain select * from t use index(d_s) where d like 'ab%'; + +--sorted_result +--replace_column 6 9 +show stats_histograms where table_name='t' and Db_name='globalindex__analyze'; + diff --git a/tests/integrationtest/t/globalindex/expression_index.test b/tests/integrationtest/t/globalindex/expression_index.test index 246aa130568ed..de5d36dc1ab2f 100644 --- a/tests/integrationtest/t/globalindex/expression_index.test +++ b/tests/integrationtest/t/globalindex/expression_index.test @@ -6,6 +6,8 @@ CREATE TABLE `t` ( ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin PARTITION BY HASH (`a`) PARTITIONS 5; +show warnings; + insert into t values (1, 'a'), (2, 'b'), (3, 'C'), (4, 'd'), (5, 'x'); --error 1062