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