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