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