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 b971c2a RATIS-1362. Intermittent NPE in RaftReconfigurationBaseTest.
(#465)
b971c2a is described below
commit b971c2a7d1b6800942723237de8c10290a70e1da
Author: Roni Juntunen <[email protected]>
AuthorDate: Mon Apr 26 11:54:14 2021 +0300
RATIS-1362. Intermittent NPE in RaftReconfigurationBaseTest. (#465)
---
.../server/impl/RaftReconfigurationBaseTest.java | 20 +++++++++-----------
1 file changed, 9 insertions(+), 11 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 22b60da..f6580e8 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
@@ -49,10 +49,7 @@ import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
+import java.util.*;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
@@ -340,7 +337,6 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER
extends MiniRaftCluste
LOG.info("Start changing the configuration: {}",
asList(c1.allPeersInNewConf));
final AtomicReference<Boolean> success = new AtomicReference<>();
-
Thread clientThread = new Thread(() -> {
try {
RaftClientReply reply =
client.admin().setConfiguration(c1.allPeersInNewConf);
@@ -352,21 +348,23 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER
extends MiniRaftCluste
clientThread.start();
if (!startNewPeer) {
- final TimeDuration delay = FIVE_SECONDS;
- LOG.info("delay {} and start new peer(s): {}", delay, c1.newPeers);
- delay.sleep();
+ // Make sure that set configuration is run inside the thread
+ RaftTestUtil.waitFor(() -> clientThread.isAlive(), 300, 5000);
+ ONE_SECOND.sleep();
+ LOG.info("start new peer(s): {}", c1.newPeers);
for(RaftPeer p : c1.newPeers) {
cluster.restartServer(p.getId(), true);
}
}
- FIVE_SECONDS.sleep();
- LOG.info(cluster.printServers());
- RaftTestUtil.waitFor(() -> success.get(), 300, 15000);
+ RaftTestUtil.waitFor(() -> success.get() != null && success.get(),
300, 15000);
+ LOG.info(cluster.printServers());
final RaftLog leaderLog = cluster.getLeader().getRaftLog();
for (RaftPeer newPeer : c1.newPeers) {
final RaftServer.Division d = cluster.getDivision(newPeer.getId());
+ RaftTestUtil.waitFor(() -> leaderLog.getEntries(0,
Long.MAX_VALUE).length
+ == d.getRaftLog().getEntries(0, Long.MAX_VALUE).length, 300,
15000);
Assert.assertArrayEquals(leaderLog.getEntries(0, Long.MAX_VALUE),
d.getRaftLog().getEntries(0, Long.MAX_VALUE));
}