This is an automated email from the ASF dual-hosted git repository.

szetszwo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ratis.git


The following commit(s) were added to refs/heads/master by this push:
     new 8d5c50e0 RATIS-1629. Fix flaky 
TestRaftReconfigurationWithSimulatedRpc#testKillLeaderDuringReconf. (#686)
8d5c50e0 is described below

commit 8d5c50e0082f8c21b79c44791b4186ed6d9c602c
Author: Sammi Chen <[email protected]>
AuthorDate: Thu Jul 21 00:49:11 2022 +0800

    RATIS-1629. Fix flaky 
TestRaftReconfigurationWithSimulatedRpc#testKillLeaderDuringReconf. (#686)
---
 .../ratis/server/impl/RaftReconfigurationBaseTest.java     | 14 ++++++++++----
 1 file changed, 10 insertions(+), 4 deletions(-)

diff --git 
a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
 
b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
index eac63d0b..b440b26d 100644
--- 
a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
+++ 
b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
@@ -44,6 +44,7 @@ import org.apache.ratis.server.raftlog.RaftLog;
 import org.apache.ratis.server.raftlog.RaftLogBase;
 import org.apache.ratis.server.storage.RaftStorageTestUtils;
 import org.apache.ratis.util.JavaUtils;
+import org.apache.ratis.util.LifeCycle;
 import org.apache.ratis.util.Log4jUtils;
 import org.apache.ratis.util.TimeDuration;
 import org.junit.Assert;
@@ -501,7 +502,7 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER 
extends MiniRaftCluste
       Assert.assertEquals(leaderId, killed);
       final RaftPeerId newLeaderId = 
RaftTestUtil.waitForLeader(cluster).getId();
       LOG.info("newLeaderId: {}", newLeaderId);
-      TimeDuration.valueOf(500, TimeUnit.MILLISECONDS).sleep();
+      TimeDuration.valueOf(1500, TimeUnit.MILLISECONDS).sleep();
 
       LOG.info("start new peers: {}", Arrays.asList(c1.newPeers));
       for (RaftPeer np : c1.newPeers) {
@@ -513,9 +514,14 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER 
extends MiniRaftCluste
       } catch(TimeoutException ignored) {
       }
 
-      // the client fails with the first leader, and then retry the same 
setConfiguration request
-      waitAndCheckNewConf(cluster, c2.allPeersInNewConf, 1, 
Collections.singletonList(leaderId));
-      setConf.get(2, TimeUnit.SECONDS);
+      RaftServerProxy newServer = cluster.getServer(c1.newPeers[0].getId());
+      if (newServer.getLifeCycleState() == LifeCycle.State.CLOSED) {
+        LOG.info("New peer {} is shutdown. Skip the check", 
c1.newPeers[0].getId());
+      } else {
+        // the client fails with the first leader, and then retry the same 
setConfiguration request
+        waitAndCheckNewConf(cluster, c2.allPeersInNewConf, 1, 
Collections.singletonList(leaderId));
+        setConf.get(2, TimeUnit.SECONDS);
+      }
     } finally {
       if (clientThread != null) {
         clientRunning.set(false);

Reply via email to