From 2067b5d943c396c6ed4520441c75c964b06f315f Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Tue, 14 Feb 2023 21:10:01 +0800 Subject: [PATCH] statistics: improve test to make test more fast (#41403) ref pingcap/tidb#41377 --- ddl/failtest/BUILD.bazel | 3 +- ddl/failtest/main_test.go | 1 + ddl/indexmergetest/BUILD.bazel | 4 +- sessiontxn/isolation/BUILD.bazel | 3 +- statistics/handle/BUILD.bazel | 13 +-- statistics/handle/dump_test.go | 3 +- statistics/handle/handletest/BUILD.bazel | 33 ++++++++ .../handle/{ => handletest}/handle_test.go | 68 ++-------------- statistics/handle/handletest/main_test.go | 33 ++++++++ statistics/handle/internal/BUILD.bazel | 12 +++ statistics/handle/internal/testutil.go | 81 +++++++++++++++++++ statistics/handle/updatetest/BUILD.bazel | 32 ++++++++ statistics/handle/updatetest/main_test.go | 33 ++++++++ .../handle/{ => updatetest}/update_test.go | 5 +- 14 files changed, 244 insertions(+), 80 deletions(-) create mode 100644 statistics/handle/handletest/BUILD.bazel rename statistics/handle/{ => handletest}/handle_test.go (98%) create mode 100644 statistics/handle/handletest/main_test.go create mode 100644 statistics/handle/internal/BUILD.bazel create mode 100644 statistics/handle/internal/testutil.go create mode 100644 statistics/handle/updatetest/BUILD.bazel create mode 100644 statistics/handle/updatetest/main_test.go rename statistics/handle/{ => updatetest}/update_test.go (99%) diff --git a/ddl/failtest/BUILD.bazel b/ddl/failtest/BUILD.bazel index 4b7980412adc0..5b3dd4403afaa 100644 --- a/ddl/failtest/BUILD.bazel +++ b/ddl/failtest/BUILD.bazel @@ -2,12 +2,13 @@ load("@io_bazel_rules_go//go:def.bzl", "go_test") go_test( name = "failtest_test", - timeout = "moderate", + timeout = "short", srcs = [ "fail_db_test.go", "main_test.go", ], flaky = True, + shard_count = 14, deps = [ "//config", "//ddl", diff --git a/ddl/failtest/main_test.go b/ddl/failtest/main_test.go index c136cb0fa6b94..327508c48274a 100644 --- a/ddl/failtest/main_test.go +++ b/ddl/failtest/main_test.go @@ -38,6 +38,7 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/ddl/indexmergetest/BUILD.bazel b/ddl/indexmergetest/BUILD.bazel index b70146ae8d461..11eac6481154f 100644 --- a/ddl/indexmergetest/BUILD.bazel +++ b/ddl/indexmergetest/BUILD.bazel @@ -2,14 +2,14 @@ load("@io_bazel_rules_go//go:def.bzl", "go_test") go_test( name = "indexmergetest_test", - timeout = "moderate", + timeout = "short", srcs = [ "main_test.go", "merge_test.go", ], flaky = True, race = "on", - shard_count = 4, + shard_count = 18, deps = [ "//config", "//ddl", diff --git a/sessiontxn/isolation/BUILD.bazel b/sessiontxn/isolation/BUILD.bazel index 21ca84f60ad0f..de291d289074f 100644 --- a/sessiontxn/isolation/BUILD.bazel +++ b/sessiontxn/isolation/BUILD.bazel @@ -38,7 +38,7 @@ go_library( go_test( name = "isolation_test", - timeout = "moderate", + timeout = "short", srcs = [ "main_test.go", "optimistic_test.go", @@ -47,6 +47,7 @@ go_test( "serializable_test.go", ], flaky = True, + shard_count = 28, deps = [ ":isolation", "//config", diff --git a/statistics/handle/BUILD.bazel b/statistics/handle/BUILD.bazel index d52847495d539..34fa9b315644e 100644 --- a/statistics/handle/BUILD.bazel +++ b/statistics/handle/BUILD.bazel @@ -59,17 +59,15 @@ go_library( go_test( name = "handle_test", - timeout = "moderate", + timeout = "short", srcs = [ "ddl_test.go", "dump_test.go", "gc_test.go", "handle_hist_test.go", - "handle_test.go", "lru_cache_test.go", "main_test.go", "update_list_test.go", - "update_test.go", ], embed = [":handle"], flaky = True, @@ -78,28 +76,21 @@ go_test( deps = [ "//config", "//domain", - "//metrics", "//parser/model", "//parser/mysql", - "//session", "//sessionctx/stmtctx", "//sessionctx/variable", "//statistics", + "//statistics/handle/internal", "//testkit", "//testkit/testsetup", "//types", "//util", - "//util/codec", - "//util/collate", - "//util/israce", "//util/mathutil", "//util/mock", - "//util/ranger", "//util/sqlexec", "@com_github_pingcap_failpoint//:failpoint", - "@com_github_prometheus_client_model//go", "@com_github_stretchr_testify//require", - "@com_github_tikv_client_go_v2//oracle", "@org_uber_go_goleak//:goleak", ], ) diff --git a/statistics/handle/dump_test.go b/statistics/handle/dump_test.go index 165ea999ae25a..ac50506fcde36 100644 --- a/statistics/handle/dump_test.go +++ b/statistics/handle/dump_test.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" + "github.com/pingcap/tidb/statistics/handle/internal" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util" "github.com/stretchr/testify/require" @@ -53,7 +54,7 @@ func requireTableEqual(t *testing.T, a *statistics.Table, b *statistics.Table) { } require.True(t, a.Indices[i].TopN.Equal(b.Indices[i].TopN)) } - require.True(t, isSameExtendedStats(a.ExtendedStats, b.ExtendedStats)) + require.True(t, internal.IsSameExtendedStats(a.ExtendedStats, b.ExtendedStats)) } func cleanStats(tk *testkit.TestKit, do *domain.Domain) { diff --git a/statistics/handle/handletest/BUILD.bazel b/statistics/handle/handletest/BUILD.bazel new file mode 100644 index 0000000000000..e8f00b0655a86 --- /dev/null +++ b/statistics/handle/handletest/BUILD.bazel @@ -0,0 +1,33 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "handletest_test", + timeout = "short", + srcs = [ + "handle_test.go", + "main_test.go", + ], + flaky = True, + shard_count = 50, + deps = [ + "//config", + "//domain", + "//parser/model", + "//session", + "//sessionctx/variable", + "//statistics", + "//statistics/handle", + "//statistics/handle/internal", + "//testkit", + "//testkit/testsetup", + "//types", + "//util/collate", + "//util/israce", + "//util/mock", + "//util/ranger", + "@com_github_pingcap_failpoint//:failpoint", + "@com_github_stretchr_testify//require", + "@com_github_tikv_client_go_v2//oracle", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/statistics/handle/handle_test.go b/statistics/handle/handletest/handle_test.go similarity index 98% rename from statistics/handle/handle_test.go rename to statistics/handle/handletest/handle_test.go index ab701ce55bb3a..ab289da80814b 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handletest/handle_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package handle_test +package handletest import ( "bytes" @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" + "github.com/pingcap/tidb/statistics/handle/internal" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" @@ -144,63 +145,6 @@ func TestStatsCacheMemTracker(t *testing.T) { require.False(t, statsTbl.Pseudo) } -func assertTableEqual(t *testing.T, a *statistics.Table, b *statistics.Table) { - require.Equal(t, b.Count, a.Count) - require.Equal(t, b.ModifyCount, a.ModifyCount) - require.Len(t, a.Columns, len(b.Columns)) - for i := range a.Columns { - require.Equal(t, b.Columns[i].Count, a.Columns[i].Count) - require.True(t, statistics.HistogramEqual(&a.Columns[i].Histogram, &b.Columns[i].Histogram, false)) - if a.Columns[i].CMSketch == nil { - require.Nil(t, b.Columns[i].CMSketch) - } else { - require.True(t, a.Columns[i].CMSketch.Equal(b.Columns[i].CMSketch)) - } - // The nil case has been considered in (*TopN).Equal() so we don't need to consider it here. - require.Truef(t, a.Columns[i].TopN.Equal(b.Columns[i].TopN), "%v, %v", a.Columns[i].TopN, b.Columns[i].TopN) - } - require.Len(t, a.Indices, len(b.Indices)) - for i := range a.Indices { - require.True(t, statistics.HistogramEqual(&a.Indices[i].Histogram, &b.Indices[i].Histogram, false)) - if a.Indices[i].CMSketch == nil { - require.Nil(t, b.Indices[i].CMSketch) - } else { - require.True(t, a.Indices[i].CMSketch.Equal(b.Indices[i].CMSketch)) - } - require.True(t, a.Indices[i].TopN.Equal(b.Indices[i].TopN)) - } - require.True(t, isSameExtendedStats(a.ExtendedStats, b.ExtendedStats)) -} - -func isSameExtendedStats(a, b *statistics.ExtendedStatsColl) bool { - aEmpty := (a == nil) || len(a.Stats) == 0 - bEmpty := (b == nil) || len(b.Stats) == 0 - if (aEmpty && !bEmpty) || (!aEmpty && bEmpty) { - return false - } - if aEmpty && bEmpty { - return true - } - if len(a.Stats) != len(b.Stats) { - return false - } - for aKey, aItem := range a.Stats { - bItem, ok := b.Stats[aKey] - if !ok { - return false - } - for i, id := range aItem.ColIDs { - if id != bItem.ColIDs[i] { - return false - } - } - if (aItem.Tp != bItem.Tp) || (aItem.ScalarVals != bItem.ScalarVals) || (aItem.StringVals != bItem.StringVals) { - return false - } - } - return true -} - func TestStatsStoreAndLoad(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) testKit := testkit.NewTestKit(t, store) @@ -226,7 +170,7 @@ func TestStatsStoreAndLoad(t *testing.T) { statsTbl2 := do.StatsHandle().GetTableStats(tableInfo) require.False(t, statsTbl2.Pseudo) require.Equal(t, int64(recordCount), statsTbl2.Count) - assertTableEqual(t, statsTbl1, statsTbl2) + internal.AssertTableEqual(t, statsTbl1, statsTbl2) } func TestEmptyTable(t *testing.T) { @@ -510,7 +454,7 @@ func TestInitStats(t *testing.T) { h.Clear() require.NoError(t, h.Update(is)) table1 := h.GetTableStats(tbl.Meta()) - assertTableEqual(t, table0, table1) + internal.AssertTableEqual(t, table0, table1) h.SetLease(0) } @@ -540,7 +484,7 @@ func TestInitStatsVer2(t *testing.T) { h.Clear() require.NoError(t, h.Update(is)) table1 := h.GetTableStats(tbl.Meta()) - assertTableEqual(t, table0, table1) + internal.AssertTableEqual(t, table0, table1) h.SetLease(0) } @@ -2484,7 +2428,7 @@ func TestFeedbackWithGlobalStats(t *testing.T) { statsTblAfter := h.GetTableStats(tblInfo) // assert that statistics not changed // the feedback can not work for the partition table in both static and dynamic mode - assertTableEqual(t, statsBefore, statsTblAfter) + internal.AssertTableEqual(t, statsBefore, statsTblAfter) } // Case 2: Feedback wouldn't be applied on version 2 and global-level statistics. diff --git a/statistics/handle/handletest/main_test.go b/statistics/handle/handletest/main_test.go new file mode 100644 index 0000000000000..75eec4666f560 --- /dev/null +++ b/statistics/handle/handletest/main_test.go @@ -0,0 +1,33 @@ +// Copyright 2023 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 handletest + +import ( + "testing" + + "github.com/pingcap/tidb/testkit/testsetup" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), + goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + } + testsetup.SetupForCommonTest() + goleak.VerifyTestMain(m, opts...) +} diff --git a/statistics/handle/internal/BUILD.bazel b/statistics/handle/internal/BUILD.bazel new file mode 100644 index 0000000000000..50a402f86cc02 --- /dev/null +++ b/statistics/handle/internal/BUILD.bazel @@ -0,0 +1,12 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "internal", + srcs = ["testutil.go"], + importpath = "github.com/pingcap/tidb/statistics/handle/internal", + visibility = ["//statistics/handle:__subpackages__"], + deps = [ + "//statistics", + "@com_github_stretchr_testify//require", + ], +) diff --git a/statistics/handle/internal/testutil.go b/statistics/handle/internal/testutil.go new file mode 100644 index 0000000000000..02ae083517e8d --- /dev/null +++ b/statistics/handle/internal/testutil.go @@ -0,0 +1,81 @@ +// Copyright 2023 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 internal + +import ( + "testing" + + "github.com/pingcap/tidb/statistics" + "github.com/stretchr/testify/require" +) + +// AssertTableEqual is to assert whether two table is equal +func AssertTableEqual(t *testing.T, a *statistics.Table, b *statistics.Table) { + require.Equal(t, b.Count, a.Count) + require.Equal(t, b.ModifyCount, a.ModifyCount) + require.Len(t, a.Columns, len(b.Columns)) + for i := range a.Columns { + require.Equal(t, b.Columns[i].Count, a.Columns[i].Count) + require.True(t, statistics.HistogramEqual(&a.Columns[i].Histogram, &b.Columns[i].Histogram, false)) + if a.Columns[i].CMSketch == nil { + require.Nil(t, b.Columns[i].CMSketch) + } else { + require.True(t, a.Columns[i].CMSketch.Equal(b.Columns[i].CMSketch)) + } + // The nil case has been considered in (*TopN).Equal() so we don't need to consider it here. + require.Truef(t, a.Columns[i].TopN.Equal(b.Columns[i].TopN), "%v, %v", a.Columns[i].TopN, b.Columns[i].TopN) + } + require.Len(t, a.Indices, len(b.Indices)) + for i := range a.Indices { + require.True(t, statistics.HistogramEqual(&a.Indices[i].Histogram, &b.Indices[i].Histogram, false)) + if a.Indices[i].CMSketch == nil { + require.Nil(t, b.Indices[i].CMSketch) + } else { + require.True(t, a.Indices[i].CMSketch.Equal(b.Indices[i].CMSketch)) + } + require.True(t, a.Indices[i].TopN.Equal(b.Indices[i].TopN)) + } + require.True(t, IsSameExtendedStats(a.ExtendedStats, b.ExtendedStats)) +} + +// IsSameExtendedStats is to judge whether the extended states is the same. +func IsSameExtendedStats(a, b *statistics.ExtendedStatsColl) bool { + aEmpty := (a == nil) || len(a.Stats) == 0 + bEmpty := (b == nil) || len(b.Stats) == 0 + if (aEmpty && !bEmpty) || (!aEmpty && bEmpty) { + return false + } + if aEmpty && bEmpty { + return true + } + if len(a.Stats) != len(b.Stats) { + return false + } + for aKey, aItem := range a.Stats { + bItem, ok := b.Stats[aKey] + if !ok { + return false + } + for i, id := range aItem.ColIDs { + if id != bItem.ColIDs[i] { + return false + } + } + if (aItem.Tp != bItem.Tp) || (aItem.ScalarVals != bItem.ScalarVals) || (aItem.StringVals != bItem.StringVals) { + return false + } + } + return true +} diff --git a/statistics/handle/updatetest/BUILD.bazel b/statistics/handle/updatetest/BUILD.bazel new file mode 100644 index 0000000000000..6ac60d66afc3d --- /dev/null +++ b/statistics/handle/updatetest/BUILD.bazel @@ -0,0 +1,32 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "updatetest_test", + timeout = "short", + srcs = [ + "main_test.go", + "update_test.go", + ], + flaky = True, + shard_count = 40, + deps = [ + "//metrics", + "//parser/model", + "//parser/mysql", + "//sessionctx/stmtctx", + "//sessionctx/variable", + "//statistics", + "//statistics/handle", + "//statistics/handle/internal", + "//testkit", + "//testkit/testsetup", + "//types", + "//util/codec", + "//util/collate", + "//util/ranger", + "@com_github_prometheus_client_model//go", + "@com_github_stretchr_testify//require", + "@com_github_tikv_client_go_v2//oracle", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/statistics/handle/updatetest/main_test.go b/statistics/handle/updatetest/main_test.go new file mode 100644 index 0000000000000..b7c7d150cf3c5 --- /dev/null +++ b/statistics/handle/updatetest/main_test.go @@ -0,0 +1,33 @@ +// Copyright 2023 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 updatetest + +import ( + "testing" + + "github.com/pingcap/tidb/testkit/testsetup" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), + goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + } + testsetup.SetupForCommonTest() + goleak.VerifyTestMain(m, opts...) +} diff --git a/statistics/handle/update_test.go b/statistics/handle/updatetest/update_test.go similarity index 99% rename from statistics/handle/update_test.go rename to statistics/handle/updatetest/update_test.go index edbef59135696..8e62cb4e7c66b 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/updatetest/update_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package handle_test +package updatetest import ( "fmt" @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" + "github.com/pingcap/tidb/statistics/handle/internal" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" @@ -1327,7 +1328,7 @@ func TestFeedbackWithStatsVer2(t *testing.T) { require.NoError(t, err) statsTblAfter := h.GetTableStats(tblInfo) // assert that statistics not changed - assertTableEqual(t, statsTblBefore, statsTblAfter) + internal.AssertTableEqual(t, statsTblBefore, statsTblAfter) // Case 3: Feedback is still effective on version 1 statistics. testKit.MustExec("set tidb_analyze_version = 1")