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