From 648068c6889b5be67f461503a3691373275d9828 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BA=8C=E6=89=8B=E6=8E=89=E5=8C=85=E5=B7=A5=E7=A8=8B?= =?UTF-8?q?=E5=B8=88?= Date: Wed, 23 Aug 2023 17:20:05 +0800 Subject: [PATCH] * : remove error rate (#46340) ref pingcap/tidb#46056 --- server/testdata/optimizer_suite_out.json | 5 -- statistics/column.go | 13 ++-- statistics/handle/handle.go | 4 +- statistics/handle/handle_hist.go | 1 - statistics/handle/update.go | 93 +++++------------------- statistics/histogram.go | 32 -------- statistics/index.go | 6 +- statistics/interact_with_storage.go | 14 +--- statistics/testdata/trace_suite_out.json | 8 -- 9 files changed, 30 insertions(+), 146 deletions(-) diff --git a/server/testdata/optimizer_suite_out.json b/server/testdata/optimizer_suite_out.json index 16b0e6f88ec4a..ccf6d0a5ef637 100644 --- a/server/testdata/optimizer_suite_out.json +++ b/server/testdata/optimizer_suite_out.json @@ -159,7 +159,6 @@ "github.com/pingcap/tidb/statistics.(*Index).IsInvalid": { "CollPseudo": true, "IsInvalid": true, - "NotAccurate": true, "TotalCount": 0 } }, @@ -214,7 +213,6 @@ "github.com/pingcap/tidb/statistics.(*Index).IsInvalid": { "CollPseudo": true, "IsInvalid": true, - "NotAccurate": true, "TotalCount": 0 } }, @@ -429,7 +427,6 @@ "github.com/pingcap/tidb/statistics.(*Index).IsInvalid": { "CollPseudo": true, "IsInvalid": true, - "NotAccurate": true, "TotalCount": 0 } }, @@ -640,7 +637,6 @@ "github.com/pingcap/tidb/statistics.(*Index).IsInvalid": { "CollPseudo": true, "IsInvalid": true, - "NotAccurate": true, "TotalCount": 0 } }, @@ -698,7 +694,6 @@ "github.com/pingcap/tidb/statistics.(*Index).IsInvalid": { "CollPseudo": true, "IsInvalid": true, - "NotAccurate": true, "TotalCount": 0 } }, diff --git a/statistics/column.go b/statistics/column.go index 67a7f8a354583..efb1e3c1d2ff8 100644 --- a/statistics/column.go +++ b/statistics/column.go @@ -41,7 +41,6 @@ type Column struct { FMSketch *FMSketch Info *model.ColumnInfo Histogram - ErrorRate // StatsLoadedStatus indicates the status of column statistics StatsLoadedStatus @@ -114,9 +113,13 @@ func (c *Column) MemoryUsage() CacheItemMemoryUsage { // Currently, we only load index/pk's Histogram from kv automatically. Columns' are loaded by needs. var HistogramNeededItems = neededStatsMap{items: map[model.TableItemID]struct{}{}} -// IsInvalid checks if this column is invalid. If this column has histogram but not loaded yet, then we mark it -// as need histogram. -func (c *Column) IsInvalid(sctx sessionctx.Context, collPseudo bool) (res bool) { +// IsInvalid checks if this column is invalid. +// If this column has histogram but not loaded yet, +// then we mark it as need histogram. +func (c *Column) IsInvalid( + sctx sessionctx.Context, + collPseudo bool, +) (res bool) { var totalCount float64 var ndv int64 var inValidForCollPseudo, essentialLoaded bool @@ -133,7 +136,7 @@ func (c *Column) IsInvalid(sctx sessionctx.Context, collPseudo bool) (res bool) debugtrace.LeaveContextCommon(sctx) }() } - if collPseudo && c.NotAccurate() { + if collPseudo { inValidForCollPseudo = true return true } diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 8d594e9e8623e..5bce85c55ccb6 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -477,7 +477,7 @@ func NewHandle(ctx, initStatsCtx sessionctx.Context, lease time.Duration, pool s handle := &Handle{ gpool: gp.New(math.MaxInt16, time.Minute), ddlEventCh: make(chan *ddlUtil.Event, 1000), - listHead: &SessionStatsCollector{mapper: make(tableDeltaMap), rateMap: make(errorRateDeltaMap)}, + listHead: NewSessionStatsCollector(), idxUsageListHead: &SessionIndexUsageCollector{mapper: make(indexUsageMap)}, pool: pool, sysProcTracker: tracker, @@ -1175,7 +1175,7 @@ func (h *Handle) loadNeededIndexHistograms(reader *statistics.StatsReader, idx m return errors.Trace(fmt.Errorf("fail to get stats version for this histogram, table_id:%v, hist_id:%v", idx.TableID, idx.ID)) } idxHist := &statistics.Index{Histogram: *hg, CMSketch: cms, TopN: topN, FMSketch: fms, - Info: index.Info, ErrorRate: index.ErrorRate, StatsVer: rows[0].GetInt64(0), + Info: index.Info, StatsVer: rows[0].GetInt64(0), Flag: index.Flag, PhysicalID: idx.TableID, StatsLoadedStatus: statistics.NewStatsFullLoadStatus()} index.LastAnalyzePos.Copy(&idxHist.LastAnalyzePos) diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index 7d8c0877cf242..c901530990ca0 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -372,7 +372,6 @@ func (*Handle) readStatsForOneItem(item model.TableItemID, w *statsWrapper, read TopN: topN, FMSketch: fms, Info: index.Info, - ErrorRate: index.ErrorRate, StatsVer: statsVer, Flag: index.Flag, PhysicalID: index.PhysicalID, diff --git a/statistics/handle/update.go b/statistics/handle/update.go index 6885e3056e439..922d2aaed03eb 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -65,67 +65,6 @@ func (m tableDeltaMap) merge(deltaMap tableDeltaMap) { } } -type errorRateDelta struct { - PkErrorRate *statistics.ErrorRate - IdxErrorRate map[int64]*statistics.ErrorRate - PkID int64 -} - -type errorRateDeltaMap map[int64]errorRateDelta - -func (m errorRateDeltaMap) update(tableID int64, histID int64, rate float64, isIndex bool) { - item := m[tableID] - if isIndex { - if item.IdxErrorRate == nil { - item.IdxErrorRate = make(map[int64]*statistics.ErrorRate) - } - if item.IdxErrorRate[histID] == nil { - item.IdxErrorRate[histID] = &statistics.ErrorRate{} - } - item.IdxErrorRate[histID].Update(rate) - } else { - if item.PkErrorRate == nil { - item.PkID = histID - item.PkErrorRate = &statistics.ErrorRate{} - } - item.PkErrorRate.Update(rate) - } - m[tableID] = item -} - -func (m errorRateDeltaMap) merge(deltaMap errorRateDeltaMap) { - for tableID, item := range deltaMap { - tbl := m[tableID] - for histID, errorRate := range item.IdxErrorRate { - if tbl.IdxErrorRate == nil { - tbl.IdxErrorRate = make(map[int64]*statistics.ErrorRate) - } - if tbl.IdxErrorRate[histID] == nil { - tbl.IdxErrorRate[histID] = &statistics.ErrorRate{} - } - tbl.IdxErrorRate[histID].Merge(errorRate) - } - if item.PkErrorRate != nil { - if tbl.PkErrorRate == nil { - tbl.PkID = item.PkID - tbl.PkErrorRate = &statistics.ErrorRate{} - } - tbl.PkErrorRate.Merge(item.PkErrorRate) - } - m[tableID] = tbl - } -} - -func (m errorRateDeltaMap) clear(tableID int64, histID int64, isIndex bool) { - item := m[tableID] - if isIndex { - delete(item.IdxErrorRate, histID) - } else { - item.PkErrorRate = nil - } - m[tableID] = item -} - // colStatsUsageMap maps (tableID, columnID) to the last time when the column stats are used(needed). type colStatsUsageMap map[model.TableItemID]time.Time @@ -137,27 +76,32 @@ func (m colStatsUsageMap) merge(other colStatsUsageMap) { } } -func merge(s *SessionStatsCollector, deltaMap tableDeltaMap, rateMap errorRateDeltaMap, colMap colStatsUsageMap) { +func merge(s *SessionStatsCollector, deltaMap tableDeltaMap, colMap colStatsUsageMap) { deltaMap.merge(s.mapper) s.mapper = make(tableDeltaMap) - rateMap.merge(s.rateMap) - s.rateMap = make(errorRateDeltaMap) colMap.merge(s.colMap) s.colMap = make(colStatsUsageMap) } // SessionStatsCollector is a list item that holds the delta mapper. If you want to write or read mapper, you must lock it. type SessionStatsCollector struct { - mapper tableDeltaMap - rateMap errorRateDeltaMap - colMap colStatsUsageMap - next *SessionStatsCollector + mapper tableDeltaMap + colMap colStatsUsageMap + next *SessionStatsCollector sync.Mutex // deleted is set to true when a session is closed. Every time we sweep the list, we will remove the useless collector. deleted bool } +// NewSessionStatsCollector initializes a new SessionStatsCollector. +func NewSessionStatsCollector() *SessionStatsCollector { + return &SessionStatsCollector{ + mapper: make(tableDeltaMap), + colMap: make(colStatsUsageMap), + } +} + // Delete only sets the deleted flag true, it will be deleted from list when DumpStatsDeltaToKV is called. func (s *SessionStatsCollector) Delete() { s.Lock() @@ -177,7 +121,6 @@ func (s *SessionStatsCollector) ClearForTest() { s.Lock() defer s.Unlock() s.mapper = make(tableDeltaMap) - s.rateMap = make(errorRateDeltaMap) s.colMap = make(colStatsUsageMap) s.next = nil s.deleted = false @@ -195,10 +138,9 @@ func (h *Handle) NewSessionStatsCollector() *SessionStatsCollector { h.listHead.Lock() defer h.listHead.Unlock() newCollector := &SessionStatsCollector{ - mapper: make(tableDeltaMap), - rateMap: make(errorRateDeltaMap), - next: h.listHead.next, - colMap: make(colStatsUsageMap), + mapper: make(tableDeltaMap), + next: h.listHead.next, + colMap: make(colStatsUsageMap), } h.listHead.next = newCollector return newCollector @@ -408,14 +350,13 @@ const ( // and remove closed session's collector. func (h *Handle) sweepList() { deltaMap := make(tableDeltaMap) - errorRateMap := make(errorRateDeltaMap) colMap := make(colStatsUsageMap) prev := h.listHead prev.Lock() for curr := prev.next; curr != nil; curr = curr.next { curr.Lock() - // Merge the session stats into deltaMap, errorRateMap respectively. - merge(curr, deltaMap, errorRateMap, colMap) + // Merge the session stats into deltaMap respectively. + merge(curr, deltaMap, colMap) if curr.deleted { prev.next = curr.next // Since the session is already closed, we can safely unlock it here. diff --git a/statistics/histogram.go b/statistics/histogram.go index d690bfd555f34..fbff0f8bedd4b 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -1007,38 +1007,6 @@ func (hg *Histogram) TruncateHistogram(numBkt int) *Histogram { return hist } -// ErrorRate is the error rate of estimate row count by bucket and cm sketch. -type ErrorRate struct { - ErrorTotal float64 - QueryTotal int64 -} - -// MaxErrorRate is the max error rate of estimate row count of a not pseudo column. -// If the table is pseudo, but the average error rate is less than MaxErrorRate, -// then the column is not pseudo. -const MaxErrorRate = 0.25 - -// NotAccurate is true when the total of query is zero or the average error -// rate is greater than MaxErrorRate. -func (e *ErrorRate) NotAccurate() bool { - if e.QueryTotal == 0 { - return true - } - return e.ErrorTotal/float64(e.QueryTotal) > MaxErrorRate -} - -// Update updates the ErrorRate. -func (e *ErrorRate) Update(rate float64) { - e.QueryTotal++ - e.ErrorTotal += rate -} - -// Merge range merges two ErrorRate. -func (e *ErrorRate) Merge(rate *ErrorRate) { - e.QueryTotal += rate.QueryTotal - e.ErrorTotal += rate.ErrorTotal -} - type countByRangeFunc = func(sessionctx.Context, int64, []*ranger.Range) (float64, error) // newHistogramBySelectivity fulfills the content of new histogram by the given selectivity result. diff --git a/statistics/index.go b/statistics/index.go index 0036a4a5c26b4..40a78c62c448f 100644 --- a/statistics/index.go +++ b/statistics/index.go @@ -44,7 +44,6 @@ type Index struct { FMSketch *FMSketch Info *model.IndexInfo Histogram - ErrorRate StatsLoadedStatus StatsVer int64 // StatsVer is the version of the current stats, used to maintain compatibility Flag int64 @@ -115,7 +114,6 @@ func (idx *Index) IsInvalid(sctx sessionctx.Context, collPseudo bool) (res bool) if !collPseudo { idx.checkStats() } - var notAccurate bool var totalCount float64 if sctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { debugtrace.EnterContextCommon(sctx) @@ -123,15 +121,13 @@ func (idx *Index) IsInvalid(sctx sessionctx.Context, collPseudo bool) (res bool) debugtrace.RecordAnyValuesWithNames(sctx, "IsInvalid", res, "CollPseudo", collPseudo, - "NotAccurate", notAccurate, "TotalCount", totalCount, ) debugtrace.LeaveContextCommon(sctx) }() } - notAccurate = idx.ErrorRate.NotAccurate() totalCount = idx.TotalRowCount() - return (collPseudo && notAccurate) || totalCount == 0 + return (collPseudo) || totalCount == 0 } // EvictAllStats evicts all stats diff --git a/statistics/interact_with_storage.go b/statistics/interact_with_storage.go index 1236a77aede21..9e9f29ab427a3 100644 --- a/statistics/interact_with_storage.go +++ b/statistics/interact_with_storage.go @@ -227,12 +227,9 @@ func indexStatsFromStorage(reader *StatsReader, row chunk.Row, table *Table, tab nullCount := row.GetInt64(5) statsVer := row.GetInt64(7) idx := table.Indices[histID] - errorRate := ErrorRate{} flag := row.GetInt64(8) lastAnalyzePos := row.GetDatum(10, types.NewFieldType(mysql.TypeBlob)) - if (!IsAnalyzed(flag) || reader.IsHistory()) && idx != nil { - errorRate = idx.ErrorRate - } + for _, idxInfo := range tableInfo.Indices { if histID != idxInfo.ID { continue @@ -249,7 +246,6 @@ func indexStatsFromStorage(reader *StatsReader, row chunk.Row, table *Table, tab if notNeedLoad { idx = &Index{ Histogram: *NewHistogram(histID, distinct, nullCount, histVer, types.NewFieldType(mysql.TypeBlob), 0, 0), - ErrorRate: errorRate, StatsVer: statsVer, Info: idxInfo, Flag: flag, @@ -285,7 +281,6 @@ func indexStatsFromStorage(reader *StatsReader, row chunk.Row, table *Table, tab TopN: topN, FMSketch: fmSketch, Info: idxInfo, - ErrorRate: errorRate, StatsVer: statsVer, Flag: flag, PhysicalID: table.PhysicalID, @@ -315,11 +310,8 @@ func columnStatsFromStorage(reader *StatsReader, row chunk.Row, table *Table, ta correlation := row.GetFloat64(9) lastAnalyzePos := row.GetDatum(10, types.NewFieldType(mysql.TypeBlob)) col := table.Columns[histID] - errorRate := ErrorRate{} flag := row.GetInt64(8) - if (!IsAnalyzed(flag) || reader.IsHistory()) && col != nil { - errorRate = col.ErrorRate - } + for _, colInfo := range tableInfo.Columns { if histID != colInfo.ID { continue @@ -350,7 +342,6 @@ func columnStatsFromStorage(reader *StatsReader, row chunk.Row, table *Table, ta PhysicalID: table.PhysicalID, Histogram: *NewHistogram(histID, distinct, nullCount, histVer, &colInfo.FieldType, 0, totColSize), Info: colInfo, - ErrorRate: errorRate, IsHandle: tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.GetFlag()), Flag: flag, StatsVer: statsVer, @@ -387,7 +378,6 @@ func columnStatsFromStorage(reader *StatsReader, row chunk.Row, table *Table, ta CMSketch: cms, TopN: topN, FMSketch: fmSketch, - ErrorRate: errorRate, IsHandle: tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.GetFlag()), Flag: flag, StatsVer: statsVer, diff --git a/statistics/testdata/trace_suite_out.json b/statistics/testdata/trace_suite_out.json index ce13a469cb5a1..4a1070905ddd8 100644 --- a/statistics/testdata/trace_suite_out.json +++ b/statistics/testdata/trace_suite_out.json @@ -338,7 +338,6 @@ "github.com/pingcap/tidb/statistics.(*Index).IsInvalid": { "CollPseudo": false, "IsInvalid": false, - "NotAccurate": true, "TotalCount": 3080 } }, @@ -552,7 +551,6 @@ "github.com/pingcap/tidb/statistics.(*Index).IsInvalid": { "CollPseudo": false, "IsInvalid": false, - "NotAccurate": true, "TotalCount": 3080 } }, @@ -903,7 +901,6 @@ "github.com/pingcap/tidb/statistics.(*Index).IsInvalid": { "CollPseudo": false, "IsInvalid": false, - "NotAccurate": true, "TotalCount": 2980 } }, @@ -995,7 +992,6 @@ "github.com/pingcap/tidb/statistics.(*Index).IsInvalid": { "CollPseudo": false, "IsInvalid": false, - "NotAccurate": true, "TotalCount": 2980 } }, @@ -1386,7 +1382,6 @@ "github.com/pingcap/tidb/statistics.(*Index).IsInvalid": { "CollPseudo": false, "IsInvalid": false, - "NotAccurate": true, "TotalCount": 3080 } }, @@ -1538,7 +1533,6 @@ "github.com/pingcap/tidb/statistics.(*Index).IsInvalid": { "CollPseudo": false, "IsInvalid": false, - "NotAccurate": true, "TotalCount": 3080 } }, @@ -2395,7 +2389,6 @@ "github.com/pingcap/tidb/statistics.(*Index).IsInvalid": { "CollPseudo": false, "IsInvalid": false, - "NotAccurate": true, "TotalCount": 2980 } }, @@ -2525,7 +2518,6 @@ "github.com/pingcap/tidb/statistics.(*Index).IsInvalid": { "CollPseudo": false, "IsInvalid": false, - "NotAccurate": true, "TotalCount": 2980 } },