6e4d536d3d10bf4b32ad5452e55626788208bcc9
[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 akka.pattern.Patterns.ask;
11 import static org.junit.Assert.assertEquals;
12 import static org.junit.Assert.assertNotNull;
13
14 import akka.actor.ActorRef;
15 import akka.actor.InvalidActorNameException;
16 import akka.actor.PoisonPill;
17 import akka.actor.Terminated;
18 import akka.dispatch.Dispatchers;
19 import akka.dispatch.Mailboxes;
20 import akka.testkit.TestActorRef;
21 import akka.testkit.javadsl.TestKit;
22 import akka.util.Timeout;
23 import com.google.common.base.Stopwatch;
24 import com.google.common.collect.ImmutableMap;
25 import com.google.common.util.concurrent.Uninterruptibles;
26 import java.io.OutputStream;
27 import java.util.ArrayList;
28 import java.util.Collections;
29 import java.util.List;
30 import java.util.Map;
31 import java.util.Optional;
32 import java.util.concurrent.ConcurrentHashMap;
33 import java.util.concurrent.TimeUnit;
34 import java.util.function.Consumer;
35 import java.util.function.Predicate;
36 import org.apache.commons.lang3.SerializationUtils;
37 import org.junit.After;
38 import org.opendaylight.controller.cluster.raft.MockRaftActor.MockSnapshotState;
39 import org.opendaylight.controller.cluster.raft.MockRaftActorContext.MockPayload;
40 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
41 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
42 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
43 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
44 import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftState;
45 import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState;
46 import org.opendaylight.controller.cluster.raft.persisted.ApplyJournalEntries;
47 import org.opendaylight.controller.cluster.raft.persisted.ServerConfigurationPayload;
48 import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
49 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
50 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
51 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
52 import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor;
53 import org.opendaylight.yangtools.concepts.Identifier;
54 import org.opendaylight.yangtools.util.AbstractStringIdentifier;
55 import org.slf4j.Logger;
56 import org.slf4j.LoggerFactory;
57 import scala.concurrent.Await;
58 import scala.concurrent.duration.FiniteDuration;
59
60 /**
61  * Abstract base for an integration test that tests end-to-end RaftActor and behavior functionality.
62  *
63  * @author Thomas Pantelis
64  */
65 public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest {
66
67     private static final class MockIdentifier extends AbstractStringIdentifier<MockIdentifier> {
68         private static final long serialVersionUID = 1L;
69
70         protected MockIdentifier(final String string) {
71             super(string);
72         }
73     }
74
75     public static class SetPeerAddress {
76         private final String peerId;
77         private final String peerAddress;
78
79         public SetPeerAddress(final String peerId, final String peerAddress) {
80             this.peerId = peerId;
81             this.peerAddress = peerAddress;
82         }
83
84         public String getPeerId() {
85             return peerId;
86         }
87
88         public String getPeerAddress() {
89             return peerAddress;
90         }
91     }
92
93     public static class TestRaftActor extends MockRaftActor {
94
95         private final ActorRef collectorActor;
96         private final Map<Class<?>, Predicate<?>> dropMessages = new ConcurrentHashMap<>();
97
98         TestRaftActor(final Builder builder) {
99             super(builder);
100             this.collectorActor = builder.collectorActor;
101         }
102
103         public void startDropMessages(final Class<?> msgClass) {
104             dropMessages.put(msgClass, msg -> true);
105         }
106
107         <T> void startDropMessages(final Class<T> msgClass, final Predicate<T> filter) {
108             dropMessages.put(msgClass, filter);
109         }
110
111         public void stopDropMessages(final Class<?> msgClass) {
112             dropMessages.remove(msgClass);
113         }
114
115         void setMockTotalMemory(final long mockTotalMemory) {
116             getRaftActorContext().setTotalMemoryRetriever(mockTotalMemory > 0 ? () -> mockTotalMemory : null);
117         }
118
119         @SuppressWarnings({ "rawtypes", "unchecked", "checkstyle:IllegalCatch" })
120         @Override
121         public void handleCommand(final Object message) {
122             if (message instanceof MockPayload) {
123                 MockPayload payload = (MockPayload) message;
124                 super.persistData(collectorActor, new MockIdentifier(payload.toString()), payload, false);
125                 return;
126             }
127
128             if (message instanceof ServerConfigurationPayload) {
129                 super.persistData(collectorActor, new MockIdentifier("serverConfig"), (Payload) message, false);
130                 return;
131             }
132
133             if (message instanceof SetPeerAddress) {
134                 setPeerAddress(((SetPeerAddress) message).getPeerId(),
135                         ((SetPeerAddress) message).getPeerAddress());
136                 return;
137             }
138
139             try {
140                 Predicate drop = dropMessages.get(message.getClass());
141                 if (drop == null || !drop.test(message)) {
142                     super.handleCommand(message);
143                 }
144             } finally {
145                 if (!(message instanceof SendHeartBeat)) {
146                     try {
147                         collectorActor.tell(message, ActorRef.noSender());
148                     } catch (Exception e) {
149                         LOG.error("MessageCollectorActor error", e);
150                     }
151                 }
152             }
153         }
154
155         @Override
156         @SuppressWarnings("checkstyle:IllegalCatch")
157         public void createSnapshot(final ActorRef actorRef, final Optional<OutputStream> installSnapshotStream) {
158             MockSnapshotState snapshotState = new MockSnapshotState(new ArrayList<>(getState()));
159             if (installSnapshotStream.isPresent()) {
160                 SerializationUtils.serialize(snapshotState, installSnapshotStream.get());
161             }
162
163             actorRef.tell(new CaptureSnapshotReply(snapshotState, installSnapshotStream), actorRef);
164         }
165
166         public ActorRef collectorActor() {
167             return collectorActor;
168         }
169
170         public static Builder newBuilder() {
171             return new Builder();
172         }
173
174         public static class Builder extends AbstractBuilder<Builder, TestRaftActor> {
175             private ActorRef collectorActor;
176
177             Builder() {
178                 super(TestRaftActor.class);
179             }
180
181             public Builder collectorActor(final ActorRef newCollectorActor) {
182                 this.collectorActor = newCollectorActor;
183                 return this;
184             }
185         }
186     }
187
188     protected static final int SNAPSHOT_CHUNK_SIZE = 100;
189
190     protected final Logger testLog = LoggerFactory.getLogger(getClass());
191
192     protected final TestActorFactory factory = new TestActorFactory(getSystem());
193
194     protected String leaderId = factory.generateActorId("leader");
195     protected DefaultConfigParamsImpl leaderConfigParams;
196     protected TestActorRef<TestRaftActor> leaderActor;
197     protected ActorRef leaderCollectorActor;
198     protected RaftActorContext leaderContext;
199     protected RaftActorBehavior leader;
200
201     protected String follower1Id = factory.generateActorId("follower");
202     protected TestActorRef<TestRaftActor> follower1Actor;
203     protected ActorRef follower1CollectorActor;
204     protected RaftActorBehavior follower1;
205     protected RaftActorContext follower1Context;
206
207     protected String follower2Id = factory.generateActorId("follower");
208     protected TestActorRef<TestRaftActor> follower2Actor;
209     protected ActorRef follower2CollectorActor;
210     protected  RaftActorBehavior follower2;
211     protected RaftActorContext follower2Context;
212
213     protected ImmutableMap<String, String> peerAddresses;
214
215     protected long initialTerm = 5;
216     protected long currentTerm;
217
218     protected int snapshotBatchCount = 4;
219     protected int snapshotChunkSize = SNAPSHOT_CHUNK_SIZE;
220
221     protected List<MockPayload> expSnapshotState = new ArrayList<>();
222
223     @After
224     public void tearDown() {
225         InMemoryJournal.clear();
226         InMemorySnapshotStore.clear();
227         factory.close();
228     }
229
230     protected DefaultConfigParamsImpl newLeaderConfigParams() {
231         DefaultConfigParamsImpl configParams = new DefaultConfigParamsImpl();
232         configParams.setHeartBeatInterval(new FiniteDuration(100, TimeUnit.MILLISECONDS));
233         configParams.setElectionTimeoutFactor(4);
234         configParams.setSnapshotBatchCount(snapshotBatchCount);
235         configParams.setSnapshotDataThresholdPercentage(70);
236         configParams.setIsolatedLeaderCheckInterval(new FiniteDuration(1, TimeUnit.DAYS));
237         configParams.setSnapshotChunkSize(snapshotChunkSize);
238         return configParams;
239     }
240
241     protected DefaultConfigParamsImpl newFollowerConfigParams() {
242         DefaultConfigParamsImpl configParams = new DefaultConfigParamsImpl();
243         configParams.setHeartBeatInterval(new FiniteDuration(500, TimeUnit.MILLISECONDS));
244         configParams.setElectionTimeoutFactor(1000);
245         return configParams;
246     }
247
248     protected void waitUntilLeader(final ActorRef actorRef) {
249         RaftActorTestKit.waitUntilLeader(actorRef);
250     }
251
252     protected TestActorRef<TestRaftActor> newTestRaftActor(final String id, final Map<String, String> newPeerAddresses,
253             final ConfigParams configParams) {
254         return newTestRaftActor(id, TestRaftActor.newBuilder().peerAddresses(newPeerAddresses != null
255                 ? newPeerAddresses : Collections.<String, String>emptyMap()).config(configParams));
256     }
257
258     protected TestActorRef<TestRaftActor> newTestRaftActor(final String id, final TestRaftActor.Builder builder) {
259         builder.collectorActor(factory.createActor(
260                 MessageCollectorActor.props(), factory.generateActorId(id + "-collector"))).id(id);
261
262         InvalidActorNameException lastEx = null;
263         for (int i = 0; i < 10; i++) {
264             try {
265                 return factory.createTestActor(builder.props().withDispatcher(Dispatchers.DefaultDispatcherId())
266                         .withMailbox(Mailboxes.DefaultMailboxId()), id);
267             } catch (InvalidActorNameException e) {
268                 lastEx = e;
269                 Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
270             }
271         }
272
273         assertNotNull(lastEx);
274         throw lastEx;
275     }
276
277     protected void killActor(final TestActorRef<TestRaftActor> actor) {
278         TestKit testkit = new TestKit(getSystem());
279         testkit.watch(actor);
280
281         actor.tell(PoisonPill.getInstance(), null);
282         testkit.expectMsgClass(testkit.duration("5 seconds"), Terminated.class);
283
284         testkit.unwatch(actor);
285     }
286
287     protected void verifyApplyJournalEntries(final ActorRef actor, final long expIndex) {
288         MessageCollectorActor.expectFirstMatching(actor, ApplyJournalEntries.class,
289             msg -> msg.getToIndex() == expIndex);
290     }
291
292     protected void verifySnapshot(final String prefix, final Snapshot snapshot, final long lastAppliedTerm,
293             final long lastAppliedIndex, final long lastTerm, final long lastIndex) {
294         assertEquals(prefix + " Snapshot getLastAppliedTerm", lastAppliedTerm, snapshot.getLastAppliedTerm());
295         assertEquals(prefix + " Snapshot getLastAppliedIndex", lastAppliedIndex, snapshot.getLastAppliedIndex());
296         assertEquals(prefix + " Snapshot getLastTerm", lastTerm, snapshot.getLastTerm());
297         assertEquals(prefix + " Snapshot getLastIndex", lastIndex, snapshot.getLastIndex());
298
299         List<Object> actualState = ((MockSnapshotState)snapshot.getState()).getState();
300         assertEquals(String.format("%s Snapshot getState size. Expected %s: . Actual: %s", prefix, expSnapshotState,
301                 actualState), expSnapshotState.size(), actualState.size());
302         for (int i = 0; i < expSnapshotState.size(); i++) {
303             assertEquals(prefix + " Snapshot state " + i, expSnapshotState.get(i), actualState.get(i));
304         }
305     }
306
307     protected void verifyPersistedJournal(final String persistenceId,
308             final List<? extends ReplicatedLogEntry> expJournal) {
309         List<ReplicatedLogEntry> journal = InMemoryJournal.get(persistenceId, ReplicatedLogEntry.class);
310         assertEquals("Journal ReplicatedLogEntry count", expJournal.size(), journal.size());
311         for (int i = 0; i < expJournal.size(); i++) {
312             ReplicatedLogEntry expected = expJournal.get(i);
313             ReplicatedLogEntry actual = journal.get(i);
314             verifyReplicatedLogEntry(expected, actual.getTerm(), actual.getIndex(), actual.getData());
315         }
316     }
317
318     protected MockPayload sendPayloadData(final ActorRef actor, final String data) {
319         return sendPayloadData(actor, data, 0);
320     }
321
322     protected MockPayload sendPayloadData(final ActorRef actor, final String data, final int size) {
323         MockPayload payload;
324         if (size > 0) {
325             payload = new MockPayload(data, size);
326         } else {
327             payload = new MockPayload(data);
328         }
329
330         actor.tell(payload, ActorRef.noSender());
331         return payload;
332     }
333
334     protected void verifyApplyState(final ApplyState applyState, final ActorRef expClientActor,
335             final String expId, final long expTerm, final long expIndex, final Payload payload) {
336         assertEquals("ApplyState getClientActor", expClientActor, applyState.getClientActor());
337
338         final Identifier id = expId == null ? null : new MockIdentifier(expId);
339         assertEquals("ApplyState getIdentifier", id, applyState.getIdentifier());
340         ReplicatedLogEntry replicatedLogEntry = applyState.getReplicatedLogEntry();
341         verifyReplicatedLogEntry(replicatedLogEntry, expTerm, expIndex, payload);
342     }
343
344     protected void verifyReplicatedLogEntry(final ReplicatedLogEntry replicatedLogEntry, final long expTerm,
345             final long expIndex, final Payload payload) {
346         assertEquals("ReplicatedLogEntry getTerm", expTerm, replicatedLogEntry.getTerm());
347         assertEquals("ReplicatedLogEntry getIndex", expIndex, replicatedLogEntry.getIndex());
348         assertEquals("ReplicatedLogEntry getData", payload, replicatedLogEntry.getData());
349     }
350
351     protected String testActorPath(final String id) {
352         return factory.createTestActorPath(id);
353     }
354
355     protected void verifyLeadersTrimmedLog(final long lastIndex) {
356         verifyTrimmedLog("Leader", leaderActor, lastIndex, lastIndex - 1);
357     }
358
359     protected void verifyLeadersTrimmedLog(final long lastIndex, final long replicatedToAllIndex) {
360         verifyTrimmedLog("Leader", leaderActor, lastIndex, replicatedToAllIndex);
361     }
362
363     protected void verifyFollowersTrimmedLog(final int num, final TestActorRef<TestRaftActor> actorRef,
364             final long lastIndex) {
365         verifyTrimmedLog("Follower " + num, actorRef, lastIndex, lastIndex - 1);
366     }
367
368     protected void verifyTrimmedLog(final String name, final TestActorRef<TestRaftActor> actorRef, final long lastIndex,
369             final long replicatedToAllIndex) {
370         TestRaftActor actor = actorRef.underlyingActor();
371         RaftActorContext context = actor.getRaftActorContext();
372         long snapshotIndex = lastIndex - 1;
373         assertEquals(name + " snapshot term", snapshotIndex < 0 ? -1 : currentTerm,
374                 context.getReplicatedLog().getSnapshotTerm());
375         assertEquals(name + " snapshot index", snapshotIndex, context.getReplicatedLog().getSnapshotIndex());
376         assertEquals(name + " journal log size", 1, context.getReplicatedLog().size());
377         assertEquals(name + " journal last index", lastIndex, context.getReplicatedLog().lastIndex());
378         assertEquals(name + " commit index", lastIndex, context.getCommitIndex());
379         assertEquals(name + " last applied", lastIndex, context.getLastApplied());
380         assertEquals(name + " replicatedToAllIndex", replicatedToAllIndex,
381                 actor.getCurrentBehavior().getReplicatedToAllIndex());
382     }
383
384     @SuppressWarnings("checkstyle:IllegalCatch")
385     static void verifyRaftState(final ActorRef raftActor, final Consumer<OnDemandRaftState> verifier) {
386         Timeout timeout = new Timeout(500, TimeUnit.MILLISECONDS);
387         AssertionError lastError = null;
388         Stopwatch sw = Stopwatch.createStarted();
389         while (sw.elapsed(TimeUnit.SECONDS) <= 5) {
390             try {
391                 OnDemandRaftState raftState = (OnDemandRaftState)Await.result(ask(raftActor,
392                         GetOnDemandRaftState.INSTANCE, timeout), timeout.duration());
393                 verifier.accept(raftState);
394                 return;
395             } catch (AssertionError e) {
396                 lastError = e;
397                 Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
398             } catch (Exception e) {
399                 lastError = new AssertionError("OnDemandRaftState failed", e);
400                 Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
401             }
402         }
403
404         throw lastError;
405     }
406 }