Bug 5740: Configure control-aware mailbox
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / test / java / org / opendaylight / controller / cluster / raft / AbstractRaftActorIntegrationTest.java
index dfaa8d55f6e913b7ce34559cc38d68a8343d71e1..5959df768fc6bd3359edfef777331b8bbcc9bb24 100644 (file)
@@ -7,37 +7,54 @@
  */
 package org.opendaylight.controller.cluster.raft;
 
-import static org.junit.Assert.assertArrayEquals;
+import static akka.pattern.Patterns.ask;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
 import akka.actor.ActorRef;
+import akka.actor.InvalidActorNameException;
 import akka.actor.PoisonPill;
-import akka.actor.Props;
 import akka.actor.Terminated;
 import akka.dispatch.Dispatchers;
+import akka.dispatch.Mailboxes;
 import akka.testkit.JavaTestKit;
 import akka.testkit.TestActorRef;
-import com.google.common.base.Optional;
-import com.google.common.base.Predicate;
+import akka.util.Timeout;
+import com.google.common.base.Stopwatch;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.util.concurrent.Uninterruptibles;
+import java.io.OutputStream;
+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.RaftActorTest.MockRaftActor;
-import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
+import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftState;
+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;
 import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor;
+import org.opendaylight.yangtools.concepts.Identifier;
+import org.opendaylight.yangtools.util.AbstractStringIdentifier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import scala.concurrent.Await;
 import scala.concurrent.duration.FiniteDuration;
 
 /**
@@ -47,56 +64,85 @@ import scala.concurrent.duration.FiniteDuration;
  */
 public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest {
 
+    private static final class MockIdentifier extends AbstractStringIdentifier<MockIdentifier> {
+        private static final long serialVersionUID = 1L;
+
+        protected MockIdentifier(String string) {
+            super(string);
+        }
+    }
+
+    public static class SetPeerAddress {
+        private final String peerId;
+        private final String peerAddress;
+
+        public SetPeerAddress(String peerId, String peerAddress) {
+            this.peerId = peerId;
+            this.peerAddress = peerAddress;
+        }
+
+        public String getPeerId() {
+            return peerId;
+        }
+
+        public String getPeerAddress() {
+            return peerAddress;
+        }
+    }
+
     public static class TestRaftActor extends MockRaftActor {
 
         private final TestActorRef<MessageCollectorActor> collectorActor;
-        private final Map<Class<?>, Boolean> dropMessages = new ConcurrentHashMap<>();
-        private volatile byte[] snapshot;
-        private volatile long mockTotalMemory;
-
-        private TestRaftActor(String id, Map<String, String> peerAddresses, ConfigParams config,
-                TestActorRef<MessageCollectorActor> collectorActor) {
-            super(id, peerAddresses, Optional.of(config), null);
-            this.collectorActor = collectorActor;
+        private final Map<Class<?>, Predicate<?>> dropMessages = new ConcurrentHashMap<>();
+
+        private TestRaftActor(Builder builder) {
+            super(builder);
+            this.collectorActor = builder.collectorActor;
         }
 
-        public static Props props(String id, Map<String, String> peerAddresses, ConfigParams config,
-                TestActorRef<MessageCollectorActor> collectorActor) {
-            return Props.create(TestRaftActor.class, id, peerAddresses, config, collectorActor).
-                    withDispatcher(Dispatchers.DefaultDispatcherId());
+        public void startDropMessages(Class<?> msgClass) {
+            dropMessages.put(msgClass, msg -> true);
         }
 
-        void startDropMessages(Class<?> msgClass) {
-            dropMessages.put(msgClass, Boolean.TRUE);
+        <T> void startDropMessages(Class<T> msgClass, Predicate<T> filter) {
+            dropMessages.put(msgClass, filter);
         }
 
-        void stopDropMessages(Class<?> msgClass) {
+        public void stopDropMessages(Class<?> msgClass) {
             dropMessages.remove(msgClass);
         }
 
-        void setMockTotalMemory(long mockTotalMemory) {
-            this.mockTotalMemory = mockTotalMemory;
-        }
-
-        @Override
-        protected long getTotalMemory() {
-            return mockTotalMemory > 0 ? mockTotalMemory : super.getTotalMemory();
+        void setMockTotalMemory(final long mockTotalMemory) {
+            getRaftActorContext().setTotalMemoryRetriever(mockTotalMemory > 0 ? () -> mockTotalMemory : null);
         }
 
+        @SuppressWarnings({ "rawtypes", "unchecked", "checkstyle:IllegalCatch" })
         @Override
         public void handleCommand(Object message) {
-            if(message instanceof MockPayload) {
-                MockPayload payload = (MockPayload)message;
-                super.persistData(collectorActor, payload.toString(), payload);
+            if (message instanceof MockPayload) {
+                MockPayload payload = (MockPayload) message;
+                super.persistData(collectorActor, new MockIdentifier(payload.toString()), payload, false);
+                return;
+            }
+
+            if (message instanceof ServerConfigurationPayload) {
+                super.persistData(collectorActor, new MockIdentifier("serverConfig"), (Payload) message, false);
+                return;
+            }
+
+            if (message instanceof SetPeerAddress) {
+                setPeerAddress(((SetPeerAddress) message).getPeerId().toString(),
+                        ((SetPeerAddress) message).getPeerAddress());
                 return;
             }
 
             try {
-                if(!dropMessages.containsKey(message.getClass())) {
+                Predicate drop = dropMessages.get(message.getClass());
+                if (drop == null || !drop.test(message)) {
                     super.handleCommand(message);
                 }
             } finally {
-                if(!(message instanceof SendHeartBeat)) {
+                if (!(message instanceof SendHeartBeat)) {
                     try {
                         collectorActor.tell(message, ActorRef.noSender());
                     } catch (Exception e) {
@@ -107,25 +153,40 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
         }
 
         @Override
-        protected void createSnapshot() {
-            if(snapshot != null) {
-                getSelf().tell(new CaptureSnapshotReply(snapshot), ActorRef.noSender());
+        @SuppressWarnings("checkstyle:IllegalCatch")
+        public void createSnapshot(ActorRef actorRef, 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);
         }
 
-        @Override
-        protected void applyRecoverySnapshot(byte[] bytes) {
+        public ActorRef collectorActor() {
+            return collectorActor;
         }
 
-        void setSnapshot(byte[] snapshot) {
-            this.snapshot = snapshot;
+        public static Builder newBuilder() {
+            return new Builder();
         }
 
-        public ActorRef collectorActor() {
-            return collectorActor;
+        public static class Builder extends AbstractBuilder<Builder, TestRaftActor> {
+            private TestActorRef<MessageCollectorActor> collectorActor;
+
+            public Builder collectorActor(TestActorRef<MessageCollectorActor> newCollectorActor) {
+                this.collectorActor = newCollectorActor;
+                return this;
+            }
+
+            private Builder() {
+                super(TestRaftActor.class);
+            }
         }
     }
 
+    protected static final int SNAPSHOT_CHUNK_SIZE = 100;
+
     protected final Logger testLog = LoggerFactory.getLogger(getClass());
 
     protected final TestActorFactory factory = new TestActorFactory(getSystem());
@@ -154,6 +215,10 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
     protected long initialTerm = 5;
     protected long currentTerm;
 
+    protected int snapshotBatchCount = 4;
+
+    protected List<MockPayload> expSnapshotState = new ArrayList<>();
+
     @After
     public void tearDown() {
         InMemoryJournal.clear();
@@ -164,10 +229,11 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
     protected DefaultConfigParamsImpl newLeaderConfigParams() {
         DefaultConfigParamsImpl configParams = new DefaultConfigParamsImpl();
         configParams.setHeartBeatInterval(new FiniteDuration(100, TimeUnit.MILLISECONDS));
-        configParams.setElectionTimeoutFactor(1);
-        configParams.setSnapshotBatchCount(4);
+        configParams.setElectionTimeoutFactor(4);
+        configParams.setSnapshotBatchCount(snapshotBatchCount);
         configParams.setSnapshotDataThresholdPercentage(70);
         configParams.setIsolatedLeaderCheckInterval(new FiniteDuration(1, TimeUnit.DAYS));
+        configParams.setSnapshotChunkSize(SNAPSHOT_CHUNK_SIZE);
         return configParams;
     }
 
@@ -179,77 +245,98 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
     }
 
     protected void waitUntilLeader(ActorRef actorRef) {
-        RaftActorTest.RaftActorTestKit.waitUntilLeader(actorRef);
+        RaftActorTestKit.waitUntilLeader(actorRef);
     }
 
-    protected TestActorRef<TestRaftActor> newTestRaftActor(String id, Map<String, String> peerAddresses,
+    protected TestActorRef<TestRaftActor> newTestRaftActor(String id, Map<String, String> newPeerAddresses,
             ConfigParams configParams) {
-        TestActorRef<MessageCollectorActor> collectorActor = factory.createTestActor(
+        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"));
-        return factory.createTestActor(TestRaftActor.props(id,
-                peerAddresses != null ? peerAddresses : Collections.<String, String>emptyMap(),
-                        configParams, collectorActor), id);
+                        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())
+                        .withMailbox(Mailboxes.DefaultMailboxId()), id);
+            } catch (InvalidActorNameException e) {
+                lastEx = e;
+                Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+            }
+        }
+
+        assertNotNull(lastEx);
+        throw lastEx;
     }
 
-    protected void killActor(TestActorRef<TestRaftActor> leaderActor) {
+    protected void killActor(TestActorRef<TestRaftActor> actor) {
         JavaTestKit testkit = new JavaTestKit(getSystem());
-        testkit.watch(leaderActor);
+        testkit.watch(actor);
 
-        leaderActor.tell(PoisonPill.getInstance(), null);
+        actor.tell(PoisonPill.getInstance(), null);
         testkit.expectMsgClass(JavaTestKit.duration("5 seconds"), Terminated.class);
 
-        testkit.unwatch(leaderActor);
+        testkit.unwatch(actor);
     }
 
     protected void verifyApplyJournalEntries(ActorRef actor, final long expIndex) {
-        MessageCollectorActor.expectFirstMatching(actor, ApplyJournalEntries.class, new Predicate<ApplyJournalEntries>() {
-            @Override
-            public boolean apply(ApplyJournalEntries msg) {
-                return msg.getToIndex() == expIndex;
-            }
-        });
+        MessageCollectorActor.expectFirstMatching(actor, ApplyJournalEntries.class,
+            msg -> msg.getToIndex() == expIndex);
     }
 
     protected void verifySnapshot(String prefix, Snapshot snapshot, long lastAppliedTerm,
-            int lastAppliedIndex, long lastTerm, long lastIndex, byte[] data) {
+            long lastAppliedIndex, long lastTerm, long lastIndex)
+                    throws Exception {
         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());
-        assertArrayEquals(prefix + " Snapshot getState", data, 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++) {
+            assertEquals(prefix + " Snapshot state " + i, expSnapshotState.get(i), actualState.get(i));
+        }
     }
 
     protected void verifyPersistedJournal(String persistenceId, 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++) {
+        for (int i = 0; i < expJournal.size(); i++) {
             ReplicatedLogEntry expected = expJournal.get(i);
             ReplicatedLogEntry actual = journal.get(i);
             verifyReplicatedLogEntry(expected, actual.getTerm(), actual.getIndex(), actual.getData());
         }
     }
 
-    protected MockPayload sendPayloadData(ActorRef leaderActor, String data) {
-        return sendPayloadData(leaderActor, data, 0);
+    protected MockPayload sendPayloadData(ActorRef actor, String data) {
+        return sendPayloadData(actor, data, 0);
     }
 
-    protected MockPayload sendPayloadData(ActorRef leaderActor, String data, int size) {
+    protected MockPayload sendPayloadData(ActorRef actor, String data, int size) {
         MockPayload payload;
-        if(size > 0) {
+        if (size > 0) {
             payload = new MockPayload(data, size);
         } else {
             payload = new MockPayload(data);
         }
 
-        leaderActor.tell(payload, ActorRef.noSender());
+        actor.tell(payload, ActorRef.noSender());
         return payload;
     }
 
     protected void verifyApplyState(ApplyState applyState, ActorRef expClientActor,
-            String expId, long expTerm, long expIndex, MockPayload payload) {
+            String expId, long expTerm, long expIndex, Payload payload) {
         assertEquals("ApplyState getClientActor", expClientActor, applyState.getClientActor());
-        assertEquals("ApplyState getIdentifier", expId, applyState.getIdentifier());
+
+        final Identifier id = expId == null ? null : new MockIdentifier(expId);
+        assertEquals("ApplyState getIdentifier", id, applyState.getIdentifier());
         ReplicatedLogEntry replicatedLogEntry = applyState.getReplicatedLogEntry();
         verifyReplicatedLogEntry(replicatedLogEntry, expTerm, expIndex, payload);
     }
@@ -261,7 +348,58 @@ public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest
         assertEquals("ReplicatedLogEntry getData", payload, replicatedLogEntry.getData());
     }
 
-    protected String testActorPath(String id){
-        return "akka://test/user" + id;
+    protected String testActorPath(String id) {
+        return factory.createTestActorPath(id);
+    }
+
+    protected void verifyLeadersTrimmedLog(long lastIndex) {
+        verifyTrimmedLog("Leader", leaderActor, lastIndex, lastIndex - 1);
+    }
+
+    protected void verifyLeadersTrimmedLog(long lastIndex, long replicatedToAllIndex) {
+        verifyTrimmedLog("Leader", leaderActor, lastIndex, replicatedToAllIndex);
+    }
+
+    protected void verifyFollowersTrimmedLog(int num, TestActorRef<TestRaftActor> actorRef, long lastIndex) {
+        verifyTrimmedLog("Follower " + num, actorRef, lastIndex, lastIndex - 1);
+    }
+
+    protected void verifyTrimmedLog(String name, TestActorRef<TestRaftActor> actorRef, long lastIndex,
+            long replicatedToAllIndex) {
+        TestRaftActor actor = actorRef.underlyingActor();
+        RaftActorContext context = actor.getRaftActorContext();
+        long snapshotIndex = lastIndex - 1;
+        assertEquals(name + " snapshot term", snapshotIndex < 0 ? -1 : currentTerm,
+                context.getReplicatedLog().getSnapshotTerm());
+        assertEquals(name + " snapshot index", snapshotIndex, context.getReplicatedLog().getSnapshotIndex());
+        assertEquals(name + " journal log size", 1, context.getReplicatedLog().size());
+        assertEquals(name + " journal last index", lastIndex, context.getReplicatedLog().lastIndex());
+        assertEquals(name + " commit index", lastIndex, context.getCommitIndex());
+        assertEquals(name + " last applied", lastIndex, context.getLastApplied());
+        assertEquals(name + " replicatedToAllIndex", replicatedToAllIndex,
+                actor.getCurrentBehavior().getReplicatedToAllIndex());
+    }
+
+    @SuppressWarnings("checkstyle:IllegalCatch")
+    static void verifyRaftState(ActorRef raftActor, Consumer<OnDemandRaftState> verifier) {
+        Timeout timeout = new Timeout(500, TimeUnit.MILLISECONDS);
+        AssertionError lastError = null;
+        Stopwatch sw = Stopwatch.createStarted();
+        while (sw.elapsed(TimeUnit.SECONDS) <= 5) {
+            try {
+                OnDemandRaftState raftState = (OnDemandRaftState)Await.result(ask(raftActor,
+                        GetOnDemandRaftState.INSTANCE, timeout), timeout.duration());
+                verifier.accept(raftState);
+                return;
+            } catch (AssertionError e) {
+                lastError = e;
+                Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
+            } catch (Exception e) {
+                lastError = new AssertionError("OnDemandRaftState failed", e);
+                Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
+            }
+        }
+
+        throw lastError;
     }
 }