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