ab44654b6fcefcee29164a7b772f91b971191286
[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.Terminated;
16 import akka.dispatch.Dispatchers;
17 import akka.testkit.JavaTestKit;
18 import akka.testkit.TestActorRef;
19 import com.google.common.collect.ImmutableMap;
20 import com.google.common.util.concurrent.Uninterruptibles;
21 import java.util.ArrayList;
22 import java.util.Collections;
23 import java.util.List;
24 import java.util.Map;
25 import java.util.concurrent.ConcurrentHashMap;
26 import java.util.concurrent.TimeUnit;
27 import org.junit.After;
28 import org.opendaylight.controller.cluster.raft.MockRaftActorContext.MockPayload;
29 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
30 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
31 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
32 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
33 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
34 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
35 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
36 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
37 import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor;
38 import org.opendaylight.yangtools.concepts.Identifier;
39 import org.opendaylight.yangtools.util.StringIdentifier;
40 import org.slf4j.Logger;
41 import org.slf4j.LoggerFactory;
42 import scala.concurrent.duration.FiniteDuration;
43
44 /**
45  * Abstract base for an integration test that tests end-to-end RaftActor and behavior functionality.
46  *
47  * @author Thomas Pantelis
48  */
49 public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest {
50
51     public static class SetPeerAddress {
52         private final String peerId;
53         private final String peerAddress;
54
55         public SetPeerAddress(String peerId, String peerAddress) {
56             this.peerId = peerId;
57             this.peerAddress = peerAddress;
58         }
59
60         public String getPeerId() {
61             return peerId;
62         }
63
64         public String getPeerAddress() {
65             return peerAddress;
66         }
67     }
68
69     public static class TestRaftActor extends MockRaftActor {
70
71         private final TestActorRef<MessageCollectorActor> collectorActor;
72         private final Map<Class<?>, Boolean> dropMessages = new ConcurrentHashMap<>();
73
74         private TestRaftActor(Builder builder) {
75             super(builder);
76             this.collectorActor = builder.collectorActor;
77         }
78
79         void startDropMessages(Class<?> msgClass) {
80             dropMessages.put(msgClass, Boolean.TRUE);
81         }
82
83         void stopDropMessages(Class<?> msgClass) {
84             dropMessages.remove(msgClass);
85         }
86
87         void setMockTotalMemory(final long mockTotalMemory) {
88             getRaftActorContext().setTotalMemoryRetriever(mockTotalMemory > 0 ? () -> mockTotalMemory : null);
89         }
90
91         @Override
92         public void handleCommand(Object message) {
93             if(message instanceof MockPayload) {
94                 MockPayload payload = (MockPayload)message;
95                 super.persistData(collectorActor, new StringIdentifier(payload.toString()), payload);
96                 return;
97             }
98
99             if(message instanceof SetPeerAddress) {
100                 setPeerAddress(((SetPeerAddress) message).getPeerId().toString(),
101                         ((SetPeerAddress) message).getPeerAddress());
102                 return;
103             }
104
105             try {
106                 if(!dropMessages.containsKey(message.getClass())) {
107                     super.handleCommand(message);
108                 }
109             } finally {
110                 if(!(message instanceof SendHeartBeat)) {
111                     try {
112                         collectorActor.tell(message, ActorRef.noSender());
113                     } catch (Exception e) {
114                         LOG.error("MessageCollectorActor error", e);
115                     }
116                 }
117             }
118         }
119
120         @Override
121         public void createSnapshot(ActorRef actorRef) {
122             try {
123                 actorRef.tell(new CaptureSnapshotReply(RaftActorTest.fromObject(getState()).toByteArray()), actorRef);
124             } catch (Exception e) {
125                 e.printStackTrace();
126             }
127         }
128
129         public ActorRef collectorActor() {
130             return collectorActor;
131         }
132
133         public static Builder newBuilder() {
134             return new Builder();
135         }
136
137         public static class Builder extends AbstractBuilder<Builder, TestRaftActor> {
138             private TestActorRef<MessageCollectorActor> collectorActor;
139
140             public Builder collectorActor(TestActorRef<MessageCollectorActor> collectorActor) {
141                 this.collectorActor = collectorActor;
142                 return this;
143             }
144
145             private Builder() {
146                 super(TestRaftActor.class);
147             }
148         }
149     }
150
151     protected static final int SNAPSHOT_CHUNK_SIZE = 100;
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 int snapshotBatchCount = 4;
182
183     protected List<MockPayload> expSnapshotState = new ArrayList<>();
184
185     @After
186     public void tearDown() {
187         InMemoryJournal.clear();
188         InMemorySnapshotStore.clear();
189         factory.close();
190     }
191
192     protected DefaultConfigParamsImpl newLeaderConfigParams() {
193         DefaultConfigParamsImpl configParams = new DefaultConfigParamsImpl();
194         configParams.setHeartBeatInterval(new FiniteDuration(100, TimeUnit.MILLISECONDS));
195         configParams.setElectionTimeoutFactor(1);
196         configParams.setSnapshotBatchCount(snapshotBatchCount);
197         configParams.setSnapshotDataThresholdPercentage(70);
198         configParams.setIsolatedLeaderCheckInterval(new FiniteDuration(1, TimeUnit.DAYS));
199         configParams.setSnapshotChunkSize(SNAPSHOT_CHUNK_SIZE);
200         return configParams;
201     }
202
203     protected DefaultConfigParamsImpl newFollowerConfigParams() {
204         DefaultConfigParamsImpl configParams = new DefaultConfigParamsImpl();
205         configParams.setHeartBeatInterval(new FiniteDuration(500, TimeUnit.MILLISECONDS));
206         configParams.setElectionTimeoutFactor(1000);
207         return configParams;
208     }
209
210     protected void waitUntilLeader(ActorRef actorRef) {
211         RaftActorTestKit.waitUntilLeader(actorRef);
212     }
213
214     protected TestActorRef<TestRaftActor> newTestRaftActor(String id, Map<String, String> peerAddresses,
215             ConfigParams configParams) {
216         return newTestRaftActor(id, TestRaftActor.newBuilder().peerAddresses(peerAddresses != null ? peerAddresses :
217             Collections.<String, String>emptyMap()).config(configParams));
218     }
219
220     protected TestActorRef<TestRaftActor> newTestRaftActor(String id, TestRaftActor.Builder builder) {
221         builder.collectorActor(factory.<MessageCollectorActor>createTestActor(
222                 MessageCollectorActor.props().withDispatcher(Dispatchers.DefaultDispatcherId()),
223                         factory.generateActorId(id + "-collector"))).id(id);
224
225         InvalidActorNameException lastEx = null;
226         for(int i = 0; i < 10; i++) {
227             try {
228                 return factory.createTestActor(builder.props().withDispatcher(Dispatchers.DefaultDispatcherId()), id);
229             } catch (InvalidActorNameException e) {
230                 lastEx = e;
231                 Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
232             }
233         }
234
235         assertNotNull(lastEx);
236         throw lastEx;
237     }
238
239     protected void killActor(TestActorRef<TestRaftActor> leaderActor) {
240         JavaTestKit testkit = new JavaTestKit(getSystem());
241         testkit.watch(leaderActor);
242
243         leaderActor.tell(PoisonPill.getInstance(), null);
244         testkit.expectMsgClass(JavaTestKit.duration("5 seconds"), Terminated.class);
245
246         testkit.unwatch(leaderActor);
247     }
248
249     protected void verifyApplyJournalEntries(ActorRef actor, final long expIndex) {
250         MessageCollectorActor.expectFirstMatching(actor, ApplyJournalEntries.class, msg -> msg.getToIndex() == expIndex);
251     }
252
253     @SuppressWarnings("unchecked")
254     protected void verifySnapshot(String prefix, Snapshot snapshot, long lastAppliedTerm,
255             long lastAppliedIndex, long lastTerm, long lastIndex)
256                     throws Exception {
257         assertEquals(prefix + " Snapshot getLastAppliedTerm", lastAppliedTerm, snapshot.getLastAppliedTerm());
258         assertEquals(prefix + " Snapshot getLastAppliedIndex", lastAppliedIndex, snapshot.getLastAppliedIndex());
259         assertEquals(prefix + " Snapshot getLastTerm", lastTerm, snapshot.getLastTerm());
260         assertEquals(prefix + " Snapshot getLastIndex", lastIndex, snapshot.getLastIndex());
261
262         List<Object> actualState = (List<Object>)MockRaftActor.toObject(snapshot.getState());
263         assertEquals(String.format("%s Snapshot getState size. Expected %s: . Actual: %s", prefix, expSnapshotState,
264                 actualState), expSnapshotState.size(), actualState.size());
265         for(int i = 0; i < expSnapshotState.size(); i++) {
266             assertEquals(prefix + " Snapshot state " + i, expSnapshotState.get(i), actualState.get(i));
267         }
268     }
269
270     protected void verifyPersistedJournal(String persistenceId, List<? extends ReplicatedLogEntry> expJournal) {
271         List<ReplicatedLogEntry> journal = InMemoryJournal.get(persistenceId, ReplicatedLogEntry.class);
272         assertEquals("Journal ReplicatedLogEntry count", expJournal.size(), journal.size());
273         for(int i = 0; i < expJournal.size(); i++) {
274             ReplicatedLogEntry expected = expJournal.get(i);
275             ReplicatedLogEntry actual = journal.get(i);
276             verifyReplicatedLogEntry(expected, actual.getTerm(), actual.getIndex(), actual.getData());
277         }
278     }
279
280     protected MockPayload sendPayloadData(ActorRef leaderActor, String data) {
281         return sendPayloadData(leaderActor, data, 0);
282     }
283
284     protected MockPayload sendPayloadData(ActorRef leaderActor, String data, int size) {
285         MockPayload payload;
286         if(size > 0) {
287             payload = new MockPayload(data, size);
288         } else {
289             payload = new MockPayload(data);
290         }
291
292         leaderActor.tell(payload, ActorRef.noSender());
293         return payload;
294     }
295
296     protected void verifyApplyState(ApplyState applyState, ActorRef expClientActor,
297             String expId, long expTerm, long expIndex, MockPayload payload) {
298         assertEquals("ApplyState getClientActor", expClientActor, applyState.getClientActor());
299
300         final Identifier id = expId == null ? null : new StringIdentifier(expId);
301         assertEquals("ApplyState getIdentifier", id, applyState.getIdentifier());
302         ReplicatedLogEntry replicatedLogEntry = applyState.getReplicatedLogEntry();
303         verifyReplicatedLogEntry(replicatedLogEntry, expTerm, expIndex, payload);
304     }
305
306     protected void verifyReplicatedLogEntry(ReplicatedLogEntry replicatedLogEntry, long expTerm, long expIndex,
307             Payload payload) {
308         assertEquals("ReplicatedLogEntry getTerm", expTerm, replicatedLogEntry.getTerm());
309         assertEquals("ReplicatedLogEntry getIndex", expIndex, replicatedLogEntry.getIndex());
310         assertEquals("ReplicatedLogEntry getData", payload, replicatedLogEntry.getData());
311     }
312
313     protected String testActorPath(String id){
314         return factory.createTestActorPath(id);
315     }
316
317     protected void verifyLeadersTrimmedLog(long lastIndex) {
318         verifyTrimmedLog("Leader", leaderActor, lastIndex, lastIndex - 1);
319     }
320
321     protected void verifyLeadersTrimmedLog(long lastIndex, long replicatedToAllIndex) {
322         verifyTrimmedLog("Leader", leaderActor, lastIndex, replicatedToAllIndex);
323     }
324
325     protected void verifyFollowersTrimmedLog(int num, TestActorRef<TestRaftActor> actorRef, long lastIndex) {
326         verifyTrimmedLog("Follower " + num, actorRef, lastIndex, lastIndex - 1);
327     }
328
329     protected void verifyTrimmedLog(String name, TestActorRef<TestRaftActor> actorRef, long lastIndex,
330             long replicatedToAllIndex) {
331         TestRaftActor actor = actorRef.underlyingActor();
332         RaftActorContext context = actor.getRaftActorContext();
333         long snapshotIndex = lastIndex - 1;
334         assertEquals(name + " snapshot term", snapshotIndex < 0 ? -1 : currentTerm,
335                 context.getReplicatedLog().getSnapshotTerm());
336         assertEquals(name + " snapshot index", snapshotIndex, context.getReplicatedLog().getSnapshotIndex());
337         assertEquals(name + " journal log size", 1, context.getReplicatedLog().size());
338         assertEquals(name + " journal last index", lastIndex, context.getReplicatedLog().lastIndex());
339         assertEquals(name + " commit index", lastIndex, context.getCommitIndex());
340         assertEquals(name + " last applied", lastIndex, context.getLastApplied());
341         assertEquals(name + " replicatedToAllIndex", replicatedToAllIndex,
342                 actor.getCurrentBehavior().getReplicatedToAllIndex());
343     }
344 }