Skip to content

Commit

Permalink
cherry pick pingcap#24382 to release-4.0
Browse files Browse the repository at this point in the history
Signed-off-by: ti-srebot <[email protected]>
  • Loading branch information
eurekaka authored and ti-srebot committed Jul 28, 2021
1 parent 4f919b0 commit 10bec34
Show file tree
Hide file tree
Showing 3 changed files with 303 additions and 2 deletions.
8 changes: 6 additions & 2 deletions statistics/handle/update.go
Original file line number Diff line number Diff line change
Expand Up @@ -685,10 +685,14 @@ func NeedAnalyzeTable(tbl *statistics.Table, limit time.Duration, autoAnalyzeRat
return false, ""
}
// No need to analyze it.
if float64(tbl.ModifyCount)/float64(tbl.Count) <= autoAnalyzeRatio {
tblCnt := float64(tbl.Count)
if histCnt := tbl.ColHistCount(); histCnt > 0 {
tblCnt = histCnt
}
if float64(tbl.ModifyCount)/tblCnt <= autoAnalyzeRatio {
return false, ""
}
return true, fmt.Sprintf("too many modifications(%v/%v>%v)", tbl.ModifyCount, tbl.Count, autoAnalyzeRatio)
return true, fmt.Sprintf("too many modifications(%v/%v>%v)", tbl.ModifyCount, tblCnt, autoAnalyzeRatio)
}

func (h *Handle) getAutoAnalyzeParameters() map[string]string {
Expand Down
274 changes: 274 additions & 0 deletions statistics/handle/update_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1817,3 +1817,277 @@ func (s *testStatsSuite) TestDeleteUpdateFeedback(c *C) {
c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil)
c.Assert(h.GetQueryFeedback().Size, Equals, 0)
}
<<<<<<< HEAD
=======

func (s *testStatsSuite) BenchmarkHandleAutoAnalyze(c *C) {
defer cleanEnv(c, s.store, s.do)
testKit := testkit.NewTestKit(c, s.store)
testKit.MustExec("use test")
h := s.do.StatsHandle()
is := s.do.InfoSchema()
for i := 0; i < c.N; i++ {
h.HandleAutoAnalyze(is)
}
}

// subtraction parses the number for counter and returns new - old.
// string for counter will be `label:<name:"type" value:"ok" > counter:<value:0 > `
func subtraction(newMetric *dto.Metric, oldMetric *dto.Metric) int {
newStr := newMetric.String()
oldStr := oldMetric.String()
newIdx := strings.LastIndex(newStr, ":")
newNum, _ := strconv.Atoi(newStr[newIdx+1 : len(newStr)-3])
oldIdx := strings.LastIndex(oldStr, ":")
oldNum, _ := strconv.Atoi(oldStr[oldIdx+1 : len(oldStr)-3])
return newNum - oldNum
}

func (s *testStatsSuite) TestDisableFeedback(c *C) {
defer cleanEnv(c, s.store, s.do)
testKit := testkit.NewTestKit(c, s.store)

oriProbability := statistics.FeedbackProbability.Load()
defer func() {
statistics.FeedbackProbability.Store(oriProbability)
}()
statistics.FeedbackProbability.Store(0.0)
oldNum := &dto.Metric{}
err := metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(oldNum)
c.Assert(err, IsNil)
testKit.MustExec("use test")
testKit.MustExec("create table t (a int, b int, index idx_a(a))")
testKit.MustExec("insert into t values (1, 1), (2, 2), (3, 3), (5, 5)")
testKit.MustExec("analyze table t with 0 topn")
for i := 0; i < 20; i++ {
testKit.MustQuery("select /*+ use_index(t, idx_a) */ * from t where a < 4")
}

newNum := &dto.Metric{}
err = metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(newNum)
c.Assert(err, IsNil)
c.Assert(subtraction(newNum, oldNum), Equals, 0)
}

func (s *testStatsSuite) TestFeedbackCounter(c *C) {
defer cleanEnv(c, s.store, s.do)
testKit := testkit.NewTestKit(c, s.store)

oriProbability := statistics.FeedbackProbability.Load()
defer func() {
statistics.FeedbackProbability.Store(oriProbability)
}()
statistics.FeedbackProbability.Store(1)
oldNum := &dto.Metric{}
err := metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(oldNum)
c.Assert(err, IsNil)
testKit.MustExec("use test")
testKit.MustExec("create table t (a int, b int, index idx_a(a))")
testKit.MustExec("insert into t values (1, 1), (2, 2), (3, 3), (5, 5)")
testKit.MustExec("analyze table t with 0 topn")
for i := 0; i < 20; i++ {
testKit.MustQuery("select /*+ use_index(t, idx_a) */ * from t where a < 4")
}

newNum := &dto.Metric{}
err = metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(newNum)
c.Assert(err, IsNil)
c.Assert(subtraction(newNum, oldNum), Equals, 20)
}

