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