Cleanup and document FindLeaderReply
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / ShardTest.java
1 /*
2  * Copyright (c) 2014, 2015 Cisco 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
9 package org.opendaylight.controller.cluster.datastore;
10
11 import static org.junit.Assert.assertEquals;
12 import static org.junit.Assert.assertFalse;
13 import static org.junit.Assert.assertNotNull;
14 import static org.junit.Assert.assertSame;
15 import static org.junit.Assert.assertTrue;
16 import static org.mockito.Mockito.doReturn;
17 import static org.mockito.Mockito.inOrder;
18 import static org.mockito.Mockito.mock;
19 import static org.mockito.Mockito.reset;
20 import static org.mockito.Mockito.verify;
21 import static org.opendaylight.controller.cluster.datastore.DataStoreVersions.CURRENT_VERSION;
22 import akka.actor.ActorRef;
23 import akka.actor.ActorSelection;
24 import akka.actor.Props;
25 import akka.actor.Status.Failure;
26 import akka.dispatch.Dispatchers;
27 import akka.dispatch.OnComplete;
28 import akka.japi.Creator;
29 import akka.pattern.Patterns;
30 import akka.persistence.SaveSnapshotSuccess;
31 import akka.testkit.TestActorRef;
32 import akka.util.Timeout;
33 import com.google.common.base.Function;
34 import com.google.common.util.concurrent.Futures;
35 import com.google.common.util.concurrent.ListenableFuture;
36 import com.google.common.util.concurrent.Uninterruptibles;
37 import java.util.Collections;
38 import java.util.HashSet;
39 import java.util.Set;
40 import java.util.concurrent.CountDownLatch;
41 import java.util.concurrent.TimeUnit;
42 import java.util.concurrent.atomic.AtomicBoolean;
43 import java.util.concurrent.atomic.AtomicReference;
44 import org.junit.Test;
45 import org.mockito.InOrder;
46 import org.opendaylight.controller.cluster.DataPersistenceProvider;
47 import org.opendaylight.controller.cluster.DelegatingPersistentDataProvider;
48 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
49 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
50 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
51 import org.opendaylight.controller.cluster.datastore.messages.AbortTransaction;
52 import org.opendaylight.controller.cluster.datastore.messages.AbortTransactionReply;
53 import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
54 import org.opendaylight.controller.cluster.datastore.messages.BatchedModificationsReply;
55 import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransaction;
56 import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransactionReply;
57 import org.opendaylight.controller.cluster.datastore.messages.CommitTransaction;
58 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
59 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
60 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
61 import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
62 import org.opendaylight.controller.cluster.datastore.messages.ReadData;
63 import org.opendaylight.controller.cluster.datastore.messages.ReadDataReply;
64 import org.opendaylight.controller.cluster.datastore.messages.ReadyLocalTransaction;
65 import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
66 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListener;
67 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListenerReply;
68 import org.opendaylight.controller.cluster.datastore.messages.RegisterDataTreeChangeListener;
69 import org.opendaylight.controller.cluster.datastore.messages.RegisterDataTreeChangeListenerReply;
70 import org.opendaylight.controller.cluster.datastore.messages.ShardLeaderStateChanged;
71 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
72 import org.opendaylight.controller.cluster.datastore.modification.DeleteModification;
73 import org.opendaylight.controller.cluster.datastore.modification.MergeModification;
74 import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
75 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
76 import org.opendaylight.controller.cluster.datastore.utils.MockDataChangeListener;
77 import org.opendaylight.controller.cluster.datastore.utils.MockDataTreeChangeListener;
78 import org.opendaylight.controller.cluster.datastore.utils.SerializationUtils;
79 import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListener;
80 import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListenerReply;
81 import org.opendaylight.controller.cluster.raft.RaftActorContext;
82 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
83 import org.opendaylight.controller.cluster.raft.ReplicatedLogImplEntry;
84 import org.opendaylight.controller.cluster.raft.Snapshot;
85 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
86 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
87 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
88 import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus;
89 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
90 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
91 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
92 import org.opendaylight.controller.cluster.raft.messages.ServerRemoved;
93 import org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy;
94 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
95 import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor;
96 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
97 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
98 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataBroker;
99 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
100 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
101 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
102 import org.opendaylight.yangtools.yang.data.api.schema.MapNode;
103 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
104 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
105 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidateNode;
106 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidateTip;
107 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification;
108 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException;
109 import org.opendaylight.yangtools.yang.data.api.schema.tree.ModificationType;
110 import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType;
111 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
112 import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder;
113 import org.opendaylight.yangtools.yang.data.impl.schema.tree.InMemoryDataTreeFactory;
114 import scala.concurrent.Await;
115 import scala.concurrent.Future;
116 import scala.concurrent.duration.FiniteDuration;
117
118 public class ShardTest extends AbstractShardTest {
119     private static final String DUMMY_DATA = "Dummy data as snapshot sequence number is set to 0 in InMemorySnapshotStore and journal recovery seq number will start from 1";
120
121     @Test
122     public void testRegisterChangeListener() throws Exception {
123         new ShardTestKit(getSystem()) {{
124             final TestActorRef<Shard> shard = actorFactory.createTestActor(
125                     newShardProps(),  "testRegisterChangeListener");
126
127             waitUntilLeader(shard);
128
129             shard.tell(new UpdateSchemaContext(SchemaContextHelper.full()), ActorRef.noSender());
130
131             final MockDataChangeListener listener = new MockDataChangeListener(1);
132             final ActorRef dclActor = actorFactory.createActor(DataChangeListener.props(listener),
133                     "testRegisterChangeListener-DataChangeListener");
134
135             shard.tell(new RegisterChangeListener(TestModel.TEST_PATH,
136                     dclActor, AsyncDataBroker.DataChangeScope.BASE, true), getRef());
137
138             final RegisterChangeListenerReply reply = expectMsgClass(duration("3 seconds"),
139                     RegisterChangeListenerReply.class);
140             final String replyPath = reply.getListenerRegistrationPath().toString();
141             assertTrue("Incorrect reply path: " + replyPath, replyPath.matches(
142                     "akka:\\/\\/test\\/user\\/testRegisterChangeListener\\/\\$.*"));
143
144             final YangInstanceIdentifier path = TestModel.TEST_PATH;
145             writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
146
147             listener.waitForChangeEvents(path);
148         }};
149     }
150
151     @SuppressWarnings("serial")
152     @Test
153     public void testChangeListenerNotifiedWhenNotTheLeaderOnRegistration() throws Exception {
154         // This test tests the timing window in which a change listener is registered before the
155         // shard becomes the leader. We verify that the listener is registered and notified of the
156         // existing data when the shard becomes the leader.
157         new ShardTestKit(getSystem()) {{
158             // For this test, we want to send the RegisterChangeListener message after the shard
159             // has recovered from persistence and before it becomes the leader. So we subclass
160             // Shard to override onReceiveCommand and, when the first ElectionTimeout is received,
161             // we know that the shard has been initialized to a follower and has started the
162             // election process. The following 2 CountDownLatches are used to coordinate the
163             // ElectionTimeout with the sending of the RegisterChangeListener message.
164             final CountDownLatch onFirstElectionTimeout = new CountDownLatch(1);
165             final CountDownLatch onChangeListenerRegistered = new CountDownLatch(1);
166             final Creator<Shard> creator = new Creator<Shard>() {
167                 boolean firstElectionTimeout = true;
168
169                 @Override
170                 public Shard create() throws Exception {
171                     // Use a non persistent provider because this test actually invokes persist on the journal
172                     // this will cause all other messages to not be queued properly after that.
173                     // The basic issue is that you cannot use TestActorRef with a persistent actor (at least when
174                     // it does do a persist)
175                     return new Shard(newShardBuilder()) {
176                         @Override
177                         public void onReceiveCommand(final Object message) throws Exception {
178                             if(message instanceof ElectionTimeout && firstElectionTimeout) {
179                                 // Got the first ElectionTimeout. We don't forward it to the
180                                 // base Shard yet until we've sent the RegisterChangeListener
181                                 // message. So we signal the onFirstElectionTimeout latch to tell
182                                 // the main thread to send the RegisterChangeListener message and
183                                 // start a thread to wait on the onChangeListenerRegistered latch,
184                                 // which the main thread signals after it has sent the message.
185                                 // After the onChangeListenerRegistered is triggered, we send the
186                                 // original ElectionTimeout message to proceed with the election.
187                                 firstElectionTimeout = false;
188                                 final ActorRef self = getSelf();
189                                 new Thread() {
190                                     @Override
191                                     public void run() {
192                                         Uninterruptibles.awaitUninterruptibly(
193                                                 onChangeListenerRegistered, 5, TimeUnit.SECONDS);
194                                         self.tell(message, self);
195                                     }
196                                 }.start();
197
198                                 onFirstElectionTimeout.countDown();
199                             } else {
200                                 super.onReceiveCommand(message);
201                             }
202                         }
203                     };
204                 }
205             };
206
207             final MockDataChangeListener listener = new MockDataChangeListener(1);
208             final ActorRef dclActor = actorFactory.createActor(DataChangeListener.props(listener),
209                     "testRegisterChangeListenerWhenNotLeaderInitially-DataChangeListener");
210
211             final TestActorRef<Shard> shard = actorFactory.createTestActor(
212                     Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
213                     "testRegisterChangeListenerWhenNotLeaderInitially");
214
215             // Write initial data into the in-memory store.
216             final YangInstanceIdentifier path = TestModel.TEST_PATH;
217             writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
218
219             // Wait until the shard receives the first ElectionTimeout message.
220             assertEquals("Got first ElectionTimeout", true,
221                     onFirstElectionTimeout.await(5, TimeUnit.SECONDS));
222
223             // Now send the RegisterChangeListener and wait for the reply.
224             shard.tell(new RegisterChangeListener(path, dclActor,
225                     AsyncDataBroker.DataChangeScope.SUBTREE, false), getRef());
226
227             final RegisterChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
228                     RegisterChangeListenerReply.class);
229             assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
230
231             // Sanity check - verify the shard is not the leader yet.
232             shard.tell(FindLeader.INSTANCE, getRef());
233             final FindLeaderReply findLeadeReply =
234                     expectMsgClass(duration("5 seconds"), FindLeaderReply.class);
235             assertFalse("Expected the shard not to be the leader", findLeadeReply.getLeaderActor().isPresent());
236
237             // Signal the onChangeListenerRegistered latch to tell the thread above to proceed
238             // with the election process.
239             onChangeListenerRegistered.countDown();
240
241             // Wait for the shard to become the leader and notify our listener with the existing
242             // data in the store.
243             listener.waitForChangeEvents(path);
244         }};
245     }
246
247     @Test
248     public void testRegisterDataTreeChangeListener() throws Exception {
249         new ShardTestKit(getSystem()) {{
250             final TestActorRef<Shard> shard = actorFactory.createTestActor(
251                     newShardProps(), "testRegisterDataTreeChangeListener");
252
253             waitUntilLeader(shard);
254
255             shard.tell(new UpdateSchemaContext(SchemaContextHelper.full()), ActorRef.noSender());
256
257             final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
258             final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener),
259                     "testRegisterDataTreeChangeListener-DataTreeChangeListener");
260
261             shard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor, false), getRef());
262
263             final RegisterDataTreeChangeListenerReply reply = expectMsgClass(duration("3 seconds"),
264                     RegisterDataTreeChangeListenerReply.class);
265             final String replyPath = reply.getListenerRegistrationPath().toString();
266             assertTrue("Incorrect reply path: " + replyPath, replyPath.matches(
267                     "akka:\\/\\/test\\/user\\/testRegisterDataTreeChangeListener\\/\\$.*"));
268
269             final YangInstanceIdentifier path = TestModel.TEST_PATH;
270             writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
271
272             listener.waitForChangeEvents();
273         }};
274     }
275
276     @SuppressWarnings("serial")
277     @Test
278     public void testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration() throws Exception {
279         new ShardTestKit(getSystem()) {{
280             final CountDownLatch onFirstElectionTimeout = new CountDownLatch(1);
281             final CountDownLatch onChangeListenerRegistered = new CountDownLatch(1);
282             final Creator<Shard> creator = new Creator<Shard>() {
283                 boolean firstElectionTimeout = true;
284
285                 @Override
286                 public Shard create() throws Exception {
287                     return new Shard(Shard.builder().id(shardID).datastoreContext(
288                             dataStoreContextBuilder.persistent(false).build()).schemaContext(SCHEMA_CONTEXT)) {
289                         @Override
290                         public void onReceiveCommand(final Object message) throws Exception {
291                             if(message instanceof ElectionTimeout && firstElectionTimeout) {
292                                 firstElectionTimeout = false;
293                                 final ActorRef self = getSelf();
294                                 new Thread() {
295                                     @Override
296                                     public void run() {
297                                         Uninterruptibles.awaitUninterruptibly(
298                                                 onChangeListenerRegistered, 5, TimeUnit.SECONDS);
299                                         self.tell(message, self);
300                                     }
301                                 }.start();
302
303                                 onFirstElectionTimeout.countDown();
304                             } else {
305                                 super.onReceiveCommand(message);
306                             }
307                         }
308                     };
309                 }
310             };
311
312             final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
313             final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener),
314                     "testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration-DataChangeListener");
315
316             final TestActorRef<Shard> shard = actorFactory.createTestActor(
317                     Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
318                     "testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration");
319
320             final YangInstanceIdentifier path = TestModel.TEST_PATH;
321             writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
322
323             assertEquals("Got first ElectionTimeout", true,
324                 onFirstElectionTimeout.await(5, TimeUnit.SECONDS));
325
326             shard.tell(new RegisterDataTreeChangeListener(path, dclActor, false), getRef());
327             final RegisterDataTreeChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
328                 RegisterDataTreeChangeListenerReply.class);
329             assertNotNull("getListenerRegistratioznPath", reply.getListenerRegistrationPath());
330
331             shard.tell(FindLeader.INSTANCE, getRef());
332             final FindLeaderReply findLeadeReply =
333                     expectMsgClass(duration("5 seconds"), FindLeaderReply.class);
334             assertFalse("Expected the shard not to be the leader", findLeadeReply.getLeaderActor().isPresent());
335
336             writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
337
338             onChangeListenerRegistered.countDown();
339
340             // TODO: investigate why we do not receive data chage events
341             listener.waitForChangeEvents();
342         }};
343     }
344
345     @Test
346     public void testCreateTransaction(){
347         new ShardTestKit(getSystem()) {{
348             final ActorRef shard = actorFactory.createActor(newShardProps(), "testCreateTransaction");
349
350             waitUntilLeader(shard);
351
352             shard.tell(new UpdateSchemaContext(TestModel.createTestContext()), getRef());
353
354             shard.tell(new CreateTransaction("txn-1", TransactionType.READ_ONLY.ordinal(), null,
355                     DataStoreVersions.CURRENT_VERSION).toSerializable(), getRef());
356
357             final CreateTransactionReply reply = expectMsgClass(duration("3 seconds"),
358                     CreateTransactionReply.class);
359
360             final String path = reply.getTransactionPath().toString();
361             assertTrue("Unexpected transaction path " + path,
362                     path.contains("akka://test/user/testCreateTransaction/shard-txn-1"));
363         }};
364     }
365
366     @Test
367     public void testCreateTransactionOnChain(){
368         new ShardTestKit(getSystem()) {{
369             final ActorRef shard = actorFactory.createActor(newShardProps(), "testCreateTransactionOnChain");
370
371             waitUntilLeader(shard);
372
373             shard.tell(new CreateTransaction("txn-1",TransactionType.READ_ONLY.ordinal(), "foobar",
374                     DataStoreVersions.CURRENT_VERSION).toSerializable(), getRef());
375
376             final CreateTransactionReply reply = expectMsgClass(duration("3 seconds"),
377                     CreateTransactionReply.class);
378
379             final String path = reply.getTransactionPath().toString();
380             assertTrue("Unexpected transaction path " + path,
381                     path.contains("akka://test/user/testCreateTransactionOnChain/shard-txn-1"));
382         }};
383     }
384
385     @SuppressWarnings("serial")
386     @Test
387     public void testPeerAddressResolved() throws Exception {
388         new ShardTestKit(getSystem()) {{
389             final CountDownLatch recoveryComplete = new CountDownLatch(1);
390             class TestShard extends Shard {
391                 TestShard() {
392                     super(Shard.builder().id(shardID).datastoreContext(newDatastoreContext()).
393                             peerAddresses(Collections.<String, String>singletonMap(shardID.toString(), null)).
394                             schemaContext(SCHEMA_CONTEXT));
395                 }
396
397                 String getPeerAddress(String id) {
398                     return getRaftActorContext().getPeerAddress(id);
399                 }
400
401                 @Override
402                 protected void onRecoveryComplete() {
403                     try {
404                         super.onRecoveryComplete();
405                     } finally {
406                         recoveryComplete.countDown();
407                     }
408                 }
409             }
410
411             final TestActorRef<Shard> shard = actorFactory.createTestActor(
412                     Props.create(new DelegatingShardCreator(new Creator<Shard>() {
413                         @Override
414                         public TestShard create() throws Exception {
415                             return new TestShard();
416                         }
417                     })), "testPeerAddressResolved");
418
419             assertEquals("Recovery complete", true,
420                 Uninterruptibles.awaitUninterruptibly(recoveryComplete, 5, TimeUnit.SECONDS));
421
422             final String address = "akka://foobar";
423             shard.underlyingActor().onReceiveCommand(new PeerAddressResolved(shardID.toString(), address));
424
425             assertEquals("getPeerAddress", address,
426                 ((TestShard) shard.underlyingActor()).getPeerAddress(shardID.toString()));
427         }};
428     }
429
430     @Test
431     public void testApplySnapshot() throws Exception {
432
433         final TestActorRef<Shard> shard = actorFactory.createTestActor(newShardProps(), "testApplySnapshot");
434
435         ShardTestKit.waitUntilLeader(shard);
436
437         final DataTree store = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL);
438         store.setSchemaContext(SCHEMA_CONTEXT);
439
440         final ContainerNode container = ImmutableContainerNodeBuilder.create().withNodeIdentifier(
441                 new YangInstanceIdentifier.NodeIdentifier(TestModel.TEST_QNAME)).
442                     withChild(ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).addChild(
443                         ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1)).build()).build();
444
445         writeToStore(store, TestModel.TEST_PATH, container);
446
447         final YangInstanceIdentifier root = YangInstanceIdentifier.builder().build();
448         final NormalizedNode<?,?> expected = readStore(store, root);
449
450         final Snapshot snapshot = Snapshot.create(SerializationUtils.serializeNormalizedNode(expected),
451                 Collections.<ReplicatedLogEntry>emptyList(), 1, 2, 3, 4);
452
453         shard.underlyingActor().getRaftActorSnapshotCohort().applySnapshot(snapshot.getState());
454
455         final NormalizedNode<?,?> actual = readStore(shard, root);
456
457         assertEquals("Root node", expected, actual);
458     }
459
460     @Test
461     public void testApplyState() throws Exception {
462         final TestActorRef<Shard> shard = actorFactory.createTestActor(newShardProps(), "testApplyState");
463
464         ShardTestKit.waitUntilLeader(shard);
465
466         final DataTree source = setupInMemorySnapshotStore();
467         final DataTreeModification writeMod = source.takeSnapshot().newModification();
468         ContainerNode node = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
469         writeMod.write(TestModel.TEST_PATH, node);
470         writeMod.ready();
471
472         final ApplyState applyState = new ApplyState(null, "test", new ReplicatedLogImplEntry(1, 2,
473                 payloadForModification(source, writeMod)));
474
475         shard.underlyingActor().onReceiveCommand(applyState);
476
477         final NormalizedNode<?,?> actual = readStore(shard, TestModel.TEST_PATH);
478         assertEquals("Applied state", node, actual);
479     }
480
481     @Test
482     public void testDataTreeCandidateRecovery() throws Exception {
483         // Set up the InMemorySnapshotStore.
484         final DataTree source = setupInMemorySnapshotStore();
485
486         final DataTreeModification writeMod = source.takeSnapshot().newModification();
487         writeMod.write(TestModel.OUTER_LIST_PATH, ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build());
488         writeMod.ready();
489         InMemoryJournal.addEntry(shardID.toString(), 0, DUMMY_DATA);
490
491         // Set up the InMemoryJournal.
492         InMemoryJournal.addEntry(shardID.toString(), 1, new ReplicatedLogImplEntry(0, 1, payloadForModification(source, writeMod)));
493
494         final int nListEntries = 16;
495         final Set<Integer> listEntryKeys = new HashSet<>();
496
497         // Add some ModificationPayload entries
498         for (int i = 1; i <= nListEntries; i++) {
499             listEntryKeys.add(Integer.valueOf(i));
500
501             final YangInstanceIdentifier path = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
502                     .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i).build();
503
504             final DataTreeModification mod = source.takeSnapshot().newModification();
505             mod.merge(path, ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i));
506             mod.ready();
507             InMemoryJournal.addEntry(shardID.toString(), i+1, new ReplicatedLogImplEntry(i, 1,
508                 payloadForModification(source, mod)));
509         }
510
511         InMemoryJournal.addEntry(shardID.toString(), nListEntries + 2,
512             new ApplyJournalEntries(nListEntries));
513
514         testRecovery(listEntryKeys);
515     }
516
517     @Test
518     public void testConcurrentThreePhaseCommits() throws Throwable {
519         new ShardTestKit(getSystem()) {{
520             final TestActorRef<Shard> shard = actorFactory.createTestActor(
521                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
522                     "testConcurrentThreePhaseCommits");
523
524             waitUntilLeader(shard);
525
526             // Setup 3 simulated transactions with mock cohorts backed by real cohorts.
527
528             final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
529
530             final String transactionID1 = "tx1";
531             final MutableCompositeModification modification1 = new MutableCompositeModification();
532             final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore,
533                     TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1);
534
535             final String transactionID2 = "tx2";
536             final MutableCompositeModification modification2 = new MutableCompositeModification();
537             final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore,
538                     TestModel.OUTER_LIST_PATH,
539                     ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(),
540                     modification2);
541
542             final String transactionID3 = "tx3";
543             final MutableCompositeModification modification3 = new MutableCompositeModification();
544             final ShardDataTreeCohort cohort3 = setupMockWriteTransaction("cohort3", dataStore,
545                     YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
546                         .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(),
547                     ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1),
548                     modification3);
549
550             final long timeoutSec = 5;
551             final FiniteDuration duration = FiniteDuration.create(timeoutSec, TimeUnit.SECONDS);
552             final Timeout timeout = new Timeout(duration);
553
554             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
555             final ReadyTransactionReply readyReply = ReadyTransactionReply.fromSerializable(
556                     expectMsgClass(duration, ReadyTransactionReply.class));
557             assertEquals("Cohort path", shard.path().toString(), readyReply.getCohortPath());
558
559             // Send the CanCommitTransaction message for the first Tx.
560
561             shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
562             final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
563                     expectMsgClass(duration, CanCommitTransactionReply.class));
564             assertEquals("Can commit", true, canCommitReply.getCanCommit());
565
566             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
567             expectMsgClass(duration, ReadyTransactionReply.class);
568
569             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort3, transactionID3, modification3), getRef());
570             expectMsgClass(duration, ReadyTransactionReply.class);
571
572             // Send the CanCommitTransaction message for the next 2 Tx's. These should get queued and
573             // processed after the first Tx completes.
574
575             final Future<Object> canCommitFuture1 = Patterns.ask(shard,
576                     new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), timeout);
577
578             final Future<Object> canCommitFuture2 = Patterns.ask(shard,
579                     new CanCommitTransaction(transactionID3, CURRENT_VERSION).toSerializable(), timeout);
580
581             // Send the CommitTransaction message for the first Tx. After it completes, it should
582             // trigger the 2nd Tx to proceed which should in turn then trigger the 3rd.
583
584             shard.tell(new CommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
585             expectMsgClass(duration, CommitTransactionReply.class);
586
587             // Wait for the next 2 Tx's to complete.
588
589             final AtomicReference<Throwable> caughtEx = new AtomicReference<>();
590             final CountDownLatch commitLatch = new CountDownLatch(2);
591
592             class OnFutureComplete extends OnComplete<Object> {
593                 private final Class<?> expRespType;
594
595                 OnFutureComplete(final Class<?> expRespType) {
596                     this.expRespType = expRespType;
597                 }
598
599                 @Override
600                 public void onComplete(final Throwable error, final Object resp) {
601                     if(error != null) {
602                         caughtEx.set(new AssertionError(getClass().getSimpleName() + " failure", error));
603                     } else {
604                         try {
605                             assertEquals("Commit response type", expRespType, resp.getClass());
606                             onSuccess(resp);
607                         } catch (final Exception e) {
608                             caughtEx.set(e);
609                         }
610                     }
611                 }
612
613                 void onSuccess(final Object resp) throws Exception {
614                 }
615             }
616
617             class OnCommitFutureComplete extends OnFutureComplete {
618                 OnCommitFutureComplete() {
619                     super(CommitTransactionReply.class);
620                 }
621
622                 @Override
623                 public void onComplete(final Throwable error, final Object resp) {
624                     super.onComplete(error, resp);
625                     commitLatch.countDown();
626                 }
627             }
628
629             class OnCanCommitFutureComplete extends OnFutureComplete {
630                 private final String transactionID;
631
632                 OnCanCommitFutureComplete(final String transactionID) {
633                     super(CanCommitTransactionReply.class);
634                     this.transactionID = transactionID;
635                 }
636
637                 @Override
638                 void onSuccess(final Object resp) throws Exception {
639                     final CanCommitTransactionReply canCommitReply =
640                             CanCommitTransactionReply.fromSerializable(resp);
641                     assertEquals("Can commit", true, canCommitReply.getCanCommit());
642
643                     final Future<Object> commitFuture = Patterns.ask(shard,
644                             new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), timeout);
645                     commitFuture.onComplete(new OnCommitFutureComplete(), getSystem().dispatcher());
646                 }
647             }
648
649             canCommitFuture1.onComplete(new OnCanCommitFutureComplete(transactionID2),
650                     getSystem().dispatcher());
651
652             canCommitFuture2.onComplete(new OnCanCommitFutureComplete(transactionID3),
653                     getSystem().dispatcher());
654
655             final boolean done = commitLatch.await(timeoutSec, TimeUnit.SECONDS);
656
657             if(caughtEx.get() != null) {
658                 throw caughtEx.get();
659             }
660
661             assertEquals("Commits complete", true, done);
662
663             final InOrder inOrder = inOrder(cohort1, cohort2, cohort3);
664             inOrder.verify(cohort1).canCommit();
665             inOrder.verify(cohort1).preCommit();
666             inOrder.verify(cohort1).commit();
667             inOrder.verify(cohort2).canCommit();
668             inOrder.verify(cohort2).preCommit();
669             inOrder.verify(cohort2).commit();
670             inOrder.verify(cohort3).canCommit();
671             inOrder.verify(cohort3).preCommit();
672             inOrder.verify(cohort3).commit();
673
674             // Verify data in the data store.
675
676             verifyOuterListEntry(shard, 1);
677
678             verifyLastApplied(shard, 2);
679         }};
680     }
681
682     @Test
683     public void testBatchedModificationsWithNoCommitOnReady() throws Throwable {
684         new ShardTestKit(getSystem()) {{
685             final TestActorRef<Shard> shard = actorFactory.createTestActor(
686                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
687                     "testBatchedModificationsWithNoCommitOnReady");
688
689             waitUntilLeader(shard);
690
691             final String transactionID = "tx";
692             final FiniteDuration duration = duration("5 seconds");
693
694             final AtomicReference<ShardDataTreeCohort> mockCohort = new AtomicReference<>();
695             final ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() {
696                 @Override
697                 public ShardDataTreeCohort decorate(final String txID, final ShardDataTreeCohort actual) {
698                     if(mockCohort.get() == null) {
699                         mockCohort.set(createDelegatingMockCohort("cohort", actual));
700                     }
701
702                     return mockCohort.get();
703                 }
704             };
705
706             shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator);
707
708             // Send a BatchedModifications to start a transaction.
709
710             shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH,
711                     ImmutableNodes.containerNode(TestModel.TEST_QNAME), false, false, 1), getRef());
712             expectMsgClass(duration, BatchedModificationsReply.class);
713
714             // Send a couple more BatchedModifications.
715
716             shard.tell(newBatchedModifications(transactionID, TestModel.OUTER_LIST_PATH,
717                     ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), false, false, 2), getRef());
718             expectMsgClass(duration, BatchedModificationsReply.class);
719
720             shard.tell(newBatchedModifications(transactionID, YangInstanceIdentifier.builder(
721                     TestModel.OUTER_LIST_PATH).nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(),
722                     ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1), true, false, 3), getRef());
723             expectMsgClass(duration, ReadyTransactionReply.class);
724
725             // Send the CanCommitTransaction message.
726
727             shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
728             final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
729                     expectMsgClass(duration, CanCommitTransactionReply.class));
730             assertEquals("Can commit", true, canCommitReply.getCanCommit());
731
732             // Send the CanCommitTransaction message.
733
734             shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
735             expectMsgClass(duration, CommitTransactionReply.class);
736
737             final InOrder inOrder = inOrder(mockCohort.get());
738             inOrder.verify(mockCohort.get()).canCommit();
739             inOrder.verify(mockCohort.get()).preCommit();
740             inOrder.verify(mockCohort.get()).commit();
741
742             // Verify data in the data store.
743
744             verifyOuterListEntry(shard, 1);
745         }};
746     }
747
748     @Test
749     public void testBatchedModificationsWithCommitOnReady() throws Throwable {
750         new ShardTestKit(getSystem()) {{
751             final TestActorRef<Shard> shard = actorFactory.createTestActor(
752                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
753                     "testBatchedModificationsWithCommitOnReady");
754
755             waitUntilLeader(shard);
756
757             final String transactionID = "tx";
758             final FiniteDuration duration = duration("5 seconds");
759
760             final AtomicReference<ShardDataTreeCohort> mockCohort = new AtomicReference<>();
761             final ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() {
762                 @Override
763                 public ShardDataTreeCohort decorate(final String txID, final ShardDataTreeCohort actual) {
764                     if(mockCohort.get() == null) {
765                         mockCohort.set(createDelegatingMockCohort("cohort", actual));
766                     }
767
768                     return mockCohort.get();
769                 }
770             };
771
772             shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator);
773
774             // Send a BatchedModifications to start a transaction.
775
776             shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH,
777                     ImmutableNodes.containerNode(TestModel.TEST_QNAME), false, false, 1), getRef());
778             expectMsgClass(duration, BatchedModificationsReply.class);
779
780             // Send a couple more BatchedModifications.
781
782             shard.tell(newBatchedModifications(transactionID, TestModel.OUTER_LIST_PATH,
783                 ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), false, false, 2), getRef());
784             expectMsgClass(duration, BatchedModificationsReply.class);
785
786             shard.tell(newBatchedModifications(transactionID, YangInstanceIdentifier.builder(
787                     TestModel.OUTER_LIST_PATH).nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(),
788                 ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1), true, true, 3), getRef());
789
790             expectMsgClass(duration, CommitTransactionReply.class);
791
792             final InOrder inOrder = inOrder(mockCohort.get());
793             inOrder.verify(mockCohort.get()).canCommit();
794             inOrder.verify(mockCohort.get()).preCommit();
795             inOrder.verify(mockCohort.get()).commit();
796
797             // Verify data in the data store.
798
799             verifyOuterListEntry(shard, 1);
800         }};
801     }
802
803     @Test(expected=IllegalStateException.class)
804     public void testBatchedModificationsReadyWithIncorrectTotalMessageCount() throws Throwable {
805         new ShardTestKit(getSystem()) {{
806             final TestActorRef<Shard> shard = actorFactory.createTestActor(
807                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
808                     "testBatchedModificationsReadyWithIncorrectTotalMessageCount");
809
810             waitUntilLeader(shard);
811
812             final String transactionID = "tx1";
813             final BatchedModifications batched = new BatchedModifications(transactionID, DataStoreVersions.CURRENT_VERSION, null);
814             batched.setReady(true);
815             batched.setTotalMessagesSent(2);
816
817             shard.tell(batched, getRef());
818
819             final Failure failure = expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
820
821             if(failure != null) {
822                 throw failure.cause();
823             }
824         }};
825     }
826
827     @Test
828     public void testBatchedModificationsWithOperationFailure() throws Throwable {
829         new ShardTestKit(getSystem()) {{
830             final TestActorRef<Shard> shard = actorFactory.createTestActor(
831                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
832                     "testBatchedModificationsWithOperationFailure");
833
834             waitUntilLeader(shard);
835
836             // Test merge with invalid data. An exception should occur when the merge is applied. Note that
837             // write will not validate the children for performance reasons.
838
839             String transactionID = "tx1";
840
841             ContainerNode invalidData = ImmutableContainerNodeBuilder.create().withNodeIdentifier(
842                     new YangInstanceIdentifier.NodeIdentifier(TestModel.TEST_QNAME)).
843                         withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build();
844
845             BatchedModifications batched = new BatchedModifications(transactionID, CURRENT_VERSION, null);
846             batched.addModification(new MergeModification(TestModel.TEST_PATH, invalidData));
847             shard.tell(batched, getRef());
848             Failure failure = expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
849
850             Throwable cause = failure.cause();
851
852             batched = new BatchedModifications(transactionID, DataStoreVersions.CURRENT_VERSION, null);
853             batched.setReady(true);
854             batched.setTotalMessagesSent(2);
855
856             shard.tell(batched, getRef());
857
858             failure = expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
859             assertEquals("Failure cause", cause, failure.cause());
860         }};
861     }
862
863     @Test
864     public void testBatchedModificationsOnTransactionChain() throws Throwable {
865         new ShardTestKit(getSystem()) {{
866             final TestActorRef<Shard> shard = actorFactory.createTestActor(
867                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
868                     "testBatchedModificationsOnTransactionChain");
869
870             waitUntilLeader(shard);
871
872             final String transactionChainID = "txChain";
873             final String transactionID1 = "tx1";
874             final String transactionID2 = "tx2";
875
876             final FiniteDuration duration = duration("5 seconds");
877
878             // Send a BatchedModifications to start a chained write transaction and ready it.
879
880             final ContainerNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
881             final YangInstanceIdentifier path = TestModel.TEST_PATH;
882             shard.tell(newBatchedModifications(transactionID1, transactionChainID, path,
883                     containerNode, true, false, 1), getRef());
884             expectMsgClass(duration, ReadyTransactionReply.class);
885
886             // Create a read Tx on the same chain.
887
888             shard.tell(new CreateTransaction(transactionID2, TransactionType.READ_ONLY.ordinal(),
889                     transactionChainID, DataStoreVersions.CURRENT_VERSION).toSerializable(), getRef());
890
891             final CreateTransactionReply createReply = expectMsgClass(duration("3 seconds"), CreateTransactionReply.class);
892
893             getSystem().actorSelection(createReply.getTransactionPath()).tell(
894                     new ReadData(path, DataStoreVersions.CURRENT_VERSION), getRef());
895             final ReadDataReply readReply = expectMsgClass(duration("3 seconds"), ReadDataReply.class);
896             assertEquals("Read node", containerNode, readReply.getNormalizedNode());
897
898             // Commit the write transaction.
899
900             shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
901             final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
902                     expectMsgClass(duration, CanCommitTransactionReply.class));
903             assertEquals("Can commit", true, canCommitReply.getCanCommit());
904
905             shard.tell(new CommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
906             expectMsgClass(duration, CommitTransactionReply.class);
907
908             // Verify data in the data store.
909
910             final NormalizedNode<?, ?> actualNode = readStore(shard, path);
911             assertEquals("Stored node", containerNode, actualNode);
912         }};
913     }
914
915     @Test
916     public void testOnBatchedModificationsWhenNotLeader() {
917         final AtomicBoolean overrideLeaderCalls = new AtomicBoolean();
918         new ShardTestKit(getSystem()) {{
919             final Creator<Shard> creator = new Creator<Shard>() {
920                 private static final long serialVersionUID = 1L;
921
922                 @Override
923                 public Shard create() throws Exception {
924                     return new Shard(newShardBuilder()) {
925                         @Override
926                         protected boolean isLeader() {
927                             return overrideLeaderCalls.get() ? false : super.isLeader();
928                         }
929
930                         @Override
931                         protected ActorSelection getLeader() {
932                             return overrideLeaderCalls.get() ? getSystem().actorSelection(getRef().path()) :
933                                 super.getLeader();
934                         }
935                     };
936                 }
937             };
938
939             final TestActorRef<Shard> shard = actorFactory.createTestActor(
940                     Props.create(new DelegatingShardCreator(creator)), "testOnBatchedModificationsWhenNotLeader");
941
942             waitUntilLeader(shard);
943
944             overrideLeaderCalls.set(true);
945
946             final BatchedModifications batched = new BatchedModifications("tx", DataStoreVersions.CURRENT_VERSION, "");
947
948             shard.tell(batched, ActorRef.noSender());
949
950             expectMsgEquals(batched);
951         }};
952     }
953
954     @Test
955     public void testTransactionMessagesWithNoLeader() {
956         new ShardTestKit(getSystem()) {{
957             dataStoreContextBuilder.customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName()).
958                 shardHeartbeatIntervalInMillis(50).shardElectionTimeoutFactor(1);
959             final TestActorRef<Shard> shard = actorFactory.createTestActor(
960                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
961                     "testTransactionMessagesWithNoLeader");
962
963             waitUntilNoLeader(shard);
964
965             shard.tell(new BatchedModifications("tx", DataStoreVersions.CURRENT_VERSION, ""), getRef());
966             Failure failure = expectMsgClass(Failure.class);
967             assertEquals("Failure cause type", NoShardLeaderException.class, failure.cause().getClass());
968
969             shard.tell(prepareForwardedReadyTransaction(mock(ShardDataTreeCohort.class), "tx",
970                     DataStoreVersions.CURRENT_VERSION, true), getRef());
971             failure = expectMsgClass(Failure.class);
972             assertEquals("Failure cause type", NoShardLeaderException.class, failure.cause().getClass());
973
974             shard.tell(new ReadyLocalTransaction("tx", mock(DataTreeModification.class), true), getRef());
975             failure = expectMsgClass(Failure.class);
976             assertEquals("Failure cause type", NoShardLeaderException.class, failure.cause().getClass());
977         }};
978     }
979
980     @Test
981     public void testReadyWithImmediateCommit() throws Exception{
982         testReadyWithImmediateCommit(true);
983         testReadyWithImmediateCommit(false);
984     }
985
986     private void testReadyWithImmediateCommit(final boolean readWrite) throws Exception{
987         new ShardTestKit(getSystem()) {{
988             final TestActorRef<Shard> shard = actorFactory.createTestActor(
989                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
990                     "testReadyWithImmediateCommit-" + readWrite);
991
992             waitUntilLeader(shard);
993
994             final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
995
996             final String transactionID = "tx1";
997             final MutableCompositeModification modification = new MutableCompositeModification();
998             final NormalizedNode<?, ?> containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
999             final ShardDataTreeCohort cohort = setupMockWriteTransaction("cohort", dataStore,
1000                     TestModel.TEST_PATH, containerNode, modification);
1001
1002             final FiniteDuration duration = duration("5 seconds");
1003
1004             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification, true), getRef());
1005
1006             expectMsgClass(duration, CommitTransactionReply.class);
1007
1008             final InOrder inOrder = inOrder(cohort);
1009             inOrder.verify(cohort).canCommit();
1010             inOrder.verify(cohort).preCommit();
1011             inOrder.verify(cohort).commit();
1012
1013             final NormalizedNode<?, ?> actualNode = readStore(shard, TestModel.TEST_PATH);
1014             assertEquals(TestModel.TEST_QNAME.getLocalName(), containerNode, actualNode);
1015         }};
1016     }
1017
1018     @Test
1019     public void testReadyLocalTransactionWithImmediateCommit() throws Exception{
1020         new ShardTestKit(getSystem()) {{
1021             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1022                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1023                     "testReadyLocalTransactionWithImmediateCommit");
1024
1025             waitUntilLeader(shard);
1026
1027             final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
1028
1029             final DataTreeModification modification = dataStore.newModification();
1030
1031             final ContainerNode writeData = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
1032             new WriteModification(TestModel.TEST_PATH, writeData).apply(modification);
1033             final MapNode mergeData = ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build();
1034             new MergeModification(TestModel.OUTER_LIST_PATH, mergeData).apply(modification);
1035
1036             final String txId = "tx1";
1037             modification.ready();
1038             final ReadyLocalTransaction readyMessage = new ReadyLocalTransaction(txId, modification, true);
1039
1040             shard.tell(readyMessage, getRef());
1041
1042             expectMsgClass(CommitTransactionReply.class);
1043
1044             final NormalizedNode<?, ?> actualNode = readStore(shard, TestModel.OUTER_LIST_PATH);
1045             assertEquals(TestModel.OUTER_LIST_QNAME.getLocalName(), mergeData, actualNode);
1046         }};
1047     }
1048
1049     @Test
1050     public void testReadyLocalTransactionWithThreePhaseCommit() throws Exception{
1051         new ShardTestKit(getSystem()) {{
1052             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1053                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1054                     "testReadyLocalTransactionWithThreePhaseCommit");
1055
1056             waitUntilLeader(shard);
1057
1058             final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
1059
1060             final DataTreeModification modification = dataStore.newModification();
1061
1062             final ContainerNode writeData = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
1063             new WriteModification(TestModel.TEST_PATH, writeData).apply(modification);
1064             final MapNode mergeData = ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build();
1065             new MergeModification(TestModel.OUTER_LIST_PATH, mergeData).apply(modification);
1066
1067             final String txId = "tx1";
1068                 modification.ready();
1069             final ReadyLocalTransaction readyMessage = new ReadyLocalTransaction(txId, modification, false);
1070
1071             shard.tell(readyMessage, getRef());
1072
1073             expectMsgClass(ReadyTransactionReply.class);
1074
1075             // Send the CanCommitTransaction message.
1076
1077             shard.tell(new CanCommitTransaction(txId, CURRENT_VERSION).toSerializable(), getRef());
1078             final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
1079                     expectMsgClass(CanCommitTransactionReply.class));
1080             assertEquals("Can commit", true, canCommitReply.getCanCommit());
1081
1082             // Send the CanCommitTransaction message.
1083
1084             shard.tell(new CommitTransaction(txId, CURRENT_VERSION).toSerializable(), getRef());
1085             expectMsgClass(CommitTransactionReply.class);
1086
1087             final NormalizedNode<?, ?> actualNode = readStore(shard, TestModel.OUTER_LIST_PATH);
1088             assertEquals(TestModel.OUTER_LIST_QNAME.getLocalName(), mergeData, actualNode);
1089         }};
1090     }
1091
1092     @Test
1093     public void testCommitWithPersistenceDisabled() throws Throwable {
1094         testCommitWithPersistenceDisabled(true);
1095         testCommitWithPersistenceDisabled(false);
1096     }
1097
1098     private void testCommitWithPersistenceDisabled(final boolean readWrite) throws Throwable {
1099         dataStoreContextBuilder.persistent(false);
1100         new ShardTestKit(getSystem()) {{
1101             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1102                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1103                     "testCommitWithPersistenceDisabled-" + readWrite);
1104
1105             waitUntilLeader(shard);
1106
1107             final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
1108
1109             // Setup a simulated transactions with a mock cohort.
1110
1111             final String transactionID = "tx";
1112             final MutableCompositeModification modification = new MutableCompositeModification();
1113             final NormalizedNode<?, ?> containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
1114             final ShardDataTreeCohort cohort = setupMockWriteTransaction("cohort", dataStore,
1115                 TestModel.TEST_PATH, containerNode, modification);
1116
1117             final FiniteDuration duration = duration("5 seconds");
1118
1119             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification), getRef());
1120             expectMsgClass(duration, ReadyTransactionReply.class);
1121
1122             // Send the CanCommitTransaction message.
1123
1124             shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
1125             final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
1126                     expectMsgClass(duration, CanCommitTransactionReply.class));
1127             assertEquals("Can commit", true, canCommitReply.getCanCommit());
1128
1129             // Send the CanCommitTransaction message.
1130
1131             shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
1132             expectMsgClass(duration, CommitTransactionReply.class);
1133
1134             final InOrder inOrder = inOrder(cohort);
1135             inOrder.verify(cohort).canCommit();
1136             inOrder.verify(cohort).preCommit();
1137             inOrder.verify(cohort).commit();
1138
1139             final NormalizedNode<?, ?> actualNode = readStore(shard, TestModel.TEST_PATH);
1140             assertEquals(TestModel.TEST_QNAME.getLocalName(), containerNode, actualNode);
1141         }};
1142     }
1143
1144     @Test
1145     public void testCommitWhenTransactionHasNoModifications() {
1146         testCommitWhenTransactionHasNoModifications(true);
1147         testCommitWhenTransactionHasNoModifications(false);
1148     }
1149
1150     private void testCommitWhenTransactionHasNoModifications(final boolean readWrite){
1151         // Note that persistence is enabled which would normally result in the entry getting written to the journal
1152         // but here that need not happen
1153         new ShardTestKit(getSystem()) {
1154             {
1155                 final TestActorRef<Shard> shard = actorFactory.createTestActor(
1156                         newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1157                         "testCommitWhenTransactionHasNoModifications-" + readWrite);
1158
1159                 waitUntilLeader(shard);
1160
1161                 final String transactionID = "tx1";
1162                 final MutableCompositeModification modification = new MutableCompositeModification();
1163                 final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1");
1164                 doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit();
1165                 doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).preCommit();
1166                 doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).commit();
1167                 doReturn(mockUnmodifiedCandidate("cohort1-candidate")).when(cohort).getCandidate();
1168
1169                 final FiniteDuration duration = duration("5 seconds");
1170
1171                 shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification), getRef());
1172                 expectMsgClass(duration, ReadyTransactionReply.class);
1173
1174                 // Send the CanCommitTransaction message.
1175
1176                 shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
1177                 final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
1178                         expectMsgClass(duration, CanCommitTransactionReply.class));
1179                 assertEquals("Can commit", true, canCommitReply.getCanCommit());
1180
1181                 shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
1182                 expectMsgClass(duration, CommitTransactionReply.class);
1183
1184                 final InOrder inOrder = inOrder(cohort);
1185                 inOrder.verify(cohort).canCommit();
1186                 inOrder.verify(cohort).preCommit();
1187                 inOrder.verify(cohort).commit();
1188
1189                 shard.tell(Shard.GET_SHARD_MBEAN_MESSAGE, getRef());
1190                 final ShardStats shardStats = expectMsgClass(duration, ShardStats.class);
1191
1192                 // Use MBean for verification
1193                 // Committed transaction count should increase as usual
1194                 assertEquals(1,shardStats.getCommittedTransactionsCount());
1195
1196                 // Commit index should not advance because this does not go into the journal
1197                 assertEquals(-1, shardStats.getCommitIndex());
1198             }
1199         };
1200     }
1201
1202     @Test
1203     public void testCommitWhenTransactionHasModifications() {
1204         testCommitWhenTransactionHasModifications(true);
1205         testCommitWhenTransactionHasModifications(false);
1206     }
1207
1208     private void testCommitWhenTransactionHasModifications(final boolean readWrite){
1209         new ShardTestKit(getSystem()) {
1210             {
1211                 final TestActorRef<Shard> shard = actorFactory.createTestActor(
1212                         newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1213                         "testCommitWhenTransactionHasModifications-" + readWrite);
1214
1215                 waitUntilLeader(shard);
1216
1217                 final String transactionID = "tx1";
1218                 final MutableCompositeModification modification = new MutableCompositeModification();
1219                 modification.addModification(new DeleteModification(YangInstanceIdentifier.builder().build()));
1220                 final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1");
1221                 doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit();
1222                 doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).preCommit();
1223                 doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).commit();
1224                 doReturn(mockCandidate("cohort1-candidate")).when(cohort).getCandidate();
1225
1226                 final FiniteDuration duration = duration("5 seconds");
1227
1228                 shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification), getRef());
1229                 expectMsgClass(duration, ReadyTransactionReply.class);
1230
1231                 // Send the CanCommitTransaction message.
1232
1233                 shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
1234                 final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
1235                         expectMsgClass(duration, CanCommitTransactionReply.class));
1236                 assertEquals("Can commit", true, canCommitReply.getCanCommit());
1237
1238                 shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
1239                 expectMsgClass(duration, CommitTransactionReply.class);
1240
1241                 final InOrder inOrder = inOrder(cohort);
1242                 inOrder.verify(cohort).canCommit();
1243                 inOrder.verify(cohort).preCommit();
1244                 inOrder.verify(cohort).commit();
1245
1246                 shard.tell(Shard.GET_SHARD_MBEAN_MESSAGE, getRef());
1247                 final ShardStats shardStats = expectMsgClass(duration, ShardStats.class);
1248
1249                 // Use MBean for verification
1250                 // Committed transaction count should increase as usual
1251                 assertEquals(1, shardStats.getCommittedTransactionsCount());
1252
1253                 // Commit index should advance as we do not have an empty modification
1254                 assertEquals(0, shardStats.getCommitIndex());
1255             }
1256         };
1257     }
1258
1259     @Test
1260     public void testCommitPhaseFailure() throws Throwable {
1261         testCommitPhaseFailure(true);
1262         testCommitPhaseFailure(false);
1263     }
1264
1265     private void testCommitPhaseFailure(final boolean readWrite) throws Throwable {
1266         new ShardTestKit(getSystem()) {{
1267             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1268                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1269                     "testCommitPhaseFailure-" + readWrite);
1270
1271             waitUntilLeader(shard);
1272
1273             // Setup 2 simulated transactions with mock cohorts. The first one fails in the
1274             // commit phase.
1275
1276             final String transactionID1 = "tx1";
1277             final MutableCompositeModification modification1 = new MutableCompositeModification();
1278             final ShardDataTreeCohort cohort1 = mock(ShardDataTreeCohort.class, "cohort1");
1279             doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort1).canCommit();
1280             doReturn(Futures.immediateFuture(null)).when(cohort1).preCommit();
1281             doReturn(Futures.immediateFailedFuture(new IllegalStateException("mock"))).when(cohort1).commit();
1282             doReturn(mockCandidate("cohort1-candidate")).when(cohort1).getCandidate();
1283
1284             final String transactionID2 = "tx2";
1285             final MutableCompositeModification modification2 = new MutableCompositeModification();
1286             final ShardDataTreeCohort cohort2 = mock(ShardDataTreeCohort.class, "cohort2");
1287             doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort2).canCommit();
1288
1289             final FiniteDuration duration = duration("5 seconds");
1290             final Timeout timeout = new Timeout(duration);
1291
1292             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
1293             expectMsgClass(duration, ReadyTransactionReply.class);
1294
1295             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
1296             expectMsgClass(duration, ReadyTransactionReply.class);
1297
1298             // Send the CanCommitTransaction message for the first Tx.
1299
1300             shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1301             final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
1302                     expectMsgClass(duration, CanCommitTransactionReply.class));
1303             assertEquals("Can commit", true, canCommitReply.getCanCommit());
1304
1305             // Send the CanCommitTransaction message for the 2nd Tx. This should get queued and
1306             // processed after the first Tx completes.
1307
1308             final Future<Object> canCommitFuture = Patterns.ask(shard,
1309                     new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), timeout);
1310
1311             // Send the CommitTransaction message for the first Tx. This should send back an error
1312             // and trigger the 2nd Tx to proceed.
1313
1314             shard.tell(new CommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1315             expectMsgClass(duration, akka.actor.Status.Failure.class);
1316
1317             // Wait for the 2nd Tx to complete the canCommit phase.
1318
1319             final CountDownLatch latch = new CountDownLatch(1);
1320             canCommitFuture.onComplete(new OnComplete<Object>() {
1321                 @Override
1322                 public void onComplete(final Throwable t, final Object resp) {
1323                     latch.countDown();
1324                 }
1325             }, getSystem().dispatcher());
1326
1327             assertEquals("2nd CanCommit complete", true, latch.await(5, TimeUnit.SECONDS));
1328
1329             final InOrder inOrder = inOrder(cohort1, cohort2);
1330             inOrder.verify(cohort1).canCommit();
1331             inOrder.verify(cohort1).preCommit();
1332             inOrder.verify(cohort1).commit();
1333             inOrder.verify(cohort2).canCommit();
1334         }};
1335     }
1336
1337     @Test
1338     public void testPreCommitPhaseFailure() throws Throwable {
1339         testPreCommitPhaseFailure(true);
1340         testPreCommitPhaseFailure(false);
1341     }
1342
1343     private void testPreCommitPhaseFailure(final boolean readWrite) throws Throwable {
1344         new ShardTestKit(getSystem()) {{
1345             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1346                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1347                     "testPreCommitPhaseFailure-" + readWrite);
1348
1349             waitUntilLeader(shard);
1350
1351             final String transactionID1 = "tx1";
1352             final MutableCompositeModification modification1 = new MutableCompositeModification();
1353             final ShardDataTreeCohort cohort1 = mock(ShardDataTreeCohort.class, "cohort1");
1354             doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort1).canCommit();
1355             doReturn(Futures.immediateFailedFuture(new IllegalStateException("mock"))).when(cohort1).preCommit();
1356
1357             final String transactionID2 = "tx2";
1358             final MutableCompositeModification modification2 = new MutableCompositeModification();
1359             final ShardDataTreeCohort cohort2 = mock(ShardDataTreeCohort.class, "cohort2");
1360             doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort2).canCommit();
1361
1362             final FiniteDuration duration = duration("5 seconds");
1363             final Timeout timeout = new Timeout(duration);
1364
1365             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
1366             expectMsgClass(duration, ReadyTransactionReply.class);
1367
1368             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
1369             expectMsgClass(duration, ReadyTransactionReply.class);
1370
1371             // Send the CanCommitTransaction message for the first Tx.
1372
1373             shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1374             final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
1375                 expectMsgClass(duration, CanCommitTransactionReply.class));
1376             assertEquals("Can commit", true, canCommitReply.getCanCommit());
1377
1378             // Send the CanCommitTransaction message for the 2nd Tx. This should get queued and
1379             // processed after the first Tx completes.
1380
1381             final Future<Object> canCommitFuture = Patterns.ask(shard,
1382                     new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), timeout);
1383
1384             // Send the CommitTransaction message for the first Tx. This should send back an error
1385             // and trigger the 2nd Tx to proceed.
1386
1387             shard.tell(new CommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1388             expectMsgClass(duration, akka.actor.Status.Failure.class);
1389
1390             // Wait for the 2nd Tx to complete the canCommit phase.
1391
1392             final CountDownLatch latch = new CountDownLatch(1);
1393             canCommitFuture.onComplete(new OnComplete<Object>() {
1394                 @Override
1395                 public void onComplete(final Throwable t, final Object resp) {
1396                     latch.countDown();
1397                 }
1398             }, getSystem().dispatcher());
1399
1400             assertEquals("2nd CanCommit complete", true, latch.await(5, TimeUnit.SECONDS));
1401
1402             final InOrder inOrder = inOrder(cohort1, cohort2);
1403             inOrder.verify(cohort1).canCommit();
1404             inOrder.verify(cohort1).preCommit();
1405             inOrder.verify(cohort2).canCommit();
1406         }};
1407     }
1408
1409     @Test
1410     public void testCanCommitPhaseFailure() throws Throwable {
1411         testCanCommitPhaseFailure(true);
1412         testCanCommitPhaseFailure(false);
1413     }
1414
1415     private void testCanCommitPhaseFailure(final boolean readWrite) throws Throwable {
1416         new ShardTestKit(getSystem()) {{
1417             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1418                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1419                     "testCanCommitPhaseFailure-" + readWrite);
1420
1421             waitUntilLeader(shard);
1422
1423             final FiniteDuration duration = duration("5 seconds");
1424
1425             final String transactionID1 = "tx1";
1426             final MutableCompositeModification modification = new MutableCompositeModification();
1427             final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1");
1428             doReturn(Futures.immediateFailedFuture(new IllegalStateException("mock"))).when(cohort).canCommit();
1429
1430             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID1, modification), getRef());
1431             expectMsgClass(duration, ReadyTransactionReply.class);
1432
1433             // Send the CanCommitTransaction message.
1434
1435             shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1436             expectMsgClass(duration, akka.actor.Status.Failure.class);
1437
1438             // Send another can commit to ensure the failed one got cleaned up.
1439
1440             reset(cohort);
1441
1442             final String transactionID2 = "tx2";
1443             doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit();
1444
1445             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID2, modification), getRef());
1446             expectMsgClass(duration, ReadyTransactionReply.class);
1447
1448             shard.tell(new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef());
1449             final CanCommitTransactionReply reply = CanCommitTransactionReply.fromSerializable(
1450                     expectMsgClass(CanCommitTransactionReply.class));
1451             assertEquals("getCanCommit", true, reply.getCanCommit());
1452         }};
1453     }
1454
1455     @Test
1456     public void testCanCommitPhaseFalseResponse() throws Throwable {
1457         testCanCommitPhaseFalseResponse(true);
1458         testCanCommitPhaseFalseResponse(false);
1459     }
1460
1461     private void testCanCommitPhaseFalseResponse(final boolean readWrite) throws Throwable {
1462         new ShardTestKit(getSystem()) {{
1463             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1464                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1465                     "testCanCommitPhaseFalseResponse-" + readWrite);
1466
1467             waitUntilLeader(shard);
1468
1469             final FiniteDuration duration = duration("5 seconds");
1470
1471             final String transactionID1 = "tx1";
1472             final MutableCompositeModification modification = new MutableCompositeModification();
1473             final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1");
1474             doReturn(Futures.immediateFuture(Boolean.FALSE)).when(cohort).canCommit();
1475
1476             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID1, modification), getRef());
1477             expectMsgClass(duration, ReadyTransactionReply.class);
1478
1479             // Send the CanCommitTransaction message.
1480
1481             shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1482             CanCommitTransactionReply reply = CanCommitTransactionReply.fromSerializable(
1483                     expectMsgClass(CanCommitTransactionReply.class));
1484             assertEquals("getCanCommit", false, reply.getCanCommit());
1485
1486             // Send another can commit to ensure the failed one got cleaned up.
1487
1488             reset(cohort);
1489
1490             final String transactionID2 = "tx2";
1491             doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit();
1492
1493             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID2, modification), getRef());
1494             expectMsgClass(duration, ReadyTransactionReply.class);
1495
1496             shard.tell(new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef());
1497             reply = CanCommitTransactionReply.fromSerializable(
1498                     expectMsgClass(CanCommitTransactionReply.class));
1499             assertEquals("getCanCommit", true, reply.getCanCommit());
1500         }};
1501     }
1502
1503     @Test
1504     public void testImmediateCommitWithCanCommitPhaseFailure() throws Throwable {
1505         testImmediateCommitWithCanCommitPhaseFailure(true);
1506         testImmediateCommitWithCanCommitPhaseFailure(false);
1507     }
1508
1509     private void testImmediateCommitWithCanCommitPhaseFailure(final boolean readWrite) throws Throwable {
1510         new ShardTestKit(getSystem()) {{
1511             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1512                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1513                     "testImmediateCommitWithCanCommitPhaseFailure-" + readWrite);
1514
1515             waitUntilLeader(shard);
1516
1517             final FiniteDuration duration = duration("5 seconds");
1518
1519             final String transactionID1 = "tx1";
1520             final MutableCompositeModification modification = new MutableCompositeModification();
1521             final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1");
1522             doReturn(Futures.immediateFailedFuture(new IllegalStateException("mock"))).when(cohort).canCommit();
1523
1524             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID1, modification, true), getRef());
1525
1526             expectMsgClass(duration, akka.actor.Status.Failure.class);
1527
1528             // Send another can commit to ensure the failed one got cleaned up.
1529
1530             reset(cohort);
1531
1532             final String transactionID2 = "tx2";
1533             doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit();
1534             doReturn(Futures.immediateFuture(null)).when(cohort).preCommit();
1535             doReturn(Futures.immediateFuture(null)).when(cohort).commit();
1536             final DataTreeCandidateTip candidate = mock(DataTreeCandidateTip.class);
1537             final DataTreeCandidateNode candidateRoot = mock(DataTreeCandidateNode.class);
1538             doReturn(ModificationType.UNMODIFIED).when(candidateRoot).getModificationType();
1539             doReturn(candidateRoot).when(candidate).getRootNode();
1540             doReturn(candidate).when(cohort).getCandidate();
1541
1542             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID2, modification, true), getRef());
1543
1544             expectMsgClass(duration, CommitTransactionReply.class);
1545         }};
1546     }
1547
1548     @Test
1549     public void testImmediateCommitWithCanCommitPhaseFalseResponse() throws Throwable {
1550         testImmediateCommitWithCanCommitPhaseFalseResponse(true);
1551         testImmediateCommitWithCanCommitPhaseFalseResponse(false);
1552     }
1553
1554     private void testImmediateCommitWithCanCommitPhaseFalseResponse(final boolean readWrite) throws Throwable {
1555         new ShardTestKit(getSystem()) {{
1556             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1557                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1558                     "testImmediateCommitWithCanCommitPhaseFalseResponse-" + readWrite);
1559
1560             waitUntilLeader(shard);
1561
1562             final FiniteDuration duration = duration("5 seconds");
1563
1564             final String transactionID = "tx1";
1565             final MutableCompositeModification modification = new MutableCompositeModification();
1566             final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1");
1567             doReturn(Futures.immediateFuture(Boolean.FALSE)).when(cohort).canCommit();
1568
1569             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification, true), getRef());
1570
1571             expectMsgClass(duration, akka.actor.Status.Failure.class);
1572
1573             // Send another can commit to ensure the failed one got cleaned up.
1574
1575             reset(cohort);
1576
1577             final String transactionID2 = "tx2";
1578             doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit();
1579             doReturn(Futures.immediateFuture(null)).when(cohort).preCommit();
1580             doReturn(Futures.immediateFuture(null)).when(cohort).commit();
1581             final DataTreeCandidateTip candidate = mock(DataTreeCandidateTip.class);
1582             final DataTreeCandidateNode candidateRoot = mock(DataTreeCandidateNode.class);
1583             doReturn(ModificationType.UNMODIFIED).when(candidateRoot).getModificationType();
1584             doReturn(candidateRoot).when(candidate).getRootNode();
1585             doReturn(candidate).when(cohort).getCandidate();
1586
1587             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID2, modification, true), getRef());
1588
1589             expectMsgClass(duration, CommitTransactionReply.class);
1590         }};
1591     }
1592
1593     @Test
1594     public void testAbortBeforeFinishCommit() throws Throwable {
1595         testAbortBeforeFinishCommit(true);
1596         testAbortBeforeFinishCommit(false);
1597     }
1598
1599     private void testAbortBeforeFinishCommit(final boolean readWrite) throws Throwable {
1600         new ShardTestKit(getSystem()) {{
1601             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1602                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1603                     "testAbortBeforeFinishCommit-" + readWrite);
1604
1605             waitUntilLeader(shard);
1606
1607             final FiniteDuration duration = duration("5 seconds");
1608             final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
1609
1610             final String transactionID = "tx1";
1611             final Function<ShardDataTreeCohort, ListenableFuture<Void>> preCommit =
1612                           new Function<ShardDataTreeCohort, ListenableFuture<Void>>() {
1613                 @Override
1614                 public ListenableFuture<Void> apply(final ShardDataTreeCohort cohort) {
1615                     final ListenableFuture<Void> preCommitFuture = cohort.preCommit();
1616
1617                     // Simulate an AbortTransaction message occurring during replication, after
1618                     // persisting and before finishing the commit to the in-memory store.
1619                     // We have no followers so due to optimizations in the RaftActor, it does not
1620                     // attempt replication and thus we can't send an AbortTransaction message b/c
1621                     // it would be processed too late after CommitTransaction completes. So we'll
1622                     // simulate an AbortTransaction message occurring during replication by calling
1623                     // the shard directly.
1624                     //
1625                     shard.underlyingActor().doAbortTransaction(transactionID, null);
1626
1627                     return preCommitFuture;
1628                 }
1629             };
1630
1631             final MutableCompositeModification modification = new MutableCompositeModification();
1632             final ShardDataTreeCohort cohort = setupMockWriteTransaction("cohort1", dataStore,
1633                     TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME),
1634                     modification, preCommit);
1635
1636             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification), getRef());
1637             expectMsgClass(duration, ReadyTransactionReply.class);
1638
1639             shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
1640             final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
1641                 expectMsgClass(duration, CanCommitTransactionReply.class));
1642             assertEquals("Can commit", true, canCommitReply.getCanCommit());
1643
1644             shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
1645             expectMsgClass(duration, CommitTransactionReply.class);
1646
1647             final NormalizedNode<?, ?> node = readStore(shard, TestModel.TEST_PATH);
1648
1649             // Since we're simulating an abort occurring during replication and before finish commit,
1650             // the data should still get written to the in-memory store since we've gotten past
1651             // canCommit and preCommit and persisted the data.
1652             assertNotNull(TestModel.TEST_QNAME.getLocalName() + " not found", node);
1653         }};
1654     }
1655
1656     @Test
1657     public void testTransactionCommitTimeout() throws Throwable {
1658         testTransactionCommitTimeout(true);
1659         testTransactionCommitTimeout(false);
1660     }
1661
1662     private void testTransactionCommitTimeout(final boolean readWrite) throws Throwable {
1663         dataStoreContextBuilder.shardTransactionCommitTimeoutInSeconds(1);
1664
1665         new ShardTestKit(getSystem()) {{
1666             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1667                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1668                     "testTransactionCommitTimeout-" + readWrite);
1669
1670             waitUntilLeader(shard);
1671
1672             final FiniteDuration duration = duration("5 seconds");
1673
1674             final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
1675
1676             writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
1677             writeToStore(shard, TestModel.OUTER_LIST_PATH,
1678                     ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build());
1679
1680             // Create 1st Tx - will timeout
1681
1682             final String transactionID1 = "tx1";
1683             final MutableCompositeModification modification1 = new MutableCompositeModification();
1684             final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore,
1685                     YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
1686                         .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(),
1687                     ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1),
1688                     modification1);
1689
1690             // Create 2nd Tx
1691
1692             final String transactionID2 = "tx3";
1693             final MutableCompositeModification modification2 = new MutableCompositeModification();
1694             final YangInstanceIdentifier listNodePath = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
1695                 .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2).build();
1696             final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort3", dataStore,
1697                     listNodePath,
1698                     ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2),
1699                     modification2);
1700
1701             // Ready the Tx's
1702
1703             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
1704             expectMsgClass(duration, ReadyTransactionReply.class);
1705
1706             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
1707             expectMsgClass(duration, ReadyTransactionReply.class);
1708
1709             // canCommit 1st Tx. We don't send the commit so it should timeout.
1710
1711             shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1712             expectMsgClass(duration, CanCommitTransactionReply.class);
1713
1714             // canCommit the 2nd Tx - it should complete after the 1st Tx times out.
1715
1716             shard.tell(new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef());
1717             expectMsgClass(duration, CanCommitTransactionReply.class);
1718
1719             // Try to commit the 1st Tx - should fail as it's not the current Tx.
1720
1721             shard.tell(new CommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1722             expectMsgClass(duration, akka.actor.Status.Failure.class);
1723
1724             // Commit the 2nd Tx.
1725
1726             shard.tell(new CommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef());
1727             expectMsgClass(duration, CommitTransactionReply.class);
1728
1729             final NormalizedNode<?, ?> node = readStore(shard, listNodePath);
1730             assertNotNull(listNodePath + " not found", node);
1731         }};
1732     }
1733
1734     @Test
1735     public void testTransactionCommitQueueCapacityExceeded() throws Throwable {
1736         dataStoreContextBuilder.shardTransactionCommitQueueCapacity(2);
1737
1738         new ShardTestKit(getSystem()) {{
1739             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1740                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1741                     "testTransactionCommitQueueCapacityExceeded");
1742
1743             waitUntilLeader(shard);
1744
1745             final FiniteDuration duration = duration("5 seconds");
1746
1747             final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
1748
1749             final String transactionID1 = "tx1";
1750             final MutableCompositeModification modification1 = new MutableCompositeModification();
1751             final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore,
1752                     TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1);
1753
1754             final String transactionID2 = "tx2";
1755             final MutableCompositeModification modification2 = new MutableCompositeModification();
1756             final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore,
1757                     TestModel.OUTER_LIST_PATH,
1758                     ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(),
1759                     modification2);
1760
1761             final String transactionID3 = "tx3";
1762             final MutableCompositeModification modification3 = new MutableCompositeModification();
1763             final ShardDataTreeCohort cohort3 = setupMockWriteTransaction("cohort3", dataStore,
1764                     TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification3);
1765
1766             // Ready the Tx's
1767
1768             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
1769             expectMsgClass(duration, ReadyTransactionReply.class);
1770
1771             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
1772             expectMsgClass(duration, ReadyTransactionReply.class);
1773
1774             // The 3rd Tx should exceed queue capacity and fail.
1775
1776             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort3, transactionID3, modification3), getRef());
1777             expectMsgClass(duration, akka.actor.Status.Failure.class);
1778
1779             // canCommit 1st Tx.
1780
1781             shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1782             expectMsgClass(duration, CanCommitTransactionReply.class);
1783
1784             // canCommit the 2nd Tx - it should get queued.
1785
1786             shard.tell(new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef());
1787
1788             // canCommit the 3rd Tx - should exceed queue capacity and fail.
1789
1790             shard.tell(new CanCommitTransaction(transactionID3, CURRENT_VERSION).toSerializable(), getRef());
1791             expectMsgClass(duration, akka.actor.Status.Failure.class);
1792         }};
1793     }
1794
1795     @Test
1796     public void testTransactionCommitWithPriorExpiredCohortEntries() throws Throwable {
1797         dataStoreContextBuilder.shardCommitQueueExpiryTimeoutInMillis(1300).shardTransactionCommitTimeoutInSeconds(1);
1798
1799         new ShardTestKit(getSystem()) {{
1800             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1801                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1802                     "testTransactionCommitWithPriorExpiredCohortEntries");
1803
1804             waitUntilLeader(shard);
1805
1806             final FiniteDuration duration = duration("5 seconds");
1807
1808             final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
1809
1810             final String transactionID1 = "tx1";
1811             final MutableCompositeModification modification1 = new MutableCompositeModification();
1812             final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore,
1813                     TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1);
1814
1815             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
1816             expectMsgClass(duration, ReadyTransactionReply.class);
1817
1818             final String transactionID2 = "tx2";
1819             final MutableCompositeModification modification2 = new MutableCompositeModification();
1820             final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore,
1821                     TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification2);
1822
1823             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
1824             expectMsgClass(duration, ReadyTransactionReply.class);
1825
1826             final String transactionID3 = "tx3";
1827             final MutableCompositeModification modification3 = new MutableCompositeModification();
1828             final ShardDataTreeCohort cohort3 = setupMockWriteTransaction("cohort3", dataStore,
1829                     TestModel.TEST2_PATH, ImmutableNodes.containerNode(TestModel.TEST2_QNAME), modification3);
1830
1831             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort3, transactionID3, modification3), getRef());
1832             expectMsgClass(duration, ReadyTransactionReply.class);
1833
1834             // All Tx's are readied. We'll send canCommit for the last one but not the others. The others
1835             // should expire from the queue and the last one should be processed.
1836
1837             shard.tell(new CanCommitTransaction(transactionID3, CURRENT_VERSION).toSerializable(), getRef());
1838             expectMsgClass(duration, CanCommitTransactionReply.class);
1839         }};
1840     }
1841
1842     @Test
1843     public void testTransactionCommitWithSubsequentExpiredCohortEntry() throws Throwable {
1844         dataStoreContextBuilder.shardCommitQueueExpiryTimeoutInMillis(1300).shardTransactionCommitTimeoutInSeconds(1);
1845
1846         new ShardTestKit(getSystem()) {{
1847             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1848                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1849                     "testTransactionCommitWithSubsequentExpiredCohortEntry");
1850
1851             waitUntilLeader(shard);
1852
1853             final FiniteDuration duration = duration("5 seconds");
1854
1855             final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
1856
1857             final String transactionID1 = "tx1";
1858             final MutableCompositeModification modification1 = new MutableCompositeModification();
1859             final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore,
1860                     TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1);
1861
1862             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
1863             expectMsgClass(duration, ReadyTransactionReply.class);
1864
1865             // CanCommit the first one so it's the current in-progress CohortEntry.
1866
1867             shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1868             expectMsgClass(duration, CanCommitTransactionReply.class);
1869
1870             // Ready the second Tx.
1871
1872             final String transactionID2 = "tx2";
1873             final MutableCompositeModification modification2 = new MutableCompositeModification();
1874             final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore,
1875                     TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification2);
1876
1877             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
1878             expectMsgClass(duration, ReadyTransactionReply.class);
1879
1880             // Ready the third Tx.
1881
1882             final String transactionID3 = "tx3";
1883             final DataTreeModification modification3 = dataStore.newModification();
1884             new WriteModification(TestModel.TEST2_PATH, ImmutableNodes.containerNode(TestModel.TEST2_QNAME))
1885                     .apply(modification3);
1886                 modification3.ready();
1887             final ReadyLocalTransaction readyMessage = new ReadyLocalTransaction(transactionID3, modification3, true);
1888
1889             shard.tell(readyMessage, getRef());
1890
1891             // Commit the first Tx. After completing, the second should expire from the queue and the third
1892             // Tx committed.
1893
1894             shard.tell(new CommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1895             expectMsgClass(duration, CommitTransactionReply.class);
1896
1897             // Expect commit reply from the third Tx.
1898
1899             expectMsgClass(duration, CommitTransactionReply.class);
1900
1901             final NormalizedNode<?, ?> node = readStore(shard, TestModel.TEST2_PATH);
1902             assertNotNull(TestModel.TEST2_PATH + " not found", node);
1903         }};
1904     }
1905
1906     @Test
1907     public void testCanCommitBeforeReadyFailure() throws Throwable {
1908         new ShardTestKit(getSystem()) {{
1909             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1910                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1911                     "testCanCommitBeforeReadyFailure");
1912
1913             shard.tell(new CanCommitTransaction("tx", CURRENT_VERSION).toSerializable(), getRef());
1914             expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
1915         }};
1916     }
1917
1918     @Test
1919     public void testAbortCurrentTransaction() throws Throwable {
1920         testAbortCurrentTransaction(true);
1921         testAbortCurrentTransaction(false);
1922     }
1923
1924     private void testAbortCurrentTransaction(final boolean readWrite) throws Throwable {
1925         new ShardTestKit(getSystem()) {{
1926             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1927                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1928                     "testAbortCurrentTransaction-" + readWrite);
1929
1930             waitUntilLeader(shard);
1931
1932             // Setup 2 simulated transactions with mock cohorts. The first one will be aborted.
1933
1934             final String transactionID1 = "tx1";
1935             final MutableCompositeModification modification1 = new MutableCompositeModification();
1936             final ShardDataTreeCohort cohort1 = mock(ShardDataTreeCohort.class, "cohort1");
1937             doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort1).canCommit();
1938             doReturn(Futures.immediateFuture(null)).when(cohort1).abort();
1939
1940             final String transactionID2 = "tx2";
1941             final MutableCompositeModification modification2 = new MutableCompositeModification();
1942             final ShardDataTreeCohort cohort2 = mock(ShardDataTreeCohort.class, "cohort2");
1943             doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort2).canCommit();
1944
1945             final FiniteDuration duration = duration("5 seconds");
1946             final Timeout timeout = new Timeout(duration);
1947
1948             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
1949             expectMsgClass(duration, ReadyTransactionReply.class);
1950
1951             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
1952             expectMsgClass(duration, ReadyTransactionReply.class);
1953
1954             // Send the CanCommitTransaction message for the first Tx.
1955
1956             shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1957             final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
1958                     expectMsgClass(duration, CanCommitTransactionReply.class));
1959             assertEquals("Can commit", true, canCommitReply.getCanCommit());
1960
1961             // Send the CanCommitTransaction message for the 2nd Tx. This should get queued and
1962             // processed after the first Tx completes.
1963
1964             final Future<Object> canCommitFuture = Patterns.ask(shard,
1965                     new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), timeout);
1966
1967             // Send the AbortTransaction message for the first Tx. This should trigger the 2nd
1968             // Tx to proceed.
1969
1970             shard.tell(new AbortTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1971             expectMsgClass(duration, AbortTransactionReply.class);
1972
1973             // Wait for the 2nd Tx to complete the canCommit phase.
1974
1975             Await.ready(canCommitFuture, duration);
1976
1977             final InOrder inOrder = inOrder(cohort1, cohort2);
1978             inOrder.verify(cohort1).canCommit();
1979             inOrder.verify(cohort2).canCommit();
1980         }};
1981     }
1982
1983     @Test
1984     public void testAbortQueuedTransaction() throws Throwable {
1985         testAbortQueuedTransaction(true);
1986         testAbortQueuedTransaction(false);
1987     }
1988
1989     private void testAbortQueuedTransaction(final boolean readWrite) throws Throwable {
1990         dataStoreContextBuilder.shardTransactionCommitTimeoutInSeconds(1);
1991         new ShardTestKit(getSystem()) {{
1992             final AtomicReference<CountDownLatch> cleaupCheckLatch = new AtomicReference<>();
1993             @SuppressWarnings("serial")
1994             final Creator<Shard> creator = new Creator<Shard>() {
1995                 @Override
1996                 public Shard create() throws Exception {
1997                     return new Shard(newShardBuilder()) {
1998                         @Override
1999                         public void onReceiveCommand(final Object message) throws Exception {
2000                             super.onReceiveCommand(message);
2001                             if(message.equals(TX_COMMIT_TIMEOUT_CHECK_MESSAGE)) {
2002                                 if(cleaupCheckLatch.get() != null) {
2003                                     cleaupCheckLatch.get().countDown();
2004                                 }
2005                             }
2006                         }
2007                     };
2008                 }
2009             };
2010
2011             final TestActorRef<Shard> shard = actorFactory.createTestActor(
2012                     Props.create(new DelegatingShardCreator(creator)).withDispatcher(
2013                             Dispatchers.DefaultDispatcherId()), "testAbortQueuedTransaction-" + readWrite);
2014
2015             waitUntilLeader(shard);
2016
2017             final String transactionID = "tx1";
2018
2019             final MutableCompositeModification modification = new MutableCompositeModification();
2020             final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort");
2021             doReturn(Futures.immediateFuture(null)).when(cohort).abort();
2022
2023             final FiniteDuration duration = duration("5 seconds");
2024
2025             // Ready the tx.
2026
2027             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification), getRef());
2028             expectMsgClass(duration, ReadyTransactionReply.class);
2029
2030             assertEquals("getPendingTxCommitQueueSize", 1, shard.underlyingActor().getPendingTxCommitQueueSize());
2031
2032             // Send the AbortTransaction message.
2033
2034             shard.tell(new AbortTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
2035             expectMsgClass(duration, AbortTransactionReply.class);
2036
2037             verify(cohort).abort();
2038
2039             // Verify the tx cohort is removed from queue at the cleanup check interval.
2040
2041             cleaupCheckLatch.set(new CountDownLatch(1));
2042             assertEquals("TX_COMMIT_TIMEOUT_CHECK_MESSAGE received", true,
2043                     cleaupCheckLatch.get().await(5, TimeUnit.SECONDS));
2044
2045             assertEquals("getPendingTxCommitQueueSize", 0, shard.underlyingActor().getPendingTxCommitQueueSize());
2046
2047             // Now send CanCommitTransaction - should fail.
2048
2049             shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
2050
2051             Throwable failure = expectMsgClass(duration, akka.actor.Status.Failure.class).cause();
2052             assertTrue("Failure type", failure instanceof IllegalStateException);
2053         }};
2054     }
2055
2056     @Test
2057     public void testCreateSnapshot() throws Exception {
2058         testCreateSnapshot(true, "testCreateSnapshot");
2059     }
2060
2061     @Test
2062     public void testCreateSnapshotWithNonPersistentData() throws Exception {
2063         testCreateSnapshot(false, "testCreateSnapshotWithNonPersistentData");
2064     }
2065
2066     @SuppressWarnings("serial")
2067     private void testCreateSnapshot(final boolean persistent, final String shardActorName) throws Exception{
2068
2069         final AtomicReference<CountDownLatch> latch = new AtomicReference<>(new CountDownLatch(1));
2070
2071         final AtomicReference<Object> savedSnapshot = new AtomicReference<>();
2072         class TestPersistentDataProvider extends DelegatingPersistentDataProvider {
2073             TestPersistentDataProvider(final DataPersistenceProvider delegate) {
2074                 super(delegate);
2075             }
2076
2077             @Override
2078             public void saveSnapshot(final Object o) {
2079                 savedSnapshot.set(o);
2080                 super.saveSnapshot(o);
2081             }
2082         }
2083
2084         dataStoreContextBuilder.persistent(persistent);
2085
2086         new ShardTestKit(getSystem()) {{
2087             class TestShard extends Shard {
2088
2089                 protected TestShard(AbstractBuilder<?, ?> builder) {
2090                     super(builder);
2091                     setPersistence(new TestPersistentDataProvider(super.persistence()));
2092                 }
2093
2094                 @Override
2095                 public void handleCommand(final Object message) {
2096                     super.handleCommand(message);
2097
2098                     if (message instanceof SaveSnapshotSuccess || message.equals("commit_snapshot")) {
2099                         latch.get().countDown();
2100                     }
2101                 }
2102
2103                 @Override
2104                 public RaftActorContext getRaftActorContext() {
2105                     return super.getRaftActorContext();
2106                 }
2107             }
2108
2109             final Creator<Shard> creator = new Creator<Shard>() {
2110                 @Override
2111                 public Shard create() throws Exception {
2112                     return new TestShard(newShardBuilder());
2113                 }
2114             };
2115
2116             final TestActorRef<Shard> shard = actorFactory.createTestActor(
2117                     Props.create(new DelegatingShardCreator(creator)), shardActorName);
2118
2119             waitUntilLeader(shard);
2120             writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
2121
2122             final NormalizedNode<?,?> expectedRoot = readStore(shard, YangInstanceIdentifier.builder().build());
2123
2124             // Trigger creation of a snapshot by ensuring
2125             final RaftActorContext raftActorContext = ((TestShard) shard.underlyingActor()).getRaftActorContext();
2126             raftActorContext.getSnapshotManager().capture(mock(ReplicatedLogEntry.class), -1);
2127             awaitAndValidateSnapshot(expectedRoot);
2128
2129             raftActorContext.getSnapshotManager().capture(mock(ReplicatedLogEntry.class), -1);
2130             awaitAndValidateSnapshot(expectedRoot);
2131         }
2132
2133         private void awaitAndValidateSnapshot(NormalizedNode<?,?> expectedRoot) throws InterruptedException {
2134             assertEquals("Snapshot saved", true, latch.get().await(5, TimeUnit.SECONDS));
2135
2136             assertTrue("Invalid saved snapshot " + savedSnapshot.get(),
2137                     savedSnapshot.get() instanceof Snapshot);
2138
2139             verifySnapshot((Snapshot)savedSnapshot.get(), expectedRoot);
2140
2141             latch.set(new CountDownLatch(1));
2142             savedSnapshot.set(null);
2143         }
2144
2145         private void verifySnapshot(final Snapshot snapshot, final NormalizedNode<?,?> expectedRoot) {
2146
2147             final NormalizedNode<?, ?> actual = SerializationUtils.deserializeNormalizedNode(snapshot.getState());
2148             assertEquals("Root node", expectedRoot, actual);
2149
2150         }};
2151     }
2152
2153     /**
2154      * This test simply verifies that the applySnapShot logic will work
2155      * @throws ReadFailedException
2156      * @throws DataValidationFailedException
2157      */
2158     @Test
2159     public void testInMemoryDataTreeRestore() throws ReadFailedException, DataValidationFailedException {
2160         final DataTree store = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL);
2161         store.setSchemaContext(SCHEMA_CONTEXT);
2162
2163         final DataTreeModification putTransaction = store.takeSnapshot().newModification();
2164         putTransaction.write(TestModel.TEST_PATH,
2165             ImmutableNodes.containerNode(TestModel.TEST_QNAME));
2166         commitTransaction(store, putTransaction);
2167
2168
2169         final NormalizedNode<?, ?> expected = readStore(store, YangInstanceIdentifier.builder().build());
2170
2171         final DataTreeModification writeTransaction = store.takeSnapshot().newModification();
2172
2173         writeTransaction.delete(YangInstanceIdentifier.builder().build());
2174         writeTransaction.write(YangInstanceIdentifier.builder().build(), expected);
2175
2176         commitTransaction(store, writeTransaction);
2177
2178         final NormalizedNode<?, ?> actual = readStore(store, YangInstanceIdentifier.builder().build());
2179
2180         assertEquals(expected, actual);
2181     }
2182
2183     @Test
2184     public void testRecoveryApplicable(){
2185
2186         final DatastoreContext persistentContext = DatastoreContext.newBuilder().
2187                 shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).persistent(true).build();
2188
2189         final Props persistentProps = Shard.builder().id(shardID).datastoreContext(persistentContext).
2190                 schemaContext(SCHEMA_CONTEXT).props();
2191
2192         final DatastoreContext nonPersistentContext = DatastoreContext.newBuilder().
2193                 shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).persistent(false).build();
2194
2195         final Props nonPersistentProps = Shard.builder().id(shardID).datastoreContext(nonPersistentContext).
2196                 schemaContext(SCHEMA_CONTEXT).props();
2197
2198         new ShardTestKit(getSystem()) {{
2199             final TestActorRef<Shard> shard1 = actorFactory.createTestActor(persistentProps, "testPersistence1");
2200
2201             assertTrue("Recovery Applicable", shard1.underlyingActor().persistence().isRecoveryApplicable());
2202
2203             final TestActorRef<Shard> shard2 = actorFactory.createTestActor(nonPersistentProps, "testPersistence2");
2204
2205             assertFalse("Recovery Not Applicable", shard2.underlyingActor().persistence().isRecoveryApplicable());
2206         }};
2207     }
2208
2209     @Test
2210     public void testOnDatastoreContext() {
2211         new ShardTestKit(getSystem()) {{
2212             dataStoreContextBuilder.persistent(true);
2213
2214             final TestActorRef<Shard> shard = actorFactory.createTestActor(newShardProps(), "testOnDatastoreContext");
2215
2216             assertEquals("isRecoveryApplicable", true,
2217                     shard.underlyingActor().persistence().isRecoveryApplicable());
2218
2219             waitUntilLeader(shard);
2220
2221             shard.tell(dataStoreContextBuilder.persistent(false).build(), ActorRef.noSender());
2222
2223             assertEquals("isRecoveryApplicable", false,
2224                 shard.underlyingActor().persistence().isRecoveryApplicable());
2225
2226             shard.tell(dataStoreContextBuilder.persistent(true).build(), ActorRef.noSender());
2227
2228             assertEquals("isRecoveryApplicable", true,
2229                 shard.underlyingActor().persistence().isRecoveryApplicable());
2230         }};
2231     }
2232
2233     @Test
2234     public void testRegisterRoleChangeListener() throws Exception {
2235         new ShardTestKit(getSystem()) {
2236             {
2237                 final TestActorRef<Shard> shard = actorFactory.createTestActor(
2238                         newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
2239                         "testRegisterRoleChangeListener");
2240
2241                 waitUntilLeader(shard);
2242
2243                 final TestActorRef<MessageCollectorActor> listener =
2244                         TestActorRef.create(getSystem(), Props.create(MessageCollectorActor.class));
2245
2246                 shard.tell(new RegisterRoleChangeListener(), listener);
2247
2248                 MessageCollectorActor.expectFirstMatching(listener, RegisterRoleChangeListenerReply.class);
2249
2250                 ShardLeaderStateChanged leaderStateChanged = MessageCollectorActor.expectFirstMatching(listener,
2251                     ShardLeaderStateChanged.class);
2252                 assertEquals("getLocalShardDataTree present", true,
2253                         leaderStateChanged.getLocalShardDataTree().isPresent());
2254                 assertSame("getLocalShardDataTree", shard.underlyingActor().getDataStore().getDataTree(),
2255                     leaderStateChanged.getLocalShardDataTree().get());
2256
2257                 MessageCollectorActor.clearMessages(listener);
2258
2259                 // Force a leader change
2260
2261                 shard.tell(new RequestVote(10000, "member2", 50, 50), getRef());
2262
2263                 leaderStateChanged = MessageCollectorActor.expectFirstMatching(listener,
2264                         ShardLeaderStateChanged.class);
2265                 assertEquals("getLocalShardDataTree present", false,
2266                         leaderStateChanged.getLocalShardDataTree().isPresent());
2267             }
2268         };
2269     }
2270
2271     @Test
2272     public void testFollowerInitialSyncStatus() throws Exception {
2273         final TestActorRef<Shard> shard = actorFactory.createTestActor(
2274                 newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
2275                 "testFollowerInitialSyncStatus");
2276
2277         shard.underlyingActor().onReceiveCommand(new FollowerInitialSyncUpStatus(false, "member-1-shard-inventory-operational"));
2278
2279         assertEquals(false, shard.underlyingActor().getShardMBean().getFollowerInitialSyncStatus());
2280
2281         shard.underlyingActor().onReceiveCommand(new FollowerInitialSyncUpStatus(true, "member-1-shard-inventory-operational"));
2282
2283         assertEquals(true, shard.underlyingActor().getShardMBean().getFollowerInitialSyncStatus());
2284     }
2285
2286     @Test
2287     public void testClusteredDataChangeListenerDelayedRegistration() throws Exception {
2288         new ShardTestKit(getSystem()) {{
2289             String testName = "testClusteredDataChangeListenerDelayedRegistration";
2290             dataStoreContextBuilder.shardElectionTimeoutFactor(1000);
2291
2292             final MockDataChangeListener listener = new MockDataChangeListener(1);
2293             final ActorRef dclActor = actorFactory.createActor(DataChangeListener.props(listener),
2294                     actorFactory.generateActorId(testName + "-DataChangeListener"));
2295
2296             final TestActorRef<Shard> shard = actorFactory.createTestActor(
2297                     newShardBuilder().props().withDispatcher(Dispatchers.DefaultDispatcherId()),
2298                     actorFactory.generateActorId(testName + "-shard"));
2299
2300             waitUntilNoLeader(shard);
2301
2302             final YangInstanceIdentifier path = TestModel.TEST_PATH;
2303
2304             shard.tell(new RegisterChangeListener(path, dclActor, AsyncDataBroker.DataChangeScope.BASE, true), getRef());
2305             final RegisterChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
2306                 RegisterChangeListenerReply.class);
2307             assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
2308
2309             writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
2310
2311             shard.tell(new ElectionTimeout(), ActorRef.noSender());
2312
2313             listener.waitForChangeEvents();
2314         }};
2315     }
2316
2317     @Test
2318     public void testClusteredDataChangeListenerRegistration() throws Exception {
2319         new ShardTestKit(getSystem()) {{
2320             String testName = "testClusteredDataChangeListenerRegistration";
2321             final ShardIdentifier followerShardID = ShardIdentifier.builder().memberName(
2322                     actorFactory.generateActorId(testName + "-follower")).shardName("inventory").type("config").build();
2323
2324             final ShardIdentifier leaderShardID = ShardIdentifier.builder().memberName(
2325                     actorFactory.generateActorId(testName + "-leader")).shardName("inventory").type("config").build();
2326
2327             final TestActorRef<Shard> followerShard = actorFactory.createTestActor(
2328                     Shard.builder().id(followerShardID).
2329                         datastoreContext(dataStoreContextBuilder.shardElectionTimeoutFactor(1000).build()).
2330                         peerAddresses(Collections.singletonMap(leaderShardID.toString(),
2331                             "akka://test/user/" + leaderShardID.toString())).schemaContext(SCHEMA_CONTEXT).props().
2332                     withDispatcher(Dispatchers.DefaultDispatcherId()), followerShardID.toString());
2333
2334             final TestActorRef<Shard> leaderShard = actorFactory.createTestActor(
2335                     Shard.builder().id(leaderShardID).datastoreContext(newDatastoreContext()).
2336                         peerAddresses(Collections.singletonMap(followerShardID.toString(),
2337                             "akka://test/user/" + followerShardID.toString())).schemaContext(SCHEMA_CONTEXT).props().
2338                     withDispatcher(Dispatchers.DefaultDispatcherId()), leaderShardID.toString());
2339
2340             leaderShard.tell(new ElectionTimeout(), ActorRef.noSender());
2341             String leaderPath = waitUntilLeader(followerShard);
2342             assertEquals("Shard leader path", leaderShard.path().toString(), leaderPath);
2343
2344             final YangInstanceIdentifier path = TestModel.TEST_PATH;
2345             final MockDataChangeListener listener = new MockDataChangeListener(1);
2346             final ActorRef dclActor = actorFactory.createActor(DataChangeListener.props(listener),
2347                     actorFactory.generateActorId(testName + "-DataChangeListener"));
2348
2349             followerShard.tell(new RegisterChangeListener(path, dclActor, AsyncDataBroker.DataChangeScope.BASE, true), getRef());
2350             final RegisterChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
2351                 RegisterChangeListenerReply.class);
2352             assertNotNull("getListenerRegistratioznPath", reply.getListenerRegistrationPath());
2353
2354             writeToStore(followerShard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
2355
2356             listener.waitForChangeEvents();
2357         }};
2358     }
2359
2360     @Test
2361     public void testClusteredDataTreeChangeListenerDelayedRegistration() throws Exception {
2362         new ShardTestKit(getSystem()) {{
2363             String testName = "testClusteredDataTreeChangeListenerDelayedRegistration";
2364             dataStoreContextBuilder.shardElectionTimeoutFactor(1000);
2365
2366             final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
2367             final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener),
2368                     actorFactory.generateActorId(testName + "-DataTreeChangeListener"));
2369
2370             final TestActorRef<Shard> shard = actorFactory.createTestActor(
2371                     newShardBuilder().props().withDispatcher(Dispatchers.DefaultDispatcherId()),
2372                     actorFactory.generateActorId(testName + "-shard"));
2373
2374             waitUntilNoLeader(shard);
2375
2376             final YangInstanceIdentifier path = TestModel.TEST_PATH;
2377
2378             shard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor, true), getRef());
2379             final RegisterDataTreeChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
2380                     RegisterDataTreeChangeListenerReply.class);
2381             assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
2382
2383             writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
2384
2385             shard.tell(new ElectionTimeout(), ActorRef.noSender());
2386
2387             listener.waitForChangeEvents();
2388         }};
2389     }
2390
2391     @Test
2392     public void testClusteredDataTreeChangeListenerRegistration() throws Exception {
2393         new ShardTestKit(getSystem()) {{
2394             String testName = "testClusteredDataTreeChangeListenerRegistration";
2395             final ShardIdentifier followerShardID = ShardIdentifier.builder().memberName(
2396                     actorFactory.generateActorId(testName + "-follower")).shardName("inventory").type("config").build();
2397
2398             final ShardIdentifier leaderShardID = ShardIdentifier.builder().memberName(
2399                     actorFactory.generateActorId(testName + "-leader")).shardName("inventory").type("config").build();
2400
2401             final TestActorRef<Shard> followerShard = actorFactory.createTestActor(
2402                     Shard.builder().id(followerShardID).
2403                         datastoreContext(dataStoreContextBuilder.shardElectionTimeoutFactor(1000).build()).
2404                         peerAddresses(Collections.singletonMap(leaderShardID.toString(),
2405                             "akka://test/user/" + leaderShardID.toString())).schemaContext(SCHEMA_CONTEXT).props().
2406                     withDispatcher(Dispatchers.DefaultDispatcherId()), followerShardID.toString());
2407
2408             final TestActorRef<Shard> leaderShard = actorFactory.createTestActor(
2409                     Shard.builder().id(leaderShardID).datastoreContext(newDatastoreContext()).
2410                         peerAddresses(Collections.singletonMap(followerShardID.toString(),
2411                             "akka://test/user/" + followerShardID.toString())).schemaContext(SCHEMA_CONTEXT).props().
2412                     withDispatcher(Dispatchers.DefaultDispatcherId()), leaderShardID.toString());
2413
2414             leaderShard.tell(new ElectionTimeout(), ActorRef.noSender());
2415             String leaderPath = waitUntilLeader(followerShard);
2416             assertEquals("Shard leader path", leaderShard.path().toString(), leaderPath);
2417
2418             final YangInstanceIdentifier path = TestModel.TEST_PATH;
2419             final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
2420             final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener),
2421                     actorFactory.generateActorId(testName + "-DataTreeChangeListener"));
2422
2423             followerShard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor, true), getRef());
2424             final RegisterDataTreeChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
2425                     RegisterDataTreeChangeListenerReply.class);
2426             assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
2427
2428             writeToStore(followerShard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
2429
2430             listener.waitForChangeEvents();
2431         }};
2432     }
2433
2434     @Test
2435     public void testServerRemoved() throws Exception {
2436         final TestActorRef<MessageCollectorActor> parent = actorFactory.createTestActor(MessageCollectorActor.props());
2437
2438         final ActorRef shard = parent.underlyingActor().context().actorOf(
2439                 newShardBuilder().props().withDispatcher(Dispatchers.DefaultDispatcherId()),
2440                 "testServerRemoved");
2441
2442         shard.tell(new ServerRemoved("test"), ActorRef.noSender());
2443
2444         MessageCollectorActor.expectFirstMatching(parent, ServerRemoved.class);
2445     }
2446 }