Modernize sal-akka-raft tests
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / test / java / org / opendaylight / controller / cluster / raft / RaftActorServerConfigurationSupportTest.java
index 29aadbb95c8fd0a3157fb2db98e982284b13c258..2f9998c56b1910cf8e3c559c93143dd0f880d852 100644 (file)
@@ -15,24 +15,21 @@ import static org.opendaylight.controller.cluster.raft.utils.MessageCollectorAct
 import static org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor.expectFirstMatching;
 import static org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor.expectMatching;
 
+import akka.actor.AbstractActor;
 import akka.actor.ActorRef;
 import akka.actor.Props;
-import akka.actor.UntypedActor;
 import akka.dispatch.Dispatchers;
 import akka.testkit.TestActorRef;
 import akka.testkit.javadsl.TestKit;
-import com.google.common.base.Optional;
 import com.google.common.base.Stopwatch;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
 import com.google.common.io.ByteSource;
+import com.google.common.util.concurrent.MoreExecutors;
 import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
+import java.time.Duration;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import org.apache.commons.lang3.SerializationUtils;
 import org.junit.After;
@@ -135,12 +132,12 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
     }
 
     @After
-    public void tearDown() throws Exception {
+    public void tearDown() {
         actorFactory.close();
     }
 
     @Test
-    public void testAddServerWithExistingFollower() throws Exception {
+    public void testAddServerWithExistingFollower() {
         LOG.info("testAddServerWithExistingFollower starting");
         setupNewFollower();
         RaftActorContextImpl followerActorContext = newFollowerContext(FOLLOWER_ID, followerActor);
@@ -154,7 +151,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         followerActorContext.setCurrentBehavior(follower);
 
         TestActorRef<MockLeaderRaftActor> leaderActor = actorFactory.createTestActor(
-                MockLeaderRaftActor.props(ImmutableMap.of(FOLLOWER_ID, followerActor.path().toString()),
+                MockLeaderRaftActor.props(Map.of(FOLLOWER_ID, followerActor.path().toString()),
                         followerActorContext).withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(LEADER_ID));
 
@@ -173,7 +170,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         List<Object> snapshotState = MockRaftActor.fromState(applySnapshot.getSnapshot().getState());
         assertEquals("Snapshot state", snapshotState, leaderRaftActor.getState());
 
-        AddServerReply addServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"), AddServerReply.class);
+        AddServerReply addServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), AddServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.OK, addServerReply.getStatus());
         assertEquals("getLeaderHint", LEADER_ID, addServerReply.getLeaderHint().get());
 
@@ -201,10 +198,9 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
 
         // Verify new server config was applied in both followers
 
-        assertEquals("Follower peers", Sets.newHashSet(LEADER_ID, NEW_SERVER_ID), followerActorContext.getPeerIds());
+        assertEquals("Follower peers", Set.of(LEADER_ID, NEW_SERVER_ID), followerActorContext.getPeerIds());
 
-        assertEquals("New follower peers", Sets.newHashSet(LEADER_ID, FOLLOWER_ID),
-                newFollowerActorContext.getPeerIds());
+        assertEquals("New follower peers", Set.of(LEADER_ID, FOLLOWER_ID), newFollowerActorContext.getPeerIds());
 
         assertEquals("Follower commit index", 3, followerActorContext.getCommitIndex());
         assertEquals("Follower last applied index", 3, followerActorContext.getLastApplied());
@@ -225,7 +221,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
     }
 
     @Test
-    public void testAddServerWithNoExistingFollower() throws Exception {
+    public void testAddServerWithNoExistingFollower() {
         LOG.info("testAddServerWithNoExistingFollower starting");
 
         setupNewFollower();
@@ -236,8 +232,8 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
                 0, 2, 1).build());
 
         TestActorRef<MockLeaderRaftActor> leaderActor = actorFactory.createTestActor(
-                MockLeaderRaftActor.props(ImmutableMap.<String, String>of(),
-                        initialActorContext).withDispatcher(Dispatchers.DefaultDispatcherId()),
+                MockLeaderRaftActor.props(Map.of(), initialActorContext)
+                    .withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(LEADER_ID));
 
         MockLeaderRaftActor leaderRaftActor = leaderActor.underlyingActor();
@@ -253,7 +249,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         List<Object> snapshotState = MockRaftActor.fromState(applySnapshot.getSnapshot().getState());
         assertEquals("Snapshot state", snapshotState, leaderRaftActor.getState());
 
-        AddServerReply addServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"), AddServerReply.class);
+        AddServerReply addServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), AddServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.OK, addServerReply.getStatus());
         assertEquals("getLeaderHint", LEADER_ID, addServerReply.getLeaderHint().get());
 
@@ -275,21 +271,21 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
 
         // Verify new server config was applied in the new follower
 
-        assertEquals("New follower peers", Sets.newHashSet(LEADER_ID), newFollowerActorContext.getPeerIds());
+        assertEquals("New follower peers", Set.of(LEADER_ID), newFollowerActorContext.getPeerIds());
 
         LOG.info("testAddServerWithNoExistingFollower ending");
     }
 
     @Test
-    public void testAddServersAsNonVoting() throws Exception {
+    public void testAddServersAsNonVoting() {
         LOG.info("testAddServersAsNonVoting starting");
 
         setupNewFollower();
         RaftActorContext initialActorContext = new MockRaftActorContext();
 
         TestActorRef<MockLeaderRaftActor> leaderActor = actorFactory.createTestActor(
-                MockLeaderRaftActor.props(ImmutableMap.<String, String>of(),
-                        initialActorContext).withDispatcher(Dispatchers.DefaultDispatcherId()),
+                MockLeaderRaftActor.props(Map.of(), initialActorContext)
+                    .withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(LEADER_ID));
 
         MockLeaderRaftActor leaderRaftActor = leaderActor.underlyingActor();
@@ -299,7 +295,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
 
         leaderActor.tell(new AddServer(NEW_SERVER_ID, newFollowerRaftActor.path().toString(), false), testKit.getRef());
 
-        AddServerReply addServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"), AddServerReply.class);
+        AddServerReply addServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), AddServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.OK, addServerReply.getStatus());
         assertEquals("getLeaderHint", LEADER_ID, addServerReply.getLeaderHint().get());
 
@@ -322,7 +318,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
 
         // Verify new server config was applied in the new follower
 
-        assertEquals("New follower peers", Sets.newHashSet(LEADER_ID), newFollowerActorContext.getPeerIds());
+        assertEquals("New follower peers", Set.of(LEADER_ID), newFollowerActorContext.getPeerIds());
 
         assertNoneMatching(newFollowerCollectorActor, InstallSnapshot.class, 500);
 
@@ -336,9 +332,9 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
 
         leaderActor.tell(new AddServer(NEW_SERVER_ID2, followerActor.path().toString(), false), testKit.getRef());
 
-        addServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"), AddServerReply.class);
+        addServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), AddServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.OK, addServerReply.getStatus());
-        assertEquals("getLeaderHint", java.util.Optional.of(LEADER_ID), addServerReply.getLeaderHint());
+        assertEquals("getLeaderHint", Optional.of(LEADER_ID), addServerReply.getLeaderHint());
 
         expectFirstMatching(leaderCollectorActor, ApplyState.class);
         assertEquals("Leader journal last index", 1, leaderActorContext.getReplicatedLog().lastIndex());
@@ -351,15 +347,15 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
     }
 
     @Test
