diff --git a/executor/hot_regions_history_table_test.go b/executor/hot_regions_history_table_test.go index 6e5fc3981305c..e53cd3e7c84ce 100644 --- a/executor/hot_regions_history_table_test.go +++ b/executor/hot_regions_history_table_test.go @@ -38,7 +38,7 @@ import ( "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/external" "github.com/pingcap/tidb/util/pdapi" - "github.com/stretchr/testify/suite" + "github.com/stretchr/testify/require" ) type mockStoreWithMultiPD struct { @@ -55,19 +55,17 @@ func (s *mockStoreWithMultiPD) Name() string { return "mockStore func (s *mockStoreWithMultiPD) Describe() string { return "" } type hotRegionsHistoryTableSuite struct { - suite.Suite store kv.Storage clean func() httpServers []*httptest.Server startTime time.Time } -func TestHotRegionsHistoryTable(t *testing.T) { - suite.Run(t, new(hotRegionsHistoryTableSuite)) -} +func createHotRegionsHistoryTableSuite(t *testing.T) *hotRegionsHistoryTableSuite { + var clean func() -func (s *hotRegionsHistoryTableSuite) SetupSuite() { - s.store, s.clean = testkit.CreateMockStore(s.T()) + s := new(hotRegionsHistoryTableSuite) + s.store, clean = testkit.CreateMockStore(t) store := &mockStoreWithMultiPD{ s.store.(helper.Storage), make([]string, 3), @@ -75,12 +73,19 @@ func (s *hotRegionsHistoryTableSuite) SetupSuite() { // start 3 PD server with hotRegionsServer and store them in s.store for i := 0; i < 3; i++ { httpServer, mockAddr := s.setUpMockPDHTTPServer() - s.Require().NotNil(httpServer) + require.NotNil(t, httpServer) s.httpServers = append(s.httpServers, httpServer) store.hosts[i] = mockAddr } s.store = store s.startTime = time.Now() + s.clean = func() { + for _, server := range s.httpServers { + server.Close() + } + clean() + } + return s } func writeResp(w http.ResponseWriter, resp interface{}) { @@ -153,24 +158,20 @@ func (s *hotRegionsHistoryTableSuite) setUpMockPDHTTPServer() (*httptest.Server, return server, mockAddr } -func (s *hotRegionsHistoryTableSuite) TearDownSuite() { - for _, server := range s.httpServers { - server.Close() - } - s.clean() -} +func TestTiDBHotRegionsHistory(t *testing.T) { + s := createHotRegionsHistoryTableSuite(t) + defer s.clean() -func (s *hotRegionsHistoryTableSuite) TestTiDBHotRegionsHistory() { var unixTimeMs = func(v string) int64 { - t, err := time.ParseInLocation("2006-01-02 15:04:05", v, time.Local) - s.Require().NoError(err) - return t.UnixNano() / int64(time.Millisecond) + tt, err := time.ParseInLocation("2006-01-02 15:04:05", v, time.Local) + require.NoError(t, err) + return tt.UnixNano() / int64(time.Millisecond) } - tk := testkit.NewTestKit(s.T(), s.store) - tablesPrivTid := external.GetTableByName(s.T(), tk, "mysql", "TABLES_PRIV").Meta().ID + tk := testkit.NewTestKit(t, s.store) + tablesPrivTid := external.GetTableByName(t, tk, "mysql", "TABLES_PRIV").Meta().ID tablesPrivTidStr := strconv.FormatInt(tablesPrivTid, 10) - statsMetaTid := external.GetTableByName(s.T(), tk, "mysql", "STATS_META").Meta().ID + statsMetaTid := external.GetTableByName(t, tk, "mysql", "STATS_META").Meta().ID statsMetaTidStr := strconv.FormatInt(statsMetaTid, 10) fullHotRegions := [][]string{ @@ -496,7 +497,7 @@ func (s *hotRegionsHistoryTableSuite) TestTiDBHotRegionsHistory() { } result := tk.MustQuery(sql) warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - s.Require().Lenf(warnings, 0, "unexpected warnings: %+v, sql: %s", warnings, sql) + require.Len(t, warnings, 0, fmt.Sprintf("unexpected warnings: %+v, sql: %s", warnings, sql)) var expected []string for _, row := range cas.expected { expectedRow := row @@ -506,20 +507,23 @@ func (s *hotRegionsHistoryTableSuite) TestTiDBHotRegionsHistory() { } } -func (s *hotRegionsHistoryTableSuite) TestTiDBHotRegionsHistoryError() { - tk := testkit.NewTestKit(s.T(), s.store) +func TestTiDBHotRegionsHistoryError(t *testing.T) { + s := createHotRegionsHistoryTableSuite(t) + defer s.clean() + + tk := testkit.NewTestKit(t, s.store) // Test without start time error rs, err := tk.Exec("select * from information_schema.tidb_hot_regions_history") - s.Require().NoError(err) + require.NoError(t, err) _, err = session.ResultSetToStringSlice(context.Background(), tk.Session(), rs) - s.Require().EqualError(err, "denied to scan hot regions, please specified the start time, such as `update_time > '2020-01-01 00:00:00'`") - s.NoError(rs.Close()) + require.EqualError(t, err, "denied to scan hot regions, please specified the start time, such as `update_time > '2020-01-01 00:00:00'`") + require.NoError(t, rs.Close()) // Test without end time error. rs, err = tk.Exec("select * from information_schema.tidb_hot_regions_history where update_time>='2019/08/26 06:18:13.011'") - s.Require().NoError(err) + require.NoError(t, err) _, err = session.ResultSetToStringSlice(context.Background(), tk.Session(), rs) - s.Require().EqualError(err, "denied to scan hot regions, please specified the end time, such as `update_time < '2020-01-01 00:00:00'`") - s.NoError(rs.Close()) + require.EqualError(t, err, "denied to scan hot regions, please specified the end time, such as `update_time < '2020-01-01 00:00:00'`") + require.NoError(t, rs.Close()) } diff --git a/executor/infoschema_cluster_table_test.go b/executor/infoschema_cluster_table_test.go index 95ab9f271b538..c790919a17004 100644 --- a/executor/infoschema_cluster_table_test.go +++ b/executor/infoschema_cluster_table_test.go @@ -38,12 +38,11 @@ import ( "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/pdapi" - "github.com/stretchr/testify/suite" + "github.com/stretchr/testify/require" "google.golang.org/grpc" ) type infosSchemaClusterTableSuite struct { - suite.Suite store kv.Storage dom *domain.Domain clean func() @@ -54,20 +53,26 @@ type infosSchemaClusterTableSuite struct { startTime time.Time } -func TestInfoSchemaClusterTable(t *testing.T) { - suite.Run(t, new(infosSchemaClusterTableSuite)) -} +func createInfosSchemaClusterTableSuite(t *testing.T) *infosSchemaClusterTableSuite { + var clean func() -func (s *infosSchemaClusterTableSuite) SetupSuite() { - s.store, s.dom, s.clean = testkit.CreateMockStoreAndDomain(s.T()) - s.rpcServer, s.listenAddr = s.setUpRPCService("127.0.0.1:0") + s := new(infosSchemaClusterTableSuite) + s.store, s.dom, clean = testkit.CreateMockStoreAndDomain(t) + s.rpcServer, s.listenAddr = setUpRPCService(t, s.dom, "127.0.0.1:0") s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer() s.startTime = time.Now() + s.clean = func() { + s.rpcServer.Stop() + s.httpServer.Close() + clean() + } + + return s } -func (s *infosSchemaClusterTableSuite) setUpRPCService(addr string) (*grpc.Server, string) { +func setUpRPCService(t *testing.T, dom *domain.Domain, addr string) (*grpc.Server, string) { lis, err := net.Listen("tcp", addr) - s.Require().NoError(err) + require.NoError(t, err) // Fix issue 9836 sm := &mockSessionManager{ @@ -80,11 +85,11 @@ func (s *infosSchemaClusterTableSuite) setUpRPCService(addr string) (*grpc.Serve Host: "127.0.0.1", Command: mysql.ComQuery, } - srv := server.NewRPCServer(config.GetGlobalConfig(), s.dom, sm) + srv := server.NewRPCServer(config.GetGlobalConfig(), dom, sm) port := lis.Addr().(*net.TCPAddr).Port addr = fmt.Sprintf("127.0.0.1:%d", port) go func() { - s.Require().NoError(srv.Serve(lis)) + require.NoError(t, srv.Serve(lis)) }() config.UpdateGlobal(func(conf *config.Config) { conf.Status.StatusPort = uint(port) @@ -165,12 +170,6 @@ func (s *infosSchemaClusterTableSuite) setUpMockPDHTTPServer() (*httptest.Server return srv, mockAddr } -func (s *infosSchemaClusterTableSuite) TearDownSuite() { - s.rpcServer.Stop() - s.httpServer.Close() - s.clean() -} - type mockSessionManager struct { processInfoMap map[uint64]*util.ProcessInfo serverID uint64 @@ -189,10 +188,10 @@ func (sm *mockSessionManager) GetProcessInfo(id uint64) (*util.ProcessInfo, bool return rs, ok } -func (sm *mockSessionManager) StoreInternalSession(se interface{}) { +func (sm *mockSessionManager) StoreInternalSession(_ interface{}) { } -func (sm *mockSessionManager) DeleteInternalSession(se interface{}) { +func (sm *mockSessionManager) DeleteInternalSession(_ interface{}) { } func (sm *mockSessionManager) GetInternalSessionStartTSList() []uint64 { @@ -224,7 +223,10 @@ func (s *mockStore) StartGCWorker() error { panic("not implemented") } func (s *mockStore) Name() string { return "mockStore" } func (s *mockStore) Describe() string { return "" } -func (s *infosSchemaClusterTableSuite) TestTiDBClusterInfo() { +func TestTiDBClusterInfo(t *testing.T) { + s := createInfosSchemaClusterTableSuite(t) + defer s.clean() + mockAddr := s.mockAddr store := &mockStore{ s.store.(helper.Storage), @@ -232,7 +234,7 @@ func (s *infosSchemaClusterTableSuite) TestTiDBClusterInfo() { } // information_schema.cluster_info - tk := testkit.NewTestKit(s.T(), store) + tk := testkit.NewTestKit(t, store) tidbStatusAddr := fmt.Sprintf(":%d", config.GetGlobalConfig().Status.StatusPort) row := func(cols ...string) string { return strings.Join(cols, " ") } tk.MustQuery("select type, instance, status_address, version, git_hash from information_schema.cluster_info").Check(testkit.Rows( @@ -246,9 +248,9 @@ func (s *infosSchemaClusterTableSuite) TestTiDBClusterInfo() { row("tikv", "store1", ""), )) - s.Require().NoError(failpoint.Enable("github.com/pingcap/tidb/infoschema/mockStoreTombstone", `return(true)`)) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockStoreTombstone", `return(true)`)) tk.MustQuery("select type, instance, start_time from information_schema.cluster_info where type = 'tikv'").Check(testkit.Rows()) - s.Require().NoError(failpoint.Disable("github.com/pingcap/tidb/infoschema/mockStoreTombstone")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/infoschema/mockStoreTombstone")) // information_schema.cluster_config instances := []string{ @@ -257,8 +259,8 @@ func (s *infosSchemaClusterTableSuite) TestTiDBClusterInfo() { "tikv,127.0.0.1:11080," + mockAddr + ",mock-version,mock-githash,0", } fpExpr := `return("` + strings.Join(instances, ";") + `")` - s.Require().NoError(failpoint.Enable("github.com/pingcap/tidb/infoschema/mockClusterInfo", fpExpr)) - defer func() { s.Require().NoError(failpoint.Disable("github.com/pingcap/tidb/infoschema/mockClusterInfo")) }() + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockClusterInfo", fpExpr)) + defer func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/infoschema/mockClusterInfo")) }() tk.MustQuery("select type, instance, status_address, version, git_hash, server_id from information_schema.cluster_info").Check(testkit.Rows( row("pd", "127.0.0.1:11080", mockAddr, "mock-version", "mock-githash", "0"), row("tidb", "127.0.0.1:11080", mockAddr, "mock-version", "mock-githash", "1001"), @@ -294,10 +296,13 @@ func (s *infosSchemaClusterTableSuite) TestTiDBClusterInfo() { )) } -func (s *infosSchemaClusterTableSuite) TestTableStorageStats() { - tk := testkit.NewTestKit(s.T(), s.store) +func TestTableStorageStats(t *testing.T) { + s := createInfosSchemaClusterTableSuite(t) + defer s.clean() + + tk := testkit.NewTestKit(t, s.store) err := tk.QueryToErr("select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'test'") - s.Require().EqualError(err, "pd unavailable") + require.EqualError(t, err, "pd unavailable") mockAddr := s.mockAddr store := &mockStore{ s.store.(helper.Storage), @@ -305,11 +310,11 @@ func (s *infosSchemaClusterTableSuite) TestTableStorageStats() { } // Test information_schema.TABLE_STORAGE_STATS. - tk = testkit.NewTestKit(s.T(), store) + tk = testkit.NewTestKit(t, store) // Test not set the schema. err = tk.QueryToErr("select * from information_schema.TABLE_STORAGE_STATS") - s.Require().EqualError(err, "Please specify the 'table_schema'") + require.EqualError(t, err, "Please specify the 'table_schema'") // Test it would get null set when get the sys schema. tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema';").Check([][]interface{}{}) @@ -331,20 +336,20 @@ func (s *infosSchemaClusterTableSuite) TestTableStorageStats() { )) rows := tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql';").Rows() result := 32 - s.Require().Len(rows, result) + require.Len(t, rows, result) // More tests about the privileges. tk.MustExec("create user 'testuser'@'localhost'") tk.MustExec("create user 'testuser2'@'localhost'") tk.MustExec("create user 'testuser3'@'localhost'") - tk1 := testkit.NewTestKit(s.T(), store) + tk1 := testkit.NewTestKit(t, store) defer tk1.MustExec("drop user 'testuser'@'localhost'") defer tk1.MustExec("drop user 'testuser2'@'localhost'") defer tk1.MustExec("drop user 'testuser3'@'localhost'") tk.MustExec("grant all privileges on *.* to 'testuser2'@'localhost'") tk.MustExec("grant select on *.* to 'testuser3'@'localhost'") - s.Require().True(tk.Session().Auth(&auth.UserIdentity{ + require.True(t, tk.Session().Auth(&auth.UserIdentity{ Username: "testuser", Hostname: "localhost", }, nil, nil)) @@ -352,14 +357,14 @@ func (s *infosSchemaClusterTableSuite) TestTableStorageStats() { // User has no access to this schema, so the result set is empty. tk.MustQuery("select count(1) from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql'").Check(testkit.Rows("0")) - s.Require().True(tk.Session().Auth(&auth.UserIdentity{ + require.True(t, tk.Session().Auth(&auth.UserIdentity{ Username: "testuser2", Hostname: "localhost", }, nil, nil)) tk.MustQuery("select count(1) from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql'").Check(testkit.Rows(strconv.Itoa(result))) - s.Require().True(tk.Session().Auth(&auth.UserIdentity{ + require.True(t, tk.Session().Auth(&auth.UserIdentity{ Username: "testuser3", Hostname: "localhost", }, nil, nil))