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