diff --git a/executor/BUILD.bazel b/executor/BUILD.bazel index d0025b488579f..059d198ce9f69 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -457,7 +457,6 @@ go_test( "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_pingcap_log//:log", "@com_github_pingcap_sysutil//:sysutil", - "@com_github_pingcap_tipb//go-binlog", "@com_github_pingcap_tipb//go-tipb", "@com_github_prometheus_client_golang//prometheus", "@com_github_prometheus_client_model//go", diff --git a/executor/executor_txn_test.go b/executor/executor_txn_test.go index d2bd8f862830e..cf078200321e2 100644 --- a/executor/executor_txn_test.go +++ b/executor/executor_txn_test.go @@ -15,7 +15,6 @@ package executor_test import ( - "context" "fmt" "strconv" "strings" @@ -25,9 +24,7 @@ import ( "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tipb/go-binlog" "github.com/stretchr/testify/require" - "google.golang.org/grpc" ) func TestInvalidReadTemporaryTable(t *testing.T) { @@ -769,7 +766,7 @@ func TestSavepointWithBinlog(t *testing.T) { tk := testkit.NewTestKit(t, store) // mock for binlog enabled. - tk.Session().GetSessionVars().BinlogClient = binloginfo.MockPumpsClient(&mockPumpClient{}) + tk.Session().GetSessionVars().BinlogClient = binloginfo.MockPumpsClient(&testkit.MockPumpClient{}) tk.MustExec("use test") tk.MustExec("create table t(id int, a int, unique index idx(id))") @@ -788,13 +785,3 @@ func TestSavepointWithBinlog(t *testing.T) { tk.MustExec("commit") tk.MustQuery("select * from t").Check(testkit.Rows("1 1")) } - -type mockPumpClient struct{} - -func (m mockPumpClient) WriteBinlog(ctx context.Context, in *binlog.WriteBinlogReq, opts ...grpc.CallOption) (*binlog.WriteBinlogResp, error) { - return &binlog.WriteBinlogResp{}, nil -} - -func (m mockPumpClient) PullBinlogs(ctx context.Context, in *binlog.PullBinlogReq, opts ...grpc.CallOption) (binlog.Pump_PullBinlogsClient, error) { - return nil, nil -} diff --git a/table/tables/BUILD.bazel b/table/tables/BUILD.bazel index 682919b06418e..6e0115b95fb6d 100644 --- a/table/tables/BUILD.bazel +++ b/table/tables/BUILD.bazel @@ -9,6 +9,7 @@ go_library( "partition.go", "state_remote.go", "tables.go", + "testutil.go", ], importpath = "github.com/pingcap/tidb/table/tables", visibility = ["//visibility:public"], @@ -61,20 +62,18 @@ go_library( go_test( name = "tables_test", - timeout = "long", + timeout = "short", srcs = [ "cache_test.go", - "export_test.go", "index_test.go", "main_test.go", "mutation_checker_test.go", - "partition_test.go", "state_remote_test.go", "tables_test.go", ], embed = [":tables"], flaky = True, - shard_count = 50, + shard_count = 36, deps = [ "//ddl", "//ddl/util/callback", @@ -91,7 +90,6 @@ go_test( "//parser/mysql", "//session", "//sessionctx", - "//sessionctx/binloginfo", "//sessionctx/stmtctx", "//sessionctx/variable", "//sessiontxn", @@ -105,20 +103,15 @@ go_test( "//util", "//util/codec", "//util/collate", - "//util/dbterror", - "//util/logutil", "//util/mock", "//util/rowcodec", "//util/stmtsummary", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", - "@com_github_pingcap_tipb//go-binlog", "@com_github_prometheus_client_model//go", "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", "@com_github_tikv_client_go_v2//oracle", - "@org_golang_google_grpc//:grpc", "@org_uber_go_goleak//:goleak", - "@org_uber_go_zap//:zap", ], ) diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index bd85589518cb4..9d4945eab2052 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -45,10 +45,8 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/rowcodec" - "github.com/pingcap/tipb/go-binlog" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "google.golang.org/grpc" ) func firstKey(t table.Table) kv.Key { @@ -81,16 +79,6 @@ func seek(t table.PhysicalTable, ctx sessionctx.Context, h kv.Handle) (kv.Handle return handle, true, nil } -type mockPumpClient struct{} - -func (m mockPumpClient) WriteBinlog(ctx context.Context, in *binlog.WriteBinlogReq, opts ...grpc.CallOption) (*binlog.WriteBinlogResp, error) { - return &binlog.WriteBinlogResp{}, nil -} - -func (m mockPumpClient) PullBinlogs(ctx context.Context, in *binlog.PullBinlogReq, opts ...grpc.CallOption) (binlog.Pump_PullBinlogsClient, error) { - return nil, nil -} - func TestBasic(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) diff --git a/table/tables/test/partition/BUILD.bazel b/table/tables/test/partition/BUILD.bazel new file mode 100644 index 0000000000000..92f560ede6da6 --- /dev/null +++ b/table/tables/test/partition/BUILD.bazel @@ -0,0 +1,32 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "partition_test", + timeout = "long", + srcs = [ + "main_test.go", + "partition_test.go", + ], + flaky = True, + shard_count = 35, + deps = [ + "//ddl", + "//domain", + "//errno", + "//kv", + "//parser/model", + "//sessionctx/binloginfo", + "//sessiontxn", + "//table", + "//table/tables", + "//testkit", + "//testkit/testsetup", + "//types", + "//util", + "//util/dbterror", + "//util/logutil", + "@com_github_stretchr_testify//require", + "@org_uber_go_goleak//:goleak", + "@org_uber_go_zap//:zap", + ], +) diff --git a/table/tables/test/partition/main_test.go b/table/tables/test/partition/main_test.go new file mode 100644 index 0000000000000..1eb8faa493b29 --- /dev/null +++ b/table/tables/test/partition/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 partition + +import ( + "testing" + + "github.com/pingcap/tidb/testkit/testsetup" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testsetup.SetupForCommonTest() + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/golang/glog.(*fileSink).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/table/tables/partition_test.go b/table/tables/test/partition/partition_test.go similarity index 99% rename from table/tables/partition_test.go rename to table/tables/test/partition/partition_test.go index 257a1049d0a89..7a266cd637c94 100644 --- a/table/tables/partition_test.go +++ b/table/tables/test/partition/partition_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package tables_test +package partition import ( "context" @@ -45,7 +45,7 @@ func TestPartitionBasic(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Session().GetSessionVars().BinlogClient = binloginfo.MockPumpsClient(mockPumpClient{}) + tk.Session().GetSessionVars().BinlogClient = binloginfo.MockPumpsClient(testkit.MockPumpClient{}) tk.MustExec("set @@session.tidb_enable_table_partition = '1'") tk.MustExec(`CREATE TABLE partition_basic (id int(11), unique index(id)) PARTITION BY RANGE COLUMNS ( id ) ( diff --git a/table/tables/export_test.go b/table/tables/testutil.go similarity index 92% rename from table/tables/export_test.go rename to table/tables/testutil.go index 5a8ee966351c2..f6d513253cab9 100644 --- a/table/tables/export_test.go +++ b/table/tables/testutil.go @@ -16,6 +16,7 @@ package tables import "github.com/pingcap/tidb/table" +// SwapReorgPartFields swaps the reorganizePartitions field of two partitioned tables. used in tests. func SwapReorgPartFields(src, dst table.Table) bool { s, ok := src.(*partitionedTable) if !ok { diff --git a/testkit/BUILD.bazel b/testkit/BUILD.bazel index aca65b9c5b78d..b6ec22fe930bc 100644 --- a/testkit/BUILD.bazel +++ b/testkit/BUILD.bazel @@ -39,11 +39,13 @@ go_library( "//util/sqlexec", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", + "@com_github_pingcap_tipb//go-binlog", "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", "@com_github_tikv_client_go_v2//tikv", "@com_github_tikv_client_go_v2//tikvrpc", "@io_opencensus_go//stats/view", + "@org_golang_google_grpc//:grpc", "@org_uber_go_atomic//:atomic", "@org_uber_go_zap//:zap", ], diff --git a/testkit/testkit.go b/testkit/testkit.go index 2cc4c01debb8f..3f492303293da 100644 --- a/testkit/testkit.go +++ b/testkit/testkit.go @@ -35,11 +35,13 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/intest" "github.com/pingcap/tidb/util/sqlexec" + "github.com/pingcap/tipb/go-binlog" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/tikvrpc" "go.uber.org/atomic" + "google.golang.org/grpc" ) var testKitIDGenerator atomic.Uint64 @@ -569,3 +571,16 @@ func (c *RegionProperityClient) SendRequest(ctx context.Context, addr string, re } return c.Client.SendRequest(ctx, addr, req, timeout) } + +// MockPumpClient is a mock pump client. +type MockPumpClient struct{} + +// WriteBinlog is a mock method. +func (m MockPumpClient) WriteBinlog(ctx context.Context, in *binlog.WriteBinlogReq, opts ...grpc.CallOption) (*binlog.WriteBinlogResp, error) { + return &binlog.WriteBinlogResp{}, nil +} + +// PullBinlogs is a mock method. +func (m MockPumpClient) PullBinlogs(ctx context.Context, in *binlog.PullBinlogReq, opts ...grpc.CallOption) (binlog.Pump_PullBinlogsClient, error) { + return nil, nil +}