From 471ac7d59e93db65fd49733d5185bc1996f7ad43 Mon Sep 17 00:00:00 2001 From: tpantelis Date: Tue, 10 Feb 2015 16:08:36 -0500 Subject: [PATCH] Refactor FollowerTest Removed the Within blocks in tests. Changed all the tests to use the leaderActor and followerActor class members for consistency. The actors are killed on tearDown by the TestActorFactory. The creation of a JavaTestKit block is no longer necessary and was removed from each test. Added some more assertions in some tests. Change-Id: Iee793c2b1129b5826dbd3c267c404aae30418fdb Signed-off-by: tpantelis --- .../cluster/raft/behaviors/Follower.java | 8 +- .../cluster/raft/MockRaftActorContext.java | 76 ++ .../AbstractRaftActorBehaviorTest.java | 20 + .../cluster/raft/behaviors/FollowerTest.java | 851 +++++++----------- .../cluster/raft/behaviors/LeaderTest.java | 33 - .../raft/utils/MessageCollectorActor.java | 21 +- .../test/resources/simplelogger.properties | 2 +- 7 files changed, 434 insertions(+), 577 deletions(-) diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java index 1e4fcf7225..c799441d60 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java @@ -10,7 +10,6 @@ package org.opendaylight.controller.cluster.raft.behaviors; import akka.actor.ActorRef; import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.ByteString; import java.util.ArrayList; import org.opendaylight.controller.cluster.raft.RaftActorContext; import org.opendaylight.controller.cluster.raft.RaftState; @@ -330,12 +329,13 @@ public class Follower extends AbstractRaftActorBehavior { } } - @Override public void close() throws Exception { + @Override + public void close() throws Exception { stopElection(); } @VisibleForTesting - ByteString getSnapshotChunksCollected(){ - return snapshotTracker != null ? snapshotTracker.getCollectedChunks() : ByteString.EMPTY; + SnapshotTracker getSnapshotTracker(){ + return snapshotTracker; } } diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/MockRaftActorContext.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/MockRaftActorContext.java index e10fb8d293..24581d6d2a 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/MockRaftActorContext.java +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/MockRaftActorContext.java @@ -256,6 +256,36 @@ public class MockRaftActorContext implements RaftActorContext { public String toString() { return value; } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((value == null) ? 0 : value.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + MockPayload other = (MockPayload) obj; + if (value == null) { + if (other.value != null) { + return false; + } + } else if (!value.equals(other.value)) { + return false; + } + return true; + } } public static class MockReplicatedLogEntry implements ReplicatedLogEntry, Serializable { @@ -288,6 +318,52 @@ public class MockRaftActorContext implements RaftActorContext { public int size() { return getData().size(); } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((data == null) ? 0 : data.hashCode()); + result = prime * result + (int) (index ^ (index >>> 32)); + result = prime * result + (int) (term ^ (term >>> 32)); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + MockReplicatedLogEntry other = (MockReplicatedLogEntry) obj; + if (data == null) { + if (other.data != null) { + return false; + } + } else if (!data.equals(other.data)) { + return false; + } + if (index != other.index) { + return false; + } + if (term != other.term) { + return false; + } + return true; + } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + builder.append("MockReplicatedLogEntry [term=").append(term).append(", index=").append(index) + .append(", data=").append(data).append("]"); + return builder.toString(); + } } public static class MockReplicatedLogBuilder { diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehaviorTest.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehaviorTest.java index f45be0b526..4cfcc3fb9e 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehaviorTest.java +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehaviorTest.java @@ -5,8 +5,13 @@ import static org.junit.Assert.assertTrue; import akka.actor.ActorRef; import akka.actor.Props; import akka.testkit.JavaTestKit; +import com.google.protobuf.ByteString; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectOutputStream; import java.util.ArrayList; import java.util.List; +import java.util.Map; import org.junit.Test; import org.opendaylight.controller.cluster.raft.AbstractActorTest; import org.opendaylight.controller.cluster.raft.MockRaftActorContext; @@ -20,6 +25,7 @@ import org.opendaylight.controller.cluster.raft.messages.RequestVote; import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply; import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload; import org.opendaylight.controller.cluster.raft.utils.DoNothingActor; +import org.slf4j.LoggerFactory; public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest { @@ -415,4 +421,18 @@ public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest { protected Object fromSerializableMessage(Object serializable){ return SerializationUtils.fromSerializable(serializable); } + + protected ByteString toByteString(Map state) { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + try(ObjectOutputStream oos = new ObjectOutputStream(bos)) { + oos.writeObject(state); + return ByteString.copyFrom(bos.toByteArray()); + } catch (IOException e) { + throw new AssertionError("IOException occurred converting Map to Bytestring", e); + } + } + + protected void logStart(String name) { + LoggerFactory.getLogger(LeaderTest.class).info("Starting " + name); + } } diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/FollowerTest.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/FollowerTest.java index 6b0857351d..34cdd5b67e 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/FollowerTest.java +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/FollowerTest.java @@ -1,25 +1,23 @@ package org.opendaylight.controller.cluster.raft.behaviors; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import akka.actor.ActorRef; import akka.actor.Props; -import akka.testkit.JavaTestKit; +import akka.testkit.TestActorRef; import com.google.protobuf.ByteString; -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 org.junit.After; +import org.junit.Assert; import org.junit.Test; -import org.opendaylight.controller.cluster.raft.DefaultConfigParamsImpl; import org.opendaylight.controller.cluster.raft.MockRaftActorContext; import org.opendaylight.controller.cluster.raft.RaftActorContext; import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry; +import org.opendaylight.controller.cluster.raft.Snapshot; +import org.opendaylight.controller.cluster.raft.TestActorFactory; 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; @@ -28,16 +26,31 @@ 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; public class FollowerTest extends AbstractRaftActorBehaviorTest { - private final ActorRef followerActor = getSystem().actorOf(Props.create( - DoNothingActor.class)); + private final TestActorFactory actorFactory = new TestActorFactory(getSystem()); + private final TestActorRef followerActor = actorFactory.createTestActor( + Props.create(MessageCollectorActor.class), actorFactory.generateActorId("follower")); - @Override protected RaftActorBehavior createBehavior(RaftActorContext actorContext) { + private final TestActorRef leaderActor = actorFactory.createTestActor( + Props.create(MessageCollectorActor.class), actorFactory.generateActorId("leader")); + + private RaftActorBehavior follower; + + @After + public void tearDown() throws Exception { + if(follower != null) { + follower.close(); + } + + actorFactory.close(); + } + + @Override + protected RaftActorBehavior createBehavior(RaftActorContext actorContext) { return new Follower(actorContext); } @@ -48,117 +61,62 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest { @Override protected MockRaftActorContext createActorContext(ActorRef actorRef){ - return new MockRaftActorContext("test", getSystem(), actorRef); + return new MockRaftActorContext("follower", getSystem(), actorRef); } @Test public void testThatAnElectionTimeoutIsTriggered(){ - new JavaTestKit(getSystem()) {{ - - new Within(DefaultConfigParamsImpl.HEART_BEAT_INTERVAL.$times(6)) { - @Override - protected void run() { - - Follower follower = new Follower(createActorContext(getTestActor())); - - final Boolean out = new ExpectMsg(DefaultConfigParamsImpl.HEART_BEAT_INTERVAL.$times(6), "ElectionTimeout") { - // do not put code outside this method, will run afterwards - @Override - protected Boolean match(Object in) { - if (in instanceof ElectionTimeout) { - return true; - } else { - throw noMatch(); - } - } - }.get(); - - assertEquals(true, out); - } - }; - }}; + MockRaftActorContext actorContext = createActorContext(); + follower = new Follower(actorContext); + + MessageCollectorActor.expectFirstMatching(followerActor, ElectionTimeout.class, + actorContext.getConfigParams().getElectionTimeOutInterval().$times(6).toMillis()); } @Test public void testHandleElectionTimeout(){ - RaftActorContext raftActorContext = createActorContext(); - Follower follower = - new Follower(raftActorContext); + logStart("testHandleElectionTimeout"); + + follower = new Follower(createActorContext()); - RaftActorBehavior raftBehavior = - follower.handleMessage(followerActor, new ElectionTimeout()); + RaftActorBehavior raftBehavior = follower.handleMessage(followerActor, new ElectionTimeout()); assertTrue(raftBehavior instanceof Candidate); } @Test public void testHandleRequestVoteWhenSenderTermEqualToCurrentTermAndVotedForIsNull(){ - new JavaTestKit(getSystem()) {{ - - new Within(duration("1 seconds")) { - @Override - protected void run() { - - RaftActorContext context = createActorContext(getTestActor()); + logStart("testHandleRequestVoteWhenSenderTermEqualToCurrentTermAndVotedForIsNull"); - context.getTermInformation().update(1000, null); + RaftActorContext context = createActorContext(); + long term = 1000; + context.getTermInformation().update(term, null); - RaftActorBehavior follower = createBehavior(context); + follower = createBehavior(context); - follower.handleMessage(getTestActor(), new RequestVote(1000, "test", 10000, 999)); + follower.handleMessage(leaderActor, new RequestVote(term, "test", 10000, 999)); - final Boolean out = new ExpectMsg(duration("1 seconds"), "RequestVoteReply") { - // do not put code outside this method, will run afterwards - @Override - protected Boolean match(Object in) { - if (in instanceof RequestVoteReply) { - RequestVoteReply reply = (RequestVoteReply) in; - return reply.isVoteGranted(); - } else { - throw noMatch(); - } - } - }.get(); + RequestVoteReply reply = MessageCollectorActor.expectFirstMatching(leaderActor, RequestVoteReply.class); - assertEquals(true, out); - } - }; - }}; + assertEquals("isVoteGranted", true, reply.isVoteGranted()); + assertEquals("getTerm", term, reply.getTerm()); } @Test public void testHandleRequestVoteWhenSenderTermEqualToCurrentTermAndVotedForIsNotTheSameAsCandidateId(){ - new JavaTestKit(getSystem()) {{ + logStart("testHandleRequestVoteWhenSenderTermEqualToCurrentTermAndVotedForIsNotTheSameAsCandidateId"); - new Within(duration("1 seconds")) { - @Override - protected void run() { + RaftActorContext context = createActorContext(); + long term = 1000; + context.getTermInformation().update(term, "test"); - RaftActorContext context = createActorContext(getTestActor()); + follower = createBehavior(context); - context.getTermInformation().update(1000, "test"); + follower.handleMessage(leaderActor, new RequestVote(term, "candidate", 10000, 999)); - RaftActorBehavior follower = createBehavior(context); + RequestVoteReply reply = MessageCollectorActor.expectFirstMatching(leaderActor, RequestVoteReply.class); - follower.handleMessage(getTestActor(), new RequestVote(1000, "candidate", 10000, 999)); - - final Boolean out = new ExpectMsg(duration("1 seconds"), "RequestVoteReply") { - // do not put code outside this method, will run afterwards - @Override - protected Boolean match(Object in) { - if (in instanceof RequestVoteReply) { - RequestVoteReply reply = (RequestVoteReply) in; - return reply.isVoteGranted(); - } else { - throw noMatch(); - } - } - }.get(); - - assertEquals(false, out); - } - }; - }}; + assertEquals("isVoteGranted", false, reply.isVoteGranted()); } /** @@ -171,32 +129,25 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest { */ @Test public void testHandleAppendEntriesWithNewerCommitIndex() throws Exception { - new JavaTestKit(getSystem()) {{ + logStart("testHandleAppendEntriesWithNewerCommitIndex"); - RaftActorContext context = - createActorContext(); + MockRaftActorContext context = createActorContext(); - context.setLastApplied(100); - setLastLogEntry((MockRaftActorContext) context, 1, 100, + context.setLastApplied(100); + setLastLogEntry(context, 1, 100, new MockRaftActorContext.MockPayload("")); - ((MockRaftActorContext) context).getReplicatedLog().setSnapshotIndex(99); - - List entries = - Arrays.asList( - (ReplicatedLogEntry) new MockRaftActorContext.MockReplicatedLogEntry(2, 101, - new MockRaftActorContext.MockPayload("foo")) - ); + context.getReplicatedLog().setSnapshotIndex(99); - // The new commitIndex is 101 - AppendEntries appendEntries = - new AppendEntries(2, "leader-1", 100, 1, entries, 101, 100); + List entries = Arrays.asList( + newReplicatedLogEntry(2, 101, "foo")); - RaftActorBehavior raftBehavior = - createBehavior(context).handleMessage(getRef(), appendEntries); + // The new commitIndex is 101 + AppendEntries appendEntries = new AppendEntries(2, "leader-1", 100, 1, entries, 101, 100); - assertEquals(101L, context.getLastApplied()); + follower = createBehavior(context); + follower.handleMessage(leaderActor, appendEntries); - }}; + assertEquals("getLastApplied", 101L, context.getLastApplied()); } /** @@ -207,58 +158,30 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest { * @throws Exception */ @Test - public void testHandleAppendEntriesSenderPrevLogTermNotSameAsReceiverPrevLogTerm() - throws Exception { - new JavaTestKit(getSystem()) {{ - - MockRaftActorContext context = createActorContext(); - - // First set the receivers term to lower number - context.getTermInformation().update(95, "test"); + public void testHandleAppendEntriesSenderPrevLogTermNotSameAsReceiverPrevLogTerm() { + logStart("testHandleAppendEntriesSenderPrevLogTermNotSameAsReceiverPrevLogTerm"); - // Set the last log entry term for the receiver to be greater than - // what we will be sending as the prevLogTerm in AppendEntries - MockRaftActorContext.SimpleReplicatedLog mockReplicatedLog = - setLastLogEntry(context, 20, 0, new MockRaftActorContext.MockPayload("")); + MockRaftActorContext context = createActorContext(); - // AppendEntries is now sent with a bigger term - // this will set the receivers term to be the same as the sender's term - AppendEntries appendEntries = - new AppendEntries(100, "leader-1", 0, 0, null, 101, -1); + // First set the receivers term to lower number + context.getTermInformation().update(95, "test"); - RaftActorBehavior behavior = createBehavior(context); + // AppendEntries is now sent with a bigger term + // this will set the receivers term to be the same as the sender's term + AppendEntries appendEntries = new AppendEntries(100, "leader", 0, 0, null, 101, -1); - // Send an unknown message so that the state of the RaftActor remains unchanged - RaftActorBehavior expected = behavior.handleMessage(getRef(), "unknown"); + follower = createBehavior(context); - RaftActorBehavior raftBehavior = - behavior.handleMessage(getRef(), appendEntries); + RaftActorBehavior newBehavior = follower.handleMessage(leaderActor, appendEntries); - assertEquals(expected, raftBehavior); + Assert.assertSame(follower, newBehavior); - // Also expect an AppendEntriesReply to be sent where success is false - final Boolean out = new ExpectMsg(duration("1 seconds"), - "AppendEntriesReply") { - // do not put code outside this method, will run afterwards - @Override - protected Boolean match(Object in) { - if (in instanceof AppendEntriesReply) { - AppendEntriesReply reply = (AppendEntriesReply) in; - return reply.isSuccess(); - } else { - throw noMatch(); - } - } - }.get(); + AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(leaderActor, + AppendEntriesReply.class); - assertEquals(false, out); - - - }}; + assertEquals("isSuccess", false, reply.isSuccess()); } - - /** * This test verifies that when a new AppendEntries message is received with * new entries and the logs of the sender and receiver match that the new @@ -268,278 +191,201 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest { * @throws Exception */ @Test - public void testHandleAppendEntriesAddNewEntries() throws Exception { - new JavaTestKit(getSystem()) {{ - - MockRaftActorContext context = createActorContext(); - - // First set the receivers term to lower number - context.getTermInformation().update(1, "test"); - - // Prepare the receivers log - MockRaftActorContext.SimpleReplicatedLog log = - new MockRaftActorContext.SimpleReplicatedLog(); - log.append( - new MockRaftActorContext.MockReplicatedLogEntry(1, 0, new MockRaftActorContext.MockPayload("zero"))); - log.append( - new MockRaftActorContext.MockReplicatedLogEntry(1, 1, new MockRaftActorContext.MockPayload("one"))); - log.append( - new MockRaftActorContext.MockReplicatedLogEntry(1, 2, new MockRaftActorContext.MockPayload("two"))); - - context.setReplicatedLog(log); - - // Prepare the entries to be sent with AppendEntries - List entries = new ArrayList<>(); - entries.add( - new MockRaftActorContext.MockReplicatedLogEntry(1, 3, new MockRaftActorContext.MockPayload("three"))); - entries.add( - new MockRaftActorContext.MockReplicatedLogEntry(1, 4, new MockRaftActorContext.MockPayload("four"))); - - // Send appendEntries with the same term as was set on the receiver - // before the new behavior was created (1 in this case) - // This will not work for a Candidate because as soon as a Candidate - // is created it increments the term - AppendEntries appendEntries = - new AppendEntries(1, "leader-1", 2, 1, entries, 4, -1); - - RaftActorBehavior behavior = createBehavior(context); - - // Send an unknown message so that the state of the RaftActor remains unchanged - RaftActorBehavior expected = behavior.handleMessage(getRef(), "unknown"); - - RaftActorBehavior raftBehavior = - behavior.handleMessage(getRef(), appendEntries); - - assertEquals(expected, raftBehavior); - assertEquals(5, log.last().getIndex() + 1); - assertNotNull(log.get(3)); - assertNotNull(log.get(4)); - - // Also expect an AppendEntriesReply to be sent where success is false - final Boolean out = new ExpectMsg(duration("1 seconds"), - "AppendEntriesReply") { - // do not put code outside this method, will run afterwards - @Override - protected Boolean match(Object in) { - if (in instanceof AppendEntriesReply) { - AppendEntriesReply reply = (AppendEntriesReply) in; - return reply.isSuccess(); - } else { - throw noMatch(); - } - } - }.get(); - - assertEquals(true, out); - - - }}; - } + public void testHandleAppendEntriesAddNewEntries() { + logStart("testHandleAppendEntriesAddNewEntries"); + MockRaftActorContext context = createActorContext(); + // First set the receivers term to lower number + context.getTermInformation().update(1, "test"); + + // Prepare the receivers log + MockRaftActorContext.SimpleReplicatedLog log = new MockRaftActorContext.SimpleReplicatedLog(); + log.append(newReplicatedLogEntry(1, 0, "zero")); + log.append(newReplicatedLogEntry(1, 1, "one")); + log.append(newReplicatedLogEntry(1, 2, "two")); + + context.setReplicatedLog(log); + + // Prepare the entries to be sent with AppendEntries + List entries = new ArrayList<>(); + entries.add(newReplicatedLogEntry(1, 3, "three")); + entries.add(newReplicatedLogEntry(1, 4, "four")); + + // Send appendEntries with the same term as was set on the receiver + // before the new behavior was created (1 in this case) + // This will not work for a Candidate because as soon as a Candidate + // is created it increments the term + AppendEntries appendEntries = new AppendEntries(1, "leader-1", 2, 1, entries, 4, -1); + + follower = createBehavior(context); + + RaftActorBehavior newBehavior = follower.handleMessage(leaderActor, appendEntries); + + Assert.assertSame(follower, newBehavior); + + assertEquals("Next index", 5, log.last().getIndex() + 1); + assertEquals("Entry 3", entries.get(0), log.get(3)); + assertEquals("Entry 4", entries.get(1), log.get(4)); + + expectAndVerifyAppendEntriesReply(1, true, context.getId(), 1, 4); + } /** * This test verifies that when a new AppendEntries message is received with * new entries and the logs of the sender and receiver are out-of-sync that * the log is first corrected by removing the out of sync entries from the * log and then adding in the new entries sent with the AppendEntries message - * - * @throws Exception */ @Test - public void testHandleAppendEntriesCorrectReceiverLogEntries() - throws Exception { - new JavaTestKit(getSystem()) {{ - - MockRaftActorContext context = createActorContext(); - - // First set the receivers term to lower number - context.getTermInformation().update(2, "test"); - - // Prepare the receivers log - MockRaftActorContext.SimpleReplicatedLog log = - new MockRaftActorContext.SimpleReplicatedLog(); - log.append( - new MockRaftActorContext.MockReplicatedLogEntry(1, 0, new MockRaftActorContext.MockPayload("zero"))); - log.append( - new MockRaftActorContext.MockReplicatedLogEntry(1, 1, new MockRaftActorContext.MockPayload("one"))); - log.append( - new MockRaftActorContext.MockReplicatedLogEntry(1, 2, new MockRaftActorContext.MockPayload("two"))); - - context.setReplicatedLog(log); - - // Prepare the entries to be sent with AppendEntries - List entries = new ArrayList<>(); - entries.add( - new MockRaftActorContext.MockReplicatedLogEntry(2, 2, new MockRaftActorContext.MockPayload("two-1"))); - entries.add( - new MockRaftActorContext.MockReplicatedLogEntry(2, 3, new MockRaftActorContext.MockPayload("three"))); - - // Send appendEntries with the same term as was set on the receiver - // before the new behavior was created (1 in this case) - // This will not work for a Candidate because as soon as a Candidate - // is created it increments the term - AppendEntries appendEntries = - new AppendEntries(2, "leader-1", 1, 1, entries, 3, -1); - - RaftActorBehavior behavior = createBehavior(context); - - // Send an unknown message so that the state of the RaftActor remains unchanged - RaftActorBehavior expected = behavior.handleMessage(getRef(), "unknown"); - - RaftActorBehavior raftBehavior = - behavior.handleMessage(getRef(), appendEntries); - - assertEquals(expected, raftBehavior); - - // The entry at index 2 will be found out-of-sync with the leader - // and will be removed - // Then the two new entries will be added to the log - // Thus making the log to have 4 entries - assertEquals(4, log.last().getIndex() + 1); - assertNotNull(log.get(2)); - - assertEquals("one", log.get(1).getData().toString()); - - // Check that the entry at index 2 has the new data - assertEquals("two-1", log.get(2).getData().toString()); - - assertEquals("three", log.get(3).getData().toString()); - - assertNotNull(log.get(3)); - - // Also expect an AppendEntriesReply to be sent where success is false - final Boolean out = new ExpectMsg(duration("1 seconds"), - "AppendEntriesReply") { - // do not put code outside this method, will run afterwards - @Override - protected Boolean match(Object in) { - if (in instanceof AppendEntriesReply) { - AppendEntriesReply reply = (AppendEntriesReply) in; - return reply.isSuccess(); - } else { - throw noMatch(); - } - } - }.get(); - - assertEquals(true, out); - - - }}; + public void testHandleAppendEntriesCorrectReceiverLogEntries() { + logStart("testHandleAppendEntriesCorrectReceiverLogEntries"); + + MockRaftActorContext context = createActorContext(); + + // First set the receivers term to lower number + context.getTermInformation().update(1, "test"); + + // Prepare the receivers log + MockRaftActorContext.SimpleReplicatedLog log = new MockRaftActorContext.SimpleReplicatedLog(); + log.append(newReplicatedLogEntry(1, 0, "zero")); + log.append(newReplicatedLogEntry(1, 1, "one")); + log.append(newReplicatedLogEntry(1, 2, "two")); + + context.setReplicatedLog(log); + + // Prepare the entries to be sent with AppendEntries + List entries = new ArrayList<>(); + entries.add(newReplicatedLogEntry(2, 2, "two-1")); + entries.add(newReplicatedLogEntry(2, 3, "three")); + + // Send appendEntries with the same term as was set on the receiver + // before the new behavior was created (1 in this case) + // This will not work for a Candidate because as soon as a Candidate + // is created it increments the term + AppendEntries appendEntries = new AppendEntries(2, "leader", 1, 1, entries, 3, -1); + + follower = createBehavior(context); + + RaftActorBehavior newBehavior = follower.handleMessage(leaderActor, appendEntries); + + Assert.assertSame(follower, newBehavior); + + // The entry at index 2 will be found out-of-sync with the leader + // and will be removed + // Then the two new entries will be added to the log + // Thus making the log to have 4 entries + assertEquals("Next index", 4, log.last().getIndex() + 1); + //assertEquals("Entry 2", entries.get(0), log.get(2)); + + assertEquals("Entry 1 data", "one", log.get(1).getData().toString()); + + // Check that the entry at index 2 has the new data + assertEquals("Entry 2", entries.get(0), log.get(2)); + + assertEquals("Entry 3", entries.get(1), log.get(3)); + + expectAndVerifyAppendEntriesReply(2, true, context.getId(), 2, 3); } @Test public void testHandleAppendEntriesPreviousLogEntryMissing(){ - new JavaTestKit(getSystem()) {{ + logStart("testHandleAppendEntriesPreviousLogEntryMissing"); - MockRaftActorContext context = createActorContext(); + MockRaftActorContext context = createActorContext(); - // Prepare the receivers log - MockRaftActorContext.SimpleReplicatedLog log = - new MockRaftActorContext.SimpleReplicatedLog(); - log.append( - new MockRaftActorContext.MockReplicatedLogEntry(1, 0, new MockRaftActorContext.MockPayload("zero"))); - log.append( - new MockRaftActorContext.MockReplicatedLogEntry(1, 1, new MockRaftActorContext.MockPayload("one"))); - log.append( - new MockRaftActorContext.MockReplicatedLogEntry(1, 2, new MockRaftActorContext.MockPayload("two"))); + // Prepare the receivers log + MockRaftActorContext.SimpleReplicatedLog log = new MockRaftActorContext.SimpleReplicatedLog(); + log.append(newReplicatedLogEntry(1, 0, "zero")); + log.append(newReplicatedLogEntry(1, 1, "one")); + log.append(newReplicatedLogEntry(1, 2, "two")); - context.setReplicatedLog(log); + context.setReplicatedLog(log); - // Prepare the entries to be sent with AppendEntries - List entries = new ArrayList<>(); - entries.add( - new MockRaftActorContext.MockReplicatedLogEntry(1, 4, new MockRaftActorContext.MockPayload("two-1"))); + // Prepare the entries to be sent with AppendEntries + List entries = new ArrayList<>(); + entries.add(newReplicatedLogEntry(1, 4, "four")); - AppendEntries appendEntries = - new AppendEntries(1, "leader-1", 3, 1, entries, 4, -1); + AppendEntries appendEntries = new AppendEntries(1, "leader", 3, 1, entries, 4, -1); - RaftActorBehavior behavior = createBehavior(context); + follower = createBehavior(context); - // Send an unknown message so that the state of the RaftActor remains unchanged - RaftActorBehavior expected = behavior.handleMessage(getRef(), "unknown"); + RaftActorBehavior newBehavior = follower.handleMessage(leaderActor, appendEntries); - RaftActorBehavior raftBehavior = - behavior.handleMessage(getRef(), appendEntries); + Assert.assertSame(follower, newBehavior); - assertEquals(expected, raftBehavior); + expectAndVerifyAppendEntriesReply(1, false, context.getId(), 1, 2); + } - // Also expect an AppendEntriesReply to be sent where success is false - final Boolean out = new ExpectMsg(duration("1 seconds"), - "AppendEntriesReply") { - // do not put code outside this method, will run afterwards - @Override - protected Boolean match(Object in) { - if (in instanceof AppendEntriesReply) { - AppendEntriesReply reply = (AppendEntriesReply) in; - return reply.isSuccess(); - } else { - throw noMatch(); - } - } - }.get(); + @Test + public void testHandleAppendEntriesWithExistingLogEntry() { + logStart("testHandleAppendEntriesWithExistingLogEntry"); - assertEquals(false, out); + MockRaftActorContext context = createActorContext(); - }}; + context.getTermInformation().update(1, "test"); - } + // Prepare the receivers log + MockRaftActorContext.SimpleReplicatedLog log = new MockRaftActorContext.SimpleReplicatedLog(); + log.append(newReplicatedLogEntry(1, 0, "zero")); + log.append(newReplicatedLogEntry(1, 1, "one")); - @Test - public void testHandleAppendAfterInstallingSnapshot(){ - new JavaTestKit(getSystem()) {{ + context.setReplicatedLog(log); + + // Send the last entry again. + List entries = Arrays.asList(newReplicatedLogEntry(1, 1, "one")); + + follower = createBehavior(context); + + follower.handleMessage(leaderActor, new AppendEntries(1, "leader", 0, 1, entries, 1, -1)); + + assertEquals("Next index", 2, log.last().getIndex() + 1); + assertEquals("Entry 1", entries.get(0), log.get(1)); - MockRaftActorContext context = createActorContext(); + expectAndVerifyAppendEntriesReply(1, true, context.getId(), 1, 1); + // Send the last entry again and also a new one. - // Prepare the receivers log - MockRaftActorContext.SimpleReplicatedLog log = - new MockRaftActorContext.SimpleReplicatedLog(); + entries = Arrays.asList(newReplicatedLogEntry(1, 1, "one"), newReplicatedLogEntry(1, 2, "two")); - // Set up a log as if it has been snapshotted - log.setSnapshotIndex(3); - log.setSnapshotTerm(1); + leaderActor.underlyingActor().clear(); + follower.handleMessage(leaderActor, new AppendEntries(1, "leader", 0, 1, entries, 2, -1)); - context.setReplicatedLog(log); + assertEquals("Next index", 3, log.last().getIndex() + 1); + assertEquals("Entry 1", entries.get(0), log.get(1)); + assertEquals("Entry 2", entries.get(1), log.get(2)); + + expectAndVerifyAppendEntriesReply(1, true, context.getId(), 1, 2); + } + + @Test + public void testHandleAppendAfterInstallingSnapshot(){ + logStart("testHandleAppendAfterInstallingSnapshot"); - // Prepare the entries to be sent with AppendEntries - List entries = new ArrayList<>(); - entries.add( - new MockRaftActorContext.MockReplicatedLogEntry(1, 4, new MockRaftActorContext.MockPayload("two-1"))); + MockRaftActorContext context = createActorContext(); - AppendEntries appendEntries = - new AppendEntries(1, "leader-1", 3, 1, entries, 4, 3); + // Prepare the receivers log + MockRaftActorContext.SimpleReplicatedLog log = new MockRaftActorContext.SimpleReplicatedLog(); - RaftActorBehavior behavior = createBehavior(context); + // Set up a log as if it has been snapshotted + log.setSnapshotIndex(3); + log.setSnapshotTerm(1); - // Send an unknown message so that the state of the RaftActor remains unchanged - RaftActorBehavior expected = behavior.handleMessage(getRef(), "unknown"); + context.setReplicatedLog(log); - RaftActorBehavior raftBehavior = - behavior.handleMessage(getRef(), appendEntries); + // Prepare the entries to be sent with AppendEntries + List entries = new ArrayList<>(); + entries.add(newReplicatedLogEntry(1, 4, "four")); - assertEquals(expected, raftBehavior); + AppendEntries appendEntries = new AppendEntries(1, "leader", 3, 1, entries, 4, 3); - // Also expect an AppendEntriesReply to be sent where success is false - final Boolean out = new ExpectMsg(duration("1 seconds"), - "AppendEntriesReply") { - // do not put code outside this method, will run afterwards - @Override - protected Boolean match(Object in) { - if (in instanceof AppendEntriesReply) { - AppendEntriesReply reply = (AppendEntriesReply) in; - return reply.isSuccess(); - } else { - throw noMatch(); - } - } - }.get(); + follower = createBehavior(context); - assertEquals(true, out); + RaftActorBehavior newBehavior = follower.handleMessage(leaderActor, appendEntries); - }}; + Assert.assertSame(follower, newBehavior); + expectAndVerifyAppendEntriesReply(1, true, context.getId(), 1, 4); } @@ -551,180 +397,121 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest { */ @Test public void testHandleInstallSnapshot() throws Exception { - JavaTestKit javaTestKit = new JavaTestKit(getSystem()) {{ - - ActorRef leaderActor = getSystem().actorOf(Props.create( - MessageCollectorActor.class)); - - MockRaftActorContext context = createActorContext(getRef()); - - Follower follower = (Follower)createBehavior(context); - - HashMap 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; - int chunkIndex = 1; - - do { - chunkData = getNextChunk(bsSnapshot, offset); - final InstallSnapshot installSnapshot = - new InstallSnapshot(1, "leader-1", i, 1, - chunkData, chunkIndex, 3); - follower.handleMessage(leaderActor, installSnapshot); - offset = offset + 50; - i++; - chunkIndex++; - } while ((offset+50) < snapshotLength); - - final InstallSnapshot installSnapshot3 = new InstallSnapshot(1, "leader-1", 3, 1, chunkData, chunkIndex, 3); - follower.handleMessage(leaderActor, installSnapshot3); - - String[] matches = new ReceiveWhile(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(); - - // Verify that after a snapshot is successfully applied the collected snapshot chunks is reset to empty - assertEquals(ByteString.EMPTY, follower.getSnapshotChunksCollected()); - - 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 listMessages = (List) messages; - - int installSnapshotReplyReceivedCount = 0; - for (Object message: listMessages) { - if (message instanceof InstallSnapshotReply) { - ++installSnapshotReplyReceivedCount; - } - } + logStart("testHandleInstallSnapshot"); + + MockRaftActorContext context = createActorContext(); + + follower = createBehavior(context); + + HashMap followerSnapshot = new HashMap<>(); + followerSnapshot.put("1", "A"); + followerSnapshot.put("2", "B"); + followerSnapshot.put("3", "C"); + + ByteString bsSnapshot = toByteString(followerSnapshot); + int offset = 0; + int snapshotLength = bsSnapshot.size(); + int chunkSize = 50; + int totalChunks = (snapshotLength / chunkSize) + ((snapshotLength % chunkSize) > 0 ? 1 : 0); + int lastIncludedIndex = 1; + int chunkIndex = 1; + InstallSnapshot lastInstallSnapshot = null; + + for(int i = 0; i < totalChunks; i++) { + ByteString chunkData = getNextChunk(bsSnapshot, offset, chunkSize); + lastInstallSnapshot = new InstallSnapshot(1, "leader", lastIncludedIndex, 1, + chunkData, chunkIndex, totalChunks); + follower.handleMessage(leaderActor, lastInstallSnapshot); + offset = offset + 50; + lastIncludedIndex++; + chunkIndex++; + } - assertEquals(3, installSnapshotReplyReceivedCount); + ApplySnapshot applySnapshot = MessageCollectorActor.expectFirstMatching(followerActor, + ApplySnapshot.class); + Snapshot snapshot = applySnapshot.getSnapshot(); + assertEquals("getLastIndex", lastInstallSnapshot.getLastIncludedIndex(), snapshot.getLastIndex()); + assertEquals("getLastIncludedTerm", lastInstallSnapshot.getLastIncludedTerm(), + snapshot.getLastAppliedTerm()); + assertEquals("getLastAppliedIndex", lastInstallSnapshot.getLastIncludedIndex(), + snapshot.getLastAppliedIndex()); + assertEquals("getLastTerm", lastInstallSnapshot.getLastIncludedTerm(), snapshot.getLastTerm()); + Assert.assertArrayEquals("getState", bsSnapshot.toByteArray(), snapshot.getState()); + + List replies = MessageCollectorActor.getAllMatching( + leaderActor, InstallSnapshotReply.class); + assertEquals("InstallSnapshotReply count", totalChunks, replies.size()); + + chunkIndex = 1; + for(InstallSnapshotReply reply: replies) { + assertEquals("getChunkIndex", chunkIndex++, reply.getChunkIndex()); + assertEquals("getTerm", 1, reply.getTerm()); + assertEquals("isSuccess", true, reply.isSuccess()); + assertEquals("getFollowerId", context.getId(), reply.getFollowerId()); + } - }}; + Assert.assertNull("Expected null SnapshotTracker", ((Follower)follower).getSnapshotTracker()); } @Test public void testHandleOutOfSequenceInstallSnapshot() throws Exception { - JavaTestKit javaTestKit = new JavaTestKit(getSystem()) { - { - - ActorRef leaderActor = getSystem().actorOf(Props.create( - MessageCollectorActor.class)); - - MockRaftActorContext context = createActorContext(getRef()); + logStart("testHandleOutOfSequenceInstallSnapshot"); - Follower follower = (Follower) createBehavior(context); + MockRaftActorContext context = createActorContext(); - HashMap followerSnapshot = new HashMap<>(); - followerSnapshot.put("1", "A"); - followerSnapshot.put("2", "B"); - followerSnapshot.put("3", "C"); + follower = createBehavior(context); - ByteString bsSnapshot = toByteString(followerSnapshot); + HashMap followerSnapshot = new HashMap<>(); + followerSnapshot.put("1", "A"); + followerSnapshot.put("2", "B"); + followerSnapshot.put("3", "C"); - final InstallSnapshot installSnapshot = new InstallSnapshot(1, "leader-1", 3, 1, getNextChunk(bsSnapshot, 10), 3, 3); - follower.handleMessage(leaderActor, installSnapshot); + ByteString bsSnapshot = toByteString(followerSnapshot); - Object messages = executeLocalOperation(leaderActor, "get-all-messages"); + InstallSnapshot installSnapshot = new InstallSnapshot(1, "leader", 3, 1, + getNextChunk(bsSnapshot, 10, 50), 3, 3); + follower.handleMessage(leaderActor, installSnapshot); - assertNotNull(messages); - assertTrue(messages instanceof List); - List listMessages = (List) messages; + InstallSnapshotReply reply = MessageCollectorActor.expectFirstMatching(leaderActor, + InstallSnapshotReply.class); - int installSnapshotReplyReceivedCount = 0; - for (Object message: listMessages) { - if (message instanceof InstallSnapshotReply) { - ++installSnapshotReplyReceivedCount; - } - } + assertEquals("isSuccess", false, reply.isSuccess()); + assertEquals("getChunkIndex", -1, reply.getChunkIndex()); + assertEquals("getTerm", 1, reply.getTerm()); + assertEquals("getFollowerId", context.getId(), reply.getFollowerId()); - assertEquals(1, installSnapshotReplyReceivedCount); - InstallSnapshotReply reply = (InstallSnapshotReply) listMessages.get(0); - assertEquals(false, reply.isSuccess()); - assertEquals(-1, reply.getChunkIndex()); - assertEquals(ByteString.EMPTY, follower.getSnapshotChunksCollected()); - - - }}; + Assert.assertNull("Expected null SnapshotTracker", ((Follower)follower).getSnapshotTracker()); } - public Object executeLocalOperation(ActorRef actor, Object message) throws Exception { - return MessageCollectorActor.getAllMessages(actor); - } - - public ByteString getNextChunk (ByteString bs, int offset){ + public ByteString getNextChunk (ByteString bs, int offset, int chunkSize){ int snapshotLength = bs.size(); int start = offset; - int size = 50; - if (50 > snapshotLength) { + int size = chunkSize; + if (chunkSize > snapshotLength) { size = snapshotLength; } else { - if ((start + 50) > snapshotLength) { + if ((start + chunkSize) > snapshotLength) { size = snapshotLength - start; } } return bs.substring(start, start + size); } - private ByteString toByteString(Map 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; + private void expectAndVerifyAppendEntriesReply(int expTerm, boolean expSuccess, + String expFollowerId, long expLogLastTerm, long expLogLastIndex) { + + AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(leaderActor, + AppendEntriesReply.class); + + assertEquals("isSuccess", expSuccess, reply.isSuccess()); + assertEquals("getTerm", expTerm, reply.getTerm()); + assertEquals("getFollowerId", expFollowerId, reply.getFollowerId()); + assertEquals("getLogLastTerm", expLogLastTerm, reply.getLogLastTerm()); + assertEquals("getLogLastIndex", expLogLastIndex, reply.getLogLastIndex()); + } + + private ReplicatedLogEntry newReplicatedLogEntry(long term, long index, String data) { + return new MockRaftActorContext.MockReplicatedLogEntry(term, index, + new MockRaftActorContext.MockPayload(data)); } } diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java index f087793674..7c37cbc502 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java @@ -13,9 +13,6 @@ 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.IOException; -import java.io.ObjectOutputStream; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -46,7 +43,6 @@ import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply; import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply; import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor; import org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages; -import org.slf4j.LoggerFactory; import scala.concurrent.duration.FiniteDuration; public class LeaderTest extends AbstractRaftActorBehaviorTest { @@ -72,10 +68,6 @@ public class LeaderTest extends AbstractRaftActorBehaviorTest { actorFactory.close(); } - private void logStart(String name) { - LoggerFactory.getLogger(LeaderTest.class).info("Starting " + name); - } - @Test public void testHandleMessageForUnknownMessage() throws Exception { logStart("testHandleMessageForUnknownMessage"); @@ -766,31 +758,6 @@ public class LeaderTest extends AbstractRaftActorBehaviorTest { return context; } - private ByteString toByteString(Map 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) { - Assert.fail("IOException in converting Hashmap to Bytestring:" + e); - } - return null; - } - public static class ForwardMessageToBehaviorActor extends MessageCollectorActor { AbstractRaftActorBehavior behavior; diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/utils/MessageCollectorActor.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/utils/MessageCollectorActor.java index 448c28e8c8..662c12180d 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/utils/MessageCollectorActor.java +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/utils/MessageCollectorActor.java @@ -75,14 +75,21 @@ public class MessageCollectorActor extends UntypedActor { return null; } - public static T expectFirstMatching(ActorRef actor, Class clazz) throws Exception { - for(int i = 0; i < 50; i++) { - T message = getFirstMatching(actor, clazz); - if(message != null) { - return message; - } + public static T expectFirstMatching(ActorRef actor, Class clazz) { + return expectFirstMatching(actor, clazz, 5000); + } + + public static T expectFirstMatching(ActorRef actor, Class clazz, long timeout) { + int count = (int) (timeout / 50); + for(int i = 0; i < count; i++) { + try { + T message = getFirstMatching(actor, clazz); + if(message != null) { + return message; + } + } catch (Exception e) {} - Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS); + Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS); } Assert.fail("Did not receive message of type " + clazz); diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/resources/simplelogger.properties b/opendaylight/md-sal/sal-akka-raft/src/test/resources/simplelogger.properties index 4e798073f6..853fc755d2 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/resources/simplelogger.properties +++ b/opendaylight/md-sal/sal-akka-raft/src/test/resources/simplelogger.properties @@ -3,4 +3,4 @@ org.slf4j.simpleLogger.dateTimeFormat=hh:mm:ss,S a org.slf4j.simpleLogger.logFile=System.out org.slf4j.simpleLogger.showShortLogName=true org.slf4j.simpleLogger.levelInBrackets=true -org.slf4j.simpleLogger.org.opendaylight.controller.cluster.raft=trace \ No newline at end of file +org.slf4j.simpleLogger.log.org.opendaylight.controller.cluster.raft=trace \ No newline at end of file -- 2.36.6