diff --git a/client/option_test.go b/client/option_test.go index 1f8c61b0f42..c5452dbfc0c 100644 --- a/client/option_test.go +++ b/client/option_test.go @@ -47,7 +47,7 @@ func (s *testClientSuite) TestDynamicOptionChange(c *C) { expectBool := true o.setEnableTSOFollowerProxy(expectBool) // Check the value changing notification. - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { <-o.enableTSOFollowerProxyCh return true }) diff --git a/pkg/mock/mockhbstream/mockhbstream_test.go b/pkg/mock/mockhbstream/mockhbstream_test.go index 0e5cd02d7ea..e6d05f19d1b 100644 --- a/pkg/mock/mockhbstream/mockhbstream_test.go +++ b/pkg/mock/mockhbstream/mockhbstream_test.go @@ -66,13 +66,13 @@ func (s *testHeartbeatStreamSuite) TestActivity(c *C) { // Active stream is stream1. hbs.BindStream(1, stream1) - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { hbs.SendMsg(region, proto.Clone(msg).(*pdpb.RegionHeartbeatResponse)) return stream1.Recv() != nil && stream2.Recv() == nil }) // Rebind to stream2. hbs.BindStream(1, stream2) - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { hbs.SendMsg(region, proto.Clone(msg).(*pdpb.RegionHeartbeatResponse)) return stream1.Recv() == nil && stream2.Recv() != nil }) @@ -83,7 +83,7 @@ func (s *testHeartbeatStreamSuite) TestActivity(c *C) { c.Assert(res.GetHeader().GetError(), NotNil) // Switch back to 1 again. hbs.BindStream(1, stream1) - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { hbs.SendMsg(region, proto.Clone(msg).(*pdpb.RegionHeartbeatResponse)) return stream1.Recv() != nil && stream2.Recv() == nil }) diff --git a/pkg/testutil/testutil.go b/pkg/testutil/testutil.go index 85e82ca9c11..c3c917d7b3a 100644 --- a/pkg/testutil/testutil.go +++ b/pkg/testutil/testutil.go @@ -31,7 +31,7 @@ const ( // CheckFunc is a condition checker that passed to WaitUntil. Its implementation // may call c.Fatal() to abort the test, or c.Log() to add more information. -type CheckFunc func(c *check.C) bool +type CheckFunc func() bool // WaitOp represents available options when execute WaitUntil type WaitOp struct { @@ -63,7 +63,7 @@ func WaitUntil(c *check.C, f CheckFunc, opts ...WaitOption) { opt(option) } for i := 0; i < option.retryTimes; i++ { - if f(c) { + if f() { return } time.Sleep(option.sleepInterval) diff --git a/server/api/server_test.go b/server/api/server_test.go index 15d0b4d652d..4e62aa86d39 100644 --- a/server/api/server_test.go +++ b/server/api/server_test.go @@ -123,7 +123,7 @@ func mustNewCluster(c *C, num int, opts ...func(cfg *config.Config)) ([]*config. } func mustWaitLeader(c *C, svrs []*server.Server) { - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { var leader *pdpb.Member for _, svr := range svrs { l := svr.GetLeader() diff --git a/server/api/tso_test.go b/server/api/tso_test.go index b11e98f04d2..59039068d24 100644 --- a/server/api/tso_test.go +++ b/server/api/tso_test.go @@ -48,7 +48,7 @@ func (s *testTsoSuite) TearDownSuite(c *C) { } func (s *testTsoSuite) TestTransferAllocator(c *C) { - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { s.svr.GetTSOAllocatorManager().ClusterDCLocationChecker() _, err := s.svr.GetTSOAllocatorManager().GetAllocator("dc-1") return err == nil diff --git a/server/cluster/coordinator_test.go b/server/cluster/coordinator_test.go index 985bfd86ced..708624688b7 100644 --- a/server/cluster/coordinator_test.go +++ b/server/cluster/coordinator_test.go @@ -894,7 +894,7 @@ func BenchmarkPatrolRegion(b *testing.B) { } func waitOperator(c *C, co *coordinator, regionID uint64) { - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { return co.opController.GetOperator(regionID) != nil }) } @@ -1206,7 +1206,7 @@ func (s *testScheduleControllerSuite) TestInterval(c *C) { func waitAddLearner(c *C, stream mockhbstream.HeartbeatStream, region *core.RegionInfo, storeID uint64) *core.RegionInfo { var res *pdpb.RegionHeartbeatResponse - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { if res = stream.Recv(); res != nil { return res.GetRegionId() == region.GetID() && res.GetChangePeer().GetChangeType() == eraftpb.ConfChangeType_AddLearnerNode && @@ -1222,7 +1222,7 @@ func waitAddLearner(c *C, stream mockhbstream.HeartbeatStream, region *core.Regi func waitPromoteLearner(c *C, stream mockhbstream.HeartbeatStream, region *core.RegionInfo, storeID uint64) *core.RegionInfo { var res *pdpb.RegionHeartbeatResponse - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { if res = stream.Recv(); res != nil { return res.GetRegionId() == region.GetID() && res.GetChangePeer().GetChangeType() == eraftpb.ConfChangeType_AddNode && @@ -1239,7 +1239,7 @@ func waitPromoteLearner(c *C, stream mockhbstream.HeartbeatStream, region *core. func waitRemovePeer(c *C, stream mockhbstream.HeartbeatStream, region *core.RegionInfo, storeID uint64) *core.RegionInfo { var res *pdpb.RegionHeartbeatResponse - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { if res = stream.Recv(); res != nil { return res.GetRegionId() == region.GetID() && res.GetChangePeer().GetChangeType() == eraftpb.ConfChangeType_RemoveNode && @@ -1255,7 +1255,7 @@ func waitRemovePeer(c *C, stream mockhbstream.HeartbeatStream, region *core.Regi func waitTransferLeader(c *C, stream mockhbstream.HeartbeatStream, region *core.RegionInfo, storeID uint64) *core.RegionInfo { var res *pdpb.RegionHeartbeatResponse - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { if res = stream.Recv(); res != nil { if res.GetRegionId() == region.GetID() { for _, peer := range append(res.GetTransferLeader().GetPeers(), res.GetTransferLeader().GetPeer()) { @@ -1273,7 +1273,7 @@ func waitTransferLeader(c *C, stream mockhbstream.HeartbeatStream, region *core. } func waitNoResponse(c *C, stream mockhbstream.HeartbeatStream) { - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { res := stream.Recv() return res == nil }) diff --git a/server/server_test.go b/server/server_test.go index b0ae5ff082f..f433ac2dc31 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -43,7 +43,7 @@ func TestMain(m *testing.M) { func mustWaitLeader(c *C, svrs []*Server) *Server { var leader *Server - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { for _, s := range svrs { if !s.IsClosed() && s.member.IsLeader() { leader = s diff --git a/tests/client/client_test.go b/tests/client/client_test.go index de8c5383e66..14207ce4242 100644 --- a/tests/client/client_test.go +++ b/tests/client/client_test.go @@ -90,7 +90,7 @@ func (s *clientTestSuite) TestClientLeaderChange(c *C) { cli := setupCli(c, s.ctx, endpoints) var ts1, ts2 uint64 - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { p1, l1, err := cli.GetTS(context.TODO()) if err == nil { ts1 = tsoutil.ComposeTS(p1, l1) @@ -111,7 +111,7 @@ func (s *clientTestSuite) TestClientLeaderChange(c *C) { waitLeader(c, cli.(client), cluster.GetServer(leader).GetConfig().ClientUrls) // Check TS won't fall back after leader changed. - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { p2, l2, err := cli.GetTS(context.TODO()) if err == nil { ts2 = tsoutil.ComposeTS(p2, l2) @@ -140,7 +140,7 @@ func (s *clientTestSuite) TestLeaderTransfer(c *C) { cli := setupCli(c, s.ctx, endpoints) var lastTS uint64 - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { physical, logical, err := cli.GetTS(context.TODO()) if err == nil { lastTS = tsoutil.ComposeTS(physical, logical) @@ -217,7 +217,7 @@ func (s *clientTestSuite) TestTSOAllocatorLeader(c *C) { var allocatorLeaderMap = make(map[string]string) for _, dcLocation := range dcLocationConfig { var pdName string - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { pdName = cluster.WaitAllocatorLeader(dcLocation) return len(pdName) > 0 }) @@ -423,7 +423,7 @@ func (s *clientTestSuite) TestGetTsoFromFollowerClient1(c *C) { c.Assert(failpoint.Enable("github.com/tikv/pd/client/unreachableNetwork", "return(true)"), IsNil) var lastTS uint64 - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { physical, logical, err := cli.GetTS(context.TODO()) if err == nil { lastTS = tsoutil.ComposeTS(physical, logical) @@ -451,7 +451,7 @@ func (s *clientTestSuite) TestGetTsoFromFollowerClient2(c *C) { c.Assert(failpoint.Enable("github.com/tikv/pd/client/unreachableNetwork", "return(true)"), IsNil) var lastTS uint64 - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { physical, logical, err := cli.GetTS(context.TODO()) if err == nil { lastTS = tsoutil.ComposeTS(physical, logical) @@ -506,7 +506,7 @@ func setupCli(c *C, ctx context.Context, endpoints []string, opts ...pd.ClientOp } func waitLeader(c *C, cli client, leader string) { - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { cli.ScheduleCheckLeader() return cli.GetLeaderAddr() == leader }) @@ -721,7 +721,7 @@ func (s *testClientSuite) TestGetRegion(c *C) { err := s.regionHeartbeat.Send(req) c.Assert(err, IsNil) - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { r, err := s.client.GetRegion(context.Background(), []byte("a")) c.Assert(err, IsNil) if r == nil { @@ -759,7 +759,7 @@ func (s *testClientSuite) TestGetPrevRegion(c *C) { } time.Sleep(500 * time.Millisecond) for i := 0; i < 20; i++ { - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { r, err := s.client.GetPrevRegion(context.Background(), []byte{byte(i)}) c.Assert(err, IsNil) if i > 0 && i < regionLen { @@ -798,7 +798,7 @@ func (s *testClientSuite) TestScanRegions(c *C) { } // Wait for region heartbeats. - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { scanRegions, err := s.client.ScanRegions(context.Background(), []byte{0}, nil, 10) return err == nil && len(scanRegions) == 10 }) @@ -865,7 +865,7 @@ func (s *testClientSuite) TestGetRegionByID(c *C) { err := s.regionHeartbeat.Send(req) c.Assert(err, IsNil) - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { r, err := s.client.GetRegionByID(context.Background(), regionID) c.Assert(err, IsNil) if r == nil { @@ -1146,7 +1146,7 @@ func (s *testClientSuite) TestScatterRegion(c *C) { err := s.regionHeartbeat.Send(req) regionsID := []uint64{regionID} c.Assert(err, IsNil) - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { scatterResp, err := s.client.ScatterRegions(context.Background(), regionsID, pd.WithGroup("test"), pd.WithRetry(1)) if c.Check(err, NotNil) { return false diff --git a/tests/cluster.go b/tests/cluster.go index 4d42ee38806..27bc32b1aba 100644 --- a/tests/cluster.go +++ b/tests/cluster.go @@ -611,7 +611,7 @@ func (c *TestCluster) WaitAllLeaders(testC *check.C, dcLocations map[string]stri for _, dcLocation := range dcLocations { wg.Add(1) go func(dc string) { - testutil.WaitUntil(testC, func(testC *check.C) bool { + testutil.WaitUntil(testC, func() bool { leaderName := c.WaitAllocatorLeader(dc) return leaderName != "" }) diff --git a/tests/pdctl/member/member_test.go b/tests/pdctl/member/member_test.go index 6a0d72d7648..1dac74ce6a4 100644 --- a/tests/pdctl/member/member_test.go +++ b/tests/pdctl/member/member_test.go @@ -74,7 +74,7 @@ func (s *memberTestSuite) TestMember(c *C) { args = []string{"-u", pdAddr, "member", "leader", "transfer", "pd2"} _, err = pdctl.ExecuteCommand(cmd, args...) c.Assert(err, IsNil) - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { return c.Check("pd2", Equals, svr.GetLeader().GetName()) }) @@ -84,7 +84,7 @@ func (s *memberTestSuite) TestMember(c *C) { output, err = pdctl.ExecuteCommand(cmd, args...) c.Assert(strings.Contains(string(output), "Success"), IsTrue) c.Assert(err, IsNil) - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { return c.Check("pd2", Not(Equals), svr.GetLeader().GetName()) }) diff --git a/tests/server/api/api_test.go b/tests/server/api/api_test.go index 299530f0924..6afac8da3d9 100644 --- a/tests/server/api/api_test.go +++ b/tests/server/api/api_test.go @@ -87,7 +87,7 @@ func (s *serverTestSuite) TestReconnect(c *C) { // Make sure they proxy requests to the new leader. for name, s := range cluster.GetServers() { if name != leader { - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { res, e := http.Get(s.GetConfig().AdvertiseClientUrls + "/pd/api/v1/version") c.Assert(e, IsNil) defer res.Body.Close() @@ -103,7 +103,7 @@ func (s *serverTestSuite) TestReconnect(c *C) { // Request will fail with no leader. for name, s := range cluster.GetServers() { if name != leader && name != newLeader { - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { res, err := http.Get(s.GetConfig().AdvertiseClientUrls + "/pd/api/v1/version") c.Assert(err, IsNil) defer res.Body.Close() diff --git a/tests/server/member/member_test.go b/tests/server/member/member_test.go index a5df2173c2a..2eb2ef19ddd 100644 --- a/tests/server/member/member_test.go +++ b/tests/server/member/member_test.go @@ -111,7 +111,7 @@ func (s *memberTestSuite) TestMemberDelete(c *C) { httpClient := &http.Client{Timeout: 15 * time.Second} for _, t := range table { c.Log(time.Now(), "try to delete:", t.path) - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { addr := leader.GetConfig().ClientUrls + "/pd/api/v1/members/" + t.path req, err := http.NewRequest(http.MethodDelete, addr, nil) c.Assert(err, IsNil) @@ -187,7 +187,7 @@ func (s *memberTestSuite) TestLeaderPriority(c *C) { server1 := cluster.GetServer(leader1) addr := server1.GetConfig().ClientUrls // PD leader should sync with etcd leader. - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { return cluster.GetLeader() == leader1 }) // Bind a lower priority to current leader. @@ -195,13 +195,13 @@ func (s *memberTestSuite) TestLeaderPriority(c *C) { // Wait etcd leader change. leader2 := s.waitEtcdLeaderChange(c, server1, leader1) // PD leader should sync with etcd leader again. - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { return cluster.GetLeader() == leader2 }) } func (s *memberTestSuite) post(c *C, url string, body string) { - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { res, err := http.Post(url, "", bytes.NewBufferString(body)) // #nosec c.Assert(err, IsNil) b, err := io.ReadAll(res.Body) @@ -214,7 +214,7 @@ func (s *memberTestSuite) post(c *C, url string, body string) { func (s *memberTestSuite) waitEtcdLeaderChange(c *C, server *tests.TestServer, old string) string { var leader string - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { var err error leader, err = server.GetEtcdLeader() if err != nil { @@ -271,7 +271,7 @@ func (s *memberTestSuite) TestLeaderResignWithBlock(c *C) { func (s *memberTestSuite) waitLeaderChange(c *C, cluster *tests.TestCluster, old string) string { var leader string - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { leader = cluster.GetLeader() if leader == old || leader == "" { return false @@ -383,7 +383,7 @@ func (s *leaderTestSuite) sendRequest(c *C, addr string) { func mustWaitLeader(c *C, svrs []*server.Server) *server.Server { var leader *server.Server - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { for _, s := range svrs { if !s.IsClosed() && s.GetMember().IsLeader() { leader = s diff --git a/tests/server/region_syncer/region_syncer_test.go b/tests/server/region_syncer/region_syncer_test.go index aa026bc3187..3a99fdabc52 100644 --- a/tests/server/region_syncer/region_syncer_test.go +++ b/tests/server/region_syncer/region_syncer_test.go @@ -140,7 +140,7 @@ func (s *regionSyncerTestSuite) TestRegionSyncer(c *C) { c.Assert(followerServer, NotNil) cacheRegions := leaderServer.GetServer().GetBasicCluster().GetRegions() c.Assert(cacheRegions, HasLen, regionLen) - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { for _, region := range cacheRegions { r := followerServer.GetServer().GetBasicCluster().GetRegion(region.GetID()) if !(c.Check(r.GetMeta(), DeepEquals, region.GetMeta()) && diff --git a/tests/server/server_test.go b/tests/server/server_test.go index e0e2dff7b8e..56b357f7fcd 100644 --- a/tests/server/server_test.go +++ b/tests/server/server_test.go @@ -138,7 +138,7 @@ func (s *serverTestSuite) TestLeader(c *C) { err = cluster.GetServer(leader1).Stop() c.Assert(err, IsNil) - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { leader := cluster.GetLeader() return leader != leader1 }) diff --git a/tests/server/tso/allocator_test.go b/tests/server/tso/allocator_test.go index 44a450409b1..b3b2002f2b0 100644 --- a/tests/server/tso/allocator_test.go +++ b/tests/server/tso/allocator_test.go @@ -156,7 +156,7 @@ func (s *testAllocatorSuite) TestPriorityAndDifferentLocalTSO(c *C) { c.Assert(err, IsNil) dcLocationConfig["pd4"] = "dc-4" cluster.CheckClusterDCLocation() - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { leaderName := cluster.WaitAllocatorLeader("dc-4") return leaderName != "" }) @@ -180,7 +180,7 @@ func (s *testAllocatorSuite) TestPriorityAndDifferentLocalTSO(c *C) { for serverName, dcLocation := range dcLocationConfig { go func(serName, dc string) { defer wg.Done() - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { leaderName := cluster.WaitAllocatorLeader(dc) return leaderName == serName }, testutil.WithRetryTimes(12), testutil.WithSleepInterval(5*time.Second)) @@ -231,7 +231,7 @@ func (s *testAllocatorSuite) testTSOSuffix(c *C, cluster *tests.TestCluster, am allocator, err := am.GetAllocator(dcLocation) c.Assert(err, IsNil) var tso pdpb.Timestamp - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { tso, err = allocator.GenerateTSO(1) c.Assert(err, IsNil) return tso.GetPhysical() != 0 diff --git a/tests/server/tso/consistency_test.go b/tests/server/tso/consistency_test.go index a912146d116..974c0bb71c2 100644 --- a/tests/server/tso/consistency_test.go +++ b/tests/server/tso/consistency_test.go @@ -345,7 +345,7 @@ func (s *testTSOConsistencySuite) TestLocalTSOAfterMemberChanged(c *C) { c.Assert(err, IsNil) dcLocationConfig["pd4"] = "dc-4" cluster.CheckClusterDCLocation() - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { leaderName := cluster.WaitAllocatorLeader("dc-4") return leaderName != "" }) diff --git a/tests/server/tso/manager_test.go b/tests/server/tso/manager_test.go index 7679cc17685..68edc073445 100644 --- a/tests/server/tso/manager_test.go +++ b/tests/server/tso/manager_test.go @@ -190,7 +190,7 @@ func (s *testManagerSuite) TestNextLeaderKey(c *C) { } err := server.GetTSOAllocatorManager().TransferAllocatorForDCLocation("dc-1", server.GetServer().GetMember().ID()) c.Assert(err, IsNil) - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { cluster.CheckClusterDCLocation() currName := cluster.WaitAllocatorLeader("dc-1") return currName == name diff --git a/tests/server/watch/leader_watch_test.go b/tests/server/watch/leader_watch_test.go index c6bc9a12756..160e089ecb7 100644 --- a/tests/server/watch/leader_watch_test.go +++ b/tests/server/watch/leader_watch_test.go @@ -82,7 +82,7 @@ func (s *watchTestSuite) TestWatcher(c *C) { cluster.WaitLeader() c.Assert(pd2.GetLeader().GetName(), Equals, pd2.GetConfig().Name) failpoint.Disable("github.com/tikv/pd/server/delayWatcher") - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { return c.Check(pd3.GetLeader().GetName(), Equals, pd2.GetConfig().Name) }) c.Succeed() @@ -107,7 +107,7 @@ func (s *watchTestSuite) TestWatcherCompacted(c *C) { c.Assert(err, IsNil) err = pd2.Run() c.Assert(err, IsNil) - testutil.WaitUntil(c, func(c *C) bool { + testutil.WaitUntil(c, func() bool { return c.Check(pd2.GetLeader().GetName(), Equals, pd1.GetConfig().Name) }) c.Succeed()