diff --git a/config/config.go b/config/config.go index b4f512da2914a..a1bfcbf0d895c 100644 --- a/config/config.go +++ b/config/config.go @@ -414,7 +414,7 @@ var defaultConf = Config{ RunAutoAnalyze: true, StmtCountLimit: 5000, FeedbackProbability: 0.05, - QueryFeedbackLimit: 1024, + QueryFeedbackLimit: 512, PseudoEstimateRatio: 0.8, ForcePriority: "NO_PRIORITY", BindInfoLease: "3s", diff --git a/config/config.toml.example b/config/config.toml.example index afdc1d27b1782..f0be6cbca3c91 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -193,7 +193,7 @@ run-auto-analyze = true feedback-probability = 0.05 # The max number of query feedback that cache in memory. -query-feedback-limit = 1024 +query-feedback-limit = 512 # Pseudo stats will be used if the ratio between the modify count and # row count in statistics of a table is greater than it. diff --git a/executor/analyze_test.go b/executor/analyze_test.go index e432dca99d72b..3d8fe6563a8ca 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -361,10 +361,13 @@ func (s *testSuite1) testAnalyzeIncremental(tk *testkit.TestKit, c *C) { // Test analyze incremental with feedback. tk.MustExec("insert into t values (3,3)") oriProbability := statistics.FeedbackProbability.Load() + oriMinLogCount := handle.MinLogScanCount defer func() { statistics.FeedbackProbability.Store(oriProbability) + handle.MinLogScanCount = oriMinLogCount }() statistics.FeedbackProbability.Store(1) + handle.MinLogScanCount = 0 is := s.dom.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) diff --git a/statistics/feedback.go b/statistics/feedback.go index 0df0d8de36ee4..13a093544b385 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -79,6 +79,64 @@ func NewQueryFeedback(physicalID int64, hist *Histogram, expected int64, desc bo } } +// QueryFeedbackKey is the key for a group of feedbacks on the same index/column. +type QueryFeedbackKey struct { + PhysicalID int64 + HistID int64 + Tp int +} + +// QueryFeedbackMap is the collection of feedbacks. +type QueryFeedbackMap struct { + Size int + Feedbacks map[QueryFeedbackKey][]*QueryFeedback +} + +// NewQueryFeedbackMap builds a feedback collection. +func NewQueryFeedbackMap() *QueryFeedbackMap { + return &QueryFeedbackMap{Feedbacks: make(map[QueryFeedbackKey][]*QueryFeedback)} +} + +// Append adds a feedback into map. +func (m *QueryFeedbackMap) Append(q *QueryFeedback) { + k := QueryFeedbackKey{ + PhysicalID: q.PhysicalID, + HistID: q.Hist.ID, + Tp: q.Tp, + } + m.append(k, []*QueryFeedback{q}) + return +} + +// MaxQueryFeedbackCount is the max number of feedbacks that are cached in memory. +var MaxQueryFeedbackCount = atomic.NewInt64(1 << 9) + +func (m *QueryFeedbackMap) append(k QueryFeedbackKey, qs []*QueryFeedback) bool { + remained := MaxQueryFeedbackCount.Load() - int64(m.Size) + if remained <= 0 { + return false + } + s, ok := m.Feedbacks[k] + if !ok || s == nil { + s = make([]*QueryFeedback, 0, 8) + } + l := mathutil.MinInt64(int64(len(qs)), remained) + s = append(s, qs[:l]...) + m.Feedbacks[k] = s + m.Size = m.Size + int(l) + return true +} + +// Merge combines 2 collections of feedbacks. +func (m *QueryFeedbackMap) Merge(r *QueryFeedbackMap) { + for k, qs := range r.Feedbacks { + if !m.append(k, qs) { + break + } + } + return +} + var ( // MaxNumberOfRanges is the max number of ranges before split to collect feedback. MaxNumberOfRanges = 20 @@ -202,7 +260,7 @@ func (q *QueryFeedback) Actual() int64 { // Update updates the query feedback. `startKey` is the start scan key of the partial result, used to find // the range for update. `counts` is the scan counts of each range, used to update the feedback count info. func (q *QueryFeedback) Update(startKey kv.Key, counts []int64) { - // Older version do not have the counts info. + // Older versions do not have the counts info. if len(counts) == 0 { q.Invalidate() return @@ -248,6 +306,43 @@ func (q *QueryFeedback) Update(startKey kv.Key, counts []int64) { return } +// NonOverlappedFeedbacks extracts a set of feedbacks which are not overlapped with each other. +func NonOverlappedFeedbacks(sc *stmtctx.StatementContext, fbs []Feedback) ([]Feedback, bool) { + // Sort feedbacks by end point and start point incrementally, then pick every feedback that is not overlapped + // with the previous chosen feedbacks. + var existsErr bool + sort.Slice(fbs, func(i, j int) bool { + res, err := fbs[i].Upper.CompareDatum(sc, fbs[j].Upper) + if err != nil { + existsErr = true + } + if existsErr || res != 0 { + return res < 0 + } + res, err = fbs[i].Lower.CompareDatum(sc, fbs[j].Lower) + if err != nil { + existsErr = true + } + return res < 0 + }) + if existsErr { + return fbs, false + } + resFBs := make([]Feedback, 0, len(fbs)) + previousEnd := &types.Datum{} + for _, fb := range fbs { + res, err := previousEnd.CompareDatum(sc, fb.Lower) + if err != nil { + return fbs, false + } + if res <= 0 { + resFBs = append(resFBs, fb) + previousEnd = fb.Upper + } + } + return resFBs, true +} + // BucketFeedback stands for all the feedback for a bucket. type BucketFeedback struct { feedback []Feedback // All the feedback info in the same bucket. @@ -482,39 +577,15 @@ func (b *BucketFeedback) mergeFullyContainedFeedback(sc *stmtctx.StatementContex if len(feedbacks) == 0 { return 0, 0, false } - // Sort feedbacks by end point and start point incrementally, then pick every feedback that is not overlapped - // with the previous chosen feedbacks. - var existsErr bool - sort.Slice(feedbacks, func(i, j int) bool { - res, err := feedbacks[i].Upper.CompareDatum(sc, feedbacks[j].Upper) - if err != nil { - existsErr = true - } - if existsErr || res != 0 { - return res < 0 - } - res, err = feedbacks[i].Lower.CompareDatum(sc, feedbacks[j].Lower) - if err != nil { - existsErr = true - } - return res < 0 - }) - if existsErr { + sortedFBs, ok := NonOverlappedFeedbacks(sc, feedbacks) + if !ok { return 0, 0, false } - previousEnd := &types.Datum{} var sumFraction, sumCount float64 - for _, fb := range feedbacks { - res, err := previousEnd.CompareDatum(sc, fb.Lower) - if err != nil { - return 0, 0, false - } - if res <= 0 { - fraction, _ := getOverlapFraction(fb, bkt) - sumFraction += fraction - sumCount += float64(fb.Count) - previousEnd = fb.Upper - } + for _, fb := range sortedFBs { + fraction, _ := getOverlapFraction(fb, bkt) + sumFraction += fraction + sumCount += float64(fb.Count) } return sumFraction, sumCount, true } diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 8d779960fc603..d6f78e62f0920 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -61,8 +61,8 @@ type Handle struct { schemaVersion int64 } - // It can be read by multiply readers at the same time without acquire lock, but it can be - // written only after acquire the lock. + // It can be read by multiple readers at the same time without acquiring lock, but it can be + // written only after acquiring the lock. statsCache struct { sync.Mutex atomic.Value @@ -78,7 +78,7 @@ type Handle struct { // globalMap contains all the delta map from collectors when we dump them to KV. globalMap tableDeltaMap // feedback is used to store query feedback info. - feedback []*statistics.QueryFeedback + feedback *statistics.QueryFeedbackMap lease atomic2.Duration } @@ -90,7 +90,7 @@ func (h *Handle) Clear() { for len(h.ddlEventCh) > 0 { <-h.ddlEventCh } - h.feedback = h.feedback[:0] + h.feedback = statistics.NewQueryFeedbackMap() h.mu.ctx.GetSessionVars().InitChunkSize = 1 h.mu.ctx.GetSessionVars().MaxChunkSize = 1 h.mu.ctx.GetSessionVars().ProjectionConcurrency = 0 @@ -100,16 +100,13 @@ func (h *Handle) Clear() { h.mu.Unlock() } -// MaxQueryFeedbackCount is the max number of feedback that cache in memory. -var MaxQueryFeedbackCount = atomic2.NewInt64(1 << 10) - // NewHandle creates a Handle for update stats. func NewHandle(ctx sessionctx.Context, lease time.Duration) *Handle { handle := &Handle{ ddlEventCh: make(chan *util.Event, 100), listHead: &SessionStatsCollector{mapper: make(tableDeltaMap), rateMap: make(errorRateDeltaMap)}, globalMap: make(tableDeltaMap), - feedback: make([]*statistics.QueryFeedback, 0, MaxQueryFeedbackCount.Load()), + feedback: statistics.NewQueryFeedbackMap(), } handle.lease.Store(lease) // It is safe to use it concurrently because the exec won't touch the ctx. @@ -132,10 +129,10 @@ func (h *Handle) SetLease(lease time.Duration) { h.lease.Store(lease) } -// GetQueryFeedback gets the query feedback. It is only use in test. -func (h *Handle) GetQueryFeedback() []*statistics.QueryFeedback { +// GetQueryFeedback gets the query feedback. It is only used in test. +func (h *Handle) GetQueryFeedback() *statistics.QueryFeedbackMap { defer func() { - h.feedback = h.feedback[:0] + h.feedback = statistics.NewQueryFeedbackMap() }() return h.feedback } diff --git a/statistics/handle/update.go b/statistics/handle/update.go index 25df50895a312..15d2e748e46b6 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -128,8 +128,8 @@ func (h *Handle) merge(s *SessionStatsCollector, rateMap errorRateDeltaMap) { s.mapper = make(tableDeltaMap) rateMap.merge(s.rateMap) s.rateMap = make(errorRateDeltaMap) - h.feedback = mergeQueryFeedback(h.feedback, s.feedback) - s.feedback = s.feedback[:0] + h.feedback.Merge(s.feedback) + s.feedback = statistics.NewQueryFeedbackMap() } // SessionStatsCollector is a list item that holds the delta mapper. If you want to write or read mapper, you must lock it. @@ -137,7 +137,7 @@ type SessionStatsCollector struct { sync.Mutex mapper tableDeltaMap - feedback []*statistics.QueryFeedback + feedback *statistics.QueryFeedbackMap rateMap errorRateDeltaMap next *SessionStatsCollector // deleted is set to true when a session is closed. Every time we sweep the list, we will remove the useless collector. @@ -158,16 +158,6 @@ func (s *SessionStatsCollector) Update(id int64, delta int64, count int64, colSi s.mapper.update(id, delta, count, colSize) } -func mergeQueryFeedback(lq []*statistics.QueryFeedback, rq []*statistics.QueryFeedback) []*statistics.QueryFeedback { - for _, q := range rq { - if len(lq) >= int(MaxQueryFeedbackCount.Load()) { - break - } - lq = append(lq, q) - } - return lq -} - var ( // MinLogScanCount is the minimum scan count for a feedback to be logged. MinLogScanCount = int64(1000) @@ -175,10 +165,9 @@ var ( MinLogErrorRate = 0.5 ) -// StoreQueryFeedback will merges the feedback into stats collector. +// StoreQueryFeedback merges the feedback into stats collector. func (s *SessionStatsCollector) StoreQueryFeedback(feedback interface{}, h *Handle) error { q := feedback.(*statistics.QueryFeedback) - // TODO: If the error rate is small or actual scan count is small, we do not need to store the feed back. if !q.Valid || q.Hist == nil { return nil } @@ -187,20 +176,19 @@ func (s *SessionStatsCollector) StoreQueryFeedback(feedback interface{}, h *Hand return errors.Trace(err) } rate := q.CalcErrorRate() - if rate >= MinLogErrorRate && (q.Actual() >= MinLogScanCount || q.Expected >= MinLogScanCount) { - metrics.SignificantFeedbackCounter.Inc() - if log.GetLevel() == zap.DebugLevel { - h.logDetailedInfo(q) - } + if !(rate >= MinLogErrorRate && (q.Actual() >= MinLogScanCount || q.Expected >= MinLogScanCount)) { + return nil } + metrics.SignificantFeedbackCounter.Inc() metrics.StatsInaccuracyRate.Observe(rate) + if log.GetLevel() == zap.DebugLevel { + h.logDetailedInfo(q) + } s.Lock() defer s.Unlock() isIndex := q.Tp == statistics.IndexType s.rateMap.update(q.PhysicalID, q.Hist.ID, rate, isIndex) - if len(s.feedback) < int(MaxQueryFeedbackCount.Load()) { - s.feedback = append(s.feedback, q) - } + s.feedback.Append(q) return nil } @@ -209,9 +197,10 @@ func (h *Handle) NewSessionStatsCollector() *SessionStatsCollector { h.listHead.Lock() defer h.listHead.Unlock() newCollector := &SessionStatsCollector{ - mapper: make(tableDeltaMap), - rateMap: make(errorRateDeltaMap), - next: h.listHead.next, + mapper: make(tableDeltaMap), + rateMap: make(errorRateDeltaMap), + next: h.listHead.next, + feedback: statistics.NewQueryFeedbackMap(), } h.listHead.next = newCollector return newCollector @@ -277,6 +266,26 @@ func (h *Handle) sweepList() { h.mu.Lock() h.mu.rateMap.merge(errorRateMap) h.mu.Unlock() + h.siftFeedbacks() +} + +// siftFeedbacks eliminates feedbacks which are overlapped with others. It is a tradeoff between +// feedback accuracy and its overhead. +func (h *Handle) siftFeedbacks() { + sc := &stmtctx.StatementContext{TimeZone: time.UTC} + for k, qs := range h.feedback.Feedbacks { + fbs := make([]statistics.Feedback, 0, len(qs)*2) + for _, q := range qs { + fbs = append(fbs, q.Feedback...) + } + if len(fbs) == 0 { + delete(h.feedback.Feedbacks, k) + continue + } + h.feedback.Feedbacks[k] = h.feedback.Feedbacks[k][:1] + h.feedback.Feedbacks[k][0].Feedback, _ = statistics.NonOverlappedFeedbacks(sc, fbs) + } + h.feedback.Size = len(h.feedback.Feedbacks) } // DumpStatsDeltaToKV sweeps the whole list and updates the global map, then we dumps every table that held in map to KV. @@ -368,22 +377,23 @@ func (h *Handle) dumpTableStatColSizeToKV(id int64, delta variable.TableDelta) e // DumpStatsFeedbackToKV dumps the stats feedback to KV. func (h *Handle) DumpStatsFeedbackToKV() error { var err error - var successCount int - for _, fb := range h.feedback { - if fb.Tp == statistics.PkType { - err = h.DumpFeedbackToKV(fb) - } else { - t, ok := h.statsCache.Load().(statsCache).tables[fb.PhysicalID] - if ok { - err = h.DumpFeedbackForIndex(fb, t) + for _, fbs := range h.feedback.Feedbacks { + for _, fb := range fbs { + if fb.Tp == statistics.PkType { + err = h.DumpFeedbackToKV(fb) + } else { + t, ok := h.statsCache.Load().(statsCache).tables[fb.PhysicalID] + if ok { + err = h.DumpFeedbackForIndex(fb, t) + } + } + if err != nil { + // For simplicity, we just drop other feedbacks in case of error. + break } } - if err != nil { - break - } - successCount++ } - h.feedback = h.feedback[successCount:] + h.feedback = statistics.NewQueryFeedbackMap() return errors.Trace(err) } @@ -417,43 +427,45 @@ func (h *Handle) DumpFeedbackToKV(fb *statistics.QueryFeedback) error { // feedback locally on this tidb-server, so it could be used more timely. func (h *Handle) UpdateStatsByLocalFeedback(is infoschema.InfoSchema) { h.sweepList() - for _, fb := range h.feedback { - h.mu.Lock() - table, ok := h.getTableByPhysicalID(is, fb.PhysicalID) - h.mu.Unlock() - if !ok { - continue - } - tblStats := h.GetPartitionStats(table.Meta(), fb.PhysicalID) - newTblStats := tblStats.Copy() - if fb.Tp == statistics.IndexType { - idx, ok := tblStats.Indices[fb.Hist.ID] - if !ok || idx.Histogram.Len() == 0 { + for _, fbs := range h.feedback.Feedbacks { + for _, fb := range fbs { + h.mu.Lock() + table, ok := h.getTableByPhysicalID(is, fb.PhysicalID) + h.mu.Unlock() + if !ok { continue } - newIdx := *idx - eqFB, ranFB := statistics.SplitFeedbackByQueryType(fb.Feedback) - newIdx.CMSketch = statistics.UpdateCMSketch(idx.CMSketch, eqFB) - newIdx.Histogram = *statistics.UpdateHistogram(&idx.Histogram, &statistics.QueryFeedback{Feedback: ranFB}) - newIdx.Histogram.PreCalculateScalar() - newIdx.Flag = statistics.ResetAnalyzeFlag(newIdx.Flag) - newTblStats.Indices[fb.Hist.ID] = &newIdx - } else { - col, ok := tblStats.Columns[fb.Hist.ID] - if !ok || col.Histogram.Len() == 0 { - continue + tblStats := h.GetPartitionStats(table.Meta(), fb.PhysicalID) + newTblStats := tblStats.Copy() + if fb.Tp == statistics.IndexType { + idx, ok := tblStats.Indices[fb.Hist.ID] + if !ok || idx.Histogram.Len() == 0 { + continue + } + newIdx := *idx + eqFB, ranFB := statistics.SplitFeedbackByQueryType(fb.Feedback) + newIdx.CMSketch = statistics.UpdateCMSketch(idx.CMSketch, eqFB) + newIdx.Histogram = *statistics.UpdateHistogram(&idx.Histogram, &statistics.QueryFeedback{Feedback: ranFB}) + newIdx.Histogram.PreCalculateScalar() + newIdx.Flag = statistics.ResetAnalyzeFlag(newIdx.Flag) + newTblStats.Indices[fb.Hist.ID] = &newIdx + } else { + col, ok := tblStats.Columns[fb.Hist.ID] + if !ok || col.Histogram.Len() == 0 { + continue + } + newCol := *col + // only use the range query to update primary key + _, ranFB := statistics.SplitFeedbackByQueryType(fb.Feedback) + newFB := &statistics.QueryFeedback{Feedback: ranFB} + newFB = newFB.DecodeIntValues() + newCol.Histogram = *statistics.UpdateHistogram(&col.Histogram, newFB) + newCol.Flag = statistics.ResetAnalyzeFlag(newCol.Flag) + newTblStats.Columns[fb.Hist.ID] = &newCol } - newCol := *col - // only use the range query to update primary key - _, ranFB := statistics.SplitFeedbackByQueryType(fb.Feedback) - newFB := &statistics.QueryFeedback{Feedback: ranFB} - newFB = newFB.DecodeIntValues() - newCol.Histogram = *statistics.UpdateHistogram(&col.Histogram, newFB) - newCol.Flag = statistics.ResetAnalyzeFlag(newCol.Flag) - newTblStats.Columns[fb.Hist.ID] = &newCol + oldCache := h.statsCache.Load().(statsCache) + h.updateStatsCache(oldCache.update([]*statistics.Table{newTblStats}, nil, oldCache.version)) } - oldCache := h.statsCache.Load().(statsCache) - h.updateStatsCache(oldCache.update([]*statistics.Table{newTblStats}, nil, oldCache.version)) } } diff --git a/statistics/handle/update_list_test.go b/statistics/handle/update_list_test.go index 3a4e7e9d22b89..d5e07b7697a74 100644 --- a/statistics/handle/update_list_test.go +++ b/statistics/handle/update_list_test.go @@ -15,6 +15,7 @@ package handle import ( . "github.com/pingcap/check" + "github.com/pingcap/tidb/statistics" ) var _ = Suite(&testUpdateListSuite{}) @@ -23,7 +24,10 @@ type testUpdateListSuite struct { } func (s *testUpdateListSuite) TestInsertAndDelete(c *C) { - h := Handle{listHead: &SessionStatsCollector{mapper: make(tableDeltaMap)}} + h := Handle{ + listHead: &SessionStatsCollector{mapper: make(tableDeltaMap)}, + feedback: statistics.NewQueryFeedbackMap(), + } var items []*SessionStatsCollector for i := 0; i < 5; i++ { items = append(items, h.NewSessionStatsCollector()) diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index fefb6be4f6249..1c86e60e5813c 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -540,12 +540,17 @@ func (s *testStatsSuite) TestUpdateErrorRate(c *C) { is := s.do.InfoSchema() h.SetLease(0) c.Assert(h.Update(is), IsNil) - oriProbability := statistics.FeedbackProbability + oriMinLogCount := handle.MinLogScanCount + oriErrorRate := handle.MinLogErrorRate defer func() { statistics.FeedbackProbability = oriProbability + handle.MinLogScanCount = oriMinLogCount + handle.MinLogErrorRate = oriErrorRate }() statistics.FeedbackProbability.Store(1) + handle.MinLogScanCount = 0 + handle.MinLogErrorRate = 0 testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") @@ -610,12 +615,17 @@ func (s *testStatsSuite) TestUpdatePartitionErrorRate(c *C) { is := s.do.InfoSchema() h.SetLease(0) c.Assert(h.Update(is), IsNil) - oriProbability := statistics.FeedbackProbability + oriMinLogCount := handle.MinLogScanCount + oriErrorRate := handle.MinLogErrorRate defer func() { statistics.FeedbackProbability = oriProbability + handle.MinLogScanCount = oriMinLogCount + handle.MinLogErrorRate = oriErrorRate }() statistics.FeedbackProbability.Store(1) + handle.MinLogScanCount = 0 + handle.MinLogErrorRate = 0 testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") @@ -729,11 +739,17 @@ func (s *testStatsSuite) TestQueryFeedback(c *C) { h := s.do.StatsHandle() oriProbability := statistics.FeedbackProbability oriNumber := statistics.MaxNumberOfRanges + oriMinLogCount := handle.MinLogScanCount + oriErrorRate := handle.MinLogErrorRate defer func() { statistics.FeedbackProbability = oriProbability statistics.MaxNumberOfRanges = oriNumber + handle.MinLogScanCount = oriMinLogCount + handle.MinLogErrorRate = oriErrorRate }() statistics.FeedbackProbability.Store(1) + handle.MinLogScanCount = 0 + handle.MinLogErrorRate = 0 tests := []struct { sql string hist string @@ -787,7 +803,7 @@ func (s *testStatsSuite) TestQueryFeedback(c *C) { testKit.MustQuery("select * from t where t.a <= 5 limit 1") c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) feedback := h.GetQueryFeedback() - c.Assert(len(feedback), Equals, 0) + c.Assert(feedback.Size, Equals, 0) // Test only collect for max number of Ranges. statistics.MaxNumberOfRanges = 0 @@ -795,7 +811,7 @@ func (s *testStatsSuite) TestQueryFeedback(c *C) { testKit.MustQuery(t.sql) c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) feedback := h.GetQueryFeedback() - c.Assert(len(feedback), Equals, 0) + c.Assert(feedback.Size, Equals, 0) } // Test collect feedback by probability. @@ -805,7 +821,7 @@ func (s *testStatsSuite) TestQueryFeedback(c *C) { testKit.MustQuery(t.sql) c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) feedback := h.GetQueryFeedback() - c.Assert(len(feedback), Equals, 0) + c.Assert(feedback.Size, Equals, 0) } // Test that after drop stats, the feedback won't cause panic. @@ -841,11 +857,18 @@ func (s *testStatsSuite) TestQueryFeedbackForPartition(c *C) { testKit.MustExec("analyze table t") oriProbability := statistics.FeedbackProbability + oriMinLogCount := handle.MinLogScanCount + oriErrorRate := handle.MinLogErrorRate defer func() { statistics.FeedbackProbability = oriProbability + handle.MinLogScanCount = oriMinLogCount + handle.MinLogErrorRate = oriErrorRate }() - h := s.do.StatsHandle() statistics.FeedbackProbability.Store(1) + handle.MinLogScanCount = 0 + handle.MinLogErrorRate = 0 + + h := s.do.StatsHandle() tests := []struct { sql string hist string @@ -919,7 +942,7 @@ func (s *testStatsSuite) TestUpdateSystemTable(c *C) { c.Assert(h.Update(s.do.InfoSchema()), IsNil) feedback := h.GetQueryFeedback() // We may have query feedback for system tables, but we do not need to store them. - c.Assert(len(feedback), Equals, 0) + c.Assert(feedback.Size, Equals, 0) } func (s *testStatsSuite) TestOutOfOrderUpdate(c *C) { @@ -962,14 +985,19 @@ func (s *testStatsSuite) TestUpdateStatsByLocalFeedback(c *C) { testKit.MustExec("analyze table t") testKit.MustExec("insert into t values (3,5)") h := s.do.StatsHandle() - oriProbability := statistics.FeedbackProbability + oriMinLogCount := handle.MinLogScanCount + oriErrorRate := handle.MinLogErrorRate oriNumber := statistics.MaxNumberOfRanges defer func() { statistics.FeedbackProbability = oriProbability + handle.MinLogScanCount = oriMinLogCount + handle.MinLogErrorRate = oriErrorRate statistics.MaxNumberOfRanges = oriNumber }() statistics.FeedbackProbability.Store(1) + handle.MinLogScanCount = 0 + handle.MinLogErrorRate = 0 is := s.do.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) @@ -1017,12 +1045,17 @@ func (s *testStatsSuite) TestUpdatePartitionStatsByLocalFeedback(c *C) { testKit.MustExec("analyze table t") testKit.MustExec("insert into t values (3,5)") h := s.do.StatsHandle() - oriProbability := statistics.FeedbackProbability + oriMinLogCount := handle.MinLogScanCount + oriErrorRate := handle.MinLogErrorRate defer func() { statistics.FeedbackProbability = oriProbability + handle.MinLogScanCount = oriMinLogCount + handle.MinLogErrorRate = oriErrorRate }() statistics.FeedbackProbability.Store(1) + handle.MinLogScanCount = 0 + handle.MinLogErrorRate = 0 is := s.do.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) @@ -1396,10 +1429,16 @@ func (s *testStatsSuite) TestIndexQueryFeedback4TopN(c *C) { testKit := testkit.NewTestKit(c, s.store) oriProbability := statistics.FeedbackProbability + oriMinLogCount := handle.MinLogScanCount + oriErrorRate := handle.MinLogErrorRate defer func() { statistics.FeedbackProbability = oriProbability + handle.MinLogScanCount = oriMinLogCount + handle.MinLogErrorRate = oriErrorRate }() statistics.FeedbackProbability.Store(1) + handle.MinLogScanCount = 0 + handle.MinLogErrorRate = 0 testKit.MustExec("use test") testKit.MustExec("create table t (a bigint(64), index idx(a))") @@ -1437,10 +1476,16 @@ func (s *testStatsSuite) TestAbnormalIndexFeedback(c *C) { testKit := testkit.NewTestKit(c, s.store) oriProbability := statistics.FeedbackProbability + oriMinLogCount := handle.MinLogScanCount + oriErrorRate := handle.MinLogErrorRate defer func() { statistics.FeedbackProbability = oriProbability + handle.MinLogScanCount = oriMinLogCount + handle.MinLogErrorRate = oriErrorRate }() statistics.FeedbackProbability.Store(1) + handle.MinLogScanCount = 0 + handle.MinLogErrorRate = 0 testKit.MustExec("use test") testKit.MustExec("create table t (a bigint(64), b bigint(64), index idx_ab(a,b))") @@ -1505,11 +1550,17 @@ func (s *testStatsSuite) TestFeedbackRanges(c *C) { h := s.do.StatsHandle() oriProbability := statistics.FeedbackProbability oriNumber := statistics.MaxNumberOfRanges + oriMinLogCount := handle.MinLogScanCount + oriErrorRate := handle.MinLogErrorRate defer func() { statistics.FeedbackProbability = oriProbability statistics.MaxNumberOfRanges = oriNumber + handle.MinLogScanCount = oriMinLogCount + handle.MinLogErrorRate = oriErrorRate }() statistics.FeedbackProbability.Store(1) + handle.MinLogScanCount = 0 + handle.MinLogErrorRate = 0 testKit.MustExec("use test") testKit.MustExec("create table t (a tinyint, b tinyint, primary key(a), index idx(a, b))") @@ -1572,13 +1623,20 @@ func (s *testStatsSuite) TestUnsignedFeedbackRanges(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) h := s.do.StatsHandle() + oriProbability := statistics.FeedbackProbability + oriMinLogCount := handle.MinLogScanCount + oriErrorRate := handle.MinLogErrorRate oriNumber := statistics.MaxNumberOfRanges defer func() { statistics.FeedbackProbability = oriProbability + handle.MinLogScanCount = oriMinLogCount + handle.MinLogErrorRate = oriErrorRate statistics.MaxNumberOfRanges = oriNumber }() statistics.FeedbackProbability.Store(1) + handle.MinLogScanCount = 0 + handle.MinLogErrorRate = 0 testKit.MustExec("use test") testKit.MustExec("create table t (a tinyint unsigned, primary key(a))") @@ -1694,11 +1752,11 @@ func (s *testStatsSuite) TestDeleteUpdateFeedback(c *C) { testKit.MustExec("delete from t where a = 1") c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(len(h.GetQueryFeedback()), Equals, 0) + c.Assert(h.GetQueryFeedback().Size, Equals, 0) testKit.MustExec("update t set a = 6 where a = 2") c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(len(h.GetQueryFeedback()), Equals, 0) + c.Assert(h.GetQueryFeedback().Size, Equals, 0) testKit.MustExec("explain analyze delete from t where a = 3") c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(len(h.GetQueryFeedback()), Equals, 0) + c.Assert(h.GetQueryFeedback().Size, Equals, 0) } diff --git a/tidb-server/main.go b/tidb-server/main.go index e7bd5a737d7aa..cb7f4b84e5769 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -44,7 +44,6 @@ import ( "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" - "github.com/pingcap/tidb/statistics/handle" kvstore "github.com/pingcap/tidb/store" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" @@ -381,7 +380,7 @@ func reloadConfig(nc, c *config.Config) { statistics.FeedbackProbability.Store(nc.Performance.FeedbackProbability) } if nc.Performance.QueryFeedbackLimit != c.Performance.QueryFeedbackLimit { - handle.MaxQueryFeedbackCount.Store(int64(nc.Performance.QueryFeedbackLimit)) + statistics.MaxQueryFeedbackCount.Store(int64(nc.Performance.QueryFeedbackLimit)) } if nc.Performance.PseudoEstimateRatio != c.Performance.PseudoEstimateRatio { statistics.RatioOfPseudoEstimate.Store(nc.Performance.PseudoEstimateRatio) @@ -494,7 +493,7 @@ func setGlobalVars() { bindinfo.Lease = parseDuration(cfg.Performance.BindInfoLease) domain.RunAutoAnalyze = cfg.Performance.RunAutoAnalyze statistics.FeedbackProbability.Store(cfg.Performance.FeedbackProbability) - handle.MaxQueryFeedbackCount.Store(int64(cfg.Performance.QueryFeedbackLimit)) + statistics.MaxQueryFeedbackCount.Store(int64(cfg.Performance.QueryFeedbackLimit)) statistics.RatioOfPseudoEstimate.Store(cfg.Performance.PseudoEstimateRatio) ddl.RunWorker = cfg.RunDDL if cfg.SplitTable {