func (s *testSerialStatsSuite) TestMergeTopN(c *C) {
// Move this test to here to avoid race test.
tests := []struct {
topnNum int
n int
maxTopNVal int
maxTopNCnt int
}{
{
topnNum: 10,
n: 5,
maxTopNVal: 50,
maxTopNCnt: 100,
},
{
topnNum: 1,
n: 5,
maxTopNVal: 50,
maxTopNCnt: 100,
},
{
topnNum: 5,
n: 5,
maxTopNVal: 5,
maxTopNCnt: 100,
},
{
topnNum: 5,
n: 5,
maxTopNVal: 10,
maxTopNCnt: 100,
},
}
for _, t := range tests {
topnNum, n := t.topnNum, t.n
maxTopNVal, maxTopNCnt := t.maxTopNVal, t.maxTopNCnt

// the number of maxTopNVal should be bigger than n.
ok := maxTopNVal >= n
c.Assert(ok, Equals, true)

topNs := make([]*statistics.TopN, 0, topnNum)
res := make(map[int]uint64)
rand.Seed(time.Now().Unix())
for i := 0; i < topnNum; i++ {
topN := statistics.NewTopN(n)
occur := make(map[int]bool)
for j := 0; j < n; j++ {
// The range of numbers in the topn structure is in [0, maxTopNVal)
// But there cannot be repeated occurrences of value in a topN structure.
randNum := rand.Intn(maxTopNVal)
for occur[randNum] {
randNum = rand.Intn(maxTopNVal)
}
occur[randNum] = true
tString := []byte(fmt.Sprintf("%d", randNum))
// The range of the number of occurrences in the topn structure is in [0, maxTopNCnt)
randCnt := uint64(rand.Intn(maxTopNCnt))
res[randNum] += randCnt
topNMeta := statistics.TopNMeta{Encoded: tString, Count: randCnt}
topN.TopN = append(topN.TopN, topNMeta)
}
topNs = append(topNs, topN)
}
topN, remainTopN := statistics.MergeTopN(topNs, uint32(n))
cnt := len(topN.TopN)
var minTopNCnt uint64
for _, topNMeta := range topN.TopN {
val, err := strconv.Atoi(string(topNMeta.Encoded))
c.Assert(err, IsNil)
c.Assert(topNMeta.Count, Equals, res[val])
minTopNCnt = topNMeta.Count
}
if remainTopN != nil {
cnt += len(remainTopN)
for _, remainTopNMeta := range remainTopN {
val, err := strconv.Atoi(string(remainTopNMeta.Encoded))
c.Assert(err, IsNil)
c.Assert(remainTopNMeta.Count, Equals, res[val])
// The count of value in remainTopN may equal to the min count of value in TopN.
ok = minTopNCnt >= remainTopNMeta.Count
c.Assert(ok, Equals, true)
}
}
c.Assert(cnt, Equals, len(res))
}
}

func (s *testSerialStatsSuite) TestAutoUpdatePartitionInDynamicOnlyMode(c *C) {
defer cleanEnv(c, s.store, s.do)
testKit := testkit.NewTestKit(c, s.store)
testkit.WithPruneMode(testKit, variable.DynamicOnly, func() {
testKit.MustExec("use test")
testKit.MustExec("set @@tidb_analyze_version = 2;")
testKit.MustExec("drop table if exists t")
testKit.MustExec(`create table t (a int, b varchar(10), index idx_ab(a, b))
partition by range (a) (
partition p0 values less than (10),
partition p1 values less than (20),
partition p2 values less than (30))`)

do := s.do
is := do.InfoSchema()
h := do.StatsHandle()
c.Assert(h.RefreshVars(), IsNil)
c.Assert(h.HandleDDLEvent(<-h.DDLEventCh()), IsNil)

testKit.MustExec("insert into t values (1, 'a'), (2, 'b'), (11, 'c'), (12, 'd'), (21, 'e'), (22, 'f')")
c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil)
testKit.MustExec("set @@tidb_analyze_version = 2")
testKit.MustExec("analyze table t")

handle.AutoAnalyzeMinCnt = 0
testKit.MustExec("set global tidb_auto_analyze_ratio = 0.1")
defer func() {
handle.AutoAnalyzeMinCnt = 1000
testKit.MustExec("set global tidb_auto_analyze_ratio = 0.0")
}()

c.Assert(h.Update(is), IsNil)
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
c.Assert(err, IsNil)
tableInfo := tbl.Meta()
pi := tableInfo.GetPartitionInfo()
globalStats := h.GetTableStats(tableInfo)
partitionStats := h.GetPartitionStats(tableInfo, pi.Definitions[0].ID)
c.Assert(globalStats.Count, Equals, int64(6))
c.Assert(globalStats.ModifyCount, Equals, int64(0))
c.Assert(partitionStats.Count, Equals, int64(2))
c.Assert(partitionStats.ModifyCount, Equals, int64(0))

testKit.MustExec("insert into t values (3, 'g')")
c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil)
c.Assert(h.Update(is), IsNil)
globalStats = h.GetTableStats(tableInfo)
partitionStats = h.GetPartitionStats(tableInfo, pi.Definitions[0].ID)
c.Assert(globalStats.Count, Equals, int64(7))
c.Assert(globalStats.ModifyCount, Equals, int64(1))
c.Assert(partitionStats.Count, Equals, int64(3))
c.Assert(partitionStats.ModifyCount, Equals, int64(1))

h.HandleAutoAnalyze(is)
c.Assert(h.Update(is), IsNil)
globalStats = h.GetTableStats(tableInfo)
partitionStats = h.GetPartitionStats(tableInfo, pi.Definitions[0].ID)
c.Assert(globalStats.Count, Equals, int64(7))
c.Assert(globalStats.ModifyCount, Equals, int64(0))
c.Assert(partitionStats.Count, Equals, int64(3))
c.Assert(partitionStats.ModifyCount, Equals, int64(0))
})
}