-    public void testAddServerWithOperationInProgress() throws Exception {
+    public void testAddServerWithOperationInProgress() {
         LOG.info("testAddServerWithOperationInProgress starting");
 
         setupNewFollower();
         RaftActorContext initialActorContext = new MockRaftActorContext();
 
         TestActorRef<MockLeaderRaftActor> leaderActor = actorFactory.createTestActor(
-                MockLeaderRaftActor.props(ImmutableMap.<String, String>of(),
-                        initialActorContext).withDispatcher(Dispatchers.DefaultDispatcherId()),
+                MockLeaderRaftActor.props(Map.of(), initialActorContext)
+                    .withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(LEADER_ID));
 
         MockLeaderRaftActor leaderRaftActor = leaderActor.underlyingActor();
@@ -389,10 +385,10 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         newFollowerRaftActor.tell(installSnapshot, leaderActor);
 
         // Verify both complete successfully
-        AddServerReply addServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"), AddServerReply.class);
+        AddServerReply addServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), AddServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.OK, addServerReply.getStatus());
 
-        addServerReply = testKit2.expectMsgClass(testKit2.duration("5 seconds"), AddServerReply.class);
+        addServerReply = testKit2.expectMsgClass(Duration.ofSeconds(5), AddServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.OK, addServerReply.getStatus());
 
         // Verify ServerConfigurationPayload entries in leader's log
@@ -407,22 +403,21 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         // Verify ServerConfigurationPayload entry in the new follower
 
         expectMatching(newFollowerCollectorActor, ApplyState.class, 2);
-        assertEquals("New follower peers", Sets.newHashSet(LEADER_ID, NEW_SERVER_ID2),
-               newFollowerActorContext.getPeerIds());
+        assertEquals("New follower peers", Set.of(LEADER_ID, NEW_SERVER_ID2), newFollowerActorContext.getPeerIds());
 
         LOG.info("testAddServerWithOperationInProgress ending");
     }
 
     @Test
-    public void testAddServerWithPriorSnapshotInProgress() throws Exception {
+    public void testAddServerWithPriorSnapshotInProgress() {
         LOG.info("testAddServerWithPriorSnapshotInProgress starting");
 
         setupNewFollower();
         RaftActorContext initialActorContext = new MockRaftActorContext();
 
         TestActorRef<MockLeaderRaftActor> leaderActor = actorFactory.createTestActor(
-                MockLeaderRaftActor.props(ImmutableMap.<String, String>of(),
-                        initialActorContext).withDispatcher(Dispatchers.DefaultDispatcherId()),
+                MockLeaderRaftActor.props(Map.of(), initialActorContext)
+                    .withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(LEADER_ID));
 
         MockLeaderRaftActor leaderRaftActor = leaderActor.underlyingActor();
@@ -442,7 +437,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         leaderRaftActor.setDropMessageOfType(null);
         leaderActor.tell(commitMsg, leaderActor);
 
-        AddServerReply addServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"), AddServerReply.class);
+        AddServerReply addServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), AddServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.OK, addServerReply.getStatus());
         assertEquals("getLeaderHint", LEADER_ID, addServerReply.getLeaderHint().get());
 
@@ -461,15 +456,15 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
     }
 
     @Test
-    public void testAddServerWithPriorSnapshotCompleteTimeout() throws Exception {
+    public void testAddServerWithPriorSnapshotCompleteTimeout() {
         LOG.info("testAddServerWithPriorSnapshotCompleteTimeout starting");
 
         setupNewFollower();
         RaftActorContext initialActorContext = new MockRaftActorContext();
 
         TestActorRef<MockLeaderRaftActor> leaderActor = actorFactory.createTestActor(
-                MockLeaderRaftActor.props(ImmutableMap.<String, String>of(),
-                        initialActorContext).withDispatcher(Dispatchers.DefaultDispatcherId()),
+                MockLeaderRaftActor.props(Map.of(), initialActorContext)
+                    .withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(LEADER_ID));
 
         MockLeaderRaftActor leaderRaftActor = leaderActor.underlyingActor();
@@ -484,7 +479,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
 
         leaderActor.tell(new AddServer(NEW_SERVER_ID, newFollowerRaftActor.path().toString(), true), testKit.getRef());
 
-        AddServerReply addServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"), AddServerReply.class);
+        AddServerReply addServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), AddServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.TIMEOUT, addServerReply.getStatus());
 
         assertEquals("Leader peers size", 0, leaderActorContext.getPeerIds().size());
@@ -493,15 +488,15 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
     }
 
     @Test
-    public void testAddServerWithLeaderChangeBeforePriorSnapshotComplete() throws Exception {
+    public void testAddServerWithLeaderChangeBeforePriorSnapshotComplete() {
         LOG.info("testAddServerWithLeaderChangeBeforePriorSnapshotComplete starting");
 
         setupNewFollower();
         RaftActorContext initialActorContext = new MockRaftActorContext();
 
         TestActorRef<MockLeaderRaftActor> leaderActor = actorFactory.createTestActor(
-                MockLeaderRaftActor.props(ImmutableMap.<String, String>of(),
-                        initialActorContext).withDispatcher(Dispatchers.DefaultDispatcherId()),
+                MockLeaderRaftActor.props(Map.of(), initialActorContext)
+                    .withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(LEADER_ID));
 
         MockLeaderRaftActor leaderRaftActor = leaderActor.underlyingActor();
@@ -532,7 +527,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
 
         leaderActor.tell(new RaftActorServerConfigurationSupport.ServerOperationTimeout(NEW_SERVER_ID), leaderActor);
 
-        AddServerReply addServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"), AddServerReply.class);
+        AddServerReply addServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), AddServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.NO_LEADER, addServerReply.getStatus());
 
         assertEquals("Leader peers size", 0, leaderActorContext.getPeerIds().size());
@@ -542,15 +537,15 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
     }
 
     @Test
-    public void testAddServerWithLeaderChangeDuringInstallSnapshot() throws Exception {
+    public void testAddServerWithLeaderChangeDuringInstallSnapshot() {
         LOG.info("testAddServerWithLeaderChangeDuringInstallSnapshot starting");
 
         setupNewFollower();
         RaftActorContext initialActorContext = new MockRaftActorContext();
 
         TestActorRef<MockLeaderRaftActor> leaderActor = actorFactory.createTestActor(
-                MockLeaderRaftActor.props(ImmutableMap.<String, String>of(),
-                        initialActorContext).withDispatcher(Dispatchers.DefaultDispatcherId()),
+                MockLeaderRaftActor.props(Map.of(), initialActorContext)
+                    .withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(LEADER_ID));
 
         MockLeaderRaftActor leaderRaftActor = leaderActor.underlyingActor();
@@ -578,7 +573,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         leaderRaftActor.setDropMessageOfType(null);
         leaderActor.tell(snapshotReply, leaderActor);
 
-        AddServerReply addServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"), AddServerReply.class);
+        AddServerReply addServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), AddServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.NO_LEADER, addServerReply.getStatus());
 
         assertEquals("Leader peers size", 0, leaderActorContext.getPeerIds().size());
@@ -587,15 +582,15 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
     }
 
     @Test
-    public void testAddServerWithInstallSnapshotTimeout() throws Exception {
+    public void testAddServerWithInstallSnapshotTimeout() {
         LOG.info("testAddServerWithInstallSnapshotTimeout starting");
 
         setupNewFollower();
         RaftActorContext initialActorContext = new MockRaftActorContext();
 
         TestActorRef<MockLeaderRaftActor> leaderActor = actorFactory.createTestActor(
-                MockLeaderRaftActor.props(ImmutableMap.<String, String>of(),
-                        initialActorContext).withDispatcher(Dispatchers.DefaultDispatcherId()),
+                MockLeaderRaftActor.props(Map.of(), initialActorContext)
+                    .withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(LEADER_ID));
 
         MockLeaderRaftActor leaderRaftActor = leaderActor.underlyingActor();
@@ -609,7 +604,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
 
         leaderActor.tell(new UnInitializedFollowerSnapshotReply("bogus"), leaderActor);
 
-        AddServerReply addServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"), AddServerReply.class);
+        AddServerReply addServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), AddServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.TIMEOUT, addServerReply.getStatus());
 
         assertEquals("Leader peers size", 0, leaderActorContext.getPeerIds().size());
