Fix test failure in ShardTest
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / ShardTest.java
1 /*
2  * Copyright (c) 2014, 2015 Cisco Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8
9 package org.opendaylight.controller.cluster.datastore;
10
11 import static org.junit.Assert.assertEquals;
12 import static org.junit.Assert.assertFalse;
13 import static org.junit.Assert.assertNotNull;
14 import static org.junit.Assert.assertSame;
15 import static org.junit.Assert.assertTrue;
16 import static org.junit.Assert.fail;
17 import static org.mockito.Mockito.doReturn;
18 import static org.mockito.Mockito.inOrder;
19 import static org.mockito.Mockito.mock;
20 import static org.mockito.Mockito.reset;
21 import static org.mockito.Mockito.verify;
22 import static org.opendaylight.controller.cluster.datastore.DataStoreVersions.CURRENT_VERSION;
23 import akka.actor.ActorRef;
24 import akka.actor.ActorSelection;
25 import akka.actor.Props;
26 import akka.actor.Status.Failure;
27 import akka.dispatch.Dispatchers;
28 import akka.dispatch.OnComplete;
29 import akka.japi.Creator;
30 import akka.pattern.Patterns;
31 import akka.persistence.SaveSnapshotSuccess;
32 import akka.testkit.TestActorRef;
33 import akka.util.Timeout;
34 import com.google.common.base.Function;
35 import com.google.common.base.Stopwatch;
36 import com.google.common.util.concurrent.Futures;
37 import com.google.common.util.concurrent.ListenableFuture;
38 import com.google.common.util.concurrent.Uninterruptibles;
39 import java.util.Collections;
40 import java.util.HashSet;
41 import java.util.Set;
42 import java.util.concurrent.CountDownLatch;
43 import java.util.concurrent.TimeUnit;
44 import java.util.concurrent.atomic.AtomicBoolean;
45 import java.util.concurrent.atomic.AtomicReference;
46 import org.junit.Test;
47 import org.mockito.InOrder;
48 import org.opendaylight.controller.cluster.DataPersistenceProvider;
49 import org.opendaylight.controller.cluster.DelegatingPersistentDataProvider;
50 import org.opendaylight.controller.cluster.access.concepts.LocalHistoryIdentifier;
51 import org.opendaylight.controller.cluster.access.concepts.MemberName;
52 import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
53 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
54 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
55 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
56 import org.opendaylight.controller.cluster.datastore.messages.AbortTransaction;
57 import org.opendaylight.controller.cluster.datastore.messages.AbortTransactionReply;
58 import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
59 import org.opendaylight.controller.cluster.datastore.messages.BatchedModificationsReply;
60 import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransaction;
61 import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransactionReply;
62 import org.opendaylight.controller.cluster.datastore.messages.CommitTransaction;
63 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
64 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
65 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
66 import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
67 import org.opendaylight.controller.cluster.datastore.messages.ReadData;
68 import org.opendaylight.controller.cluster.datastore.messages.ReadDataReply;
69 import org.opendaylight.controller.cluster.datastore.messages.ReadyLocalTransaction;
70 import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
71 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListener;
72 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListenerReply;
73 import org.opendaylight.controller.cluster.datastore.messages.RegisterDataTreeChangeListener;
74 import org.opendaylight.controller.cluster.datastore.messages.RegisterDataTreeChangeListenerReply;
75 import org.opendaylight.controller.cluster.datastore.messages.ShardLeaderStateChanged;
76 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
77 import org.opendaylight.controller.cluster.datastore.modification.DeleteModification;
78 import org.opendaylight.controller.cluster.datastore.modification.MergeModification;
79 import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
80 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
81 import org.opendaylight.controller.cluster.datastore.utils.MockDataChangeListener;
82 import org.opendaylight.controller.cluster.datastore.utils.MockDataTreeChangeListener;
83 import org.opendaylight.controller.cluster.datastore.utils.SerializationUtils;
84 import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListener;
85 import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListenerReply;
86 import org.opendaylight.controller.cluster.raft.RaftActorContext;
87 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
88 import org.opendaylight.controller.cluster.raft.ReplicatedLogImplEntry;
89 import org.opendaylight.controller.cluster.raft.Snapshot;
90 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
91 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
92 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
93 import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus;
94 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
95 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
96 import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftState;
97 import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState;
98 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
99 import org.opendaylight.controller.cluster.raft.messages.ServerRemoved;
100 import org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy;
101 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
102 import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor;
103 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
104 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
105 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataBroker;
106 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
107 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
108 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
109 import org.opendaylight.yangtools.yang.data.api.schema.MapNode;
110 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
111 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
112 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidateNode;
113 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidateTip;
114 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification;
115 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException;
116 import org.opendaylight.yangtools.yang.data.api.schema.tree.ModificationType;
117 import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType;
118 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
119 import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder;
120 import org.opendaylight.yangtools.yang.data.impl.schema.tree.InMemoryDataTreeFactory;
121 import scala.concurrent.Await;
122 import scala.concurrent.Future;
123 import scala.concurrent.duration.FiniteDuration;
124
125 public class ShardTest extends AbstractShardTest {
126     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";
127
128     @Test
129     public void testRegisterChangeListener() throws Exception {
130         new ShardTestKit(getSystem()) {{
131             final TestActorRef<Shard> shard = actorFactory.createTestActor(
132                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), "testRegisterChangeListener");
133
134             waitUntilLeader(shard);
135
136             shard.tell(new UpdateSchemaContext(SchemaContextHelper.full()), ActorRef.noSender());
137
138             final MockDataChangeListener listener = new MockDataChangeListener(1);
139             final ActorRef dclActor = actorFactory.createActor(DataChangeListener.props(listener),
140                     "testRegisterChangeListener-DataChangeListener");
141
142             shard.tell(new RegisterChangeListener(TestModel.TEST_PATH,
143                     dclActor, AsyncDataBroker.DataChangeScope.BASE, true), getRef());
144
145             final RegisterChangeListenerReply reply = expectMsgClass(duration("3 seconds"),
146                     RegisterChangeListenerReply.class);
147             final String replyPath = reply.getListenerRegistrationPath().toString();
148             assertTrue("Incorrect reply path: " + replyPath, replyPath.matches(
149                     "akka:\\/\\/test\\/user\\/testRegisterChangeListener\\/\\$.*"));
150
151             final YangInstanceIdentifier path = TestModel.TEST_PATH;
152             writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
153
154             listener.waitForChangeEvents(path);
155         }};
156     }
157
158     @SuppressWarnings("serial")
159     @Test
160     public void testChangeListenerNotifiedWhenNotTheLeaderOnRegistration() throws Exception {
161         // This test tests the timing window in which a change listener is registered before the
162         // shard becomes the leader. We verify that the listener is registered and notified of the
163         // existing data when the shard becomes the leader.
164         new ShardTestKit(getSystem()) {{
165             // For this test, we want to send the RegisterChangeListener message after the shard
166             // has recovered from persistence and before it becomes the leader. So we subclass
167             // Shard to override onReceiveCommand and, when the first ElectionTimeout is received,
168             // we know that the shard has been initialized to a follower and has started the
169             // election process. The following 2 CountDownLatches are used to coordinate the
170             // ElectionTimeout with the sending of the RegisterChangeListener message.
171             final CountDownLatch onFirstElectionTimeout = new CountDownLatch(1);
172             final CountDownLatch onChangeListenerRegistered = new CountDownLatch(1);
173             final Creator<Shard> creator = new Creator<Shard>() {
174                 boolean firstElectionTimeout = true;
175
176                 @Override
177                 public Shard create() throws Exception {
178                     // Use a non persistent provider because this test actually invokes persist on the journal
179                     // this will cause all other messages to not be queued properly after that.
180                     // The basic issue is that you cannot use TestActorRef with a persistent actor (at least when
181                     // it does do a persist)
182                     return new Shard(newShardBuilder()) {
183                         @Override
184                         public void handleCommand(final Object message) {
185                             if(message instanceof ElectionTimeout && firstElectionTimeout) {
186                                 // Got the first ElectionTimeout. We don't forward it to the
187                                 // base Shard yet until we've sent the RegisterChangeListener
188                                 // message. So we signal the onFirstElectionTimeout latch to tell
189                                 // the main thread to send the RegisterChangeListener message and
190                                 // start a thread to wait on the onChangeListenerRegistered latch,
191                                 // which the main thread signals after it has sent the message.
192                                 // After the onChangeListenerRegistered is triggered, we send the
193                                 // original ElectionTimeout message to proceed with the election.
194                                 firstElectionTimeout = false;
195                                 final ActorRef self = getSelf();
196                                 new Thread() {
197                                     @Override
198                                     public void run() {
199                                         Uninterruptibles.awaitUninterruptibly(
200                                                 onChangeListenerRegistered, 5, TimeUnit.SECONDS);
201                                         self.tell(message, self);
202                                     }
203                                 }.start();
204
205                                 onFirstElectionTimeout.countDown();
206                             } else {
207                                 super.handleCommand(message);
208                             }
209                         }
210                     };
211                 }
212             };
213
214             setupInMemorySnapshotStore();
215
216             final MockDataChangeListener listener = new MockDataChangeListener(1);
217             final ActorRef dclActor = actorFactory.createActor(DataChangeListener.props(listener),
218                     "testRegisterChangeListenerWhenNotLeaderInitially-DataChangeListener");
219
220             final TestActorRef<Shard> shard = actorFactory.createTestActor(
221                     Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
222                     "testRegisterChangeListenerWhenNotLeaderInitially");
223
224             final YangInstanceIdentifier path = TestModel.TEST_PATH;
225
226             // Wait until the shard receives the first ElectionTimeout message.
227             assertEquals("Got first ElectionTimeout", true,
228                     onFirstElectionTimeout.await(5, TimeUnit.SECONDS));
229
230             // Now send the RegisterChangeListener and wait for the reply.
231             shard.tell(new RegisterChangeListener(path, dclActor,
232                     AsyncDataBroker.DataChangeScope.SUBTREE, false), getRef());
233
234             final RegisterChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
235                     RegisterChangeListenerReply.class);
236             assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
237
238             // Sanity check - verify the shard is not the leader yet.
239             shard.tell(FindLeader.INSTANCE, getRef());
240             final FindLeaderReply findLeadeReply =
241                     expectMsgClass(duration("5 seconds"), FindLeaderReply.class);
242             assertFalse("Expected the shard not to be the leader", findLeadeReply.getLeaderActor().isPresent());
243
244             // Signal the onChangeListenerRegistered latch to tell the thread above to proceed
245             // with the election process.
246             onChangeListenerRegistered.countDown();
247
248             // Wait for the shard to become the leader and notify our listener with the existing
249             // data in the store.
250             listener.waitForChangeEvents(path);
251         }};
252     }
253
254     @Test
255     public void testRegisterDataTreeChangeListener() throws Exception {
256         new ShardTestKit(getSystem()) {{
257             final TestActorRef<Shard> shard = actorFactory.createTestActor(
258                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), "testRegisterDataTreeChangeListener");
259
260             waitUntilLeader(shard);
261
262             shard.tell(new UpdateSchemaContext(SchemaContextHelper.full()), ActorRef.noSender());
263
264             final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
265             final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener),
266                     "testRegisterDataTreeChangeListener-DataTreeChangeListener");
267
268             shard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor, false), getRef());
269
270             final RegisterDataTreeChangeListenerReply reply = expectMsgClass(duration("3 seconds"),
271                     RegisterDataTreeChangeListenerReply.class);
272             final String replyPath = reply.getListenerRegistrationPath().toString();
273             assertTrue("Incorrect reply path: " + replyPath, replyPath.matches(
274                     "akka:\\/\\/test\\/user\\/testRegisterDataTreeChangeListener\\/\\$.*"));
275
276             final YangInstanceIdentifier path = TestModel.TEST_PATH;
277             writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
278
279             listener.waitForChangeEvents();
280         }};
281     }
282
283     @SuppressWarnings("serial")
284     @Test
285     public void testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration() throws Exception {
286         new ShardTestKit(getSystem()) {{
287             final CountDownLatch onFirstElectionTimeout = new CountDownLatch(1);
288             final CountDownLatch onChangeListenerRegistered = new CountDownLatch(1);
289             final Creator<Shard> creator = new Creator<Shard>() {
290                 boolean firstElectionTimeout = true;
291
292                 @Override
293                 public Shard create() throws Exception {
294                     return new Shard(newShardBuilder()) {
295                         @Override
296                         public void handleCommand(final Object message) {
297                             if(message instanceof ElectionTimeout && firstElectionTimeout) {
298                                 firstElectionTimeout = false;
299                                 final ActorRef self = getSelf();
300                                 new Thread() {
301                                     @Override
302                                     public void run() {
303                                         Uninterruptibles.awaitUninterruptibly(
304                                                 onChangeListenerRegistered, 5, TimeUnit.SECONDS);
305                                         self.tell(message, self);
306                                     }
307                                 }.start();
308
309                                 onFirstElectionTimeout.countDown();
310                             } else {
311                                 super.handleCommand(message);
312                             }
313                         }
314                     };
315                 }
316             };
317
318             setupInMemorySnapshotStore();
319
320             final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
321             final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener),
322                     "testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration-DataChangeListener");
323
324             final TestActorRef<Shard> shard = actorFactory.createTestActor(
325                     Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
326                     "testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration");
327
328             final YangInstanceIdentifier path = TestModel.TEST_PATH;
329
330             assertEquals("Got first ElectionTimeout", true,
331                 onFirstElectionTimeout.await(5, TimeUnit.SECONDS));
332
333             shard.tell(new RegisterDataTreeChangeListener(path, dclActor, false), getRef());
334             final RegisterDataTreeChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
335                 RegisterDataTreeChangeListenerReply.class);
336             assertNotNull("getListenerRegistratioznPath", reply.getListenerRegistrationPath());
337
338             shard.tell(FindLeader.INSTANCE, getRef());
339             final FindLeaderReply findLeadeReply =
340                     expectMsgClass(duration("5 seconds"), FindLeaderReply.class);
341             assertFalse("Expected the shard not to be the leader", findLeadeReply.getLeaderActor().isPresent());
342
343
344             onChangeListenerRegistered.countDown();
345
346             // TODO: investigate why we do not receive data chage events
347             listener.waitForChangeEvents();
348         }};
349     }
350
351     @Test
352     public void testCreateTransaction(){
353         new ShardTestKit(getSystem()) {{
354             final ActorRef shard = actorFactory.createActor(newShardProps(), "testCreateTransaction");
355
356             waitUntilLeader(shard);
357
358             shard.tell(new UpdateSchemaContext(TestModel.createTestContext()), getRef());
359
360             shard.tell(new CreateTransaction(nextTransactionId(), TransactionType.READ_ONLY.ordinal(),
361                     DataStoreVersions.CURRENT_VERSION).toSerializable(), getRef());
362
363             final CreateTransactionReply reply = expectMsgClass(duration("3 seconds"),
364                     CreateTransactionReply.class);
365
366             final String path = reply.getTransactionPath().toString();
367             assertTrue("Unexpected transaction path " + path,
368                     path.startsWith("akka://test/user/testCreateTransaction/shard-member-1:ShardTransactionTest@0:"));
369         }};
370     }
371
372     @Test
373     public void testCreateTransactionOnChain(){
374         new ShardTestKit(getSystem()) {{
375             final ActorRef shard = actorFactory.createActor(newShardProps(), "testCreateTransactionOnChain");
376
377             waitUntilLeader(shard);
378
379             shard.tell(new CreateTransaction(nextTransactionId(),TransactionType.READ_ONLY.ordinal(),
380                     DataStoreVersions.CURRENT_VERSION).toSerializable(), getRef());
381
382             final CreateTransactionReply reply = expectMsgClass(duration("3 seconds"),
383                     CreateTransactionReply.class);
384
385             final String path = reply.getTransactionPath().toString();
386             assertTrue("Unexpected transaction path " + path,
387                     path.startsWith("akka://test/user/testCreateTransactionOnChain/shard-member-1:ShardTransactionTest@0:"));
388         }};
389     }
390
391     @Test
392     public void testPeerAddressResolved() throws Exception {
393         new ShardTestKit(getSystem()) {{
394             ShardIdentifier peerID = ShardIdentifier.create("inventory", MemberName.forName("member-2"), "config");
395             final TestActorRef<Shard> shard = actorFactory.createTestActor(newShardBuilder().
396                     peerAddresses(Collections.<String, String>singletonMap(peerID.toString(), null)).props().
397                         withDispatcher(Dispatchers.DefaultDispatcherId()), "testPeerAddressResolved");
398
399             final String address = "akka://foobar";
400             shard.tell(new PeerAddressResolved(peerID.toString(), address), ActorRef.noSender());
401
402             shard.tell(GetOnDemandRaftState.INSTANCE, getRef());
403             OnDemandRaftState state = expectMsgClass(OnDemandRaftState.class);
404             assertEquals("getPeerAddress", address, state.getPeerAddresses().get(peerID.toString()));
405         }};
406     }
407
408     @Test
409     public void testApplySnapshot() throws Exception {
410
411         final TestActorRef<Shard> shard = actorFactory.createTestActor(newShardProps(), "testApplySnapshot");
412
413         ShardTestKit.waitUntilLeader(shard);
414
415         final DataTree store = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL);
416         store.setSchemaContext(SCHEMA_CONTEXT);
417
418         final ContainerNode container = ImmutableContainerNodeBuilder.create().withNodeIdentifier(
419                 new YangInstanceIdentifier.NodeIdentifier(TestModel.TEST_QNAME)).
420                     withChild(ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).addChild(
421                         ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1)).build()).build();
422
423         writeToStore(store, TestModel.TEST_PATH, container);
424
425         final YangInstanceIdentifier root = YangInstanceIdentifier.EMPTY;
426         final NormalizedNode<?,?> expected = readStore(store, root);
427
428         final Snapshot snapshot = Snapshot.create(SerializationUtils.serializeNormalizedNode(expected),
429                 Collections.<ReplicatedLogEntry>emptyList(), 1, 2, 3, 4);
430
431         shard.underlyingActor().getRaftActorSnapshotCohort().applySnapshot(snapshot.getState());
432
433         final NormalizedNode<?,?> actual = readStore(shard, root);
434
435         assertEquals("Root node", expected, actual);
436     }
437
438     @Test
439     public void testApplyState() throws Exception {
440         final TestActorRef<Shard> shard = actorFactory.createTestActor(
441                 newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), "testApplyState");
442
443         ShardTestKit.waitUntilLeader(shard);
444
445         final DataTree source = setupInMemorySnapshotStore();
446         final DataTreeModification writeMod = source.takeSnapshot().newModification();
447         ContainerNode node = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
448         writeMod.write(TestModel.TEST_PATH, node);
449         writeMod.ready();
450
451         final TransactionIdentifier tx = nextTransactionId();
452         final ApplyState applyState = new ApplyState(null, tx,
453             new ReplicatedLogImplEntry(1, 2, payloadForModification(source, writeMod, tx)));
454
455         shard.tell(applyState, shard);
456
457         Stopwatch sw = Stopwatch.createStarted();
458         while(sw.elapsed(TimeUnit.SECONDS) <= 5) {
459             Uninterruptibles.sleepUninterruptibly(75, TimeUnit.MILLISECONDS);
460
461             final NormalizedNode<?,?> actual = readStore(shard, TestModel.TEST_PATH);
462             if(actual != null) {
463                 assertEquals("Applied state", node, actual);
464                 return;
465             }
466         }
467
468         fail("State was not applied");
469     }
470
471     @Test
472     public void testDataTreeCandidateRecovery() throws Exception {
473         // Set up the InMemorySnapshotStore.
474         final DataTree source = setupInMemorySnapshotStore();
475
476         final DataTreeModification writeMod = source.takeSnapshot().newModification();
477         writeMod.write(TestModel.OUTER_LIST_PATH, ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build());
478         writeMod.ready();
479         InMemoryJournal.addEntry(shardID.toString(), 0, DUMMY_DATA);
480
481         // Set up the InMemoryJournal.
482         InMemoryJournal.addEntry(shardID.toString(), 1, new ReplicatedLogImplEntry(0, 1,
483             payloadForModification(source, writeMod, nextTransactionId())));
484
485         final int nListEntries = 16;
486         final Set<Integer> listEntryKeys = new HashSet<>();
487
488         // Add some ModificationPayload entries
489         for (int i = 1; i <= nListEntries; i++) {
490             listEntryKeys.add(Integer.valueOf(i));
491
492             final YangInstanceIdentifier path = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
493                     .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i).build();
494
495             final DataTreeModification mod = source.takeSnapshot().newModification();
496             mod.merge(path, ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i));
497             mod.ready();
498
499             InMemoryJournal.addEntry(shardID.toString(), i+1, new ReplicatedLogImplEntry(i, 1,
500                 payloadForModification(source, mod, nextTransactionId())));
501         }
502
503         InMemoryJournal.addEntry(shardID.toString(), nListEntries + 2,
504             new ApplyJournalEntries(nListEntries));
505
506         testRecovery(listEntryKeys);
507     }
508
509     @Test
510     public void testConcurrentThreePhaseCommits() throws Throwable {
511         new ShardTestKit(getSystem()) {{
512             final TestActorRef<Shard> shard = actorFactory.createTestActor(
513                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
514                     "testConcurrentThreePhaseCommits");
515
516             waitUntilLeader(shard);
517
518             // Setup 3 simulated transactions with mock cohorts backed by real cohorts.
519
520             final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
521
522             final TransactionIdentifier transactionID1 = nextTransactionId();
523             final MutableCompositeModification modification1 = new MutableCompositeModification();
524             final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore,
525                     TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1);
526
527             final TransactionIdentifier transactionID2 = nextTransactionId();
528             final MutableCompositeModification modification2 = new MutableCompositeModification();
529             final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore,
530                     TestModel.OUTER_LIST_PATH,
531                     ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(),
532                     modification2);
533
534             final TransactionIdentifier transactionID3 = nextTransactionId();
535             final MutableCompositeModification modification3 = new MutableCompositeModification();
536             final ShardDataTreeCohort cohort3 = setupMockWriteTransaction("cohort3", dataStore,
537                     YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
538                         .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(),
539                     ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1),
540                     modification3);
541
542             final long timeoutSec = 5;
543             final FiniteDuration duration = FiniteDuration.create(timeoutSec, TimeUnit.SECONDS);
544             final Timeout timeout = new Timeout(duration);
545
546             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
547             final ReadyTransactionReply readyReply = ReadyTransactionReply.fromSerializable(
548                     expectMsgClass(duration, ReadyTransactionReply.class));
549             assertEquals("Cohort path", shard.path().toString(), readyReply.getCohortPath());
550
551             // Send the CanCommitTransaction message for the first Tx.
552
553             shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
554             final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
555                     expectMsgClass(duration, CanCommitTransactionReply.class));
556             assertEquals("Can commit", true, canCommitReply.getCanCommit());
557
558             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
559             expectMsgClass(duration, ReadyTransactionReply.class);
560
561             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort3, transactionID3, modification3), getRef());
562             expectMsgClass(duration, ReadyTransactionReply.class);
563
564             // Send the CanCommitTransaction message for the next 2 Tx's. These should get queued and
565             // processed after the first Tx completes.
566
567             final Future<Object> canCommitFuture1 = Patterns.ask(shard,
568                     new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), timeout);
569
570             final Future<Object> canCommitFuture2 = Patterns.ask(shard,
571                     new CanCommitTransaction(transactionID3, CURRENT_VERSION).toSerializable(), timeout);
572
573             // Send the CommitTransaction message for the first Tx. After it completes, it should
574             // trigger the 2nd Tx to proceed which should in turn then trigger the 3rd.
575
576             shard.tell(new CommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
577             expectMsgClass(duration, CommitTransactionReply.class);
578
579             // Wait for the next 2 Tx's to complete.
580
581             final AtomicReference<Throwable> caughtEx = new AtomicReference<>();
582             final CountDownLatch commitLatch = new CountDownLatch(2);
583
584             class OnFutureComplete extends OnComplete<Object> {
585                 private final Class<?> expRespType;
586
587                 OnFutureComplete(final Class<?> expRespType) {
588                     this.expRespType = expRespType;
589                 }
590
591                 @Override
592                 public void onComplete(final Throwable error, final Object resp) {
593                     if(error != null) {
594                         caughtEx.set(new AssertionError(getClass().getSimpleName() + " failure", error));
595                     } else {
596                         try {
597                             assertEquals("Commit response type", expRespType, resp.getClass());
598                             onSuccess(resp);
599                         } catch (final Exception e) {
600                             caughtEx.set(e);
601                         }
602                     }
603                 }
604
605                 void onSuccess(final Object resp) throws Exception {
606                 }
607             }
608
609             class OnCommitFutureComplete extends OnFutureComplete {
610                 OnCommitFutureComplete() {
611                     super(CommitTransactionReply.class);
612                 }
613
614                 @Override
615                 public void onComplete(final Throwable error, final Object resp) {
616                     super.onComplete(error, resp);
617                     commitLatch.countDown();
618                 }
619             }
620
621             class OnCanCommitFutureComplete extends OnFutureComplete {
622                 private final TransactionIdentifier transactionID;
623
624                 OnCanCommitFutureComplete(final TransactionIdentifier transactionID) {
625                     super(CanCommitTransactionReply.class);
626                     this.transactionID = transactionID;
627                 }
628
629                 @Override
630                 void onSuccess(final Object resp) throws Exception {
631                     final CanCommitTransactionReply canCommitReply =
632                             CanCommitTransactionReply.fromSerializable(resp);
633                     assertEquals("Can commit", true, canCommitReply.getCanCommit());
634
635                     final Future<Object> commitFuture = Patterns.ask(shard,
636                             new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), timeout);
637                     commitFuture.onComplete(new OnCommitFutureComplete(), getSystem().dispatcher());
638                 }
639             }
640
641             canCommitFuture1.onComplete(new OnCanCommitFutureComplete(transactionID2),
642                     getSystem().dispatcher());
643
644             canCommitFuture2.onComplete(new OnCanCommitFutureComplete(transactionID3),
645                     getSystem().dispatcher());
646
647             final boolean done = commitLatch.await(timeoutSec, TimeUnit.SECONDS);
648
649             if(caughtEx.get() != null) {
650                 throw caughtEx.get();
651             }
652
653             assertEquals("Commits complete", true, done);
654
655             final InOrder inOrder = inOrder(cohort1, cohort2, cohort3);
656             inOrder.verify(cohort1).canCommit();
657             inOrder.verify(cohort1).preCommit();
658             inOrder.verify(cohort1).commit();
659             inOrder.verify(cohort2).canCommit();
660             inOrder.verify(cohort2).preCommit();
661             inOrder.verify(cohort2).commit();
662             inOrder.verify(cohort3).canCommit();
663             inOrder.verify(cohort3).preCommit();
664             inOrder.verify(cohort3).commit();
665
666             // Verify data in the data store.
667
668             verifyOuterListEntry(shard, 1);
669
670             verifyLastApplied(shard, 2);
671         }};
672     }
673
674     @Test
675     public void testBatchedModificationsWithNoCommitOnReady() throws Throwable {
676         new ShardTestKit(getSystem()) {{
677             final TestActorRef<Shard> shard = actorFactory.createTestActor(
678                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
679                     "testBatchedModificationsWithNoCommitOnReady");
680
681             waitUntilLeader(shard);
682
683             final TransactionIdentifier transactionID = nextTransactionId();
684             final FiniteDuration duration = duration("5 seconds");
685
686             final AtomicReference<ShardDataTreeCohort> mockCohort = new AtomicReference<>();
687             final ShardCommitCoordinator.CohortDecorator cohortDecorator = (txID, actual) -> {
688                 if(mockCohort.get() == null) {
689                     mockCohort.set(createDelegatingMockCohort("cohort", actual));
690                 }
691
692                 return mockCohort.get();
693             };
694
695             shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator);
696
697             // Send a BatchedModifications to start a transaction.
698
699             shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH,
700                     ImmutableNodes.containerNode(TestModel.TEST_QNAME), false, false, 1), getRef());
701             expectMsgClass(duration, BatchedModificationsReply.class);
702
703             // Send a couple more BatchedModifications.
704
705             shard.tell(newBatchedModifications(transactionID, TestModel.OUTER_LIST_PATH,
706                     ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), false, false, 2), getRef());
707             expectMsgClass(duration, BatchedModificationsReply.class);
708
709             shard.tell(newBatchedModifications(transactionID, YangInstanceIdentifier.builder(
710                     TestModel.OUTER_LIST_PATH).nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(),
711                     ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1), true, false, 3), getRef());
712             expectMsgClass(duration, ReadyTransactionReply.class);
713
714             // Send the CanCommitTransaction message.
715
716             shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
717             final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
718                     expectMsgClass(duration, CanCommitTransactionReply.class));
719             assertEquals("Can commit", true, canCommitReply.getCanCommit());
720
721             // Send the CanCommitTransaction message.
722
723             shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
724             expectMsgClass(duration, CommitTransactionReply.class);
725
726             final InOrder inOrder = inOrder(mockCohort.get());
727             inOrder.verify(mockCohort.get()).canCommit();
728             inOrder.verify(mockCohort.get()).preCommit();
729             inOrder.verify(mockCohort.get()).commit();
730
731             // Verify data in the data store.
732
733             verifyOuterListEntry(shard, 1);
734         }};
735     }
736
737     @Test
738     public void testBatchedModificationsWithCommitOnReady() throws Throwable {
739         new ShardTestKit(getSystem()) {{
740             final TestActorRef<Shard> shard = actorFactory.createTestActor(
741                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
742                     "testBatchedModificationsWithCommitOnReady");
743
744             waitUntilLeader(shard);
745
746             final TransactionIdentifier transactionID = nextTransactionId();
747             final FiniteDuration duration = duration("5 seconds");
748
749             final AtomicReference<ShardDataTreeCohort> mockCohort = new AtomicReference<>();
750             final ShardCommitCoordinator.CohortDecorator cohortDecorator = (txID, actual) -> {
751                 if(mockCohort.get() == null) {
752                     mockCohort.set(createDelegatingMockCohort("cohort", actual));
753                 }
754
755                 return mockCohort.get();
756             };
757
758             shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator);
759
760             // Send a BatchedModifications to start a transaction.
761
762             shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH,
763                     ImmutableNodes.containerNode(TestModel.TEST_QNAME), false, false, 1), getRef());
764             expectMsgClass(duration, BatchedModificationsReply.class);
765
766             // Send a couple more BatchedModifications.
767
768             shard.tell(newBatchedModifications(transactionID, TestModel.OUTER_LIST_PATH,
769                 ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), false, false, 2), getRef());
770             expectMsgClass(duration, BatchedModificationsReply.class);
771
772             shard.tell(newBatchedModifications(transactionID, YangInstanceIdentifier.builder(
773                     TestModel.OUTER_LIST_PATH).nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(),
774                 ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1), true, true, 3), getRef());
775
776             expectMsgClass(duration, CommitTransactionReply.class);
777
778             final InOrder inOrder = inOrder(mockCohort.get());
779             inOrder.verify(mockCohort.get()).canCommit();
780             inOrder.verify(mockCohort.get()).preCommit();
781             inOrder.verify(mockCohort.get()).commit();
782
783             // Verify data in the data store.
784
785             verifyOuterListEntry(shard, 1);
786         }};
787     }
788
789     @Test(expected=IllegalStateException.class)
790     public void testBatchedModificationsReadyWithIncorrectTotalMessageCount() throws Throwable {
791         new ShardTestKit(getSystem()) {{
792             final TestActorRef<Shard> shard = actorFactory.createTestActor(
793                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
794                     "testBatchedModificationsReadyWithIncorrectTotalMessageCount");
795
796             waitUntilLeader(shard);
797
798             final TransactionIdentifier transactionID = nextTransactionId();
799             final BatchedModifications batched = new BatchedModifications(transactionID, DataStoreVersions.CURRENT_VERSION);
800             batched.setReady(true);
801             batched.setTotalMessagesSent(2);
802
803             shard.tell(batched, getRef());
804
805             final Failure failure = expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
806
807             if(failure != null) {
808                 throw failure.cause();
809             }
810         }};
811     }
812
813     @Test
814     public void testBatchedModificationsWithOperationFailure() throws Throwable {
815         new ShardTestKit(getSystem()) {{
816             final TestActorRef<Shard> shard = actorFactory.createTestActor(
817                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
818                     "testBatchedModificationsWithOperationFailure");
819
820             waitUntilLeader(shard);
821
822             // Test merge with invalid data. An exception should occur when the merge is applied. Note that
823             // write will not validate the children for performance reasons.
824
825             TransactionIdentifier transactionID = nextTransactionId();
826
827             ContainerNode invalidData = ImmutableContainerNodeBuilder.create().withNodeIdentifier(
828                     new YangInstanceIdentifier.NodeIdentifier(TestModel.TEST_QNAME)).
829                         withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build();
830
831             BatchedModifications batched = new BatchedModifications(transactionID, CURRENT_VERSION);
832             batched.addModification(new MergeModification(TestModel.TEST_PATH, invalidData));
833             shard.tell(batched, getRef());
834             Failure failure = expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
835
836             Throwable cause = failure.cause();
837
838             batched = new BatchedModifications(transactionID, DataStoreVersions.CURRENT_VERSION);
839             batched.setReady(true);
840             batched.setTotalMessagesSent(2);
841
842             shard.tell(batched, getRef());
843
844             failure = expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
845             assertEquals("Failure cause", cause, failure.cause());
846         }};
847     }
848
849     @Test
850     public void testBatchedModificationsOnTransactionChain() throws Throwable {
851         new ShardTestKit(getSystem()) {{
852             final TestActorRef<Shard> shard = actorFactory.createTestActor(
853                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
854                     "testBatchedModificationsOnTransactionChain");
855
856             waitUntilLeader(shard);
857
858             final LocalHistoryIdentifier historyId = nextHistoryId();
859             final TransactionIdentifier transactionID1 = new TransactionIdentifier(historyId, 0);
860             final TransactionIdentifier transactionID2 = new TransactionIdentifier(historyId, 1);
861
862             final FiniteDuration duration = duration("5 seconds");
863
864             // Send a BatchedModifications to start a chained write transaction and ready it.
865
866             final ContainerNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
867             final YangInstanceIdentifier path = TestModel.TEST_PATH;
868             shard.tell(newBatchedModifications(transactionID1, path, containerNode, true, false, 1), getRef());
869             expectMsgClass(duration, ReadyTransactionReply.class);
870
871             // Create a read Tx on the same chain.
872
873             shard.tell(new CreateTransaction(transactionID2, TransactionType.READ_ONLY.ordinal(),
874                     DataStoreVersions.CURRENT_VERSION).toSerializable(), getRef());
875
876             final CreateTransactionReply createReply = expectMsgClass(duration("3 seconds"), CreateTransactionReply.class);
877
878             getSystem().actorSelection(createReply.getTransactionPath()).tell(
879                     new ReadData(path, DataStoreVersions.CURRENT_VERSION), getRef());
880             final ReadDataReply readReply = expectMsgClass(duration("3 seconds"), ReadDataReply.class);
881             assertEquals("Read node", containerNode, readReply.getNormalizedNode());
882
883             // Commit the write transaction.
884
885             shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
886             final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
887                     expectMsgClass(duration, CanCommitTransactionReply.class));
888             assertEquals("Can commit", true, canCommitReply.getCanCommit());
889
890             shard.tell(new CommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
891             expectMsgClass(duration, CommitTransactionReply.class);
892
893             // Verify data in the data store.
894
895             final NormalizedNode<?, ?> actualNode = readStore(shard, path);
896             assertEquals("Stored node", containerNode, actualNode);
897         }};
898     }
899
900     @Test
901     public void testOnBatchedModificationsWhenNotLeader() {
902         final AtomicBoolean overrideLeaderCalls = new AtomicBoolean();
903         new ShardTestKit(getSystem()) {{
904             final Creator<Shard> creator = new Creator<Shard>() {
905                 private static final long serialVersionUID = 1L;
906
907                 @Override
908                 public Shard create() throws Exception {
909                     return new Shard(newShardBuilder()) {
910                         @Override
911                         protected boolean isLeader() {
912                             return overrideLeaderCalls.get() ? false : super.isLeader();
913                         }
914
915                         @Override
916                         protected ActorSelection getLeader() {
917                             return overrideLeaderCalls.get() ? getSystem().actorSelection(getRef().path()) :
918                                 super.getLeader();
919                         }
920                     };
921                 }
922             };
923
924             final TestActorRef<Shard> shard = actorFactory.createTestActor(
925                     Props.create(new DelegatingShardCreator(creator)).
926                         withDispatcher(Dispatchers.DefaultDispatcherId()), "testOnBatchedModificationsWhenNotLeader");
927
928             waitUntilLeader(shard);
929
930             overrideLeaderCalls.set(true);
931
932             final BatchedModifications batched = new BatchedModifications(nextTransactionId(), DataStoreVersions.CURRENT_VERSION);
933
934             shard.tell(batched, ActorRef.noSender());
935
936             expectMsgEquals(batched);
937         }};
938     }
939
940     @Test
941     public void testTransactionMessagesWithNoLeader() {
942         new ShardTestKit(getSystem()) {{
943             dataStoreContextBuilder.customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName()).
944                 shardHeartbeatIntervalInMillis(50).shardElectionTimeoutFactor(1);
945             final TestActorRef<Shard> shard = actorFactory.createTestActor(
946                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
947                     "testTransactionMessagesWithNoLeader");
948
949             waitUntilNoLeader(shard);
950
951             final TransactionIdentifier txId = nextTransactionId();
952             shard.tell(new BatchedModifications(txId, 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), txId,
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(txId, 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 TransactionIdentifier transactionID = nextTransactionId();
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 TransactionIdentifier txId = nextTransactionId();
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 TransactionIdentifier txId = nextTransactionId();
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 TransactionIdentifier transactionID = nextTransactionId();
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 TransactionIdentifier transactionID = nextTransactionId();
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 TransactionIdentifier transactionID = nextTransactionId();
1221                 final MutableCompositeModification modification = new MutableCompositeModification();
1222                 modification.addModification(new DeleteModification(YangInstanceIdentifier.EMPTY));
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 TransactionIdentifier transactionID1 = nextTransactionId();
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 TransactionIdentifier transactionID2 = nextTransactionId();
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 TransactionIdentifier transactionID1 = nextTransactionId();
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 TransactionIdentifier transactionID2 = nextTransactionId();
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 TransactionIdentifier transactionID1 = nextTransactionId();
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 TransactionIdentifier transactionID2 = nextTransactionId();
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 TransactionIdentifier transactionID1 = nextTransactionId();
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 TransactionIdentifier transactionID2 = nextTransactionId();
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 TransactionIdentifier transactionID1 = nextTransactionId();
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 TransactionIdentifier transactionID2 = nextTransactionId();
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 TransactionIdentifier transactionID1 = nextTransactionId();
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, transactionID1, 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 TransactionIdentifier transactionID2 = nextTransactionId();
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 TransactionIdentifier transactionID = nextTransactionId();
1616             final Function<ShardDataTreeCohort, ListenableFuture<Void>> preCommit =
1617                           cohort -> {
1618                 final ListenableFuture<Void> preCommitFuture = cohort.preCommit();
1619
1620                 // Simulate an AbortTransaction message occurring during replication, after
1621                 // persisting and before finishing the commit to the in-memory store.
1622                 // We have no followers so due to optimizations in the RaftActor, it does not
1623                 // attempt replication and thus we can't send an AbortTransaction message b/c
1624                 // it would be processed too late after CommitTransaction completes. So we'll
1625                 // simulate an AbortTransaction message occurring during replication by calling
1626                 // the shard directly.
1627                 //
1628                 shard.underlyingActor().doAbortTransaction(transactionID, null);
1629
1630                 return preCommitFuture;
1631             };
1632
1633             final MutableCompositeModification modification = new MutableCompositeModification();
1634             final ShardDataTreeCohort cohort = setupMockWriteTransaction("cohort1", dataStore,
1635                     TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME),
1636                     modification, preCommit);
1637
1638             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification), getRef());
1639             expectMsgClass(duration, ReadyTransactionReply.class);
1640
1641             shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
1642             final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
1643                 expectMsgClass(duration, CanCommitTransactionReply.class));
1644             assertEquals("Can commit", true, canCommitReply.getCanCommit());
1645
1646             shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
1647             expectMsgClass(duration, CommitTransactionReply.class);
1648
1649             final NormalizedNode<?, ?> node = readStore(shard, TestModel.TEST_PATH);
1650
1651             // Since we're simulating an abort occurring during replication and before finish commit,
1652             // the data should still get written to the in-memory store since we've gotten past
1653             // canCommit and preCommit and persisted the data.
1654             assertNotNull(TestModel.TEST_QNAME.getLocalName() + " not found", node);
1655         }};
1656     }
1657
1658     @Test
1659     public void testTransactionCommitTimeout() throws Throwable {
1660         testTransactionCommitTimeout(true);
1661         testTransactionCommitTimeout(false);
1662     }
1663
1664     private void testTransactionCommitTimeout(final boolean readWrite) throws Throwable {
1665         dataStoreContextBuilder.shardTransactionCommitTimeoutInSeconds(1);
1666
1667         new ShardTestKit(getSystem()) {{
1668             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1669                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1670                     "testTransactionCommitTimeout-" + readWrite);
1671
1672             waitUntilLeader(shard);
1673
1674             final FiniteDuration duration = duration("5 seconds");
1675
1676             final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
1677
1678             writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
1679             writeToStore(shard, TestModel.OUTER_LIST_PATH,
1680                     ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build());
1681
1682             // Create 1st Tx - will timeout
1683
1684             final TransactionIdentifier transactionID1 = nextTransactionId();
1685             final MutableCompositeModification modification1 = new MutableCompositeModification();
1686             final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore,
1687                     YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
1688                         .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(),
1689                     ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1),
1690                     modification1);
1691
1692             // Create 2nd Tx
1693
1694             final TransactionIdentifier transactionID2 = nextTransactionId();
1695             final MutableCompositeModification modification2 = new MutableCompositeModification();
1696             final YangInstanceIdentifier listNodePath = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
1697                 .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2).build();
1698             final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort3", dataStore,
1699                     listNodePath,
1700                     ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2),
1701                     modification2);
1702
1703             // Ready the Tx's
1704
1705             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
1706             expectMsgClass(duration, ReadyTransactionReply.class);
1707
1708             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
1709             expectMsgClass(duration, ReadyTransactionReply.class);
1710
1711             // canCommit 1st Tx. We don't send the commit so it should timeout.
1712
1713             shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1714             expectMsgClass(duration, CanCommitTransactionReply.class);
1715
1716             // canCommit the 2nd Tx - it should complete after the 1st Tx times out.
1717
1718             shard.tell(new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef());
1719             expectMsgClass(duration, CanCommitTransactionReply.class);
1720
1721             // Try to commit the 1st Tx - should fail as it's not the current Tx.
1722
1723             shard.tell(new CommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1724             expectMsgClass(duration, akka.actor.Status.Failure.class);
1725
1726             // Commit the 2nd Tx.
1727
1728             shard.tell(new CommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef());
1729             expectMsgClass(duration, CommitTransactionReply.class);
1730
1731             final NormalizedNode<?, ?> node = readStore(shard, listNodePath);
1732             assertNotNull(listNodePath + " not found", node);
1733         }};
1734     }
1735
1736     @Test
1737     public void testTransactionCommitQueueCapacityExceeded() throws Throwable {
1738         dataStoreContextBuilder.shardTransactionCommitQueueCapacity(2);
1739
1740         new ShardTestKit(getSystem()) {{
1741             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1742                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1743                     "testTransactionCommitQueueCapacityExceeded");
1744
1745             waitUntilLeader(shard);
1746
1747             final FiniteDuration duration = duration("5 seconds");
1748
1749             final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
1750
1751             final TransactionIdentifier transactionID1 = nextTransactionId();
1752             final MutableCompositeModification modification1 = new MutableCompositeModification();
1753             final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore,
1754                     TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1);
1755
1756             final TransactionIdentifier transactionID2 = nextTransactionId();
1757             final MutableCompositeModification modification2 = new MutableCompositeModification();
1758             final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore,
1759                     TestModel.OUTER_LIST_PATH,
1760                     ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(),
1761                     modification2);
1762
1763             final TransactionIdentifier transactionID3 = nextTransactionId();
1764             final MutableCompositeModification modification3 = new MutableCompositeModification();
1765             final ShardDataTreeCohort cohort3 = setupMockWriteTransaction("cohort3", dataStore,
1766                     TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification3);
1767
1768             // Ready the Tx's
1769
1770             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
1771             expectMsgClass(duration, ReadyTransactionReply.class);
1772
1773             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
1774             expectMsgClass(duration, ReadyTransactionReply.class);
1775
1776             // The 3rd Tx should exceed queue capacity and fail.
1777
1778             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort3, transactionID3, modification3), getRef());
1779             expectMsgClass(duration, akka.actor.Status.Failure.class);
1780
1781             // canCommit 1st Tx.
1782
1783             shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1784             expectMsgClass(duration, CanCommitTransactionReply.class);
1785
1786             // canCommit the 2nd Tx - it should get queued.
1787
1788             shard.tell(new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef());
1789
1790             // canCommit the 3rd Tx - should exceed queue capacity and fail.
1791
1792             shard.tell(new CanCommitTransaction(transactionID3, CURRENT_VERSION).toSerializable(), getRef());
1793             expectMsgClass(duration, akka.actor.Status.Failure.class);
1794         }};
1795     }
1796
1797     @Test
1798     public void testTransactionCommitWithPriorExpiredCohortEntries() throws Throwable {
1799         dataStoreContextBuilder.shardCommitQueueExpiryTimeoutInMillis(1300).shardTransactionCommitTimeoutInSeconds(1);
1800
1801         new ShardTestKit(getSystem()) {{
1802             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1803                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1804                     "testTransactionCommitWithPriorExpiredCohortEntries");
1805
1806             waitUntilLeader(shard);
1807
1808             final FiniteDuration duration = duration("5 seconds");
1809
1810             final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
1811
1812             final TransactionIdentifier transactionID1 = nextTransactionId();
1813             final MutableCompositeModification modification1 = new MutableCompositeModification();
1814             final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore,
1815                     TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1);
1816
1817             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
1818             expectMsgClass(duration, ReadyTransactionReply.class);
1819
1820             final TransactionIdentifier transactionID2 = nextTransactionId();
1821             final MutableCompositeModification modification2 = new MutableCompositeModification();
1822             final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore,
1823                     TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification2);
1824
1825             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
1826             expectMsgClass(duration, ReadyTransactionReply.class);
1827
1828             final TransactionIdentifier transactionID3 = nextTransactionId();
1829             final MutableCompositeModification modification3 = new MutableCompositeModification();
1830             final ShardDataTreeCohort cohort3 = setupMockWriteTransaction("cohort3", dataStore,
1831                     TestModel.TEST2_PATH, ImmutableNodes.containerNode(TestModel.TEST2_QNAME), modification3);
1832
1833             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort3, transactionID3, modification3), getRef());
1834             expectMsgClass(duration, ReadyTransactionReply.class);
1835
1836             // All Tx's are readied. We'll send canCommit for the last one but not the others. The others
1837             // should expire from the queue and the last one should be processed.
1838
1839             shard.tell(new CanCommitTransaction(transactionID3, CURRENT_VERSION).toSerializable(), getRef());
1840             expectMsgClass(duration, CanCommitTransactionReply.class);
1841         }};
1842     }
1843
1844     @Test
1845     public void testTransactionCommitWithSubsequentExpiredCohortEntry() throws Throwable {
1846         dataStoreContextBuilder.shardCommitQueueExpiryTimeoutInMillis(1300).shardTransactionCommitTimeoutInSeconds(1);
1847
1848         new ShardTestKit(getSystem()) {{
1849             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1850                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1851                     "testTransactionCommitWithSubsequentExpiredCohortEntry");
1852
1853             waitUntilLeader(shard);
1854
1855             final FiniteDuration duration = duration("5 seconds");
1856
1857             final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
1858
1859             final TransactionIdentifier transactionID1 = nextTransactionId();
1860             final MutableCompositeModification modification1 = new MutableCompositeModification();
1861             final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore,
1862                     TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1);
1863
1864             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
1865             expectMsgClass(duration, ReadyTransactionReply.class);
1866
1867             // CanCommit the first one so it's the current in-progress CohortEntry.
1868
1869             shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1870             expectMsgClass(duration, CanCommitTransactionReply.class);
1871
1872             // Ready the second Tx.
1873
1874             final TransactionIdentifier transactionID2 = nextTransactionId();
1875             final MutableCompositeModification modification2 = new MutableCompositeModification();
1876             final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore,
1877                     TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification2);
1878
1879             shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
1880             expectMsgClass(duration, ReadyTransactionReply.class);
1881
1882             // Ready the third Tx.
1883
1884             final TransactionIdentifier transactionID3 = nextTransactionId();
1885             final DataTreeModification modification3 = dataStore.newModification();
1886             new WriteModification(TestModel.TEST2_PATH, ImmutableNodes.containerNode(TestModel.TEST2_QNAME))
1887                     .apply(modification3);
1888                 modification3.ready();
1889             final ReadyLocalTransaction readyMessage = new ReadyLocalTransaction(transactionID3, modification3, true);
1890
1891             shard.tell(readyMessage, getRef());
1892
1893             // Commit the first Tx. After completing, the second should expire from the queue and the third
1894             // Tx committed.
1895
1896             shard.tell(new CommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1897             expectMsgClass(duration, CommitTransactionReply.class);
1898
1899             // Expect commit reply from the third Tx.
1900
1901             expectMsgClass(duration, CommitTransactionReply.class);
1902
1903             final NormalizedNode<?, ?> node = readStore(shard, TestModel.TEST2_PATH);
1904             assertNotNull(TestModel.TEST2_PATH + " not found", node);
1905         }};
1906     }
1907
1908     @Test
1909     public void testCanCommitBeforeReadyFailure() throws Throwable {
1910         new ShardTestKit(getSystem()) {{
1911             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1912                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1913                     "testCanCommitBeforeReadyFailure");
1914
1915             shard.tell(new CanCommitTransaction(nextTransactionId(), CURRENT_VERSION).toSerializable(), getRef());
1916             expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
1917         }};
1918     }
1919
1920     @Test
1921     public void testAbortCurrentTransaction() throws Throwable {
1922         testAbortCurrentTransaction(true);
1923         testAbortCurrentTransaction(false);
1924     }
1925
1926     private void testAbortCurrentTransaction(final boolean readWrite) throws Throwable {
1927         new ShardTestKit(getSystem()) {{
1928             final TestActorRef<Shard> shard = actorFactory.createTestActor(
1929                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1930                     "testAbortCurrentTransaction-" + readWrite);
1931
1932             waitUntilLeader(shard);
1933
1934             // Setup 2 simulated transactions with mock cohorts. The first one will be aborted.
1935
1936             final TransactionIdentifier transactionID1 = nextTransactionId();
1937             final MutableCompositeModification modification1 = new MutableCompositeModification();
1938             final ShardDataTreeCohort cohort1 = mock(ShardDataTreeCohort.class, "cohort1");
1939             doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort1).canCommit();
1940             doReturn(Futures.immediateFuture(null)).when(cohort1).abort();
1941
1942             final TransactionIdentifier transactionID2 = nextTransactionId();
1943             final MutableCompositeModification modification2 = new MutableCompositeModification();
1944             final ShardDataTreeCohort cohort2 = mock(ShardDataTreeCohort.class, "cohort2");
1945             doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort2).canCommit();
1946
1947             final FiniteDuration duration = duration("5 seconds");
1948             final Timeout timeout = new Timeout(duration);
1949
1950             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
1951             expectMsgClass(duration, ReadyTransactionReply.class);
1952
1953             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
1954             expectMsgClass(duration, ReadyTransactionReply.class);
1955
1956             // Send the CanCommitTransaction message for the first Tx.
1957
1958             shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1959             final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
1960                     expectMsgClass(duration, CanCommitTransactionReply.class));
1961             assertEquals("Can commit", true, canCommitReply.getCanCommit());
1962
1963             // Send the CanCommitTransaction message for the 2nd Tx. This should get queued and
1964             // processed after the first Tx completes.
1965
1966             final Future<Object> canCommitFuture = Patterns.ask(shard,
1967                     new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), timeout);
1968
1969             // Send the AbortTransaction message for the first Tx. This should trigger the 2nd
1970             // Tx to proceed.
1971
1972             shard.tell(new AbortTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
1973             expectMsgClass(duration, AbortTransactionReply.class);
1974
1975             // Wait for the 2nd Tx to complete the canCommit phase.
1976
1977             Await.ready(canCommitFuture, duration);
1978
1979             final InOrder inOrder = inOrder(cohort1, cohort2);
1980             inOrder.verify(cohort1).canCommit();
1981             inOrder.verify(cohort2).canCommit();
1982         }};
1983     }
1984
1985     @Test
1986     public void testAbortQueuedTransaction() throws Throwable {
1987         testAbortQueuedTransaction(true);
1988         testAbortQueuedTransaction(false);
1989     }
1990
1991     private void testAbortQueuedTransaction(final boolean readWrite) throws Throwable {
1992         dataStoreContextBuilder.shardTransactionCommitTimeoutInSeconds(1);
1993         new ShardTestKit(getSystem()) {{
1994             final AtomicReference<CountDownLatch> cleaupCheckLatch = new AtomicReference<>();
1995             @SuppressWarnings("serial")
1996             final Creator<Shard> creator = () -> new Shard(newShardBuilder()) {
1997                 @Override
1998                 public void handleCommand(final Object message) {
1999                     super.handleCommand(message);
2000                     if(TX_COMMIT_TIMEOUT_CHECK_MESSAGE.equals(message)) {
2001                         if(cleaupCheckLatch.get() != null) {
2002                             cleaupCheckLatch.get().countDown();
2003                         }
2004                     }
2005                 }
2006             };
2007
2008             final TestActorRef<Shard> shard = actorFactory.createTestActor(
2009                     Props.create(new DelegatingShardCreator(creator)).withDispatcher(
2010                             Dispatchers.DefaultDispatcherId()), "testAbortQueuedTransaction-" + readWrite);
2011
2012             waitUntilLeader(shard);
2013
2014             final TransactionIdentifier transactionID = nextTransactionId();
2015             final MutableCompositeModification modification = new MutableCompositeModification();
2016             final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort");
2017             doReturn(Futures.immediateFuture(null)).when(cohort).abort();
2018
2019             final FiniteDuration duration = duration("5 seconds");
2020
2021             // Ready the tx.
2022
2023             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification), getRef());
2024             expectMsgClass(duration, ReadyTransactionReply.class);
2025
2026             assertEquals("getPendingTxCommitQueueSize", 1, shard.underlyingActor().getPendingTxCommitQueueSize());
2027
2028             // Send the AbortTransaction message.
2029
2030             shard.tell(new AbortTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
2031             expectMsgClass(duration, AbortTransactionReply.class);
2032
2033             verify(cohort).abort();
2034
2035             // Verify the tx cohort is removed from queue at the cleanup check interval.
2036
2037             cleaupCheckLatch.set(new CountDownLatch(1));
2038             assertEquals("TX_COMMIT_TIMEOUT_CHECK_MESSAGE received", true,
2039                     cleaupCheckLatch.get().await(5, TimeUnit.SECONDS));
2040
2041             assertEquals("getPendingTxCommitQueueSize", 0, shard.underlyingActor().getPendingTxCommitQueueSize());
2042
2043             // Now send CanCommitTransaction - should fail.
2044
2045             shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
2046
2047             Throwable failure = expectMsgClass(duration, akka.actor.Status.Failure.class).cause();
2048             assertTrue("Failure type", failure instanceof IllegalStateException);
2049         }};
2050     }
2051
2052     @Test
2053     public void testCreateSnapshot() throws Exception {
2054         testCreateSnapshot(true, "testCreateSnapshot");
2055     }
2056
2057     @Test
2058     public void testCreateSnapshotWithNonPersistentData() throws Exception {
2059         testCreateSnapshot(false, "testCreateSnapshotWithNonPersistentData");
2060     }
2061
2062     private void testCreateSnapshot(final boolean persistent, final String shardActorName) throws Exception{
2063
2064         final AtomicReference<CountDownLatch> latch = new AtomicReference<>(new CountDownLatch(1));
2065
2066         final AtomicReference<Object> savedSnapshot = new AtomicReference<>();
2067         class TestPersistentDataProvider extends DelegatingPersistentDataProvider {
2068             TestPersistentDataProvider(final DataPersistenceProvider delegate) {
2069                 super(delegate);
2070             }
2071
2072             @Override
2073             public void saveSnapshot(final Object o) {
2074                 savedSnapshot.set(o);
2075                 super.saveSnapshot(o);
2076             }
2077         }
2078
2079         dataStoreContextBuilder.persistent(persistent);
2080
2081         new ShardTestKit(getSystem()) {{
2082             class TestShard extends Shard {
2083
2084                 protected TestShard(AbstractBuilder<?, ?> builder) {
2085                     super(builder);
2086                     setPersistence(new TestPersistentDataProvider(super.persistence()));
2087                 }
2088
2089                 @Override
2090                 public void handleCommand(final Object message) {
2091                     super.handleCommand(message);
2092
2093                     // XXX:  commit_snapshot equality check references RaftActorSnapshotMessageSupport.COMMIT_SNAPSHOT
2094                     if (message instanceof SaveSnapshotSuccess || "commit_snapshot".equals(message.toString())) {
2095                         latch.get().countDown();
2096                     }
2097                 }
2098
2099                 @Override
2100                 public RaftActorContext getRaftActorContext() {
2101                     return super.getRaftActorContext();
2102                 }
2103             }
2104
2105             final Creator<Shard> creator = () -> new TestShard(newShardBuilder());
2106
2107             final TestActorRef<Shard> shard = actorFactory.createTestActor(
2108                     Props.create(new DelegatingShardCreator(creator)).
2109                         withDispatcher(Dispatchers.DefaultDispatcherId()), shardActorName);
2110
2111             waitUntilLeader(shard);
2112             writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
2113
2114             final NormalizedNode<?,?> expectedRoot = readStore(shard, YangInstanceIdentifier.EMPTY);
2115
2116             // Trigger creation of a snapshot by ensuring
2117             final RaftActorContext raftActorContext = ((TestShard) shard.underlyingActor()).getRaftActorContext();
2118             raftActorContext.getSnapshotManager().capture(mock(ReplicatedLogEntry.class), -1);
2119             awaitAndValidateSnapshot(expectedRoot);
2120
2121             raftActorContext.getSnapshotManager().capture(mock(ReplicatedLogEntry.class), -1);
2122             awaitAndValidateSnapshot(expectedRoot);
2123         }
2124
2125         private void awaitAndValidateSnapshot(NormalizedNode<?,?> expectedRoot) throws InterruptedException {
2126             assertEquals("Snapshot saved", true, latch.get().await(5, TimeUnit.SECONDS));
2127
2128             assertTrue("Invalid saved snapshot " + savedSnapshot.get(),
2129                     savedSnapshot.get() instanceof Snapshot);
2130
2131             verifySnapshot((Snapshot)savedSnapshot.get(), expectedRoot);
2132
2133             latch.set(new CountDownLatch(1));
2134             savedSnapshot.set(null);
2135         }
2136
2137         private void verifySnapshot(final Snapshot snapshot, final NormalizedNode<?,?> expectedRoot) {
2138
2139             final NormalizedNode<?, ?> actual = SerializationUtils.deserializeNormalizedNode(snapshot.getState());
2140             assertEquals("Root node", expectedRoot, actual);
2141
2142         }};
2143     }
2144
2145     /**
2146      * This test simply verifies that the applySnapShot logic will work
2147      * @throws ReadFailedException
2148      * @throws DataValidationFailedException
2149      */
2150     @Test
2151     public void testInMemoryDataTreeRestore() throws ReadFailedException, DataValidationFailedException {
2152         final DataTree store = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL);
2153         store.setSchemaContext(SCHEMA_CONTEXT);
2154
2155         final DataTreeModification putTransaction = store.takeSnapshot().newModification();
2156         putTransaction.write(TestModel.TEST_PATH,
2157             ImmutableNodes.containerNode(TestModel.TEST_QNAME));
2158         commitTransaction(store, putTransaction);
2159
2160
2161         final NormalizedNode<?, ?> expected = readStore(store, YangInstanceIdentifier.EMPTY);
2162
2163         final DataTreeModification writeTransaction = store.takeSnapshot().newModification();
2164
2165         writeTransaction.delete(YangInstanceIdentifier.EMPTY);
2166         writeTransaction.write(YangInstanceIdentifier.EMPTY, expected);
2167
2168         commitTransaction(store, writeTransaction);
2169
2170         final NormalizedNode<?, ?> actual = readStore(store, YangInstanceIdentifier.EMPTY);
2171
2172         assertEquals(expected, actual);
2173     }
2174
2175     @Test
2176     public void testRecoveryApplicable(){
2177
2178         final DatastoreContext persistentContext = DatastoreContext.newBuilder().
2179                 shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).persistent(true).build();
2180
2181         final Props persistentProps = Shard.builder().id(shardID).datastoreContext(persistentContext).
2182                 schemaContext(SCHEMA_CONTEXT).props();
2183
2184         final DatastoreContext nonPersistentContext = DatastoreContext.newBuilder().
2185                 shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).persistent(false).build();
2186
2187         final Props nonPersistentProps = Shard.builder().id(shardID).datastoreContext(nonPersistentContext).
2188                 schemaContext(SCHEMA_CONTEXT).props();
2189
2190         new ShardTestKit(getSystem()) {{
2191             final TestActorRef<Shard> shard1 = actorFactory.createTestActor(persistentProps, "testPersistence1");
2192
2193             assertTrue("Recovery Applicable", shard1.underlyingActor().persistence().isRecoveryApplicable());
2194
2195             final TestActorRef<Shard> shard2 = actorFactory.createTestActor(nonPersistentProps, "testPersistence2");
2196
2197             assertFalse("Recovery Not Applicable", shard2.underlyingActor().persistence().isRecoveryApplicable());
2198         }};
2199     }
2200
2201     @Test
2202     public void testOnDatastoreContext() {
2203         new ShardTestKit(getSystem()) {{
2204             dataStoreContextBuilder.persistent(true);
2205
2206             final TestActorRef<Shard> shard = actorFactory.createTestActor(newShardProps(), "testOnDatastoreContext");
2207
2208             assertEquals("isRecoveryApplicable", true,
2209                     shard.underlyingActor().persistence().isRecoveryApplicable());
2210
2211             waitUntilLeader(shard);
2212
2213             shard.tell(dataStoreContextBuilder.persistent(false).build(), ActorRef.noSender());
2214
2215             assertEquals("isRecoveryApplicable", false,
2216                 shard.underlyingActor().persistence().isRecoveryApplicable());
2217
2218             shard.tell(dataStoreContextBuilder.persistent(true).build(), ActorRef.noSender());
2219
2220             assertEquals("isRecoveryApplicable", true,
2221                 shard.underlyingActor().persistence().isRecoveryApplicable());
2222         }};
2223     }
2224
2225     @Test
2226     public void testRegisterRoleChangeListener() throws Exception {
2227         new ShardTestKit(getSystem()) {
2228             {
2229                 final TestActorRef<Shard> shard = actorFactory.createTestActor(
2230                         newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
2231                         "testRegisterRoleChangeListener");
2232
2233                 waitUntilLeader(shard);
2234
2235                 final TestActorRef<MessageCollectorActor> listener =
2236                         TestActorRef.create(getSystem(), Props.create(MessageCollectorActor.class));
2237
2238                 shard.tell(new RegisterRoleChangeListener(), listener);
2239
2240                 MessageCollectorActor.expectFirstMatching(listener, RegisterRoleChangeListenerReply.class);
2241
2242                 ShardLeaderStateChanged leaderStateChanged = MessageCollectorActor.expectFirstMatching(listener,
2243                     ShardLeaderStateChanged.class);
2244                 assertEquals("getLocalShardDataTree present", true,
2245                         leaderStateChanged.getLocalShardDataTree().isPresent());
2246                 assertSame("getLocalShardDataTree", shard.underlyingActor().getDataStore().getDataTree(),
2247                     leaderStateChanged.getLocalShardDataTree().get());
2248
2249                 MessageCollectorActor.clearMessages(listener);
2250
2251                 // Force a leader change
2252
2253                 shard.tell(new RequestVote(10000, "member2", 50, 50), getRef());
2254
2255                 leaderStateChanged = MessageCollectorActor.expectFirstMatching(listener,
2256                         ShardLeaderStateChanged.class);
2257                 assertEquals("getLocalShardDataTree present", false,
2258                         leaderStateChanged.getLocalShardDataTree().isPresent());
2259             }
2260         };
2261     }
2262
2263     @Test
2264     public void testFollowerInitialSyncStatus() throws Exception {
2265         final TestActorRef<Shard> shard = actorFactory.createTestActor(
2266                 newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
2267                 "testFollowerInitialSyncStatus");
2268
2269         shard.underlyingActor().handleNonRaftCommand(new FollowerInitialSyncUpStatus(false, "member-1-shard-inventory-operational"));
2270
2271         assertEquals(false, shard.underlyingActor().getShardMBean().getFollowerInitialSyncStatus());
2272
2273         shard.underlyingActor().handleNonRaftCommand(new FollowerInitialSyncUpStatus(true, "member-1-shard-inventory-operational"));
2274
2275         assertEquals(true, shard.underlyingActor().getShardMBean().getFollowerInitialSyncStatus());
2276     }
2277
2278     @Test
2279     public void testClusteredDataChangeListenerDelayedRegistration() throws Exception {
2280         new ShardTestKit(getSystem()) {{
2281             String testName = "testClusteredDataChangeListenerDelayedRegistration";
2282             dataStoreContextBuilder.shardElectionTimeoutFactor(1000).
2283                     customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName());
2284
2285             final MockDataChangeListener listener = new MockDataChangeListener(1);
2286             final ActorRef dclActor = actorFactory.createActor(DataChangeListener.props(listener),
2287                     actorFactory.generateActorId(testName + "-DataChangeListener"));
2288
2289             setupInMemorySnapshotStore();
2290
2291             final TestActorRef<Shard> shard = actorFactory.createTestActor(
2292                     newShardBuilder().props().withDispatcher(Dispatchers.DefaultDispatcherId()),
2293                     actorFactory.generateActorId(testName + "-shard"));
2294
2295             waitUntilNoLeader(shard);
2296
2297             final YangInstanceIdentifier path = TestModel.TEST_PATH;
2298
2299             shard.tell(new RegisterChangeListener(path, dclActor, AsyncDataBroker.DataChangeScope.BASE, true), getRef());
2300             final RegisterChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
2301                 RegisterChangeListenerReply.class);
2302             assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
2303
2304             shard.tell(DatastoreContext.newBuilderFrom(dataStoreContextBuilder.build()).
2305                     customRaftPolicyImplementation(null).build(), ActorRef.noSender());
2306
2307             listener.waitForChangeEvents();
2308         }};
2309     }
2310
2311     @Test
2312     public void testClusteredDataChangeListenerRegistration() throws Exception {
2313         new ShardTestKit(getSystem()) {{
2314             String testName = "testClusteredDataChangeListenerRegistration";
2315             final ShardIdentifier followerShardID = ShardIdentifier.create("inventory",
2316                     MemberName.forName(actorFactory.generateActorId(testName + "-follower")), "config");
2317
2318             final ShardIdentifier leaderShardID = ShardIdentifier.create("inventory",
2319                     MemberName.forName(actorFactory.generateActorId(testName + "-leader")), "config");
2320
2321             final TestActorRef<Shard> followerShard = actorFactory.createTestActor(
2322                     Shard.builder().id(followerShardID).
2323                         datastoreContext(dataStoreContextBuilder.shardElectionTimeoutFactor(1000).build()).
2324                         peerAddresses(Collections.singletonMap(leaderShardID.toString(),
2325                             "akka://test/user/" + leaderShardID.toString())).schemaContext(SCHEMA_CONTEXT).props().
2326                     withDispatcher(Dispatchers.DefaultDispatcherId()), followerShardID.toString());
2327
2328             final TestActorRef<Shard> leaderShard = actorFactory.createTestActor(
2329                     Shard.builder().id(leaderShardID).datastoreContext(newDatastoreContext()).
2330                         peerAddresses(Collections.singletonMap(followerShardID.toString(),
2331                             "akka://test/user/" + followerShardID.toString())).schemaContext(SCHEMA_CONTEXT).props().
2332                     withDispatcher(Dispatchers.DefaultDispatcherId()), leaderShardID.toString());
2333
2334             leaderShard.tell(ElectionTimeout.INSTANCE, ActorRef.noSender());
2335             String leaderPath = waitUntilLeader(followerShard);
2336             assertEquals("Shard leader path", leaderShard.path().toString(), leaderPath);
2337
2338             final YangInstanceIdentifier path = TestModel.TEST_PATH;
2339             final MockDataChangeListener listener = new MockDataChangeListener(1);
2340             final ActorRef dclActor = actorFactory.createActor(DataChangeListener.props(listener),
2341                     actorFactory.generateActorId(testName + "-DataChangeListener"));
2342
2343             followerShard.tell(new RegisterChangeListener(path, dclActor, AsyncDataBroker.DataChangeScope.BASE, true), getRef());
2344             final RegisterChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
2345                 RegisterChangeListenerReply.class);
2346             assertNotNull("getListenerRegistratioznPath", reply.getListenerRegistrationPath());
2347
2348             writeToStore(followerShard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
2349
2350             listener.waitForChangeEvents();
2351         }};
2352     }
2353
2354     @Test
2355     public void testClusteredDataTreeChangeListenerDelayedRegistration() throws Exception {
2356         new ShardTestKit(getSystem()) {{
2357             String testName = "testClusteredDataTreeChangeListenerDelayedRegistration";
2358             dataStoreContextBuilder.shardElectionTimeoutFactor(1000).
2359                     customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName());
2360
2361             final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
2362             final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener),
2363                     actorFactory.generateActorId(testName + "-DataTreeChangeListener"));
2364
2365             setupInMemorySnapshotStore();
2366
2367             final TestActorRef<Shard> shard = actorFactory.createTestActor(
2368                     newShardBuilder().props().withDispatcher(Dispatchers.DefaultDispatcherId()),
2369                     actorFactory.generateActorId(testName + "-shard"));
2370
2371             waitUntilNoLeader(shard);
2372
2373             final YangInstanceIdentifier path = TestModel.TEST_PATH;
2374
2375             shard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor, true), getRef());
2376             final RegisterDataTreeChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
2377                     RegisterDataTreeChangeListenerReply.class);
2378             assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
2379
2380             shard.tell(DatastoreContext.newBuilderFrom(dataStoreContextBuilder.build()).
2381                     customRaftPolicyImplementation(null).build(), ActorRef.noSender());
2382
2383             listener.waitForChangeEvents();
2384         }};
2385     }
2386
2387     @Test
2388     public void testClusteredDataTreeChangeListenerRegistration() throws Exception {
2389         new ShardTestKit(getSystem()) {{
2390             String testName = "testClusteredDataTreeChangeListenerRegistration";
2391             final ShardIdentifier followerShardID = ShardIdentifier.create("inventory",
2392                     MemberName.forName(actorFactory.generateActorId(testName + "-follower")), "config");
2393
2394             final ShardIdentifier leaderShardID = ShardIdentifier.create("inventory",
2395                     MemberName.forName(actorFactory.generateActorId(testName + "-leader")), "config");
2396
2397             final TestActorRef<Shard> followerShard = actorFactory.createTestActor(
2398                     Shard.builder().id(followerShardID).
2399                         datastoreContext(dataStoreContextBuilder.shardElectionTimeoutFactor(1000).build()).
2400                         peerAddresses(Collections.singletonMap(leaderShardID.toString(),
2401                             "akka://test/user/" + leaderShardID.toString())).schemaContext(SCHEMA_CONTEXT).props().
2402                     withDispatcher(Dispatchers.DefaultDispatcherId()), followerShardID.toString());
2403
2404             final TestActorRef<Shard> leaderShard = actorFactory.createTestActor(
2405                     Shard.builder().id(leaderShardID).datastoreContext(newDatastoreContext()).
2406                         peerAddresses(Collections.singletonMap(followerShardID.toString(),
2407                             "akka://test/user/" + followerShardID.toString())).schemaContext(SCHEMA_CONTEXT).props().
2408                     withDispatcher(Dispatchers.DefaultDispatcherId()), leaderShardID.toString());
2409
2410             leaderShard.tell(ElectionTimeout.INSTANCE, ActorRef.noSender());
2411             String leaderPath = waitUntilLeader(followerShard);
2412             assertEquals("Shard leader path", leaderShard.path().toString(), leaderPath);
2413
2414             final YangInstanceIdentifier path = TestModel.TEST_PATH;
2415             final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
2416             final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener),
2417                     actorFactory.generateActorId(testName + "-DataTreeChangeListener"));
2418
2419             followerShard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor, true), getRef());
2420             final RegisterDataTreeChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
2421                     RegisterDataTreeChangeListenerReply.class);
2422             assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
2423
2424             writeToStore(followerShard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
2425
2426             listener.waitForChangeEvents();
2427         }};
2428     }
2429
2430     @Test
2431     public void testServerRemoved() throws Exception {
2432         final TestActorRef<MessageCollectorActor> parent = actorFactory.createTestActor(MessageCollectorActor.props());
2433
2434         final ActorRef shard = parent.underlyingActor().context().actorOf(
2435                 newShardBuilder().props().withDispatcher(Dispatchers.DefaultDispatcherId()),
2436                 "testServerRemoved");
2437
2438         shard.tell(new ServerRemoved("test"), ActorRef.noSender());
2439
2440         MessageCollectorActor.expectFirstMatching(parent, ServerRemoved.class);
2441     }
2442 }