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