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