Do not break actor containment
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / test / java / org / opendaylight / controller / cluster / raft / AbstractRaftActorIntegrationTest.java
index 9bc8a5361556820a6eb808966ceb75765a9ac890..d8d0ce57721d0c5e11876910e1f721c493628a4f 100644 (file)
@@ -16,22 +16,27 @@ import akka.actor.InvalidActorNameException;
 import akka.actor.PoisonPill;
 import akka.actor.Terminated;
 import akka.dispatch.Dispatchers;
-import akka.testkit.JavaTestKit;
+import akka.dispatch.Mailboxes;
 import akka.testkit.TestActorRef;
+import akka.testkit.javadsl.TestKit;
 import akka.util.Timeout;
 import com.google.common.base.Stopwatch;
-import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.Uninterruptibles;
+import java.io.OutputStream;
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
 import java.util.function.Predicate;
+import org.apache.commons.lang3.SerializationUtils;
 import org.junit.After;
+import org.opendaylight.controller.cluster.raft.MockRaftActor.MockSnapshotState;
 import org.opendaylight.controller.cluster.raft.MockRaftActorContext.MockPayload;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
@@ -41,6 +46,7 @@ import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftS
 import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState;
 import org.opendaylight.controller.cluster.raft.persisted.ApplyJournalEntries;
 import org.opendaylight.controller.cluster.raft.persisted.ServerConfigurationPayload;
+import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
@@ -62,7 +68,7 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
     private static final class MockIdentifier extends AbstractStringIdentifier<MockIdentifier> {
         private static final long serialVersionUID = 1L;
 
-        protected MockIdentifier(String string) {
+        protected MockIdentifier(final String string) {
             super(string);
         }
     }
@@ -71,7 +77,7 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
         private final String peerId;
         private final String peerAddress;
 
-        public SetPeerAddress(String peerId, String peerAddress) {
+        public SetPeerAddress(final String peerId, final String peerAddress) {
             this.peerId = peerId;
             this.peerAddress = peerAddress;
         }
@@ -85,25 +91,53 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
         }
     }
 
