Skip to content

Commit

Permalink
errno, statistics: show partition name whose stats are missing when m…
Browse files Browse the repository at this point in the history
…erging global stats fails (pingcap#39348)
  • Loading branch information
xuyifangreeneyes authored Nov 25, 2022
1 parent ddfa6db commit c34ee82
Show file tree
Hide file tree
Showing 5 changed files with 25 additions and 29 deletions.
4 changes: 2 additions & 2 deletions errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -1072,8 +1072,8 @@ 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),
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),
ErrPartitionStatsMissing: mysql.Message("Build global-level stats failed due to missing partition-level stats: %s", nil),
ErrPartitionColumnStatsMissing: mysql.Message("Build global-level stats failed due to missing partition-level column stats: %s, please run analyze table to refresh columns of all partitions", nil),
ErrDDLSetting: mysql.Message("Error happened when enable/disable DDL: %s", nil),
ErrNotSupportedWithSem: mysql.Message("Feature '%s' is not supported when security enhanced mode is enabled", nil),

Expand Down
4 changes: 2 additions & 2 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -2778,12 +2778,12 @@ TiDB does not yet support JSON objects with the key length >= 65536

["types:8131"]
error = '''
Build table: %s global-level stats failed due to missing partition-level stats
Build global-level stats failed due to missing partition-level stats: %s
'''

