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