Bug 3570: Persist snapshot on follower ApplySnapshot
[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 SetPeerAddress {
54         private final String peerId;
55         private final String peerAddress;
56
57         public SetPeerAddress(String peerId, String peerAddress) {
58             this.peerId = peerId;
59             this.peerAddress = peerAddress;
60         }
61
62         public String getPeerId() {
63             return peerId;
64         }
65
66         public String getPeerAddress() {
67             return peerAddress;
68         }
69     }
70
71     public static class TestRaftActor extends MockRaftActor {
72
73         private final TestActorRef<MessageCollectorActor> collectorActor;
74         private final Map<Class<?>, Boolean> dropMessages = new ConcurrentHashMap<>();
75
76         private TestRaftActor(String id, Map<String, String> peerAddresses, ConfigParams config,
77                 TestActorRef<MessageCollectorActor> collectorActor) {
78             super(id, peerAddresses, Optional.of(config), null);
79             this.collectorActor = collectorActor;
80         }
81
82         public static Props props(String id, Map<String, String> peerAddresses, ConfigParams config,
83                 TestActorRef<MessageCollectorActor> collectorActor) {
84             return Props.create(TestRaftActor.class, id, peerAddresses, config, collectorActor).
85                     withDispatcher(Dispatchers.DefaultDispatcherId());
86         }
87
88         void startDropMessages(Class<?> msgClass) {
89             dropMessages.put(msgClass, Boolean.TRUE);
90         }
91
92         void stopDropMessages(Class<?> msgClass) {
93             dropMessages.remove(msgClass);
94         }
95
96         void setMockTotalMemory(final long mockTotalMemory) {
97             if(mockTotalMemory > 0) {
98                 getRaftActorContext().setTotalMemoryRetriever(new Supplier<Long>() {
99                     @Override
100                     public Long get() {
101                         return mockTotalMemory;
102                     }
103
104                 });
105             } else {
106                 getRaftActorContext().setTotalMemoryRetriever(null);
107             }
108         }
109
110         @Override
111         public void handleCommand(Object message) {
112             if(message instanceof MockPayload) {
113                 MockPayload payload = (MockPayload)message;
114                 super.persistData(collectorActor, payload.toString(), payload);
115                 return;
116             }
117
118             if(message instanceof SetPeerAddress) {
119                 setPeerAddress(((SetPeerAddress) message).getPeerId().toString(),
120                         ((SetPeerAddress) message).getPeerAddress());
121                 return;
122             }
123
124             try {
125                 if(!dropMessages.containsKey(message.getClass())) {
126                     super.handleCommand(message);
127                 }
128             } finally {
129                 if(!(message instanceof SendHeartBeat)) {
130                     try {
131                         collectorActor.tell(message, ActorRef.noSender());
132                     } catch (Exception e) {
133                         LOG.error("MessageCollectorActor error", e);
134                     }
135                 }
136             }
137         }
138
139         @Override
140         public void createSnapshot(ActorRef actorRef) {
141             try {
142                 actorRef.tell(new CaptureSnapshotReply(RaftActorTest.fromObject(getState()).toByteArray()), actorRef);
143             } catch (Exception e) {
144                 e.printStackTrace();
145             }
146         }
147
148         public ActorRef collectorActor() {
149             return collectorActor;
150         }
151     }
152
153     protected final Logger testLog = LoggerFactory.getLogger(getClass());
154
155     protected final TestActorFactory factory = new TestActorFactory(getSystem());
156
157     protected String leaderId = factory.generateActorId("leader");
158     protected DefaultConfigParamsImpl leaderConfigParams;
159     protected TestActorRef<TestRaftActor> leaderActor;
160     protected ActorRef leaderCollectorActor;
161     protected RaftActorContext leaderContext;
162     protected RaftActorBehavior leader;
163
164     protected String follower1Id = factory.generateActorId("follower");
165     protected TestActorRef<TestRaftActor> follower1Actor;
166     protected ActorRef follower1CollectorActor;
167     protected RaftActorBehavior follower1;
168     protected RaftActorContext follower1Context;
169
170     protected String follower2Id = factory.generateActorId("follower");
171     protected TestActorRef<TestRaftActor> follower2Actor;
172     protected ActorRef follower2CollectorActor;
173     protected  RaftActorBehavior follower2;
174     protected RaftActorContext follower2Context;
175
176     protected ImmutableMap<String, String> peerAddresses;
177
178     protected long initialTerm = 5;
179     protected long currentTerm;
180
181     protected List<Object> expSnapshotState = new ArrayList<>();
182
183     @After
184     public void tearDown() {
185         InMemoryJournal.clear();
186         InMemorySnapshotStore.clear();
187         factory.close();
188     }
189
190     protected DefaultConfigParamsImpl newLeaderConfigParams() {
191         DefaultConfigParamsImpl configParams = new DefaultConfigParamsImpl();
192         configParams.setHeartBeatInterval(new FiniteDuration(100, TimeUnit.MILLISECONDS));
193         configParams.setElectionTimeoutFactor(1);
194         configParams.setSnapshotBatchCount(4);
195         configParams.setSnapshotDataThresholdPercentage(70);
196         configParams.setIsolatedLeaderCheckInterval(new FiniteDuration(1, TimeUnit.DAYS));
197         return configParams;
198     }
199
200     protected DefaultConfigParamsImpl newFollowerConfigParams() {
201         DefaultConfigParamsImpl configParams = new DefaultConfigParamsImpl();
202         configParams.setHeartBeatInterval(new FiniteDuration(500, TimeUnit.MILLISECONDS));
203         configParams.setElectionTimeoutFactor(1000);
204         return configParams;
205     }
206
207     protected void waitUntilLeader(ActorRef actorRef) {
208         RaftActorTestKit.waitUntilLeader(actorRef);
209     }
210
211     protected TestActorRef<TestRaftActor> newTestRaftActor(String id, Map<String, String> peerAddresses,
212             ConfigParams configParams) {
213         TestActorRef<MessageCollectorActor> collectorActor = factory.createTestActor(
214                 MessageCollectorActor.props().withDispatcher(Dispatchers.DefaultDispatcherId()),
215                         factory.generateActorId(id + "-collector"));
216
217         InvalidActorNameException lastEx = null;
218         for(int i = 0; i < 10; i++) {
219             try {
220                 return factory.createTestActor(TestRaftActor.props(id,
221                         peerAddresses != null ? peerAddresses : Collections.<String, String>emptyMap(),
222                                 configParams, collectorActor), id);
223             } catch (InvalidActorNameException e) {
224                 lastEx = e;
225                 Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
226             }
227         }
228
229         assertNotNull(lastEx);
230         throw lastEx;
231     }
232
233     protected void killActor(TestActorRef<TestRaftActor> leaderActor) {
234         JavaTestKit testkit = new JavaTestKit(getSystem());
235         testkit.watch(leaderActor);
236
237         leaderActor.tell(PoisonPill.getInstance(), null);
238         testkit.expectMsgClass(JavaTestKit.duration("5 seconds"), Terminated.class);
239
240         testkit.unwatch(leaderActor);
241     }
242
243     protected void verifyApplyJournalEntries(ActorRef actor, final long expIndex) {
244         MessageCollectorActor.expectFirstMatching(actor, ApplyJournalEntries.class, new Predicate<ApplyJournalEntries>() {
245             @Override
246             public boolean apply(ApplyJournalEntries msg) {
247                 return msg.getToIndex() == expIndex;
248             }
249         });
250     }
251
252     @SuppressWarnings("unchecked")
253     protected void verifySnapshot(String prefix, Snapshot snapshot, long lastAppliedTerm,
254             int lastAppliedIndex, long lastTerm, long lastIndex)
255                     throws Exception {
256         assertEquals(prefix + " Snapshot getLastAppliedTerm", lastAppliedTerm, snapshot.getLastAppliedTerm());
257         assertEquals(prefix + " Snapshot getLastAppliedIndex", lastAppliedIndex, snapshot.getLastAppliedIndex());
258         assertEquals(prefix + " Snapshot getLastTerm", lastTerm, snapshot.getLastTerm());
259         assertEquals(prefix + " Snapshot getLastIndex", lastIndex, snapshot.getLastIndex());
260
261         List<Object> actualState = (List<Object>)MockRaftActor.toObject(snapshot.getState());
262         assertEquals(prefix + " Snapshot getState size", expSnapshotState.size(), actualState.size());
263         for(int i = 0; i < expSnapshotState.size(); i++) {
264             assertEquals(prefix + " Snapshot state " + i, expSnapshotState.get(i), actualState.get(i));
265         }
266     }
267
268     protected void verifyPersistedJournal(String persistenceId, List<? extends ReplicatedLogEntry> expJournal) {
269         List<ReplicatedLogEntry> journal = InMemoryJournal.get(persistenceId, ReplicatedLogEntry.class);
270         assertEquals("Journal ReplicatedLogEntry count", expJournal.size(), journal.size());
271         for(int i = 0; i < expJournal.size(); i++) {
272             ReplicatedLogEntry expected = expJournal.get(i);
273             ReplicatedLogEntry actual = journal.get(i);
274             verifyReplicatedLogEntry(expected, actual.getTerm(), actual.getIndex(), actual.getData());
275         }
276     }
277
278     protected MockPayload sendPayloadData(ActorRef leaderActor, String data) {
279         return sendPayloadData(leaderActor, data, 0);
280     }
281
282     protected MockPayload sendPayloadData(ActorRef leaderActor, String data, int size) {
283         MockPayload payload;
284         if(size > 0) {
285             payload = new MockPayload(data, size);
286         } else {
287             payload = new MockPayload(data);
288         }
289
290         leaderActor.tell(payload, ActorRef.noSender());
291         return payload;
292     }
293
294     protected void verifyApplyState(ApplyState applyState, ActorRef expClientActor,
295             String expId, long expTerm, long expIndex, MockPayload payload) {
296         assertEquals("ApplyState getClientActor", expClientActor, applyState.getClientActor());
297         assertEquals("ApplyState getIdentifier", expId, applyState.getIdentifier());
298         ReplicatedLogEntry replicatedLogEntry = applyState.getReplicatedLogEntry();
299         verifyReplicatedLogEntry(replicatedLogEntry, expTerm, expIndex, payload);
300     }
301
302     protected void verifyReplicatedLogEntry(ReplicatedLogEntry replicatedLogEntry, long expTerm, long expIndex,
303             Payload payload) {
304         assertEquals("ReplicatedLogEntry getTerm", expTerm, replicatedLogEntry.getTerm());
305         assertEquals("ReplicatedLogEntry getIndex", expIndex, replicatedLogEntry.getIndex());
306         assertEquals("ReplicatedLogEntry getData", payload, replicatedLogEntry.getData());
307     }
308
309     protected String testActorPath(String id){
310         return "akka://test/user" + id;
311     }
312 }