X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Ftest%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2FRaftActorTest.java;h=2c975ecf08e1d5aecdd13685f0be90ad4b3eed7e;hp=3def4f01f1859f8cbe856cffc546228ac570f4bf;hb=2fd1fa721510a30f58b3bc277deb05fce58badd6;hpb=917283d6f5633ad981bebb6dfc6d6e0245876ab5 diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/RaftActorTest.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/RaftActorTest.java index 3def4f01f1..2c975ecf08 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/RaftActorTest.java +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/RaftActorTest.java @@ -5,7 +5,6 @@ * terms of the Eclipse Public License v1.0 which accompanies this distribution, * and is available at http://www.eclipse.org/legal/epl-v10.html */ - package org.opendaylight.controller.cluster.raft; import static org.junit.Assert.assertEquals; @@ -14,10 +13,9 @@ import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyObject; -import static org.mockito.Matchers.eq; -import static org.mockito.Matchers.same; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.same; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -27,7 +25,6 @@ import static org.mockito.Mockito.verify; import akka.actor.ActorRef; import akka.actor.PoisonPill; -import akka.actor.Props; import akka.actor.Status.Failure; import akka.actor.Terminated; import akka.dispatch.Dispatchers; @@ -36,14 +33,15 @@ import akka.persistence.SaveSnapshotFailure; import akka.persistence.SaveSnapshotSuccess; import akka.persistence.SnapshotMetadata; import akka.persistence.SnapshotOffer; -import akka.testkit.JavaTestKit; +import akka.protobuf.ByteString; import akka.testkit.TestActorRef; +import akka.testkit.javadsl.TestKit; import com.google.common.base.Optional; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.Uninterruptibles; -import com.google.protobuf.ByteString; import java.io.ByteArrayOutputStream; import java.io.ObjectOutputStream; +import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -52,7 +50,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import org.apache.commons.lang3.SerializationUtils; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -93,7 +90,6 @@ import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor; import org.opendaylight.yangtools.concepts.Identifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.concurrent.duration.Duration; import scala.concurrent.duration.FiniteDuration; public class RaftActorTest extends AbstractActorTest { @@ -108,7 +104,7 @@ public class RaftActorTest extends AbstractActorTest { } @After - public void tearDown() throws Exception { + public void tearDown() { factory.close(); InMemoryJournal.clear(); InMemorySnapshotStore.clear(); @@ -127,10 +123,10 @@ public class RaftActorTest extends AbstractActorTest { @Test - public void testRaftActorRecoveryWithPersistenceEnabled() throws Exception { + public void testRaftActorRecoveryWithPersistenceEnabled() { TEST_LOG.info("testRaftActorRecoveryWithPersistenceEnabled starting"); - JavaTestKit kit = new JavaTestKit(getSystem()); + TestKit kit = new TestKit(getSystem()); String persistenceId = factory.generateActorId("follower-"); DefaultConfigParamsImpl config = new DefaultConfigParamsImpl(); @@ -185,7 +181,7 @@ public class RaftActorTest extends AbstractActorTest { // kill the actor followerActor.tell(PoisonPill.getInstance(), null); - kit.expectMsgClass(JavaTestKit.duration("5 seconds"), Terminated.class); + kit.expectMsgClass(Duration.ofSeconds(5), Terminated.class); kit.unwatch(followerActor); @@ -214,7 +210,7 @@ public class RaftActorTest extends AbstractActorTest { } @Test - public void testRaftActorRecoveryWithPersistenceDisabled() throws Exception { + public void testRaftActorRecoveryWithPersistenceDisabled() { String persistenceId = factory.generateActorId("follower-"); DefaultConfigParamsImpl config = new DefaultConfigParamsImpl(); @@ -223,7 +219,7 @@ public class RaftActorTest extends AbstractActorTest { TestActorRef ref = factory.createTestActor(MockRaftActor.props(persistenceId, ImmutableMap.builder().put("member1", "address").build(), - config, new NonPersistentDataProvider()), persistenceId); + config, createProvider()), persistenceId); MockRaftActor mockRaftActor = ref.underlyingActor(); @@ -235,8 +231,8 @@ public class RaftActorTest extends AbstractActorTest { } @Test - public void testUpdateElectionTermPersistedWithPersistenceDisabled() throws Exception { - final JavaTestKit kit = new JavaTestKit(getSystem()); + public void testUpdateElectionTermPersistedWithPersistenceDisabled() { + final TestKit kit = new TestKit(getSystem()); String persistenceId = factory.generateActorId("follower-"); DefaultConfigParamsImpl config = new DefaultConfigParamsImpl(); config.setHeartBeatInterval(new FiniteDuration(100, TimeUnit.MILLISECONDS)); @@ -246,7 +242,7 @@ public class RaftActorTest extends AbstractActorTest { TestActorRef ref = factory.createTestActor(MockRaftActor.props(persistenceId, ImmutableMap.builder().put("member1", "address").build(), - config, new NonPersistentDataProvider()) + config, createProvider()) .withDispatcher(Dispatchers.DefaultDispatcherId()), persistenceId); InMemoryJournal.waitForWriteMessagesComplete(persistenceId); @@ -259,7 +255,7 @@ public class RaftActorTest extends AbstractActorTest { config.setHeartBeatInterval(new FiniteDuration(1, TimeUnit.DAYS)); ref = factory.createTestActor(MockRaftActor.props(persistenceId, ImmutableMap.builder().put("member1", "address").build(), config, - new NonPersistentDataProvider()).withDispatcher(Dispatchers.DefaultDispatcherId()), + createProvider()).withDispatcher(Dispatchers.DefaultDispatcherId()), factory.generateActorId("follower-")); MockRaftActor actor = ref.underlyingActor(); @@ -395,7 +391,7 @@ public class RaftActorTest extends AbstractActorTest { } @Test - public void testApplyState() throws Exception { + public void testApplyState() { String persistenceId = factory.generateActorId("leader-"); DefaultConfigParamsImpl config = new DefaultConfigParamsImpl(); @@ -416,13 +412,12 @@ public class RaftActorTest extends AbstractActorTest { final Identifier id = new MockIdentifier("apply-state"); mockRaftActor.getRaftActorContext().getApplyStateConsumer().accept(new ApplyState(mockActorRef, id, entry)); - verify(mockRaftActor.actorDelegate).applyState(eq(mockActorRef), eq(id), anyObject()); + verify(mockRaftActor.actorDelegate).applyState(eq(mockActorRef), eq(id), any()); } @Test public void testRaftRoleChangeNotifierWhenRaftActorHasNoPeers() throws Exception { - TestActorRef notifierActor = factory.createTestActor( - Props.create(MessageCollectorActor.class)); + ActorRef notifierActor = factory.createActor(MessageCollectorActor.props()); MessageCollectorActor.waitUntilReady(notifierActor); DefaultConfigParamsImpl config = new DefaultConfigParamsImpl(); @@ -434,7 +429,7 @@ public class RaftActorTest extends AbstractActorTest { final TestActorRef raftActorRef = factory.createTestActor(MockRaftActor.builder() .id(persistenceId).config(config).roleChangeNotifier(notifierActor).dataPersistenceProvider( - new NonPersistentDataProvider()).props().withDispatcher(Dispatchers.DefaultDispatcherId()), + createProvider()).props().withDispatcher(Dispatchers.DefaultDispatcherId()), persistenceId); List matches = MessageCollectorActor.expectMatching(notifierActor, RoleChanged.class, 3); @@ -464,14 +459,14 @@ public class RaftActorTest extends AbstractActorTest { assertEquals(raftRoleChanged.getMemberId(), leaderStateChange.getLeaderId()); assertEquals(MockRaftActor.PAYLOAD_VERSION, leaderStateChange.getLeaderPayloadVersion()); - notifierActor.underlyingActor().clear(); + MessageCollectorActor.clearMessages(notifierActor); MockRaftActor raftActor = raftActorRef.underlyingActor(); final String newLeaderId = "new-leader"; final short newLeaderVersion = 6; Follower follower = new Follower(raftActor.getRaftActorContext()) { @Override - public RaftActorBehavior handleMessage(ActorRef sender, Object message) { + public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) { setLeaderId(newLeaderId); setLeaderPayloadVersion(newLeaderVersion); return this; @@ -488,7 +483,7 @@ public class RaftActorTest extends AbstractActorTest { assertEquals(RaftState.Leader.name(), raftRoleChanged.getOldRole()); assertEquals(RaftState.Follower.name(), raftRoleChanged.getNewRole()); - notifierActor.underlyingActor().clear(); + MessageCollectorActor.clearMessages(notifierActor); raftActor.handleCommand("any"); @@ -497,7 +492,7 @@ public class RaftActorTest extends AbstractActorTest { assertEquals(newLeaderId, leaderStateChange.getLeaderId()); assertEquals(newLeaderVersion, leaderStateChange.getLeaderPayloadVersion()); - notifierActor.underlyingActor().clear(); + MessageCollectorActor.clearMessages(notifierActor); raftActor.handleCommand("any"); @@ -508,7 +503,7 @@ public class RaftActorTest extends AbstractActorTest { @Test public void testRaftRoleChangeNotifierWhenRaftActorHasPeers() throws Exception { - ActorRef notifierActor = factory.createActor(Props.create(MessageCollectorActor.class)); + ActorRef notifierActor = factory.createActor(MessageCollectorActor.props()); MessageCollectorActor.waitUntilReady(notifierActor); DefaultConfigParamsImpl config = new DefaultConfigParamsImpl(); @@ -553,8 +548,7 @@ public class RaftActorTest extends AbstractActorTest { final String persistenceId = factory.generateActorId("leader-"); final String follower1Id = factory.generateActorId("follower-"); - ActorRef followerActor1 = - factory.createActor(Props.create(MessageCollectorActor.class)); + ActorRef followerActor1 = factory.createActor(MessageCollectorActor.props()); DefaultConfigParamsImpl config = new DefaultConfigParamsImpl(); config.setHeartBeatInterval(new FiniteDuration(1, TimeUnit.DAYS)); @@ -590,7 +584,7 @@ public class RaftActorTest extends AbstractActorTest { leaderActor.getRaftActorContext().getSnapshotManager().capture( new SimpleReplicatedLogEntry(6, 1, new MockRaftActorContext.MockPayload("x")), 4); - verify(leaderActor.snapshotCohortDelegate).createSnapshot(anyObject(), anyObject()); + verify(leaderActor.snapshotCohortDelegate).createSnapshot(any(), any()); assertEquals(8, leaderActor.getReplicatedLog().size()); @@ -643,9 +637,7 @@ public class RaftActorTest extends AbstractActorTest { final String persistenceId = factory.generateActorId("follower-"); final String leaderId = factory.generateActorId("leader-"); - - ActorRef leaderActor1 = - factory.createActor(Props.create(MessageCollectorActor.class)); + ActorRef leaderActor1 = factory.createActor(MessageCollectorActor.props()); DefaultConfigParamsImpl config = new DefaultConfigParamsImpl(); config.setHeartBeatInterval(new FiniteDuration(1, TimeUnit.DAYS)); @@ -682,7 +674,7 @@ public class RaftActorTest extends AbstractActorTest { followerActor.getRaftActorContext().getSnapshotManager().capture( new SimpleReplicatedLogEntry(5, 1, new MockRaftActorContext.MockPayload("D")), 4); - verify(followerActor.snapshotCohortDelegate).createSnapshot(anyObject(), anyObject()); + verify(followerActor.snapshotCohortDelegate).createSnapshot(any(), any()); assertEquals(6, followerActor.getReplicatedLog().size()); @@ -735,8 +727,8 @@ public class RaftActorTest extends AbstractActorTest { final String follower1Id = factory.generateActorId("follower-"); final String follower2Id = factory.generateActorId("follower-"); - final ActorRef followerActor1 = factory.createActor(Props.create(MessageCollectorActor.class), follower1Id); - final ActorRef followerActor2 = factory.createActor(Props.create(MessageCollectorActor.class), follower2Id); + final ActorRef followerActor1 = factory.createActor(MessageCollectorActor.props(), follower1Id); + final ActorRef followerActor2 = factory.createActor(MessageCollectorActor.props(), follower2Id); DefaultConfigParamsImpl config = new DefaultConfigParamsImpl(); config.setHeartBeatInterval(new FiniteDuration(1, TimeUnit.DAYS)); @@ -821,7 +813,7 @@ public class RaftActorTest extends AbstractActorTest { config.setIsolatedLeaderCheckInterval(new FiniteDuration(1, TimeUnit.DAYS)); config.setSnapshotBatchCount(5); - DataPersistenceProvider dataPersistenceProvider = new NonPersistentDataProvider(); + DataPersistenceProvider dataPersistenceProvider = createProvider(); Map peerAddresses = ImmutableMap.builder().put("member1", "address").build(); @@ -865,7 +857,7 @@ public class RaftActorTest extends AbstractActorTest { config.setIsolatedLeaderCheckInterval(new FiniteDuration(1, TimeUnit.DAYS)); config.setSnapshotBatchCount(5); - DataPersistenceProvider dataPersistenceProvider = new NonPersistentDataProvider(); + DataPersistenceProvider dataPersistenceProvider = createProvider(); Map peerAddresses = ImmutableMap.builder().put("member1", "address").build(); @@ -898,6 +890,10 @@ public class RaftActorTest extends AbstractActorTest { assertEquals(3, leader.getReplicatedToAllIndex()); } + private static DataPersistenceProvider createProvider() { + return new NonPersistentDataProvider(Runnable::run); + } + @Test public void testSwitchBehavior() { String persistenceId = factory.generateActorId("leader-"); @@ -907,7 +903,7 @@ public class RaftActorTest extends AbstractActorTest { config.setIsolatedLeaderCheckInterval(new FiniteDuration(1, TimeUnit.DAYS)); config.setSnapshotBatchCount(5); - DataPersistenceProvider dataPersistenceProvider = new NonPersistentDataProvider(); + DataPersistenceProvider dataPersistenceProvider = createProvider(); Map peerAddresses = ImmutableMap.builder().build(); @@ -939,7 +935,7 @@ public class RaftActorTest extends AbstractActorTest { assertEquals(RaftState.Leader, leaderActor.getCurrentBehavior().state()); } - public static ByteString fromObject(Object snapshot) throws Exception { + public static ByteString fromObject(final Object snapshot) throws Exception { ByteArrayOutputStream bos = null; ObjectOutputStream os = null; try { @@ -960,7 +956,7 @@ public class RaftActorTest extends AbstractActorTest { } @Test - public void testUpdateConfigParam() throws Exception { + public void testUpdateConfigParam() { DefaultConfigParamsImpl emptyConfig = new DefaultConfigParamsImpl(); String persistenceId = factory.generateActorId("follower-"); ImmutableMap peerAddresses = @@ -1008,10 +1004,10 @@ public class RaftActorTest extends AbstractActorTest { } @Test - public void testGetSnapshot() throws Exception { + public void testGetSnapshot() { TEST_LOG.info("testGetSnapshot starting"); - final JavaTestKit kit = new JavaTestKit(getSystem()); + final TestKit kit = new TestKit(getSystem()); String persistenceId = factory.generateActorId("test-actor-"); DefaultConfigParamsImpl config = new DefaultConfigParamsImpl(); @@ -1050,7 +1046,7 @@ public class RaftActorTest extends AbstractActorTest { GetSnapshotReply reply = kit.expectMsgClass(GetSnapshotReply.class); assertEquals("getId", persistenceId, reply.getId()); - Snapshot replySnapshot = SerializationUtils.deserialize(reply.getSnapshot()); + Snapshot replySnapshot = reply.getSnapshot(); assertEquals("getElectionTerm", term, replySnapshot.getElectionTerm()); assertEquals("getElectionVotedFor", "member-1", replySnapshot.getElectionVotedFor()); assertEquals("getLastAppliedIndex", 1L, replySnapshot.getLastAppliedIndex()); @@ -1064,14 +1060,15 @@ public class RaftActorTest extends AbstractActorTest { // Test with timeout mockRaftActor.getSnapshotMessageSupport().setSnapshotReplyActorTimeout( - Duration.create(200, TimeUnit.MILLISECONDS)); + FiniteDuration.create(200, TimeUnit.MILLISECONDS)); reset(mockRaftActor.snapshotCohortDelegate); raftActorRef.tell(GetSnapshot.INSTANCE, kit.getRef()); Failure failure = kit.expectMsgClass(akka.actor.Status.Failure.class); assertEquals("Failure cause type", TimeoutException.class, failure.cause().getClass()); - mockRaftActor.getSnapshotMessageSupport().setSnapshotReplyActorTimeout(Duration.create(30, TimeUnit.SECONDS)); + mockRaftActor.getSnapshotMessageSupport().setSnapshotReplyActorTimeout( + FiniteDuration.create(30, TimeUnit.SECONDS)); // Test with persistence disabled. @@ -1080,10 +1077,10 @@ public class RaftActorTest extends AbstractActorTest { raftActorRef.tell(GetSnapshot.INSTANCE, kit.getRef()); reply = kit.expectMsgClass(GetSnapshotReply.class); - verify(mockRaftActor.snapshotCohortDelegate, never()).createSnapshot(anyObject(), anyObject()); + verify(mockRaftActor.snapshotCohortDelegate, never()).createSnapshot(any(), any()); assertEquals("getId", persistenceId, reply.getId()); - replySnapshot = SerializationUtils.deserialize(reply.getSnapshot()); + replySnapshot = reply.getSnapshot(); assertEquals("getElectionTerm", term, replySnapshot.getElectionTerm()); assertEquals("getElectionVotedFor", "member-1", replySnapshot.getElectionVotedFor()); assertEquals("getLastAppliedIndex", -1L, replySnapshot.getLastAppliedIndex()); @@ -1097,7 +1094,7 @@ public class RaftActorTest extends AbstractActorTest { } @Test - public void testRestoreFromSnapshot() throws Exception { + public void testRestoreFromSnapshot() { TEST_LOG.info("testRestoreFromSnapshot starting"); String persistenceId = factory.generateActorId("test-actor-"); @@ -1122,7 +1119,7 @@ public class RaftActorTest extends AbstractActorTest { InMemorySnapshotStore.addSnapshotSavedLatch(persistenceId); TestActorRef raftActorRef = factory.createTestActor(MockRaftActor.builder().id(persistenceId) - .config(config).restoreFromSnapshot(SerializationUtils.serialize(snapshot)).props() + .config(config).restoreFromSnapshot(snapshot).props() .withDispatcher(Dispatchers.DefaultDispatcherId()), persistenceId); MockRaftActor mockRaftActor = raftActorRef.underlyingActor(); @@ -1157,7 +1154,7 @@ public class RaftActorTest extends AbstractActorTest { persistenceId = factory.generateActorId("test-actor-"); raftActorRef = factory.createTestActor(MockRaftActor.builder().id(persistenceId) - .config(config).restoreFromSnapshot(SerializationUtils.serialize(snapshot)) + .config(config).restoreFromSnapshot(snapshot) .persistent(Optional.of(Boolean.FALSE)).props() .withDispatcher(Dispatchers.DefaultDispatcherId()), persistenceId); mockRaftActor = raftActorRef.underlyingActor(); @@ -1189,7 +1186,7 @@ public class RaftActorTest extends AbstractActorTest { new MockRaftActorContext.MockPayload("B"))); TestActorRef raftActorRef = factory.createTestActor(MockRaftActor.builder().id(persistenceId) - .config(config).restoreFromSnapshot(SerializationUtils.serialize(snapshot)).props() + .config(config).restoreFromSnapshot(snapshot).props() .withDispatcher(Dispatchers.DefaultDispatcherId()), persistenceId); MockRaftActor mockRaftActor = raftActorRef.underlyingActor(); @@ -1210,7 +1207,7 @@ public class RaftActorTest extends AbstractActorTest { } @Test - public void testNonVotingOnRecovery() throws Exception { + public void testNonVotingOnRecovery() { TEST_LOG.info("testNonVotingOnRecovery starting"); DefaultConfigParamsImpl config = new DefaultConfigParamsImpl(); @@ -1236,11 +1233,10 @@ public class RaftActorTest extends AbstractActorTest { } @Test - public void testLeaderTransitioning() throws Exception { + public void testLeaderTransitioning() { TEST_LOG.info("testLeaderTransitioning starting"); - TestActorRef notifierActor = factory.createTestActor( - Props.create(MessageCollectorActor.class)); + ActorRef notifierActor = factory.createActor(MessageCollectorActor.props()); DefaultConfigParamsImpl config = new DefaultConfigParamsImpl(); config.setCustomRaftPolicyImplementationClass(DisableElectionsRaftPolicy.class.getName()); @@ -1277,7 +1273,7 @@ public class RaftActorTest extends AbstractActorTest { final String leaderId = factory.generateActorId("leader-"); final String followerId = factory.generateActorId("follower-"); - final ActorRef followerActor = factory.createActor(Props.create(MessageCollectorActor.class)); + final ActorRef followerActor = factory.createActor(MessageCollectorActor.props()); DefaultConfigParamsImpl config = new DefaultConfigParamsImpl(); config.setHeartBeatInterval(new FiniteDuration(1, TimeUnit.DAYS)); @@ -1322,7 +1318,7 @@ public class RaftActorTest extends AbstractActorTest { final String leaderId = factory.generateActorId("leader-"); final String followerId = factory.generateActorId("follower-"); - final ActorRef followerActor = factory.createActor(Props.create(MessageCollectorActor.class)); + final ActorRef followerActor = factory.createActor(MessageCollectorActor.props()); DefaultConfigParamsImpl config = new DefaultConfigParamsImpl(); config.setHeartBeatInterval(new FiniteDuration(1, TimeUnit.DAYS));