From 2fab3f3ae5fba86d7b9934911c25230367be01b6 Mon Sep 17 00:00:00 2001 From: Piotr Tabor Date: Sun, 3 Apr 2022 09:16:55 +0200 Subject: [PATCH 1/6] Make naming of test-nodes consistent and positive: m0, m1, m2 The nodes used to be named: m-1, m0, m1, that was generating very confusing logs in integration tests. --- tests/framework/integration/cluster.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/framework/integration/cluster.go b/tests/framework/integration/cluster.go index a04796894..19618aff9 100644 --- a/tests/framework/integration/cluster.go +++ b/tests/framework/integration/cluster.go @@ -259,7 +259,7 @@ func (c *Cluster) mustNewMember(t testutil.TB) *Member { c.LastMemberNum++ m := MustNewMember(t, MemberConfig{ - Name: fmt.Sprintf("m%v", memberNumber-1), + Name: fmt.Sprintf("m%v", memberNumber), MemberNumber: memberNumber, AuthToken: c.Cfg.AuthToken, PeerTLS: c.Cfg.PeerTLS, From d57f8dba628a37418680adbce53340d5f7d31945 Mon Sep 17 00:00:00 2001 From: Piotr Tabor Date: Sun, 3 Apr 2022 09:17:10 +0200 Subject: [PATCH 2/6] Deflaking: Make WaitLeader (and WaitMembersForLeader) aggressively (30s) wait for leader being established. Nearly none of the tests was checking the value... just assuming WaitLeader success. ``` maintenance_test.go:277: Waiting for leader... logger.go:130: 2022-04-03T08:01:09.914+0200 INFO m0 cluster version differs from storage version. {"member": "m0", "cluster-version": "3.6.0", "storage-version": "3.5.0"} logger.go:130: 2022-04-03T08:01:09.915+0200 WARN m0 leader failed to send out heartbeat on time; took too long, leader is overloaded likely from slow disk {"member": "m0", "to": "2acc3d3b521981", "heartbeat-interval": "10ms", "expected-duration": "20ms", "exceeded-duration": "103.756219ms"} logger.go:130: 2022-04-03T08:01:09.916+0200 INFO m0 updated storage version {"member": "m0", "new-storage-version": "3.6.0"} ... logger.go:130: 2022-04-03T08:01:09.926+0200 INFO grpc [[roundrobin] roundrobinPicker: Build called with info: {map[0xc002630ac0:{{unix:localhost:m0 localhost 0 }} 0xc002630af0:{{unix:localhost:m1 localhost 0 }} 0xc002630b20:{{unix:localhost:m2 localhost 0 }}]}] logger.go:130: 2022-04-03T08:01:09.926+0200 WARN m0 apply request took too long {"member": "m0", "took": "114.661766ms", "expected-duration": "100ms", "prefix": "", "request": "header: cluster_version_set: ", "response": ""} logger.go:130: 2022-04-03T08:01:09.927+0200 INFO m0 cluster version is updated {"member": "m0", "cluster-version": "3.6"} logger.go:130: 2022-04-03T08:01:09.955+0200 INFO m2.raft 9f96af25a04e2ec3 [logterm: 2, index: 8, vote: 9903a56eaf96afac] ignored MsgVote from 2acc3d3b521981 [logterm: 2, index: 8] at term 2: lease is not expired (remaining ticks: 10) {"member": "m2"} logger.go:130: 2022-04-03T08:01:09.955+0200 INFO m0.raft 9903a56eaf96afac [logterm: 2, index: 8, vote: 9903a56eaf96afac] ignored MsgVote from 2acc3d3b521981 [logterm: 2, index: 8] at term 2: lease is not expired (remaining ticks: 5) {"member": "m0"} logger.go:130: 2022-04-03T08:01:09.955+0200 INFO m0.raft 9903a56eaf96afac [term: 2] received a MsgAppResp message with higher term from 2acc3d3b521981 [term: 3] {"member": "m0"} logger.go:130: 2022-04-03T08:01:09.955+0200 INFO m0.raft 9903a56eaf96afac became follower at term 3 {"member": "m0"} logger.go:130: 2022-04-03T08:01:09.955+0200 INFO m0.raft raft.node: 9903a56eaf96afac lost leader 9903a56eaf96afac at term 3 {"member": "m0"} maintenance_test.go:279: Leader established. ``` Tmp --- tests/framework/integration/cluster.go | 41 +++++++++++++++++-- .../integration/clientv3/maintenance_test.go | 5 +++ tests/integration/clientv3/watch_test.go | 10 +---- 3 files changed, 43 insertions(+), 13 deletions(-) diff --git a/tests/framework/integration/cluster.go b/tests/framework/integration/cluster.go index 19618aff9..b09d05470 100644 --- a/tests/framework/integration/cluster.go +++ b/tests/framework/integration/cluster.go @@ -398,12 +398,43 @@ func (c *Cluster) WaitMembersMatch(t testutil.TB, membs []*pb.Member) { } } -// WaitLeader returns index of the member in c.Members that is leader (or -1). -func (c *Cluster) WaitLeader(t testutil.TB) int { return c.WaitMembersForLeader(t, c.Members) } +// WaitLeader returns index of the member in c.Members that is leader +// or fails the test (if not established in 30min). +func (c *Cluster) WaitLeader(t testutil.TB) int { + return c.WaitMembersForLeader(t, c.Members) +} // WaitMembersForLeader waits until given members agree on the same leader, -// and returns its 'index' in the 'membs' list (or -1). +// and returns its 'index' in the 'membs' list func (c *Cluster) WaitMembersForLeader(t testutil.TB, membs []*Member) int { + t.Logf("WaitMembersForLeader") + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + l := 0 + for l = c.waitMembersForLeader(t, ctx, membs); l < 0; { + if ctx.Err() != nil { + t.Fatal("WaitLeader FAILED: %v", ctx.Err()) + } + } + t.Logf("WaitMembersForLeader succeeded. Cluster leader index: %v", l) + + // TODO: Consider second pass check as sometimes leadership is lost + // soon after election: + // + // We perform multiple attempts, as some-times just after successful WaitLLeader + // there is a race and leadership is quickly lost: + // - MsgAppResp message with higher term from 2acc3d3b521981 [term: 3] {"member": "m0"} + // - 9903a56eaf96afac became follower at term 3 {"member": "m0"} + // - 9903a56eaf96afac lost leader 9903a56eaf96afac at term 3 {"member": "m0"} + + + return l +} + +// WaitMembersForLeader waits until given members agree on the same leader, +// and returns its 'index' in the 'membs' list +func (c *Cluster) waitMembersForLeader(t testutil.TB, ctx context.Context, membs []*Member) int { + t.Logf("WaitMembersForLeader...") possibleLead := make(map[uint64]bool) var lead uint64 for _, m := range membs { @@ -415,7 +446,7 @@ func (c *Cluster) WaitMembersForLeader(t testutil.TB, membs []*Member) int { } // ensure leader is up via linearizable get for { - ctx, cancel := context.WithTimeout(context.Background(), 10*TickDuration+time.Second) + ctx, cancel := context.WithTimeout(ctx, 10*TickDuration+time.Second) _, err := cc.Get(ctx, "0") cancel() if err == nil || strings.Contains(err.Error(), "Key not found") { @@ -442,10 +473,12 @@ func (c *Cluster) WaitMembersForLeader(t testutil.TB, membs []*Member) int { for i, m := range membs { if uint64(m.Server.ID()) == lead { + t.Logf("WaitMembersForLeader found leader. Member: %v lead: %x", i, lead) return i } } + t.Logf("WaitMembersForLeader FAILED (-1)") return -1 } diff --git a/tests/integration/clientv3/maintenance_test.go b/tests/integration/clientv3/maintenance_test.go index ce1e629c0..75b134413 100644 --- a/tests/integration/clientv3/maintenance_test.go +++ b/tests/integration/clientv3/maintenance_test.go @@ -274,18 +274,22 @@ func TestMaintenanceStatus(t *testing.T) { clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3}) defer clus.Terminate(t) + t.Logf("Waiting for leader...") clus.WaitLeader(t) + t.Logf("Leader established.") eps := make([]string, 3) for i := 0; i < 3; i++ { eps[i] = clus.Members[i].GRPCURL() } + t.Logf("Creating client...") cli, err := integration2.NewClient(t, clientv3.Config{Endpoints: eps, DialOptions: []grpc.DialOption{grpc.WithBlock()}}) if err != nil { t.Fatal(err) } defer cli.Close() + t.Logf("Creating client [DONE]") prevID, leaderFound := uint64(0), false for i := 0; i < 3; i++ { @@ -293,6 +297,7 @@ func TestMaintenanceStatus(t *testing.T) { if err != nil { t.Fatal(err) } + t.Logf("Response from %v: %v", i, resp) if prevID == 0 { prevID, leaderFound = resp.Header.MemberId, resp.Header.MemberId == resp.Leader continue diff --git a/tests/integration/clientv3/watch_test.go b/tests/integration/clientv3/watch_test.go index f464ec9bb..d52cb8a7c 100644 --- a/tests/integration/clientv3/watch_test.go +++ b/tests/integration/clientv3/watch_test.go @@ -415,15 +415,7 @@ func TestWatchResumeCompacted(t *testing.T) { } clus.Members[0].Stop(t) - ticker := time.After(time.Second * 10) - for clus.WaitLeader(t) <= 0 { - select { - case <-ticker: - t.Fatalf("failed to wait for new leader") - default: - time.Sleep(10 * time.Millisecond) - } - } + clus.WaitLeader(t) // put some data and compact away numPuts := 5 From 68f2cb8c7736e2b80fb9aceef6127563884d9ea4 Mon Sep 17 00:00:00 2001 From: Piotr Tabor Date: Sun, 3 Apr 2022 10:46:51 +0200 Subject: [PATCH 3/6] Fix ExampleAuth from integration/clientv3/examples (on OsX) The code now ensures that each of the test is running in its own directory as opposed to shared os.tempdir. ``` $ (cd tests && env go test -timeout=15m --race go.etcd.io/etcd/tests/v3/integration/clientv3/examples -run ExampleAuth) 2022/04/03 10:24:59 Running tests (examples): ... 2022/04/03 10:24:59 the function can be called only in the test context. Was integration.BeforeTest() called ? 2022/04/03 10:24:59 2022-04-03T10:24:59.462+0200 INFO m0 LISTEN GRPC {"member": "m0", "grpcAddr": "localhost:m0", "m.Name": "m0"} ``` --- client/pkg/testutil/before.go | 20 +++++++++++++ tests/framework/integration/cluster.go | 28 +++++++++++++------ .../clientv3/concurrency/main_test.go | 3 +- .../clientv3/examples/main_test.go | 13 ++++++--- tests/integration/lazy_cluster.go | 2 ++ 5 files changed, 52 insertions(+), 14 deletions(-) diff --git a/client/pkg/testutil/before.go b/client/pkg/testutil/before.go index 801aa042d..c3a01b1e6 100644 --- a/client/pkg/testutil/before.go +++ b/client/pkg/testutil/before.go @@ -15,6 +15,8 @@ package testutil import ( + "io/ioutil" + "log" "os" "testing" @@ -38,3 +40,21 @@ func BeforeTest(t testing.TB) { t.Cleanup(func() { assert.NoError(t, os.Chdir(path)) }) } + +func BeforeIntegrationExamples(*testing.M) func() { + ExitInShortMode("Skipping: the tests require real cluster") + + tempDir, err := ioutil.TempDir(os.TempDir(), "etcd-integration") + if err != nil { + log.Printf("Failed to obtain tempDir: %v", tempDir) + os.Exit(1) + } + + err = os.Chdir(tempDir) + if err != nil { + log.Printf("Failed to change working dir to: %s: %v", tempDir, err) + os.Exit(1) + } + log.Printf("Running tests (examples) in dir(%v): ...", tempDir) + return func() { os.RemoveAll(tempDir) } +} diff --git a/tests/framework/integration/cluster.go b/tests/framework/integration/cluster.go index b09d05470..089a0d29a 100644 --- a/tests/framework/integration/cluster.go +++ b/tests/framework/integration/cluster.go @@ -214,6 +214,7 @@ func (c *Cluster) fillClusterForMembers() error { } func (c *Cluster) Launch(t testutil.TB) { + t.Logf("Launching new cluster...") errc := make(chan error) for _, m := range c.Members { // Members are launched in separate goroutines because if they boot @@ -411,7 +412,7 @@ func (c *Cluster) WaitMembersForLeader(t testutil.TB, membs []*Member) int { ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() l := 0 - for l = c.waitMembersForLeader(t, ctx, membs); l < 0; { + for l = c.waitMembersForLeader(ctx, t, membs); l < 0; { if ctx.Err() != nil { t.Fatal("WaitLeader FAILED: %v", ctx.Err()) } @@ -427,14 +428,12 @@ func (c *Cluster) WaitMembersForLeader(t testutil.TB, membs []*Member) int { // - 9903a56eaf96afac became follower at term 3 {"member": "m0"} // - 9903a56eaf96afac lost leader 9903a56eaf96afac at term 3 {"member": "m0"} - return l } // WaitMembersForLeader waits until given members agree on the same leader, // and returns its 'index' in the 'membs' list -func (c *Cluster) waitMembersForLeader(t testutil.TB, ctx context.Context, membs []*Member) int { - t.Logf("WaitMembersForLeader...") +func (c *Cluster) waitMembersForLeader(ctx context.Context, t testutil.TB, membs []*Member) int { possibleLead := make(map[uint64]bool) var lead uint64 for _, m := range membs { @@ -473,12 +472,12 @@ func (c *Cluster) waitMembersForLeader(t testutil.TB, ctx context.Context, membs for i, m := range membs { if uint64(m.Server.ID()) == lead { - t.Logf("WaitMembersForLeader found leader. Member: %v lead: %x", i, lead) + t.Logf("waitMembersForLeader found leader. Member: %v lead: %x", i, lead) return i } } - t.Logf("WaitMembersForLeader FAILED (-1)") + t.Logf("waitMembersForLeader failed (-1)") return -1 } @@ -531,6 +530,7 @@ func newLocalListener(t testutil.TB) net.Listener { } func NewListenerWithAddr(t testutil.TB, addr string) net.Listener { + t.Logf("Creating listener with addr: %v", addr) l, err := transport.NewUnixListener(addr) if err != nil { t.Fatal(err) @@ -722,7 +722,7 @@ func MustNewMember(t testutil.TB, mcfg MemberConfig) *Member { m.Logger = memberLogger(t, mcfg.Name) m.StrictReconfigCheck = mcfg.StrictReconfigCheck if err := m.listenGRPC(); err != nil { - t.Fatal(err) + t.Fatalf("listenGRPC FAILED: %v", err) } t.Cleanup(func() { // if we didn't cleanup the logger, the consecutive test @@ -747,7 +747,11 @@ func (m *Member) listenGRPC() error { // prefix with localhost so cert has right domain network, host, port := m.grpcAddr() grpcAddr := host + ":" + port - m.Logger.Info("LISTEN GRPC", zap.String("grpcAddr", grpcAddr), zap.String("m.Name", m.Name)) + wd, err := os.Getwd() + if err != nil { + return err + } + m.Logger.Info("LISTEN GRPC", zap.String("grpcAddr", grpcAddr), zap.String("m.Name", m.Name), zap.String("workdir", wd)) grpcListener, err := net.Listen(network, grpcAddr) if err != nil { return fmt.Errorf("listen failed on grpc socket %s (%v)", grpcAddr, err) @@ -1346,7 +1350,7 @@ func NewCluster(t testutil.TB, cfg *ClusterConfig) *Cluster { } c.Members = ms if err := c.fillClusterForMembers(); err != nil { - t.Fatal(err) + t.Fatalf("fillClusterForMembers failed: %v", err) } c.Launch(t) @@ -1360,6 +1364,9 @@ func (c *Cluster) TakeClient(idx int) { } func (c *Cluster) Terminate(t testutil.TB) { + if t != nil { + t.Logf("========= Cluster termination started =====================") + } c.mu.Lock() if c.clusterClient != nil { if err := c.clusterClient.Close(); err != nil { @@ -1381,6 +1388,9 @@ func (c *Cluster) Terminate(t testutil.TB) { }(m) } wg.Wait() + if t != nil { + t.Logf("========= Cluster termination succeeded ===================") + } } func (c *Cluster) RandClient() *clientv3.Client { diff --git a/tests/integration/clientv3/concurrency/main_test.go b/tests/integration/clientv3/concurrency/main_test.go index c54df6b4b..62a6e73a0 100644 --- a/tests/integration/clientv3/concurrency/main_test.go +++ b/tests/integration/clientv3/concurrency/main_test.go @@ -33,12 +33,13 @@ func forUnitTestsRunInMockedContext(mocking func(), example func()) { // TestMain sets up an etcd cluster if running the examples. func TestMain(m *testing.M) { - testutil.ExitInShortMode("Skipping: the tests require real cluster") + cleanup := testutil.BeforeIntegrationExamples(m) v := m.Run() lazyCluster.Terminate() if v == 0 { testutil.MustCheckLeakedGoroutine() } + cleanup() os.Exit(v) } diff --git a/tests/integration/clientv3/examples/main_test.go b/tests/integration/clientv3/examples/main_test.go index 223ec2cdb..5803739f0 100644 --- a/tests/integration/clientv3/examples/main_test.go +++ b/tests/integration/clientv3/examples/main_test.go @@ -15,6 +15,7 @@ package clientv3_test import ( + "io/ioutil" "log" "os" "testing" @@ -37,7 +38,7 @@ var lazyCluster = integration.NewLazyClusterWithConfig( func exampleEndpoints() []string { return lazyCluster.EndpointsV3() } -func forUnitTestsRunInMockedContext(mocking func(), example func()) { +func forUnitTestsRunInMockedContext(_ func(), example func()) { // For integration tests runs in the provided environment example() } @@ -46,15 +47,19 @@ func forUnitTestsRunInMockedContext(mocking func(), example func()) { func TestMain(m *testing.M) { testutil.ExitInShortMode("Skipping: the tests require real cluster") - tempDir := os.TempDir() + tempDir, err := ioutil.TempDir(os.TempDir(), "etcd-integration") + if err != nil { + log.Printf("Failed to obtain tempDir: %v", tempDir) + os.Exit(1) + } defer os.RemoveAll(tempDir) - err := os.Chdir(tempDir) + err = os.Chdir(tempDir) if err != nil { log.Printf("Failed to change working dir to: %s: %v", tempDir, err) os.Exit(1) } - + log.Printf("Running tests (examples) in dir(%v): ...", tempDir) v := m.Run() lazyCluster.Terminate() diff --git a/tests/integration/lazy_cluster.go b/tests/integration/lazy_cluster.go index 02fc759dc..1d16d2d38 100644 --- a/tests/integration/lazy_cluster.go +++ b/tests/integration/lazy_cluster.go @@ -77,12 +77,14 @@ func NewLazyClusterWithConfig(cfg integration.ClusterConfig) LazyCluster { func (lc *lazyCluster) mustLazyInit() { lc.once.Do(func() { + lc.tb.Logf("LazyIniting ...") var err error lc.transport, err = transport.NewTransport(transport.TLSInfo{}, time.Second) if err != nil { log.Fatal(err) } lc.cluster = integration.NewCluster(lc.tb, &lc.cfg) + lc.tb.Logf("LazyIniting [Done]") }) } From ed1bc447c7540d61b61e5561cecb6fb877305f81 Mon Sep 17 00:00:00 2001 From: Piotr Tabor Date: Sun, 3 Apr 2022 13:20:29 +0200 Subject: [PATCH 4/6] Flakes: Additional logging and timeouts to understand common flakes. --- tests/common/lease_test.go | 5 +++- .../experimental/recipes/v3_lock_test.go | 26 +++++++++++-------- tests/integration/cluster_test.go | 12 +++++++-- 3 files changed, 29 insertions(+), 14 deletions(-) diff --git a/tests/common/lease_test.go b/tests/common/lease_test.go index e64f3b86e..34260c6c2 100644 --- a/tests/common/lease_test.go +++ b/tests/common/lease_test.go @@ -120,19 +120,22 @@ func TestLeaseGrantAndList(t *testing.T) { for _, nc := range nestedCases { t.Run(tc.name+"/"+nc.name, func(t *testing.T) { + t.Logf("Creating cluster...") clus := testRunner.NewCluster(t, tc.config) defer clus.Close() cc := clus.Client() - + t.Logf("Created cluster and client") testutils.ExecuteWithTimeout(t, 10*time.Second, func() { createdLeases := []clientv3.LeaseID{} for i := 0; i < nc.leaseCount; i++ { leaseResp, err := cc.Grant(10) + t.Logf("Grant returned: resp:%s err:%v", leaseResp.String(), err) require.NoError(t, err) createdLeases = append(createdLeases, leaseResp.ID) } resp, err := cc.LeaseList() + t.Logf("Lease list returned: resp:%s err:%v", resp.String(), err) require.NoError(t, err) require.Len(t, resp.Leases, nc.leaseCount) diff --git a/tests/integration/clientv3/experimental/recipes/v3_lock_test.go b/tests/integration/clientv3/experimental/recipes/v3_lock_test.go index ab7e7ff9e..8a8229a87 100644 --- a/tests/integration/clientv3/experimental/recipes/v3_lock_test.go +++ b/tests/integration/clientv3/experimental/recipes/v3_lock_test.go @@ -96,7 +96,7 @@ func TestMutexTryLockSingleNode(t *testing.T) { integration2.BeforeTest(t) clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3}) defer clus.Terminate(t) - + t.Logf("3 nodes cluster created...") var clients []*clientv3.Client testMutexTryLock(t, 5, integration2.MakeSingleNodeClients(t, clus, &clients)) integration2.CloseClients(t, clients) @@ -113,35 +113,39 @@ func TestMutexTryLockMultiNode(t *testing.T) { } func testMutexTryLock(t *testing.T, lockers int, chooseClient func() *clientv3.Client) { + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + lockedC := make(chan *concurrency.Mutex) notlockedC := make(chan *concurrency.Mutex) - stopC := make(chan struct{}) - defer close(stopC) + for i := 0; i < lockers; i++ { - go func() { + go func(i int) { session, err := concurrency.NewSession(chooseClient()) if err != nil { t.Error(err) } m := concurrency.NewMutex(session, "test-mutex-try-lock") - err = m.TryLock(context.TODO()) + err = m.TryLock(ctx) if err == nil { select { case lockedC <- m: - case <-stopC: + case <-ctx.Done(): + t.Errorf("Thread: %v, Context failed: %v", i, err) } } else if err == concurrency.ErrLocked { select { case notlockedC <- m: - case <-stopC: + case <-ctx.Done(): + t.Errorf("Thread: %v, Context failed: %v", i, err) } } else { - t.Errorf("Unexpected Error %v", err) + t.Errorf("Thread: %v; Unexpected Error %v", i, err) } - }() + }(i) } - timerC := time.After(time.Second) + timerC := time.After(30 * time.Second) select { case <-lockedC: for i := 0; i < lockers-1; i++ { @@ -154,7 +158,7 @@ func testMutexTryLock(t *testing.T, lockers int, chooseClient func() *clientv3.C } } case <-timerC: - t.Errorf("timed out waiting for lock") + t.Errorf("timed out waiting for lock (30s)") } } diff --git a/tests/integration/cluster_test.go b/tests/integration/cluster_test.go index 4e0d4a0f0..5a7ff4592 100644 --- a/tests/integration/cluster_test.go +++ b/tests/integration/cluster_test.go @@ -284,6 +284,8 @@ func TestIssue3699(t *testing.T) { // add node d c.AddMember(t) + t.Logf("Disturbing cluster till member:3 will become a leader") + // electing node d as leader makes node a unable to participate leaderID := c.WaitMembersForLeader(t, c.Members) for leaderID != 3 { @@ -297,11 +299,16 @@ func TestIssue3699(t *testing.T) { leaderID = c.WaitMembersForLeader(t, c.Members) } + t.Logf("Finally elected member 3 as the leader.") + + t.Logf("Restarting member '0'...") // bring back node a // node a will remain useless as long as d is the leader. if err := c.Members[0].Restart(t); err != nil { t.Fatal(err) } + t.Logf("Restarted member '0'.") + select { // waiting for ReadyNotify can take several seconds case <-time.After(10 * time.Second): @@ -311,12 +318,13 @@ func TestIssue3699(t *testing.T) { case <-c.Members[0].Server.ReadyNotify(): } // must WaitMembersForLeader so goroutines don't leak on terminate - c.WaitMembersForLeader(t, c.Members) + c.WaitLeader(t) + t.Logf("Expecting successful put...") // try to participate in Cluster ctx, cancel := context.WithTimeout(context.Background(), integration.RequestTimeout) if _, err := c.Members[0].Client.Put(ctx, "/foo", "bar"); err != nil { - t.Fatalf("unexpected error on Set (%v)", err) + t.Fatalf("unexpected error on Put (%v)", err) } cancel() } From 90796720c1224d266b7e4883d59b3a1c4f7e11d2 Mon Sep 17 00:00:00 2001 From: Piotr Tabor Date: Sun, 3 Apr 2022 13:49:02 +0200 Subject: [PATCH 5/6] Reduce integration test parallelism to 2 packages at once. Especially with 'race' detection, running O(cpu) integrational tests was causing CPU overloads and timeouts. --- scripts/test.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scripts/test.sh b/scripts/test.sh index 687b32433..932ab953a 100755 --- a/scripts/test.sh +++ b/scripts/test.sh @@ -107,8 +107,8 @@ function integration_extra { } function integration_pass { - run_for_module "tests" go_test "./integration/..." "parallel" : -timeout="${TIMEOUT:-15m}" "${COMMON_TEST_FLAGS[@]}" "${RUN_ARG[@]}" "$@" || return $? - run_for_module "tests" go_test "./common/..." "parallel" : --tags=integration -timeout="${TIMEOUT:-15m}" "${COMMON_TEST_FLAGS[@]}" "${RUN_ARG[@]}" "$@" || return $? + run_for_module "tests" go_test "./integration/..." "parallel" : -timeout="${TIMEOUT:-15m}" "${COMMON_TEST_FLAGS[@]}" "${RUN_ARG[@]}" -p=2 "$@" || return $? + run_for_module "tests" go_test "./common/..." "parallel" : --tags=integration -timeout="${TIMEOUT:-15m}" "${COMMON_TEST_FLAGS[@]}" "${RUN_ARG[@]}" -p=2 "$@" || return $? integration_extra "$@" } From d4dcd3061dff943418f8d413cd431916f9807fd7 Mon Sep 17 00:00:00 2001 From: Piotr Tabor Date: Sun, 3 Apr 2022 23:13:01 +0200 Subject: [PATCH 6/6] Fix flakes in TestV3LeaseCheckpoint/Checkpointing_disabled,_lease_TTL_is_reset I think strong (not-equal) relationship was too restrictive when expressed with 1s granularity. ``` logger.go:130: 2022-04-03T22:15:15.242+0200 WARN m1 leader failed to send out heartbeat on time; took too long, leader is overloaded likely from slow disk {"member": "m1", "to": "cb785755eb80ac1", "heartbeat-interval": "10ms", "expected-duration": "20ms", "exceeded-duration": "24.666613ms"} logger.go:130: 2022-04-03T22:15:15.262+0200 INFO m-1 published local member to cluster through raft {"member": "m-1", "local-member-id": "e2dd9f523aa7be87", "local-member-attributes": "{Name:m-1 ClientURLs:[unix://127.0.0.1:2196386040]}", "cluster-id": "b4b8e7e41c23c8b5", "publish-timeout": "5.2s"} v3_lease_test.go:415: Expected lease ttl (4m58s) to be greather than (4m58s) ``` --- scripts/test.sh | 2 +- tests/integration/v3_lease_test.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/scripts/test.sh b/scripts/test.sh index 932ab953a..9f5b38772 100755 --- a/scripts/test.sh +++ b/scripts/test.sh @@ -108,7 +108,7 @@ function integration_extra { function integration_pass { run_for_module "tests" go_test "./integration/..." "parallel" : -timeout="${TIMEOUT:-15m}" "${COMMON_TEST_FLAGS[@]}" "${RUN_ARG[@]}" -p=2 "$@" || return $? - run_for_module "tests" go_test "./common/..." "parallel" : --tags=integration -timeout="${TIMEOUT:-15m}" "${COMMON_TEST_FLAGS[@]}" "${RUN_ARG[@]}" -p=2 "$@" || return $? + run_for_module "tests" go_test "./common/..." "parallel" : --tags=integration -timeout="${TIMEOUT:-15m}" "${COMMON_TEST_FLAGS[@]}" -p=2 "${RUN_ARG[@]}" "$@" || return $? integration_extra "$@" } diff --git a/tests/integration/v3_lease_test.go b/tests/integration/v3_lease_test.go index 520c983a3..df277267c 100644 --- a/tests/integration/v3_lease_test.go +++ b/tests/integration/v3_lease_test.go @@ -411,8 +411,8 @@ func TestV3LeaseCheckpoint(t *testing.T) { } } - if tc.expectTTLIsGT != 0 && time.Duration(ttlresp.TTL)*time.Second <= tc.expectTTLIsGT { - t.Errorf("Expected lease ttl (%v) to be greather than (%v)", time.Duration(ttlresp.TTL)*time.Second, tc.expectTTLIsGT) + if tc.expectTTLIsGT != 0 && time.Duration(ttlresp.TTL)*time.Second < tc.expectTTLIsGT { + t.Errorf("Expected lease ttl (%v) to be >= than (%v)", time.Duration(ttlresp.TTL)*time.Second, tc.expectTTLIsGT) } if tc.expectTTLIsLT != 0 && time.Duration(ttlresp.TTL)*time.Second > tc.expectTTLIsLT {