Skip to content

Commit

Permalink
*: global stats consistency for dynamic prune (#34695)
Browse files Browse the repository at this point in the history
close #34730
  • Loading branch information
chrysan committed May 19, 2022
1 parent 2106361 commit 2cf1354
Show file tree
Hide file tree
Showing 13 changed files with 605 additions and 241 deletions.
1 change: 1 addition & 0 deletions errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -1067,6 +1067,7 @@ const (
ErrPlacementPolicyInUse = 8241
ErrOptOnCacheTable = 8242
ErrHTTPServiceError = 8243
ErrPartitionColumnStatsMissing = 8244
// TiKV/PD/TiFlash errors.
ErrPDServerTimeout = 9001
ErrTiKVServerTimeout = 9002
Expand Down
7 changes: 4 additions & 3 deletions errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -1059,9 +1059,10 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{

ErrInvalidTableSample: mysql.Message("Invalid TABLESAMPLE: %s", nil),

ErrJSONObjectKeyTooLong: mysql.Message("TiDB does not yet support JSON objects with the key length >= 65536", nil),
ErrPartitionStatsMissing: mysql.Message("Build table: %s global-level stats failed due to missing partition-level stats", nil),
ErrNotSupportedWithSem: mysql.Message("Feature '%s' is not supported when security enhanced mode is enabled", nil),
ErrJSONObjectKeyTooLong: mysql.Message("TiDB does not yet support JSON objects with the key length >= 65536", nil),
ErrPartitionStatsMissing: mysql.Message("Build table: %s global-level stats failed due to missing partition-level stats", nil),
ErrPartitionColumnStatsMissing: mysql.Message("Build table: %s global-level stats failed due to missing partition-level column stats, please run analyze table to refresh columns of all partitions", nil),
ErrNotSupportedWithSem: mysql.Message("Feature '%s' is not supported when security enhanced mode is enabled", nil),

ErrPlacementPolicyCheck: mysql.Message("Placement policy didn't meet the constraint, reason: %s", nil),
ErrMultiStatementDisabled: mysql.Message("client has multi-statement capability disabled. Run SET GLOBAL tidb_multi_statement_mode='ON' after you understand the security risk", nil),
Expand Down
5 changes: 5 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -2586,6 +2586,11 @@ error = '''
Build table: %s global-level stats failed due to missing partition-level stats
'''

["types:8244"]
error = '''
Build table: %s global-level stats failed due to missing partition-level column stats, please run analyze table to refresh columns of all partitions
'''

["variable:1193"]
error = '''
Unknown system variable '%-.64s'
Expand Down
18 changes: 13 additions & 5 deletions executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -209,7 +209,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error {
}
globalStats, err := statsHandle.MergePartitionStats2GlobalStatsByTableID(e.ctx, globalOpts, e.ctx.GetInfoSchema().(infoschema.InfoSchema), globalStatsID.tableID, info.isIndex, info.histIDs)
if err != nil {
if types.ErrPartitionStatsMissing.Equal(err) {
if types.ErrPartitionStatsMissing.Equal(err) || types.ErrPartitionColumnStatsMissing.Equal(err) {
// When we find some partition-level stats are missing, we need to report warning.
e.ctx.GetSessionVars().StmtCtx.AppendWarning(err)
continue
Expand All @@ -230,7 +230,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error {
}
}
}
err = e.saveAnalyzeOptsV2()
err = e.saveV2AnalyzeOpts()
if err != nil {
e.ctx.GetSessionVars().StmtCtx.AppendWarning(err)
}
Expand Down Expand Up @@ -258,14 +258,22 @@ func finishJobWithLog(sctx sessionctx.Context, job *statistics.AnalyzeJob, analy
}
}

func (e *AnalyzeExec) saveAnalyzeOptsV2() error {
func (e *AnalyzeExec) saveV2AnalyzeOpts() error {
if !variable.PersistAnalyzeOptions.Load() || len(e.OptionsMap) == 0 {
return nil
}
// only to save table options if dynamic prune mode
dynamicPrune := variable.PartitionPruneMode(e.ctx.GetSessionVars().PartitionPruneMode.Load()) == variable.Dynamic
toSaveMap := make(map[int64]core.V2AnalyzeOptions)
for id, opts := range e.OptionsMap {
if !opts.IsPartition || !dynamicPrune {
toSaveMap[id] = opts
}
}
sql := new(strings.Builder)
sqlexec.MustFormatSQL(sql, "REPLACE INTO mysql.analyze_options (table_id,sample_num,sample_rate,buckets,topn,column_choice,column_ids) VALUES ")
idx := 0
for _, opts := range e.OptionsMap {
for _, opts := range toSaveMap {
sampleNum := opts.RawOpts[ast.AnalyzeOptNumSamples]
sampleRate := float64(0)
if val, ok := opts.RawOpts[ast.AnalyzeOptSampleRate]; ok {
Expand All @@ -283,7 +291,7 @@ func (e *AnalyzeExec) saveAnalyzeOptsV2() error {
}
colIDStrs := strings.Join(colIDs, ",")
sqlexec.MustFormatSQL(sql, "(%?,%?,%?,%?,%?,%?,%?)", opts.PhyTableID, sampleNum, sampleRate, buckets, topn, colChoice, colIDStrs)
if idx < len(e.OptionsMap)-1 {
if idx < len(toSaveMap)-1 {
sqlexec.MustFormatSQL(sql, ",")
}
idx += 1
Expand Down
689 changes: 492 additions & 197 deletions executor/analyze_test.go

Large diffs are not rendered by default.

5 changes: 0 additions & 5 deletions executor/show_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1588,11 +1588,6 @@ func TestShowVar(t *testing.T) {
res = tk.MustQuery(showSQL)
require.Len(t, res.Rows(), len(globalVars))

// Test a known hidden variable.
res = tk.MustQuery("show variables like '" + variable.TiDBPartitionPruneMode + "'")
require.Len(t, res.Rows(), 0)
res = tk.MustQuery("show global variables like '" + variable.TiDBPartitionPruneMode + "'")
require.Len(t, res.Rows(), 0)
// Test Hidden tx_read_ts
res = tk.MustQuery("show variables like '%tx_read_ts'")
require.Len(t, res.Rows(), 0)
Expand Down
28 changes: 28 additions & 0 deletions parser/model/model.go
Original file line number Diff line number Diff line change
Expand Up @@ -299,6 +299,16 @@ func FindColumnInfoByID(cols []*ColumnInfo, id int64) *ColumnInfo {
return nil
}

// FindIndexInfoByID finds IndexInfo in indices by id.
func FindIndexInfoByID(indices []*IndexInfo, id int64) *IndexInfo {
for _, idx := range indices {
if idx.ID == id {
return idx
}
}
return nil
}

// ExtraHandleID is the column ID of column which we need to append to schema to occupy the handle's position
// for use of execution phase.
const ExtraHandleID = -1
Expand Down Expand Up @@ -1224,6 +1234,24 @@ func (t *TableInfo) FindConstraintInfoByName(constrName string) *ConstraintInfo
return nil
}

// FindIndexNameByID finds index name by id.
func (t *TableInfo) FindIndexNameByID(id int64) string {
indexInfo := FindIndexInfoByID(t.Indices, id)
if indexInfo != nil {
return indexInfo.Name.L
}
return ""
}

// FindColumnNameByID finds column name by id.
func (t *TableInfo) FindColumnNameByID(id int64) string {
colInfo := FindColumnInfoByID(t.Columns, id)
if colInfo != nil {
return colInfo.Name.L
}
return ""
}

// FKInfo provides meta data describing a foreign key constraint.
type FKInfo struct {
ID int64 `json:"id"`
Expand Down
11 changes: 6 additions & 5 deletions planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -1117,11 +1117,12 @@ type AnalyzeInfo struct {

// V2AnalyzeOptions is used to hold analyze options information.
type V2AnalyzeOptions struct {
PhyTableID int64
RawOpts map[ast.AnalyzeOptionType]uint64
FilledOpts map[ast.AnalyzeOptionType]uint64
ColChoice model.ColumnChoice
ColumnList []*model.ColumnInfo
PhyTableID int64
RawOpts map[ast.AnalyzeOptionType]uint64
FilledOpts map[ast.AnalyzeOptionType]uint64
ColChoice model.ColumnChoice
ColumnList []*model.ColumnInfo
IsPartition bool
}

// AnalyzeColumnsTask is used for analyze columns.
Expand Down
31 changes: 26 additions & 5 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -2181,6 +2181,13 @@ func (b *PlanBuilder) genV2AnalyzeOptions(
if !persist {
return optionsMap, colsInfoMap, nil
}
dynamicPrune := variable.PartitionPruneMode(b.ctx.GetSessionVars().PartitionPruneMode.Load()) == variable.Dynamic
if !isAnalyzeTable && dynamicPrune && (len(astOpts) > 0 || astColChoice != model.DefaultChoice) {
astOpts = make(map[ast.AnalyzeOptionType]uint64, 0)
astColChoice = model.DefaultChoice
astColList = make([]*model.ColumnInfo, 0)
b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.New("Ignore columns and options when analyze partition in dynamic mode"))
}
tblSavedOpts, tblSavedColChoice, tblSavedColList, err := b.getSavedAnalyzeOpts(tbl.TableInfo.ID, tbl.TableInfo)
if err != nil {
return nil, nil, err
Expand All @@ -2198,16 +2205,30 @@ func (b *PlanBuilder) genV2AnalyzeOptions(
return nil, nil, err
}
tblAnalyzeOptions := V2AnalyzeOptions{
PhyTableID: tbl.TableInfo.ID,
RawOpts: tblOpts,
FilledOpts: tblFilledOpts,
ColChoice: tblColChoice,
ColumnList: tblColList,
PhyTableID: tbl.TableInfo.ID,
RawOpts: tblOpts,
FilledOpts: tblFilledOpts,
ColChoice: tblColChoice,
ColumnList: tblColList,
IsPartition: false,
}
optionsMap[tbl.TableInfo.ID] = tblAnalyzeOptions
colsInfoMap[tbl.TableInfo.ID] = tblColsInfo
for _, id := range physicalIDs {
if id != tbl.TableInfo.ID {
if dynamicPrune {
parV2Options := V2AnalyzeOptions{
PhyTableID: id,
RawOpts: tblOpts,
FilledOpts: tblFilledOpts,
ColChoice: tblColChoice,
ColumnList: tblColList,
IsPartition: true,
}
optionsMap[id] = parV2Options
colsInfoMap[id] = tblColsInfo
continue
}
parSavedOpts, parSavedColChoice, parSavedColList, err := b.getSavedAnalyzeOpts(id, tbl.TableInfo)
if err != nil {
return nil, nil, err
Expand Down
16 changes: 14 additions & 2 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -1359,14 +1359,26 @@ var defaultSysVars = []*SysVar{
s.EnableClusteredIndex = TiDBOptEnableClustered(val)
return nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBPartitionPruneMode, Value: DefTiDBPartitionPruneMode, Hidden: true, Type: TypeStr, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
{Scope: ScopeGlobal | ScopeSession, Name: TiDBPartitionPruneMode, Value: DefTiDBPartitionPruneMode, Type: TypeStr, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
mode := PartitionPruneMode(normalizedValue).Update()
if !mode.Valid() {
return normalizedValue, ErrWrongTypeForVar.GenWithStackByArgs(TiDBPartitionPruneMode)
}
return string(mode), nil
}, GetSession: func(s *SessionVars) (string, error) {
return s.PartitionPruneMode.Load(), nil
}, SetSession: func(s *SessionVars, val string) error {
s.PartitionPruneMode.Store(strings.ToLower(strings.TrimSpace(val)))
newMode := strings.ToLower(strings.TrimSpace(val))
if PartitionPruneMode(s.PartitionPruneMode.Load()) == Static && PartitionPruneMode(newMode) == Dynamic {
s.StmtCtx.AppendWarning(errors.New("Please analyze all partition tables again for consistency between partition and global stats"))
}
s.PartitionPruneMode.Store(newMode)
return nil
}, SetGlobal: func(s *SessionVars, val string) error {
newMode := strings.ToLower(strings.TrimSpace(val))
if PartitionPruneMode(newMode) == Dynamic {
s.StmtCtx.AppendWarning(errors.New("Please analyze all partition tables again for consistency between partition and global stats"))
}
return nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBRedactLog, Value: BoolToOnOff(DefTiDBRedactLog), Type: TypeBool, SetSession: func(s *SessionVars, val string) error {
Expand Down
19 changes: 12 additions & 7 deletions statistics/handle/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -466,19 +466,24 @@ func (h *Handle) mergePartitionStats2GlobalStats(sc sessionctx.Context, opts map
if isIndex == 0 {
errMsg = fmt.Sprintf("`%s`", tableInfo.Name.L)
} else {
indexName := ""
for _, idx := range tableInfo.Indices {
if idx.ID == histIDs[0] {
indexName = idx.Name.L
}
}
errMsg = fmt.Sprintf("`%s` index: `%s`", tableInfo.Name.L, indexName)
errMsg = fmt.Sprintf("`%s` index: `%s`", tableInfo.Name.L, tableInfo.FindIndexNameByID(histIDs[0]))
}
err = types.ErrPartitionStatsMissing.GenWithStackByArgs(errMsg)
return
}
for i := 0; i < globalStats.Num; i++ {
count, hg, cms, topN, fms := partitionStats.GetStatsInfo(histIDs[i], isIndex == 1)
// partition is not empty but column stats(hist, topn) is missing
if partitionStats.Count > 0 && (hg == nil || hg.TotalRowCount() <= 0) && (topN == nil || topN.TotalCount() <= 0) {
var errMsg string
if isIndex == 0 {
errMsg = fmt.Sprintf("`%s` column: `%s`", tableInfo.Name.L, tableInfo.FindColumnNameByID(histIDs[i]))
} else {
errMsg = fmt.Sprintf("`%s` index: `%s`", tableInfo.Name.L, tableInfo.FindIndexNameByID(histIDs[i]))
}
err = types.ErrPartitionColumnStatsMissing.GenWithStackByArgs(errMsg)
return
}
if i == 0 {
// In a partition, we will only update globalStats.Count once
globalStats.Count += count
Expand Down
13 changes: 1 addition & 12 deletions statistics/handle/handle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1070,18 +1070,6 @@ partition by range (a) (
checkHealthy(60, 50, 66)
}

func TestHideGlobalStatsSwitch(t *testing.T) {
// NOTICE: remove this test when this global-stats is GA.
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)
rs := tk.MustQuery("show variables").Rows()
for _, r := range rs {
require.NotEqual(t, "tidb_partition_prune_mode", strings.ToLower(r[0].(string)))
}
require.Len(t, tk.MustQuery("show variables where variable_name like '%tidb_partition_prune_mode%'").Rows(), 0)
}

func TestGlobalStatsData(t *testing.T) {
store, dom, clean := testkit.CreateMockStoreAndDomain(t)
defer clean()
Expand Down Expand Up @@ -2220,6 +2208,7 @@ func TestFMSWithAnalyzePartition(t *testing.T) {
tk.MustExec("analyze table t partition p0 with 1 topn, 2 buckets")
tk.MustQuery("show warnings").Sort().Check(testkit.Rows(
"Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0",
"Warning 1105 Ignore columns and options when analyze partition in dynamic mode",
"Warning 8131 Build table: `t` global-level stats failed due to missing partition-level stats",
"Warning 8131 Build table: `t` index: `a` global-level stats failed due to missing partition-level stats",
))
Expand Down
3 changes: 3 additions & 0 deletions types/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -87,4 +87,7 @@ var (
// ErrPartitionStatsMissing is returned when the partition-level stats is missing and the build global-level stats fails.
// Put this error here is to prevent `import cycle not allowed`.
ErrPartitionStatsMissing = dbterror.ClassTypes.NewStd(mysql.ErrPartitionStatsMissing)
// ErrPartitionColumnStatsMissing is returned when the partition-level column stats is missing and the build global-level stats fails.
// Put this error here is to prevent `import cycle not allowed`.
ErrPartitionColumnStatsMissing = dbterror.ClassTypes.NewStd(mysql.ErrPartitionColumnStatsMissing)
)

0 comments on commit 2cf1354

Please sign in to comment.