Skip to content

Commit

Permalink
* : remove error rate (pingcap#46340)
Browse files Browse the repository at this point in the history
  • Loading branch information
Rustin170506 authored Aug 23, 2023
1 parent 6078d99 commit 648068c
Show file tree
Hide file tree
Showing 9 changed files with 30 additions and 146 deletions.
5 changes: 0 additions & 5 deletions server/testdata/optimizer_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -159,7 +159,6 @@
"github.com/pingcap/tidb/statistics.(*Index).IsInvalid": {
"CollPseudo": true,
"IsInvalid": true,
"NotAccurate": true,
"TotalCount": 0
}
},
Expand Down Expand Up @@ -214,7 +213,6 @@
"github.com/pingcap/tidb/statistics.(*Index).IsInvalid": {
"CollPseudo": true,
"IsInvalid": true,
"NotAccurate": true,
"TotalCount": 0
}
},
Expand Down Expand Up @@ -429,7 +427,6 @@
"github.com/pingcap/tidb/statistics.(*Index).IsInvalid": {
"CollPseudo": true,
"IsInvalid": true,
"NotAccurate": true,
"TotalCount": 0
}
},
Expand Down Expand Up @@ -640,7 +637,6 @@
"github.com/pingcap/tidb/statistics.(*Index).IsInvalid": {
"CollPseudo": true,
"IsInvalid": true,
"NotAccurate": true,
"TotalCount": 0
}
},
Expand Down Expand Up @@ -698,7 +694,6 @@
"github.com/pingcap/tidb/statistics.(*Index).IsInvalid": {
"CollPseudo": true,
"IsInvalid": true,
"NotAccurate": true,
"TotalCount": 0
}
},
Expand Down
13 changes: 8 additions & 5 deletions statistics/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,6 @@ type Column struct {
FMSketch *FMSketch
Info *model.ColumnInfo
Histogram
ErrorRate

// StatsLoadedStatus indicates the status of column statistics
StatsLoadedStatus
Expand Down Expand Up @@ -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
Expand All @@ -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
}
Expand Down
4 changes: 2 additions & 2 deletions statistics/handle/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down Expand Up @@ -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)
Expand Down
1 change: 0 additions & 1 deletion statistics/handle/handle_hist.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
93 changes: 17 additions & 76 deletions statistics/handle/update.go
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand All @@ -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()
Expand All @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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.
Expand Down
32 changes: 0 additions & 32 deletions statistics/histogram.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
6 changes: 1 addition & 5 deletions statistics/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -115,23 +114,20 @@ 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)
defer func() {
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
Expand Down
14 changes: 2 additions & 12 deletions statistics/interact_with_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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,
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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,
Expand Down
Loading

0 comments on commit 648068c

Please sign in to comment.