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