// Copyright 2017 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. // You may obtain a copy of the License at // // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software // distributed under the License is distributed on an "AS IS" BASIS, // See the License for the specific language governing permissions and // limitations under the License. package statistics_test import ( "fmt" "strings" "time" . "github.com/pingcap/check" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" log "github.com/sirupsen/logrus" ) var _ = Suite(&testStatsSuite{}) type testStatsSuite struct { store kv.Storage do *domain.Domain hook logHook } func (s *testStatsSuite) SetUpSuite(c *C) { testleak.BeforeTest() // Add the hook here to avoid data race. log.AddHook(&s.hook) var err error s.store, s.do, err = newStoreWithBootstrap(0) c.Assert(err, IsNil) } func (s *testStatsSuite) TearDownSuite(c *C) { s.do.Close() s.store.Close() testleak.AfterTest(c)() } func (s *testStatsSuite) TestSingleSessionInsert(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("create table t1 (c1 int, c2 int)") testKit.MustExec("create table t2 (c1 int, c2 int)") rowCount1 := 10 rowCount2 := 20 for i := 0; i < rowCount1; i++ { testKit.MustExec("insert into t1 values(1, 2)") } for i := 0; i < rowCount2; i++ { testKit.MustExec("insert into t2 values(1, 2)") } is := s.do.InfoSchema() tbl1, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) c.Assert(err, IsNil) tableInfo1 := tbl1.Meta() h := s.do.StatsHandle() h.HandleDDLEvent(<-h.DDLEventCh()) h.HandleDDLEvent(<-h.DDLEventCh()) h.DumpStatsDeltaToKV(statistics.DumpAll) h.Update(is) stats1 := h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1)) tbl2, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) c.Assert(err, IsNil) tableInfo2 := tbl2.Meta() stats2 := h.GetTableStats(tableInfo2) c.Assert(stats2.Count, Equals, int64(rowCount2)) testKit.MustExec("analyze table t1") // Test update in a txn. for i := 0; i < rowCount1; i++ { testKit.MustExec("insert into t1 values(1, 2)") } h.DumpStatsDeltaToKV(statistics.DumpAll) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1*2)) // Test IncreaseFactor. count, err := stats1.ColumnEqualRowCount(testKit.Se.GetSessionVars().StmtCtx, types.NewIntDatum(1), tableInfo1.Columns[0].ID) c.Assert(err, IsNil) c.Assert(count, Equals, float64(rowCount1*2)) testKit.MustExec("begin") for i := 0; i < rowCount1; i++ { testKit.MustExec("insert into t1 values(1, 2)") } testKit.MustExec("commit") h.DumpStatsDeltaToKV(statistics.DumpAll) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1*3)) testKit.MustExec("begin") for i := 0; i < rowCount1; i++ { testKit.MustExec("insert into t1 values(1, 2)") } for i := 0; i < rowCount1; i++ { testKit.MustExec("delete from t1 limit 1") } for i := 0; i < rowCount2; i++ { testKit.MustExec("update t2 set c2 = c1") } testKit.MustExec("commit") h.DumpStatsDeltaToKV(statistics.DumpAll) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1*3)) stats2 = h.GetTableStats(tableInfo2) c.Assert(stats2.Count, Equals, int64(rowCount2)) testKit.MustExec("begin") testKit.MustExec("delete from t1") testKit.MustExec("commit") h.DumpStatsDeltaToKV(statistics.DumpAll) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(0)) rs := testKit.MustQuery("select modify_count from mysql.stats_meta") rs.Check(testkit.Rows("40", "70")) rs = testKit.MustQuery("select tot_col_size from mysql.stats_histograms") rs.Check(testkit.Rows("0", "0", "10", "10")) // test dump delta only when `modify count / count` is greater than the ratio. originValue := statistics.DumpStatsDeltaRatio statistics.DumpStatsDeltaRatio = 0.5 defer func() { statistics.DumpStatsDeltaRatio = originValue }() statistics.DumpStatsDeltaRatio = 0.5 for i := 0; i < rowCount1; i++ { testKit.MustExec("insert into t1 values (1,2)") } h.DumpStatsDeltaToKV(statistics.DumpDelta) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1)) // not dumped testKit.MustExec("insert into t1 values (1,2)") h.DumpStatsDeltaToKV(statistics.DumpDelta) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1)) h.FlushStats() h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1+1)) } func (s *testStatsSuite) TestRollback(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("create table t (a int, b int)") testKit.MustExec("begin") testKit.MustExec("insert into t values (1,2)") testKit.MustExec("rollback") is := s.do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := tbl.Meta() h := s.do.StatsHandle() h.HandleDDLEvent(<-h.DDLEventCh()) h.DumpStatsDeltaToKV(statistics.DumpAll) h.Update(is) stats := h.GetTableStats(tableInfo) c.Assert(stats.Count, Equals, int64(0)) c.Assert(stats.ModifyCount, Equals, int64(0)) } func (s *testStatsSuite) TestMultiSession(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("create table t1 (c1 int, c2 int)") rowCount1 := 10 for i := 0; i < rowCount1; i++ { testKit.MustExec("insert into t1 values(1, 2)") } testKit1 := testkit.NewTestKit(c, s.store) for i := 0; i < rowCount1; i++ { testKit1.MustExec("insert into test.t1 values(1, 2)") } testKit2 := testkit.NewTestKit(c, s.store) for i := 0; i < rowCount1; i++ { testKit2.MustExec("delete from test.t1 limit 1") } is := s.do.InfoSchema() tbl1, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) c.Assert(err, IsNil) tableInfo1 := tbl1.Meta() h := s.do.StatsHandle() h.HandleDDLEvent(<-h.DDLEventCh()) h.DumpStatsDeltaToKV(statistics.DumpAll) h.Update(is) stats1 := h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1)) for i := 0; i < rowCount1; i++ { testKit.MustExec("insert into t1 values(1, 2)") } for i := 0; i < rowCount1; i++ { testKit1.MustExec("insert into test.t1 values(1, 2)") } for i := 0; i < rowCount1; i++ { testKit2.MustExec("delete from test.t1 limit 1") } testKit.Se.Close() testKit2.Se.Close() h.DumpStatsDeltaToKV(statistics.DumpAll) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1*2)) // The session in testKit is already Closed, set it to nil will create a new session. testKit.Se = nil rs := testKit.MustQuery("select modify_count from mysql.stats_meta") rs.Check(testkit.Rows("60")) } func (s *testStatsSuite) TestTxnWithFailure(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("create table t1 (c1 int primary key, c2 int)") is := s.do.InfoSchema() tbl1, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) c.Assert(err, IsNil) tableInfo1 := tbl1.Meta() h := s.do.StatsHandle() h.HandleDDLEvent(<-h.DDLEventCh()) rowCount1 := 10 testKit.MustExec("begin") for i := 0; i < rowCount1; i++ { testKit.MustExec("insert into t1 values(?, 2)", i) } h.DumpStatsDeltaToKV(statistics.DumpAll) h.Update(is) stats1 := h.GetTableStats(tableInfo1) // have not commit c.Assert(stats1.Count, Equals, int64(0)) testKit.MustExec("commit") h.DumpStatsDeltaToKV(statistics.DumpAll) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1)) _, err = testKit.Exec("insert into t1 values(0, 2)") c.Assert(err, NotNil) h.DumpStatsDeltaToKV(statistics.DumpAll) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1)) testKit.MustExec("insert into t1 values(-1, 2)") h.DumpStatsDeltaToKV(statistics.DumpAll) h.Update(is) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1+1)) } func (s *testStatsSuite) TestUpdatePartition(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") createTable := `CREATE TABLE t (a int, b char(5)) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (6),PARTITION p1 VALUES LESS THAN (11))` testKit.MustExec(createTable) do := s.do is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := tbl.Meta() h := do.StatsHandle() err = h.HandleDDLEvent(<-h.DDLEventCh()) c.Assert(err, IsNil) pi := tableInfo.GetPartitionInfo() c.Assert(len(pi.Definitions), Equals, 2) bColID := tableInfo.Columns[1].ID testKit.MustExec(`insert into t values (1, "a"), (7, "a")`) c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) c.Assert(h.Update(is), IsNil) for _, def := range pi.Definitions { statsTbl := h.GetPartitionStats(tableInfo, def.ID) c.Assert(statsTbl.ModifyCount, Equals, int64(1)) c.Assert(statsTbl.Count, Equals, int64(1)) c.Assert(statsTbl.Columns[bColID].TotColSize, Equals, int64(1)) } testKit.MustExec(`update t set a = a + 1, b = "aa"`) c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) c.Assert(h.Update(is), IsNil) for _, def := range pi.Definitions { statsTbl := h.GetPartitionStats(tableInfo, def.ID) c.Assert(statsTbl.ModifyCount, Equals, int64(2)) c.Assert(statsTbl.Count, Equals, int64(1)) c.Assert(statsTbl.Columns[bColID].TotColSize, Equals, int64(2)) } testKit.MustExec("delete from t") c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) c.Assert(h.Update(is), IsNil) for _, def := range pi.Definitions { statsTbl := h.GetPartitionStats(tableInfo, def.ID) c.Assert(statsTbl.ModifyCount, Equals, int64(3)) c.Assert(statsTbl.Count, Equals, int64(0)) c.Assert(statsTbl.Columns[bColID].TotColSize, Equals, int64(0)) } } func (s *testStatsSuite) TestAutoUpdate(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("create table t (a varchar(20))") statistics.AutoAnalyzeMinCnt = 0 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.6") defer func() { statistics.AutoAnalyzeMinCnt = 1000 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.0") }() do := s.do is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := tbl.Meta() h := do.StatsHandle() h.HandleDDLEvent(<-h.DDLEventCh()) h.Update(is) stats := h.GetTableStats(tableInfo) c.Assert(stats.Count, Equals, int64(0)) _, err = testKit.Exec("insert into t values ('ss')") c.Assert(err, IsNil) h.DumpStatsDeltaToKV(statistics.DumpAll) h.Update(is) h.HandleAutoAnalyze(is) h.Update(is) stats = h.GetTableStats(tableInfo) c.Assert(stats.Count, Equals, int64(1)) c.Assert(stats.ModifyCount, Equals, int64(0)) for _, item := range stats.Columns { // TotColSize = 2(length of 'ss') + 1(size of len byte). c.Assert(item.TotColSize, Equals, int64(3)) break } // Test that even if the table is recently modified, we can still analyze the table. h.Lease = time.Second defer func() { h.Lease = 0 }() _, err = testKit.Exec("insert into t values ('fff')") c.Assert(err, IsNil) c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) c.Assert(h.Update(is), IsNil) h.HandleAutoAnalyze(is) h.Update(is) stats = h.GetTableStats(tableInfo) c.Assert(stats.Count, Equals, int64(2)) c.Assert(stats.ModifyCount, Equals, int64(1)) _, err = testKit.Exec("insert into t values ('fff')") c.Assert(err, IsNil) c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) c.Assert(h.Update(is), IsNil) h.HandleAutoAnalyze(is) h.Update(is) stats = h.GetTableStats(tableInfo) c.Assert(stats.Count, Equals, int64(3)) c.Assert(stats.ModifyCount, Equals, int64(0)) _, err = testKit.Exec("insert into t values ('eee')") c.Assert(err, IsNil) h.DumpStatsDeltaToKV(statistics.DumpAll) h.Update(is) h.HandleAutoAnalyze(is) h.Update(is) stats = h.GetTableStats(tableInfo) c.Assert(stats.Count, Equals, int64(4)) // Modify count is non-zero means that we do not analyze the table. c.Assert(stats.ModifyCount, Equals, int64(1)) for _, item := range stats.Columns { // TotColSize = 6, because the table has not been analyzed, and insert statement will add 3(length of 'eee') to TotColSize. c.Assert(item.TotColSize, Equals, int64(14)) break } testKit.MustExec("analyze table t") _, err = testKit.Exec("create index idx on t(a)") c.Assert(err, IsNil) is = do.InfoSchema() tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo = tbl.Meta() h.HandleAutoAnalyze(is) h.Update(is) stats = h.GetTableStats(tableInfo) c.Assert(stats.Count, Equals, int64(4)) c.Assert(stats.ModifyCount, Equals, int64(0)) hg, ok := stats.Indices[tableInfo.Indices[0].ID] c.Assert(ok, IsTrue) c.Assert(hg.NDV, Equals, int64(3)) c.Assert(hg.Len(), Equals, 3) } func (s *testStatsSuite) TestAutoUpdatePartition(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") testKit.MustExec("create table t (a int) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (6))") testKit.MustExec("analyze table t") statistics.AutoAnalyzeMinCnt = 0 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.6") defer func() { statistics.AutoAnalyzeMinCnt = 1000 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.0") }() do := s.do is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := tbl.Meta() pi := tableInfo.GetPartitionInfo() h := do.StatsHandle() h.Update(is) stats := h.GetPartitionStats(tableInfo, pi.Definitions[0].ID) c.Assert(stats.Count, Equals, int64(0)) testKit.MustExec("insert into t values (1)") h.DumpStatsDeltaToKV(statistics.DumpAll) h.Update(is) h.HandleAutoAnalyze(is) stats = h.GetPartitionStats(tableInfo, pi.Definitions[0].ID) c.Assert(stats.Count, Equals, int64(1)) c.Assert(stats.ModifyCount, Equals, int64(0)) } func (s *testStatsSuite) TestTableAnalyzed(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("create table t (a int)") testKit.MustExec("insert into t values (1)") is := s.do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := tbl.Meta() h := s.do.StatsHandle() h.Update(is) statsTbl := h.GetTableStats(tableInfo) c.Assert(statistics.TableAnalyzed(statsTbl), IsFalse) testKit.MustExec("analyze table t") h.Update(is) statsTbl = h.GetTableStats(tableInfo) c.Assert(statistics.TableAnalyzed(statsTbl), IsTrue) h.Clear() oriLease := h.Lease // set it to non-zero so we will use load by need strategy h.Lease = 1 defer func() { h.Lease = oriLease }() h.Update(is) statsTbl = h.GetTableStats(tableInfo) c.Assert(statistics.TableAnalyzed(statsTbl), IsTrue) } func (s *testStatsSuite) TestUpdateErrorRate(c *C) { defer cleanEnv(c, s.store, s.do) h := s.do.StatsHandle() is := s.do.InfoSchema() h.Lease = 0 h.Update(is) oriProbability := statistics.FeedbackProbability defer func() { statistics.FeedbackProbability = oriProbability }() statistics.FeedbackProbability = 1 testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))") h.HandleDDLEvent(<-h.DDLEventCh()) testKit.MustExec("insert into t values (1, 3)") c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) testKit.MustExec("analyze table t") testKit.MustExec("insert into t values (2, 3)") testKit.MustExec("insert into t values (5, 3)") testKit.MustExec("insert into t values (8, 3)") testKit.MustExec("insert into t values (12, 3)") c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) is = s.do.InfoSchema() h.Update(is) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tblInfo := table.Meta() tbl := h.GetTableStats(tblInfo) aID := tblInfo.Columns[0].ID bID := tblInfo.Indices[0].ID // The statistic table is outdated now. c.Assert(tbl.Columns[aID].NotAccurate(), IsTrue) testKit.MustQuery("select * from t where a between 1 and 10") c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(is), IsNil) h.UpdateErrorRate(is) h.Update(is) tbl = h.GetTableStats(tblInfo) // The error rate of this column is not larger than MaxErrorRate now. c.Assert(tbl.Columns[aID].NotAccurate(), IsFalse) c.Assert(tbl.Indices[bID].NotAccurate(), IsTrue) testKit.MustQuery("select * from t where b between 2 and 10") c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(is), IsNil) h.UpdateErrorRate(is) h.Update(is) tbl = h.GetTableStats(tblInfo) c.Assert(tbl.Indices[bID].NotAccurate(), IsFalse) c.Assert(tbl.Indices[bID].QueryTotal, Equals, int64(1)) testKit.MustExec("analyze table t") c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) h.Update(is) tbl = h.GetTableStats(tblInfo) c.Assert(tbl.Indices[bID].QueryTotal, Equals, int64(0)) } func (s *testStatsSuite) TestUpdatePartitionErrorRate(c *C) { defer cleanEnv(c, s.store, s.do) h := s.do.StatsHandle() is := s.do.InfoSchema() h.Lease = 0 h.Update(is) oriProbability := statistics.FeedbackProbability defer func() { statistics.FeedbackProbability = oriProbability }() statistics.FeedbackProbability = 1 testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("set @@session.tidb_enable_table_partition=1") testKit.MustExec("create table t (a bigint(64), primary key(a)) partition by range (a) (partition p0 values less than (30))") h.HandleDDLEvent(<-h.DDLEventCh()) testKit.MustExec("insert into t values (1)") c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) testKit.MustExec("analyze table t") testKit.MustExec("insert into t values (2)") testKit.MustExec("insert into t values (5)") testKit.MustExec("insert into t values (8)") testKit.MustExec("insert into t values (12)") c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) is = s.do.InfoSchema() h.Update(is) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tblInfo := table.Meta() pid := tblInfo.Partition.Definitions[0].ID tbl := h.GetPartitionStats(tblInfo, pid) aID := tblInfo.Columns[0].ID // The statistic table is outdated now. c.Assert(tbl.Columns[aID].NotAccurate(), IsTrue) testKit.MustQuery("select * from t where a between 1 and 10") c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(is), IsNil) h.UpdateErrorRate(is) h.Update(is) tbl = h.GetPartitionStats(tblInfo, pid) // The error rate of this column is not larger than MaxErrorRate now. c.Assert(tbl.Columns[aID].NotAccurate(), IsFalse) } func appendBucket(h *statistics.Histogram, l, r int64) { lower, upper := types.NewIntDatum(l), types.NewIntDatum(r) h.AppendBucket(&lower, &upper, 0, 0) } func (s *testStatsSuite) TestSplitRange(c *C) { h := statistics.NewHistogram(0, 0, 0, 0, types.NewFieldType(mysql.TypeLong), 5, 0) appendBucket(h, 1, 1) appendBucket(h, 2, 5) appendBucket(h, 7, 7) appendBucket(h, 8, 8) appendBucket(h, 10, 13) tests := []struct { points []int64 exclude []bool result string }{ { points: []int64{1, 1}, exclude: []bool{false, false}, result: "[1,1]", }, { points: []int64{0, 1, 3, 8, 8, 20}, exclude: []bool{true, false, true, false, true, false}, result: "(0,1],(3,5],(5,7],(7,8],(8,20]", }, { points: []int64{8, 10, 20, 30}, exclude: []bool{false, false, true, true}, result: "[8,8],(8,10],(20,30)", }, { // test remove invalid range points: []int64{8, 9}, exclude: []bool{false, true}, result: "[8,8]", }, } for _, t := range tests { ranges := make([]*ranger.Range, 0, len(t.points)/2) for i := 0; i < len(t.points); i += 2 { ranges = append(ranges, &ranger.Range{ LowVal: []types.Datum{types.NewIntDatum(t.points[i])}, LowExclude: t.exclude[i], HighVal: []types.Datum{types.NewIntDatum(t.points[i+1])}, HighExclude: t.exclude[i+1], }) } ranges = h.SplitRange(nil, ranges, false) var ranStrs []string for _, ran := range ranges { ranStrs = append(ranStrs, ran.String()) } c.Assert(strings.Join(ranStrs, ","), Equals, t.result) } } func (s *testStatsSuite) TestQueryFeedback(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))") testKit.MustExec("insert into t values (1,2),(2,2),(4,5)") testKit.MustExec("analyze table t") testKit.MustExec("insert into t values (3,4)") h := s.do.StatsHandle() oriProbability := statistics.FeedbackProbability oriNumber := statistics.MaxNumberOfRanges defer func() { statistics.FeedbackProbability = oriProbability statistics.MaxNumberOfRanges = oriNumber }() statistics.FeedbackProbability = 1 tests := []struct { sql string hist string idxCols int }{ { // test primary key feedback sql: "select * from t where t.a <= 5", hist: "column:1 ndv:4 totColSize:0\n" + "num: 1 lower_bound: -9223372036854775808 upper_bound: 1 repeats: 0\n" + "num: 1 lower_bound: 2 upper_bound: 2 repeats: 1\n" + "num: 2 lower_bound: 3 upper_bound: 5 repeats: 0", idxCols: 0, }, { // test index feedback by double read sql: "select * from t use index(idx) where t.b <= 5", hist: "index:1 ndv:2\n" + "num: 2 lower_bound: -inf upper_bound: 2 repeats: 0\n" + "num: 2 lower_bound: 3 upper_bound: 6 repeats: 0", idxCols: 1, }, { // test index feedback by single read sql: "select b from t use index(idx) where t.b <= 5", hist: "index:1 ndv:2\n" + "num: 2 lower_bound: -inf upper_bound: 2 repeats: 0\n" + "num: 2 lower_bound: 3 upper_bound: 6 repeats: 0", idxCols: 1, }, } is := s.do.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) for i, t := range tests { testKit.MustQuery(t.sql) c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) c.Assert(err, IsNil) h.Update(is) tblInfo := table.Meta() tbl := h.GetTableStats(tblInfo) if t.idxCols == 0 { c.Assert(tbl.Columns[tblInfo.Columns[0].ID].ToString(0), Equals, tests[i].hist) } else { c.Assert(tbl.Indices[tblInfo.Indices[0].ID].ToString(1), Equals, tests[i].hist) } } // Feedback from limit executor may not be accurate. testKit.MustQuery("select * from t where t.a <= 5 limit 1") h.DumpStatsDeltaToKV(statistics.DumpAll) feedback := h.GetQueryFeedback() c.Assert(len(feedback), Equals, 0) // Test only collect for max number of Ranges. statistics.MaxNumberOfRanges = 0 for _, t := range tests { testKit.MustQuery(t.sql) h.DumpStatsDeltaToKV(statistics.DumpAll) feedback := h.GetQueryFeedback() c.Assert(len(feedback), Equals, 0) } // Test collect feedback by probability. statistics.FeedbackProbability = 0 statistics.MaxNumberOfRanges = oriNumber for _, t := range tests { testKit.MustQuery(t.sql) h.DumpStatsDeltaToKV(statistics.DumpAll) feedback := h.GetQueryFeedback() c.Assert(len(feedback), Equals, 0) } // Test that after drop stats, the feedback won't cause panic. statistics.FeedbackProbability = 1 for _, t := range tests { testKit.MustQuery(t.sql) } c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) testKit.MustExec("drop stats t") c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) // Test that the outdated feedback won't cause panic. testKit.MustExec("analyze table t") for _, t := range tests { testKit.MustQuery(t.sql) } c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) testKit.MustExec("drop table t") c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) } func (s *testStatsSuite) TestQueryFeedbackForPartition(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec(`create table t (a bigint(64), b bigint(64), primary key(a), index idx(b)) partition by range (a) ( partition p0 values less than (3), partition p1 values less than (6))`) testKit.MustExec("insert into t values (1,2),(2,2),(3,4),(4,1),(5,6)") testKit.MustExec("analyze table t") oriProbability := statistics.FeedbackProbability defer func() { statistics.FeedbackProbability = oriProbability }() h := s.do.StatsHandle() statistics.FeedbackProbability = 1 tests := []struct { sql string hist string idxCols int }{ { // test primary key feedback sql: "select * from t where t.a <= 5", hist: "column:1 ndv:2 totColSize:0\n" + "num: 1 lower_bound: -9223372036854775808 upper_bound: 1 repeats: 0\n" + "num: 1 lower_bound: 2 upper_bound: 5 repeats: 0", idxCols: 0, }, { // test index feedback by double read sql: "select * from t use index(idx) where t.b <= 5", hist: "index:1 ndv:1\n" + "num: 2 lower_bound: -inf upper_bound: 6 repeats: 0", idxCols: 1, }, { // test index feedback by single read sql: "select b from t use index(idx) where t.b <= 5", hist: "index:1 ndv:1\n" + "num: 2 lower_bound: -inf upper_bound: 6 repeats: 0", idxCols: 1, }, } is := s.do.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tblInfo := table.Meta() pi := tblInfo.GetPartitionInfo() c.Assert(pi, NotNil) // This test will check the result of partition p0. var pid int64 for _, def := range pi.Definitions { if def.Name.L == "p0" { pid = def.ID break } } for i, t := range tests { testKit.MustQuery(t.sql) c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) c.Assert(err, IsNil) h.Update(is) tbl := h.GetPartitionStats(tblInfo, pid) if t.idxCols == 0 { c.Assert(tbl.Columns[tblInfo.Columns[0].ID].ToString(0), Equals, tests[i].hist) } else { c.Assert(tbl.Indices[tblInfo.Indices[0].ID].ToString(1), Equals, tests[i].hist) } } testKit.MustExec("drop table t") } func (s *testStatsSuite) TestUpdateSystemTable(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("create table t (a int, b int)") testKit.MustExec("insert into t values (1,2)") testKit.MustExec("analyze table t") testKit.MustExec("analyze table mysql.stats_histograms") h := s.do.StatsHandle() 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) } func (s *testStatsSuite) TestOutOfOrderUpdate(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("create table t (a int, b int)") testKit.MustExec("insert into t values (1,2)") do := s.do is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := tbl.Meta() h := do.StatsHandle() h.HandleDDLEvent(<-h.DDLEventCh()) // Simulate the case that another tidb has inserted some value, but delta info has not been dumped to kv yet. testKit.MustExec("insert into t values (2,2),(4,5)") c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) testKit.MustExec(fmt.Sprintf("update mysql.stats_meta set count = 1 where table_id = %d", tableInfo.ID)) testKit.MustExec("delete from t") c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) testKit.MustQuery("select count from mysql.stats_meta").Check(testkit.Rows("1")) // Now another tidb has updated the delta info. testKit.MustExec(fmt.Sprintf("update mysql.stats_meta set count = 3 where table_id = %d", tableInfo.ID)) c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) testKit.MustQuery("select count from mysql.stats_meta").Check(testkit.Rows("0")) } func (s *testStatsSuite) TestUpdateStatsByLocalFeedback(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))") testKit.MustExec("insert into t values (1,2),(2,2),(4,5)") testKit.MustExec("analyze table t") testKit.MustExec("insert into t values (3,5)") h := s.do.StatsHandle() oriProbability := statistics.FeedbackProbability oriNumber := statistics.MaxNumberOfRanges defer func() { statistics.FeedbackProbability = oriProbability statistics.MaxNumberOfRanges = oriNumber }() statistics.FeedbackProbability = 1 is := s.do.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tblInfo := table.Meta() h.GetTableStats(tblInfo) testKit.MustQuery("select * from t use index(idx) where b <= 5") testKit.MustQuery("select * from t where a > 1") testKit.MustQuery("select * from t use index(idx) where b = 5") h.UpdateStatsByLocalFeedback(s.do.InfoSchema()) tbl := h.GetTableStats(tblInfo) c.Assert(tbl.Columns[tblInfo.Columns[0].ID].ToString(0), Equals, "column:1 ndv:3 totColSize:0\n"+ "num: 1 lower_bound: 1 upper_bound: 1 repeats: 1\n"+ "num: 1 lower_bound: 2 upper_bound: 2 repeats: 1\n"+ "num: 2 lower_bound: 3 upper_bound: 9223372036854775807 repeats: 0") sc := &stmtctx.StatementContext{TimeZone: time.Local} low, err := codec.EncodeKey(sc, nil, types.NewIntDatum(5)) c.Assert(err, IsNil) c.Assert(tbl.Indices[tblInfo.Indices[0].ID].CMSketch.QueryBytes(low), Equals, uint32(2)) c.Assert(tbl.Indices[tblInfo.Indices[0].ID].ToString(1), Equals, "index:1 ndv:2\n"+ "num: 2 lower_bound: -inf upper_bound: 2 repeats: 0\n"+ "num: 2 lower_bound: 3 upper_bound: 6 repeats: 0") // Test that it won't cause panic after update. testKit.MustQuery("select * from t use index(idx) where b > 0") // Test that after drop stats, it won't cause panic. testKit.MustExec("drop stats t") h.UpdateStatsByLocalFeedback(s.do.InfoSchema()) } func (s *testStatsSuite) TestUpdatePartitionStatsByLocalFeedback(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("set @@session.tidb_enable_table_partition=1") testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a)) partition by range (a) (partition p0 values less than (6))") testKit.MustExec("insert into t values (1,2),(2,2),(4,5)") testKit.MustExec("analyze table t") testKit.MustExec("insert into t values (3,5)") h := s.do.StatsHandle() oriProbability := statistics.FeedbackProbability defer func() { statistics.FeedbackProbability = oriProbability }() statistics.FeedbackProbability = 1 is := s.do.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) testKit.MustQuery("select * from t where a > 1") h.UpdateStatsByLocalFeedback(s.do.InfoSchema()) tblInfo := table.Meta() pid := tblInfo.Partition.Definitions[0].ID tbl := h.GetPartitionStats(tblInfo, pid) c.Assert(tbl.Columns[tblInfo.Columns[0].ID].ToString(0), Equals, "column:1 ndv:3 totColSize:0\n"+ "num: 1 lower_bound: 1 upper_bound: 1 repeats: 1\n"+ "num: 1 lower_bound: 2 upper_bound: 2 repeats: 1\n"+ "num: 2 lower_bound: 3 upper_bound: 9223372036854775807 repeats: 0") } type logHook struct { results string } func (hook *logHook) Levels() []log.Level { return []log.Level{log.DebugLevel} } func (hook *logHook) Fire(entry *log.Entry) error { message := entry.Message if idx := strings.Index(message, "[stats"); idx != -1 { hook.results = hook.results + message[idx:] } return nil } func (s *testStatsSuite) TestLogDetailedInfo(c *C) { defer cleanEnv(c, s.store, s.do) oriProbability := statistics.FeedbackProbability oriMinLogCount := statistics.MinLogScanCount oriMinError := statistics.MinLogErrorRate oriLevel := log.GetLevel() oriLease := s.do.StatsHandle().Lease defer func() { statistics.FeedbackProbability = oriProbability statistics.MinLogScanCount = oriMinLogCount statistics.MinLogErrorRate = oriMinError s.do.StatsHandle().Lease = oriLease log.SetLevel(oriLevel) }() statistics.FeedbackProbability = 1 statistics.MinLogScanCount = 0 statistics.MinLogErrorRate = 0 s.do.StatsHandle().Lease = 1 testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("create table t (a bigint(64), b bigint(64), c bigint(64), primary key(a), index idx(b), index idx_ba(b,a), index idx_bc(b,c))") for i := 0; i < 20; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d, %d)", i, i, i)) } testKit.MustExec("analyze table t with 4 buckets") tests := []struct { sql string result string }{ { sql: "select * from t where t.a <= 15", result: "[stats-feedback] test.t, column: a, range: [-inf,7), actual: 8, expected: 7, buckets: {num: 8 lower_bound: 0 upper_bound: 7 repeats: 1}" + "[stats-feedback] test.t, column: a, range: [8,15), actual: 8, expected: 7, buckets: {num: 8 lower_bound: 8 upper_bound: 15 repeats: 1}", }, { sql: "select * from t use index(idx) where t.b <= 15", result: "[stats-feedback] test.t, index: idx, range: [-inf,7), actual: 8, expected: 7, histogram: {num: 8 lower_bound: 0 upper_bound: 7 repeats: 1}" + "[stats-feedback] test.t, index: idx, range: [8,15), actual: 8, expected: 7, histogram: {num: 8 lower_bound: 8 upper_bound: 15 repeats: 1}", }, { sql: "select b from t use index(idx_ba) where b = 1 and a <= 5", result: "[stats-feedback] test.t, index: idx_ba, actual: 1, equality: 1, expected equality: 1, range: [-inf,6], actual: -1, expected: 6, buckets: {num: 8 lower_bound: 0 upper_bound: 7 repeats: 1}", }, { sql: "select b from t use index(idx_bc) where b = 1 and c <= 5", result: "[stats-feedback] test.t, index: idx_bc, actual: 1, equality: 1, expected equality: 1, range: [-inf,6], pseudo count: 7", }, { sql: "select b from t use index(idx_ba) where b = 1", result: "[stats-feedback] test.t, index: idx_ba, value: 1, actual: 1, expected: 1", }, } log.SetLevel(log.DebugLevel) for _, t := range tests { s.hook.results = "" testKit.MustQuery(t.sql) c.Assert(s.hook.results, Equals, t.result) } } func (s *testStatsSuite) TestNeedAnalyzeTable(c *C) { columns := map[int64]*statistics.Column{} columns[1] = &statistics.Column{Count: 1} tests := []struct { tbl *statistics.Table ratio float64 limit time.Duration start string end string now string result bool reason string }{ // table was never analyzed and has reach the limit { tbl: &statistics.Table{Version: oracle.EncodeTSO(oracle.GetPhysical(time.Now()))}, limit: 0, ratio: 0, start: "00:00 +0800", end: "00:01 +0800", now: "00:00 +0800", result: true, reason: "table unanalyzed", }, // table was never analyzed but has not reach the limit { tbl: &statistics.Table{Version: oracle.EncodeTSO(oracle.GetPhysical(time.Now()))}, limit: time.Hour, ratio: 0, start: "00:00 +0800", end: "00:01 +0800", now: "00:00 +0800", result: false, reason: "", }, // table was already analyzed but auto analyze is disabled { tbl: &statistics.Table{HistColl: statistics.HistColl{Columns: columns, ModifyCount: 1, Count: 1}}, limit: 0, ratio: 0, start: "00:00 +0800", end: "00:01 +0800", now: "00:00 +0800", result: false, reason: "", }, // table was already analyzed and but modify count is small { tbl: &statistics.Table{HistColl: statistics.HistColl{Columns: columns, ModifyCount: 0, Count: 1}}, limit: 0, ratio: 0.3, start: "00:00 +0800", end: "00:01 +0800", now: "00:00 +0800", result: false, reason: "", }, // table was already analyzed and but not within time period { tbl: &statistics.Table{HistColl: statistics.HistColl{Columns: columns, ModifyCount: 1, Count: 1}}, limit: 0, ratio: 0.3, start: "00:00 +0800", end: "00:01 +0800", now: "00:02 +0800", result: false, reason: "", }, // table was already analyzed and but not within time period { tbl: &statistics.Table{HistColl: statistics.HistColl{Columns: columns, ModifyCount: 1, Count: 1}}, limit: 0, ratio: 0.3, start: "22:00 +0800", end: "06:00 +0800", now: "10:00 +0800", result: false, reason: "", }, // table was already analyzed and within time period { tbl: &statistics.Table{HistColl: statistics.HistColl{Columns: columns, ModifyCount: 1, Count: 1}}, limit: 0, ratio: 0.3, start: "00:00 +0800", end: "00:01 +0800", now: "00:00 +0800", result: true, reason: "too many modifications", }, // table was already analyzed and within time period { tbl: &statistics.Table{HistColl: statistics.HistColl{Columns: columns, ModifyCount: 1, Count: 1}}, limit: 0, ratio: 0.3, start: "22:00 +0800", end: "06:00 +0800", now: "23:00 +0800", result: true, reason: "too many modifications", }, } for _, test := range tests { start, err := time.ParseInLocation(variable.AnalyzeFullTimeFormat, test.start, time.UTC) c.Assert(err, IsNil) end, err := time.ParseInLocation(variable.AnalyzeFullTimeFormat, test.end, time.UTC) c.Assert(err, IsNil) now, err := time.ParseInLocation(variable.AnalyzeFullTimeFormat, test.now, time.UTC) c.Assert(err, IsNil) needAnalyze, reason := statistics.NeedAnalyzeTable(test.tbl, test.limit, test.ratio, start, end, now) c.Assert(needAnalyze, Equals, test.result) c.Assert(strings.HasPrefix(reason, test.reason), IsTrue) } } func (s *testStatsSuite) TestIndexQueryFeedback(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) oriProbability := statistics.FeedbackProbability defer func() { statistics.FeedbackProbability = oriProbability }() statistics.FeedbackProbability = 1 testKit.MustExec("use test") testKit.MustExec("create table t (a bigint(64), b bigint(64), c bigint(64), index idx_ab(a,b), index idx_ac(a,c), index idx_b(b))") for i := 0; i < 20; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (1, %d, %d)", i, i)) } h := s.do.StatsHandle() h.HandleDDLEvent(<-h.DDLEventCh()) c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) testKit.MustExec("analyze table t with 3 buckets") for i := 0; i < 20; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (1, %d, %d)", i, i)) } c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) is := s.do.InfoSchema() h.Update(is) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tblInfo := table.Meta() tests := []struct { sql string hist string idxCols int rangeID int64 idxID int64 eqCount uint32 }{ { sql: "select * from t use index(idx_ab) where a = 1 and b < 21", hist: "index:3 ndv:20\n" + "num: 16 lower_bound: -inf upper_bound: 7 repeats: 0\n" + "num: 16 lower_bound: 8 upper_bound: 15 repeats: 0\n" + "num: 8 lower_bound: 16 upper_bound: 21 repeats: 0", rangeID: tblInfo.Indices[2].ID, idxID: tblInfo.Indices[0].ID, idxCols: 1, eqCount: 39, }, { sql: "select * from t use index(idx_ac) where a = 1 and c < 21", hist: "column:3 ndv:20 totColSize:20\n" + "num: 13 lower_bound: -9223372036854775808 upper_bound: 6 repeats: 0\n" + "num: 13 lower_bound: 7 upper_bound: 13 repeats: 0\n" + "num: 12 lower_bound: 14 upper_bound: 21 repeats: 0", rangeID: tblInfo.Columns[2].ID, idxID: tblInfo.Indices[1].ID, idxCols: 0, eqCount: 35, }, } for i, t := range tests { testKit.MustQuery(t.sql) c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) h.Update(is) tbl := h.GetTableStats(tblInfo) if t.idxCols == 0 { c.Assert(tbl.Columns[t.rangeID].ToString(0), Equals, tests[i].hist) } else { c.Assert(tbl.Indices[t.rangeID].ToString(1), Equals, tests[i].hist) } val, err := codec.EncodeKey(testKit.Se.GetSessionVars().StmtCtx, nil, types.NewIntDatum(1)) c.Assert(err, IsNil) c.Assert(tbl.Indices[t.idxID].CMSketch.QueryBytes(val), Equals, t.eqCount) } } func (s *testStatsSuite) TestAbnormalIndexFeedback(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) oriProbability := statistics.FeedbackProbability defer func() { statistics.FeedbackProbability = oriProbability }() statistics.FeedbackProbability = 1 testKit.MustExec("use test") testKit.MustExec("create table t (a bigint(64), b bigint(64), index idx_ab(a,b))") for i := 0; i < 20; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i/5, i)) } testKit.MustExec("analyze table t with 3 buckets") testKit.MustExec("delete from t where a = 1") testKit.MustExec("delete from t where b > 10") is := s.do.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tblInfo := table.Meta() h := s.do.StatsHandle() tests := []struct { sql string hist string rangeID int64 idxID int64 eqCount uint32 }{ { // The real count of `a = 1` is 0. sql: "select * from t where a = 1 and b < 21", hist: "column:2 ndv:20 totColSize:20\n" + "num: 4 lower_bound: -9223372036854775808 upper_bound: 6 repeats: 0\n" + "num: 3 lower_bound: 7 upper_bound: 13 repeats: 0\n" + "num: 6 lower_bound: 14 upper_bound: 19 repeats: 1", rangeID: tblInfo.Columns[1].ID, idxID: tblInfo.Indices[0].ID, eqCount: 3, }, { // The real count of `b > 10` is 0. sql: "select * from t where a = 2 and b > 10", hist: "column:2 ndv:20 totColSize:20\n" + "num: 4 lower_bound: -9223372036854775808 upper_bound: 6 repeats: 0\n" + "num: 2 lower_bound: 7 upper_bound: 13 repeats: 0\n" + "num: 6 lower_bound: 14 upper_bound: 19 repeats: 1", rangeID: tblInfo.Columns[1].ID, idxID: tblInfo.Indices[0].ID, eqCount: 3, }, } for i, t := range tests { testKit.MustQuery(t.sql) c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) h.Update(is) tbl := h.GetTableStats(tblInfo) c.Assert(tbl.Columns[t.rangeID].ToString(0), Equals, tests[i].hist) val, err := codec.EncodeKey(testKit.Se.GetSessionVars().StmtCtx, nil, types.NewIntDatum(1)) c.Assert(err, IsNil) c.Assert(tbl.Indices[t.idxID].CMSketch.QueryBytes(val), Equals, t.eqCount) } } func (s *testStatsSuite) TestFeedbackRanges(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) h := s.do.StatsHandle() oriProbability := statistics.FeedbackProbability oriNumber := statistics.MaxNumberOfRanges defer func() { statistics.FeedbackProbability = oriProbability statistics.MaxNumberOfRanges = oriNumber }() statistics.FeedbackProbability = 1 testKit.MustExec("use test") testKit.MustExec("create table t (a tinyint, b tinyint, primary key(a), index idx(a, b))") for i := 0; i < 20; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) } h.HandleDDLEvent(<-h.DDLEventCh()) c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) testKit.MustExec("analyze table t with 3 buckets") for i := 30; i < 40; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) } c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) tests := []struct { sql string hist string colID int64 }{ { sql: "select * from t where a <= 50 or (a > 130 and a < 140)", hist: "column:1 ndv:30 totColSize:0\n" + "num: 8 lower_bound: -128 upper_bound: 7 repeats: 0\n" + "num: 8 lower_bound: 8 upper_bound: 15 repeats: 0\n" + "num: 14 lower_bound: 16 upper_bound: 50 repeats: 0", colID: 1, }, { sql: "select * from t where a >= 10", hist: "column:1 ndv:30 totColSize:0\n" + "num: 8 lower_bound: -128 upper_bound: 7 repeats: 0\n" + "num: 8 lower_bound: 8 upper_bound: 15 repeats: 0\n" + "num: 14 lower_bound: 16 upper_bound: 127 repeats: 0", colID: 1, }, { sql: "select * from t use index(idx) where a = 1 and (b <= 50 or (b > 130 and b < 140))", hist: "column:2 ndv:20 totColSize:20\n" + "num: 7 lower_bound: -128 upper_bound: 6 repeats: 0\n" + "num: 7 lower_bound: 7 upper_bound: 13 repeats: 1\n" + "num: 6 lower_bound: 14 upper_bound: 19 repeats: 1", colID: 2, }, } is := s.do.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) for i, t := range tests { testKit.MustQuery(t.sql) c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) c.Assert(err, IsNil) h.Update(is) tblInfo := table.Meta() tbl := h.GetTableStats(tblInfo) c.Assert(tbl.Columns[t.colID].ToString(0), Equals, tests[i].hist) } } 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 oriNumber := statistics.MaxNumberOfRanges defer func() { statistics.FeedbackProbability = oriProbability statistics.MaxNumberOfRanges = oriNumber }() statistics.FeedbackProbability = 1 testKit.MustExec("use test") testKit.MustExec("create table t (a tinyint unsigned, primary key(a))") for i := 0; i < 20; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d)", i)) } h.HandleDDLEvent(<-h.DDLEventCh()) c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) testKit.MustExec("analyze table t with 3 buckets") for i := 30; i < 40; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d)", i)) } c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) tests := []struct { sql string hist string }{ { sql: "select * from t where a <= 50", hist: "column:1 ndv:30 totColSize:0\n" + "num: 8 lower_bound: 0 upper_bound: 7 repeats: 0\n" + "num: 8 lower_bound: 8 upper_bound: 15 repeats: 0\n" + "num: 14 lower_bound: 16 upper_bound: 50 repeats: 0", }, { sql: "select count(*) from t", hist: "column:1 ndv:30 totColSize:0\n" + "num: 8 lower_bound: 0 upper_bound: 7 repeats: 0\n" + "num: 8 lower_bound: 8 upper_bound: 15 repeats: 0\n" + "num: 14 lower_bound: 16 upper_bound: 255 repeats: 0", }, } is := s.do.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) for i, t := range tests { testKit.MustQuery(t.sql) c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) c.Assert(err, IsNil) h.Update(is) tblInfo := table.Meta() tbl := h.GetTableStats(tblInfo) c.Assert(tbl.Columns[1].ToString(0), Equals, tests[i].hist) } }