Bug 1726 : Adding test-case to check Install Snapshot functionality is handled correctly
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / test / java / org / opendaylight / controller / cluster / raft / behaviors / FollowerTest.java
index c5a81aa1c9225ea03fa548bf1950d5e73a7e3329..fd4a75a22f735c06d2ab8042eb13d6e92de572a0 100644 (file)
@@ -3,6 +3,8 @@ package org.opendaylight.controller.cluster.raft.behaviors;
 import akka.actor.ActorRef;
 import akka.actor.Props;
 import akka.testkit.JavaTestKit;
+import akka.util.Timeout;
+import com.google.protobuf.ByteString;
 import junit.framework.Assert;
 import org.junit.Test;
 import org.opendaylight.controller.cluster.raft.DefaultConfigParamsImpl;
@@ -10,19 +12,35 @@ import org.opendaylight.controller.cluster.raft.MockRaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftState;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
+import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
+import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
 import org.opendaylight.controller.cluster.raft.utils.DoNothingActor;
-
+import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.Duration;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectOutputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
+import static akka.pattern.Patterns.ask;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 public class FollowerTest extends AbstractRaftActorBehaviorTest {
 
@@ -34,8 +52,12 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
         return new Follower(actorContext);
     }
 
-    @Override protected RaftActorContext createActorContext() {
-        return new MockRaftActorContext("test", getSystem(), followerActor);
+    @Override protected  RaftActorContext createActorContext() {
+        return createActorContext(followerActor);
+    }
+
+    protected  RaftActorContext createActorContext(ActorRef actorRef){
+        return new MockRaftActorContext("test", getSystem(), actorRef);
     }
 
     @Test
@@ -158,17 +180,19 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
                 createActorContext();
 
             context.setLastApplied(100);
-            setLastLogEntry((MockRaftActorContext) context, 0, 0, new MockRaftActorContext.MockPayload(""));
+            setLastLogEntry((MockRaftActorContext) context, 1, 100,
+                new MockRaftActorContext.MockPayload(""));
+            ((MockRaftActorContext) context).getReplicatedLog().setSnapshotIndex(99);
 
             List<ReplicatedLogEntry> entries =
                 Arrays.asList(
-                    (ReplicatedLogEntry) new MockRaftActorContext.MockReplicatedLogEntry(100, 101,
-                        new MockRaftActorContext.MockPayload("foo"))
+                        (ReplicatedLogEntry) new MockRaftActorContext.MockReplicatedLogEntry(2, 101,
+                                new MockRaftActorContext.MockPayload("foo"))
                 );
 
             // The new commitIndex is 101
             AppendEntries appendEntries =
-                new AppendEntries(100, "leader-1", 0, 0, entries, 101);
+                new AppendEntries(2, "leader-1", 100, 1, entries, 101);
 
             RaftState raftState =
                 createBehavior(context).handleMessage(getRef(), appendEntries);
@@ -408,4 +432,148 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
         }};
     }
 
