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