Merge "BUG-2953: do not use a complete InMemoryDataStore"
[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.assertArrayEquals;
11 import static org.junit.Assert.assertEquals;
12 import akka.actor.ActorRef;
13 import akka.actor.PoisonPill;
14 import akka.actor.Props;
15 import akka.actor.Terminated;
16 import akka.dispatch.Dispatchers;
17 import akka.testkit.JavaTestKit;
18 import akka.testkit.TestActorRef;
19 import com.google.common.base.Optional;
20 import com.google.common.base.Predicate;
21 import com.google.common.base.Supplier;
22 import com.google.common.collect.ImmutableMap;
23 import java.util.Collections;
24 import java.util.List;
25 import java.util.Map;
26 import java.util.concurrent.ConcurrentHashMap;
27 import java.util.concurrent.TimeUnit;
28 import org.junit.After;
29 import org.opendaylight.controller.cluster.raft.MockRaftActorContext.MockPayload;
30 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
31 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
32 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
33 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
34 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
35 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
36 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
37 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
38 import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor;
39 import org.slf4j.Logger;
40 import org.slf4j.LoggerFactory;
41 import scala.concurrent.duration.FiniteDuration;
42
43 /**
44  * Abstract base for an integration test that tests end-to-end RaftActor and behavior functionality.
45  *
46  * @author Thomas Pantelis
47  */
48 public abstract class AbstractRaftActorIntegrationTest extends AbstractActorTest {
49
50     public static class TestRaftActor extends MockRaftActor {
51
52         private final TestActorRef<MessageCollectorActor> collectorActor;
53         private final Map<Class<?>, Boolean> dropMessages = new ConcurrentHashMap<>();
54         private volatile byte[] snapshot;
55
56         private TestRaftActor(String id, Map<String, String> peerAddresses, ConfigParams config,
57                 TestActorRef<MessageCollectorActor> collectorActor) {
58             super(id, peerAddresses, Optional.of(config), null);
59             this.collectorActor = collectorActor;
60         }
61
62         public static Props props(String id, Map<String, String> peerAddresses, ConfigParams config,
63                 TestActorRef<MessageCollectorActor> collectorActor) {
64             return Props.create(TestRaftActor.class, id, peerAddresses, config, collectorActor).
65                     withDispatcher(Dispatchers.DefaultDispatcherId());
66         }
67
68         void startDropMessages(Class<?> msgClass) {
69             dropMessages.put(msgClass, Boolean.TRUE);
70         }
71
72         void stopDropMessages(Class<?> msgClass) {
73             dropMessages.remove(msgClass);
74         }
75
76         void setMockTotalMemory(final long mockTotalMemory) {
77             if(mockTotalMemory > 0) {
78                 getRaftActorContext().setTotalMemoryRetriever(new Supplier<Long>() {
79                     @Override
80                     public Long get() {
81                         return mockTotalMemory;
82                     }
83
84                 });
85             } else {
86                 getRaftActorContext().setTotalMemoryRetriever(null);
87             }
88         }
89
90         @Override
91         public void handleCommand(Object message) {
92             if(message instanceof MockPayload) {
93                 MockPayload payload = (MockPayload)message;
94                 super.persistData(collectorActor, payload.toString(), payload);
95                 return;
96             }
97
98             try {
99                 if(!dropMessages.containsKey(message.getClass())) {
100                     super.handleCommand(message);
101                 }
102             } finally {
103                 if(!(message instanceof SendHeartBeat)) {
104                     try {
105                         collectorActor.tell(message, ActorRef.noSender());
106                     } catch (Exception e) {
107                         LOG.error("MessageCollectorActor error", e);
108                     }
109                 }
110             }
111         }
112
113         @Override
114         public void createSnapshot(ActorRef actorRef) {
115             if(snapshot != null) {
116                 getSelf().tell(new CaptureSnapshotReply(snapshot), ActorRef.noSender());
117             }
118         }
119
120         @Override
121         public void applyRecoverySnapshot(byte[] bytes) {
122         }
123
124         void setSnapshot(byte[] snapshot) {
125             this.snapshot = snapshot;
126         }
127
128         public ActorRef collectorActor() {
129             return collectorActor;
130         }
131     }
132
133     protected final Logger testLog = LoggerFactory.getLogger(getClass());
134
135     protected final TestActorFactory factory = new TestActorFactory(getSystem());
136
137     protected String leaderId = factory.generateActorId("leader");
138     protected DefaultConfigParamsImpl leaderConfigParams;
139     protected TestActorRef<TestRaftActor> leaderActor;
140     protected ActorRef leaderCollectorActor;
141     protected RaftActorContext leaderContext;
142     protected RaftActorBehavior leader;
143
144     protected String follower1Id = factory.generateActorId("follower");
145     protected TestActorRef<TestRaftActor> follower1Actor;
146     protected ActorRef follower1CollectorActor;
147     protected RaftActorBehavior follower1;
148     protected RaftActorContext follower1Context;
149
150     protected String follower2Id = factory.generateActorId("follower");
151     protected TestActorRef<TestRaftActor> follower2Actor;
152     protected ActorRef follower2CollectorActor;
153     protected  RaftActorBehavior follower2;
154     protected RaftActorContext follower2Context;
155
156     protected ImmutableMap<String, String> peerAddresses;
157
158     protected long initialTerm = 5;
159     protected long currentTerm;
160
161     @After
162     public void tearDown() {
163         InMemoryJournal.clear();
164         InMemorySnapshotStore.clear();
165         factory.close();
166     }
167
168     protected DefaultConfigParamsImpl newLeaderConfigParams() {
169         DefaultConfigParamsImpl configParams = new DefaultConfigParamsImpl();
170         configParams.setHeartBeatInterval(new FiniteDuration(100, TimeUnit.MILLISECONDS));
171         configParams.setElectionTimeoutFactor(1);
172         configParams.setSnapshotBatchCount(4);
173         configParams.setSnapshotDataThresholdPercentage(70);
174         configParams.setIsolatedLeaderCheckInterval(new FiniteDuration(1, TimeUnit.DAYS));
175         return configParams;
176     }
177
178     protected DefaultConfigParamsImpl newFollowerConfigParams() {
179         DefaultConfigParamsImpl configParams = new DefaultConfigParamsImpl();
180         configParams.setHeartBeatInterval(new FiniteDuration(500, TimeUnit.MILLISECONDS));
181         configParams.setElectionTimeoutFactor(1000);
182         return configParams;
183     }
184
185     protected void waitUntilLeader(ActorRef actorRef) {
186         RaftActorTestKit.waitUntilLeader(actorRef);
187     }
188
189     protected TestActorRef<TestRaftActor> newTestRaftActor(String id, Map<String, String> peerAddresses,
190             ConfigParams configParams) {
191         TestActorRef<MessageCollectorActor> collectorActor = factory.createTestActor(
192                 MessageCollectorActor.props().withDispatcher(Dispatchers.DefaultDispatcherId()),
193                         factory.generateActorId(id + "-collector"));
194         return factory.createTestActor(TestRaftActor.props(id,
195                 peerAddresses != null ? peerAddresses : Collections.<String, String>emptyMap(),
196                         configParams, collectorActor), id);
197     }
198
199     protected void killActor(TestActorRef<TestRaftActor> leaderActor) {
200         JavaTestKit testkit = new JavaTestKit(getSystem());
201         testkit.watch(leaderActor);
202
203         leaderActor.tell(PoisonPill.getInstance(), null);
204         testkit.expectMsgClass(JavaTestKit.duration("5 seconds"), Terminated.class);
205
206         testkit.unwatch(leaderActor);
207     }
208
209     protected void verifyApplyJournalEntries(ActorRef actor, final long expIndex) {
210         MessageCollectorActor.expectFirstMatching(actor, ApplyJournalEntries.class, new Predicate<ApplyJournalEntries>() {
211             @Override
212             public boolean apply(ApplyJournalEntries msg) {
213                 return msg.getToIndex() == expIndex;
214             }
215         });
216     }
217
218     protected void verifySnapshot(String prefix, Snapshot snapshot, long lastAppliedTerm,
219             int lastAppliedIndex, long lastTerm, long lastIndex, byte[] data) {
220         assertEquals(prefix + " Snapshot getLastAppliedTerm", lastAppliedTerm, snapshot.getLastAppliedTerm());
221         assertEquals(prefix + " Snapshot getLastAppliedIndex", lastAppliedIndex, snapshot.getLastAppliedIndex());
222         assertEquals(prefix + " Snapshot getLastTerm", lastTerm, snapshot.getLastTerm());
223         assertEquals(prefix + " Snapshot getLastIndex", lastIndex, snapshot.getLastIndex());
224         assertArrayEquals(prefix + " Snapshot getState", data, snapshot.getState());
225     }
226
227     protected void verifyPersistedJournal(String persistenceId, List<? extends ReplicatedLogEntry> expJournal) {
228         List<ReplicatedLogEntry> journal = InMemoryJournal.get(persistenceId, ReplicatedLogEntry.class);
229         assertEquals("Journal ReplicatedLogEntry count", expJournal.size(), journal.size());
230         for(int i = 0; i < expJournal.size(); i++) {
231             ReplicatedLogEntry expected = expJournal.get(i);
232             ReplicatedLogEntry actual = journal.get(i);
233             verifyReplicatedLogEntry(expected, actual.getTerm(), actual.getIndex(), actual.getData());
234         }
235     }
236
237     protected MockPayload sendPayloadData(ActorRef leaderActor, String data) {
238         return sendPayloadData(leaderActor, data, 0);
239     }
240
241     protected MockPayload sendPayloadData(ActorRef leaderActor, String data, int size) {
242         MockPayload payload;
243         if(size > 0) {
244             payload = new MockPayload(data, size);
245         } else {
246             payload = new MockPayload(data);
247         }
248
249         leaderActor.tell(payload, ActorRef.noSender());
250         return payload;
251     }
252
253     protected void verifyApplyState(ApplyState applyState, ActorRef expClientActor,
254             String expId, long expTerm, long expIndex, MockPayload payload) {
255         assertEquals("ApplyState getClientActor", expClientActor, applyState.getClientActor());
256         assertEquals("ApplyState getIdentifier", expId, applyState.getIdentifier());
257         ReplicatedLogEntry replicatedLogEntry = applyState.getReplicatedLogEntry();
258         verifyReplicatedLogEntry(replicatedLogEntry, expTerm, expIndex, payload);
259     }
260
261     protected void verifyReplicatedLogEntry(ReplicatedLogEntry replicatedLogEntry, long expTerm, long expIndex,
262             Payload payload) {
263         assertEquals("ReplicatedLogEntry getTerm", expTerm, replicatedLogEntry.getTerm());
264         assertEquals("ReplicatedLogEntry getIndex", expIndex, replicatedLogEntry.getIndex());
265         assertEquals("ReplicatedLogEntry getData", payload, replicatedLogEntry.getData());
266     }
267
268     protected String testActorPath(String id){
269         return "akka://test/user" + id;
270     }
271 }