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