+
+    /**
+     * This test verifies that when InstallSnapshot is received by
+     * the follower its applied correctly.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testHandleInstallSnapshot() throws Exception {
+        JavaTestKit javaTestKit = new JavaTestKit(getSystem()) {{
+
+            ActorRef leaderActor = getSystem().actorOf(Props.create(
+                MessageCollectorActor.class));
+
+            MockRaftActorContext context = (MockRaftActorContext)
+                createActorContext(getRef());
+
+            Follower follower = (Follower)createBehavior(context);
+
+            HashMap<String, String> followerSnapshot = new HashMap<>();
+            followerSnapshot.put("1", "A");
+            followerSnapshot.put("2", "B");
+            followerSnapshot.put("3", "C");
+
+            ByteString bsSnapshot  = toByteString(followerSnapshot);
+            ByteString chunkData = ByteString.EMPTY;
+            int offset = 0;
+            int snapshotLength = bsSnapshot.size();
+            int i = 1;
+
+            do {
+                chunkData = getNextChunk(bsSnapshot, offset);
+                final InstallSnapshot installSnapshot =
+                    new InstallSnapshot(1, "leader-1", i, 1,
+                        chunkData, i, 3);
+                follower.handleMessage(leaderActor, installSnapshot);
+                offset = offset + 50;
+                i++;
+            } while ((offset+50) < snapshotLength);
+
+            final InstallSnapshot installSnapshot3 = new InstallSnapshot(1, "leader-1", 3, 1, chunkData, 3, 3);
+            follower.handleMessage(leaderActor, installSnapshot3);
+
+            String[] matches = new ReceiveWhile<String>(String.class, duration("2 seconds")) {
+                @Override
+                protected String match(Object o) throws Exception {
+                    if (o instanceof ApplySnapshot) {
+                        ApplySnapshot as = (ApplySnapshot)o;
+                        if (as.getSnapshot().getLastIndex() != installSnapshot3.getLastIncludedIndex()) {
+                            return "applySnapshot-lastIndex-mismatch";
+                        }
+                        if (as.getSnapshot().getLastAppliedTerm() != installSnapshot3.getLastIncludedTerm()) {
+                            return "applySnapshot-lastAppliedTerm-mismatch";
+                        }
+                        if (as.getSnapshot().getLastAppliedIndex() != installSnapshot3.getLastIncludedIndex()) {
+                            return "applySnapshot-lastAppliedIndex-mismatch";
+                        }
+                        if (as.getSnapshot().getLastTerm() != installSnapshot3.getLastIncludedTerm()) {
+                            return "applySnapshot-lastTerm-mismatch";
+                        }
+                        return "applySnapshot";
+                    }
+
+                    return "ignoreCase";
+                }
+            }.get();
+
+            String applySnapshotMatch = "";
+            for (String reply: matches) {
+                if (reply.startsWith("applySnapshot")) {
+                    applySnapshotMatch = reply;
+                }
+            }
+
+            assertEquals("applySnapshot", applySnapshotMatch);
+
+            Object messages = executeLocalOperation(leaderActor, "get-all-messages");
+
+            assertNotNull(messages);
+            assertTrue(messages instanceof List);
+            List<Object> listMessages = (List<Object>) messages;
+
+            int installSnapshotReplyReceivedCount = 0;
+            for (Object message: listMessages) {
+                if (message instanceof InstallSnapshotReply) {
+                    ++installSnapshotReplyReceivedCount;
+                }
+            }
+
+            assertEquals(3, installSnapshotReplyReceivedCount);
+
+        }};
+    }
+
+    public Object executeLocalOperation(ActorRef actor, Object message) throws Exception {
+        FiniteDuration operationDuration = Duration.create(5, TimeUnit.SECONDS);
+        Timeout operationTimeout = new Timeout(operationDuration);
+        Future<Object> future = ask(actor, message, operationTimeout);
+
+        try {
+            return Await.result(future, operationDuration);
+        } catch (Exception e) {
+            throw e;
+        }
+    }
+
+    public ByteString getNextChunk (ByteString bs, int offset){
+        int snapshotLength = bs.size();
+        int start = offset;
+        int size = 50;
+        if (50 > snapshotLength) {
+            size = snapshotLength;
+        } else {
+            if ((start + 50) > snapshotLength) {
+                size = snapshotLength - start;
+            }
+        }
+        return bs.substring(start, start + size);
+    }
+
+    private ByteString toByteString(Map<String, String> state) {
+        ByteArrayOutputStream b = null;
+        ObjectOutputStream o = null;
+        try {
+            try {
+                b = new ByteArrayOutputStream();
+                o = new ObjectOutputStream(b);
+                o.writeObject(state);
+                byte[] snapshotBytes = b.toByteArray();
+                return ByteString.copyFrom(snapshotBytes);
+            } finally {
+                if (o != null) {
+                    o.flush();
+                    o.close();
+                }
+                if (b != null) {
+                    b.close();
+                }
+            }
+        } catch (IOException e) {
+            org.junit.Assert.fail("IOException in converting Hashmap to Bytestring:" + e);
+        }
+        return null;
+    }
 }