// Copyright 2018 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 executor_test import ( "context" "fmt" "strings" "sync" "time" . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/testkit" ) var _ = Suite(&testFastAnalyze{}) func (s *testSuite1) TestAnalyzePartition(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") createTable := `CREATE TABLE t (a int, b int, c varchar(10), primary key(a), index idx(b)) PARTITION BY RANGE ( a ) ( PARTITION p0 VALUES LESS THAN (6), PARTITION p1 VALUES LESS THAN (11), PARTITION p2 VALUES LESS THAN (16), PARTITION p3 VALUES LESS THAN (21) )` tk.MustExec(createTable) for i := 1; i < 21; i++ { tk.MustExec(fmt.Sprintf(`insert into t values (%d, %d, "hello")`, i, i)) } tk.MustExec("analyze table t") is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) pi := table.Meta().GetPartitionInfo() c.Assert(pi, NotNil) do, err := session.GetDomain(s.store) c.Assert(err, IsNil) handle := do.StatsHandle() for _, def := range pi.Definitions { statsTbl := handle.GetPartitionStats(table.Meta(), def.ID) c.Assert(statsTbl.Pseudo, IsFalse) c.Assert(len(statsTbl.Columns), Equals, 3) c.Assert(len(statsTbl.Indices), Equals, 1) for _, col := range statsTbl.Columns { c.Assert(col.Len(), Greater, 0) } for _, idx := range statsTbl.Indices { c.Assert(idx.Len(), Greater, 0) } } tk.MustExec("drop table t") tk.MustExec(createTable) for i := 1; i < 21; i++ { tk.MustExec(fmt.Sprintf(`insert into t values (%d, %d, "hello")`, i, i)) } tk.MustExec("alter table t analyze partition p0") is = infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) table, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) pi = table.Meta().GetPartitionInfo() c.Assert(pi, NotNil) for i, def := range pi.Definitions { statsTbl := handle.GetPartitionStats(table.Meta(), def.ID) if i == 0 { c.Assert(statsTbl.Pseudo, IsFalse) c.Assert(len(statsTbl.Columns), Equals, 3) c.Assert(len(statsTbl.Indices), Equals, 1) } else { c.Assert(statsTbl.Pseudo, IsTrue) } } } func (s *testSuite1) TestAnalyzeReplicaReadFollower(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") ctx := tk.Se.(sessionctx.Context) ctx.GetSessionVars().SetReplicaRead(kv.ReplicaReadFollower) tk.MustExec("analyze table t") } func (s *testSuite1) TestAnalyzeRestrict(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") ctx := tk.Se.(sessionctx.Context) ctx.GetSessionVars().InRestrictedSQL = true tk.MustExec("analyze table t") } func (s *testSuite1) TestAnalyzeParameters(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") for i := 0; i < 20; i++ { tk.MustExec(fmt.Sprintf("insert into t values (%d)", i)) } tk.MustExec("insert into t values (19), (19), (19)") tk.MustExec("set @@tidb_enable_fast_analyze = 1") tk.MustExec("analyze table t with 30 samples") is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() tbl := s.dom.StatsHandle().GetTableStats(tableInfo) col := tbl.Columns[1] c.Assert(col.Len(), Equals, 20) c.Assert(len(col.CMSketch.TopN()), Equals, 1) width, depth := col.CMSketch.GetWidthAndDepth() c.Assert(depth, Equals, int32(5)) c.Assert(width, Equals, int32(2048)) tk.MustExec("analyze table t with 4 buckets, 0 topn, 4 cmsketch width, 4 cmsketch depth") tbl = s.dom.StatsHandle().GetTableStats(tableInfo) col = tbl.Columns[1] c.Assert(col.Len(), Equals, 4) c.Assert(len(col.CMSketch.TopN()), Equals, 0) width, depth = col.CMSketch.GetWidthAndDepth() c.Assert(depth, Equals, int32(4)) c.Assert(width, Equals, int32(4)) // Test very large cmsketch tk.MustExec(fmt.Sprintf("analyze table t with %d cmsketch width, %d cmsketch depth", plannercore.CMSketchSizeLimit, 1)) tbl = s.dom.StatsHandle().GetTableStats(tableInfo) col = tbl.Columns[1] c.Assert(col.Len(), Equals, 20) c.Assert(len(col.CMSketch.TopN()), Equals, 1) width, depth = col.CMSketch.GetWidthAndDepth() c.Assert(depth, Equals, int32(1)) c.Assert(width, Equals, int32(plannercore.CMSketchSizeLimit)) // Test very large cmsketch tk.MustExec("analyze table t with 20480 cmsketch width, 50 cmsketch depth") tbl = s.dom.StatsHandle().GetTableStats(tableInfo) col = tbl.Columns[1] c.Assert(col.Len(), Equals, 20) c.Assert(len(col.CMSketch.TopN()), Equals, 1) width, depth = col.CMSketch.GetWidthAndDepth() c.Assert(depth, Equals, int32(50)) c.Assert(width, Equals, int32(20480)) } func (s *testSuite1) TestAnalyzeTooLongColumns(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a json)") value := fmt.Sprintf(`{"x":"%s"}`, strings.Repeat("x", mysql.MaxFieldVarCharLength)) tk.MustExec(fmt.Sprintf("insert into t values ('%s')", value)) tk.MustExec("analyze table t") is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() tbl := s.dom.StatsHandle().GetTableStats(tableInfo) c.Assert(tbl.Columns[1].Len(), Equals, 0) c.Assert(tbl.Columns[1].TotColSize, Equals, int64(65559)) } func (s *testFastAnalyze) TestAnalyzeFastSample(c *C) { cluster := mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(cluster) store, err := mockstore.NewMockTikvStore( mockstore.WithCluster(cluster), ) c.Assert(err, IsNil) defer store.Close() var dom *domain.Domain session.DisableStats4Test() session.SetSchemaLease(0) dom, err = session.BootstrapSession(store) c.Assert(err, IsNil) defer dom.Close() tk := testkit.NewTestKit(c, store) executor.RandSeed = 123 tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b int, index index_b(b))") tbl, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tblInfo := tbl.Meta() tid := tblInfo.ID // construct 5 regions split by {12, 24, 36, 48} splitKeys := generateTableSplitKeyForInt(tid, []int{12, 24, 36, 48}) manipulateCluster(cluster, splitKeys) for i := 0; i < 60; i++ { tk.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) } var pkCol *model.ColumnInfo var colsInfo []*model.ColumnInfo var indicesInfo []*model.IndexInfo for _, col := range tblInfo.Columns { if tblInfo.PKIsHandle && mysql.HasPriKeyFlag(col.Flag) { pkCol = col } else { colsInfo = append(colsInfo, col) } } for _, idx := range tblInfo.Indices { if idx.State == model.StatePublic { indicesInfo = append(indicesInfo, idx) } } opts := make(map[ast.AnalyzeOptionType]uint64) opts[ast.AnalyzeOptNumSamples] = 20 mockExec := &executor.AnalyzeTestFastExec{ Ctx: tk.Se.(sessionctx.Context), PKInfo: pkCol, ColsInfo: colsInfo, IdxsInfo: indicesInfo, Concurrency: 1, PhysicalTableID: tbl.(table.PhysicalTable).GetPhysicalID(), TblInfo: tblInfo, Opts: opts, } err = mockExec.TestFastSample() c.Assert(err, IsNil) c.Assert(len(mockExec.Collectors), Equals, 3) for i := 0; i < 2; i++ { samples := mockExec.Collectors[i].Samples c.Assert(len(samples), Equals, 20) for j := 1; j < 20; j++ { cmp, err := samples[j].Value.CompareDatum(tk.Se.GetSessionVars().StmtCtx, &samples[j-1].Value) c.Assert(err, IsNil) c.Assert(cmp, Greater, 0) } } } func checkHistogram(sc *stmtctx.StatementContext, hg *statistics.Histogram) (bool, error) { for i := 0; i < len(hg.Buckets); i++ { lower, upper := hg.GetLower(i), hg.GetUpper(i) cmp, err := upper.CompareDatum(sc, lower) if cmp < 0 || err != nil { return false, err } if i == 0 { continue } previousUpper := hg.GetUpper(i - 1) cmp, err = lower.CompareDatum(sc, previousUpper) if cmp <= 0 || err != nil { return false, err } } return true, nil } func (s *testFastAnalyze) TestFastAnalyze(c *C) { cluster := mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(cluster) store, err := mockstore.NewMockTikvStore( mockstore.WithCluster(cluster), ) c.Assert(err, IsNil) defer store.Close() var dom *domain.Domain session.DisableStats4Test() session.SetSchemaLease(0) dom, err = session.BootstrapSession(store) c.Assert(err, IsNil) defer dom.Close() tk := testkit.NewTestKit(c, store) executor.RandSeed = 123 tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b int, c char(10), index index_b(b))") tk.MustExec("set @@session.tidb_enable_fast_analyze=1") tk.MustExec("set @@session.tidb_build_stats_concurrency=1") // Should not panic. tk.MustExec("analyze table t") tblInfo, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tid := tblInfo.Meta().ID // construct 6 regions split by {10, 20, 30, 40, 50} splitKeys := generateTableSplitKeyForInt(tid, []int{10, 20, 30, 40, 50}) manipulateCluster(cluster, splitKeys) for i := 0; i < 20; i++ { tk.MustExec(fmt.Sprintf(`insert into t values (%d, %d, "char")`, i*3, i*3)) } tk.MustExec("analyze table t with 5 buckets, 6 samples") is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() tbl := dom.StatsHandle().GetTableStats(tableInfo) c.Assert(tbl.Count, Equals, int64(20)) for _, col := range tbl.Columns { ok, err := checkHistogram(tk.Se.GetSessionVars().StmtCtx, &col.Histogram) c.Assert(err, IsNil) c.Assert(ok, IsTrue) } for _, idx := range tbl.Indices { ok, err := checkHistogram(tk.Se.GetSessionVars().StmtCtx, &idx.Histogram) c.Assert(err, IsNil) c.Assert(ok, IsTrue) } // Test CM Sketch built from fast analyze. tk.MustExec("create table t1(a int, b int, index idx(a, b))") // Should not panic. tk.MustExec("analyze table t1") tk.MustExec("insert into t1 values (1,1),(1,1),(1,2),(1,2)") tk.MustExec("analyze table t1") tk.MustQuery("explain select a from t1 where a = 1").Check(testkit.Rows( "IndexReader_6 4.00 root index:IndexRangeScan_5", "└─IndexRangeScan_5 4.00 cop[tikv] table:t1, index:idx(a, b) range:[1,1], keep order:false")) tk.MustQuery("explain select a, b from t1 where a = 1 and b = 1").Check(testkit.Rows( "IndexReader_6 2.00 root index:IndexRangeScan_5", "└─IndexRangeScan_5 2.00 cop[tikv] table:t1, index:idx(a, b) range:[1 1,1 1], keep order:false")) tk.MustQuery("explain select a, b from t1 where a = 1 and b = 2").Check(testkit.Rows( "IndexReader_6 2.00 root index:IndexRangeScan_5", "└─IndexRangeScan_5 2.00 cop[tikv] table:t1, index:idx(a, b) range:[1 2,1 2], keep order:false")) tk.MustExec("create table t2 (a bigint unsigned, primary key(a))") tk.MustExec("insert into t2 values (0), (18446744073709551615)") tk.MustExec("analyze table t2") tk.MustQuery("show stats_buckets where table_name = 't2'").Check(testkit.Rows( "test t2 a 0 0 1 1 0 0", "test t2 a 0 1 2 1 18446744073709551615 18446744073709551615")) } func (s *testSuite1) TestIssue15993(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t0") tk.MustExec("CREATE TABLE t0(c0 INT PRIMARY KEY);") tk.MustExec("set @@tidb_enable_fast_analyze=1;") tk.MustExec("ANALYZE TABLE t0 INDEX PRIMARY;") } func (s *testSuite1) TestIssue15751(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t0") tk.MustExec("CREATE TABLE t0(c0 INT, c1 INT, PRIMARY KEY(c0, c1))") tk.MustExec("INSERT INTO t0 VALUES (0, 0)") tk.MustExec("set @@tidb_enable_fast_analyze=1") tk.MustExec("ANALYZE TABLE t0") } func (s *testSuite1) TestIssue15752(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t0") tk.MustExec("CREATE TABLE t0(c0 INT)") tk.MustExec("INSERT INTO t0 VALUES (0)") tk.MustExec("CREATE INDEX i0 ON t0(c0)") tk.MustExec("set @@tidb_enable_fast_analyze=1") tk.MustExec("ANALYZE TABLE t0 INDEX i0") } func (s *testSuite1) TestAnalyzeIncremental(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.Se.GetSessionVars().EnableStreaming = false s.testAnalyzeIncremental(tk, c) } func (s *testSuite1) TestAnalyzeIncrementalStreaming(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.Se.GetSessionVars().EnableStreaming = true s.testAnalyzeIncremental(tk, c) } func (s *testSuite1) testAnalyzeIncremental(tk *testkit.TestKit, c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int, primary key(a), index idx(b))") tk.MustExec("analyze incremental table t index") tk.MustQuery("show stats_buckets").Check(testkit.Rows()) tk.MustExec("insert into t values (1,1)") tk.MustExec("analyze incremental table t index") tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1", "test t idx 1 0 1 1 1 1")) tk.MustExec("insert into t values (2,2)") tk.MustExec("analyze incremental table t index") tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1", "test t a 0 1 2 1 2 2", "test t idx 1 0 1 1 1 1", "test t idx 1 1 2 1 2 2")) tk.MustExec("analyze incremental table t index") // Result should not change. tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1", "test t a 0 1 2 1 2 2", "test t idx 1 0 1 1 1 1", "test t idx 1 1 2 1 2 2")) // 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) tblInfo := table.Meta() tk.MustQuery("select * from t use index(idx) where b = 3") tk.MustQuery("select * from t where a > 1") h := s.dom.StatsHandle() c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.DumpStatsFeedbackToKV(), IsNil) c.Assert(h.HandleUpdateStats(is), IsNil) c.Assert(h.Update(is), IsNil) tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1", "test t a 0 1 3 0 2 2147483647", "test t idx 1 0 1 1 1 1", "test t idx 1 1 2 1 2 2")) tblStats := h.GetTableStats(tblInfo) val, err := codec.EncodeKey(tk.Se.GetSessionVars().StmtCtx, nil, types.NewIntDatum(3)) c.Assert(err, IsNil) c.Assert(tblStats.Indices[tblInfo.Indices[0].ID].CMSketch.QueryBytes(val), Equals, uint64(1)) c.Assert(statistics.IsAnalyzed(tblStats.Indices[tblInfo.Indices[0].ID].Flag), IsFalse) c.Assert(statistics.IsAnalyzed(tblStats.Columns[tblInfo.Columns[0].ID].Flag), IsFalse) tk.MustExec("analyze incremental table t index") tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1", "test t a 0 1 2 1 2 2", "test t a 0 2 3 1 3 3", "test t idx 1 0 1 1 1 1", "test t idx 1 1 2 1 2 2", "test t idx 1 2 3 1 3 3")) tblStats = h.GetTableStats(tblInfo) c.Assert(tblStats.Indices[tblInfo.Indices[0].ID].CMSketch.QueryBytes(val), Equals, uint64(1)) } type testFastAnalyze struct { } type regionProperityClient struct { tikv.Client mu struct { sync.Mutex failedOnce bool count int64 } } func (c *regionProperityClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { if req.Type == tikvrpc.CmdDebugGetRegionProperties { c.mu.Lock() defer c.mu.Unlock() c.mu.count++ // Mock failure once. if !c.mu.failedOnce { c.mu.failedOnce = true return &tikvrpc.Response{}, nil } } return c.Client.SendRequest(ctx, addr, req, timeout) } func (s *testFastAnalyze) TestFastAnalyzeRetryRowCount(c *C) { cli := ®ionProperityClient{} hijackClient := func(c tikv.Client) tikv.Client { cli.Client = c return cli } cluster := mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(cluster) mvccStore := mocktikv.MustNewMVCCStore() store, err := mockstore.NewMockTikvStore( mockstore.WithHijackClient(hijackClient), mockstore.WithCluster(cluster), mockstore.WithMVCCStore(mvccStore), ) c.Assert(err, IsNil) defer store.Close() dom, err := session.BootstrapSession(store) c.Assert(err, IsNil) defer dom.Close() tk := testkit.NewTestKit(c, store) tk.MustExec("use test") tk.MustExec("drop table if exists retry_row_count") tk.MustExec("create table retry_row_count(a int primary key)") tblInfo, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("retry_row_count")) c.Assert(err, IsNil) tid := tblInfo.Meta().ID c.Assert(dom.StatsHandle().Update(dom.InfoSchema()), IsNil) tk.MustExec("set @@session.tidb_enable_fast_analyze=1") tk.MustExec("set @@session.tidb_build_stats_concurrency=1") for i := 0; i < 30; i++ { tk.MustExec(fmt.Sprintf("insert into retry_row_count values (%d)", i)) } cluster.SplitTable(mvccStore, tid, 6) // Flush the region cache first. tk.MustQuery("select * from retry_row_count") tk.MustExec("analyze table retry_row_count") // 4 regions will be sampled, and it will retry the last failed region. c.Assert(cli.mu.count, Equals, int64(5)) row := tk.MustQuery(`show stats_meta where db_name = "test" and table_name = "retry_row_count"`).Rows()[0] c.Assert(row[5], Equals, "30") } func (s *testSuite9) TestFailedAnalyzeRequest(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b int, index index_b(b))") c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/buildStatsFromResult", `return(true)`), IsNil) _, err := tk.Exec("analyze table t") c.Assert(err.Error(), Equals, "mock buildStatsFromResult error") c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/buildStatsFromResult"), IsNil) } func (s *testSuite1) TestExtractTopN(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b int, index index_b(b))") for i := 0; i < 10; i++ { tk.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) } for i := 0; i < 10; i++ { tk.MustExec(fmt.Sprintf("insert into t values (%d, 0)", i+10)) } tk.MustExec("analyze table t") is := s.dom.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tblInfo := table.Meta() tblStats := s.dom.StatsHandle().GetTableStats(tblInfo) colStats := tblStats.Columns[tblInfo.Columns[1].ID] c.Assert(len(colStats.CMSketch.TopN()), Equals, 1) item := colStats.CMSketch.TopN()[0] c.Assert(item.Count, Equals, uint64(11)) idxStats := tblStats.Indices[tblInfo.Indices[0].ID] c.Assert(len(idxStats.CMSketch.TopN()), Equals, 1) item = idxStats.CMSketch.TopN()[0] c.Assert(item.Count, Equals, uint64(11)) } func (s *testSuite1) TestHashInTopN(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b float, c decimal(30, 10), d varchar(20))") tk.MustExec(`insert into t values (1, 1.1, 11.1, "0110"), (2, 2.2, 22.2, "0110"), (3, 3.3, 33.3, "0110"), (4, 4.4, 44.4, "0440")`) for i := 0; i < 3; i++ { tk.MustExec("insert into t select * from t") } // get stats of normal analyze tk.MustExec("analyze table t") is := s.dom.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tblInfo := tbl.Meta() tblStats1 := s.dom.StatsHandle().GetTableStats(tblInfo).Copy() // get stats of fast analyze tk.MustExec("set @@tidb_enable_fast_analyze = 1") tk.MustExec("analyze table t") tblStats2 := s.dom.StatsHandle().GetTableStats(tblInfo).Copy() // check the hash for topn for _, col := range tblInfo.Columns { topn1 := tblStats1.Columns[col.ID].CMSketch.TopNMap() cm2 := tblStats2.Columns[col.ID].CMSketch for h1, topnMetas := range topn1 { for _, topnMeta1 := range topnMetas { count2, exists := cm2.QueryTopN(h1, topnMeta1.GetH2(), topnMeta1.Data) c.Assert(exists, Equals, true) c.Assert(count2, Equals, topnMeta1.Count) } } } } func (s *testSuite1) TestDefaultValForAnalyze(c *C) { c.Skip("skip race test") tk := testkit.NewTestKit(c, s.store) tk.MustExec("drop database if exists test_default_val_for_analyze;") tk.MustExec("create database test_default_val_for_analyze;") tk.MustExec("use test_default_val_for_analyze") tk.MustExec("create table t (a int, key(a));") for i := 0; i < 2048; i++ { tk.MustExec("insert into t values (0)") } for i := 1; i < 4; i++ { tk.MustExec("insert into t values (?)", i) } tk.MustExec("analyze table t with 0 topn;") tk.MustQuery("explain select * from t where a = 1").Check(testkit.Rows("IndexReader_6 512.00 root index:IndexRangeScan_5", "└─IndexRangeScan_5 512.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false")) tk.MustQuery("explain select * from t where a = 999").Check(testkit.Rows("IndexReader_6 0.00 root index:IndexRangeScan_5", "└─IndexRangeScan_5 0.00 cop[tikv] table:t, index:a(a) range:[999,999], keep order:false")) tk.MustExec("drop table t;") tk.MustExec("create table t (a int, key(a));") for i := 0; i < 2048; i++ { tk.MustExec("insert into t values (0)") } for i := 1; i < 2049; i++ { tk.MustExec("insert into t values (?)", i) } tk.MustExec("analyze table t with 0 topn;") tk.MustQuery("explain select * from t where a = 1").Check(testkit.Rows("IndexReader_6 1.00 root index:IndexRangeScan_5", "└─IndexRangeScan_5 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false")) }