@@ -628,7 +623,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         configParams.setHeartBeatInterval(new FiniteDuration(1, TimeUnit.DAYS));
 
         TestActorRef<MockRaftActor> noLeaderActor = actorFactory.createTestActor(
-                MockRaftActor.builder().id(LEADER_ID).peerAddresses(ImmutableMap.of(FOLLOWER_ID,
+                MockRaftActor.builder().id(LEADER_ID).peerAddresses(Map.of(FOLLOWER_ID,
                         followerActor.path().toString())).config(configParams).persistent(Optional.of(false))
                         .props().withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(LEADER_ID));
@@ -636,7 +631,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
 
         noLeaderActor.tell(new AddServer(NEW_SERVER_ID, newFollowerRaftActor.path().toString(), true),
                 testKit.getRef());
-        AddServerReply addServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"), AddServerReply.class);
+        AddServerReply addServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), AddServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.NO_LEADER, addServerReply.getStatus());
 
         LOG.info("testAddServerWithNoLeader ending");
@@ -650,8 +645,8 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         RaftActorContext initialActorContext = new MockRaftActorContext();
 
         TestActorRef<MockLeaderRaftActor> leaderActor = actorFactory.createTestActor(
-                MockLeaderRaftActor.props(ImmutableMap.<String, String>of(),
-                        initialActorContext).withDispatcher(Dispatchers.DefaultDispatcherId()),
+                MockLeaderRaftActor.props(Map.of(), initialActorContext)
+                    .withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(LEADER_ID));
 
         MockLeaderRaftActor leaderRaftActor = leaderActor.underlyingActor();
@@ -683,7 +678,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         leaderActor.tell(new AddServer(NEW_SERVER_ID2, "", false), testKit.getRef());
 
         // The first AddServer should succeed with OK even though consensus wasn't reached
-        AddServerReply addServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"), AddServerReply.class);
+        AddServerReply addServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), AddServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.OK, addServerReply.getStatus());
         assertEquals("getLeaderHint", LEADER_ID, addServerReply.getLeaderHint().get());
 
@@ -692,12 +687,12 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
                 votingServer(NEW_SERVER_ID));
 
         // The second AddServer should fail since consensus wasn't reached for the first
-        addServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"), AddServerReply.class);
+        addServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), AddServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.PRIOR_REQUEST_CONSENSUS_TIMEOUT, addServerReply.getStatus());
 
         // Re-send the second AddServer - should also fail
         leaderActor.tell(new AddServer(NEW_SERVER_ID2, "", false), testKit.getRef());
-        addServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"), AddServerReply.class);
+        addServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), AddServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.PRIOR_REQUEST_CONSENSUS_TIMEOUT, addServerReply.getStatus());
 
         LOG.info("testAddServerWithNoConsensusReached ending");
@@ -710,13 +705,13 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         RaftActorContext initialActorContext = new MockRaftActorContext();
 
         TestActorRef<MockLeaderRaftActor> leaderActor = actorFactory.createTestActor(
-                MockLeaderRaftActor.props(ImmutableMap.of(FOLLOWER_ID, followerActor.path().toString()),
+                MockLeaderRaftActor.props(Map.of(FOLLOWER_ID, followerActor.path().toString()),
                         initialActorContext).withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(LEADER_ID));
 
         leaderActor.tell(new AddServer(FOLLOWER_ID, followerActor.path().toString(), true), testKit.getRef());
 
-        AddServerReply addServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"), AddServerReply.class);
+        AddServerReply addServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), AddServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.ALREADY_EXISTS, addServerReply.getStatus());
 
         LOG.info("testAddServerWithExistingServer ending");
@@ -734,14 +729,13 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
                 MessageCollectorActor.props(), actorFactory.generateActorId(LEADER_ID));
 
         TestActorRef<MockRaftActor> followerRaftActor = actorFactory.createTestActor(
-                MockRaftActor.builder().id(FOLLOWER_ID).peerAddresses(ImmutableMap.of(LEADER_ID,
+                MockRaftActor.builder().id(FOLLOWER_ID).peerAddresses(Map.of(LEADER_ID,
                         leaderActor.path().toString())).config(configParams).persistent(Optional.of(false))
                         .props().withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(FOLLOWER_ID));
         followerRaftActor.underlyingActor().waitForInitializeBehaviorComplete();
 
-        followerRaftActor.tell(new AppendEntries(1, LEADER_ID, 0, 1, Collections.<ReplicatedLogEntry>emptyList(),
-                -1, -1, (short)0), leaderActor);
+        followerRaftActor.tell(new AppendEntries(1, LEADER_ID, 0, 1, List.of(), -1, -1, (short)0), leaderActor);
 
         followerRaftActor.tell(new AddServer(NEW_SERVER_ID, newFollowerRaftActor.path().toString(), true),
                 testKit.getRef());
@@ -757,7 +751,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         DefaultConfigParamsImpl configParams = new DefaultConfigParamsImpl();
         configParams.setHeartBeatInterval(new FiniteDuration(1, TimeUnit.DAYS));
         TestActorRef<MockRaftActor> noLeaderActor = actorFactory.createTestActor(
-                MockRaftActor.builder().id(LEADER_ID).peerAddresses(ImmutableMap.of(FOLLOWER_ID,
+                MockRaftActor.builder().id(LEADER_ID).peerAddresses(Map.of(FOLLOWER_ID,
                         followerActor.path().toString())).config(configParams).persistent(Optional.of(false))
                         .props().withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(LEADER_ID));
@@ -766,7 +760,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
                 noLeaderActor.underlyingActor());
 
         ReplicatedLogEntry serverConfigEntry = new SimpleReplicatedLogEntry(1, 1,
-                new ServerConfigurationPayload(Collections.<ServerInfo>emptyList()));
+                new ServerConfigurationPayload(List.of()));
         boolean handled = support.handleMessage(new ApplyState(null, null, serverConfigEntry), ActorRef.noSender());
         assertEquals("Message handled", true, handled);
 
@@ -786,15 +780,14 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         configParams.setHeartBeatInterval(new FiniteDuration(1, TimeUnit.DAYS));
 
         TestActorRef<MockRaftActor> leaderActor = actorFactory.createTestActor(
-                MockRaftActor.builder().id(LEADER_ID).peerAddresses(ImmutableMap.of(FOLLOWER_ID,
+                MockRaftActor.builder().id(LEADER_ID).peerAddresses(Map.of(FOLLOWER_ID,
                         followerActor.path().toString())).config(configParams).persistent(Optional.of(false))
                         .props().withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(LEADER_ID));
         leaderActor.underlyingActor().waitForInitializeBehaviorComplete();
 
         leaderActor.tell(new RemoveServer(FOLLOWER_ID), testKit.getRef());
-        RemoveServerReply removeServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"),
-                RemoveServerReply.class);
+        RemoveServerReply removeServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), RemoveServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.NO_LEADER, removeServerReply.getStatus());
 
         LOG.info("testRemoveServerWithNoLeader ending");
@@ -807,13 +800,12 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         RaftActorContext initialActorContext = new MockRaftActorContext();
 
         TestActorRef<MockLeaderRaftActor> leaderActor = actorFactory.createTestActor(
-                MockLeaderRaftActor.props(ImmutableMap.of(FOLLOWER_ID, followerActor.path().toString()),
+                MockLeaderRaftActor.props(Map.of(FOLLOWER_ID, followerActor.path().toString()),
                         initialActorContext).withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(LEADER_ID));
 
         leaderActor.tell(new RemoveServer(NEW_SERVER_ID), testKit.getRef());
-        RemoveServerReply removeServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"),
-                RemoveServerReply.class);
+        RemoveServerReply removeServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), RemoveServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.DOES_NOT_EXIST, removeServerReply.getStatus());
 
         LOG.info("testRemoveServerNonExistentServer ending");
