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