Move ServerConfigurationPayload to cluster.raft.persisted
[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.persisted.ServerConfigurationPayload;
35 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
36 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
37 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
38 import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor;
39 import org.opendaylight.yangtools.concepts.Identifier;
40 import org.opendaylight.yangtools.util.AbstractStringIdentifier;
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     private static final class MockIdentifier extends AbstractStringIdentifier<MockIdentifier> {
53         private static final long serialVersionUID = 1L;
54
55         protected MockIdentifier(String string) {
56             super(string);
57         }
58     }
59
60     public static class SetPeerAddress {
61         private final String peerId;
62         private final String peerAddress;
63
64         public SetPeerAddress(String peerId, String peerAddress) {
65             this.peerId = peerId;
66             this.peerAddress = peerAddress;
67         }
68
69         public String getPeerId() {
70             return peerId;
71         }
72
73         public String getPeerAddress() {
74             return peerAddress;
75         }
76     }
77
78     public static class TestRaftActor extends MockRaftActor {
79
80         private final TestActorRef<MessageCollectorActor> collectorActor;
81         private final Map<Class<?>, Boolean> dropMessages = new ConcurrentHashMap<>();
82
83         private TestRaftActor(Builder builder) {
84             super(builder);
85             this.collectorActor = builder.collectorActor;
86         }
87
88         public void startDropMessages(Class<?> msgClass) {
89             dropMessages.put(msgClass, Boolean.TRUE);
90         }
91
92         public void stopDropMessages(Class<?> msgClass) {
93             dropMessages.remove(msgClass);
94         }
95
96         void setMockTotalMemory(final long mockTotalMemory) {
97             getRaftActorContext().setTotalMemoryRetriever(mockTotalMemory > 0 ? () -> mockTotalMemory : null);
98         }
99
100         @Override
101         public void handleCommand(Object message) {
102             if(message instanceof MockPayload) {
103                 MockPayload payload = (MockPayload)message;
104                 super.persistData(collectorActor, new MockIdentifier(payload.toString()), payload);
105                 return;
106             }
107
108             if(message instanceof ServerConfigurationPayload) {
109                 super.persistData(collectorActor, new MockIdentifier("serverConfig"), (Payload)message);
110                 return;
111             }
112
113             if(message instanceof SetPeerAddress) {
114                 setPeerAddress(((SetPeerAddress) message).getPeerId().toString(),
115                         ((SetPeerAddress) message).getPeerAddress());
116                 return;
117             }
118
119             try {
120                 if(!dropMessages.containsKey(message.getClass())) {
121                     super.handleCommand(message);
122                 }
123             } finally {
124                 if(!(message instanceof SendHeartBeat)) {
125                     try {
126                         collectorActor.tell(message, ActorRef.noSender());
127                     } catch (Exception e) {
128                         LOG.error("MessageCollectorActor error", e);
129                     }
130                 }
131             }
132         }
133
134         @Override
135         public void createSnapshot(ActorRef actorRef) {
136             try {
137                 actorRef.tell(new CaptureSnapshotReply(RaftActorTest.fromObject(getState()).toByteArray()), actorRef);
138             } catch (Exception e) {
139                 e.printStackTrace();
140             }
141         }
142
143         public ActorRef collectorActor() {
144             return collectorActor;
145         }
146
147         public static Builder newBuilder() {
148             return new Builder();
149         }
150
151         public static class Builder extends AbstractBuilder<Builder, TestRaftActor> {
152             private TestActorRef<MessageCollectorActor> collectorActor;
153
154             public Builder collectorActor(TestActorRef<MessageCollectorActor> collectorActor) {
155                 this.collectorActor = collectorActor;
156                 return this;
157             }
158
159             private Builder() {
160                 super(TestRaftActor.class);
161             }
162         }
163     }
164
165     protected static final int SNAPSHOT_CHUNK_SIZE = 100;
166
167     protected final Logger testLog = LoggerFactory.getLogger(getClass());
168
169     protected final TestActorFactory factory = new TestActorFactory(getSystem());
170
171     protected String leaderId = factory.generateActorId("leader");
172     protected DefaultConfigParamsImpl leaderConfigParams;
173     protected TestActorRef<TestRaftActor> leaderActor;
174     protected ActorRef leaderCollectorActor;
175     protected RaftActorContext leaderContext;
176     protected RaftActorBehavior leader;
177
178     protected String follower1Id = factory.generateActorId("follower");
179     protected TestActorRef<TestRaftActor> follower1Actor;
180     protected ActorRef follower1CollectorActor;
181     protected RaftActorBehavior follower1;
182     protected RaftActorContext follower1Context;
183
184     protected String follower2Id = factory.generateActorId("follower");
185     protected TestActorRef<TestRaftActor> follower2Actor;
186     protected ActorRef follower2CollectorActor;
187     protected  RaftActorBehavior follower2;
188     protected RaftActorContext follower2Context;
189
190     protected ImmutableMap<String, String> peerAddresses;
191
192     protected long initialTerm = 5;
193     protected long currentTerm;
194
195     protected int snapshotBatchCount = 4;
196
197     protected List<MockPayload> expSnapshotState = new ArrayList<>();
198
199     @After
200     public void tearDown() {
201         InMemoryJournal.clear();
202         InMemorySnapshotStore.clear();
203         factory.close();
204     }
205
206     protected DefaultConfigParamsImpl newLeaderConfigParams() {
207         DefaultConfigParamsImpl configParams = new DefaultConfigParamsImpl();
208         configParams.setHeartBeatInterval(new FiniteDuration(100, TimeUnit.MILLISECONDS));
209         configParams.setElectionTimeoutFactor(1);
210         configParams.setSnapshotBatchCount(snapshotBatchCount);
211         configParams.setSnapshotDataThresholdPercentage(70);
212         configParams.setIsolatedLeaderCheckInterval(new FiniteDuration(1, TimeUnit.DAYS));
213         configParams.setSnapshotChunkSize(SNAPSHOT_CHUNK_SIZE);
214         return configParams;
215     }
216
217     protected DefaultConfigParamsImpl newFollowerConfigParams() {
218         DefaultConfigParamsImpl configParams = new DefaultConfigParamsImpl();
219         configParams.setHeartBeatInterval(new FiniteDuration(500, TimeUnit.MILLISECONDS));
220         configParams.setElectionTimeoutFactor(1000);
221         return configParams;
222     }
223
224     protected void waitUntilLeader(ActorRef actorRef) {
225         RaftActorTestKit.waitUntilLeader(actorRef);
226     }
227
228     protected TestActorRef<TestRaftActor> newTestRaftActor(String id, Map<String, String> peerAddresses,
229             ConfigParams configParams) {
230         return newTestRaftActor(id, TestRaftActor.newBuilder().peerAddresses(peerAddresses != null ? peerAddresses :
231             Collections.<String, String>emptyMap()).config(configParams));
232     }
233
234     protected TestActorRef<TestRaftActor> newTestRaftActor(String id, TestRaftActor.Builder builder) {
235         builder.collectorActor(factory.<MessageCollectorActor>createTestActor(
236                 MessageCollectorActor.props().withDispatcher(Dispatchers.DefaultDispatcherId()),
237                         factory.generateActorId(id + "-collector"))).id(id);
238
239         InvalidActorNameException lastEx = null;
240         for(int i = 0; i < 10; i++) {
241             try {
242                 return factory.createTestActor(builder.props().withDispatcher(Dispatchers.DefaultDispatcherId()), id);
243             } catch (InvalidActorNameException e) {
244                 lastEx = e;
245                 Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
246             }
247         }
248
249         assertNotNull(lastEx);
250         throw lastEx;
251     }
252
253     protected void killActor(TestActorRef<TestRaftActor> leaderActor) {
254         JavaTestKit testkit = new JavaTestKit(getSystem());
255         testkit.watch(leaderActor);
256
257         leaderActor.tell(PoisonPill.getInstance(), null);
258         testkit.expectMsgClass(JavaTestKit.duration("5 seconds"), Terminated.class);
259
260         testkit.unwatch(leaderActor);
261     }
262
263     protected void verifyApplyJournalEntries(ActorRef actor, final long expIndex) {
264         MessageCollectorActor.expectFirstMatching(actor, ApplyJournalEntries.class, msg -> msg.getToIndex() == expIndex);
265     }
266
267     @SuppressWarnings("unchecked")
268     protected void verifySnapshot(String prefix, Snapshot snapshot, long lastAppliedTerm,
269             long lastAppliedIndex, long lastTerm, long lastIndex)
270                     throws Exception {
271         assertEquals(prefix + " Snapshot getLastAppliedTerm", lastAppliedTerm, snapshot.getLastAppliedTerm());
272         assertEquals(prefix + " Snapshot getLastAppliedIndex", lastAppliedIndex, snapshot.getLastAppliedIndex());
273         assertEquals(prefix + " Snapshot getLastTerm", lastTerm, snapshot.getLastTerm());
274         assertEquals(prefix + " Snapshot getLastIndex", lastIndex, snapshot.getLastIndex());
275
276         List<Object> actualState = (List<Object>)MockRaftActor.toObject(snapshot.getState());
277         assertEquals(String.format("%s Snapshot getState size. Expected %s: . Actual: %s", prefix, expSnapshotState,
278                 actualState), expSnapshotState.size(), actualState.size());
279         for(int i = 0; i < expSnapshotState.size(); i++) {
280             assertEquals(prefix + " Snapshot state " + i, expSnapshotState.get(i), actualState.get(i));
281         }
282     }
283
284     protected void verifyPersistedJournal(String persistenceId, List<? extends ReplicatedLogEntry> expJournal) {
285         List<ReplicatedLogEntry> journal = InMemoryJournal.get(persistenceId, ReplicatedLogEntry.class);
286         assertEquals("Journal ReplicatedLogEntry count", expJournal.size(), journal.size());
287         for(int i = 0; i < expJournal.size(); i++) {
288             ReplicatedLogEntry expected = expJournal.get(i);
289             ReplicatedLogEntry actual = journal.get(i);
290             verifyReplicatedLogEntry(expected, actual.getTerm(), actual.getIndex(), actual.getData());
291         }
292     }
293
294     protected MockPayload sendPayloadData(ActorRef leaderActor, String data) {
295         return sendPayloadData(leaderActor, data, 0);
296     }
297
298     protected MockPayload sendPayloadData(ActorRef leaderActor, String data, int size) {
299         MockPayload payload;
300         if(size > 0) {
301             payload = new MockPayload(data, size);
302         } else {
303             payload = new MockPayload(data);
304         }
305
306         leaderActor.tell(payload, ActorRef.noSender());
307         return payload;
308     }
309
310     protected void verifyApplyState(ApplyState applyState, ActorRef expClientActor,
311             String expId, long expTerm, long expIndex, Payload payload) {
312         assertEquals("ApplyState getClientActor", expClientActor, applyState.getClientActor());
313
314         final Identifier id = expId == null ? null : new MockIdentifier(expId);
315         assertEquals("ApplyState getIdentifier", id, applyState.getIdentifier());
316         ReplicatedLogEntry replicatedLogEntry = applyState.getReplicatedLogEntry();
317         verifyReplicatedLogEntry(replicatedLogEntry, expTerm, expIndex, payload);
318     }
319
320     protected void verifyReplicatedLogEntry(ReplicatedLogEntry replicatedLogEntry, long expTerm, long expIndex,
321             Payload payload) {
322         assertEquals("ReplicatedLogEntry getTerm", expTerm, replicatedLogEntry.getTerm());
323         assertEquals("ReplicatedLogEntry getIndex", expIndex, replicatedLogEntry.getIndex());
324         assertEquals("ReplicatedLogEntry getData", payload, replicatedLogEntry.getData());
325     }
326
327     protected String testActorPath(String id){
328         return factory.createTestActorPath(id);
329     }
330
331     protected void verifyLeadersTrimmedLog(long lastIndex) {
332         verifyTrimmedLog("Leader", leaderActor, lastIndex, lastIndex - 1);
333     }
334
335     protected void verifyLeadersTrimmedLog(long lastIndex, long replicatedToAllIndex) {
336         verifyTrimmedLog("Leader", leaderActor, lastIndex, replicatedToAllIndex);
337     }
338
339     protected void verifyFollowersTrimmedLog(int num, TestActorRef<TestRaftActor> actorRef, long lastIndex) {
340         verifyTrimmedLog("Follower " + num, actorRef, lastIndex, lastIndex - 1);
341     }
342
343     protected void verifyTrimmedLog(String name, TestActorRef<TestRaftActor> actorRef, long lastIndex,
344             long replicatedToAllIndex) {
345         TestRaftActor actor = actorRef.underlyingActor();
346         RaftActorContext context = actor.getRaftActorContext();
347         long snapshotIndex = lastIndex - 1;
348         assertEquals(name + " snapshot term", snapshotIndex < 0 ? -1 : currentTerm,
349                 context.getReplicatedLog().getSnapshotTerm());
350         assertEquals(name + " snapshot index", snapshotIndex, context.getReplicatedLog().getSnapshotIndex());
351         assertEquals(name + " journal log size", 1, context.getReplicatedLog().size());
352         assertEquals(name + " journal last index", lastIndex, context.getReplicatedLog().lastIndex());
353         assertEquals(name + " commit index", lastIndex, context.getCommitIndex());
354         assertEquals(name + " last applied", lastIndex, context.getLastApplied());
355         assertEquals(name + " replicatedToAllIndex", replicatedToAllIndex,
356                 actor.getCurrentBehavior().getReplicatedToAllIndex());
357     }
358 }