func (s *testSerialStatsSuite) TestAutoAnalyzeRatio(c *C) {
defer cleanEnv(c, s.store, s.do)
tk := testkit.NewTestKit(c, s.store)

oriStart := tk.MustQuery("select @@tidb_auto_analyze_start_time").Rows()[0][0].(string)
oriEnd := tk.MustQuery("select @@tidb_auto_analyze_end_time").Rows()[0][0].(string)
handle.AutoAnalyzeMinCnt = 0
defer func() {
handle.AutoAnalyzeMinCnt = 1000
tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_start_time='%v'", oriStart))
tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='%v'", oriEnd))
}()

h := s.do.StatsHandle()
tk.MustExec("use test")
tk.MustExec("create table t (a int)")
c.Assert(h.HandleDDLEvent(<-h.DDLEventCh()), IsNil)
tk.MustExec("insert into t values (1)" + strings.Repeat(", (1)", 19))
c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil)
is := s.do.InfoSchema()
c.Assert(h.Update(is), IsNil)
// To pass the stats.Pseudo check in autoAnalyzeTable
tk.MustExec("analyze table t")
tk.MustExec("explain select * from t where a = 1")
c.Assert(h.LoadNeededHistograms(), IsNil)
tk.MustExec("set global tidb_auto_analyze_start_time='00:00 +0000'")
tk.MustExec("set global tidb_auto_analyze_end_time='23:59 +0000'")

tk.MustExec("insert into t values (1)" + strings.Repeat(", (1)", 10))
c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil)
c.Assert(h.Update(is), IsNil)
c.Assert(h.HandleAutoAnalyze(is), IsTrue)

tk.MustExec("delete from t limit 12")
c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil)
c.Assert(h.Update(is), IsNil)
c.Assert(h.HandleAutoAnalyze(is), IsFalse)

tk.MustExec("delete from t limit 4")
c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil)
c.Assert(h.Update(is), IsNil)
c.Assert(h.HandleAutoAnalyze(s.do.InfoSchema()), IsTrue)
}
>>>>>>> 50ae2d826... statistics: trigger auto-analyze based on histogram row count (#24382)
23 changes: 23 additions & 0 deletions statistics/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,6 +156,29 @@ func (t *Table) ColumnByName(colName string) *Column {
return nil
}

<<<<<<< HEAD
=======
// GetStatsInfo returns their statistics according to the ID of the column or index, including histogram, CMSketch, TopN and FMSketch.
func (t *Table) GetStatsInfo(ID int64, isIndex bool) (int64, *Histogram, *CMSketch, *TopN, *FMSketch) {
if isIndex {
idxStatsInfo := t.Indices[ID]
return int64(idxStatsInfo.TotalRowCount()), idxStatsInfo.Histogram.Copy(), idxStatsInfo.CMSketch.Copy(), idxStatsInfo.TopN.Copy(), idxStatsInfo.FMSketch.Copy()
}
colStatsInfo := t.Columns[ID]
return int64(colStatsInfo.TotalRowCount()), colStatsInfo.Histogram.Copy(), colStatsInfo.CMSketch.Copy(), colStatsInfo.TopN.Copy(), colStatsInfo.FMSketch.Copy()
}

// ColHistCount returns the count of the column histograms.
func (t *Table) ColHistCount() float64 {
for _, col := range t.Columns {
if col != nil {
return col.TotalRowCount()
}
}
return -1
}

>>>>>>> 50ae2d826... statistics: trigger auto-analyze based on histogram row count (#24382)
type tableColumnID struct {
TableID int64
ColumnID int64
Expand Down

0 comments on commit 10bec34

Please sign in to comment.