Skip to content

Commit

Permalink
manual cherry-pick #39014
Browse files Browse the repository at this point in the history
  • Loading branch information
time-and-fate committed Nov 9, 2022
1 parent bb2953e commit 0c67ba3
Show file tree
Hide file tree
Showing 7 changed files with 42 additions and 21 deletions.
13 changes: 12 additions & 1 deletion executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -219,7 +219,18 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error {
for i := 0; i < globalStats.Num; i++ {
hg, cms, topN, fms := globalStats.Hg[i], globalStats.Cms[i], globalStats.TopN[i], globalStats.Fms[i]
// fms for global stats doesn't need to dump to kv.
err = statsHandle.SaveStatsToStorage(globalStatsID.tableID, globalStats.Count, info.isIndex, hg, cms, topN, fms, info.statsVersion, 1, false, true)
err = statsHandle.SaveStatsToStorage(globalStatsID.tableID,
globalStats.Count,
globalStats.ModifyCount,
info.isIndex,
hg,
cms,
topN,
fms,
info.statsVersion,
1,
false,
true)
if err != nil {
logutil.Logger(ctx).Error("save global-level stats to storage failed", zap.Error(err))
}
Expand Down
4 changes: 2 additions & 2 deletions statistics/handle/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,7 @@ func (h *Handle) updateGlobalStats(tblInfo *model.TableInfo) error {
for i := 0; i < newColGlobalStats.Num; i++ {
hg, cms, topN, fms := newColGlobalStats.Hg[i], newColGlobalStats.Cms[i], newColGlobalStats.TopN[i], newColGlobalStats.Fms[i]
// fms for global stats doesn't need to dump to kv.
err = h.SaveStatsToStorage(tableID, newColGlobalStats.Count, 0, hg, cms, topN, fms, 2, 1, false, false)
err = h.SaveStatsToStorage(tableID, newColGlobalStats.Count, newColGlobalStats.ModifyCount, 0, hg, cms, topN, fms, 2, 1, false, false)
if err != nil {
return err
}
Expand Down Expand Up @@ -151,7 +151,7 @@ func (h *Handle) updateGlobalStats(tblInfo *model.TableInfo) error {
for i := 0; i < newIndexGlobalStats.Num; i++ {
hg, cms, topN, fms := newIndexGlobalStats.Hg[i], newIndexGlobalStats.Cms[i], newIndexGlobalStats.TopN[i], newIndexGlobalStats.Fms[i]
// fms for global stats doesn't need to dump to kv.
err = h.SaveStatsToStorage(tableID, newIndexGlobalStats.Count, 1, hg, cms, topN, fms, 2, 1, false, false)
err = h.SaveStatsToStorage(tableID, newIndexGlobalStats.Count, newColGlobalStats.ModifyCount, 1, hg, cms, topN, fms, 2, 1, false, false)
if err != nil {
return err
}
Expand Down
8 changes: 6 additions & 2 deletions statistics/handle/dump.go
Original file line number Diff line number Diff line change
Expand Up @@ -236,14 +236,18 @@ func (h *Handle) loadStatsFromJSON(tableInfo *model.TableInfo, physicalID int64,

for _, col := range tbl.Columns {
// loadStatsFromJSON doesn't support partition table now.
err = h.SaveStatsToStorage(tbl.PhysicalID, tbl.Count, 0, &col.Histogram, col.CMSketch, col.TopN, col.FMSketch, int(col.StatsVer), 1, false, false)
// The table level Count and Modify_count would be overridden by the SaveMetaToStorage below, so we don't need
// to care about them here.
err = h.SaveStatsToStorage(tbl.PhysicalID, tbl.Count, 0, 0, &col.Histogram, col.CMSketch, col.TopN, col.FMSketch, int(col.StatsVer), 1, false, false)
if err != nil {
return errors.Trace(err)
}
}
for _, idx := range tbl.Indices {
// loadStatsFromJSON doesn't support partition table now.
err = h.SaveStatsToStorage(tbl.PhysicalID, tbl.Count, 1, &idx.Histogram, idx.CMSketch, idx.TopN, nil, int(idx.StatsVer), 1, false, false)
// The table level Count and Modify_count would be overridden by the SaveMetaToStorage below, so we don't need
// to care about them here.
err = h.SaveStatsToStorage(tbl.PhysicalID, tbl.Count, 0, 1, &idx.Histogram, idx.CMSketch, idx.TopN, nil, int(idx.StatsVer), 1, false, false)
if err != nil {
return errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion statistics/handle/dump_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -267,7 +267,7 @@ func TestDumpCMSketchWithTopN(t *testing.T) {
cms, _, _, _ := statistics.NewCMSketchAndTopN(5, 2048, fakeData, 20, 100)

stat := h.GetTableStats(tableInfo)
err = h.SaveStatsToStorage(tableInfo.ID, 1, 0, &stat.Columns[tableInfo.Columns[0].ID].Histogram, cms, nil, nil, statistics.Version2, 1, false, false)
err = h.SaveStatsToStorage(tableInfo.ID, 1, 0, 0, &stat.Columns[tableInfo.Columns[0].ID].Histogram, cms, nil, nil, statistics.Version2, 1, false, false)
require.NoError(t, err)
require.Nil(t, h.Update(is))

Expand Down
24 changes: 14 additions & 10 deletions statistics/handle/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -383,12 +383,13 @@ func (h *Handle) UpdateSessionVar() error {
// In the column statistics, the variable `num` is equal to the number of columns in the partition table.
// In the index statistics, the variable `num` is always equal to one.
type GlobalStats struct {
Num int
Count int64
Hg []*statistics.Histogram
Cms []*statistics.CMSketch
TopN []*statistics.TopN
Fms []*statistics.FMSketch
Num int
Count int64
ModifyCount int64
Hg []*statistics.Histogram
Cms []*statistics.CMSketch
TopN []*statistics.TopN
Fms []*statistics.FMSketch
}

// MergePartitionStats2GlobalStatsByTableID merge the partition-level stats to global-level stats based on the tableID.
Expand Down Expand Up @@ -472,7 +473,7 @@ func (h *Handle) mergePartitionStats2GlobalStats(sc sessionctx.Context, opts map
return
}
for i := 0; i < globalStats.Num; i++ {
count, hg, cms, topN, fms := partitionStats.GetStatsInfo(histIDs[i], isIndex == 1)
_, hg, cms, topN, fms := partitionStats.GetStatsInfo(histIDs[i], isIndex == 1)
// partition stats 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
Expand All @@ -486,7 +487,8 @@ func (h *Handle) mergePartitionStats2GlobalStats(sc sessionctx.Context, opts map
}
if i == 0 {
// In a partition, we will only update globalStats.Count once
globalStats.Count += count
globalStats.Count += partitionStats.Count
globalStats.ModifyCount += partitionStats.ModifyCount
}
allHg[i] = append(allHg[i], hg)
allCms[i] = append(allCms[i], cms)
Expand Down Expand Up @@ -1197,8 +1199,10 @@ func (h *Handle) SaveTableStatsToStorage(results *statistics.AnalyzeResults, nee
}

// SaveStatsToStorage saves the stats to storage.
// If count is negative, both count and modify count would not be used and not be written to the table. Unless, corresponding
// fields in the stats_meta table will be updated.
// TODO: refactor to reduce the number of parameters
func (h *Handle) SaveStatsToStorage(tableID int64, count int64, isIndex int, hg *statistics.Histogram, cms *statistics.CMSketch, topN *statistics.TopN, fms *statistics.FMSketch, statsVersion int, isAnalyzed int64, needDumpFMS bool, updateAnalyzeTime bool) (err error) {
func (h *Handle) SaveStatsToStorage(tableID int64, count, modifyCount int64, isIndex int, hg *statistics.Histogram, cms *statistics.CMSketch, topN *statistics.TopN, fms *statistics.FMSketch, statsVersion int, isAnalyzed int64, needDumpFMS bool, updateAnalyzeTime bool) (err error) {
statsVer := uint64(0)
defer func() {
if err == nil && statsVer != 0 {
Expand All @@ -1224,7 +1228,7 @@ func (h *Handle) SaveStatsToStorage(tableID int64, count int64, isIndex int, hg
version := txn.StartTS()
// If the count is less than 0, then we do not want to update the modify count and count.
if count >= 0 {
_, err = exec.ExecuteInternal(ctx, "replace into mysql.stats_meta (version, table_id, count) values (%?, %?, %?)", version, tableID, count)
_, err = exec.ExecuteInternal(ctx, "replace into mysql.stats_meta (version, table_id, count, modify_count) values (%?, %?, %?, %?)", version, tableID, count, modifyCount)
} else {
_, err = exec.ExecuteInternal(ctx, "update mysql.stats_meta set version = %? where table_id = %?", version, tableID)
}
Expand Down
10 changes: 6 additions & 4 deletions statistics/handle/handle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1635,6 +1635,7 @@ partition by range (a) (
partition p0 values less than (10),
partition p1 values less than (20)
)`)
require.NoError(t, dom.StatsHandle().HandleDDLEvent(<-dom.StatsHandle().DDLEventCh()))
tk.MustExec("insert into t values (1), (5), (null), (11), (15)")
require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll))

Expand Down Expand Up @@ -1670,14 +1671,15 @@ partition by range (a) (
require.NoError(t, err)
tableInfo := tbl.Meta()
globalStats := h.GetTableStats(tableInfo)
// global.count = p0.count(3) + p1.count(2) + p2.count(2)
// We did not analyze partition p1, so the value here has not changed
require.Equal(t, int64(7), globalStats.Count)
// global.count = p0.count(3) + p1.count(4) + p2.count(2)
// modify count is 2 because we didn't analyze p1 after the second insert
require.Equal(t, int64(9), globalStats.Count)
require.Equal(t, int64(2), globalStats.ModifyCount)

tk.MustExec("analyze table t partition p1;")
globalStats = h.GetTableStats(tableInfo)
// global.count = p0.count(3) + p1.count(4) + p2.count(4)
// The value of p1.Count is correct now.
// The value of modify count is 0 now.
require.Equal(t, int64(9), globalStats.Count)
require.Equal(t, int64(0), globalStats.ModifyCount)

Expand Down
2 changes: 1 addition & 1 deletion statistics/handle/update.go
Original file line number Diff line number Diff line change
Expand Up @@ -882,7 +882,7 @@ func (h *Handle) deleteOutdatedFeedback(tableID, histID, isIndex int64) error {
func (h *Handle) dumpStatsUpdateToKV(tableID, isIndex int64, q *statistics.QueryFeedback, hist *statistics.Histogram, cms *statistics.CMSketch, topN *statistics.TopN, fms *statistics.FMSketch, statsVersion int64) error {
hist = statistics.UpdateHistogram(hist, q, int(statsVersion))
// feedback for partition is not ready.
err := h.SaveStatsToStorage(tableID, -1, int(isIndex), hist, cms, topN, fms, int(statsVersion), 0, false, false)
err := h.SaveStatsToStorage(tableID, -1, 0, int(isIndex), hist, cms, topN, fms, int(statsVersion), 0, false, false)
metrics.UpdateStatsCounter.WithLabelValues(metrics.RetLabel(err)).Inc()
return errors.Trace(err)
}
Expand Down

0 comments on commit 0c67ba3

Please sign in to comment.