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
commit 7e22578d7af15cfcee9d1279025f644f420e4f36 Author: slfan1989 <[email protected]> AuthorDate: Wed May 21 00:09:01 2025 +0800 RATIS-2251. Migrate ratis-test tests to Junit 5 - Part 3. (#1227) --- .../ratis/InstallSnapshotNotificationTests.java | 80 ++++++------ .../java/org/apache/ratis/LogAppenderTests.java | 14 +- .../org/apache/ratis/MessageStreamApiTests.java | 16 +-- .../org/apache/ratis/OutputStreamBaseTest.java | 30 ++--- .../org/apache/ratis/RaftAsyncExceptionTests.java | 14 +- .../test/java/org/apache/ratis/RaftAsyncTests.java | 54 ++++---- .../org/apache/ratis/RaftExceptionBaseTest.java | 20 +-- .../test/java/org/apache/ratis/RaftTestUtil.java | 23 ++-- .../org/apache/ratis/ReadOnlyRequestTests.java | 46 +++---- .../ratis/ReadOnlyRequestWithLongTimeoutTests.java | 20 +-- .../apache/ratis/RequestLimitAsyncBaseTest.java | 8 +- .../org/apache/ratis/TestReConfigProperty.java | 142 ++++++++------------- .../java/org/apache/ratis/WatchRequestTests.java | 62 ++++----- .../ratis/server/impl/GroupInfoBaseTest.java | 12 +- .../ratis/server/impl/GroupManagementBaseTest.java | 70 +++++----- .../apache/ratis/server/impl/LogMetadataTests.java | 6 +- .../server/impl/PreAppendLeaderStepDownTest.java | 14 +- .../server/impl/RaftReconfigurationBaseTest.java | 94 +++++++------- .../ratis/server/impl/RaftServerTestUtil.java | 22 ++-- .../impl/RaftStateMachineExceptionTests.java | 32 ++--- .../ratis/server/impl/RetryCacheTestUtil.java | 4 +- .../ratis/server/impl/ServerPauseResumeTest.java | 22 ++-- .../server/impl/TestRatisServerMetricsBase.java | 8 +- .../ratis/statemachine/RaftSnapshotBaseTest.java | 2 +- .../ratis/statemachine/SnapshotManagementTest.java | 44 +++---- ratis-test/pom.xml | 10 ++ .../apache/ratis/grpc/TestRaftAsyncWithGrpc.java | 2 + .../apache/ratis/netty/TestRaftAsyncWithNetty.java | 2 + 28 files changed, 431 insertions(+), 442 deletions(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java index ae5d79224..411c93120 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java @@ -39,8 +39,8 @@ import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; @@ -194,7 +194,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC try(final RaftClient client = cluster.createClient(leaderId)) { for (; i < numRequests; i++) { final RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -206,7 +206,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC final List<File> snapshotFiles = RaftSnapshotBaseTest.getSnapshotFiles(cluster, nextIndex - SNAPSHOT_TRIGGER_THRESHOLD, nextIndex); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); + Assertions.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); return null; }, 10, ONE_SECOND, "snapshotFile.exist", LOG); logs = LogSegmentPath.getLogSegmentPaths(leader.getRaftStorage()); @@ -220,7 +220,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC // delete the log segments from the leader LOG.info("Delete logs {}", logs); for (LogSegmentPath path : logs) { - FileUtils.deleteFully(path.getPath()); // the log may be already purged + FileUtils.deleteFully(path.getPath()); // the log may be already puged } // restart the peer @@ -231,13 +231,13 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC // generate some more traffic try(final RaftClient client = cluster.createClient(cluster.getLeader().getId())) { - Assert.assertTrue(client.io().send(new RaftTestUtil.SimpleMessage("m" + i)).isSuccess()); + Assertions.assertTrue(client.io().send(new RaftTestUtil.SimpleMessage("m" + i)).isSuccess()); } final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); LOG.info("LeaderSnapshotInfo: {}", leaderSnapshotInfo.getTermIndex()); final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); - Assert.assertTrue(set); + Assertions.assertTrue(set); // add two more peers final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(2, true, @@ -253,7 +253,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC // leader snapshot. for (RaftServer.Division follower : cluster.getFollowers()) { final long expected = leaderSnapshotInfo.getIndex(); - Assert.assertEquals(expected, RaftServerTestUtil.getLatestInstalledSnapshotIndex(follower)); + Assertions.assertEquals(expected, RaftServerTestUtil.getLatestInstalledSnapshotIndex(follower)); RaftSnapshotBaseTest.assertLogContent(follower, false); } @@ -279,7 +279,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC try (final RaftClient client = cluster.createClient(leaderId)) { for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) { final RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -290,7 +290,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC final List<File> snapshotFiles = RaftSnapshotBaseTest.getSnapshotFiles(cluster, oldLeaderNextIndex - SNAPSHOT_TRIGGER_THRESHOLD, oldLeaderNextIndex); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); + Assertions.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); return null; }, 10, ONE_SECOND, "snapshotFile.exist", LOG); } @@ -300,7 +300,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC // generate some more traffic try (final RaftClient client = cluster.createClient(leader.getId())) { - Assert.assertTrue(client.io().send(new RaftTestUtil.SimpleMessage("m" + i)).isSuccess()); + Assertions.assertTrue(client.io().send(new RaftTestUtil.SimpleMessage("m" + i)).isSuccess()); } FIVE_SECONDS.sleep(); @@ -309,8 +309,8 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC JavaUtils.attempt(() -> { final long newLeaderNextIndex = leader.getRaftLog().getNextIndex(); LOG.info("{}: newLeaderNextIndex = {}", leaderId, newLeaderNextIndex); - Assert.assertTrue(newLeaderNextIndex > oldLeaderNextIndex); - Assert.assertEquals(newLeaderNextIndex, follower.getRaftLog().getNextIndex()); + Assertions.assertTrue(newLeaderNextIndex > oldLeaderNextIndex); + Assertions.assertEquals(newLeaderNextIndex, follower.getRaftLog().getNextIndex()); }, 10, ONE_SECOND, "followerNextIndex", LOG); } @@ -331,14 +331,14 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC // Let a few heartbeats pass. ONE_SECOND.sleep(); - Assert.assertEquals(0, numSnapshotRequests.get()); + Assertions.assertEquals(0, numSnapshotRequests.get()); // Generate data. try(final RaftClient client = cluster.createClient(leaderId)) { for (; i < 10; i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -349,42 +349,42 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC // Take snapshot and check result. long snapshotIndex = cluster.getLeader().getStateMachine().takeSnapshot(); - Assert.assertEquals(20, snapshotIndex); + Assertions.assertEquals(20, snapshotIndex); final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); - Assert.assertEquals(20, leaderSnapshotInfo.getIndex()); + Assertions.assertEquals(20, leaderSnapshotInfo.getIndex()); final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); - Assert.assertTrue(set); + Assertions.assertTrue(set); // Wait for the snapshot to be done. final RaftServer.Division leader = cluster.getLeader(); final long nextIndex = leader.getRaftLog().getNextIndex(); - Assert.assertEquals(21, nextIndex); + Assertions.assertEquals(21, nextIndex); // End index is exclusive. final List<File> snapshotFiles = RaftSnapshotBaseTest.getSnapshotFiles(cluster, 0, nextIndex); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); + Assertions.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); return null; }, 10, ONE_SECOND, "snapshotFile.exist", LOG); // Clear all log files and reset cached log start index. long snapshotInstallIndex = leader.getRaftLog().onSnapshotInstalled(leader.getRaftLog().getLastCommittedIndex()).get(); - Assert.assertEquals(20, snapshotInstallIndex); + Assertions.assertEquals(20, snapshotInstallIndex); // Check that logs are gone. - Assert.assertEquals(0, + Assertions.assertEquals(0, LogSegmentPath.getLogSegmentPaths(leader.getRaftStorage()).size()); - Assert.assertEquals(RaftLog.INVALID_LOG_INDEX, leader.getRaftLog().getStartIndex()); + Assertions.assertEquals(RaftLog.INVALID_LOG_INDEX, leader.getRaftLog().getStartIndex()); // Allow some heartbeats to go through, then make sure none of them had // snapshot requests. ONE_SECOND.sleep(); - Assert.assertEquals(0, numSnapshotRequests.get()); + Assertions.assertEquals(0, numSnapshotRequests.get()); // Make sure leader and followers are still up to date. for (RaftServer.Division follower : cluster.getFollowers()) { - Assert.assertEquals( + Assertions.assertEquals( leader.getRaftLog().getNextIndex(), follower.getRaftLog().getNextIndex()); } @@ -400,7 +400,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC // Generate more data. try (final RaftClient client = cluster.createClient(leader.getId())) { - Assert.assertTrue(client.io().send(new RaftTestUtil.SimpleMessage("m" + i)).isSuccess()); + Assertions.assertTrue(client.io().send(new RaftTestUtil.SimpleMessage("m" + i)).isSuccess()); } // Make sure leader and followers are still up to date. @@ -412,7 +412,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC } // Make sure each new peer got one snapshot notification. - Assert.assertEquals(2, numSnapshotRequests.get()); + Assertions.assertEquals(2, numSnapshotRequests.get()); } finally { cluster.shutdown(); @@ -437,7 +437,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -448,7 +448,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC final List<File> snapshotFiles = RaftSnapshotBaseTest.getSnapshotFiles(cluster, nextIndex - SNAPSHOT_TRIGGER_THRESHOLD, nextIndex); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); + Assertions.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); return null; }, 10, ONE_SECOND, "snapshotFile.exist", LOG); logs = LogSegmentPath.getLogSegmentPaths(leader.getRaftStorage()); @@ -470,13 +470,13 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC // generate some more traffic try(final RaftClient client = cluster.createClient(cluster.getLeader().getId())) { - Assert.assertTrue(client.io().send(new RaftTestUtil.SimpleMessage("m" + i)).isSuccess()); + Assertions.assertTrue(client.io().send(new RaftTestUtil.SimpleMessage("m" + i)).isSuccess()); } final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); LOG.info("LeaderSnapshotInfo: {}", leaderSnapshotInfo.getTermIndex()); final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); - Assert.assertTrue(set); + Assertions.assertTrue(set); // add one new peer final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(1, true, true); @@ -490,7 +490,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC // Check the installed snapshot index on each Follower matches with the // leader snapshot. for (RaftServer.Division follower : cluster.getFollowers()) { - Assert.assertEquals(leaderSnapshotInfo.getIndex(), + Assertions.assertEquals(leaderSnapshotInfo.getIndex(), RaftServerTestUtil.getLatestInstalledSnapshotIndex(follower)); } @@ -500,9 +500,9 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC (SimpleStateMachine4Testing) cluster.getFollowers().get(0).getStateMachine(); final File followerSnapshotFile = new File(followerStateMachine.getStateMachineDir(), leaderSnapshotFile.getName()); - Assert.assertEquals(numNotifyInstallSnapshotFinished.get(), 2); - Assert.assertTrue(leaderSnapshotFile.exists()); - Assert.assertFalse(followerSnapshotFile.exists()); + Assertions.assertEquals(numNotifyInstallSnapshotFinished.get(), 2); + Assertions.assertTrue(leaderSnapshotFile.exists()); + Assertions.assertFalse(followerSnapshotFile.exists()); // restart the peer and check if it can correctly handle conf change cluster.restartServer(cluster.getLeader().getId(), false); @@ -535,7 +535,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -546,7 +546,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC final List<File> snapshotFiles = RaftSnapshotBaseTest.getSnapshotFiles(cluster, nextIndex - SNAPSHOT_TRIGGER_THRESHOLD, nextIndex); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); + Assertions.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); return null; }, 10, ONE_SECOND, "snapshotFile.exist", LOG); @@ -554,7 +554,7 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); - Assert.assertTrue(set); + Assertions.assertTrue(set); // add two more peers final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(2, true, @@ -568,12 +568,12 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC // Check the installed snapshot index on each Follower matches with the // leader snapshot. for (RaftServer.Division follower : cluster.getFollowers()) { - Assert.assertEquals(leaderSnapshotInfo.getIndex(), + Assertions.assertEquals(leaderSnapshotInfo.getIndex(), RaftServerTestUtil.getLatestInstalledSnapshotIndex(follower)); } // Make sure each new peer got at least one snapshot notification. - Assert.assertTrue(2 <= numSnapshotRequests.get()); + Assertions.assertTrue(2 <= numSnapshotRequests.get()); } finally { cluster.shutdown(); } diff --git a/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java b/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java index ce08e4a74..c7a7849e6 100644 --- a/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java @@ -18,7 +18,7 @@ package org.apache.ratis; import static org.apache.ratis.RaftTestUtil.waitForLeader; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; import org.apache.ratis.RaftTestUtil.SimpleMessage; import org.apache.ratis.client.RaftClient; @@ -40,8 +40,8 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.ArrayList; @@ -199,7 +199,7 @@ public abstract class LogAppenderTests<CLUSTER extends MiniRaftCluster> if (e != null) { throw e; } - Assert.assertTrue(s.succeed.get()); + Assertions.assertTrue(s.succeed.get()); } } finally { for (int i = 0; i < clients.size(); i ++) { @@ -215,11 +215,11 @@ public abstract class LogAppenderTests<CLUSTER extends MiniRaftCluster> final RaftLog leaderLog = cluster.getLeader().getRaftLog(); final EnumMap<LogEntryBodyCase, AtomicLong> counts = RaftTestUtil.countEntries(leaderLog); LOG.info("counts = " + counts); - Assert.assertEquals(6 * numMsgs * numClients, counts.get(LogEntryBodyCase.STATEMACHINELOGENTRY).get()); + Assertions.assertEquals(6 * numMsgs * numClients, counts.get(LogEntryBodyCase.STATEMACHINELOGENTRY).get()); final LogEntryProto last = RaftTestUtil.getLastEntry(LogEntryBodyCase.STATEMACHINELOGENTRY, leaderLog); LOG.info("last = {}", LogProtoUtils.toLogEntryString(last)); - Assert.assertNotNull(last); - Assert.assertTrue(last.getIndex() <= leader.getInfo().getLastAppliedIndex()); + Assertions.assertNotNull(last); + Assertions.assertTrue(last.getIndex() <= leader.getInfo().getLastAppliedIndex()); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/MessageStreamApiTests.java b/ratis-server/src/test/java/org/apache/ratis/MessageStreamApiTests.java index 816611573..50f59b2d2 100644 --- a/ratis-server/src/test/java/org/apache/ratis/MessageStreamApiTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/MessageStreamApiTests.java @@ -31,8 +31,8 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.nio.charset.StandardCharsets; @@ -75,11 +75,11 @@ public abstract class MessageStreamApiTests<CLUSTER extends MiniRaftCluster> ext try(RaftClient client = cluster.createClient()) { final String k1 = k.substring(0, endOfRequest); final RaftClientReply r1= client.io().sendReadOnly(new SimpleMessage(k1)); - Assert.assertTrue(r1.isSuccess()); + Assertions.assertTrue(r1.isSuccess()); final String k2 = k.substring(endOfRequest); final RaftClientReply r2 = client.io().sendReadOnly(new SimpleMessage(k2)); - Assert.assertTrue(r2.isSuccess()); + Assertions.assertTrue(r2.isSuccess()); } } @@ -104,24 +104,24 @@ public abstract class MessageStreamApiTests<CLUSTER extends MiniRaftCluster> ext final String s = (char)('A' + i) + "1234567"; LOG.info("s=" + s); final ByteString b = ByteString.copyFrom(s, StandardCharsets.UTF_8); - Assert.assertEquals(8, b.size()); + Assertions.assertEquals(8, b.size()); for(int j = 0; j < 128; j++) { bytes = bytes.concat(b); } i++; - Assert.assertEquals(i*SUBMESSAGE_SIZE.getSizeInt(), bytes.size()); + Assertions.assertEquals(i*SUBMESSAGE_SIZE.getSizeInt(), bytes.size()); } try(RaftClient client = cluster.createClient()) { final RaftClientReply reply = client.getMessageStreamApi().streamAsync(Message.valueOf(bytes)).get(); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } // check if all the parts are streamed as a single message. try(RaftClient client = cluster.createClient()) { final RaftClientReply reply = client.io().sendReadOnly(new SimpleMessage(bytes.toString(StandardCharsets.UTF_8))); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } } diff --git a/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java index 0c1a5164f..a17cdb0d5 100644 --- a/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java @@ -26,8 +26,8 @@ import org.apache.ratis.server.raftlog.LogEntryHeader; import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.StringUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.io.OutputStream; @@ -47,7 +47,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; import static org.apache.ratis.RaftTestUtil.waitForLeader; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.fail; public abstract class OutputStreamBaseTest<CLUSTER extends MiniRaftCluster> extends BaseTest @@ -92,7 +92,7 @@ public abstract class OutputStreamBaseTest<CLUSTER extends MiniRaftCluster> private void checkLog(RaftLog raftLog, long expectedCommittedIndex, Supplier<byte[]> s) throws IOException { long committedIndex = raftLog.getLastCommittedIndex(); - Assert.assertTrue(committedIndex >= expectedCommittedIndex); + Assertions.assertTrue(committedIndex >= expectedCommittedIndex); // check the log content final LogEntryHeader[] entries = raftLog.getEntries(0, Long.MAX_VALUE); int count = 0; @@ -106,10 +106,10 @@ public abstract class OutputStreamBaseTest<CLUSTER extends MiniRaftCluster> final String message = "log " + entry + " " + log.getLogEntryBodyCase() + " " + StringUtils.bytes2HexString(logData) + ", expected=" + StringUtils.bytes2HexString(expected); - Assert.assertArrayEquals(message, expected, logData); + Assertions.assertArrayEquals(expected, logData, message); count++; } - Assert.assertEquals(expectedCommittedIndex, count); + Assertions.assertEquals(expectedCommittedIndex, count); } @Test @@ -155,12 +155,12 @@ public abstract class OutputStreamBaseTest<CLUSTER extends MiniRaftCluster> private RaftLog assertRaftLog(int expectedEntries, RaftServer.Division server) throws Exception { final RaftLog raftLog = server.getRaftLog(); final EnumMap<LogEntryBodyCase, AtomicLong> counts = RaftTestUtil.countEntries(raftLog); - Assert.assertEquals(expectedEntries, counts.get(LogEntryBodyCase.STATEMACHINELOGENTRY).get()); + Assertions.assertEquals(expectedEntries, counts.get(LogEntryBodyCase.STATEMACHINELOGENTRY).get()); final LogEntryProto last = RaftTestUtil.getLastEntry(LogEntryBodyCase.STATEMACHINELOGENTRY, raftLog); - Assert.assertNotNull(last); - Assert.assertTrue(raftLog.getLastCommittedIndex() >= last.getIndex()); - Assert.assertTrue(server.getInfo().getLastAppliedIndex() >= last.getIndex()); + Assertions.assertNotNull(last); + Assertions.assertTrue(raftLog.getLastCommittedIndex() >= last.getIndex()); + Assertions.assertTrue(server.getInfo().getLastAppliedIndex() >= last.getIndex()); return raftLog; } @@ -249,12 +249,12 @@ public abstract class OutputStreamBaseTest<CLUSTER extends MiniRaftCluster> final LogEntryProto e = raftLog.get(ti.getIndex()); if (e.hasStateMachineLogEntry()) { final byte[] eValue = e.getStateMachineLogEntry().getLogData().toByteArray(); - Assert.assertEquals(ByteValue.BUFFERSIZE, eValue.length); + Assertions.assertEquals(ByteValue.BUFFERSIZE, eValue.length); System.arraycopy(eValue, 0, actual, totalSize, eValue.length); totalSize += eValue.length; } } - Assert.assertArrayEquals(expected, actual); + Assertions.assertArrayEquals(expected, actual); } /** @@ -296,18 +296,18 @@ public abstract class OutputStreamBaseTest<CLUSTER extends MiniRaftCluster> Thread.sleep(500); RaftTestUtil.waitAndKillLeader(cluster); final RaftServer.Division newLeader = waitForLeader(cluster); - Assert.assertNotEquals(leader.getId(), newLeader.getId()); + Assertions.assertNotEquals(leader.getId(), newLeader.getId()); Thread.sleep(500); running.set(false); latch.await(5, TimeUnit.SECONDS); LOG.info("Writer success? " + success.get()); - Assert.assertTrue(success.get()); + Assertions.assertTrue(success.get()); // total number of tx should be >= result + 2, where 2 means two NoOp from // leaders. It may be larger than result+2 because the client may resend // requests and we do not have retry cache on servers yet. LOG.info("last applied index: {}. total number of requests: {}", newLeader.getInfo().getLastAppliedIndex(), result.get()); - Assert.assertTrue(newLeader.getInfo().getLastAppliedIndex() >= result.get() + 1); + Assertions.assertTrue(newLeader.getInfo().getLastAppliedIndex() >= result.get() + 1); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncExceptionTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncExceptionTests.java index 33e085733..044ddc342 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncExceptionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncExceptionTests.java @@ -30,8 +30,8 @@ import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.ArrayList; @@ -62,13 +62,13 @@ public abstract class RaftAsyncExceptionTests<CLUSTER extends MiniRaftCluster> // send a message to make sure the cluster is working try(RaftClient client = cluster.createClient()) { final RaftClientReply reply = client.async().send(new SimpleMessage("first")).get(); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } // create another group final RaftGroup clusterGroup = cluster.getGroup(); final RaftGroup anotherGroup = RaftGroup.valueOf(RaftGroupId.randomId(), clusterGroup.getPeers()); - Assert.assertNotEquals(clusterGroup.getGroupId(), anotherGroup.getGroupId()); + Assertions.assertNotEquals(clusterGroup.getGroupId(), anotherGroup.getGroupId()); // create another client using another group final SimpleMessage[] messages = SimpleMessage.create(5); @@ -78,7 +78,7 @@ public abstract class RaftAsyncExceptionTests<CLUSTER extends MiniRaftCluster> for(SimpleMessage m : messages) { futures.add(client.async().send(m)); } - Assert.assertEquals(messages.length, futures.size()); + Assertions.assertEquals(messages.length, futures.size()); // check replies final Iterator<CompletableFuture<RaftClientReply>> i = futures.iterator(); @@ -102,7 +102,7 @@ public abstract class RaftAsyncExceptionTests<CLUSTER extends MiniRaftCluster> // send a message to make sure the cluster is working try(RaftClient client = cluster.createClient()) { final RaftClientReply reply = client.io().send(new SimpleMessage("m0")); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); RaftClientConfigKeys.Rpc.setRequestTimeout(PROPERTIES.get(), ONE_SECOND); // Block StartTransaction @@ -118,7 +118,7 @@ public abstract class RaftAsyncExceptionTests<CLUSTER extends MiniRaftCluster> .map(SimpleStateMachine4Testing::get) .forEach(SimpleStateMachine4Testing::unblockStartTransaction); // The request should succeed after start transaction is unblocked - Assert.assertTrue(replyFuture.get(FIVE_SECONDS.getDuration(), FIVE_SECONDS.getUnit()).isSuccess()); + Assertions.assertTrue(replyFuture.get(FIVE_SECONDS.getDuration(), FIVE_SECONDS.getUnit()).isSuccess()); } } } diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java index a1c16df8f..4119bea71 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java @@ -51,8 +51,9 @@ import org.apache.ratis.util.PlatformUtils; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedRunnable; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; import java.io.IOException; @@ -70,6 +71,7 @@ import java.util.stream.StreamSupport; import static org.apache.ratis.RaftTestUtil.waitForLeader; +@Timeout(100) public abstract class RaftAsyncTests<CLUSTER extends MiniRaftCluster> extends BaseTest implements MiniRaftCluster.Factory.Get<CLUSTER> { { @@ -112,9 +114,9 @@ public abstract class RaftAsyncTests<CLUSTER extends MiniRaftCluster> extends Ba } static void assertRaftRetryFailureException(RaftRetryFailureException rfe, RetryPolicy retryPolicy, String name) { - Assert.assertNotNull(name + " does not have RaftRetryFailureException", rfe); - Assert.assertTrue(name + ": unexpected error message, rfe=" + rfe + ", retryPolicy=" + retryPolicy, - rfe.getMessage().contains(retryPolicy.toString())); + Assertions.assertNotNull(rfe, name + " does not have RaftRetryFailureException"); + Assertions.assertTrue(rfe.getMessage().contains(retryPolicy.toString()), + name + ": unexpected error message, rfe=" + rfe + ", retryPolicy=" + retryPolicy); } @Test @@ -172,7 +174,7 @@ public abstract class RaftAsyncTests<CLUSTER extends MiniRaftCluster> extends Ba for (; i < messages.length; i++) { replies.add(client.async().send(messages[i])); } - Assert.assertEquals(messages.length, replies.size()); + Assertions.assertEquals(messages.length, replies.size()); } // sleep again so that the first half calls will fail retries. @@ -229,7 +231,7 @@ public abstract class RaftAsyncTests<CLUSTER extends MiniRaftCluster> extends Ba futures[i] = client.async().send(messages[i]); blockedRequestsCount.decrementAndGet(); } - Assert.assertEquals(0, blockedRequestsCount.get()); + Assertions.assertEquals(0, blockedRequestsCount.get()); futures[numMessages] = CompletableFuture.supplyAsync(() -> { blockedRequestsCount.incrementAndGet(); @@ -242,7 +244,7 @@ public abstract class RaftAsyncTests<CLUSTER extends MiniRaftCluster> extends Ba while (blockedRequestsCount.get() != 1) { Thread.sleep(1000); } - Assert.assertEquals(1, blockedRequestsCount.get()); + Assertions.assertEquals(1, blockedRequestsCount.get()); //Since all semaphore permits are acquired the last message sent is in queue RaftClientTestUtil.assertAsyncRequestSemaphore(client, 0, 1); @@ -255,7 +257,7 @@ public abstract class RaftAsyncTests<CLUSTER extends MiniRaftCluster> extends Ba for (int i = 0; i <= numMessages; i++) { futures[i].join(); } - Assert.assertEquals(0, blockedRequestsCount.get()); + Assertions.assertEquals(0, blockedRequestsCount.get()); } } @@ -297,11 +299,11 @@ public abstract class RaftAsyncTests<CLUSTER extends MiniRaftCluster> extends Ba LOG.info("sendAsync " + s); futures.add(client.async().send(new SimpleMessage(s))); } - Assert.assertEquals(numMessages, futures.size()); + Assertions.assertEquals(numMessages, futures.size()); final List<RaftClientReply> replies = new ArrayList<>(); for (CompletableFuture<RaftClientReply> f : futures) { final RaftClientReply r = f.join(); - Assert.assertTrue(r.isSuccess()); + Assertions.assertTrue(r.isSuccess()); replies.add(r); } futures.clear(); @@ -309,7 +311,7 @@ public abstract class RaftAsyncTests<CLUSTER extends MiniRaftCluster> extends Ba // Use a follower with the max commit index final RaftClientReply lastWriteReply = replies.get(replies.size() - 1); final RaftPeerId leader = lastWriteReply.getServerId(); - Assert.assertEquals(leader, lastWriteReply.getServerId()); + Assertions.assertEquals(leader, lastWriteReply.getServerId()); LOG.info("leader = " + leader); final Collection<CommitInfoProto> commitInfos = lastWriteReply.getCommitInfos(); LOG.info("commitInfos = " + commitInfos); @@ -356,7 +358,7 @@ public abstract class RaftAsyncTests<CLUSTER extends MiniRaftCluster> extends Ba throw new CompletionException(e); } - Assert.assertEquals("log entry mismatch for query=" + query, expected, computed); + Assertions.assertEquals(expected, computed, "log entry mismatch for query=" + query); return null; })); } @@ -383,14 +385,14 @@ public abstract class RaftAsyncTests<CLUSTER extends MiniRaftCluster> extends Ba if (exception != null) { LOG.error("Failed to send message " + s, exception); // reply should be null in case of exception - Assert.assertNull(reply); + Assertions.assertNull(reply); return; } - Assert.assertTrue(reply.isSuccess()); - Assert.assertNull(reply.getException()); + Assertions.assertTrue(reply.isSuccess()); + Assertions.assertNull(reply.getException()); // verify that all servers have caught up to log index when the reply is returned reply.getCommitInfos().forEach(commitInfoProto -> - Assert.assertTrue(commitInfoProto.getCommitIndex() >= reply.getLogIndex())); + Assertions.assertTrue(commitInfoProto.getCommitIndex() >= reply.getLogIndex())); }); } } @@ -436,15 +438,15 @@ public abstract class RaftAsyncTests<CLUSTER extends MiniRaftCluster> extends Ba CompletableFuture<RaftClientReply> replyFuture = client.async().send(new SimpleMessage("abc")); Thread.sleep(waitTime); // replyFuture should not be completed until append request is unblocked. - Assert.assertFalse(replyFuture.isDone()); + Assertions.assertFalse(replyFuture.isDone()); // unblock append request. cluster.getServerAliveStream() .filter(impl -> !impl.getInfo().isLeader() && !impl.getPeer().getId().equals(leader)) .map(SimpleStateMachine4Testing::get) .forEach(SimpleStateMachine4Testing::unblockWriteStateMachineData); - Assert.assertTrue(replyFuture.get().isSuccess()); - Assert.assertTrue(System.currentTimeMillis() - time > waitTime); + Assertions.assertTrue(replyFuture.get().isSuccess()); + Assertions.assertTrue(System.currentTimeMillis() - time > waitTime); } //reset for the other tests @@ -481,7 +483,7 @@ public abstract class RaftAsyncTests<CLUSTER extends MiniRaftCluster> extends Ba // previous leader should not there. cluster.getServerAliveStream() .map(RaftServer.Division::getInfo) - .forEach(info -> Assert.assertTrue(!info.isLeader() || info.getCurrentTerm() > termOfPrevLeader)); + .forEach(info -> Assertions.assertTrue(!info.isLeader() || info.getCurrentTerm() > termOfPrevLeader)); } finally { // unblock append entries request @@ -494,7 +496,7 @@ public abstract class RaftAsyncTests<CLUSTER extends MiniRaftCluster> extends Ba LOG.info("Current Leader is elected on term {}", termOfCurrLeader); // leader on termOfPrevLeader should step-down. - Assert.assertTrue(termOfPrevLeader < termOfCurrLeader); + Assertions.assertTrue(termOfPrevLeader < termOfCurrLeader); } @Test @@ -507,10 +509,10 @@ public abstract class RaftAsyncTests<CLUSTER extends MiniRaftCluster> extends Ba private void runTestNoRetryWaitOnNotLeaderException(MiniRaftCluster cluster) throws Exception { final RaftServer.Division leader = waitForLeader(cluster); final List<RaftServer.Division> followers = cluster.getFollowers(); - Assert.assertNotNull(followers); - Assert.assertEquals(2, followers.size()); - Assert.assertNotSame(leader, followers.get(0)); - Assert.assertNotSame(leader, followers.get(1)); + Assertions.assertNotNull(followers); + Assertions.assertEquals(2, followers.size()); + Assertions.assertNotSame(leader, followers.get(0)); + Assertions.assertNotSame(leader, followers.get(1)); // send a message to make sure that the leader is ready try (final RaftClient client = cluster.createClient(leader.getId())) { diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java index 9b6d81121..3d2837cd6 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java @@ -33,9 +33,9 @@ import org.apache.ratis.server.raftlog.RaftLogIOException; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.io.IOException; @@ -83,17 +83,17 @@ public abstract class RaftExceptionBaseTest<CLUSTER extends MiniRaftCluster> String messageId, RaftPeerId server, RaftClientRpc rpc, CLUSTER cluster) throws IOException { final SimpleMessage message = new SimpleMessage(messageId); final RaftClientReply reply = rpc.sendRequest(cluster.newRaftClientRequest(ClientId.randomId(), server, message)); - Assert.assertNotNull(reply); + Assertions.assertNotNull(reply); Assume.assumeFalse(reply.isSuccess()); final NotLeaderException nle = reply.getNotLeaderException(); Objects.requireNonNull(nle); - Assert.assertEquals(expectedSuggestedLeader, nle.getSuggestedLeader().getId()); + Assertions.assertEquals(expectedSuggestedLeader, nle.getSuggestedLeader().getId()); return reply; } static void sendMessage(String message, RaftClient client) throws IOException { final RaftClientReply reply = client.io().send(new SimpleMessage(message)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } @Test @@ -115,7 +115,7 @@ public abstract class RaftExceptionBaseTest<CLUSTER extends MiniRaftCluster> LOG.info("Start changing the configuration: {}", Arrays.asList(change.allPeersInNewConf)); try (final RaftClient c2 = cluster.createClient(newLeader)) { RaftClientReply reply = c2.admin().setConfiguration(change.allPeersInNewConf); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } LOG.info(cluster.printServers()); @@ -127,9 +127,9 @@ public abstract class RaftExceptionBaseTest<CLUSTER extends MiniRaftCluster> final Collection<RaftPeer> peers = cluster.getPeers(); final Collection<RaftPeer> peersFromReply = reply.getNotLeaderException().getPeers(); - Assert.assertEquals(peers.size(), peersFromReply.size()); + Assertions.assertEquals(peers.size(), peersFromReply.size()); for (RaftPeer p : peersFromReply) { - Assert.assertTrue(peers.contains(p)); + Assertions.assertTrue(peers.contains(p)); } sendMessage("m2", client); @@ -143,10 +143,10 @@ public abstract class RaftExceptionBaseTest<CLUSTER extends MiniRaftCluster> void runTestGroupMismatchException(CLUSTER cluster) throws Exception { final RaftGroup clusterGroup = cluster.getGroup(); - Assert.assertEquals(NUM_PEERS, clusterGroup.getPeers().size()); + Assertions.assertEquals(NUM_PEERS, clusterGroup.getPeers().size()); final RaftGroup anotherGroup = RaftGroup.valueOf(RaftGroupId.randomId(), clusterGroup.getPeers()); - Assert.assertNotEquals(clusterGroup.getGroupId(), anotherGroup.getGroupId()); + Assertions.assertNotEquals(clusterGroup.getGroupId(), anotherGroup.getGroupId()); // Create client using another group try(RaftClient client = cluster.createClient(anotherGroup)) { diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java index 58dedaf8c..f8d92cf08 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java @@ -43,7 +43,6 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.ProtoUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; import org.junit.AssumptionViolatedException; import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; @@ -142,7 +141,7 @@ public interface RaftTestUtil { static RaftPeerId waitAndKillLeader(MiniRaftCluster cluster) throws InterruptedException { final RaftServer.Division leader = waitForLeader(cluster); - Assert.assertNotNull(leader); + Assertions.assertNotNull(leader); LOG.info("killing leader = " + leader); cluster.killServer(leader.getId()); @@ -229,7 +228,7 @@ public interface RaftTestUtil { e = log.get(termIndices[i].getIndex()); if (Arrays.equals(expectedMessages[j].getContent().toByteArray(), e.getStateMachineLogEntry().getLogData().toByteArray())) { - Assert.assertTrue(predicate.test(e)); + Assertions.assertTrue(predicate.test(e)); } } catch (IOException exception) { exception.printStackTrace(); @@ -342,16 +341,16 @@ public interface RaftTestUtil { static void assertLogEntries(List<LogEntryProto> entries, long expectedTerm, SimpleMessage... expectedMessages) { long logIndex = 0; - Assert.assertEquals(expectedMessages.length, entries.size()); + Assertions.assertEquals(expectedMessages.length, entries.size()); for (int i = 0; i < expectedMessages.length; i++) { final LogEntryProto e = entries.get(i); - Assert.assertTrue(e.getTerm() >= expectedTerm); + Assertions.assertTrue(e.getTerm() >= expectedTerm); if (e.getTerm() > expectedTerm) { expectedTerm = e.getTerm(); } - Assert.assertTrue(e.getIndex() > logIndex); + Assertions.assertTrue(e.getIndex() > logIndex); logIndex = e.getIndex(); - Assert.assertEquals(expectedMessages[i].getContent(), e.getStateMachineLogEntry().getLogData()); + Assertions.assertEquals(expectedMessages[i].getContent(), e.getStateMachineLogEntry().getLogData()); } } @@ -553,11 +552,11 @@ public interface RaftTestUtil { } static void assertSameLog(RaftLog expected, RaftLog computed) throws Exception { - Assert.assertEquals(expected.getLastEntryTermIndex(), computed.getLastEntryTermIndex()); + Assertions.assertEquals(expected.getLastEntryTermIndex(), computed.getLastEntryTermIndex()); final long lastIndex = expected.getNextIndex() - 1; - Assert.assertEquals(expected.getLastEntryTermIndex().getIndex(), lastIndex); + Assertions.assertEquals(expected.getLastEntryTermIndex().getIndex(), lastIndex); for(long i = 0; i < lastIndex; i++) { - Assert.assertEquals(expected.get(i), computed.get(i)); + Assertions.assertEquals(expected.get(i), computed.get(i)); } } @@ -588,8 +587,8 @@ public interface RaftTestUtil { } static void assertSuccessReply(RaftClientReply reply) { - Assert.assertNotNull("reply == null", reply); - Assert.assertTrue("reply is not success: " + reply, reply.isSuccess()); + Assertions.assertNotNull(reply, "reply == null"); + Assertions.assertTrue(reply.isSuccess(), "reply is not success: " + reply); } static void gc() throws InterruptedException { diff --git a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java index eea75592e..17fd28cbe 100644 --- a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java @@ -35,9 +35,9 @@ import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.statemachine.impl.BaseStateMachine; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.nio.charset.StandardCharsets; @@ -63,7 +63,7 @@ public abstract class ReadOnlyRequestTests<CLUSTER extends MiniRaftCluster> final Message waitAndIncrementMessage = new RaftTestUtil.SimpleMessage(WAIT_AND_INCREMENT); final Message queryMessage = new RaftTestUtil.SimpleMessage(QUERY); - @Before + @BeforeEach public void setup() { final RaftProperties p = getProperties(); p.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, @@ -90,9 +90,9 @@ public abstract class ReadOnlyRequestTests<CLUSTER extends MiniRaftCluster> try (final RaftClient client = cluster.createClient(leaderId)) { for (int i = 1; i <= 10; i++) { RaftClientReply reply = client.io().send(incrementMessage); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); reply = client.io().sendReadOnly(queryMessage); - Assert.assertEquals(i, retrieve(reply)); + Assertions.assertEquals(i, retrieve(reply)); } } } finally { @@ -123,10 +123,10 @@ public abstract class ReadOnlyRequestTests<CLUSTER extends MiniRaftCluster> CompletableFuture<RaftClientReply> result = client.async().send(incrementMessage); client.admin().transferLeadership(null, 200); - Assert.assertThrows(ReadIndexException.class, () -> { + Assertions.assertThrows(ReadIndexException.class, () -> { RaftClientReply timeoutReply = noRetry.io().sendReadOnly(queryMessage); - Assert.assertNotNull(timeoutReply.getException()); - Assert.assertTrue(timeoutReply.getException() instanceof ReadException); + Assertions.assertNotNull(timeoutReply.getException()); + Assertions.assertTrue(timeoutReply.getException() instanceof ReadException); }); } @@ -152,18 +152,18 @@ public abstract class ReadOnlyRequestTests<CLUSTER extends MiniRaftCluster> RaftTestUtil.waitForLeader(cluster); List<RaftServer.Division> followers = cluster.getFollowers(); - Assert.assertEquals(2, followers.size()); + Assertions.assertEquals(2, followers.size()); final RaftPeerId f0 = followers.get(0).getId(); final RaftPeerId f1 = followers.get(1).getId(); try (RaftClient client = cluster.createClient(cluster.getLeader().getId())) { for (int i = 1; i <= 10; i++) { final RaftClientReply reply = client.io().send(incrementMessage); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); final RaftClientReply read1 = client.io().sendReadOnly(queryMessage, f0); - Assert.assertEquals(i, retrieve(read1)); + Assertions.assertEquals(i, retrieve(read1)); final CompletableFuture<RaftClientReply> read2 = client.async().sendReadOnly(queryMessage, f1); - Assert.assertEquals(i, retrieve(read2.get(1, TimeUnit.SECONDS))); + Assertions.assertEquals(i, retrieve(read2.get(1, TimeUnit.SECONDS))); } } } finally { @@ -188,7 +188,7 @@ public abstract class ReadOnlyRequestTests<CLUSTER extends MiniRaftCluster> RaftTestUtil.waitForLeader(cluster); List<RaftServer.Division> followers = cluster.getFollowers(); - Assert.assertEquals(2, followers.size()); + Assertions.assertEquals(2, followers.size()); try (RaftClient leaderClient = cluster.createClient(cluster.getLeader().getId()); RaftClient followerClient1 = cluster.createClient(followers.get(0).getId())) { @@ -198,7 +198,7 @@ public abstract class ReadOnlyRequestTests<CLUSTER extends MiniRaftCluster> Thread.sleep(100); RaftClientReply clientReply = followerClient1.io().sendReadOnly(queryMessage, followers.get(0).getId()); - Assert.assertEquals(2, retrieve(clientReply)); + Assertions.assertEquals(2, retrieve(clientReply)); } } finally { @@ -223,20 +223,20 @@ public abstract class ReadOnlyRequestTests<CLUSTER extends MiniRaftCluster> RaftTestUtil.waitForLeader(cluster); List<RaftServer.Division> followers = cluster.getFollowers(); - Assert.assertEquals(2, followers.size()); + Assertions.assertEquals(2, followers.size()); try (RaftClient leaderClient = cluster.createClient(cluster.getLeader().getId()); RaftClient followerClient1 = cluster.createClient(followers.get(0).getId(), RetryPolicies.noRetry())) { leaderClient.io().send(incrementMessage); RaftClientReply clientReply = followerClient1.io().sendReadOnly(queryMessage); - Assert.assertEquals(1, retrieve(clientReply)); + Assertions.assertEquals(1, retrieve(clientReply)); // kill the leader // read timeout quicker than election timeout leaderClient.admin().transferLeadership(null, 200); - Assert.assertThrows(ReadIndexException.class, () -> { + Assertions.assertThrows(ReadIndexException.class, () -> { followerClient1.io().sendReadOnly(queryMessage, followers.get(0).getId()); }); } @@ -273,14 +273,14 @@ public abstract class ReadOnlyRequestTests<CLUSTER extends MiniRaftCluster> client.io().send(incrementMessage); final RaftClientReply clientReply = client.io().sendReadOnly(queryMessage); - Assert.assertEquals(1, retrieve(clientReply)); + Assertions.assertEquals(1, retrieve(clientReply)); // kill the leader client.admin().transferLeadership(null, 200); // readOnly will success after re-election final RaftClientReply replySuccess = client.io().sendReadOnly(queryMessage); - Assert.assertEquals(1, retrieve(clientReply)); + Assertions.assertEquals(1, retrieve(clientReply)); } } @@ -295,12 +295,12 @@ public abstract class ReadOnlyRequestTests<CLUSTER extends MiniRaftCluster> // test blocking read-after-write client.io().send(incrementMessage); final RaftClientReply blockReply = client.io().sendReadAfterWrite(queryMessage); - Assert.assertEquals(1, retrieve(blockReply)); + Assertions.assertEquals(1, retrieve(blockReply)); // test asynchronous read-after-write client.async().send(incrementMessage); client.async().sendReadAfterWrite(queryMessage).thenAccept(reply -> { - Assert.assertEquals(2, retrieve(reply)); + Assertions.assertEquals(2, retrieve(reply)); }); for (int i = 0; i < 20; i++) { @@ -311,7 +311,7 @@ public abstract class ReadOnlyRequestTests<CLUSTER extends MiniRaftCluster> CompletableFuture.allOf(linearizable, readAfterWrite).get(); // read-after-write is more consistent than linearizable read - Assert.assertTrue(retrieve(readAfterWrite.get()) >= retrieve(linearizable.get())); + Assertions.assertTrue(retrieve(readAfterWrite.get()) >= retrieve(linearizable.get())); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestWithLongTimeoutTests.java b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestWithLongTimeoutTests.java index 14aa1bca2..4a6498f93 100644 --- a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestWithLongTimeoutTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestWithLongTimeoutTests.java @@ -32,9 +32,9 @@ import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.concurrent.CompletableFuture; @@ -59,7 +59,7 @@ public abstract class ReadOnlyRequestWithLongTimeoutTests<CLUSTER extends MiniRa final Message timeoutMessage = new RaftTestUtil.SimpleMessage(TIMEOUT_INCREMENT); final Message queryMessage = new RaftTestUtil.SimpleMessage(QUERY); - @Before + @BeforeEach public void setup() { final RaftProperties p = getProperties(); p.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, @@ -86,17 +86,17 @@ public abstract class ReadOnlyRequestWithLongTimeoutTests<CLUSTER extends MiniRa try (RaftClient client = cluster.createClient(leaderId, RetryPolicies.noRetry())) { final RaftClientReply reply = client.io().send(incrementMessage); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); client.async().send(waitAndIncrementMessage); Thread.sleep(100); RaftClientReply staleValueBefore = client.io().sendStaleRead(queryMessage, 0, leaderId); - Assert.assertEquals(1, ReadOnlyRequestTests.retrieve(staleValueBefore)); + Assertions.assertEquals(1, ReadOnlyRequestTests.retrieve(staleValueBefore)); RaftClientReply linearizableReadValue = client.io().sendReadOnly(queryMessage); - Assert.assertEquals(2, ReadOnlyRequestTests.retrieve(linearizableReadValue)); + Assertions.assertEquals(2, ReadOnlyRequestTests.retrieve(linearizableReadValue)); } } @@ -111,14 +111,14 @@ public abstract class ReadOnlyRequestWithLongTimeoutTests<CLUSTER extends MiniRa try (RaftClient client = cluster.createClient(leaderId, RetryPolicies.noRetry())) { final RaftClientReply reply = client.io().send(incrementMessage); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); final CompletableFuture<RaftClientReply> asyncTimeoutReply = client.async().send(timeoutMessage); Thread.sleep(100); - Assert.assertThrows(ReadException.class, () -> { + Assertions.assertThrows(ReadException.class, () -> { final RaftClientReply timeoutReply = client.io().sendReadOnly(queryMessage); - Assert.assertTrue(timeoutReply.getException().getCause() instanceof TimeoutIOException); + Assertions.assertTrue(timeoutReply.getException().getCause() instanceof TimeoutIOException); }); asyncTimeoutReply.join(); diff --git a/ratis-server/src/test/java/org/apache/ratis/RequestLimitAsyncBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/RequestLimitAsyncBaseTest.java index 19a46a065..ce339faff 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RequestLimitAsyncBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/RequestLimitAsyncBaseTest.java @@ -31,8 +31,8 @@ import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.server.impl.RaftServerTestUtil; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.ArrayList; @@ -74,7 +74,7 @@ public abstract class RequestLimitAsyncBaseTest<CLUSTER extends MiniRaftCluster> final SimpleMessage message = new SimpleMessage("first"); final CompletableFuture<RaftClientReply> future = c1.async().send(message); final RaftClientReply reply = getWithDefaultTimeout(future); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } // collecting futures returned from StateMachine.applyTransaction @@ -118,7 +118,7 @@ public abstract class RequestLimitAsyncBaseTest<CLUSTER extends MiniRaftCluster> // check replies for(CompletableFuture<RaftClientReply> f : writeFutures) { final RaftClientReply reply = getWithDefaultTimeout(f); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } } diff --git a/ratis-server/src/test/java/org/apache/ratis/TestReConfigProperty.java b/ratis-server/src/test/java/org/apache/ratis/TestReConfigProperty.java index 4535406a7..7d1aec901 100644 --- a/ratis-server/src/test/java/org/apache/ratis/TestReConfigProperty.java +++ b/ratis-server/src/test/java/org/apache/ratis/TestReConfigProperty.java @@ -27,9 +27,10 @@ import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; import java.io.IOException; @@ -59,7 +60,7 @@ public abstract class TestReConfigProperty<CLUSTER extends MiniRaftCluster> exte private static final String VAL2 = "val2"; private static final String DEFAULT = "default"; - @Before + @BeforeEach public void setup () { conf1 = new RaftProperties(); conf2 = new RaftProperties(); @@ -81,8 +82,7 @@ public abstract class TestReConfigProperty<CLUSTER extends MiniRaftCluster> exte Collection<PropertyChange> changes = ReconfigurationBase.getChangedProperties(conf2, conf1); - Assert.assertTrue("expected 3 changed properties but got " + changes.size(), - changes.size() == 3); + Assertions.assertEquals(3, changes.size(), "Unexpected changed size"); boolean changeFound = false; boolean unsetFound = false; @@ -100,8 +100,7 @@ public abstract class TestReConfigProperty<CLUSTER extends MiniRaftCluster> exte setFound = true; } } - Assert.assertTrue("not all changes have been applied", - changeFound && unsetFound && setFound); + Assertions.assertTrue(changeFound && unsetFound && setFound, "not all changes have been applied"); } /** @@ -157,161 +156,133 @@ public abstract class TestReConfigProperty<CLUSTER extends MiniRaftCluster> exte public void testReconfigure() { ReconfigurableDummy dummy = new ReconfigurableDummy(conf1); - Assert.assertEquals(PROP1 + " set to wrong value ", VAL1, dummy.getProperties().get(PROP1)); - Assert.assertEquals(PROP2 + " set to wrong value ", VAL1, dummy.getProperties().get(PROP2)); - Assert.assertEquals(PROP3 + " set to wrong value ", VAL1, dummy.getProperties().get(PROP3)); - Assert.assertNull(PROP4 + " set to wrong value ", dummy.getProperties().get(PROP4)); - Assert.assertNull(PROP5 + " set to wrong value ", dummy.getProperties().get(PROP5)); - - Assert.assertTrue(PROP1 + " should be reconfigurable ", - dummy.isPropertyReconfigurable(PROP1)); - Assert.assertTrue(PROP2 + " should be reconfigurable ", - dummy.isPropertyReconfigurable(PROP2)); - Assert.assertFalse(PROP3 + " should not be reconfigurable ", - dummy.isPropertyReconfigurable(PROP3)); - Assert.assertTrue(PROP4 + " should be reconfigurable ", - dummy.isPropertyReconfigurable(PROP4)); - Assert.assertFalse(PROP5 + " should not be reconfigurable ", - dummy.isPropertyReconfigurable(PROP5)); + Assertions.assertEquals(VAL1, dummy.getProperties().get(PROP1), PROP1 + " set to wrong value "); + Assertions.assertEquals(VAL1, dummy.getProperties().get(PROP2), PROP2 + " set to wrong value "); + Assertions.assertEquals(VAL1, dummy.getProperties().get(PROP3), PROP3 + " set to wrong value "); + Assertions.assertNull(dummy.getProperties().get(PROP4), PROP4 + " set to wrong value "); + Assertions.assertNull(dummy.getProperties().get(PROP5), PROP5 + " set to wrong value "); + + Assertions.assertTrue(dummy.isPropertyReconfigurable(PROP1), PROP1 + " should be reconfigurable "); + Assertions.assertTrue(dummy.isPropertyReconfigurable(PROP2), PROP2 + " should be reconfigurable "); + Assertions.assertFalse(dummy.isPropertyReconfigurable(PROP3), PROP3 + " should not be reconfigurable "); + Assertions.assertTrue(dummy.isPropertyReconfigurable(PROP4), PROP4 + " should be reconfigurable "); + Assertions.assertFalse(dummy.isPropertyReconfigurable(PROP5), PROP5 + " should not be reconfigurable "); // change something to the same value as before { - boolean exceptionCaught = false; try { dummy.reconfigureProperty(PROP1, VAL1); dummy.startReconfiguration(); RaftTestUtil.waitFor(() -> dummy.getReconfigurationStatus().ended(), 100, 60000); - Assert.assertEquals(PROP1 + " set to wrong value ", VAL1, dummy.getProperties().get(PROP1)); + Assertions.assertEquals(VAL1, dummy.getProperties().get(PROP1), PROP1 + " set to wrong value "); } catch (ReconfigurationException | IOException | TimeoutException | InterruptedException e) { - exceptionCaught = true; + Assertions.fail("Unexpected exception", e); } - Assert.assertFalse("received unexpected exception", - exceptionCaught); } // change something to null { - boolean exceptionCaught = false; try { dummy.reconfigureProperty(PROP1, null); dummy.startReconfiguration(); RaftTestUtil.waitFor(() -> dummy.getReconfigurationStatus().ended(), 100, 60000); - Assert.assertEquals(PROP1 + "set to wrong value ", DEFAULT, - dummy.getProperties().get(PROP1)); + Assertions.assertEquals(DEFAULT +, dummy.getProperties().get(PROP1), PROP1 + "set to wrong value "); } catch (ReconfigurationException | IOException | InterruptedException | TimeoutException e) { - exceptionCaught = true; + Assertions.fail("Unexpected exception", e); } - Assert.assertFalse("received unexpected exception", - exceptionCaught); } // change something to a different value than before { - boolean exceptionCaught = false; try { dummy.reconfigureProperty(PROP1, VAL2); dummy.startReconfiguration(); RaftTestUtil.waitFor(() -> dummy.getReconfigurationStatus().ended(), 100, 60000); - Assert.assertEquals(PROP1 + "set to wrong value ", VAL2, dummy.getProperties().get(PROP1)); + Assertions.assertEquals(VAL2, dummy.getProperties().get(PROP1), PROP1 + "set to wrong value "); } catch (ReconfigurationException | IOException | InterruptedException | TimeoutException e) { - exceptionCaught = true; + Assertions.fail("Unexpected exception", e); } - Assert.assertFalse("received unexpected exception", - exceptionCaught); } // set unset property to null { - boolean exceptionCaught = false; try { dummy.reconfigureProperty(PROP4, null); dummy.startReconfiguration(); RaftTestUtil.waitFor(() -> dummy.getReconfigurationStatus().ended(), 100, 60000); - Assert.assertSame(PROP4 + "set to wrong value ", DEFAULT, dummy.getProperties().get(PROP4)); + Assertions.assertSame(DEFAULT, dummy.getProperties().get(PROP4), PROP4 + "set to wrong value "); } catch (ReconfigurationException | IOException | InterruptedException | TimeoutException e) { - exceptionCaught = true; + Assertions.fail("Unexpected exception", e); } - Assert.assertFalse("received unexpected exception", - exceptionCaught); } // set unset property { - boolean exceptionCaught = false; try { dummy.reconfigureProperty(PROP4, VAL1); dummy.startReconfiguration(); RaftTestUtil.waitFor(() -> dummy.getReconfigurationStatus().ended(), 100, 60000); - Assert.assertEquals(PROP4 + "set to wrong value ", VAL1, dummy.getProperties().get(PROP4)); + Assertions.assertEquals(VAL1, dummy.getProperties().get(PROP4), PROP4 + "set to wrong value "); } catch (ReconfigurationException | IOException | InterruptedException | TimeoutException e) { - exceptionCaught = true; + Assertions.fail("Unexpected exception", e); } - Assert.assertFalse("received unexpected exception", - exceptionCaught); } // try to set unset property to null (not reconfigurable) { - boolean exceptionCaught = false; try { dummy.reconfigureProperty(PROP5, null); dummy.startReconfiguration(); RaftTestUtil.waitFor(() -> dummy.getReconfigurationStatus().ended(), 100, 60000); } catch (ReconfigurationException | IOException | InterruptedException | TimeoutException e) { - exceptionCaught = true; + Assertions.fail("Unexpected exception", e); } - Assert.assertTrue("did not receive expected exception", - dummy.getReconfigurationStatus().getChanges() + Assertions.assertTrue(dummy.getReconfigurationStatus().getChanges() .get(new PropertyChange(PROP5, DEFAULT, null)) - .getMessage().contains("Property is not reconfigurable.") && !exceptionCaught); + .getMessage().contains("Property is not reconfigurable."), "did not receive expected exception"); } // try to set unset property to value (not reconfigurable) { - boolean exceptionCaught = false; try { dummy.reconfigureProperty(PROP5, VAL1); dummy.startReconfiguration(); RaftTestUtil.waitFor(() -> dummy.getReconfigurationStatus().ended(), 100, 60000); } catch (ReconfigurationException | IOException | InterruptedException | TimeoutException e) { - exceptionCaught = true; + Assertions.fail("Unexpected exception", e); } - Assert.assertTrue("did not receive expected exception", - dummy.getReconfigurationStatus().getChanges() + Assertions.assertTrue(dummy.getReconfigurationStatus().getChanges() .get(new PropertyChange(PROP5, VAL1, null)) - .getMessage().contains("Property is not reconfigurable.") && !exceptionCaught); + .getMessage().contains("Property is not reconfigurable."), "did not receive expected exception"); } // try to change property to value (not reconfigurable) { - boolean exceptionCaught = false; try { dummy.reconfigureProperty(PROP3, VAL2); dummy.startReconfiguration(); RaftTestUtil.waitFor(() -> dummy.getReconfigurationStatus().ended(), 100, 60000); } catch (ReconfigurationException | IOException | InterruptedException | TimeoutException e) { - exceptionCaught = true; + Assertions.fail("Unexpected exception", e); } - Assert.assertTrue("did not receive expected exception", - dummy.getReconfigurationStatus().getChanges() + Assertions.assertTrue(dummy.getReconfigurationStatus().getChanges() .get(new PropertyChange(PROP3, VAL2, VAL1)) - .getMessage().contains("Property is not reconfigurable.") && !exceptionCaught); + .getMessage().contains("Property is not reconfigurable."), "did not receive expected exception"); } // try to change property to null (not reconfigurable) { - boolean exceptionCaught = false; try { dummy.reconfigureProperty(PROP3, null); dummy.startReconfiguration(); RaftTestUtil.waitFor(() -> dummy.getReconfigurationStatus().ended(), 100, 60000); } catch (ReconfigurationException | IOException | InterruptedException | TimeoutException e) { - exceptionCaught = true; + Assertions.fail("Unexpected exception", e); } - Assert.assertTrue("did not receive expected exception", - dummy.getReconfigurationStatus().getChanges() + Assertions.assertTrue(dummy.getReconfigurationStatus().getChanges() .get(new PropertyChange(PROP3, DEFAULT, VAL1)) - .getMessage().contains("Property is not reconfigurable.") && !exceptionCaught); + .getMessage().contains("Property is not reconfigurable."), "did not receive expected exception"); } } @@ -321,7 +292,7 @@ public abstract class TestReConfigProperty<CLUSTER extends MiniRaftCluster> exte @Test public void testThread() throws ReconfigurationException, IOException { ReconfigurableDummy dummy = new ReconfigurableDummy(conf1); - Assert.assertEquals(VAL1, dummy.getProperties().get(PROP1)); + Assertions.assertEquals(VAL1, dummy.getProperties().get(PROP1)); Thread dummyThread = new Thread(dummy); dummyThread.start(); try { @@ -341,25 +312,23 @@ public abstract class TestReConfigProperty<CLUSTER extends MiniRaftCluster> exte } } - Assert.assertFalse("dummy thread should not be alive", - dummyThread.isAlive()); + Assertions.assertFalse(dummyThread.isAlive(), "dummy thread should not be alive"); dummy.running = false; try { dummyThread.join(); } catch (InterruptedException ignore) { // do nothing } - Assert.assertTrue(PROP1 + " is set to wrong value", - dummy.getProperties().get(PROP1).equals(VAL2)); + Assertions.assertEquals(VAL2, dummy.getProperties().get(PROP1), PROP1 + " is set to wrong value"); } /** * Ensure that {@link ReconfigurationBase#reconfigureProperty} updates the * parent's cached configuration on success. - * @throws IOException */ - @Test (timeout=300000) + @Test + @Timeout(value = 300) public void testConfIsUpdatedOnSuccess() throws ReconfigurationException, IOException, InterruptedException, TimeoutException { final String property = "FOO"; @@ -377,15 +346,15 @@ public abstract class TestReConfigProperty<CLUSTER extends MiniRaftCluster> exte reconfigurable.reconfigureProperty(property, value2); reconfigurable.startReconfiguration(); RaftTestUtil.waitFor(() -> reconfigurable.getReconfigurationStatus().ended(), 100, 60000); - Assert.assertEquals(value2, reconfigurable.getProperties().get(property)); + Assertions.assertEquals(value2, reconfigurable.getProperties().get(property)); } /** * Ensure that {@link ReconfigurationBase#startReconfiguration} updates * its parent's cached configuration on success. - * @throws IOException */ - @Test (timeout=300000) + @Test + @Timeout(value = 300) public void testConfIsUpdatedOnSuccessAsync() throws InterruptedException, IOException, TimeoutException { final String property = "FOO"; @@ -404,7 +373,7 @@ public abstract class TestReConfigProperty<CLUSTER extends MiniRaftCluster> exte reconfigurable.startReconfiguration(); RaftTestUtil.waitFor(() -> reconfigurable.getReconfigurationStatus().ended(), 100, 60000); - Assert.assertEquals(value2, reconfigurable.getProperties().get(property)); + Assertions.assertEquals(value2, reconfigurable.getProperties().get(property)); } /** @@ -412,7 +381,8 @@ public abstract class TestReConfigProperty<CLUSTER extends MiniRaftCluster> exte * property in its parent's configuration when the new value is null. * @throws IOException */ - @Test (timeout=300000) + @Test + @Timeout(value = 300) public void testConfIsUnset() throws InterruptedException, TimeoutException, IOException { final String property = "FOO"; @@ -427,15 +397,15 @@ public abstract class TestReConfigProperty<CLUSTER extends MiniRaftCluster> exte reconfigurable.startReconfiguration(); RaftTestUtil.waitFor(() -> reconfigurable.getReconfigurationStatus().ended(), 100, 60000); - Assert.assertNull(reconfigurable.getProperties().get(property)); + Assertions.assertNull(reconfigurable.getProperties().get(property)); } /** * Ensure that {@link ReconfigurationBase#startReconfiguration} unsets the * property in its parent's configuration when the new value is null. - * @throws IOException */ - @Test (timeout=300000) + @Test + @Timeout(value = 300) public void testConfIsUnsetAsync() throws ReconfigurationException, IOException, TimeoutException, InterruptedException { final String property = "FOO"; @@ -451,7 +421,7 @@ public abstract class TestReConfigProperty<CLUSTER extends MiniRaftCluster> exte // Kick off a reconfiguration task and wait until it completes. reconfigurable.startReconfiguration(); RaftTestUtil.waitFor(() -> reconfigurable.getReconfigurationStatus().ended(), 100, 60000); - Assert.assertNull(reconfigurable.getProperties().get(property)); + Assertions.assertNull(reconfigurable.getProperties().get(property)); } private ReconfigurationBase makeReconfigurable( diff --git a/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java b/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java index b842ee9db..a2c0cd2e1 100644 --- a/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java @@ -42,9 +42,9 @@ import org.apache.ratis.util.ProtoUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedConsumer; import org.apache.ratis.util.function.CheckedSupplier; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.event.Level; @@ -57,7 +57,7 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.stream.Stream; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.fail; public abstract class WatchRequestTests<CLUSTER extends MiniRaftCluster> extends BaseTest @@ -70,7 +70,7 @@ public abstract class WatchRequestTests<CLUSTER extends MiniRaftCluster> static final int NUM_SERVERS = 3; static final int GET_TIMEOUT_SECOND = 10; - @Before + @BeforeEach public void setup() { final RaftProperties p = getProperties(); p.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, @@ -128,7 +128,7 @@ public abstract class WatchRequestTests<CLUSTER extends MiniRaftCluster> CompletableFuture<RaftClientReply> reply = watchClient.async().send(new RaftTestUtil.SimpleMessage("message")); long writeIndex = reply.get().getLogIndex(); - Assert.assertTrue(writeIndex > 0); + Assertions.assertTrue(writeIndex > 0); watchClient.async().watch(writeIndex, ReplicationLevel.MAJORITY_COMMITTED); return watchClient.async().watch(logIndex, ReplicationLevel.MAJORITY); } @@ -212,8 +212,8 @@ public abstract class WatchRequestTests<CLUSTER extends MiniRaftCluster> } log.info("{}-Watch({}) returns {}", name, logIndex, reply); - Assert.assertTrue(reply.isSuccess()); - Assert.assertTrue(reply.getLogIndex() >= logIndex); + Assertions.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.getLogIndex() >= logIndex); return reply; } } @@ -240,8 +240,8 @@ public abstract class WatchRequestTests<CLUSTER extends MiniRaftCluster> p.sendRequests(replies, watches); - Assert.assertEquals(numMessages, replies.size()); - Assert.assertEquals(numMessages, watches.size()); + Assertions.assertEquals(numMessages, replies.size()); + Assertions.assertEquals(numMessages, watches.size()); // since leader is blocked, nothing can be done. TimeUnit.SECONDS.sleep(1); @@ -254,7 +254,7 @@ public abstract class WatchRequestTests<CLUSTER extends MiniRaftCluster> checkMajority(replies, watches, log); - Assert.assertEquals(numMessages, watches.size()); + Assertions.assertEquals(numMessages, watches.size()); // but not replicated/committed to all. TimeUnit.SECONDS.sleep(1); @@ -273,26 +273,26 @@ public abstract class WatchRequestTests<CLUSTER extends MiniRaftCluster> final RaftClientReply reply = replies.get(i).get(GET_TIMEOUT_SECOND, TimeUnit.SECONDS); log.info("checkMajority {}: receive {}", i, reply); final long logIndex = reply.getLogIndex(); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); final WatchReplies watchReplies = watches.get(i).get(GET_TIMEOUT_SECOND, TimeUnit.SECONDS); - Assert.assertEquals(logIndex, watchReplies.logIndex); + Assertions.assertEquals(logIndex, watchReplies.logIndex); final RaftClientReply watchMajorityReply = watchReplies.getMajority(); final RaftClientReply watchMajorityCommittedReply = watchReplies.getMajorityCommitted(); { // check commit infos final Collection<CommitInfoProto> commitInfos = watchMajorityCommittedReply.getCommitInfos(); final String message = "logIndex=" + logIndex + ", " + ProtoUtils.toString(commitInfos); - Assert.assertEquals(NUM_SERVERS, commitInfos.size()); + Assertions.assertEquals(NUM_SERVERS, commitInfos.size()); // One follower has not committed, so min must be less than logIndex final long min = commitInfos.stream().map(CommitInfoProto::getCommitIndex).min(Long::compare).get(); - Assert.assertTrue(message, logIndex > min); + Assertions.assertTrue(logIndex > min, message); // All other followers have committed commitInfos.stream() .map(CommitInfoProto::getCommitIndex).sorted(Long::compare) - .skip(1).forEach(ci -> Assert.assertTrue(message, logIndex <= ci)); + .skip(1).forEach(ci -> Assertions.assertTrue(logIndex <= ci, message)); } } } @@ -308,8 +308,8 @@ public abstract class WatchRequestTests<CLUSTER extends MiniRaftCluster> { // check commit infos final Collection<CommitInfoProto> commitInfos = watchAllCommittedReply.getCommitInfos(); final String message = "logIndex=" + logIndex + ", " + ProtoUtils.toString(commitInfos); - Assert.assertEquals(NUM_SERVERS, commitInfos.size()); - commitInfos.forEach(info -> Assert.assertTrue(message, logIndex <= info.getCommitIndex())); + Assertions.assertEquals(NUM_SERVERS, commitInfos.size()); + commitInfos.forEach(info -> Assertions.assertTrue(logIndex <= info.getCommitIndex(), message)); } } } @@ -352,8 +352,8 @@ public abstract class WatchRequestTests<CLUSTER extends MiniRaftCluster> p.sendRequests(replies, watches); - Assert.assertEquals(numMessages, replies.size()); - Assert.assertEquals(numMessages, watches.size()); + Assertions.assertEquals(numMessages, replies.size()); + Assertions.assertEquals(numMessages, watches.size()); // since only one follower is blocked commit, requests can be committed MAJORITY and ALL but not ALL_COMMITTED. checkMajority(replies, watches, log); @@ -410,8 +410,8 @@ public abstract class WatchRequestTests<CLUSTER extends MiniRaftCluster> p.sendRequests(replies, watches); - Assert.assertEquals(numMessages, replies.size()); - Assert.assertEquals(numMessages, watches.size()); + Assertions.assertEquals(numMessages, replies.size()); + Assertions.assertEquals(numMessages, watches.size()); watchTimeout.sleep(); watchTimeoutDenomination.sleep(); // for roundup error @@ -461,11 +461,11 @@ public abstract class WatchRequestTests<CLUSTER extends MiniRaftCluster> fail("runTestWatchRequestClientTimeout failed"); } catch (Exception ex) { log.error("error occurred", ex); - Assert.assertTrue(ex.getCause().getClass() == AlreadyClosedException.class || + Assertions.assertTrue(ex.getCause().getClass() == AlreadyClosedException.class || ex.getCause().getClass() == RaftRetryFailureException.class); if (ex.getCause() != null) { if (ex.getCause().getCause() != null) { - Assert.assertEquals(TimeoutIOException.class, + Assertions.assertEquals(TimeoutIOException.class, ex.getCause().getCause().getClass()); } } @@ -531,10 +531,10 @@ public abstract class WatchRequestTests<CLUSTER extends MiniRaftCluster> final RaftClientReply reply = replies.get(i).get(GET_TIMEOUT_SECOND, TimeUnit.SECONDS); log.info("checkTimeout {}: receive {}", i, reply); final long logIndex = reply.getLogIndex(); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); final WatchReplies watchReplies = watches.get(i).get(GET_TIMEOUT_SECOND, TimeUnit.SECONDS); - Assert.assertEquals(logIndex, watchReplies.logIndex); + Assertions.assertEquals(logIndex, watchReplies.logIndex); assertNotReplicatedException(logIndex, ReplicationLevel.ALL, watchReplies::getAll); @@ -554,11 +554,11 @@ public abstract class WatchRequestTests<CLUSTER extends MiniRaftCluster> } static void assertNotReplicatedException(long logIndex, ReplicationLevel replication, Throwable t) { - Assert.assertSame(NotReplicatedException.class, t.getClass()); + Assertions.assertSame(NotReplicatedException.class, t.getClass()); final NotReplicatedException nre = (NotReplicatedException) t; - Assert.assertNotNull(nre); - Assert.assertEquals(logIndex, nre.getLogIndex()); - Assert.assertEquals(replication, nre.getRequiredReplication()); - Assert.assertNotNull(nre.getCommitInfos()); + Assertions.assertNotNull(nre); + Assertions.assertEquals(logIndex, nre.getLogIndex()); + Assertions.assertEquals(replication, nre.getRequiredReplication()); + Assertions.assertNotNull(nre.getCommitInfos()); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupInfoBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupInfoBaseTest.java index 72eedf0f3..218b6e067 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupInfoBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupInfoBaseTest.java @@ -24,8 +24,8 @@ import org.apache.ratis.protocol.*; import org.apache.ratis.proto.RaftProtos.CommitInfoProto; import org.apache.ratis.server.RaftServer; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.Collection; @@ -92,7 +92,7 @@ public abstract class GroupInfoBaseTest<CLUSTER extends MiniRaftCluster> final RaftClientReply reply = sendMessages(numMessages, cluster); for(CommitInfoProto i : reply.getCommitInfos()) { if (!RaftPeerId.valueOf(i.getServer().getId()).equals(killedFollower)) { - Assert.assertTrue(i.getCommitIndex() > maxCommit); + Assertions.assertTrue(i.getCommitIndex() > maxCommit); } } } @@ -104,12 +104,12 @@ public abstract class GroupInfoBaseTest<CLUSTER extends MiniRaftCluster> } try(final RaftClient client = cluster.createClient(peer.getId())) { final GroupListReply info = client.getGroupManagementApi(peer.getId()).list(); - Assert.assertEquals(1, info.getGroupIds().stream().filter(id -> group.getGroupId().equals(id)).count()); + Assertions.assertEquals(1, info.getGroupIds().stream().filter(id -> group.getGroupId().equals(id)).count()); for(CommitInfoProto i : info.getCommitInfos()) { if (RaftPeerId.valueOf(i.getServer().getId()).equals(killedFollower)) { - Assert.assertTrue(i.getCommitIndex() <= maxCommit); + Assertions.assertTrue(i.getCommitIndex() <= maxCommit); } else { - Assert.assertTrue(i.getCommitIndex() > maxCommit); + Assertions.assertTrue(i.getCommitIndex() > maxCommit); } } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java index a06336f91..a86ab2a51 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java @@ -36,8 +36,8 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedBiConsumer; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; @@ -97,7 +97,7 @@ public abstract class GroupManagementBaseTest extends BaseTest { // Make sure that there are no leaders. TimeUnit.SECONDS.sleep(1); LOG.info("start: " + cluster.printServers()); - Assert.assertNull(cluster.getLeader()); + Assertions.assertNull(cluster.getLeader()); // Add groups List<RaftPeer> peers = cluster.getPeers(); @@ -108,7 +108,7 @@ public abstract class GroupManagementBaseTest extends BaseTest { LOG.info("add new group: " + newGroup); try (final RaftClient client = cluster.createClient(newGroup)) { // Before request, client try leader with the highest priority - Assert.assertTrue(client.getLeaderId() == peersWithPriority.get(suggestedLeaderIndex).getId()); + Assertions.assertTrue(client.getLeaderId() == peersWithPriority.get(suggestedLeaderIndex).getId()); for (RaftPeer p : newGroup.getPeers()) { client.getGroupManagementApi(p.getId()).add(newGroup); } @@ -116,7 +116,7 @@ public abstract class GroupManagementBaseTest extends BaseTest { JavaUtils.attempt(() -> { final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster, newGroup.getGroupId()); - Assert.assertTrue(leader.getId() == peers.get(suggestedLeaderIndex).getId()); + Assertions.assertTrue(leader.getId() == peers.get(suggestedLeaderIndex).getId()); }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testMultiGroupWithPriority", LOG); String suggestedLeader = peers.get(suggestedLeaderIndex).getId().toString(); @@ -130,7 +130,7 @@ public abstract class GroupManagementBaseTest extends BaseTest { JavaUtils.attempt(() -> { final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster, newGroup.getGroupId()); - Assert.assertTrue(leader.getId() != peers.get(suggestedLeaderIndex).getId()); + Assertions.assertTrue(leader.getId() != peers.get(suggestedLeaderIndex).getId()); }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testMultiGroupWithPriority", LOG); // send request so that suggested leader's log lag behind new leader's, @@ -138,7 +138,7 @@ public abstract class GroupManagementBaseTest extends BaseTest { try (final RaftClient client = cluster.createClient(newGroup)) { for (int i = 0; i < 10; i ++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -150,7 +150,7 @@ public abstract class GroupManagementBaseTest extends BaseTest { // leadership to suggested leader when suggested leader catch up the log. JavaUtils.attempt(() -> { final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster, newGroup.getGroupId()); - Assert.assertTrue(leader.getId() == peers.get(suggestedLeaderIndex).getId()); + Assertions.assertTrue(leader.getId() == peers.get(suggestedLeaderIndex).getId()); }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testMultiGroupWithPriority", LOG); // change the suggest leader @@ -158,18 +158,18 @@ public abstract class GroupManagementBaseTest extends BaseTest { List<RaftPeer> peersWithNewPriority = getPeersWithPriority(peers, peers.get(newSuggestedLeaderIndex)); try (final RaftClient client = cluster.createClient(newGroup)) { RaftClientReply reply = client.admin().setConfiguration(peersWithNewPriority.toArray(new RaftPeer[0])); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } JavaUtils.attempt(() -> { final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster, newGroup.getGroupId()); - Assert.assertTrue(leader.getId() == peers.get(newSuggestedLeaderIndex).getId()); + Assertions.assertTrue(leader.getId() == peers.get(newSuggestedLeaderIndex).getId()); }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testMultiGroupWithPriority", LOG); cluster.killServer(peers.get(newSuggestedLeaderIndex).getId()); JavaUtils.attempt(() -> { final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster, newGroup.getGroupId()); - Assert.assertTrue(leader.getId() != peers.get(newSuggestedLeaderIndex).getId()); + Assertions.assertTrue(leader.getId() != peers.get(newSuggestedLeaderIndex).getId()); }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testMultiGroupWithPriority", LOG); cluster.shutdown(); @@ -191,7 +191,7 @@ public abstract class GroupManagementBaseTest extends BaseTest { // Make sure that there are no leaders. TimeUnit.SECONDS.sleep(1); LOG.info("start: " + cluster.printServers()); - Assert.assertNull(cluster.getLeader()); + Assertions.assertNull(cluster.getLeader()); // Add groups final RaftGroup newGroup = RaftGroup.valueOf(RaftGroupId.randomId(), cluster.getPeers()); @@ -201,7 +201,7 @@ public abstract class GroupManagementBaseTest extends BaseTest { client.getGroupManagementApi(p.getId()).add(newGroup); } } - Assert.assertNotNull(RaftTestUtil.waitForLeader(cluster)); + Assertions.assertNotNull(RaftTestUtil.waitForLeader(cluster)); TimeUnit.SECONDS.sleep(1); // restart the servers with null group @@ -211,7 +211,7 @@ public abstract class GroupManagementBaseTest extends BaseTest { } // the servers should retrieve the conf from the log. - Assert.assertNotNull(RaftTestUtil.waitForLeader(cluster)); + Assertions.assertNotNull(RaftTestUtil.waitForLeader(cluster)); cluster.shutdown(); } @@ -267,7 +267,7 @@ public abstract class GroupManagementBaseTest extends BaseTest { // Make sure that there are no leaders. TimeUnit.SECONDS.sleep(1); LOG.info("start: " + cluster.printServers()); - Assert.assertNull(cluster.getLeader()); + Assertions.assertNull(cluster.getLeader()); try { // Reinitialize servers to three groups @@ -286,7 +286,7 @@ public abstract class GroupManagementBaseTest extends BaseTest { client.getGroupManagementApi(p.getId()).add(groups[i]); } } - Assert.assertNotNull(RaftTestUtil.waitForLeader(cluster, gid)); + Assertions.assertNotNull(RaftTestUtil.waitForLeader(cluster, gid)); checker.accept(cluster, groups[i]); } printThreadCount(type, "start groups"); @@ -302,15 +302,15 @@ public abstract class GroupManagementBaseTest extends BaseTest { for (RaftPeer p : g.getPeers()) { final RaftServer.Division d = cluster.getDivision(p.getId(), g.getGroupId()); final File root = d.getRaftStorage().getStorageDir().getRoot(); - Assert.assertTrue(root.exists()); - Assert.assertTrue(root.isDirectory()); + Assertions.assertTrue(root.exists()); + Assertions.assertTrue(root.isDirectory()); final RaftClientReply r; try (final RaftClient client = cluster.createClient(p.getId(), g)) { r = client.getGroupManagementApi(p.getId()).remove(g.getGroupId(), true, false); } - Assert.assertTrue(r.isSuccess()); - Assert.assertFalse(root.exists()); + Assertions.assertTrue(r.isSuccess()); + Assertions.assertFalse(root.exists()); } } } @@ -335,7 +335,7 @@ public abstract class GroupManagementBaseTest extends BaseTest { peers -> client.admin().setConfiguration(peers.toArray(RaftPeer.emptyArray()))); } - Assert.assertNotNull(RaftTestUtil.waitForLeader(cluster)); + Assertions.assertNotNull(RaftTestUtil.waitForLeader(cluster)); checker.accept(cluster, groups[chosen]); LOG.info("update groups: " + cluster.printServers()); printThreadCount(type, "update groups"); @@ -358,15 +358,15 @@ public abstract class GroupManagementBaseTest extends BaseTest { final RaftPeerId peerId = peer.getId(); final RaftGroup group = RaftGroup.valueOf(cluster.getGroupId(), peer); try (final RaftClient client = cluster.createClient()) { - Assert.assertEquals(group, cluster.getDivision(peerId).getGroup()); + Assertions.assertEquals(group, cluster.getDivision(peerId).getGroup()); try { client.getGroupManagementApi(peer.getId()).add(group); } catch (IOException ex) { // HadoopRPC throws RemoteException, which makes it hard to check if // the exception is instance of AlreadyExistsException - Assert.assertTrue(ex.toString().contains(AlreadyExistsException.class.getCanonicalName())); + Assertions.assertTrue(ex.toString().contains(AlreadyExistsException.class.getCanonicalName())); } - Assert.assertEquals(group, cluster.getDivision(peerId).getGroup()); + Assertions.assertEquals(group, cluster.getDivision(peerId).getGroup()); cluster.shutdown(); } } @@ -383,32 +383,32 @@ public abstract class GroupManagementBaseTest extends BaseTest { final RaftGroup group1 = RaftGroup.valueOf(cluster1.getGroupId(), peer1); final RaftGroup group2 = RaftGroup.valueOf(cluster2.getGroupId(), peer1); try (final RaftClient client = cluster1.createClient()) { - Assert.assertEquals(group1, cluster1.getDivision(peerId1).getGroup()); + Assertions.assertEquals(group1, cluster1.getDivision(peerId1).getGroup()); try { // Group2 is added to one of the peers in Group1 final GroupManagementApi api1 = client.getGroupManagementApi(peerId1); api1.add(group2); List<RaftGroupId> groupIds1 = cluster1.getServer(peerId1).getGroupIds(); - Assert.assertEquals(groupIds1.size(), 2); + Assertions.assertEquals(groupIds1.size(), 2); // Group2 is renamed from the peer1 of Group1 api1.remove(group2.getGroupId(), false, true); groupIds1 = cluster1.getServer(peerId1).getGroupIds(); - Assert.assertEquals(groupIds1.size(), 1); + Assertions.assertEquals(groupIds1.size(), 1); cluster1.restart(false); List<RaftGroupId> groupIdsAfterRestart = cluster1.getServer(peerId1).getGroupIds(); - Assert.assertEquals(groupIds1.size(), groupIdsAfterRestart.size()); + Assertions.assertEquals(groupIds1.size(), groupIdsAfterRestart.size()); File renamedGroup = new File(RaftServerConfigKeys.removedGroupsDir( cluster1.getProperties()), group2.getGroupId().getUuid().toString()); - Assert.assertTrue(renamedGroup.isDirectory()); + Assertions.assertTrue(renamedGroup.isDirectory()); } catch (IOException ex) { - Assert.fail(); + Assertions.fail(); } finally { cluster1.shutdown(); // Clean up @@ -430,7 +430,7 @@ public abstract class GroupManagementBaseTest extends BaseTest { final RaftGroup group1 = RaftGroup.valueOf(cluster1.getGroupId(), peer1); final RaftGroup group2 = RaftGroup.valueOf(cluster2.getGroupId(), peer1); try (final RaftClient client = cluster1.createClient()) { - Assert.assertEquals(group1, + Assertions.assertEquals(group1, cluster1.getDivision(peerId1).getGroup()); try { @@ -438,20 +438,20 @@ public abstract class GroupManagementBaseTest extends BaseTest { final GroupManagementApi api1 = client.getGroupManagementApi(peerId1); api1.add(group2); List<RaftGroupId> groupIds1 = cluster1.getServer(peerId1).getGroupIds(); - Assert.assertEquals(groupIds1.size(), 2); + Assertions.assertEquals(groupIds1.size(), 2); // Group2 is deleted from the peer1 of Group1 api1.remove(group2.getGroupId(), true, false); groupIds1 = cluster1.getServer(peerId1).getGroupIds(); - Assert.assertEquals(groupIds1.size(), 1); + Assertions.assertEquals(groupIds1.size(), 1); cluster1.restart(false); List<RaftGroupId> groupIdsAfterRestart = cluster1.getServer(peerId1).getGroupIds(); - Assert.assertEquals(groupIds1.size(), groupIdsAfterRestart.size()); + Assertions.assertEquals(groupIds1.size(), groupIdsAfterRestart.size()); } catch (IOException ex) { - Assert.fail(); + Assertions.fail(); } finally { cluster1.shutdown(); FileUtils.deleteFully(RaftServerConfigKeys.removedGroupsDir( diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LogMetadataTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LogMetadataTests.java index a834e2714..537ece931 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LogMetadataTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LogMetadataTests.java @@ -29,8 +29,8 @@ import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.raftlog.RaftLog; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public abstract class LogMetadataTests<CLUSTER extends MiniRaftCluster> extends BaseTest @@ -72,7 +72,7 @@ public abstract class LogMetadataTests<CLUSTER extends MiniRaftCluster> long commitIndex = cluster.getServer(leaderId).getDivision(cluster.getGroupId()).getRaftLog() .getLastCommittedIndex(); - Assert.assertTrue(checker.test(commitIndex)); + Assertions.assertTrue(checker.test(commitIndex)); } finally { cluster.shutdown(); } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/PreAppendLeaderStepDownTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/PreAppendLeaderStepDownTest.java index 4a62844c0..0211bfed0 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/PreAppendLeaderStepDownTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/PreAppendLeaderStepDownTest.java @@ -34,8 +34,8 @@ import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.io.IOException; @@ -105,9 +105,9 @@ public abstract class PreAppendLeaderStepDownTest<CLUSTER extends MiniRaftCluste RaftTestUtil.waitForLeader(cluster).getRaftLog().getLastEntryTermIndex().getTerm(); if (leaderShouldStepDown) { - Assert.assertTrue(newTerm > oldTerm); + Assertions.assertTrue(newTerm > oldTerm); } else { - Assert.assertEquals(newTerm, oldTerm); + Assertions.assertEquals(newTerm, oldTerm); } cluster.shutdown(); @@ -124,11 +124,11 @@ public abstract class PreAppendLeaderStepDownTest<CLUSTER extends MiniRaftCluste RaftPeerId leaderId = leader.getId(); RaftServerImpl l = (RaftServerImpl) leader; try (RaftClient client = cluster.createClient(leader.getId())) { - JavaUtils.attempt(() -> Assert.assertEquals(leaderId, leader.getId()), + JavaUtils.attempt(() -> Assertions.assertEquals(leaderId, leader.getId()), 20, ONE_SECOND, "check leader id", LOG); RaftClientReply reply = client.admin().transferLeadership(null, 3000); - Assert.assertTrue(reply.isSuccess()); - Assert.assertEquals(2, ((RaftServerImpl) leader).getRole().getCurrentRole().getNumber()); + Assertions.assertTrue(reply.isSuccess()); + Assertions.assertEquals(2, ((RaftServerImpl) leader).getRole().getCurrentRole().getNumber()); } } } 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 843c271b1..68ad4ebd2 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 @@ -42,12 +42,14 @@ import org.apache.ratis.server.raftlog.LogProtoUtils; 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.test.tag.Flaky; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; import java.io.IOException; @@ -64,7 +66,7 @@ import java.util.concurrent.atomic.AtomicReference; import static java.util.Arrays.asList; import static org.apache.ratis.server.impl.RaftServerTestUtil.waitAndCheckNewConf; -import static org.junit.Assert.assertThrows; +import static org.junit.jupiter.api.Assertions.assertThrows; public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluster> extends BaseTest @@ -94,7 +96,7 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste for (int j = 0; j < peersWithPriority.size(); j ++) { int priorityInConf = conf.getPeer(peersWithPriority.get(j).getId()).getPriority(); - Assert.assertEquals(priorityInConf, peersWithPriority.get(j).getPriority()); + Assertions.assertEquals(priorityInConf, peersWithPriority.get(j).getPriority()); } } } @@ -169,15 +171,15 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste .setConf(new PeerConfiguration(Arrays.asList(curPeer, newPeer))) .setLogEntryIndex(Long.MAX_VALUE / 2) .build(); - Assert.assertTrue(oldNewConf.isSingleMode(curPeer.getId())); + Assertions.assertTrue(oldNewConf.isSingleMode(curPeer.getId())); RaftServerTestUtil.setRaftConf(leaderServer, groupId, oldNewConf); try(RaftClient client = cluster.createClient()) { client.admin().transferLeadership(null, leaderServer.getId(), 1000); } final RaftServer.Division newLeader = RaftTestUtil.waitForLeader(cluster); - Assert.assertEquals(leaderServer.getId(), newLeader.getId()); - Assert.assertEquals(oldNewConf, newLeader.getRaftConf()); + Assertions.assertEquals(leaderServer.getId(), newLeader.getId()); + Assertions.assertEquals(oldNewConf, newLeader.getRaftConf()); }); } @@ -195,8 +197,8 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste .setServersInNewConf(c1.allPeersInNewConf) .setMode(SetConfigurationRequest.Mode.COMPARE_AND_SET) .build(); - assertThrows("Expect change majority error.", SetConfigurationException.class, - () -> client.admin().setConfiguration(arguments)); + assertThrows(SetConfigurationException.class, () -> client.admin().setConfiguration(arguments), + "Expect change majority error."); } }); } @@ -262,13 +264,13 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste try (final RaftClient client = cluster.createClient(leader.getId())) { for (int i = 0; i < 10; i++) { RaftClientReply reply = client.io().send(new SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } RaftClientReply reply = client.admin().setConfiguration( SetConfigurationRequest.Arguments.newBuilder() .setServersInNewConf(peers) .setMode(SetConfigurationRequest.Mode.ADD).build()); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); } cluster.close(); @@ -289,7 +291,7 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste try (final RaftClient client = cluster.createClient(leader.getId())) { for (int i = 0; i < 10; i++) { RaftClientReply reply = client.io().send(new SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } testFailureCase("Can't set configuration in CAS mode ", @@ -306,7 +308,7 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste .setServersInCurrentConf(oldPeers) .setMode(SetConfigurationRequest.Mode.COMPARE_AND_SET) .build()); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); } cluster.close(); @@ -314,7 +316,8 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testReconfTwice() throws Exception { runWithNewCluster(3, this::runTestReconfTwice); } @@ -326,7 +329,7 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste // submit some msgs before reconf for (int i = 0; i < STAGING_CATCHUP_GAP * 2; i++) { RaftClientReply reply = client.io().send(new SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } final AtomicBoolean reconf1 = new AtomicBoolean(false); @@ -360,8 +363,8 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste clientThread.start(); latch.await(); - Assert.assertTrue(reconf1.get()); - Assert.assertTrue(reconf2.get()); + Assertions.assertTrue(reconf1.get()); + Assertions.assertTrue(reconf2.get()); waitAndCheckNewConf(cluster, finalPeers.get(), 2, null); final RaftPeerId leader2 = RaftTestUtil.waitForLeader(cluster).getId(); @@ -373,7 +376,7 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste // each leader change generates one configuration. // expectedConf = 1 (init) + 2*2 (two conf changes) + #leader final int expectedConf = leader2.equals(leaderId) ? 6 : 7; - Assert.assertEquals(server.getId() + ": " + confManager, expectedConf, confManager.numOfConf()); + Assertions.assertEquals(expectedConf, confManager.numOfConf(), server.getId() + ": " + confManager); }); } } @@ -392,17 +395,17 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste LOG.info("Start changing the configuration: {}", asList(c1.allPeersInNewConf)); - Assert.assertFalse(((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).isTransitional()); + Assertions.assertFalse(((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).isTransitional()); final RaftClientRpc sender = client.getClientRpc(); final SetConfigurationRequest request = cluster.newSetConfigurationRequest( client.getId(), leaderId, c1.allPeersInNewConf); try { RaftClientReply reply = sender.sendRequest(request); - Assert.fail("did not get expected exception " + reply.toString()); + Assertions.fail("did not get expected exception " + reply.toString()); } catch (IOException e) { - Assert.assertTrue("Got exception " + e, - e instanceof ReconfigurationTimeoutException); + Assertions.assertTrue( + e instanceof ReconfigurationTimeoutException, "Got exception " + e); } // the two new peers have not started yet, the bootstrapping must timeout @@ -412,10 +415,10 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste // state so that we still get timeout instead of in-progress exception try { sender.sendRequest(request); - Assert.fail("did not get expected exception"); + Assertions.fail("did not get expected exception"); } catch (IOException e) { - Assert.assertTrue("Got exception " + e, - e instanceof ReconfigurationTimeoutException); + Assertions.assertTrue( + e instanceof ReconfigurationTimeoutException, "Got exception " + e); } // start the two new peers @@ -423,7 +426,7 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste for (RaftPeer np : c1.newPeers) { cluster.restartServer(np.getId(), false); } - Assert.assertTrue(client.admin().setConfiguration(c1.allPeersInNewConf).isSuccess()); + Assertions.assertTrue(client.admin().setConfiguration(c1.allPeersInNewConf).isSuccess()); } } @@ -443,8 +446,8 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste try (final RaftClient client = cluster.createClient(leaderId)) { final PeerChanges c1 = cluster.addNewPeers(2, true); - assertThrows("Expect change majority error.", SetConfigurationException.class, - () -> client.admin().setConfiguration(c1.allPeersInNewConf)); + assertThrows(SetConfigurationException.class, () -> client.admin().setConfiguration(c1.allPeersInNewConf), + "Expect change majority error."); } }); } @@ -465,7 +468,7 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste // submit some msgs before reconf for (int i = 0; i < STAGING_CATCHUP_GAP * 2; i++) { RaftClientReply reply = client.io().send(new SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } final PeerChanges c1 = cluster.addNewPeers(numNewPeer, startNewPeer); @@ -501,7 +504,7 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste 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), + Assertions.assertArrayEquals(leaderLog.getEntries(0, Long.MAX_VALUE), d.getRaftLog().getEntries(0, Long.MAX_VALUE)); } } @@ -512,6 +515,7 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste * retrying. */ @Test + @Flaky("RATIS-2251") public void testKillLeaderDuringReconf() throws Exception { // originally 3 peers runWithNewCluster(3, this::runTestKillLeaderDuringReconf); @@ -549,7 +553,7 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste // the leader cannot generate the (old, new) conf, and it will keep // bootstrapping the 1 new peer since it has not started yet. - Assert.assertFalse(((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).isTransitional()); + Assertions.assertFalse(((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).isTransitional()); // (0) the first conf entry, (1) the 1st setConf entry, (2) a metadata entry // (3) new current conf entry (4) a metadata entry @@ -559,11 +563,11 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste LOG.info("{}", LogProtoUtils.toLogEntryString(e)); } final long commitIndex = leaderLog.getLastCommittedIndex(); - Assert.assertTrue("commitIndex = " + commitIndex + " > 2", commitIndex <= 2); + Assertions.assertTrue(commitIndex <= 2, "commitIndex = " + commitIndex + " > 2"); } final RaftPeerId killed = RaftTestUtil.waitAndKillLeader(cluster); - Assert.assertEquals(leaderId, killed); + Assertions.assertEquals(leaderId, killed); final RaftPeerId newLeaderId = RaftTestUtil.waitForLeader(cluster).getId(); LOG.info("newLeaderId: {}", newLeaderId); TimeDuration.valueOf(1500, TimeUnit.MILLISECONDS).sleep(); @@ -615,13 +619,13 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste // no real configuration change in the request final RaftClientReply reply = client.admin().setConfiguration(cluster.getPeers().toArray(RaftPeer.emptyArray())); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); final long newCommittedIndex = leaderLog.getLastCommittedIndex(); for(long i = committedIndex + 1; i <= newCommittedIndex; i++) { final LogEntryProto e = leaderLog.get(i); - Assert.assertTrue(e.hasMetadataEntry()); + Assertions.assertTrue(e.hasMetadataEntry()); } - Assert.assertSame(confBefore, cluster.getLeader().getRaftConf()); + Assertions.assertSame(confBefore, cluster.getLeader().getRaftConf()); } } @@ -691,8 +695,8 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste for (int i = 0; i < 10 && !confChanged.get(); i++) { Thread.sleep(1000); } - Assert.assertTrue(confChanged.get()); - Assert.assertTrue(caughtException.get()); + Assertions.assertTrue(confChanged.get()); + Assertions.assertTrue(caughtException.get()); } finally { LOG_SYNC_DELAY.clear(); } @@ -758,11 +762,11 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste // wait till the old leader persist the new conf JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(log.getFlushIndex() >= confIndex); + Assertions.assertTrue(log.getFlushIndex() >= confIndex); return null; }, 10, sleepTime, "FLUSH", LOG); final long committed = log.getLastCommittedIndex(); - Assert.assertTrue(committed < confIndex); + Assertions.assertTrue(committed < confIndex); // unblock the old leader BlockRequestHandlingInjection.getInstance().unblockReplier(leaderId.toString()); @@ -770,14 +774,14 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste // the client should get NotLeaderException clientThread.join(5000); - Assert.assertTrue(gotNotLeader.get()); + Assertions.assertTrue(gotNotLeader.get()); // the old leader should have truncated the setConf from the log JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(log.getLastCommittedIndex() >= confIndex); + Assertions.assertTrue(log.getLastCommittedIndex() >= confIndex); return null; }, 10, ONE_SECOND, "COMMIT", LOG); - Assert.assertTrue(log.get(confIndex).hasConfigurationEntry()); + Assertions.assertTrue(log.get(confIndex).hasConfigurationEntry()); log2 = null; } finally { RaftStorageTestUtils.printLog(log2, s -> LOG.info(s)); @@ -833,8 +837,8 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste for (int i = 0; !success.get() && i < 5; i++) { Thread.sleep(1000); } - Assert.assertTrue(success.get()); - Assert.assertTrue(caughtNotReady.get()); + Assertions.assertTrue(success.get()); + Assertions.assertTrue(caughtNotReady.get()); } finally { LEADER_PLACE_HOLDER_DELAY.clear(); cluster.shutdown(); diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java index 0054e1601..db22c4854 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java @@ -40,7 +40,7 @@ import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedConsumer; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -99,7 +99,7 @@ public final class RaftServerTestUtil { private static void waitAndCheckNewConf(MiniRaftCluster cluster, Collection<RaftPeer> peers, Collection<RaftPeerId> deadPeers) { LOG.info("waitAndCheckNewConf: peers={}, deadPeers={}, {}", peers, deadPeers, cluster.printServers()); - Assert.assertNotNull(cluster.getLeader()); + Assertions.assertNotNull(cluster.getLeader()); int numIncluded = 0; int deadIncluded = 0; @@ -117,16 +117,16 @@ public final class RaftServerTestUtil { final RaftConfigurationImpl conf = server.getState().getRaftConf(); if (current.containsInConf(server.getId())) { numIncluded++; - Assert.assertTrue(conf.isStable()); - Assert.assertTrue(conf.hasNoChange(peers, Collections.emptyList())); + Assertions.assertTrue(conf.isStable()); + Assertions.assertTrue(conf.hasNoChange(peers, Collections.emptyList())); } else if (server.getInfo().isAlive()) { // The server is successfully removed from the conf // It may not be shutdown since it may not be able to talk to the new leader (who is not in its conf). - Assert.assertTrue(conf.isStable()); - Assert.assertFalse(conf.containsInConf(server.getId())); + Assertions.assertTrue(conf.isStable()); + Assertions.assertFalse(conf.containsInConf(server.getId())); } } - Assert.assertEquals(peers.size(), numIncluded + deadIncluded); + Assertions.assertEquals(peers.size(), numIncluded + deadIncluded); } public static long getNextIndex(RaftServer.Division server) { @@ -176,8 +176,8 @@ public final class RaftServerTestUtil { public static void assertLeaderLease(RaftServer.Division leader, boolean hasLease) { final LeaderStateImpl l = getLeaderState(leader).orElse(null); - Assert.assertNotNull(l); - Assert.assertEquals(l.hasLease(), hasLease); + Assertions.assertNotNull(l); + Assertions.assertEquals(l.hasLease(), hasLease); } public static void restartLogAppenders(RaftServer.Division server) { @@ -200,8 +200,8 @@ public final class RaftServerTestUtil { public static void assertLostMajorityHeartbeatsRecently(RaftServer.Division leader) { final FollowerState f = ((RaftServerImpl)leader).getRole().getFollowerState().orElse(null); - Assert.assertNotNull(f); - Assert.assertTrue(f.lostMajorityHeartbeatsRecently()); + Assertions.assertNotNull(f); + Assertions.assertTrue(f.lostMajorityHeartbeatsRecently()); } public static SegmentedRaftLog newSegmentedRaftLog(RaftGroupMemberId memberId, DivisionInfo info, diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java index be857141c..3a58f4e7c 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java @@ -33,15 +33,15 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.io.IOException; import java.util.Objects; import java.util.concurrent.CompletableFuture; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.fail; public abstract class RaftStateMachineExceptionTests<CLUSTER extends MiniRaftCluster> extends BaseTest implements MiniRaftCluster.Factory.Get<CLUSTER> { @@ -92,7 +92,7 @@ public abstract class RaftStateMachineExceptionTests<CLUSTER extends MiniRaftClu fail("Exception expected"); } catch (StateMachineException e) { e.printStackTrace(); - Assert.assertTrue(e.getCause().getMessage().contains("Fake Exception")); + Assertions.assertTrue(e.getCause().getMessage().contains("Fake Exception")); } cluster.shutdown(); } @@ -114,23 +114,23 @@ public abstract class RaftStateMachineExceptionTests<CLUSTER extends MiniRaftClu final SimpleMessage message = new SimpleMessage("message"); final RaftClientRequest r = cluster.newRaftClientRequest(client.getId(), leaderId, callId, message); RaftClientReply reply = rpc.sendRequest(r); - Assert.assertFalse(reply.isSuccess()); - Assert.assertNotNull(reply.getStateMachineException()); + Assertions.assertFalse(reply.isSuccess()); + Assertions.assertNotNull(reply.getStateMachineException()); // retry with the same callId for (int i = 0; i < 5; i++) { reply = rpc.sendRequest(r); - Assert.assertEquals(client.getId(), reply.getClientId()); - Assert.assertEquals(callId, reply.getCallId()); - Assert.assertFalse(reply.isSuccess()); - Assert.assertNotNull(reply.getStateMachineException()); + Assertions.assertEquals(client.getId(), reply.getClientId()); + Assertions.assertEquals(callId, reply.getCallId()); + Assertions.assertFalse(reply.isSuccess()); + Assertions.assertNotNull(reply.getStateMachineException()); } for (RaftServer.Division server : cluster.iterateDivisions()) { LOG.info("check server " + server.getId()); JavaUtils.attemptRepeatedly(() -> { - Assert.assertNotNull(RetryCacheTestUtil.get(server, client.getId(), callId)); + Assertions.assertNotNull(RetryCacheTestUtil.get(server, client.getId(), callId)); return null; }, 5, BaseTest.ONE_SECOND, "GetRetryEntry", LOG); @@ -161,8 +161,8 @@ public abstract class RaftStateMachineExceptionTests<CLUSTER extends MiniRaftClu Objects.requireNonNull(reply.getStateMachineException()); final RetryCache.Entry oldEntry = RetryCacheTestUtil.get(oldLeader, client.getId(), callId); - Assert.assertNotNull(oldEntry); - Assert.assertTrue(RetryCacheTestUtil.isFailed(oldEntry)); + Assertions.assertNotNull(oldEntry); + Assertions.assertTrue(RetryCacheTestUtil.isFailed(oldEntry)); Thread.sleep(100); // At this point of time the old leader would have stepped down. wait for leader election to complete @@ -173,9 +173,9 @@ public abstract class RaftStateMachineExceptionTests<CLUSTER extends MiniRaftClu Objects.requireNonNull(reply.getStateMachineException()); final RetryCache.Entry currentEntry = RetryCacheTestUtil.get(leader, client.getId(), callId); - Assert.assertNotNull(currentEntry); - Assert.assertTrue(RetryCacheTestUtil.isFailed(currentEntry)); - Assert.assertNotEquals(oldEntry, currentEntry); + Assertions.assertNotNull(currentEntry); + Assertions.assertTrue(RetryCacheTestUtil.isFailed(currentEntry)); + Assertions.assertNotEquals(oldEntry, currentEntry); failPreAppend = false; } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java index 439245b49..f90f91417 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java @@ -27,7 +27,7 @@ import org.apache.ratis.server.RetryCache; import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLog; import org.apache.ratis.server.storage.RaftStorage; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import java.util.concurrent.TimeUnit; @@ -59,7 +59,7 @@ public final class RetryCacheTestUtil { public static void assertFailure(RetryCache cache, LogEntryProto logEntry, boolean isFailed) { if(logEntry.hasStateMachineLogEntry()) { final ClientInvocationId invocationId = ClientInvocationId.valueOf(logEntry.getStateMachineLogEntry()); - Assert.assertEquals(isFailed, get(cache, invocationId).isFailed()); + Assertions.assertEquals(isFailed, get(cache, invocationId).isFailed()); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/ServerPauseResumeTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/ServerPauseResumeTest.java index d2584c631..185b0e3cd 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/ServerPauseResumeTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/ServerPauseResumeTest.java @@ -29,8 +29,8 @@ import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.raftlog.RaftLog; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; /** Test server pause and resume. */ public abstract class ServerPauseResumeTest <CLUSTER extends MiniRaftCluster> @@ -50,7 +50,7 @@ public abstract class ServerPauseResumeTest <CLUSTER extends MiniRaftCluster> final RaftServer.Division leader = waitForLeader(cluster); RaftPeerId leaderId = leader.getId(); final List<RaftServer.Division> followers = cluster.getFollowers(); - Assert.assertTrue(followers.size() >= 1); + Assertions.assertTrue(followers.size() >= 1); final RaftServerImpl follower = (RaftServerImpl)followers.get(0); SimpleMessage[] batch1 = SimpleMessage.create(100, "batch1"); @@ -60,15 +60,15 @@ public abstract class ServerPauseResumeTest <CLUSTER extends MiniRaftCluster> Thread.sleep(cluster.getTimeoutMax().toLong(TimeUnit.MILLISECONDS) * 5); final RaftLog leaderLog = leader.getRaftLog(); // leader should contain all logs. - Assert.assertTrue(RaftTestUtil.logEntriesContains(leaderLog, batch1)); + Assertions.assertTrue(RaftTestUtil.logEntriesContains(leaderLog, batch1)); RaftLog followerLog = follower.getRaftLog(); // follower should contain all logs. - Assert.assertTrue(RaftTestUtil.logEntriesContains(followerLog, batch1)); + Assertions.assertTrue(RaftTestUtil.logEntriesContains(followerLog, batch1)); // pause follower. boolean isSuccess = follower.pause(); - Assert.assertTrue(isSuccess); - Assert.assertTrue(follower.getInfo().getLifeCycleState().isPausingOrPaused()); + Assertions.assertTrue(isSuccess); + Assertions.assertTrue(follower.getInfo().getLifeCycleState().isPausingOrPaused()); SimpleMessage[] batch2 = SimpleMessage.create(100, "batch2"); Thread writeThread2 = RaftTestUtil.sendMessageInNewThread(cluster, leaderId, batch2); @@ -76,15 +76,15 @@ public abstract class ServerPauseResumeTest <CLUSTER extends MiniRaftCluster> writeThread2.join(); Thread.sleep(cluster.getTimeoutMax().toLong(TimeUnit.MILLISECONDS) * 5); // paused follower should not have any batch2 message in its raftlog. - Assert.assertTrue(RaftTestUtil.logEntriesNotContains(followerLog, batch2)); + Assertions.assertTrue(RaftTestUtil.logEntriesNotContains(followerLog, batch2)); // resume follower. isSuccess = follower.resume(); - Assert.assertTrue(isSuccess); - Assert.assertFalse(follower.getInfo().getLifeCycleState().isPausingOrPaused()); + Assertions.assertTrue(isSuccess); + Assertions.assertFalse(follower.getInfo().getLifeCycleState().isPausingOrPaused()); Thread.sleep(cluster.getTimeoutMax().toLong(TimeUnit.MILLISECONDS) * 5); // follower should contain all logs. - Assert.assertTrue(RaftTestUtil.logEntriesContains(followerLog, batch2)); + Assertions.assertTrue(RaftTestUtil.logEntriesContains(followerLog, batch2)); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/TestRatisServerMetricsBase.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/TestRatisServerMetricsBase.java index a465b3c4d..2fe2ed0b6 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/TestRatisServerMetricsBase.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/TestRatisServerMetricsBase.java @@ -17,7 +17,7 @@ */ package org.apache.ratis.server.impl; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import java.io.IOException; import java.util.concurrent.CompletableFuture; @@ -32,8 +32,8 @@ import org.apache.ratis.protocol.RaftClientRequest; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.metrics.RaftServerMetricsImpl; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; /** Tests on Ratis server metrics. */ @@ -66,7 +66,7 @@ public abstract class TestRatisServerMetricsBase<CLUSTER extends MiniRaftCluster .setType(RaftClientRequest.staleReadRequestType(Long.MAX_VALUE)) .build(); final CompletableFuture<RaftClientReply> f = leaderImpl.getRaftServer().submitClientRequestAsync(r); - Assert.assertFalse(f.get().isSuccess()); + Assertions.assertFalse(f.get().isSuccess()); assertEquals(1L, ((RaftServerMetricsImpl)leaderImpl.getRaftServerMetrics()) .getNumFailedClientStaleRead().getCount()); } diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java index 507dd63ad..2c4ac2eee 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java @@ -49,8 +49,8 @@ import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.Slf4jUtils; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.slf4j.Logger; diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/SnapshotManagementTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/SnapshotManagementTest.java index dffeb4228..8be669bf1 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/SnapshotManagementTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/SnapshotManagementTest.java @@ -31,9 +31,9 @@ import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; @@ -51,7 +51,7 @@ public abstract class SnapshotManagementTest<CLUSTER extends MiniRaftCluster> static final Logger LOG = LoggerFactory.getLogger(SnapshotManagementTest.class); - @Before + @BeforeEach public void setup() { final RaftProperties p = getProperties(); p.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, @@ -74,18 +74,18 @@ public abstract class SnapshotManagementTest<CLUSTER extends MiniRaftCluster> try (final RaftClient client = cluster.createClient(leaderId)) { for (int i = 0; i < RaftServerConfigKeys.Snapshot.creationGap(getProperties()); i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } snapshotReply = client.getSnapshotManagementApi().create(3000); } - Assert.assertTrue(snapshotReply.isSuccess()); + Assertions.assertTrue(snapshotReply.isSuccess()); final long snapshotIndex = snapshotReply.getLogIndex(); LOG.info("snapshotIndex = {}", snapshotIndex); final File snapshotFile = SimpleStateMachine4Testing.get(leader) .getStateMachineStorage().getSnapshotFile(leader.getInfo().getCurrentTerm(), snapshotIndex); - Assert.assertTrue(snapshotFile.exists()); + Assertions.assertTrue(snapshotFile.exists()); } void runTestTakeSnapshotWithConfigurableGap(CLUSTER cluster) throws Exception { @@ -95,29 +95,29 @@ public abstract class SnapshotManagementTest<CLUSTER extends MiniRaftCluster> try (final RaftClient client = cluster.createClient(leaderId)) { for (int i = 0; i < RaftServerConfigKeys.Snapshot.creationGap(getProperties())/2-1; i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } - Assert.assertTrue(leader.getStateMachine().getLastAppliedTermIndex().getIndex() + Assertions.assertTrue(leader.getStateMachine().getLastAppliedTermIndex().getIndex() < RaftServerConfigKeys.Snapshot.creationGap(getProperties())); snapshotReply = client.getSnapshotManagementApi(leaderId).create(3000); - Assert.assertTrue(snapshotReply.isSuccess()); - Assert.assertEquals(0,snapshotReply.getLogIndex()); + Assertions.assertTrue(snapshotReply.isSuccess()); + Assertions.assertEquals(0,snapshotReply.getLogIndex()); for (int i = 0; i < RaftServerConfigKeys.Snapshot.creationGap(getProperties())/2-1; i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } final SnapshotManagementRequest r1 = SnapshotManagementRequest.newCreate(client.getId(), leaderId, cluster.getGroupId(), CallId.getAndIncrement(), 3000); snapshotReply = client.getSnapshotManagementApi(leaderId).create(3000); } - Assert.assertTrue(snapshotReply.isSuccess()); + Assertions.assertTrue(snapshotReply.isSuccess()); final long snapshotIndex = snapshotReply.getLogIndex(); LOG.info("snapshotIndex = {}", snapshotIndex); final File snapshotFile = SimpleStateMachine4Testing.get(leader) .getStateMachineStorage() .getSnapshotFile(leader.getInfo().getCurrentTerm(), snapshotIndex); - Assert.assertTrue(snapshotFile.exists()); + Assertions.assertTrue(snapshotFile.exists()); } void runTestTakeSnapshotOnSpecificServer(CLUSTER cluster) throws Exception { @@ -125,23 +125,23 @@ public abstract class SnapshotManagementTest<CLUSTER extends MiniRaftCluster> final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); final RaftServer.Division follower = cluster.getFollowers().get(0); final RaftPeerId followerId = follower.getId(); - Assert.assertTrue(follower.getInfo().isFollower()); + Assertions.assertTrue(follower.getInfo().isFollower()); try (final RaftClient client = cluster.createClient(followerId)) { for (int i = 0; i < RaftServerConfigKeys.Snapshot.creationGap(getProperties()); i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } snapshotReply = client.getSnapshotManagementApi(followerId).create(3000); } - Assert.assertTrue(snapshotReply.isSuccess()); + Assertions.assertTrue(snapshotReply.isSuccess()); final long snapshotIndex = snapshotReply.getLogIndex(); LOG.info("snapshotIndex = {} on {} server {}", snapshotIndex, follower.getInfo().getCurrentRole(), follower.getId()); final File snapshotFile = SimpleStateMachine4Testing.get(follower) .getStateMachineStorage().getSnapshotFile(follower.getInfo().getCurrentTerm(), snapshotIndex); - Assert.assertTrue(snapshotFile.exists()); + Assertions.assertTrue(snapshotFile.exists()); } @@ -155,22 +155,22 @@ public abstract class SnapshotManagementTest<CLUSTER extends MiniRaftCluster> final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); final RaftServer.Division listener = cluster.getListeners().get(0); final RaftPeerId listenerId = listener.getId(); - Assert.assertTrue(listener.getInfo().isListener()); + Assertions.assertTrue(listener.getInfo().isListener()); try (final RaftClient client = cluster.createClient(listenerId)) { for (int i = 0; i < RaftServerConfigKeys.Snapshot.creationGap(getProperties()); i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } snapshotReply = client.getSnapshotManagementApi(listenerId).create(3000); } - Assert.assertTrue(snapshotReply.isSuccess()); + Assertions.assertTrue(snapshotReply.isSuccess()); final long snapshotIndex = snapshotReply.getLogIndex(); LOG.info("snapshotIndex = {} on {} server {}", snapshotIndex, listener.getInfo().getCurrentRole(), listener.getId()); final File snapshotFile = SimpleStateMachine4Testing.get(listener) .getStateMachineStorage().getSnapshotFile(listener.getInfo().getCurrentTerm(), snapshotIndex); - Assert.assertTrue(snapshotFile.exists()); + Assertions.assertTrue(snapshotFile.exists()); } } diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index 69470bf0d..a47e9bd35 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -151,6 +151,16 @@ <artifactId>junit-jupiter-params</artifactId> <scope>test</scope> </dependency> + <dependency> + <groupId>org.junit.vintage</groupId> + <artifactId>junit-vintage-engine</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.junit.platform</groupId> + <artifactId>junit-platform-launcher</artifactId> + <scope>test</scope> + </dependency> <dependency> <groupId>org.mockito</groupId> <artifactId>mockito-core</artifactId> diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpc.java index a12c52f10..9f9cda701 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpc.java @@ -18,7 +18,9 @@ package org.apache.ratis.grpc; import org.apache.ratis.RaftAsyncTests; +import org.junit.jupiter.api.Timeout; +@Timeout(100) public class TestRaftAsyncWithGrpc extends RaftAsyncTests<MiniRaftClusterWithGrpc> implements MiniRaftClusterWithGrpc.FactoryGet { } \ No newline at end of file diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java index ebaa33d50..c09b07876 100644 --- a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java @@ -18,7 +18,9 @@ package org.apache.ratis.netty; import org.apache.ratis.RaftAsyncTests; +import org.junit.jupiter.api.Timeout; +@Timeout(100) public class TestRaftAsyncWithNetty extends RaftAsyncTests<MiniRaftClusterWithNetty> implements MiniRaftClusterWithNetty.FactoryGet {
