diff --git a/executor/analyze_serial_test.go b/executor/analyze_serial_test.go new file mode 100644 index 0000000000000..3cb412c55b65a --- /dev/null +++ b/executor/analyze_serial_test.go @@ -0,0 +1,302 @@ +// Copyright 2021 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package executor_test + +import ( + "testing" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/statistics/handle" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/collate" + "github.com/stretchr/testify/require" +) + +func TestFastAnalyze4GlobalStats(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec(`create database if not exists test_fast_gstats`) + tk.MustExec("use test_fast_gstats") + tk.MustExec("set @@session.tidb_enable_fast_analyze=1") + tk.MustExec("set @@session.tidb_build_stats_concurrency=1") + // test fast analyze in dynamic mode + tk.MustExec("set @@session.tidb_analyze_version = 2;") + tk.MustExec("set @@session.tidb_partition_prune_mode = 'dynamic';") + tk.MustExec("drop table if exists test_fast_gstats;") + tk.MustExec("create table test_fast_gstats(a int, b int) PARTITION BY HASH(a) PARTITIONS 2;") + tk.MustExec("insert into test_fast_gstats values(1,1),(3,3),(4,4),(2,2),(5,5);") + err := tk.ExecToErr("analyze table test_fast_gstats;") + require.EqualError(t, err, "Fast analyze hasn't reached General Availability and only support analyze version 1 currently.") +} + +func TestAnalyzeIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1 (id int, v int, primary key(id), index k(v))") + tk.MustExec("insert into t1(id, v) values(1, 2), (2, 2), (3, 2), (4, 2), (5, 1), (6, 3), (7, 4)") + tk.MustExec("set @@tidb_analyze_version=1") + tk.MustExec("analyze table t1 index k") + require.Greater(t, len(tk.MustQuery("show stats_buckets where table_name = 't1' and column_name = 'k' and is_index = 1").Rows()), 0) + tk.MustExec("set @@tidb_analyze_version=default") + tk.MustExec("analyze table t1") + require.Greater(t, len(tk.MustQuery("show stats_topn where table_name = 't1' and column_name = 'k' and is_index = 1").Rows()), 0) + + func() { + defer tk.MustExec("set @@session.tidb_enable_fast_analyze=0") + tk.MustExec("drop stats t1") + tk.MustExec("set @@session.tidb_enable_fast_analyze=1") + tk.MustExec("set @@tidb_analyze_version=1") + tk.MustExec("analyze table t1 index k") + require.Greater(t, len(tk.MustQuery("show stats_buckets where table_name = 't1' and column_name = 'k' and is_index = 1").Rows()), 1) + }() +} + +func TestAnalyzeIncremental(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + dom, err := session.BootstrapSession(store) + require.NoError(t, err) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_analyze_version = 1") + tk.Session().GetSessionVars().EnableStreaming = false + testAnalyzeIncremental(tk, t, dom) +} + +func TestAnalyzeIncrementalStreaming(t *testing.T) { + t.Skip("unistore hasn't support streaming yet.") + store, clean := testkit.CreateMockStore(t) + dom, err := session.BootstrapSession(store) + require.NoError(t, err) + + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.Session().GetSessionVars().EnableStreaming = true + testAnalyzeIncremental(tk, t, dom) +} + +func testAnalyzeIncremental(tk *testkit.TestKit, t *testing.T, dom *domain.Domain) { + 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 0", "test t idx 1 0 1 1 1 1 0")) + 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 0", "test t a 0 1 2 1 2 2 0", "test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0")) + 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 0", "test t a 0 1 2 1 2 2 0", "test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0")) + + // Test analyze incremental with feedback. + tk.MustExec("insert into t values (3,3)") + oriProbability := statistics.FeedbackProbability.Load() + oriMinLogCount := handle.MinLogScanCount.Load() + defer func() { + statistics.FeedbackProbability.Store(oriProbability) + handle.MinLogScanCount.Store(oriMinLogCount) + }() + statistics.FeedbackProbability.Store(1) + handle.MinLogScanCount.Store(0) + is := dom.InfoSchema() + table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo := table.Meta() + tk.MustQuery("select * from t use index(idx) where b = 3") + tk.MustQuery("select * from t where a > 1") + h := dom.StatsHandle() + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.DumpStatsFeedbackToKV()) + require.NoError(t, h.HandleUpdateStats(is)) + require.NoError(t, h.Update(is)) + require.NoError(t, h.LoadNeededHistograms()) + tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1 0", "test t a 0 1 3 0 2 2147483647 0", "test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0")) + tblStats := h.GetTableStats(tblInfo) + val, err := codec.EncodeKey(tk.Session().GetSessionVars().StmtCtx, nil, types.NewIntDatum(3)) + require.NoError(t, err) + require.Equal(t, uint64(1), tblStats.Indices[tblInfo.Indices[0].ID].QueryBytes(val)) + require.False(t, statistics.IsAnalyzed(tblStats.Indices[tblInfo.Indices[0].ID].Flag)) + require.False(t, statistics.IsAnalyzed(tblStats.Columns[tblInfo.Columns[0].ID].Flag)) + + tk.MustExec("analyze incremental table t index") + require.NoError(t, h.LoadNeededHistograms()) + tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1 0", "test t a 0 1 2 1 2 2 0", "test t a 0 2 3 1 3 3 0", + "test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0", "test t idx 1 2 3 1 3 3 0")) + tblStats = h.GetTableStats(tblInfo) + require.Equal(t, uint64(1), tblStats.Indices[tblInfo.Indices[0].ID].QueryBytes(val)) + + // test analyzeIndexIncremental for global-level stats; + tk.MustExec("set @@session.tidb_analyze_version = 1;") + tk.MustQuery("select @@tidb_analyze_version").Check(testkit.Rows("1")) + tk.MustExec("set @@tidb_partition_prune_mode = 'static';") + tk.MustExec("drop table if exists t;") + tk.MustExec(`create table t (a int, b int, primary key(a), index idx(b)) partition by range (a) ( + partition p0 values less than (10), + partition p1 values less than (20), + partition p2 values less than (30) + );`) + tk.MustExec("analyze incremental table t index") + require.NoError(t, h.LoadNeededHistograms()) + 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 warnings").Check(testkit.Rows()) // no warning + require.NoError(t, h.LoadNeededHistograms()) + tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t p0 a 0 0 1 1 1 1 0", "test t p0 idx 1 0 1 1 1 1 0")) + tk.MustExec("insert into t values (2,2)") + tk.MustExec("analyze incremental table t index") + require.NoError(t, h.LoadNeededHistograms()) + tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t p0 a 0 0 1 1 1 1 0", "test t p0 a 0 1 2 1 2 2 0", "test t p0 idx 1 0 1 1 1 1 0", "test t p0 idx 1 1 2 1 2 2 0")) + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic';") + tk.MustExec("insert into t values (11,11)") + err = tk.ExecToErr("analyze incremental table t index") + require.Equal(t, "[stats]: global statistics for partitioned tables unavailable in ANALYZE INCREMENTAL", err.Error()) +} + +func TestIssue27429(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table test.t(id int, value varchar(20) charset utf8mb4 collate utf8mb4_general_ci, value1 varchar(20) charset utf8mb4 collate utf8mb4_bin)") + tk.MustExec("insert into test.t values (1, 'abc', 'abc '),(4, 'Abc', 'abc'),(3,'def', 'def ');") + + tk.MustQuery("select upper(group_concat(distinct value order by 1)) from test.t;").Check(testkit.Rows("ABC,DEF")) + tk.MustQuery("select upper(group_concat(distinct value)) from test.t;").Check(testkit.Rows("ABC,DEF")) +} + +func TestIssue20874(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("delete from mysql.stats_histograms") + tk.MustExec("create table t (a char(10) collate utf8mb4_unicode_ci not null, b char(20) collate utf8mb4_general_ci not null, key idxa(a), key idxb(b))") + tk.MustExec("insert into t values ('#', 'C'), ('$', 'c'), ('a', 'a')") + tk.MustExec("set @@tidb_analyze_version=1") + tk.MustExec("analyze table t") + tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check(testkit.Rows( + "test t a 0 0 1 1 \x02\xd2 \x02\xd2 0", + "test t a 0 1 2 1 \x0e\x0f \x0e\x0f 0", + "test t a 0 2 3 1 \x0e3 \x0e3 0", + "test t b 0 0 1 1 \x00A \x00A 0", + "test t b 0 1 3 2 \x00C \x00C 0", + "test t idxa 1 0 1 1 \x02\xd2 \x02\xd2 0", + "test t idxa 1 1 2 1 \x0e\x0f \x0e\x0f 0", + "test t idxa 1 2 3 1 \x0e3 \x0e3 0", + "test t idxb 1 0 1 1 \x00A \x00A 0", + "test t idxb 1 1 3 2 \x00C \x00C 0", + )) + tk.MustQuery("select is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, correlation from mysql.stats_histograms").Sort().Check(testkit.Rows( + "0 1 3 0 9 1 1", + "0 2 2 0 9 1 -0.5", + "1 1 3 0 0 1 0", + "1 2 2 0 0 1 0", + )) + tk.MustExec("set @@tidb_analyze_version=2") + tk.MustExec("analyze table t") + tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't'").Sort().Check(testkit.Rows( + "test t a 0 \x02\xd2 1", + "test t a 0 \x0e\x0f 1", + "test t a 0 \x0e3 1", + "test t b 0 \x00A 1", + "test t b 0 \x00C 2", + "test t idxa 1 \x02\xd2 1", + "test t idxa 1 \x0e\x0f 1", + "test t idxa 1 \x0e3 1", + "test t idxb 1 \x00A 1", + "test t idxb 1 \x00C 2", + )) + tk.MustQuery("select is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, correlation from mysql.stats_histograms").Sort().Check(testkit.Rows( + "0 1 3 0 6 2 1", + "0 2 2 0 6 2 -0.5", + "1 1 3 0 6 2 0", + "1 2 2 0 6 2 0", + )) +} + +func TestAnalyzeClusteredIndexPrimary(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t0") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t0(a varchar(20), primary key(a) clustered)") + tk.MustExec("create table t1(a varchar(20), primary key(a))") + tk.MustExec("insert into t0 values('1111')") + tk.MustExec("insert into t1 values('1111')") + tk.MustExec("set @@session.tidb_analyze_version = 1") + tk.MustExec("analyze table t0 index primary") + tk.MustExec("analyze table t1 index primary") + tk.MustQuery("show stats_buckets").Check(testkit.Rows( + "test t0 PRIMARY 1 0 1 1 1111 1111 0", + "test t1 PRIMARY 1 0 1 1 1111 1111 0")) + tk.MustExec("set @@session.tidb_analyze_version = 2") + tk.MustExec("analyze table t0") + tk.MustExec("analyze table t1") + tk.MustQuery("show stats_topn").Sort().Check(testkit.Rows(""+ + "test t0 PRIMARY 1 1111 1", + "test t0 a 0 1111 1", + "test t1 PRIMARY 1 1111 1", + "test t1 a 0 1111 1")) +} + +func TestAnalyzeSamplingWorkPanic(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@session.tidb_analyze_version = 2") + tk.MustExec("create table t(a int)") + tk.MustExec("insert into t values(1), (2), (3), (4), (5), (6), (7), (8), (9), (10), (11), (12)") + tk.MustExec("split table t between (-9223372036854775808) and (9223372036854775807) regions 12") + + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/mockAnalyzeSamplingBuildWorkerPanic", "return(1)")) + err := tk.ExecToErr("analyze table t") + require.NotNil(t, err) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/mockAnalyzeSamplingBuildWorkerPanic")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/mockAnalyzeSamplingMergeWorkerPanic", "return(1)")) + err = tk.ExecToErr("analyze table t") + require.NotNil(t, err) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/mockAnalyzeSamplingMergeWorkerPanic")) +} diff --git a/executor/analyze_test.go b/executor/analyze_test.go index f8c574f36d53a..b35c1321315e3 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -21,9 +21,9 @@ import ( "strings" "sync" "sync/atomic" + "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" @@ -42,20 +42,21 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/israce" - "github.com/pingcap/tidb/util/testkit" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/testutils" "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/tikvrpc" ) -var _ = Suite(&testFastAnalyze{}) - -func (s *testSuite1) TestAnalyzePartition(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAnalyzePartition(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) testkit.WithPruneMode(tk, variable.Static, func() { tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -73,24 +74,24 @@ PARTITION BY RANGE ( a ) ( } tk.MustExec("analyze table t") - is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) + is := tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) pi := table.Meta().GetPartitionInfo() - c.Assert(pi, NotNil) - do, err := session.GetDomain(s.store) - c.Assert(err, IsNil) + require.NotNil(t, pi) + do, err := session.GetDomain(store) + require.NoError(t, err) 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) + require.False(t, statsTbl.Pseudo) + require.Len(t, statsTbl.Columns, 3) + require.Len(t, statsTbl.Indices, 1) for _, col := range statsTbl.Columns { - c.Assert(col.Len()+col.Num(), Greater, 0) + require.Greater(t, col.Len()+col.Num(), 0) } for _, idx := range statsTbl.Indices { - c.Assert(idx.Len()+idx.Num(), Greater, 0) + require.Greater(t, idx.Len()+idx.Num(), 0) } } @@ -100,41 +101,49 @@ PARTITION BY RANGE ( a ) ( tk.MustExec(fmt.Sprintf(`insert into t values (%d, %d, "hello")`, i, i)) } tk.MustExec("alter table t analyze partition p0") - is = tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) + is = tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) table, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) pi = table.Meta().GetPartitionInfo() - c.Assert(pi, NotNil) + require.NotNil(t, pi) 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) + require.False(t, statsTbl.Pseudo) + require.Len(t, statsTbl.Columns, 3) + require.Len(t, statsTbl.Indices, 1) } else { - c.Assert(statsTbl.Pseudo, IsTrue) + require.True(t, statsTbl.Pseudo) } } }) } -func (s *testSuite1) TestAnalyzeReplicaReadFollower(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAnalyzeReplicaReadFollower(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, 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 := tk.Session().(sessionctx.Context) ctx.GetSessionVars().SetReplicaRead(kv.ReplicaReadFollower) tk.MustExec("analyze table t") } -func (s *testSuite1) TestClusterIndexAnalyze(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestClusterIndexAnalyze(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("drop database if exists test_cluster_index_analyze;") tk.MustExec("create database test_cluster_index_analyze;") tk.MustExec("use test_cluster_index_analyze;") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("create table t (a int, b int, c int, primary key(a, b));") for i := 0; i < 100; i++ { @@ -158,18 +167,28 @@ func (s *testSuite1) TestClusterIndexAnalyze(c *C) { tk.MustExec("drop table t;") } -func (s *testSuite1) TestAnalyzeRestrict(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAnalyzeRestrict(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, 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 := tk.Session().(sessionctx.Context) ctx.GetSessionVars().InRestrictedSQL = true tk.MustExec("analyze table t") } -func (s *testSuite1) TestAnalyzeParameters(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAnalyzeParameters(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + dom, err := session.BootstrapSession(store) + require.NoError(t, err) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") @@ -181,50 +200,57 @@ func (s *testSuite1) TestAnalyzeParameters(c *C) { tk.MustExec("set @@tidb_enable_fast_analyze = 1") tk.MustExec("set @@tidb_analyze_version = 1") tk.MustExec("analyze table t with 30 samples") - is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) + is := tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := table.Meta() - tbl := s.dom.StatsHandle().GetTableStats(tableInfo) + tbl := dom.StatsHandle().GetTableStats(tableInfo) col := tbl.Columns[1] - c.Assert(col.Len(), Equals, 20) - c.Assert(len(col.TopN.TopN), Equals, 1) + require.Equal(t, 20, col.Len()) + require.Len(t, col.TopN.TopN, 1) width, depth := col.CMSketch.GetWidthAndDepth() - c.Assert(depth, Equals, int32(5)) - c.Assert(width, Equals, int32(2048)) + require.Equal(t, int32(5), depth) + require.Equal(t, int32(2048), width) tk.MustExec("analyze table t with 4 buckets, 0 topn, 4 cmsketch width, 4 cmsketch depth") - tbl = s.dom.StatsHandle().GetTableStats(tableInfo) + tbl = dom.StatsHandle().GetTableStats(tableInfo) col = tbl.Columns[1] - c.Assert(col.Len(), Equals, 4) - c.Assert(col.TopN, IsNil) + require.Equal(t, 4, col.Len()) + require.Nil(t, col.TopN) width, depth = col.CMSketch.GetWidthAndDepth() - c.Assert(depth, Equals, int32(4)) - c.Assert(width, Equals, int32(4)) + require.Equal(t, int32(4), depth) + require.Equal(t, int32(4), width) // Test very large cmsketch tk.MustExec(fmt.Sprintf("analyze table t with %d cmsketch width, %d cmsketch depth", core.CMSketchSizeLimit, 1)) - tbl = s.dom.StatsHandle().GetTableStats(tableInfo) + tbl = dom.StatsHandle().GetTableStats(tableInfo) col = tbl.Columns[1] - c.Assert(col.Len(), Equals, 20) - c.Assert(len(col.TopN.TopN), Equals, 1) + require.Equal(t, 20, col.Len()) + + require.Len(t, col.TopN.TopN, 1) width, depth = col.CMSketch.GetWidthAndDepth() - c.Assert(depth, Equals, int32(1)) - c.Assert(width, Equals, int32(core.CMSketchSizeLimit)) + require.Equal(t, int32(1), depth) + require.Equal(t, int32(core.CMSketchSizeLimit), width) // Test very large cmsketch tk.MustExec("analyze table t with 20480 cmsketch width, 50 cmsketch depth") - tbl = s.dom.StatsHandle().GetTableStats(tableInfo) + tbl = dom.StatsHandle().GetTableStats(tableInfo) col = tbl.Columns[1] - c.Assert(col.Len(), Equals, 20) - c.Assert(len(col.TopN.TopN), Equals, 1) + require.Equal(t, 20, col.Len()) + require.Len(t, col.TopN.TopN, 1) width, depth = col.CMSketch.GetWidthAndDepth() - c.Assert(depth, Equals, int32(50)) - c.Assert(width, Equals, int32(20480)) + require.Equal(t, int32(50), depth) + require.Equal(t, int32(20480), width) } -func (s *testSuite1) TestAnalyzeTooLongColumns(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAnalyzeTooLongColumns(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + dom, err := session.BootstrapSession(store) + require.NoError(t, err) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a json)") @@ -232,31 +258,32 @@ func (s *testSuite1) TestAnalyzeTooLongColumns(c *C) { tk.MustExec(fmt.Sprintf("insert into t values ('%s')", value)) tk.MustExec("analyze table t") - is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) + is := tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) 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)) + tbl := dom.StatsHandle().GetTableStats(tableInfo) + require.Equal(t, 0, tbl.Columns[1].Len()) + require.Equal(t, int64(65559), tbl.Columns[1].TotColSize) } -func (s *testSuite1) TestAnalyzeIndexExtractTopN(c *C) { +func TestAnalyzeIndexExtractTopN(t *testing.T) { + t.Parallel() _ = checkHistogram - c.Skip("unstable, skip it and fix it before 20210618") + t.Skip("unstable, skip it and fix it before 20210618") store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(t, err) }() var dom *domain.Domain session.DisableStats4Test() session.SetSchemaLease(0) dom, err = session.BootstrapSession(store) - c.Assert(err, IsNil) + require.NoError(t, err) defer dom.Close() - tk := testkit.NewTestKit(c, store) + tk := testkit.NewTestKit(t, store) tk.MustExec("create database test_index_extract_topn") tk.MustExec("use test_index_extract_topn") @@ -266,31 +293,32 @@ func (s *testSuite1) TestAnalyzeIndexExtractTopN(c *C) { tk.MustExec("set @@session.tidb_analyze_version=2") tk.MustExec("analyze table t") - is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) + is := tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test_index_extract_topn"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := table.Meta() tbl := dom.StatsHandle().GetTableStats(tableInfo) // Construct TopN, should be (1, 1) -> 2 and (1, 2) -> 2 topn := statistics.NewTopN(2) { - key1, err := codec.EncodeKey(tk.Se.GetSessionVars().StmtCtx, nil, types.NewIntDatum(1), types.NewIntDatum(1)) - c.Assert(err, IsNil) + key1, err := codec.EncodeKey(tk.Session().GetSessionVars().StmtCtx, nil, types.NewIntDatum(1), types.NewIntDatum(1)) + require.NoError(t, err) topn.AppendTopN(key1, 2) - key2, err := codec.EncodeKey(tk.Se.GetSessionVars().StmtCtx, nil, types.NewIntDatum(1), types.NewIntDatum(2)) - c.Assert(err, IsNil) + key2, err := codec.EncodeKey(tk.Session().GetSessionVars().StmtCtx, nil, types.NewIntDatum(1), types.NewIntDatum(2)) + require.NoError(t, err) topn.AppendTopN(key2, 2) } for _, idx := range tbl.Indices { - ok, err := checkHistogram(tk.Se.GetSessionVars().StmtCtx, &idx.Histogram) - c.Assert(err, IsNil) - c.Assert(ok, IsTrue) - c.Assert(idx.TopN.Equal(topn), IsTrue) + ok, err := checkHistogram(tk.Session().GetSessionVars().StmtCtx, &idx.Histogram) + require.NoError(t, err) + require.True(t, ok) + require.True(t, idx.TopN.Equal(topn)) } } -func (s *testFastAnalyze) TestAnalyzeFastSample(c *C) { +func TestAnalyzeFastSample(t *testing.T) { + t.Parallel() var cls testutils.Cluster store, err := mockstore.NewMockStore( mockstore.WithClusterInspector(func(c testutils.Cluster) { @@ -298,25 +326,25 @@ func (s *testFastAnalyze) TestAnalyzeFastSample(c *C) { cls = c }), ) - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(t, err) }() var dom *domain.Domain session.DisableStats4Test() session.SetSchemaLease(0) dom, err = session.BootstrapSession(store) - c.Assert(err, IsNil) + require.NoError(t, err) defer dom.Close() - tk := testkit.NewTestKit(c, store) + tk := testkit.NewTestKit(t, store) atomic.StoreInt64(&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) + require.NoError(t, err) tblInfo := tbl.Meta() tid := tblInfo.ID @@ -328,7 +356,7 @@ func (s *testFastAnalyze) TestAnalyzeFastSample(c *C) { tk.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) } - handleCols := core.BuildHandleColsForAnalyze(tk.Se, tblInfo, true, nil) + handleCols := core.BuildHandleColsForAnalyze(tk.Session(), tblInfo, true, nil) var colsInfo []*model.ColumnInfo var indicesInfo []*model.IndexInfo for _, col := range tblInfo.Columns { @@ -346,12 +374,12 @@ func (s *testFastAnalyze) TestAnalyzeFastSample(c *C) { opts[ast.AnalyzeOptNumSamples] = 20 // Get a start_ts later than the above inserts. tk.MustExec("begin") - txn, err := tk.Se.Txn(false) - c.Assert(err, IsNil) + txn, err := tk.Session().Txn(false) + require.NoError(t, err) ts := txn.StartTS() tk.MustExec("commit") mockExec := &executor.AnalyzeTestFastExec{ - Ctx: tk.Se.(sessionctx.Context), + Ctx: tk.Session().(sessionctx.Context), HandleCols: handleCols, ColsInfo: colsInfo, IdxsInfo: indicesInfo, @@ -365,15 +393,15 @@ func (s *testFastAnalyze) TestAnalyzeFastSample(c *C) { Opts: opts, } err = mockExec.TestFastSample() - c.Assert(err, IsNil) - c.Assert(len(mockExec.Collectors), Equals, 3) + require.NoError(t, err) + require.Len(t, mockExec.Collectors, 3) for i := 0; i < 2; i++ { samples := mockExec.Collectors[i].Samples - c.Assert(len(samples), Equals, 20) + require.Len(t, samples, 20) for j := 1; j < 20; j++ { - cmp, err := samples[j].Value.Compare(tk.Se.GetSessionVars().StmtCtx, &samples[j-1].Value, collate.GetBinaryCollator()) - c.Assert(err, IsNil) - c.Assert(cmp, Greater, 0) + cmp, err := samples[j].Value.Compare(tk.Session().GetSessionVars().StmtCtx, &samples[j-1].Value, collate.GetBinaryCollator()) + require.NoError(t, err) + require.Greater(t, cmp, 0) } } } @@ -397,8 +425,9 @@ func checkHistogram(sc *stmtctx.StatementContext, hg *statistics.Histogram) (boo return true, nil } -func (s *testFastAnalyze) TestFastAnalyze(c *C) { - c.Skip("Skip this unstable test(#25782) and bring it back before 2021-07-29.") +func TestFastAnalyze(t *testing.T) { + t.Parallel() + t.Skip("Skip this unstable test(#25782) and bring it back before 2021-07-29.") var cls testutils.Cluster store, err := mockstore.NewMockStore( mockstore.WithClusterInspector(func(c testutils.Cluster) { @@ -406,19 +435,19 @@ func (s *testFastAnalyze) TestFastAnalyze(c *C) { cls = c }), ) - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(t, err) }() var dom *domain.Domain session.DisableStats4Test() session.SetSchemaLease(0) dom, err = session.BootstrapSession(store) - c.Assert(err, IsNil) + require.NoError(t, err) dom.SetStatsUpdating(true) defer dom.Close() - tk := testkit.NewTestKit(c, store) + tk := testkit.NewTestKit(t, store) atomic.StoreInt64(&executor.RandSeed, 123) tk.MustExec("use test") @@ -430,7 +459,7 @@ func (s *testFastAnalyze) TestFastAnalyze(c *C) { // Should not panic. tk.MustExec("analyze table t") tblInfo, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tid := tblInfo.Meta().ID // construct 6 regions split by {10, 20, 30, 40, 50} @@ -442,21 +471,21 @@ func (s *testFastAnalyze) TestFastAnalyze(c *C) { } tk.MustExec("analyze table t with 5 buckets, 6 samples") - is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) + is := tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := table.Meta() tbl := dom.StatsHandle().GetTableStats(tableInfo) // TODO(tangenta): add stats_meta.row_count assertion. for _, col := range tbl.Columns { - ok, err := checkHistogram(tk.Se.GetSessionVars().StmtCtx, &col.Histogram) - c.Assert(err, IsNil) - c.Assert(ok, IsTrue) + ok, err := checkHistogram(tk.Session().GetSessionVars().StmtCtx, &col.Histogram) + require.NoError(t, err) + require.True(t, ok) } for _, idx := range tbl.Indices { - ok, err := checkHistogram(tk.Se.GetSessionVars().StmtCtx, &idx.Histogram) - c.Assert(err, IsNil) - c.Assert(ok, IsTrue) + ok, err := checkHistogram(tk.Session().GetSessionVars().StmtCtx, &idx.Histogram) + require.NoError(t, err) + require.True(t, ok) } // Test CM Sketch built from fast analyze. @@ -517,27 +546,12 @@ func (s *testFastAnalyze) TestFastAnalyze(c *C) { */ } -func (s *testSerialSuite2) TestFastAnalyze4GlobalStats(c *C) { - if israce.RaceEnabled { - c.Skip("unstable, skip race test") - } - tk := testkit.NewTestKit(c, s.store) - tk.MustExec(`create database if not exists test_fast_gstats`) - tk.MustExec("use test_fast_gstats") - tk.MustExec("set @@session.tidb_enable_fast_analyze=1") - tk.MustExec("set @@session.tidb_build_stats_concurrency=1") - // test fast analyze in dynamic mode - tk.MustExec("set @@session.tidb_analyze_version = 2;") - tk.MustExec("set @@session.tidb_partition_prune_mode = 'dynamic';") - tk.MustExec("drop table if exists test_fast_gstats;") - tk.MustExec("create table test_fast_gstats(a int, b int) PARTITION BY HASH(a) PARTITIONS 2;") - tk.MustExec("insert into test_fast_gstats values(1,1),(3,3),(4,4),(2,2),(5,5);") - err := tk.ExecToErr("analyze table test_fast_gstats;") - c.Assert(err, ErrorMatches, ".*Fast analyze hasn't reached General Availability and only support analyze version 1 currently.*") -} +func TestIssue15993(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) -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);") @@ -546,8 +560,12 @@ func (s *testSuite1) TestIssue15993(c *C) { tk.MustExec("ANALYZE TABLE t0 INDEX PRIMARY;") } -func (s *testSuite1) TestIssue15751(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue15751(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, 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))") @@ -557,8 +575,12 @@ func (s *testSuite1) TestIssue15751(c *C) { tk.MustExec("ANALYZE TABLE t0") } -func (s *testSuite1) TestIssue15752(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue15752(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t0") tk.MustExec("CREATE TABLE t0(c0 INT)") @@ -569,130 +591,6 @@ func (s *testSuite1) TestIssue15752(c *C) { tk.MustExec("ANALYZE TABLE t0 INDEX i0") } -func (s *testSerialSuite2) TestAnalyzeIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1") - tk.MustExec("create table t1 (id int, v int, primary key(id), index k(v))") - tk.MustExec("insert into t1(id, v) values(1, 2), (2, 2), (3, 2), (4, 2), (5, 1), (6, 3), (7, 4)") - tk.MustExec("set @@tidb_analyze_version=1") - tk.MustExec("analyze table t1 index k") - c.Assert(len(tk.MustQuery("show stats_buckets where table_name = 't1' and column_name = 'k' and is_index = 1").Rows()), Greater, 0) - tk.MustExec("set @@tidb_analyze_version=default") - tk.MustExec("analyze table t1") - c.Assert(len(tk.MustQuery("show stats_topn where table_name = 't1' and column_name = 'k' and is_index = 1").Rows()), Greater, 0) - - func() { - defer tk.MustExec("set @@session.tidb_enable_fast_analyze=0") - tk.MustExec("drop stats t1") - tk.MustExec("set @@session.tidb_enable_fast_analyze=1") - tk.MustExec("set @@tidb_analyze_version=1") - tk.MustExec("analyze table t1 index k") - c.Assert(len(tk.MustQuery("show stats_buckets where table_name = 't1' and column_name = 'k' and is_index = 1").Rows()), Greater, 1) - }() -} - -func (s *testSerialSuite2) TestAnalyzeIncremental(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("set @@tidb_analyze_version = 1") - tk.Se.GetSessionVars().EnableStreaming = false - s.testAnalyzeIncremental(tk, c) -} - -func (s *testSerialSuite2) TestAnalyzeIncrementalStreaming(c *C) { - c.Skip("unistore hasn't support streaming yet.") - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.Se.GetSessionVars().EnableStreaming = true - s.testAnalyzeIncremental(tk, c) -} - -// nolint:unused -func (s *testSerialSuite2) 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 0", "test t idx 1 0 1 1 1 1 0")) - 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 0", "test t a 0 1 2 1 2 2 0", "test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0")) - 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 0", "test t a 0 1 2 1 2 2 0", "test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0")) - - // Test analyze incremental with feedback. - tk.MustExec("insert into t values (3,3)") - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(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) - c.Assert(h.LoadNeededHistograms(), IsNil) - tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1 0", "test t a 0 1 3 0 2 2147483647 0", "test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0")) - 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].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") - c.Assert(h.LoadNeededHistograms(), IsNil) - tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1 0", "test t a 0 1 2 1 2 2 0", "test t a 0 2 3 1 3 3 0", - "test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0", "test t idx 1 2 3 1 3 3 0")) - tblStats = h.GetTableStats(tblInfo) - c.Assert(tblStats.Indices[tblInfo.Indices[0].ID].QueryBytes(val), Equals, uint64(1)) - - // test analyzeIndexIncremental for global-level stats; - tk.MustExec("set @@session.tidb_analyze_version = 1;") - tk.MustQuery("select @@tidb_analyze_version").Check(testkit.Rows("1")) - tk.MustExec("set @@tidb_partition_prune_mode = 'static';") - tk.MustExec("drop table if exists t;") - tk.MustExec(`create table t (a int, b int, primary key(a), index idx(b)) partition by range (a) ( - partition p0 values less than (10), - partition p1 values less than (20), - partition p2 values less than (30) - );`) - tk.MustExec("analyze incremental table t index") - c.Assert(h.LoadNeededHistograms(), IsNil) - 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 warnings").Check(testkit.Rows()) // no warning - c.Assert(h.LoadNeededHistograms(), IsNil) - tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t p0 a 0 0 1 1 1 1 0", "test t p0 idx 1 0 1 1 1 1 0")) - tk.MustExec("insert into t values (2,2)") - tk.MustExec("analyze incremental table t index") - c.Assert(h.LoadNeededHistograms(), IsNil) - tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t p0 a 0 0 1 1 1 1 0", "test t p0 a 0 1 2 1 2 2 0", "test t p0 idx 1 0 1 1 1 1 0", "test t p0 idx 1 1 2 1 2 2 0")) - tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic';") - tk.MustExec("insert into t values (11,11)") - err = tk.ExecToErr("analyze incremental table t index") - c.Assert(err.Error(), Equals, "[stats]: global statistics for partitioned tables unavailable in ANALYZE INCREMENTAL") -} - -type testFastAnalyze struct { -} - type regionProperityClient struct { tikv.Client mu struct { @@ -716,7 +614,8 @@ func (c *regionProperityClient) SendRequest(ctx context.Context, addr string, re return c.Client.SendRequest(ctx, addr, req, timeout) } -func (s *testFastAnalyze) TestFastAnalyzeRetryRowCount(c *C) { +func TestFastAnalyzeRetryRowCount(t *testing.T) { + t.Parallel() cli := ®ionProperityClient{} hijackClient := func(c tikv.Client) tikv.Client { cli.Client = c @@ -731,23 +630,23 @@ func (s *testFastAnalyze) TestFastAnalyzeRetryRowCount(c *C) { }), mockstore.WithClientHijacker(hijackClient), ) - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(t, err) }() dom, err := session.BootstrapSession(store) - c.Assert(err, IsNil) + require.NoError(t, err) defer dom.Close() + tk := testkit.NewTestKit(t, store) - 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) + require.NoError(t, err) tid := tblInfo.Meta().ID - c.Assert(dom.StatsHandle().Update(dom.InfoSchema()), IsNil) + require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema())) tk.MustExec("set @@session.tidb_enable_fast_analyze=1") tk.MustExec("set @@session.tidb_build_stats_concurrency=1") tk.MustExec("set @@tidb_analyze_version = 1") @@ -760,26 +659,32 @@ func (s *testFastAnalyze) TestFastAnalyzeRetryRowCount(c *C) { tk.MustQuery("select * from retry_row_count") tk.MustExec("analyze table retry_row_count") row := tk.MustQuery(`show stats_meta where db_name = "test" and table_name = "retry_row_count"`).Rows()[0] - c.Assert(row[5], Equals, "30") + require.Equal(t, "30", row[5]) } -func (s *testSuite10) TestFailedAnalyzeRequest(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestFailedAnalyzeRequest(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, 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))") tk.MustExec("set @@tidb_analyze_version = 1") - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/buildStatsFromResult", `return(true)`), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/buildStatsFromResult", `return(true)`)) _, 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) + require.Equal(t, "mock buildStatsFromResult error", err.Error()) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/buildStatsFromResult")) } -func (s *testSuite1) TestExtractTopN(c *C) { - if israce.RaceEnabled { - c.Skip("unstable, skip race test") - } - tk := testkit.NewTestKit(c, s.store) +func TestExtractTopN(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + dom, err := session.BootstrapSession(store) + require.NoError(t, err) + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists test_extract_topn") tk.MustExec("use test_extract_topn") tk.MustExec("drop table if exists test_extract_topn") @@ -792,19 +697,19 @@ func (s *testSuite1) TestExtractTopN(c *C) { tk.MustExec(fmt.Sprintf("insert into test_extract_topn values (%d, 0)", i+10)) } tk.MustExec("analyze table test_extract_topn") - is := s.dom.InfoSchema() + is := dom.InfoSchema() table, err := is.TableByName(model.NewCIStr("test_extract_topn"), model.NewCIStr("test_extract_topn")) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := table.Meta() - tblStats := s.dom.StatsHandle().GetTableStats(tblInfo) + tblStats := dom.StatsHandle().GetTableStats(tblInfo) colStats := tblStats.Columns[tblInfo.Columns[1].ID] - c.Assert(len(colStats.TopN.TopN), Equals, 10) + require.Len(t, colStats.TopN.TopN, 10) item := colStats.TopN.TopN[0] - c.Assert(item.Count, Equals, uint64(11)) + require.Equal(t, uint64(11), item.Count) idxStats := tblStats.Indices[tblInfo.Indices[0].ID] - c.Assert(len(idxStats.TopN.TopN), Equals, 10) + require.Len(t, idxStats.TopN.TopN, 10) idxItem := idxStats.TopN.TopN[0] - c.Assert(idxItem.Count, Equals, uint64(11)) + require.Equal(t, uint64(11), idxItem.Count) // The columns are: DBName, table name, column name, is index, value, count. tk.MustQuery("show stats_topn where column_name in ('b', 'index_b')").Sort().Check(testkit.Rows("test_extract_topn test_extract_topn b 0 0 11", "test_extract_topn test_extract_topn b 0 1 1", @@ -829,8 +734,14 @@ func (s *testSuite1) TestExtractTopN(c *C) { )) } -func (s *testSuite1) TestHashInTopN(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestHashInTopN(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + dom, err := session.BootstrapSession(store) + require.NoError(t, err) + tk := testkit.NewTestKit(t, 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))") @@ -845,32 +756,36 @@ func (s *testSuite1) TestHashInTopN(c *C) { tk.MustExec("set @@tidb_analyze_version = 1") // get stats of normal analyze tk.MustExec("analyze table t") - is := s.dom.InfoSchema() + is := dom.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := tbl.Meta() - tblStats1 := s.dom.StatsHandle().GetTableStats(tblInfo).Copy() + tblStats1 := 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() + tblStats2 := dom.StatsHandle().GetTableStats(tblInfo).Copy() // check the hash for topn for _, col := range tblInfo.Columns { topn1 := tblStats1.Columns[col.ID].TopN.TopN cm2 := tblStats2.Columns[col.ID].TopN for _, topnMeta := range topn1 { count2, exists := cm2.QueryTopN(topnMeta.Encoded) - c.Assert(exists, Equals, true) - c.Assert(count2, Equals, topnMeta.Count) + require.True(t, exists) + require.Equal(t, topnMeta.Count, count2) } } } -func (s *testSuite1) TestNormalAnalyzeOnCommonHandle(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestNormalAnalyzeOnCommonHandle(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2, t3, t4") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("CREATE TABLE t1 (a int primary key, b int)") tk.MustExec("insert into t1 values(1,1), (2,2), (3,3)") tk.MustExec("CREATE TABLE t2 (a varchar(255) primary key, b int)") @@ -915,9 +830,13 @@ func (s *testSuite1) TestNormalAnalyzeOnCommonHandle(c *C) { "test t3 c 1 2 3 1 3 3 0")) } -func (s *testSuite1) TestDefaultValForAnalyze(c *C) { - c.Skip("skip race test") - tk := testkit.NewTestKit(c, s.store) +func TestDefaultValForAnalyze(t *testing.T) { + t.Parallel() + t.Skip("skip race test") + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, 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") @@ -950,98 +869,13 @@ func (s *testSuite1) TestDefaultValForAnalyze(c *C) { "└─IndexRangeScan_5 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false")) } -func (s *testSerialSuite2) TestIssue27429(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table test.t(id int, value varchar(20) charset utf8mb4 collate utf8mb4_general_ci, value1 varchar(20) charset utf8mb4 collate utf8mb4_bin)") - tk.MustExec("insert into test.t values (1, 'abc', 'abc '),(4, 'Abc', 'abc'),(3,'def', 'def ');") - - tk.MustQuery("select upper(group_concat(distinct value order by 1)) from test.t;").Check(testkit.Rows("ABC,DEF")) - tk.MustQuery("select upper(group_concat(distinct value)) from test.t;").Check(testkit.Rows("ABC,DEF")) -} +func TestAnalyzeFullSamplingOnIndexWithVirtualColumnOrPrefixColumn(t *testing.T) { + t.Parallel() + t.Skip("unstable, skip it and fix it before 20210624") + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) -func (s *testSerialSuite2) TestIssue20874(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("delete from mysql.stats_histograms") - tk.MustExec("create table t (a char(10) collate utf8mb4_unicode_ci not null, b char(20) collate utf8mb4_general_ci not null, key idxa(a), key idxb(b))") - tk.MustExec("insert into t values ('#', 'C'), ('$', 'c'), ('a', 'a')") - tk.MustExec("set @@tidb_analyze_version=1") - tk.MustExec("analyze table t") - tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check(testkit.Rows( - "test t a 0 0 1 1 \x02\xd2 \x02\xd2 0", - "test t a 0 1 2 1 \x0e\x0f \x0e\x0f 0", - "test t a 0 2 3 1 \x0e3 \x0e3 0", - "test t b 0 0 1 1 \x00A \x00A 0", - "test t b 0 1 3 2 \x00C \x00C 0", - "test t idxa 1 0 1 1 \x02\xd2 \x02\xd2 0", - "test t idxa 1 1 2 1 \x0e\x0f \x0e\x0f 0", - "test t idxa 1 2 3 1 \x0e3 \x0e3 0", - "test t idxb 1 0 1 1 \x00A \x00A 0", - "test t idxb 1 1 3 2 \x00C \x00C 0", - )) - tk.MustQuery("select is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, correlation from mysql.stats_histograms").Sort().Check(testkit.Rows( - "0 1 3 0 9 1 1", - "0 2 2 0 9 1 -0.5", - "1 1 3 0 0 1 0", - "1 2 2 0 0 1 0", - )) - tk.MustExec("set @@tidb_analyze_version=2") - tk.MustExec("analyze table t") - tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't'").Sort().Check(testkit.Rows( - "test t a 0 \x02\xd2 1", - "test t a 0 \x0e\x0f 1", - "test t a 0 \x0e3 1", - "test t b 0 \x00A 1", - "test t b 0 \x00C 2", - "test t idxa 1 \x02\xd2 1", - "test t idxa 1 \x0e\x0f 1", - "test t idxa 1 \x0e3 1", - "test t idxb 1 \x00A 1", - "test t idxb 1 \x00C 2", - )) - tk.MustQuery("select is_index, hist_id, distinct_count, null_count, tot_col_size, stats_ver, correlation from mysql.stats_histograms").Sort().Check(testkit.Rows( - "0 1 3 0 6 2 1", - "0 2 2 0 6 2 -0.5", - "1 1 3 0 6 2 0", - "1 2 2 0 6 2 0", - )) -} - -func (s *testSerialSuite2) TestAnalyzeClusteredIndexPrimary(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t0") - tk.MustExec("drop table if exists t1") - tk.MustExec("create table t0(a varchar(20), primary key(a) clustered)") - tk.MustExec("create table t1(a varchar(20), primary key(a))") - tk.MustExec("insert into t0 values('1111')") - tk.MustExec("insert into t1 values('1111')") - tk.MustExec("set @@session.tidb_analyze_version = 1") - tk.MustExec("analyze table t0 index primary") - tk.MustExec("analyze table t1 index primary") - tk.MustQuery("show stats_buckets").Check(testkit.Rows( - "test t0 PRIMARY 1 0 1 1 1111 1111 0", - "test t1 PRIMARY 1 0 1 1 1111 1111 0")) - tk.MustExec("set @@session.tidb_analyze_version = 2") - tk.MustExec("analyze table t0") - tk.MustExec("analyze table t1") - tk.MustQuery("show stats_topn").Sort().Check(testkit.Rows(""+ - "test t0 PRIMARY 1 1111 1", - "test t0 a 0 1111 1", - "test t1 PRIMARY 1 1111 1", - "test t1 a 0 1111 1")) -} - -func (s *testSuite1) TestAnalyzeFullSamplingOnIndexWithVirtualColumnOrPrefixColumn(c *C) { - c.Skip("unstable, skip it and fix it before 20210624") - tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists sampling_index_virtual_col") tk.MustExec("create table sampling_index_virtual_col(a int, b int as (a+1), index idx(b))") @@ -1056,9 +890,9 @@ func (s *testSuite1) TestAnalyzeFullSamplingOnIndexWithVirtualColumnOrPrefixColu tk.MustQuery("show stats_topn where table_name = 'sampling_index_virtual_col' and column_name = 'idx'").Check(testkit.Rows("test sampling_index_virtual_col idx 1 6 4")) row := tk.MustQuery(`show stats_histograms where db_name = "test" and table_name = "sampling_index_virtual_col"`).Rows()[0] // The NDV. - c.Assert(row[6], Equals, "5") + require.Equal(t, "5", row[6]) // The NULLs. - c.Assert(row[7], Equals, "2") + require.Equal(t, "2", row[7]) tk.MustExec("drop table if exists sampling_index_prefix_col") tk.MustExec("create table sampling_index_prefix_col(a varchar(3), index idx(a(1)))") tk.MustExec("insert into sampling_index_prefix_col (a) values ('aa'), ('ab'), ('ac'), ('bb')") @@ -1069,94 +903,83 @@ func (s *testSuite1) TestAnalyzeFullSamplingOnIndexWithVirtualColumnOrPrefixColu tk.MustQuery("show stats_topn where table_name = 'sampling_index_prefix_col' and column_name = 'idx'").Check(testkit.Rows("test sampling_index_prefix_col idx 1 a 3")) } -func (s *testSerialSuite2) TestAnalyzeSamplingWorkPanic(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("set @@session.tidb_analyze_version = 2") - tk.MustExec("create table t(a int)") - tk.MustExec("insert into t values(1), (2), (3), (4), (5), (6), (7), (8), (9), (10), (11), (12)") - tk.MustExec("split table t between (-9223372036854775808) and (9223372036854775807) regions 12") - - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockAnalyzeSamplingBuildWorkerPanic", "return(1)"), IsNil) - err := tk.ExecToErr("analyze table t") - c.Assert(err, NotNil) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/mockAnalyzeSamplingBuildWorkerPanic"), IsNil) - - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockAnalyzeSamplingMergeWorkerPanic", "return(1)"), IsNil) - err = tk.ExecToErr("analyze table t") - c.Assert(err, NotNil) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/mockAnalyzeSamplingMergeWorkerPanic"), IsNil) -} +func TestSnapshotAnalyze(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) -func (s *testSuite10) TestSnapshotAnalyze(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, index index_a(a))") - is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) + is := tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := tbl.Meta() tid := tblInfo.ID tk.MustExec("insert into t values(1),(1),(1)") tk.MustExec("begin") - txn, err := tk.Se.Txn(false) - c.Assert(err, IsNil) + txn, err := tk.Session().Txn(false) + require.NoError(t, err) startTS1 := txn.StartTS() tk.MustExec("commit") tk.MustExec("insert into t values(2),(2),(2)") tk.MustExec("begin") - txn, err = tk.Se.Txn(false) - c.Assert(err, IsNil) + txn, err = tk.Session().Txn(false) + require.NoError(t, err) startTS2 := txn.StartTS() tk.MustExec("commit") - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/injectAnalyzeSnapshot", fmt.Sprintf("return(%d)", startTS1)), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/injectAnalyzeSnapshot", fmt.Sprintf("return(%d)", startTS1))) tk.MustExec("analyze table t") rows := tk.MustQuery(fmt.Sprintf("select count, snapshot from mysql.stats_meta where table_id = %d", tid)).Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][0], Equals, "3") + require.Len(t, rows, 1) + require.Equal(t, "3", rows[0][0]) s1Str := rows[0][1].(string) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/injectAnalyzeSnapshot", fmt.Sprintf("return(%d)", startTS2)), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/injectAnalyzeSnapshot", fmt.Sprintf("return(%d)", startTS2))) tk.MustExec("analyze table t") rows = tk.MustQuery(fmt.Sprintf("select count, snapshot from mysql.stats_meta where table_id = %d", tid)).Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][0], Equals, "6") + require.Len(t, rows, 1) + require.Equal(t, "6", rows[0][0]) s2Str := rows[0][1].(string) - c.Assert(s1Str != s2Str, IsTrue) + require.True(t, s1Str != s2Str) tk.MustExec("set @@session.tidb_analyze_version = 2") - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/injectAnalyzeSnapshot", fmt.Sprintf("return(%d)", startTS1)), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/injectAnalyzeSnapshot", fmt.Sprintf("return(%d)", startTS1))) tk.MustExec("analyze table t") rows = tk.MustQuery(fmt.Sprintf("select count, snapshot from mysql.stats_meta where table_id = %d", tid)).Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][0], Equals, "6") + require.Len(t, rows, 1) + require.Equal(t, "6", rows[0][0]) s3Str := rows[0][1].(string) - c.Assert(s3Str, Equals, s2Str) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/injectAnalyzeSnapshot"), IsNil) + require.Equal(t, s2Str, s3Str) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/injectAnalyzeSnapshot")) } -func (s *testSuite10) TestAdjustSampleRateNote(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAdjustSampleRateNote(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") - statsHandle := domain.GetDomain(tk.Se.(sessionctx.Context)).StatsHandle() + statsHandle := domain.GetDomain(tk.Session().(sessionctx.Context)).StatsHandle() tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, index index_a(a))") - c.Assert(statsHandle.HandleDDLEvent(<-statsHandle.DDLEventCh()), IsNil) - is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) + require.NoError(t, statsHandle.HandleDDLEvent(<-statsHandle.DDLEventCh())) + is := tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := tbl.Meta() tid := tblInfo.ID tk.MustExec(fmt.Sprintf("update mysql.stats_meta set count = 220000 where table_id=%d", tid)) - c.Assert(statsHandle.Update(is), IsNil) + require.NoError(t, statsHandle.Update(is)) result := tk.MustQuery("show stats_meta where table_name = 't'") - c.Assert(result.Rows()[0][5], Equals, "220000") + require.Equal(t, "220000", result.Rows()[0][5]) tk.MustExec("analyze table t") tk.MustQuery("show warnings").Check(testkit.Rows("Note 1105 Analyze use auto adjusted sample rate 0.500000 for table test.t.")) tk.MustExec("insert into t values(1),(1),(1)") - c.Assert(statsHandle.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(statsHandle.Update(is), IsNil) + require.NoError(t, statsHandle.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, statsHandle.Update(is)) result = tk.MustQuery("show stats_meta where table_name = 't'") - c.Assert(result.Rows()[0][5], Equals, "3") + require.Equal(t, "3", result.Rows()[0][5]) tk.MustExec("analyze table t") tk.MustQuery("show warnings").Check(testkit.Rows("Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t.")) }