["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
Build global-level stats failed due to missing partition-level column stats: %s, please run analyze table to refresh columns of all partitions
'''

["variable:1193"]
Expand Down
18 changes: 9 additions & 9 deletions executor/analyzetest/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2836,8 +2836,8 @@ PARTITION BY RANGE ( a ) (
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: `idx` global-level stats failed due to missing partition-level stats",
"Warning 8131 Build global-level stats failed due to missing partition-level stats: table `t` partition `p1`",
"Warning 8131 Build global-level stats failed due to missing partition-level stats: table `t` partition `p1`",
))
tk.MustQuery("select * from t where a > 1 and b > 1 and c > 1 and d > 1")
require.NoError(t, h.LoadNeededHistograms())
Expand All @@ -2849,8 +2849,8 @@ PARTITION BY RANGE ( a ) (
tk.MustExec("analyze table t partition p0")
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 8131 Build table: `t` global-level stats failed due to missing partition-level stats",
"Warning 8131 Build table: `t` index: `idx` global-level stats failed due to missing partition-level stats",
"Warning 8131 Build global-level stats failed due to missing partition-level stats: table `t` partition `p1`",
"Warning 8131 Build global-level stats failed due to missing partition-level stats: table `t` partition `p1`",
))
tbl = h.GetTableStats(tableInfo)
require.Equal(t, tbl.Version, lastVersion) // global stats not updated
Expand Down Expand Up @@ -2904,7 +2904,7 @@ PARTITION BY RANGE ( a ) (
tk.MustExec("analyze table t partition p1 columns a,b,d with 1 topn, 3 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 p1",
"Warning 8244 Build table: `t` column: `d` global-level stats failed due to missing partition-level column stats, please run analyze table to refresh columns of all partitions",
"Warning 8244 Build global-level stats failed due to missing partition-level column stats: table `t` partition `p0` column `d`, please run analyze table to refresh columns of all partitions",
))

// analyze partition with existing table-level options and existing partition stats under dynamic
Expand All @@ -2914,7 +2914,7 @@ PARTITION BY RANGE ( a ) (
tk.MustQuery("show warnings").Sort().Check(testkit.Rows(
"Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1",
"Warning 1105 Ignore columns and options when analyze partition in dynamic mode",
"Warning 8244 Build table: `t` column: `d` global-level stats failed due to missing partition-level column stats, please run analyze table to refresh columns of all partitions",
"Warning 8244 Build global-level stats failed due to missing partition-level column stats: table `t` partition `p0` column `d`, please run analyze table to refresh columns of all partitions",
))

// analyze partition with existing table-level & partition-level options and existing partition stats under dynamic
Expand All @@ -2923,7 +2923,7 @@ PARTITION BY RANGE ( a ) (
tk.MustQuery("show warnings").Sort().Check(testkit.Rows(
"Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1",
"Warning 1105 Ignore columns and options when analyze partition in dynamic mode",
"Warning 8244 Build table: `t` column: `d` global-level stats failed due to missing partition-level column stats, please run analyze table to refresh columns of all partitions",
"Warning 8244 Build global-level stats failed due to missing partition-level column stats: table `t` partition `p0` column `d`, please run analyze table to refresh columns of all partitions",
))
tk.MustQuery("select * from t where a > 1 and b > 1 and c > 1 and d > 1")
require.NoError(t, h.LoadNeededHistograms())
Expand Down Expand Up @@ -2974,8 +2974,8 @@ PARTITION BY RANGE ( a ) (
// analyze partition with index and with options are allowed under dynamic V1
tk.MustExec("analyze table t partition p0 with 1 topn, 3 buckets")
tk.MustQuery("show warnings").Sort().Check(testkit.Rows(
"Warning 8131 Build table: `t` global-level stats failed due to missing partition-level stats",
"Warning 8131 Build table: `t` index: `idx` global-level stats failed due to missing partition-level stats",
"Warning 8131 Build global-level stats failed due to missing partition-level stats: table `t` partition `p1`",
"Warning 8131 Build global-level stats failed due to missing partition-level stats: table `t` partition `p1`",
))
tk.MustExec("analyze table t partition p1 with 1 topn, 3 buckets")
tk.MustQuery("show warnings").Sort().Check(testkit.Rows())
Expand Down
24 changes: 10 additions & 14 deletions statistics/handle/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -686,20 +686,15 @@ func (h *Handle) MergePartitionStats2GlobalStatsByTableID(sc sessionctx.Context,
return h.mergePartitionStats2GlobalStats(sc, opts, is, globalTableInfo, isIndex, histIDs, tablePartitionStats)
}

func (h *Handle) loadTablePartitionStats(tableInfo *model.TableInfo, partitionID int64, isIndex int, histIDs []int64) (*statistics.Table, error) {
func (h *Handle) loadTablePartitionStats(tableInfo *model.TableInfo, partitionDef *model.PartitionDefinition) (*statistics.Table, error) {
var partitionStats *statistics.Table
partitionStats, err := h.TableStatsFromStorage(tableInfo, partitionID, true, 0)
partitionStats, err := h.TableStatsFromStorage(tableInfo, partitionDef.ID, true, 0)
if err != nil {
return nil, err
}
// if the err == nil && partitionStats == nil, it means we lack the partition-level stats which the physicalID is equal to partitionID.
if partitionStats == nil {
var errMsg string
if isIndex == 0 {
errMsg = fmt.Sprintf("`%s`", tableInfo.Name.L)
} else {
errMsg = fmt.Sprintf("`%s` index: `%s`", tableInfo.Name.L, tableInfo.FindIndexNameByID(histIDs[0]))
}
errMsg := fmt.Sprintf("table `%s` partition `%s`", tableInfo.Name.L, partitionDef.Name.L)
err = types.ErrPartitionStatsMissing.GenWithStackByArgs(errMsg)
return nil, err
}
Expand Down Expand Up @@ -750,7 +745,8 @@ func (h *Handle) mergePartitionStats2GlobalStats(sc sessionctx.Context,
allFms[i] = make([]*statistics.FMSketch, 0, partitionNum)
}

for _, partitionID := range partitionIDs {
for _, def := range globalTableInfo.Partition.Definitions {
partitionID := def.ID
h.mu.Lock()
partitionTable, ok := h.getTableByPhysicalID(is, partitionID)
h.mu.Unlock()
Expand All @@ -765,7 +761,7 @@ func (h *Handle) mergePartitionStats2GlobalStats(sc sessionctx.Context,
}
// If pre-load partition stats isn't provided, then we load partition stats directly and set it into allPartitionStats
if allPartitionStats == nil || partitionStats == nil || !ok {
partitionStats, err = h.loadTablePartitionStats(tableInfo, partitionID, isIndex, histIDs)
partitionStats, err = h.loadTablePartitionStats(tableInfo, &def)
if err != nil {
return
}
Expand All @@ -779,9 +775,9 @@ func (h *Handle) mergePartitionStats2GlobalStats(sc sessionctx.Context,
if !analyzed {
var errMsg string
if isIndex == 0 {
errMsg = fmt.Sprintf("`%s`", tableInfo.Name.L)
errMsg = fmt.Sprintf("table `%s` partition `%s` column `%s`", tableInfo.Name.L, def.Name.L, tableInfo.FindColumnNameByID(histIDs[i]))
} else {
errMsg = fmt.Sprintf("`%s` index: `%s`", tableInfo.Name.L, tableInfo.FindIndexNameByID(histIDs[0]))
errMsg = fmt.Sprintf("table `%s` partition `%s` index `%s`", tableInfo.Name.L, def.Name.L, tableInfo.FindIndexNameByID(histIDs[i]))
}
err = types.ErrPartitionStatsMissing.GenWithStackByArgs(errMsg)
return
Expand All @@ -790,9 +786,9 @@ func (h *Handle) mergePartitionStats2GlobalStats(sc sessionctx.Context,
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]))
errMsg = fmt.Sprintf("table `%s` partition `%s` column `%s`", tableInfo.Name.L, def.Name.L, tableInfo.FindColumnNameByID(histIDs[i]))
} else {
errMsg = fmt.Sprintf("`%s` index: `%s`", tableInfo.Name.L, tableInfo.FindIndexNameByID(histIDs[i]))
errMsg = fmt.Sprintf("table `%s` partition `%s` index `%s`", tableInfo.Name.L, def.Name.L, tableInfo.FindIndexNameByID(histIDs[i]))
}
err = types.ErrPartitionColumnStatsMissing.GenWithStackByArgs(errMsg)
return
Expand Down
4 changes: 2 additions & 2 deletions statistics/handle/handle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2227,8 +2227,8 @@ func TestFMSWithAnalyzePartition(t *testing.T) {
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",
"Warning 8131 Build global-level stats failed due to missing partition-level stats: table `t` partition `p1`",
"Warning 8131 Build global-level stats failed due to missing partition-level stats: table `t` partition `p1`",
))
tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("2"))
}
Expand Down

0 comments on commit c34ee82

Please sign in to comment.