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