a47cf118963e1698c68ef2c2706a44c2baaeeb59
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / test / java / org / opendaylight / controller / cluster / raft / AbstractRaftActorIntegrationTest.java
1 /*
2  * Copyright (c) 2015 Brocade Communications Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.controller.cluster.raft;
9
10 import static org.junit.Assert.assertEquals;
11 import static org.junit.Assert.assertNotNull;
12 import akka.actor.ActorRef;
13 import akka.actor.InvalidActorNameException;
14 import akka.actor.PoisonPill;
15 import akka.actor.Props;
16 import akka.actor.Terminated;
17 import akka.dispatch.Dispatchers;
18 import akka.testkit.JavaTestKit;
19 import akka.testkit.TestActorRef;
20 import com.google.common.base.Optional;
21 import com.google.common.base.Predicate;
22 import com.google.common.base.Supplier;
23 import com.google.common.collect.ImmutableMap;
24 import com.google.common.util.concurrent.Uninterruptibles;
25 import java.util.ArrayList;
26 import java.util.Collections;
27 import java.util.List;
28 import java.util.Map;
29 import java.util.concurrent.ConcurrentHashMap;
30 import java.util.concurrent.TimeUnit;
31 import org.junit.After;
32 import org.opendaylight.controller.cluster.raft.MockRaftActorContext.MockPayload;
33 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
34 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
35 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
36 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
37 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
38 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
39 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
40 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
41 import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor;
42 import org.slf4j.Logger;
43 import org.slf4j.LoggerFactory;
44 import scala.concurrent.duration.FiniteDuration;
45
46 /**
47  * Abstract base for an integration test that tests end-to-end RaftActor and behavior functionality.
48  *
49  * @author Thomas Pantelis
50  */
51 public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest {
52
53     public static class TestRaftActor extends MockRaftActor {
54
55         private final TestActorRef<MessageCollectorActor> collectorActor;
56         private final Map<Class<?>, Boolean> dropMessages = new ConcurrentHashMap<>();
57
58         private TestRaftActor(String id, Map<String, String> peerAddresses, ConfigParams config,
59                 TestActorRef<MessageCollectorActor> collectorActor) {
60             super(id, peerAddresses, Optional.of(config), null);
61             this.collectorActor = collectorActor;
62         }
63
64         public static Props props(String id, Map<String, String> peerAddresses, ConfigParams config,
65                 TestActorRef<MessageCollectorActor> collectorActor) {
66             return Props.create(TestRaftActor.class, id, peerAddresses, config, collectorActor).
67                     withDispatcher(Dispatchers.DefaultDispatcherId());
68         }
69
70         void startDropMessages(Class<?> msgClass) {
71             dropMessages.put(msgClass, Boolean.TRUE);
72         }
73
74         void stopDropMessages(Class<?> msgClass) {
75             dropMessages.remove(msgClass);
76         }
77
78         void setMockTotalMemory(final long mockTotalMemory) {
79             if(mockTotalMemory > 0) {
80                 getRaftActorContext().setTotalMemoryRetriever(new Supplier<Long>() {
81                     @Override
82                     public Long get() {
83                         return mockTotalMemory;
84                     }
85
86                 });
87             } else {
88                 getRaftActorContext().setTotalMemoryRetriever(null);
89             }
90         }
91
92         @Override
93         public void handleCommand(Object message) {
94             if(message instanceof MockPayload) {
95                 MockPayload payload = (MockPayload)message;
96                 super.persistData(collectorActor, payload.toString(), payload);
97                 return;
98             }
99
100             try {
101                 if(!dropMessages.containsKey(message.getClass())) {
102                     super.handleCommand(message);
103                 }
104             } finally {
105                 if(!(message instanceof SendHeartBeat)) {
106                     try {
107                         collectorActor.tell(message, ActorRef.noSender());
108                     } catch (Exception e) {
109                         LOG.error("MessageCollectorActor error", e);
110                     }
111                 }
112             }
113         }
114
115         @Override
116         public void createSnapshot(ActorRef actorRef) {
117             try {
118                 actorRef.tell(new CaptureSnapshotReply(RaftActorTest.fromObject(getState()).toByteArray()), actorRef);
119             } catch (Exception e) {
120                 e.printStackTrace();
121             }
122         }
123
124         public ActorRef collectorActor() {
125             return collectorActor;
126         }
127     }
128
129     protected final Logger testLog = LoggerFactory.getLogger(getClass());
130
131     protected final TestActorFactory factory = new TestActorFactory(getSystem());
132
133     protected String leaderId = factory.generateActorId("leader");
134     protected DefaultConfigParamsImpl leaderConfigParams;
135     protected TestActorRef<TestRaftActor> leaderActor;
136     protected ActorRef leaderCollectorActor;
137     protected RaftActorContext leaderContext;
138     protected RaftActorBehavior leader;
139
140     protected String follower1Id = factory.generateActorId("follower");
141     protected TestActorRef<TestRaftActor> follower1Actor;
142     protected ActorRef follower1CollectorActor;
143     protected RaftActorBehavior follower1;
144     protected RaftActorContext follower1Context;
145
146     protected String follower2Id = factory.generateActorId("follower");
147     protected TestActorRef<TestRaftActor> follower2Actor;
148     protected ActorRef follower2CollectorActor;
149     protected  RaftActorBehavior follower2;
150     protected RaftActorContext follower2Context;
151
152     protected ImmutableMap<String, String> peerAddresses;
153
154     protected long initialTerm = 5;
155     protected long currentTerm;
156
157     protected List<Object> expSnapshotState = new ArrayList<>();
158
159     @After
160     public void tearDown() {
161         InMemoryJournal.clear();
162         InMemorySnapshotStore.clear();
163         factory.close();
164     }
165
166     protected DefaultConfigParamsImpl newLeaderConfigParams() {
167         DefaultConfigParamsImpl configParams = new DefaultConfigParamsImpl();
168         configParams.setHeartBeatInterval(new FiniteDuration(100, TimeUnit.MILLISECONDS));
169         configParams.setElectionTimeoutFactor(1);
170         configParams.setSnapshotBatchCount(4);
171         configParams.setSnapshotDataThresholdPercentage(70);
172         configParams.setIsolatedLeaderCheckInterval(new FiniteDuration(1, TimeUnit.DAYS));
173         return configParams;
174     }
175
176     protected DefaultConfigParamsImpl newFollowerConfigParams() {
177         DefaultConfigParamsImpl configParams = new DefaultConfigParamsImpl();
178         configParams.setHeartBeatInterval(new FiniteDuration(500, TimeUnit.MILLISECONDS));
179         configParams.setElectionTimeoutFactor(1000);
180         return configParams;
181     }
182
183     protected void waitUntilLeader(ActorRef actorRef) {
184         RaftActorTestKit.waitUntilLeader(actorRef);
185     }
186
187     protected TestActorRef<TestRaftActor> newTestRaftActor(String id, Map<String, String> peerAddresses,
188             ConfigParams configParams) {
189         TestActorRef<MessageCollectorActor> collectorActor = factory.createTestActor(
190                 MessageCollectorActor.props().withDispatcher(Dispatchers.DefaultDispatcherId()),
191                         factory.generateActorId(id + "-collector"));
192
193         InvalidActorNameException lastEx = null;
194         for(int i = 0; i < 10; i++) {
195             try {
196                 return factory.createTestActor(TestRaftActor.props(id,
197                         peerAddresses != null ? peerAddresses : Collections.<String, String>emptyMap(),
198                                 configParams, collectorActor), id);
199             } catch (InvalidActorNameException e) {
200                 lastEx = e;
201                 Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
202             }
203         }
204
205         assertNotNull(lastEx);
206         throw lastEx;
207     }
208
209     protected void killActor(TestActorRef<TestRaftActor> leaderActor) {
210         JavaTestKit testkit = new JavaTestKit(getSystem());
211         testkit.watch(leaderActor);
212
213         leaderActor.tell(PoisonPill.getInstance(), null);
214         testkit.expectMsgClass(JavaTestKit.duration("5 seconds"), Terminated.class);
215
216         testkit.unwatch(leaderActor);
217     }
218
219     protected void verifyApplyJournalEntries(ActorRef actor, final long expIndex) {
220         MessageCollectorActor.expectFirstMatching(actor, ApplyJournalEntries.class, new Predicate<ApplyJournalEntries>() {
221             @Override
222             public boolean apply(ApplyJournalEntries msg) {
223                 return msg.getToIndex() == expIndex;
224             }
225         });
226     }
227
228     @SuppressWarnings("unchecked")
229     protected void verifySnapshot(String prefix, Snapshot snapshot, long lastAppliedTerm,
230             int lastAppliedIndex, long lastTerm, long lastIndex)
231                     throws Exception {
232         assertEquals(prefix + " Snapshot getLastAppliedTerm", lastAppliedTerm, snapshot.getLastAppliedTerm());
233         assertEquals(prefix + " Snapshot getLastAppliedIndex", lastAppliedIndex, snapshot.getLastAppliedIndex());
234         assertEquals(prefix + " Snapshot getLastTerm", lastTerm, snapshot.getLastTerm());
235         assertEquals(prefix + " Snapshot getLastIndex", lastIndex, snapshot.getLastIndex());
236
237         List<Object> actualState = (List<Object>)MockRaftActor.toObject(snapshot.getState());
238         assertEquals(prefix + " Snapshot getState size", expSnapshotState.size(), actualState.size());
239         for(int i = 0; i < expSnapshotState.size(); i++) {
240             assertEquals(prefix + " Snapshot state " + i, expSnapshotState.get(i), actualState.get(i));
241         }
242     }
243
244     protected void verifyPersistedJournal(String persistenceId, List<? extends ReplicatedLogEntry> expJournal) {
245         List<ReplicatedLogEntry> journal = InMemoryJournal.get(persistenceId, ReplicatedLogEntry.class);
246         assertEquals("Journal ReplicatedLogEntry count", expJournal.size(), journal.size());
247         for(int i = 0; i < expJournal.size(); i++) {
248             ReplicatedLogEntry expected = expJournal.get(i);
249             ReplicatedLogEntry actual = journal.get(i);
250             verifyReplicatedLogEntry(expected, actual.getTerm(), actual.getIndex(), actual.getData());
251         }
252     }
253
254     protected MockPayload sendPayloadData(ActorRef leaderActor, String data) {
255         return sendPayloadData(leaderActor, data, 0);
256     }
257
258     protected MockPayload sendPayloadData(ActorRef leaderActor, String data, int size) {
259         MockPayload payload;
260         if(size > 0) {
261             payload = new MockPayload(data, size);
262         } else {
263             payload = new MockPayload(data);
264         }
265
266         leaderActor.tell(payload, ActorRef.noSender());
267         return payload;
268     }
269
270     protected void verifyApplyState(ApplyState applyState, ActorRef expClientActor,
271             String expId, long expTerm, long expIndex, MockPayload payload) {
272         assertEquals("ApplyState getClientActor", expClientActor, applyState.getClientActor());
273         assertEquals("ApplyState getIdentifier", expId, applyState.getIdentifier());
274         ReplicatedLogEntry replicatedLogEntry = applyState.getReplicatedLogEntry();
275         verifyReplicatedLogEntry(replicatedLogEntry, expTerm, expIndex, payload);
276     }
277
278     protected void verifyReplicatedLogEntry(ReplicatedLogEntry replicatedLogEntry, long expTerm, long expIndex,
279             Payload payload) {
280         assertEquals("ReplicatedLogEntry getTerm", expTerm, replicatedLogEntry.getTerm());
281         assertEquals("ReplicatedLogEntry getIndex", expIndex, replicatedLogEntry.getIndex());
282         assertEquals("ReplicatedLogEntry getData", payload, replicatedLogEntry.getData());
283     }
284
285     protected String testActorPath(String id){
286         return "akka://test/user" + id;
287     }
288 }