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