@@ -830,14 +822,13 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
                 MessageCollectorActor.props(), actorFactory.generateActorId(LEADER_ID));
 
         TestActorRef<MockRaftActor> followerRaftActor = actorFactory.createTestActor(
-                MockRaftActor.builder().id(FOLLOWER_ID).peerAddresses(ImmutableMap.of(LEADER_ID,
+                MockRaftActor.builder().id(FOLLOWER_ID).peerAddresses(Map.of(LEADER_ID,
                         leaderActor.path().toString())).config(configParams).persistent(Optional.of(false))
                         .props().withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(FOLLOWER_ID));
         followerRaftActor.underlyingActor().waitForInitializeBehaviorComplete();
 
-        followerRaftActor.tell(new AppendEntries(1, LEADER_ID, 0, 1, Collections.<ReplicatedLogEntry>emptyList(),
-                -1, -1, (short)0), leaderActor);
+        followerRaftActor.tell(new AppendEntries(1, LEADER_ID, 0, 1, List.of(), -1, -1, (short)0), leaderActor);
 
         followerRaftActor.tell(new RemoveServer(FOLLOWER_ID), testKit.getRef());
         expectFirstMatching(leaderActor, RemoveServer.class);
@@ -846,7 +837,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
     }
 
     @Test
-    public void testRemoveServer() throws Exception {
+    public void testRemoveServer() {
         LOG.info("testRemoveServer starting");
 
         DefaultConfigParamsImpl configParams = new DefaultConfigParamsImpl();
@@ -861,7 +852,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
 
         final String downNodeId = "downNode";
         TestActorRef<MockLeaderRaftActor> leaderActor = actorFactory.createTestActor(MockLeaderRaftActor.props(
-                ImmutableMap.of(FOLLOWER_ID, follower1ActorPath, FOLLOWER_ID2, follower2ActorPath, downNodeId, ""),
+                Map.of(FOLLOWER_ID, follower1ActorPath, FOLLOWER_ID2, follower2ActorPath, downNodeId, ""),
                         initialActorContext).withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(LEADER_ID));
 
@@ -870,14 +861,14 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         ActorRef follower1Collector = actorFactory.createActor(
                 MessageCollectorActor.props(), actorFactory.generateActorId("collector"));
         final TestActorRef<CollectingMockRaftActor> follower1Actor = actorFactory.createTestActor(
-                CollectingMockRaftActor.props(FOLLOWER_ID, ImmutableMap.of(LEADER_ID, leaderActor.path().toString(),
+                CollectingMockRaftActor.props(FOLLOWER_ID, Map.of(LEADER_ID, leaderActor.path().toString(),
                         FOLLOWER_ID2, follower2ActorPath, downNodeId, ""), configParams, NO_PERSISTENCE,
                         follower1Collector).withDispatcher(Dispatchers.DefaultDispatcherId()), follower1ActorId);
 
         ActorRef follower2Collector = actorFactory.createActor(
                 MessageCollectorActor.props(), actorFactory.generateActorId("collector"));
         final TestActorRef<CollectingMockRaftActor> follower2Actor = actorFactory.createTestActor(
-                CollectingMockRaftActor.props(FOLLOWER_ID2, ImmutableMap.of(LEADER_ID, leaderActor.path().toString(),
+                CollectingMockRaftActor.props(FOLLOWER_ID2, Map.of(LEADER_ID, leaderActor.path().toString(),
                         FOLLOWER_ID, follower1ActorPath, downNodeId, ""), configParams, NO_PERSISTENCE,
                         follower2Collector).withDispatcher(Dispatchers.DefaultDispatcherId()), follower2ActorId);
 
@@ -886,8 +877,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         follower2Actor.underlyingActor().waitForInitializeBehaviorComplete();
 
         leaderActor.tell(new RemoveServer(FOLLOWER_ID), testKit.getRef());
-        RemoveServerReply removeServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"),
-                RemoveServerReply.class);
+        RemoveServerReply removeServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), RemoveServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.OK, removeServerReply.getStatus());
 
         ApplyState applyState = MessageCollectorActor.expectFirstMatching(leaderCollector, ApplyState.class);
@@ -922,7 +912,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         RaftActorContext initialActorContext = new MockRaftActorContext();
 
         TestActorRef<MockLeaderRaftActor> leaderActor = actorFactory.createTestActor(
-                MockLeaderRaftActor.props(ImmutableMap.of(FOLLOWER_ID, followerActorPath),
+                MockLeaderRaftActor.props(Map.of(FOLLOWER_ID, followerActorPath),
                         initialActorContext).withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(LEADER_ID));
 
@@ -931,14 +921,13 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         final ActorRef followerCollector =
                 actorFactory.createActor(MessageCollectorActor.props(), actorFactory.generateActorId("collector"));
         actorFactory.createTestActor(
-                CollectingMockRaftActor.props(FOLLOWER_ID, ImmutableMap.of(LEADER_ID, leaderActor.path().toString()),
+                CollectingMockRaftActor.props(FOLLOWER_ID, Map.of(LEADER_ID, leaderActor.path().toString()),
                         configParams, NO_PERSISTENCE, followerCollector)
                         .withDispatcher(Dispatchers.DefaultDispatcherId()),
                 followerActorId);
 
         leaderActor.tell(new RemoveServer(LEADER_ID), testKit.getRef());
-        RemoveServerReply removeServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"),
-                RemoveServerReply.class);
+        RemoveServerReply removeServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), RemoveServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.OK, removeServerReply.getStatus());
 
         final ApplyState applyState = MessageCollectorActor.expectFirstMatching(followerCollector, ApplyState.class);
@@ -956,13 +945,12 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         LOG.info("testRemoveServerLeaderWithNoFollowers starting");
 
         TestActorRef<MockLeaderRaftActor> leaderActor = actorFactory.createTestActor(
-                MockLeaderRaftActor.props(Collections.<String, String>emptyMap(),
+                MockLeaderRaftActor.props(Map.of(),
                         new MockRaftActorContext()).withDispatcher(Dispatchers.DefaultDispatcherId()),
                 actorFactory.generateActorId(LEADER_ID));
 
         leaderActor.tell(new RemoveServer(LEADER_ID), testKit.getRef());
-        RemoveServerReply removeServerReply = testKit.expectMsgClass(testKit.duration("5 seconds"),
-                RemoveServerReply.class);
+        RemoveServerReply removeServerReply = testKit.expectMsgClass(Duration.ofSeconds(5), RemoveServerReply.class);
         assertEquals("getStatus", ServerChangeStatus.NOT_SUPPORTED, removeServerReply.getStatus());
 
         LOG.info("testRemoveServerLeaderWithNoFollowers ending");
@@ -982,7 +970,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         final String follower2ActorPath = actorFactory.createTestActorPath(follower2ActorId);
 
         TestActorRef<MockLeaderRaftActor> leaderActor = actorFactory.createTestActor(
-                MockLeaderRaftActor.props(ImmutableMap.of(FOLLOWER_ID, follower1ActorPath,
+                MockLeaderRaftActor.props(Map.of(FOLLOWER_ID, follower1ActorPath,
                         FOLLOWER_ID2, follower2ActorPath), new MockRaftActorContext())
                         .withDispatcher(Dispatchers.DefaultDispatcherId()), actorFactory.generateActorId(LEADER_ID));
         ActorRef leaderCollector = newLeaderCollectorActor(leaderActor.underlyingActor());
@@ -990,22 +978,22 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         ActorRef follower1Collector = actorFactory.createActor(
                 MessageCollectorActor.props(), actorFactory.generateActorId("collector"));
         final TestActorRef<CollectingMockRaftActor> follower1RaftActor = actorFactory.createTestActor(
-                CollectingMockRaftActor.props(FOLLOWER_ID, ImmutableMap.of(LEADER_ID, leaderActor.path().toString(),
+                CollectingMockRaftActor.props(FOLLOWER_ID, Map.of(LEADER_ID, leaderActor.path().toString(),
                         FOLLOWER_ID2, follower2ActorPath), configParams, NO_PERSISTENCE, follower1Collector)
                         .withDispatcher(Dispatchers.DefaultDispatcherId()), follower1ActorId);
 
         ActorRef follower2Collector = actorFactory.createActor(
                 MessageCollectorActor.props(), actorFactory.generateActorId("collector"));
         final TestActorRef<CollectingMockRaftActor> follower2RaftActor = actorFactory.createTestActor(
-                CollectingMockRaftActor.props(FOLLOWER_ID2, ImmutableMap.of(LEADER_ID, leaderActor.path().toString(),
+                CollectingMockRaftActor.props(FOLLOWER_ID2, Map.of(LEADER_ID, leaderActor.path().toString(),
                         FOLLOWER_ID, follower1ActorPath), configParams, NO_PERSISTENCE, follower2Collector)
                         .withDispatcher(Dispatchers.DefaultDispatcherId()), follower2ActorId);
 
         // Send first ChangeServersVotingStatus message
 
-        leaderActor.tell(new ChangeServersVotingStatus(ImmutableMap.of(FOLLOWER_ID, false, FOLLOWER_ID2, false)),
+        leaderActor.tell(new ChangeServersVotingStatus(Map.of(FOLLOWER_ID, false, FOLLOWER_ID2, false)),
                 testKit.getRef());
-        ServerChangeReply reply = testKit.expectMsgClass(testKit.duration("5 seconds"), ServerChangeReply.class);
+        ServerChangeReply reply = testKit.expectMsgClass(Duration.ofSeconds(5), ServerChangeReply.class);
         assertEquals("getStatus", ServerChangeStatus.OK, reply.getStatus());
 
         final ApplyState applyState = MessageCollectorActor.expectFirstMatching(leaderCollector, ApplyState.class);
@@ -1029,8 +1017,8 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
 
         // Send second ChangeServersVotingStatus message
 
-        leaderActor.tell(new ChangeServersVotingStatus(ImmutableMap.of(FOLLOWER_ID, true)), testKit.getRef());
-        reply = testKit.expectMsgClass(testKit.duration("5 seconds"), ServerChangeReply.class);
+        leaderActor.tell(new ChangeServersVotingStatus(Map.of(FOLLOWER_ID, true)), testKit.getRef());
+        reply = testKit.expectMsgClass(Duration.ofSeconds(5), ServerChangeReply.class);
         assertEquals("getStatus", ServerChangeStatus.OK, reply.getStatus());
 
         MessageCollectorActor.expectFirstMatching(leaderCollector, ApplyState.class);
@@ -1061,7 +1049,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         final String follower2ActorPath = actorFactory.createTestActorPath(follower2ActorId);
 
         TestActorRef<MockLeaderRaftActor> leaderActor = actorFactory.createTestActor(
-                MockLeaderRaftActor.props(ImmutableMap.of(FOLLOWER_ID, follower1ActorPath,
+                MockLeaderRaftActor.props(Map.of(FOLLOWER_ID, follower1ActorPath,
                         FOLLOWER_ID2, follower2ActorPath), new MockRaftActorContext())
                         .withDispatcher(Dispatchers.DefaultDispatcherId()), actorFactory.generateActorId(LEADER_ID));
         ActorRef leaderCollector = newLeaderCollectorActor(leaderActor.underlyingActor());
@@ -1069,21 +1057,21 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         ActorRef follower1Collector = actorFactory.createActor(
                 MessageCollectorActor.props(), actorFactory.generateActorId("collector"));
         final TestActorRef<CollectingMockRaftActor> follower1RaftActor = actorFactory.createTestActor(
-                CollectingMockRaftActor.props(FOLLOWER_ID, ImmutableMap.of(LEADER_ID, leaderActor.path().toString(),
+                CollectingMockRaftActor.props(FOLLOWER_ID, Map.of(LEADER_ID, leaderActor.path().toString(),
                         FOLLOWER_ID2, follower2ActorPath), configParams, NO_PERSISTENCE, follower1Collector)
                         .withDispatcher(Dispatchers.DefaultDispatcherId()), follower1ActorId);
 
         ActorRef follower2Collector = actorFactory.createActor(
                 MessageCollectorActor.props(), actorFactory.generateActorId("collector"));
         final TestActorRef<CollectingMockRaftActor> follower2RaftActor = actorFactory.createTestActor(
-                CollectingMockRaftActor.props(FOLLOWER_ID2, ImmutableMap.of(LEADER_ID, leaderActor.path().toString(),
+                CollectingMockRaftActor.props(FOLLOWER_ID2, Map.of(LEADER_ID, leaderActor.path().toString(),
                         FOLLOWER_ID, follower1ActorPath), configParams, NO_PERSISTENCE, follower2Collector)
                         .withDispatcher(Dispatchers.DefaultDispatcherId()), follower2ActorId);
 
         // Send ChangeServersVotingStatus message
 
-        leaderActor.tell(new ChangeServersVotingStatus(ImmutableMap.of(LEADER_ID, false)), testKit.getRef());
-        ServerChangeReply reply = testKit.expectMsgClass(testKit.duration("5 seconds"), ServerChangeReply.class);
+        leaderActor.tell(new ChangeServersVotingStatus(Map.of(LEADER_ID, false)), testKit.getRef());
+        ServerChangeReply reply = testKit.expectMsgClass(Duration.ofSeconds(5), ServerChangeReply.class);
         assertEquals("getStatus", ServerChangeStatus.OK, reply.getStatus());
 
         MessageCollectorActor.expectFirstMatching(leaderCollector, ApplyState.class);
@@ -1111,11 +1099,11 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         LOG.info("testChangeLeaderToNonVotingInSingleNode starting");
 
         TestActorRef<MockLeaderRaftActor> leaderActor = actorFactory.createTestActor(
-                MockLeaderRaftActor.props(ImmutableMap.of(), new MockRaftActorContext())
+                MockLeaderRaftActor.props(Map.of(), new MockRaftActorContext())
                         .withDispatcher(Dispatchers.DefaultDispatcherId()), actorFactory.generateActorId(LEADER_ID));
 
-        leaderActor.tell(new ChangeServersVotingStatus(ImmutableMap.of(LEADER_ID, false)), testKit.getRef());
-        ServerChangeReply reply = testKit.expectMsgClass(testKit.duration("5 seconds"), ServerChangeReply.class);
+        leaderActor.tell(new ChangeServersVotingStatus(Map.of(LEADER_ID, false)), testKit.getRef());
+        ServerChangeReply reply = testKit.expectMsgClass(Duration.ofSeconds(5), ServerChangeReply.class);
         assertEquals("getStatus", ServerChangeStatus.INVALID_REQUEST, reply.getStatus());
 
         LOG.info("testChangeLeaderToNonVotingInSingleNode ending");
@@ -1136,7 +1124,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         // via the server config. The server config will also contain 2 voting peers that are down (ie no
         // actors created).
 
-        ServerConfigurationPayload persistedServerConfig = new ServerConfigurationPayload(Arrays.asList(
+        ServerConfigurationPayload persistedServerConfig = new ServerConfigurationPayload(List.of(
                 new ServerInfo(node1ID, false), new ServerInfo(node2ID, false),
                 new ServerInfo("downNode1", true), new ServerInfo("downNode2", true)));
         SimpleReplicatedLogEntry persistedServerConfigEntry = new SimpleReplicatedLogEntry(0, 1, persistedServerConfig);
@@ -1151,14 +1139,14 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         ActorRef node1Collector = actorFactory.createActor(
                 MessageCollectorActor.props(), actorFactory.generateActorId("collector"));
         TestActorRef<CollectingMockRaftActor> node1RaftActorRef = actorFactory.createTestActor(
-                CollectingMockRaftActor.props(node1ID, ImmutableMap.<String, String>of(), configParams,
+                CollectingMockRaftActor.props(node1ID, Map.of(), configParams,
                         PERSISTENT, node1Collector).withDispatcher(Dispatchers.DefaultDispatcherId()), node1ID);
         CollectingMockRaftActor node1RaftActor = node1RaftActorRef.underlyingActor();
 
         ActorRef node2Collector = actorFactory.createActor(
                 MessageCollectorActor.props(), actorFactory.generateActorId("collector"));
         TestActorRef<CollectingMockRaftActor> node2RaftActorRef = actorFactory.createTestActor(
-                CollectingMockRaftActor.props(node2ID, ImmutableMap.<String, String>of(), configParams,
+                CollectingMockRaftActor.props(node2ID, Map.of(), configParams,
                         PERSISTENT, node2Collector).withDispatcher(Dispatchers.DefaultDispatcherId()), node2ID);
         CollectingMockRaftActor node2RaftActor = node2RaftActorRef.underlyingActor();
 
@@ -1185,10 +1173,10 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
 
         // First send the message such that node1 has no peer address for node2 - should fail.
 
-        ChangeServersVotingStatus changeServers = new ChangeServersVotingStatus(ImmutableMap.of(node1ID, true,
+        ChangeServersVotingStatus changeServers = new ChangeServersVotingStatus(Map.of(node1ID, true,
                 node2ID, true, "downNode1", false, "downNode2", false));
         node1RaftActorRef.tell(changeServers, testKit.getRef());
-        ServerChangeReply reply = testKit.expectMsgClass(testKit.duration("5 seconds"), ServerChangeReply.class);
+        ServerChangeReply reply = testKit.expectMsgClass(Duration.ofSeconds(5), ServerChangeReply.class);
         assertEquals("getStatus", ServerChangeStatus.NO_LEADER, reply.getStatus());
         assertEquals("getRaftState", RaftState.Follower, node1RaftActor.getRaftState());
 
@@ -1196,7 +1184,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
 
         long term = node1RaftActor.getRaftActorContext().getTermInformation().getCurrentTerm();
         node1RaftActorRef.tell(new AppendEntries(term, "downNode1", -1L, -1L,
-                Collections.<ReplicatedLogEntry>emptyList(), 0, -1, (short)1), ActorRef.noSender());
+                List.of(), 0, -1, (short)1), ActorRef.noSender());
 
         // Wait for the ElectionTimeout to clear the leaderId. The leaderId must be null so on the next
         // ChangeServersVotingStatus message, it will try to elect a leader.
@@ -1209,7 +1197,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         node1RaftActor.setPeerAddress(node2ID, node2RaftActorRef.path().toString());
 
         node1RaftActorRef.tell(changeServers, testKit.getRef());
-        reply = testKit.expectMsgClass(testKit.duration("5 seconds"), ServerChangeReply.class);
+        reply = testKit.expectMsgClass(Duration.ofSeconds(5), ServerChangeReply.class);
         assertEquals("getStatus", ServerChangeStatus.OK, reply.getStatus());
 
         ApplyJournalEntries apply = MessageCollectorActor.expectFirstMatching(node1Collector,
@@ -1243,7 +1231,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
                 ? actorFactory.createTestActorPath(node1ID) : peerId.equals(node2ID)
                         ? actorFactory.createTestActorPath(node2ID) : null;
 
-        ServerConfigurationPayload persistedServerConfig = new ServerConfigurationPayload(Arrays.asList(
+        ServerConfigurationPayload persistedServerConfig = new ServerConfigurationPayload(List.of(
                 new ServerInfo(node1ID, false), new ServerInfo(node2ID, true)));
         SimpleReplicatedLogEntry persistedServerConfigEntry = new SimpleReplicatedLogEntry(0, 1, persistedServerConfig);
 
@@ -1259,7 +1247,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         ActorRef node1Collector = actorFactory.createActor(
                 MessageCollectorActor.props(), actorFactory.generateActorId("collector"));
         TestActorRef<CollectingMockRaftActor> node1RaftActorRef = actorFactory.createTestActor(
-                CollectingMockRaftActor.props(node1ID, ImmutableMap.<String, String>of(), configParams1,
+                CollectingMockRaftActor.props(node1ID, Map.of(), configParams1,
                         PERSISTENT, node1Collector).withDispatcher(Dispatchers.DefaultDispatcherId()), node1ID);
         final CollectingMockRaftActor node1RaftActor = node1RaftActorRef.underlyingActor();
 
@@ -1269,7 +1257,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         ActorRef node2Collector = actorFactory.createActor(
                 MessageCollectorActor.props(), actorFactory.generateActorId("collector"));
         TestActorRef<CollectingMockRaftActor> node2RaftActorRef = actorFactory.createTestActor(
-                CollectingMockRaftActor.props(node2ID, ImmutableMap.<String, String>of(), configParams2,
+                CollectingMockRaftActor.props(node2ID, Map.of(), configParams2,
                         PERSISTENT, node2Collector).withDispatcher(Dispatchers.DefaultDispatcherId()), node2ID);
         CollectingMockRaftActor node2RaftActor = node2RaftActorRef.underlyingActor();
 
@@ -1281,13 +1269,13 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
 
         node2RaftActor.setDropMessageOfType(RequestVote.class);
 
-        ChangeServersVotingStatus changeServers = new ChangeServersVotingStatus(ImmutableMap.of(node1ID, true));
+        ChangeServersVotingStatus changeServers = new ChangeServersVotingStatus(Map.of(node1ID, true));
         node1RaftActorRef.tell(changeServers, testKit.getRef());
-        ServerChangeReply reply = testKit.expectMsgClass(testKit.duration("5 seconds"), ServerChangeReply.class);
+        ServerChangeReply reply = testKit.expectMsgClass(Duration.ofSeconds(5), ServerChangeReply.class);
         assertEquals("getStatus", ServerChangeStatus.NO_LEADER, reply.getStatus());
 
-        assertEquals("Server config", Sets.newHashSet(nonVotingServer(node1ID), votingServer(node2ID)),
-                Sets.newHashSet(node1RaftActor.getRaftActorContext().getPeerServerInfo(true).getServerConfig()));
+        assertEquals("Server config", Set.of(nonVotingServer(node1ID), votingServer(node2ID)),
+            Set.copyOf(node1RaftActor.getRaftActorContext().getPeerServerInfo(true).getServerConfig()));
         assertEquals("getRaftState", RaftState.Follower, node1RaftActor.getRaftState());
 
         LOG.info("testChangeToVotingWithNoLeaderAndElectionTimeout ending");
@@ -1309,7 +1297,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         configParams.setElectionTimeoutFactor(3);
         configParams.setPeerAddressResolver(peerAddressResolver);
 
-        ServerConfigurationPayload persistedServerConfig = new ServerConfigurationPayload(Arrays.asList(
+        ServerConfigurationPayload persistedServerConfig = new ServerConfigurationPayload(List.of(
                 new ServerInfo(node1ID, false), new ServerInfo(node2ID, false)));
         SimpleReplicatedLogEntry persistedServerConfigEntry = new SimpleReplicatedLogEntry(0, 1, persistedServerConfig);
 
@@ -1324,14 +1312,14 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         ActorRef node1Collector = actorFactory.createActor(
                 MessageCollectorActor.props(), actorFactory.generateActorId("collector"));
         TestActorRef<CollectingMockRaftActor> node1RaftActorRef = actorFactory.createTestActor(
-                CollectingMockRaftActor.props(node1ID, ImmutableMap.<String, String>of(), configParams,
+                CollectingMockRaftActor.props(node1ID, Map.of(), configParams,
                         PERSISTENT, node1Collector).withDispatcher(Dispatchers.DefaultDispatcherId()), node1ID);
         final CollectingMockRaftActor node1RaftActor = node1RaftActorRef.underlyingActor();
 
         ActorRef node2Collector = actorFactory.createActor(
                 MessageCollectorActor.props(), actorFactory.generateActorId("collector"));
         TestActorRef<CollectingMockRaftActor> node2RaftActorRef = actorFactory.createTestActor(
-                CollectingMockRaftActor.props(node2ID, ImmutableMap.<String, String>of(), configParams,
+                CollectingMockRaftActor.props(node2ID, Map.of(), configParams,
                         PERSISTENT, node2Collector).withDispatcher(Dispatchers.DefaultDispatcherId()), node2ID);
         final CollectingMockRaftActor node2RaftActor = node2RaftActorRef.underlyingActor();
 
@@ -1341,9 +1329,9 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         // forward the request to node2.
 
         ChangeServersVotingStatus changeServers = new ChangeServersVotingStatus(
-                ImmutableMap.of(node1ID, true, node2ID, true));
+                Map.of(node1ID, true, node2ID, true));
         node1RaftActorRef.tell(changeServers, testKit.getRef());
-        ServerChangeReply reply = testKit.expectMsgClass(testKit.duration("5 seconds"), ServerChangeReply.class);
+        ServerChangeReply reply = testKit.expectMsgClass(Duration.ofSeconds(5), ServerChangeReply.class);
         assertEquals("getStatus", ServerChangeStatus.OK, reply.getStatus());
 
         MessageCollectorActor.expectFirstMatching(node2Collector, ApplyJournalEntries.class);
@@ -1375,7 +1363,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
                 ? actorFactory.createTestActorPath(node1ID) : peerId.equals(node2ID)
                         ? actorFactory.createTestActorPath(node2ID) : null);
 
-        ServerConfigurationPayload persistedServerConfig = new ServerConfigurationPayload(Arrays.asList(
+        ServerConfigurationPayload persistedServerConfig = new ServerConfigurationPayload(List.of(
                 new ServerInfo(node1ID, false), new ServerInfo(node2ID, true)));
         SimpleReplicatedLogEntry persistedServerConfigEntry = new SimpleReplicatedLogEntry(0, 1, persistedServerConfig);
 
@@ -1387,14 +1375,14 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         ActorRef node1Collector = actorFactory.createActor(
                 MessageCollectorActor.props(), actorFactory.generateActorId("collector"));
         TestActorRef<CollectingMockRaftActor> node1RaftActorRef = actorFactory.createTestActor(
-                CollectingMockRaftActor.props(node1ID, ImmutableMap.<String, String>of(), configParams,
+                CollectingMockRaftActor.props(node1ID, Map.of(), configParams,
                         PERSISTENT, node1Collector).withDispatcher(Dispatchers.DefaultDispatcherId()), node1ID);
         final CollectingMockRaftActor node1RaftActor = node1RaftActorRef.underlyingActor();
 
         ActorRef node2Collector = actorFactory.createActor(
                 MessageCollectorActor.props(), actorFactory.generateActorId("collector"));
         TestActorRef<CollectingMockRaftActor> node2RaftActorRef = actorFactory.createTestActor(
-                CollectingMockRaftActor.props(node2ID, ImmutableMap.<String, String>of(), configParams,
+                CollectingMockRaftActor.props(node2ID, Map.of(), configParams,
                         PERSISTENT, node2Collector).withDispatcher(Dispatchers.DefaultDispatcherId()), node2ID);
         CollectingMockRaftActor node2RaftActor = node2RaftActorRef.underlyingActor();
 
@@ -1405,7 +1393,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
 
         node2RaftActor.setDropMessageOfType(RequestVote.class);
 
-        ChangeServersVotingStatus changeServers = new ChangeServersVotingStatus(ImmutableMap.of(node1ID, true,
+        ChangeServersVotingStatus changeServers = new ChangeServersVotingStatus(Map.of(node1ID, true,
                 node2ID, true));
         node1RaftActorRef.tell(changeServers, testKit.getRef());
 
@@ -1413,7 +1401,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
 
         node2RaftActorRef.tell(TimeoutNow.INSTANCE, ActorRef.noSender());
 
-        ServerChangeReply reply = testKit.expectMsgClass(testKit.duration("5 seconds"), ServerChangeReply.class);
+        ServerChangeReply reply = testKit.expectMsgClass(Duration.ofSeconds(5), ServerChangeReply.class);
         assertEquals("getStatus", ServerChangeStatus.OK, reply.getStatus());
 
         MessageCollectorActor.expectFirstMatching(node1Collector, ApplyJournalEntries.class);
@@ -1430,7 +1418,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         LOG.info("testChangeToVotingWithNoLeaderAndOtherLeaderElected ending");
     }
 
-    private static void verifyRaftState(RaftState expState, RaftActor... raftActors) {
+    private static void verifyRaftState(final RaftState expState, final RaftActor... raftActors) {
         Stopwatch sw = Stopwatch.createStarted();
         while (sw.elapsed(TimeUnit.SECONDS) <= 5) {
             for (RaftActor raftActor : raftActors) {
@@ -1443,33 +1431,34 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         fail("None of the RaftActors have state " + expState);
     }
 
-    private static ServerInfo votingServer(String id) {
+    private static ServerInfo votingServer(final String id) {
         return new ServerInfo(id, true);
     }
 
-    private static ServerInfo nonVotingServer(String id) {
+    private static ServerInfo nonVotingServer(final String id) {
         return new ServerInfo(id, false);
     }
 
-    private ActorRef newLeaderCollectorActor(MockLeaderRaftActor leaderRaftActor) {
+    private ActorRef newLeaderCollectorActor(final MockLeaderRaftActor leaderRaftActor) {
         return newCollectorActor(leaderRaftActor, LEADER_ID);
     }
 
-    private ActorRef newCollectorActor(AbstractMockRaftActor raftActor, String id) {
+    private ActorRef newCollectorActor(final AbstractMockRaftActor raftActor, final String id) {
         ActorRef collectorActor = actorFactory.createTestActor(
                 MessageCollectorActor.props(), actorFactory.generateActorId(id + "Collector"));
         raftActor.setCollectorActor(collectorActor);
         return collectorActor;
     }
 
-    private static void verifyServerConfigurationPayloadEntry(ReplicatedLog log, ServerInfo... expected) {
+    private static void verifyServerConfigurationPayloadEntry(final ReplicatedLog log, final ServerInfo... expected) {
         ReplicatedLogEntry logEntry = log.get(log.lastIndex());
         assertEquals("Last log entry payload class", ServerConfigurationPayload.class, logEntry.getData().getClass());
         ServerConfigurationPayload payload = (ServerConfigurationPayload)logEntry.getData();
-        assertEquals("Server config", Sets.newHashSet(expected), Sets.newHashSet(payload.getServerConfig()));
+        assertEquals("Server config", Set.of(expected), Set.copyOf(payload.getServerConfig()));
     }
 
-    private static RaftActorContextImpl newFollowerContext(String id, TestActorRef<? extends UntypedActor> actor) {
+    private static RaftActorContextImpl newFollowerContext(final String id,
+            final TestActorRef<? extends AbstractActor> actor) {
         DefaultConfigParamsImpl configParams = new DefaultConfigParamsImpl();
         configParams.setHeartBeatInterval(new FiniteDuration(100, TimeUnit.MILLISECONDS));
         configParams.setElectionTimeoutFactor(100000);
@@ -1477,31 +1466,31 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
         ElectionTermImpl termInfo = new ElectionTermImpl(noPersistence, id, LOG);
         termInfo.update(1, LEADER_ID);
         return new RaftActorContextImpl(actor, actor.underlyingActor().getContext(),
-                id, termInfo, -1, -1, ImmutableMap.of(LEADER_ID, ""), configParams,
-                noPersistence, applyState -> actor.tell(applyState, actor), LOG);
+                id, termInfo, -1, -1, Map.of(LEADER_ID, ""), configParams,
+                noPersistence, applyState -> actor.tell(applyState, actor), LOG,  MoreExecutors.directExecutor());
     }
 
     abstract static class AbstractMockRaftActor extends MockRaftActor {
         private volatile ActorRef collectorActor;
         private volatile Class<?> dropMessageOfType;
 
-        AbstractMockRaftActor(String id, Map<String, String> peerAddresses, Optional<ConfigParams> config,
-                boolean persistent, ActorRef collectorActor) {
+        AbstractMockRaftActor(final String id, final Map<String, String> peerAddresses,
+                final Optional<ConfigParams> config, final boolean persistent, final ActorRef collectorActor) {
             super(builder().id(id).peerAddresses(peerAddresses).config(config.get())
                     .persistent(Optional.of(persistent)));
             this.collectorActor = collectorActor;
         }
 
-        void setDropMessageOfType(Class<?> dropMessageOfType) {
+        void setDropMessageOfType(final Class<?> dropMessageOfType) {
             this.dropMessageOfType = dropMessageOfType;
         }
 
-        void setCollectorActor(ActorRef collectorActor) {
+        void setCollectorActor(final ActorRef collectorActor) {
             this.collectorActor = collectorActor;
         }
 
         @Override
-        public void handleCommand(Object message) {
+        public void handleCommand(final Object message) {
             if (dropMessageOfType == null || !dropMessageOfType.equals(message.getClass())) {
                 super.handleCommand(message);
             }
@@ -1514,30 +1503,31 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
 
     public static class CollectingMockRaftActor extends AbstractMockRaftActor {
 
-        CollectingMockRaftActor(String id, Map<String, String> peerAddresses, Optional<ConfigParams> config,
-                boolean persistent, ActorRef collectorActor) {
+        CollectingMockRaftActor(final String id, final Map<String, String> peerAddresses,
+                final Optional<ConfigParams> config, final boolean persistent, final ActorRef collectorActor) {
             super(id, peerAddresses, config, persistent, collectorActor);
             snapshotCohortDelegate = new RaftActorSnapshotCohort() {
                 @Override
-                public void createSnapshot(ActorRef actorRef, java.util.Optional<OutputStream> installSnapshotStream) {
+                public void createSnapshot(final ActorRef actorRef,
+                        final Optional<OutputStream> installSnapshotStream) {
                     actorRef.tell(new CaptureSnapshotReply(ByteState.empty(), installSnapshotStream), actorRef);
                 }
 
                 @Override
                 public void applySnapshot(
-                        org.opendaylight.controller.cluster.raft.persisted.Snapshot.State snapshotState) {
+                        final org.opendaylight.controller.cluster.raft.persisted.Snapshot.State snapshotState) {
                 }
 
                 @Override
                 public org.opendaylight.controller.cluster.raft.persisted.Snapshot.State deserializeSnapshot(
-                        ByteSource snapshotBytes) {
+                        final ByteSource snapshotBytes) {
                     throw new UnsupportedOperationException();
                 }
             };
         }
 
         public static Props props(final String id, final Map<String, String> peerAddresses,
-                ConfigParams config, boolean persistent, ActorRef collectorActor) {
+                final ConfigParams config, final boolean persistent, final ActorRef collectorActor) {
 
             return Props.create(CollectingMockRaftActor.class, id, peerAddresses, Optional.of(config),
                     persistent, collectorActor);
@@ -1546,8 +1536,8 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
     }
 
     public static class MockLeaderRaftActor extends AbstractMockRaftActor {
-        public MockLeaderRaftActor(Map<String, String> peerAddresses, ConfigParams config,
-                RaftActorContext fromContext) {
+        public MockLeaderRaftActor(final Map<String, String> peerAddresses, final ConfigParams config,
+                final RaftActorContext fromContext) {
             super(LEADER_ID, peerAddresses, Optional.of(config), NO_PERSISTENCE, null);
             setPersistence(false);
 
@@ -1572,8 +1562,8 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
 
         @Override
         @SuppressWarnings("checkstyle:IllegalCatch")
-        public void createSnapshot(ActorRef actorRef, java.util.Optional<OutputStream> installSnapshotStream) {
-            MockSnapshotState snapshotState = new MockSnapshotState(new ArrayList<>(getState()));
+        public void createSnapshot(final ActorRef actorRef, final Optional<OutputStream> installSnapshotStream) {
+            MockSnapshotState snapshotState = new MockSnapshotState(List.copyOf(getState()));
             if (installSnapshotStream.isPresent()) {
                 SerializationUtils.serialize(snapshotState, installSnapshotStream.get());
             }
@@ -1581,7 +1571,7 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
             actorRef.tell(new CaptureSnapshotReply(snapshotState, installSnapshotStream), actorRef);
         }
 
-        static Props props(Map<String, String> peerAddresses, RaftActorContext fromContext) {
+        static Props props(final Map<String, String> peerAddresses, final RaftActorContext fromContext) {
             DefaultConfigParamsImpl configParams = new DefaultConfigParamsImpl();
             configParams.setHeartBeatInterval(new FiniteDuration(100, TimeUnit.MILLISECONDS));
             configParams.setElectionTimeoutFactor(10);
@@ -1590,13 +1580,12 @@ public class RaftActorServerConfigurationSupportTest extends AbstractActorTest {
     }
 
     public static class MockNewFollowerRaftActor extends AbstractMockRaftActor {
-        public MockNewFollowerRaftActor(ConfigParams config, ActorRef collectorActor) {
-            super(NEW_SERVER_ID, Maps.<String, String>newHashMap(), Optional.of(config), NO_PERSISTENCE,
-                    collectorActor);
+        public MockNewFollowerRaftActor(final ConfigParams config, final ActorRef collectorActor) {
+            super(NEW_SERVER_ID, Map.of(), Optional.of(config), NO_PERSISTENCE, collectorActor);
             setPersistence(false);
         }
 
-        static Props props(ConfigParams config, ActorRef collectorActor) {
+        static Props props(final ConfigParams config, final ActorRef collectorActor) {
             return Props.create(MockNewFollowerRaftActor.class, config, collectorActor);
         }
     }