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