+    /**
+     * Message intended for testing to allow triggering persistData via the mailbox.
+     */
+    public static final class TestPersist {
+
+        private final ActorRef actorRef;
+        private final Identifier identifier;
+        private final Payload payload;
+
+        TestPersist(final ActorRef actorRef, final Identifier identifier, final Payload payload) {
+            this.actorRef = actorRef;
+            this.identifier = identifier;
+            this.payload = payload;
+        }
+
+        public ActorRef getActorRef() {
+            return actorRef;
+        }
+
+        public Identifier getIdentifier() {
+            return identifier;
+        }
+
+        public Payload getPayload() {
+            return payload;
+        }
+    }
+
     public static class TestRaftActor extends MockRaftActor {
 
-        private final TestActorRef<MessageCollectorActor> collectorActor;
+        private final ActorRef collectorActor;
         private final Map<Class<?>, Predicate<?>> dropMessages = new ConcurrentHashMap<>();
 
-        private TestRaftActor(Builder builder) {
+        TestRaftActor(final Builder builder) {
             super(builder);
             this.collectorActor = builder.collectorActor;
         }
 
-        public void startDropMessages(Class<?> msgClass) {
+        public void startDropMessages(final Class<?> msgClass) {
             dropMessages.put(msgClass, msg -> true);
         }
 
-        <T> void startDropMessages(Class<T> msgClass, Predicate<T> filter) {
+        <T> void startDropMessages(final Class<T> msgClass, final Predicate<T> filter) {
             dropMessages.put(msgClass, filter);
         }
 
-        public void stopDropMessages(Class<?> msgClass) {
+        public void stopDropMessages(final Class<?> msgClass) {
             dropMessages.remove(msgClass);
         }
 
@@ -113,24 +147,30 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
 
         @SuppressWarnings({ "rawtypes", "unchecked", "checkstyle:IllegalCatch" })
         @Override
-        public void handleCommand(Object message) {
+        public void handleCommand(final Object message) {
             if (message instanceof MockPayload) {
                 MockPayload payload = (MockPayload) message;
-                super.persistData(collectorActor, new MockIdentifier(payload.toString()), payload);
+                super.persistData(collectorActor, new MockIdentifier(payload.toString()), payload, false);
                 return;
             }
 
             if (message instanceof ServerConfigurationPayload) {
-                super.persistData(collectorActor, new MockIdentifier("serverConfig"), (Payload) message);
+                super.persistData(collectorActor, new MockIdentifier("serverConfig"), (Payload) message, false);
                 return;
             }
 
             if (message instanceof SetPeerAddress) {
-                setPeerAddress(((SetPeerAddress) message).getPeerId().toString(),
+                setPeerAddress(((SetPeerAddress) message).getPeerId(),
                         ((SetPeerAddress) message).getPeerAddress());
                 return;
             }
 
+            if (message instanceof TestPersist) {
+                persistData(((TestPersist) message).getActorRef(), ((TestPersist) message).getIdentifier(),
+                        ((TestPersist) message).getPayload(), false);
+                return;
+            }
+
             try {
                 Predicate drop = dropMessages.get(message.getClass());
                 if (drop == null || !drop.test(message)) {
@@ -149,12 +189,13 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
 
         @Override
         @SuppressWarnings("checkstyle:IllegalCatch")
-        public void createSnapshot(ActorRef actorRef) {
-            try {
-                actorRef.tell(new CaptureSnapshotReply(RaftActorTest.fromObject(getState()).toByteArray()), actorRef);
-            } catch (Exception e) {
-                Throwables.propagate(e);
+        public void createSnapshot(final ActorRef actorRef, final Optional<OutputStream> installSnapshotStream) {
+            MockSnapshotState snapshotState = new MockSnapshotState(new ArrayList<>(getState()));
+            if (installSnapshotStream.isPresent()) {
+                SerializationUtils.serialize(snapshotState, installSnapshotStream.get());
             }
+
+            actorRef.tell(new CaptureSnapshotReply(snapshotState, installSnapshotStream), actorRef);
         }
 
         public ActorRef collectorActor() {
@@ -166,15 +207,15 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
         }
 
         public static class Builder extends AbstractBuilder<Builder, TestRaftActor> {
-            private TestActorRef<MessageCollectorActor> collectorActor;
+            private ActorRef collectorActor;
 
-            public Builder collectorActor(TestActorRef<MessageCollectorActor> newCollectorActor) {
-                this.collectorActor = newCollectorActor;
-                return this;
+            Builder() {
+                super(TestRaftActor.class);
             }
 
-            private Builder() {
-                super(TestRaftActor.class);
+            public Builder collectorActor(final ActorRef newCollectorActor) {
+                this.collectorActor = newCollectorActor;
+                return this;
             }
         }
     }
@@ -210,6 +251,7 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
     protected long currentTerm;
 
     protected int snapshotBatchCount = 4;
+    protected int snapshotChunkSize = SNAPSHOT_CHUNK_SIZE;
 
     protected List<MockPayload> expSnapshotState = new ArrayList<>();
 
@@ -227,7 +269,7 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
         configParams.setSnapshotBatchCount(snapshotBatchCount);
         configParams.setSnapshotDataThresholdPercentage(70);
         configParams.setIsolatedLeaderCheckInterval(new FiniteDuration(1, TimeUnit.DAYS));
-        configParams.setSnapshotChunkSize(SNAPSHOT_CHUNK_SIZE);
+        configParams.setSnapshotChunkSize(snapshotChunkSize);
         return configParams;
     }
 
@@ -238,25 +280,25 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
         return configParams;
     }
 
-    protected void waitUntilLeader(ActorRef actorRef) {
+    protected void waitUntilLeader(final ActorRef actorRef) {
         RaftActorTestKit.waitUntilLeader(actorRef);
     }
 
-    protected TestActorRef<TestRaftActor> newTestRaftActor(String id, Map<String, String> newPeerAddresses,
-            ConfigParams configParams) {
+    protected TestActorRef<TestRaftActor> newTestRaftActor(final String id, final Map<String, String> newPeerAddresses,
+            final ConfigParams configParams) {
         return newTestRaftActor(id, TestRaftActor.newBuilder().peerAddresses(newPeerAddresses != null
                 ? newPeerAddresses : Collections.<String, String>emptyMap()).config(configParams));
     }
 
-    protected TestActorRef<TestRaftActor> newTestRaftActor(String id, TestRaftActor.Builder builder) {
-        builder.collectorActor(factory.<MessageCollectorActor>createTestActor(
-                MessageCollectorActor.props().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                        factory.generateActorId(id + "-collector"))).id(id);
+    protected TestActorRef<TestRaftActor> newTestRaftActor(final String id, final TestRaftActor.Builder builder) {
+        builder.collectorActor(factory.createActor(
+                MessageCollectorActor.props(), factory.generateActorId(id + "-collector"))).id(id);
 
         InvalidActorNameException lastEx = null;
         for (int i = 0; i < 10; i++) {
             try {
-                return factory.createTestActor(builder.props().withDispatcher(Dispatchers.DefaultDispatcherId()), id);
+                return factory.createTestActor(builder.props().withDispatcher(Dispatchers.DefaultDispatcherId())
+                        .withMailbox(Mailboxes.DefaultMailboxId()), id);
             } catch (InvalidActorNameException e) {
                 lastEx = e;
                 Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
@@ -267,31 +309,29 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
         throw lastEx;
     }
 
-    protected void killActor(TestActorRef<TestRaftActor> actor) {
-        JavaTestKit testkit = new JavaTestKit(getSystem());
+    protected void killActor(final TestActorRef<TestRaftActor> actor) {
+        TestKit testkit = new TestKit(getSystem());
         testkit.watch(actor);
 
         actor.tell(PoisonPill.getInstance(), null);
-        testkit.expectMsgClass(JavaTestKit.duration("5 seconds"), Terminated.class);
+        testkit.expectMsgClass(Duration.ofSeconds(5), Terminated.class);
 
         testkit.unwatch(actor);
     }
 
-    protected void verifyApplyJournalEntries(ActorRef actor, final long expIndex) {
+    protected void verifyApplyJournalEntries(final ActorRef actor, final long expIndex) {
         MessageCollectorActor.expectFirstMatching(actor, ApplyJournalEntries.class,
             msg -> msg.getToIndex() == expIndex);
     }
 
-    @SuppressWarnings("unchecked")
-    protected void verifySnapshot(String prefix, Snapshot snapshot, long lastAppliedTerm,
-            long lastAppliedIndex, long lastTerm, long lastIndex)
-                    throws Exception {
+    protected void verifySnapshot(final String prefix, final Snapshot snapshot, final long lastAppliedTerm,
+            final long lastAppliedIndex, final long lastTerm, final long lastIndex) {
         assertEquals(prefix + " Snapshot getLastAppliedTerm", lastAppliedTerm, snapshot.getLastAppliedTerm());
         assertEquals(prefix + " Snapshot getLastAppliedIndex", lastAppliedIndex, snapshot.getLastAppliedIndex());
         assertEquals(prefix + " Snapshot getLastTerm", lastTerm, snapshot.getLastTerm());
         assertEquals(prefix + " Snapshot getLastIndex", lastIndex, snapshot.getLastIndex());
 
-        List<Object> actualState = (List<Object>)MockRaftActor.toObject(snapshot.getState());
+        List<Object> actualState = ((MockSnapshotState)snapshot.getState()).getState();
         assertEquals(String.format("%s Snapshot getState size. Expected %s: . Actual: %s", prefix, expSnapshotState,
                 actualState), expSnapshotState.size(), actualState.size());
         for (int i = 0; i < expSnapshotState.size(); i++) {
@@ -299,7 +339,8 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
         }
     }
 
-    protected void verifyPersistedJournal(String persistenceId, List<? extends ReplicatedLogEntry> expJournal) {
+    protected void verifyPersistedJournal(final String persistenceId,
+            final List<? extends ReplicatedLogEntry> expJournal) {
         List<ReplicatedLogEntry> journal = InMemoryJournal.get(persistenceId, ReplicatedLogEntry.class);
         assertEquals("Journal ReplicatedLogEntry count", expJournal.size(), journal.size());
         for (int i = 0; i < expJournal.size(); i++) {
@@ -309,11 +350,11 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
         }
     }
 
-    protected MockPayload sendPayloadData(ActorRef actor, String data) {
+    protected MockPayload sendPayloadData(final ActorRef actor, final String data) {
         return sendPayloadData(actor, data, 0);
     }
 
-    protected MockPayload sendPayloadData(ActorRef actor, String data, int size) {
+    protected MockPayload sendPayloadData(final ActorRef actor, final String data, final int size) {
         MockPayload payload;
         if (size > 0) {
             payload = new MockPayload(data, size);
@@ -325,8 +366,8 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
         return payload;
     }
 
-    protected void verifyApplyState(ApplyState applyState, ActorRef expClientActor,
-            String expId, long expTerm, long expIndex, Payload payload) {
+    protected void verifyApplyState(final ApplyState applyState, final ActorRef expClientActor,
+            final String expId, final long expTerm, final long expIndex, final Payload payload) {
         assertEquals("ApplyState getClientActor", expClientActor, applyState.getClientActor());
 
         final Identifier id = expId == null ? null : new MockIdentifier(expId);
@@ -335,31 +376,32 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
         verifyReplicatedLogEntry(replicatedLogEntry, expTerm, expIndex, payload);
     }
 
-    protected void verifyReplicatedLogEntry(ReplicatedLogEntry replicatedLogEntry, long expTerm, long expIndex,
-            Payload payload) {
+    protected void verifyReplicatedLogEntry(final ReplicatedLogEntry replicatedLogEntry, final long expTerm,
+            final long expIndex, final Payload payload) {
         assertEquals("ReplicatedLogEntry getTerm", expTerm, replicatedLogEntry.getTerm());
         assertEquals("ReplicatedLogEntry getIndex", expIndex, replicatedLogEntry.getIndex());
         assertEquals("ReplicatedLogEntry getData", payload, replicatedLogEntry.getData());
     }
 
-    protected String testActorPath(String id) {
+    protected String testActorPath(final String id) {
         return factory.createTestActorPath(id);
     }
 
-    protected void verifyLeadersTrimmedLog(long lastIndex) {
+    protected void verifyLeadersTrimmedLog(final long lastIndex) {
         verifyTrimmedLog("Leader", leaderActor, lastIndex, lastIndex - 1);
     }
 
-    protected void verifyLeadersTrimmedLog(long lastIndex, long replicatedToAllIndex) {
+    protected void verifyLeadersTrimmedLog(final long lastIndex, final long replicatedToAllIndex) {
         verifyTrimmedLog("Leader", leaderActor, lastIndex, replicatedToAllIndex);
     }
 
-    protected void verifyFollowersTrimmedLog(int num, TestActorRef<TestRaftActor> actorRef, long lastIndex) {
+    protected void verifyFollowersTrimmedLog(final int num, final TestActorRef<TestRaftActor> actorRef,
+            final long lastIndex) {
         verifyTrimmedLog("Follower " + num, actorRef, lastIndex, lastIndex - 1);
     }
 
-    protected void verifyTrimmedLog(String name, TestActorRef<TestRaftActor> actorRef, long lastIndex,
-            long replicatedToAllIndex) {
+    protected void verifyTrimmedLog(final String name, final TestActorRef<TestRaftActor> actorRef, final long lastIndex,
+            final long replicatedToAllIndex) {
         TestRaftActor actor = actorRef.underlyingActor();
         RaftActorContext context = actor.getRaftActorContext();
         long snapshotIndex = lastIndex - 1;
@@ -375,7 +417,7 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
     }
 
     @SuppressWarnings("checkstyle:IllegalCatch")
-    static void verifyRaftState(ActorRef raftActor, Consumer<OnDemandRaftState> verifier) {
+    static void verifyRaftState(final ActorRef raftActor, final Consumer<OnDemandRaftState> verifier) {
         Timeout timeout = new Timeout(500, TimeUnit.MILLISECONDS);
         AssertionError lastError = null;
         Stopwatch sw = Stopwatch.createStarted();