Refactor DataPersistenceProviders and RaftActor#persistence
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / ShardTest.java
1 package org.opendaylight.controller.cluster.datastore;
2
3 import static org.junit.Assert.assertEquals;
4 import static org.junit.Assert.assertFalse;
5 import static org.junit.Assert.assertNotNull;
6 import static org.junit.Assert.assertNull;
7 import static org.junit.Assert.assertTrue;
8 import static org.mockito.Mockito.doReturn;
9 import static org.mockito.Mockito.inOrder;
10 import static org.mockito.Mockito.mock;
11 import static org.opendaylight.controller.cluster.datastore.DataStoreVersions.CURRENT_VERSION;
12 import akka.actor.ActorRef;
13 import akka.actor.ActorSelection;
14 import akka.actor.PoisonPill;
15 import akka.actor.Props;
16 import akka.dispatch.Dispatchers;
17 import akka.dispatch.OnComplete;
18 import akka.japi.Creator;
19 import akka.pattern.Patterns;
20 import akka.testkit.TestActorRef;
21 import akka.util.Timeout;
22 import com.google.common.base.Function;
23 import com.google.common.base.Optional;
24 import com.google.common.util.concurrent.Futures;
25 import com.google.common.util.concurrent.ListenableFuture;
26 import com.google.common.util.concurrent.MoreExecutors;
27 import com.google.common.util.concurrent.Uninterruptibles;
28 import java.io.IOException;
29 import java.util.Collections;
30 import java.util.HashSet;
31 import java.util.List;
32 import java.util.Map;
33 import java.util.Set;
34 import java.util.concurrent.CountDownLatch;
35 import java.util.concurrent.ExecutionException;
36 import java.util.concurrent.TimeUnit;
37 import java.util.concurrent.atomic.AtomicBoolean;
38 import java.util.concurrent.atomic.AtomicReference;
39 import org.junit.Test;
40 import org.mockito.InOrder;
41 import org.opendaylight.controller.cluster.DataPersistenceProvider;
42 import org.opendaylight.controller.cluster.DelegatingPersistentDataProvider;
43 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
44 import org.opendaylight.controller.cluster.datastore.messages.AbortTransaction;
45 import org.opendaylight.controller.cluster.datastore.messages.AbortTransactionReply;
46 import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
47 import org.opendaylight.controller.cluster.datastore.messages.BatchedModificationsReply;
48 import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransaction;
49 import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransactionReply;
50 import org.opendaylight.controller.cluster.datastore.messages.CommitTransaction;
51 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
52 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
53 import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction;
54 import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
55 import org.opendaylight.controller.cluster.datastore.messages.ReadData;
56 import org.opendaylight.controller.cluster.datastore.messages.ReadDataReply;
57 import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
58 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListener;
59 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListenerReply;
60 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
61 import org.opendaylight.controller.cluster.datastore.modification.DeleteModification;
62 import org.opendaylight.controller.cluster.datastore.modification.MergeModification;
63 import org.opendaylight.controller.cluster.datastore.modification.Modification;
64 import org.opendaylight.controller.cluster.datastore.modification.ModificationPayload;
65 import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
66 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
67 import org.opendaylight.controller.cluster.datastore.utils.MessageCollectorActor;
68 import org.opendaylight.controller.cluster.datastore.utils.MockDataChangeListener;
69 import org.opendaylight.controller.cluster.datastore.utils.SerializationUtils;
70 import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListener;
71 import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListenerReply;
72 import org.opendaylight.controller.cluster.raft.RaftActorContext;
73 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
74 import org.opendaylight.controller.cluster.raft.ReplicatedLogImplEntry;
75 import org.opendaylight.controller.cluster.raft.Snapshot;
76 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
77 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
78 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
79 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
80 import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus;
81 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
82 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
83 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
84 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
85 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
86 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
87 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataBroker;
88 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
89 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore;
90 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStoreFactory;
91 import org.opendaylight.controller.protobuff.messages.cohort3pc.ThreePhaseCommitCohortMessages;
92 import org.opendaylight.controller.protobuff.messages.transaction.ShardTransactionMessages.CreateTransactionReply;
93 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
94 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
95 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
96 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.PathArgument;
97 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
98 import org.opendaylight.yangtools.yang.data.api.schema.DataContainerChild;
99 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
100 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
101 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
102 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
103 import scala.concurrent.Await;
104 import scala.concurrent.Future;
105 import scala.concurrent.duration.FiniteDuration;
106
107 public class ShardTest extends AbstractShardTest {
108
109     @Test
110     public void testRegisterChangeListener() throws Exception {
111         new ShardTestKit(getSystem()) {{
112             TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
113                     newShardProps(),  "testRegisterChangeListener");
114
115             waitUntilLeader(shard);
116
117             shard.tell(new UpdateSchemaContext(SchemaContextHelper.full()), ActorRef.noSender());
118
119             MockDataChangeListener listener = new MockDataChangeListener(1);
120             ActorRef dclActor = getSystem().actorOf(DataChangeListener.props(listener),
121                     "testRegisterChangeListener-DataChangeListener");
122
123             shard.tell(new RegisterChangeListener(TestModel.TEST_PATH,
124                     dclActor, AsyncDataBroker.DataChangeScope.BASE), getRef());
125
126             RegisterChangeListenerReply reply = expectMsgClass(duration("3 seconds"),
127                     RegisterChangeListenerReply.class);
128             String replyPath = reply.getListenerRegistrationPath().toString();
129             assertTrue("Incorrect reply path: " + replyPath, replyPath.matches(
130                     "akka:\\/\\/test\\/user\\/testRegisterChangeListener\\/\\$.*"));
131
132             YangInstanceIdentifier path = TestModel.TEST_PATH;
133             writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
134
135             listener.waitForChangeEvents(path);
136
137             dclActor.tell(PoisonPill.getInstance(), ActorRef.noSender());
138             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
139         }};
140     }
141
142     @SuppressWarnings("serial")
143     @Test
144     public void testChangeListenerNotifiedWhenNotTheLeaderOnRegistration() throws Exception {
145         // This test tests the timing window in which a change listener is registered before the
146         // shard becomes the leader. We verify that the listener is registered and notified of the
147         // existing data when the shard becomes the leader.
148         new ShardTestKit(getSystem()) {{
149             // For this test, we want to send the RegisterChangeListener message after the shard
150             // has recovered from persistence and before it becomes the leader. So we subclass
151             // Shard to override onReceiveCommand and, when the first ElectionTimeout is received,
152             // we know that the shard has been initialized to a follower and has started the
153             // election process. The following 2 CountDownLatches are used to coordinate the
154             // ElectionTimeout with the sending of the RegisterChangeListener message.
155             final CountDownLatch onFirstElectionTimeout = new CountDownLatch(1);
156             final CountDownLatch onChangeListenerRegistered = new CountDownLatch(1);
157             Creator<Shard> creator = new Creator<Shard>() {
158                 boolean firstElectionTimeout = true;
159
160                 @Override
161                 public Shard create() throws Exception {
162                     // Use a non persistent provider because this test actually invokes persist on the journal
163                     // this will cause all other messages to not be queued properly after that.
164                     // The basic issue is that you cannot use TestActorRef with a persistent actor (at least when
165                     // it does do a persist)
166                     return new Shard(shardID, Collections.<String,String>emptyMap(),
167                             dataStoreContextBuilder.persistent(false).build(), SCHEMA_CONTEXT) {
168                         @Override
169                         public void onReceiveCommand(final Object message) throws Exception {
170                             if(message instanceof ElectionTimeout && firstElectionTimeout) {
171                                 // Got the first ElectionTimeout. We don't forward it to the
172                                 // base Shard yet until we've sent the RegisterChangeListener
173                                 // message. So we signal the onFirstElectionTimeout latch to tell
174                                 // the main thread to send the RegisterChangeListener message and
175                                 // start a thread to wait on the onChangeListenerRegistered latch,
176                                 // which the main thread signals after it has sent the message.
177                                 // After the onChangeListenerRegistered is triggered, we send the
178                                 // original ElectionTimeout message to proceed with the election.
179                                 firstElectionTimeout = false;
180                                 final ActorRef self = getSelf();
181                                 new Thread() {
182                                     @Override
183                                     public void run() {
184                                         Uninterruptibles.awaitUninterruptibly(
185                                                 onChangeListenerRegistered, 5, TimeUnit.SECONDS);
186                                         self.tell(message, self);
187                                     }
188                                 }.start();
189
190                                 onFirstElectionTimeout.countDown();
191                             } else {
192                                 super.onReceiveCommand(message);
193                             }
194                         }
195                     };
196                 }
197             };
198
199             MockDataChangeListener listener = new MockDataChangeListener(1);
200             ActorRef dclActor = getSystem().actorOf(DataChangeListener.props(listener),
201                     "testRegisterChangeListenerWhenNotLeaderInitially-DataChangeListener");
202
203             TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
204                     Props.create(new DelegatingShardCreator(creator)),
205                     "testRegisterChangeListenerWhenNotLeaderInitially");
206
207             // Write initial data into the in-memory store.
208             YangInstanceIdentifier path = TestModel.TEST_PATH;
209             writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
210
211             // Wait until the shard receives the first ElectionTimeout message.
212             assertEquals("Got first ElectionTimeout", true,
213                     onFirstElectionTimeout.await(5, TimeUnit.SECONDS));
214
215             // Now send the RegisterChangeListener and wait for the reply.
216             shard.tell(new RegisterChangeListener(path, dclActor,
217                     AsyncDataBroker.DataChangeScope.SUBTREE), getRef());
218
219             RegisterChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
220                     RegisterChangeListenerReply.class);
221             assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
222
223             // Sanity check - verify the shard is not the leader yet.
224             shard.tell(new FindLeader(), getRef());
225             FindLeaderReply findLeadeReply =
226                     expectMsgClass(duration("5 seconds"), FindLeaderReply.class);
227             assertNull("Expected the shard not to be the leader", findLeadeReply.getLeaderActor());
228
229             // Signal the onChangeListenerRegistered latch to tell the thread above to proceed
230             // with the election process.
231             onChangeListenerRegistered.countDown();
232
233             // Wait for the shard to become the leader and notify our listener with the existing
234             // data in the store.
235             listener.waitForChangeEvents(path);
236
237             dclActor.tell(PoisonPill.getInstance(), ActorRef.noSender());
238             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
239         }};
240     }
241
242     @Test
243     public void testCreateTransaction(){
244         new ShardTestKit(getSystem()) {{
245             ActorRef shard = getSystem().actorOf(newShardProps(), "testCreateTransaction");
246
247             waitUntilLeader(shard);
248
249             shard.tell(new UpdateSchemaContext(TestModel.createTestContext()), getRef());
250
251             shard.tell(new CreateTransaction("txn-1",
252                     TransactionProxy.TransactionType.READ_ONLY.ordinal() ).toSerializable(), getRef());
253
254             CreateTransactionReply reply = expectMsgClass(duration("3 seconds"),
255                     CreateTransactionReply.class);
256
257             String path = reply.getTransactionActorPath().toString();
258             assertTrue("Unexpected transaction path " + path,
259                     path.contains("akka://test/user/testCreateTransaction/shard-txn-1"));
260
261             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
262         }};
263     }
264
265     @Test
266     public void testCreateTransactionOnChain(){
267         new ShardTestKit(getSystem()) {{
268             final ActorRef shard = getSystem().actorOf(newShardProps(), "testCreateTransactionOnChain");
269
270             waitUntilLeader(shard);
271
272             shard.tell(new CreateTransaction("txn-1",
273                     TransactionProxy.TransactionType.READ_ONLY.ordinal() , "foobar").toSerializable(),
274                     getRef());
275
276             CreateTransactionReply reply = expectMsgClass(duration("3 seconds"),
277                     CreateTransactionReply.class);
278
279             String path = reply.getTransactionActorPath().toString();
280             assertTrue("Unexpected transaction path " + path,
281                     path.contains("akka://test/user/testCreateTransactionOnChain/shard-txn-1"));
282
283             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
284         }};
285     }
286
287     @SuppressWarnings("serial")
288     @Test
289     public void testPeerAddressResolved() throws Exception {
290         new ShardTestKit(getSystem()) {{
291             final CountDownLatch recoveryComplete = new CountDownLatch(1);
292             class TestShard extends Shard {
293                 TestShard() {
294                     super(shardID, Collections.<String, String>singletonMap(shardID.toString(), null),
295                             newDatastoreContext(), SCHEMA_CONTEXT);
296                 }
297
298                 Map<String, String> getPeerAddresses() {
299                     return getRaftActorContext().getPeerAddresses();
300                 }
301
302                 @Override
303                 protected void onRecoveryComplete() {
304                     try {
305                         super.onRecoveryComplete();
306                     } finally {
307                         recoveryComplete.countDown();
308                     }
309                 }
310             }
311
312             final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
313                     Props.create(new DelegatingShardCreator(new Creator<Shard>() {
314                         @Override
315                         public TestShard create() throws Exception {
316                             return new TestShard();
317                         }
318                     })), "testPeerAddressResolved");
319
320             //waitUntilLeader(shard);
321             assertEquals("Recovery complete", true,
322                     Uninterruptibles.awaitUninterruptibly(recoveryComplete, 5, TimeUnit.SECONDS));
323
324             String address = "akka://foobar";
325             shard.underlyingActor().onReceiveCommand(new PeerAddressResolved(shardID.toString(), address));
326
327             assertEquals("getPeerAddresses", address,
328                     ((TestShard)shard.underlyingActor()).getPeerAddresses().get(shardID.toString()));
329
330             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
331         }};
332     }
333
334     @Test
335     public void testApplySnapshot() throws Exception {
336         TestActorRef<Shard> shard = TestActorRef.create(getSystem(), newShardProps(),
337                 "testApplySnapshot");
338
339         InMemoryDOMDataStore store = new InMemoryDOMDataStore("OPER", MoreExecutors.sameThreadExecutor());
340         store.onGlobalContextUpdated(SCHEMA_CONTEXT);
341
342         writeToStore(store, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
343
344         YangInstanceIdentifier root = YangInstanceIdentifier.builder().build();
345         NormalizedNode<?,?> expected = readStore(store, root);
346
347         ApplySnapshot applySnapshot = new ApplySnapshot(Snapshot.create(
348                 SerializationUtils.serializeNormalizedNode(expected),
349                 Collections.<ReplicatedLogEntry>emptyList(), 1, 2, 3, 4));
350
351         shard.underlyingActor().onReceiveCommand(applySnapshot);
352
353         NormalizedNode<?,?> actual = readStore(shard, root);
354
355         assertEquals("Root node", expected, actual);
356
357         shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
358     }
359
360     @Test
361     public void testApplyState() throws Exception {
362
363         TestActorRef<Shard> shard = TestActorRef.create(getSystem(), newShardProps(), "testApplyState");
364
365         NormalizedNode<?, ?> node = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
366
367         ApplyState applyState = new ApplyState(null, "test", new ReplicatedLogImplEntry(1, 2,
368                 newModificationPayload(new WriteModification(TestModel.TEST_PATH, node))));
369
370         shard.underlyingActor().onReceiveCommand(applyState);
371
372         NormalizedNode<?,?> actual = readStore(shard, TestModel.TEST_PATH);
373         assertEquals("Applied state", node, actual);
374
375         shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
376     }
377
378     @Test
379     public void testRecovery() throws Exception {
380
381         // Set up the InMemorySnapshotStore.
382
383         InMemoryDOMDataStore testStore = InMemoryDOMDataStoreFactory.create("Test", null, null);
384         testStore.onGlobalContextUpdated(SCHEMA_CONTEXT);
385
386         writeToStore(testStore, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
387
388         NormalizedNode<?, ?> root = readStore(testStore, YangInstanceIdentifier.builder().build());
389
390         InMemorySnapshotStore.addSnapshot(shardID.toString(), Snapshot.create(
391                 SerializationUtils.serializeNormalizedNode(root),
392                 Collections.<ReplicatedLogEntry>emptyList(), 0, 1, -1, -1));
393
394         // Set up the InMemoryJournal.
395
396         InMemoryJournal.addEntry(shardID.toString(), 0, new ReplicatedLogImplEntry(0, 1, newModificationPayload(
397                   new WriteModification(TestModel.OUTER_LIST_PATH,
398                           ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build()))));
399
400         int nListEntries = 16;
401         Set<Integer> listEntryKeys = new HashSet<>();
402
403         // Add some ModificationPayload entries
404         for(int i = 1; i <= nListEntries; i++) {
405             listEntryKeys.add(Integer.valueOf(i));
406             YangInstanceIdentifier path = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
407                     .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i).build();
408             Modification mod = new MergeModification(path,
409                     ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i));
410             InMemoryJournal.addEntry(shardID.toString(), i, new ReplicatedLogImplEntry(i, 1,
411                     newModificationPayload(mod)));
412         }
413
414         InMemoryJournal.addEntry(shardID.toString(), nListEntries + 1,
415                 new ApplyJournalEntries(nListEntries));
416
417         testRecovery(listEntryKeys);
418     }
419
420     private ModificationPayload newModificationPayload(final Modification... mods) throws IOException {
421         MutableCompositeModification compMod = new MutableCompositeModification();
422         for(Modification mod: mods) {
423             compMod.addModification(mod);
424         }
425
426         return new ModificationPayload(compMod);
427     }
428
429     @SuppressWarnings({ "unchecked" })
430     @Test
431     public void testConcurrentThreePhaseCommits() throws Throwable {
432         new ShardTestKit(getSystem()) {{
433             final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
434                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
435                     "testConcurrentThreePhaseCommits");
436
437             waitUntilLeader(shard);
438
439             final String transactionID1 = "tx1";
440             final String transactionID2 = "tx2";
441             final String transactionID3 = "tx3";
442
443             final AtomicReference<DOMStoreThreePhaseCommitCohort> mockCohort1 = new AtomicReference<>();
444             final AtomicReference<DOMStoreThreePhaseCommitCohort> mockCohort2 = new AtomicReference<>();
445             final AtomicReference<DOMStoreThreePhaseCommitCohort> mockCohort3 = new AtomicReference<>();
446             ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() {
447                 @Override
448                 public DOMStoreThreePhaseCommitCohort decorate(String transactionID, DOMStoreThreePhaseCommitCohort actual) {
449                     if(transactionID.equals(transactionID1)) {
450                         mockCohort1.set(createDelegatingMockCohort("cohort1", actual));
451                         return mockCohort1.get();
452                     } else if(transactionID.equals(transactionID2)) {
453                         mockCohort2.set(createDelegatingMockCohort("cohort2", actual));
454                         return mockCohort2.get();
455                     } else {
456                         mockCohort3.set(createDelegatingMockCohort("cohort3", actual));
457                         return mockCohort3.get();
458                     }
459                 }
460             };
461
462             shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator);
463
464             long timeoutSec = 5;
465             final FiniteDuration duration = FiniteDuration.create(timeoutSec, TimeUnit.SECONDS);
466             final Timeout timeout = new Timeout(duration);
467
468             // Send a BatchedModifications message for the first transaction.
469
470             shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH,
471                     ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef());
472             BatchedModificationsReply batchedReply = expectMsgClass(duration, BatchedModificationsReply.class);
473             assertEquals("getCohortPath", shard.path().toString(), batchedReply.getCohortPath());
474             assertEquals("getNumBatched", 1, batchedReply.getNumBatched());
475
476             // Send the CanCommitTransaction message for the first Tx.
477
478             shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef());
479             CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
480                     expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
481             assertEquals("Can commit", true, canCommitReply.getCanCommit());
482
483             // Send BatchedModifications for the next 2 Tx's.
484
485             shard.tell(newBatchedModifications(transactionID2, TestModel.OUTER_LIST_PATH,
486                     ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), true), getRef());
487             expectMsgClass(duration, BatchedModificationsReply.class);
488
489             shard.tell(newBatchedModifications(transactionID3, YangInstanceIdentifier.builder(
490                     TestModel.OUTER_LIST_PATH).nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(),
491                     ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1), true), getRef());
492             expectMsgClass(duration, BatchedModificationsReply.class);
493
494             // Send the CanCommitTransaction message for the next 2 Tx's. These should get queued and
495             // processed after the first Tx completes.
496
497             Future<Object> canCommitFuture1 = Patterns.ask(shard,
498                     new CanCommitTransaction(transactionID2).toSerializable(), timeout);
499
500             Future<Object> canCommitFuture2 = Patterns.ask(shard,
501                     new CanCommitTransaction(transactionID3).toSerializable(), timeout);
502
503             // Send the CommitTransaction message for the first Tx. After it completes, it should
504             // trigger the 2nd Tx to proceed which should in turn then trigger the 3rd.
505
506             shard.tell(new CommitTransaction(transactionID1).toSerializable(), getRef());
507             expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
508
509             // Wait for the next 2 Tx's to complete.
510
511             final AtomicReference<Throwable> caughtEx = new AtomicReference<>();
512             final CountDownLatch commitLatch = new CountDownLatch(2);
513
514             class OnFutureComplete extends OnComplete<Object> {
515                 private final Class<?> expRespType;
516
517                 OnFutureComplete(final Class<?> expRespType) {
518                     this.expRespType = expRespType;
519                 }
520
521                 @Override
522                 public void onComplete(final Throwable error, final Object resp) {
523                     if(error != null) {
524                         caughtEx.set(new AssertionError(getClass().getSimpleName() + " failure", error));
525                     } else {
526                         try {
527                             assertEquals("Commit response type", expRespType, resp.getClass());
528                             onSuccess(resp);
529                         } catch (Exception e) {
530                             caughtEx.set(e);
531                         }
532                     }
533                 }
534
535                 void onSuccess(final Object resp) throws Exception {
536                 }
537             }
538
539             class OnCommitFutureComplete extends OnFutureComplete {
540                 OnCommitFutureComplete() {
541                     super(CommitTransactionReply.SERIALIZABLE_CLASS);
542                 }
543
544                 @Override
545                 public void onComplete(final Throwable error, final Object resp) {
546                     super.onComplete(error, resp);
547                     commitLatch.countDown();
548                 }
549             }
550
551             class OnCanCommitFutureComplete extends OnFutureComplete {
552                 private final String transactionID;
553
554                 OnCanCommitFutureComplete(final String transactionID) {
555                     super(CanCommitTransactionReply.SERIALIZABLE_CLASS);
556                     this.transactionID = transactionID;
557                 }
558
559                 @Override
560                 void onSuccess(final Object resp) throws Exception {
561                     CanCommitTransactionReply canCommitReply =
562                             CanCommitTransactionReply.fromSerializable(resp);
563                     assertEquals("Can commit", true, canCommitReply.getCanCommit());
564
565                     Future<Object> commitFuture = Patterns.ask(shard,
566                             new CommitTransaction(transactionID).toSerializable(), timeout);
567                     commitFuture.onComplete(new OnCommitFutureComplete(), getSystem().dispatcher());
568                 }
569             }
570
571             canCommitFuture1.onComplete(new OnCanCommitFutureComplete(transactionID2),
572                     getSystem().dispatcher());
573
574             canCommitFuture2.onComplete(new OnCanCommitFutureComplete(transactionID3),
575                     getSystem().dispatcher());
576
577             boolean done = commitLatch.await(timeoutSec, TimeUnit.SECONDS);
578
579             if(caughtEx.get() != null) {
580                 throw caughtEx.get();
581             }
582
583             assertEquals("Commits complete", true, done);
584
585             InOrder inOrder = inOrder(mockCohort1.get(), mockCohort2.get(), mockCohort3.get());
586             inOrder.verify(mockCohort1.get()).canCommit();
587             inOrder.verify(mockCohort1.get()).preCommit();
588             inOrder.verify(mockCohort1.get()).commit();
589             inOrder.verify(mockCohort2.get()).canCommit();
590             inOrder.verify(mockCohort2.get()).preCommit();
591             inOrder.verify(mockCohort2.get()).commit();
592             inOrder.verify(mockCohort3.get()).canCommit();
593             inOrder.verify(mockCohort3.get()).preCommit();
594             inOrder.verify(mockCohort3.get()).commit();
595
596             // Verify data in the data store.
597
598             NormalizedNode<?, ?> outerList = readStore(shard, TestModel.OUTER_LIST_PATH);
599             assertNotNull(TestModel.OUTER_LIST_QNAME.getLocalName() + " not found", outerList);
600             assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " value is not Iterable",
601                     outerList.getValue() instanceof Iterable);
602             Object entry = ((Iterable<Object>)outerList.getValue()).iterator().next();
603             assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " entry is not MapEntryNode",
604                        entry instanceof MapEntryNode);
605             MapEntryNode mapEntry = (MapEntryNode)entry;
606             Optional<DataContainerChild<? extends PathArgument, ?>> idLeaf =
607                     mapEntry.getChild(new YangInstanceIdentifier.NodeIdentifier(TestModel.ID_QNAME));
608             assertTrue("Missing leaf " + TestModel.ID_QNAME.getLocalName(), idLeaf.isPresent());
609             assertEquals(TestModel.ID_QNAME.getLocalName() + " value", 1, idLeaf.get().getValue());
610
611             verifyLastApplied(shard, 2);
612
613             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
614         }};
615     }
616
617     private BatchedModifications newBatchedModifications(String transactionID, YangInstanceIdentifier path,
618             NormalizedNode<?, ?> data, boolean ready) {
619         return newBatchedModifications(transactionID, null, path, data, ready);
620     }
621
622     private BatchedModifications newBatchedModifications(String transactionID, String transactionChainID,
623             YangInstanceIdentifier path, NormalizedNode<?, ?> data, boolean ready) {
624         BatchedModifications batched = new BatchedModifications(transactionID, CURRENT_VERSION, transactionChainID);
625         batched.addModification(new WriteModification(path, data));
626         batched.setReady(ready);
627         return batched;
628     }
629
630     @SuppressWarnings("unchecked")
631     @Test
632     public void testMultipleBatchedModifications() throws Throwable {
633         new ShardTestKit(getSystem()) {{
634             final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
635                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
636                     "testMultipleBatchedModifications");
637
638             waitUntilLeader(shard);
639
640             final String transactionID = "tx";
641             FiniteDuration duration = duration("5 seconds");
642
643             final AtomicReference<DOMStoreThreePhaseCommitCohort> mockCohort = new AtomicReference<>();
644             ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() {
645                 @Override
646                 public DOMStoreThreePhaseCommitCohort decorate(String txID, DOMStoreThreePhaseCommitCohort actual) {
647                     if(mockCohort.get() == null) {
648                         mockCohort.set(createDelegatingMockCohort("cohort", actual));
649                     }
650
651                     return mockCohort.get();
652                 }
653             };
654
655             shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator);
656
657             // Send a BatchedModifications to start a transaction.
658
659             shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH,
660                     ImmutableNodes.containerNode(TestModel.TEST_QNAME), false), getRef());
661             expectMsgClass(duration, BatchedModificationsReply.class);
662
663             // Send a couple more BatchedModifications.
664
665             shard.tell(newBatchedModifications(transactionID, TestModel.OUTER_LIST_PATH,
666                     ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), false), getRef());
667             expectMsgClass(duration, BatchedModificationsReply.class);
668
669             shard.tell(newBatchedModifications(transactionID, YangInstanceIdentifier.builder(
670                     TestModel.OUTER_LIST_PATH).nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(),
671                     ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1), true), getRef());
672             expectMsgClass(duration, BatchedModificationsReply.class);
673
674             // Send the CanCommitTransaction message.
675
676             shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
677             CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
678                     expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
679             assertEquals("Can commit", true, canCommitReply.getCanCommit());
680
681             // Send the CanCommitTransaction message.
682
683             shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef());
684             expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
685
686             InOrder inOrder = inOrder(mockCohort.get());
687             inOrder.verify(mockCohort.get()).canCommit();
688             inOrder.verify(mockCohort.get()).preCommit();
689             inOrder.verify(mockCohort.get()).commit();
690
691             // Verify data in the data store.
692
693             NormalizedNode<?, ?> outerList = readStore(shard, TestModel.OUTER_LIST_PATH);
694             assertNotNull(TestModel.OUTER_LIST_QNAME.getLocalName() + " not found", outerList);
695             assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " value is not Iterable",
696                     outerList.getValue() instanceof Iterable);
697             Object entry = ((Iterable<Object>)outerList.getValue()).iterator().next();
698             assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " entry is not MapEntryNode",
699                        entry instanceof MapEntryNode);
700             MapEntryNode mapEntry = (MapEntryNode)entry;
701             Optional<DataContainerChild<? extends PathArgument, ?>> idLeaf =
702                     mapEntry.getChild(new YangInstanceIdentifier.NodeIdentifier(TestModel.ID_QNAME));
703             assertTrue("Missing leaf " + TestModel.ID_QNAME.getLocalName(), idLeaf.isPresent());
704             assertEquals(TestModel.ID_QNAME.getLocalName() + " value", 1, idLeaf.get().getValue());
705
706             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
707         }};
708     }
709
710     @Test
711     public void testBatchedModificationsOnTransactionChain() throws Throwable {
712         new ShardTestKit(getSystem()) {{
713             final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
714                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
715                     "testBatchedModificationsOnTransactionChain");
716
717             waitUntilLeader(shard);
718
719             String transactionChainID = "txChain";
720             String transactionID1 = "tx1";
721             String transactionID2 = "tx2";
722
723             FiniteDuration duration = duration("5 seconds");
724
725             // Send a BatchedModifications to start a chained write transaction and ready it.
726
727             ContainerNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
728             YangInstanceIdentifier path = TestModel.TEST_PATH;
729             shard.tell(newBatchedModifications(transactionID1, transactionChainID, path,
730                     containerNode, true), getRef());
731             expectMsgClass(duration, BatchedModificationsReply.class);
732
733             // Create a read Tx on the same chain.
734
735             shard.tell(new CreateTransaction(transactionID2, TransactionProxy.TransactionType.READ_ONLY.ordinal() ,
736                     transactionChainID).toSerializable(), getRef());
737
738             CreateTransactionReply createReply = expectMsgClass(duration("3 seconds"), CreateTransactionReply.class);
739
740             getSystem().actorSelection(createReply.getTransactionActorPath()).tell(new ReadData(path), getRef());
741             ReadDataReply readReply = expectMsgClass(duration("3 seconds"), ReadDataReply.class);
742             assertEquals("Read node", containerNode, readReply.getNormalizedNode());
743
744             // Commit the write transaction.
745
746             shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef());
747             CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
748                     expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
749             assertEquals("Can commit", true, canCommitReply.getCanCommit());
750
751             shard.tell(new CommitTransaction(transactionID1).toSerializable(), getRef());
752             expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
753
754             // Verify data in the data store.
755
756             NormalizedNode<?, ?> actualNode = readStore(shard, path);
757             assertEquals("Stored node", containerNode, actualNode);
758
759             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
760         }};
761     }
762
763     @Test
764     public void testOnBatchedModificationsWhenNotLeader() {
765         final AtomicBoolean overrideLeaderCalls = new AtomicBoolean();
766         new ShardTestKit(getSystem()) {{
767             Creator<Shard> creator = new Creator<Shard>() {
768                 @Override
769                 public Shard create() throws Exception {
770                     return new Shard(shardID, Collections.<String,String>emptyMap(),
771                             newDatastoreContext(), SCHEMA_CONTEXT) {
772                         @Override
773                         protected boolean isLeader() {
774                             return overrideLeaderCalls.get() ? false : super.isLeader();
775                         }
776
777                         @Override
778                         protected ActorSelection getLeader() {
779                             return overrideLeaderCalls.get() ? getSystem().actorSelection(getRef().path()) :
780                                 super.getLeader();
781                         }
782                     };
783                 }
784             };
785
786             TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
787                     Props.create(new DelegatingShardCreator(creator)), "testOnBatchedModificationsWhenNotLeader");
788
789             waitUntilLeader(shard);
790
791             overrideLeaderCalls.set(true);
792
793             BatchedModifications batched = new BatchedModifications("tx", DataStoreVersions.CURRENT_VERSION, "");
794
795             shard.tell(batched, ActorRef.noSender());
796
797             expectMsgEquals(batched);
798
799             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
800         }};
801     }
802
803     @Test
804     public void testCommitWithPersistenceDisabled() throws Throwable {
805         dataStoreContextBuilder.persistent(false);
806         new ShardTestKit(getSystem()) {{
807             final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
808                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
809                     "testCommitWithPersistenceDisabled");
810
811             waitUntilLeader(shard);
812
813             String transactionID = "tx";
814             FiniteDuration duration = duration("5 seconds");
815
816             // Send a BatchedModifications to start a transaction.
817
818             NormalizedNode<?, ?> containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
819             shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH, containerNode, true), getRef());
820             expectMsgClass(duration, BatchedModificationsReply.class);
821
822             // Send the CanCommitTransaction message.
823
824             shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
825             CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
826                     expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
827             assertEquals("Can commit", true, canCommitReply.getCanCommit());
828
829             // Send the CanCommitTransaction message.
830
831             shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef());
832             expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
833
834             NormalizedNode<?, ?> actualNode = readStore(shard, TestModel.TEST_PATH);
835             assertEquals(TestModel.TEST_QNAME.getLocalName(), containerNode, actualNode);
836
837             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
838         }};
839     }
840
841     @Test
842     public void testCommitWhenTransactionHasNoModifications(){
843         // Note that persistence is enabled which would normally result in the entry getting written to the journal
844         // but here that need not happen
845         new ShardTestKit(getSystem()) {
846             {
847                 final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
848                         newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
849                         "testCommitWhenTransactionHasNoModifications");
850
851                 waitUntilLeader(shard);
852
853                 String transactionID = "tx1";
854                 MutableCompositeModification modification = new MutableCompositeModification();
855                 DOMStoreThreePhaseCommitCohort cohort = mock(DOMStoreThreePhaseCommitCohort.class, "cohort1");
856                 doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit();
857                 doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).preCommit();
858                 doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).commit();
859
860                 FiniteDuration duration = duration("5 seconds");
861
862                 // Simulate the ForwardedReadyTransaction messages that would be sent
863                 // by the ShardTransaction.
864
865                 shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION,
866                         cohort, modification, true), getRef());
867                 expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
868
869                 // Send the CanCommitTransaction message.
870
871                 shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
872                 CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
873                         expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
874                 assertEquals("Can commit", true, canCommitReply.getCanCommit());
875
876                 shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef());
877                 expectMsgClass(duration, ThreePhaseCommitCohortMessages.CommitTransactionReply.class);
878
879                 InOrder inOrder = inOrder(cohort);
880                 inOrder.verify(cohort).canCommit();
881                 inOrder.verify(cohort).preCommit();
882                 inOrder.verify(cohort).commit();
883
884                 // Use MBean for verification
885                 // Committed transaction count should increase as usual
886                 assertEquals(1,shard.underlyingActor().getShardMBean().getCommittedTransactionsCount());
887
888                 // Commit index should not advance because this does not go into the journal
889                 assertEquals(-1, shard.underlyingActor().getShardMBean().getCommitIndex());
890
891                 shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
892
893             }
894         };
895     }
896
897     @Test
898     public void testCommitWhenTransactionHasModifications(){
899         new ShardTestKit(getSystem()) {
900             {
901                 final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
902                         newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
903                         "testCommitWhenTransactionHasModifications");
904
905                 waitUntilLeader(shard);
906
907                 String transactionID = "tx1";
908                 MutableCompositeModification modification = new MutableCompositeModification();
909                 modification.addModification(new DeleteModification(YangInstanceIdentifier.builder().build()));
910                 DOMStoreThreePhaseCommitCohort cohort = mock(DOMStoreThreePhaseCommitCohort.class, "cohort1");
911                 doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit();
912                 doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).preCommit();
913                 doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).commit();
914
915                 FiniteDuration duration = duration("5 seconds");
916
917                 // Simulate the ForwardedReadyTransaction messages that would be sent
918                 // by the ShardTransaction.
919
920                 shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION,
921                         cohort, modification, true), getRef());
922                 expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
923
924                 // Send the CanCommitTransaction message.
925
926                 shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
927                 CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
928                         expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
929                 assertEquals("Can commit", true, canCommitReply.getCanCommit());
930
931                 shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef());
932                 expectMsgClass(duration, ThreePhaseCommitCohortMessages.CommitTransactionReply.class);
933
934                 InOrder inOrder = inOrder(cohort);
935                 inOrder.verify(cohort).canCommit();
936                 inOrder.verify(cohort).preCommit();
937                 inOrder.verify(cohort).commit();
938
939                 // Use MBean for verification
940                 // Committed transaction count should increase as usual
941                 assertEquals(1, shard.underlyingActor().getShardMBean().getCommittedTransactionsCount());
942
943                 // Commit index should advance as we do not have an empty modification
944                 assertEquals(0, shard.underlyingActor().getShardMBean().getCommitIndex());
945
946                 shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
947
948             }
949         };
950     }
951
952     @Test
953     public void testCommitPhaseFailure() throws Throwable {
954         new ShardTestKit(getSystem()) {{
955             final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
956                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
957                     "testCommitPhaseFailure");
958
959             waitUntilLeader(shard);
960
961             // Setup 2 mock cohorts. The first one fails in the commit phase.
962
963             final String transactionID1 = "tx1";
964             final DOMStoreThreePhaseCommitCohort cohort1 = mock(DOMStoreThreePhaseCommitCohort.class, "cohort1");
965             doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort1).canCommit();
966             doReturn(Futures.immediateFuture(null)).when(cohort1).preCommit();
967             doReturn(Futures.immediateFailedFuture(new IllegalStateException("mock"))).when(cohort1).commit();
968
969             final String transactionID2 = "tx2";
970             final DOMStoreThreePhaseCommitCohort cohort2 = mock(DOMStoreThreePhaseCommitCohort.class, "cohort2");
971             doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort2).canCommit();
972
973             ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() {
974                 @Override
975                 public DOMStoreThreePhaseCommitCohort decorate(String transactionID,
976                         DOMStoreThreePhaseCommitCohort actual) {
977                     return transactionID1.equals(transactionID) ? cohort1 : cohort2;
978                 }
979             };
980
981             shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator);
982
983             FiniteDuration duration = duration("5 seconds");
984             final Timeout timeout = new Timeout(duration);
985
986             // Send BatchedModifications to start and ready each transaction.
987
988             shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH,
989                     ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef());
990             expectMsgClass(duration, BatchedModificationsReply.class);
991
992             shard.tell(newBatchedModifications(transactionID2, TestModel.TEST_PATH,
993                     ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef());
994             expectMsgClass(duration, BatchedModificationsReply.class);
995
996             // Send the CanCommitTransaction message for the first Tx.
997
998             shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef());
999             CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
1000                     expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
1001             assertEquals("Can commit", true, canCommitReply.getCanCommit());
1002
1003             // Send the CanCommitTransaction message for the 2nd Tx. This should get queued and
1004             // processed after the first Tx completes.
1005
1006             Future<Object> canCommitFuture = Patterns.ask(shard,
1007                     new CanCommitTransaction(transactionID2).toSerializable(), timeout);
1008
1009             // Send the CommitTransaction message for the first Tx. This should send back an error
1010             // and trigger the 2nd Tx to proceed.
1011
1012             shard.tell(new CommitTransaction(transactionID1).toSerializable(), getRef());
1013             expectMsgClass(duration, akka.actor.Status.Failure.class);
1014
1015             // Wait for the 2nd Tx to complete the canCommit phase.
1016
1017             final CountDownLatch latch = new CountDownLatch(1);
1018             canCommitFuture.onComplete(new OnComplete<Object>() {
1019                 @Override
1020                 public void onComplete(final Throwable t, final Object resp) {
1021                     latch.countDown();
1022                 }
1023             }, getSystem().dispatcher());
1024
1025             assertEquals("2nd CanCommit complete", true, latch.await(5, TimeUnit.SECONDS));
1026
1027             InOrder inOrder = inOrder(cohort1, cohort2);
1028             inOrder.verify(cohort1).canCommit();
1029             inOrder.verify(cohort1).preCommit();
1030             inOrder.verify(cohort1).commit();
1031             inOrder.verify(cohort2).canCommit();
1032
1033             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
1034         }};
1035     }
1036
1037     @Test
1038     public void testPreCommitPhaseFailure() throws Throwable {
1039         new ShardTestKit(getSystem()) {{
1040             final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
1041                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1042                     "testPreCommitPhaseFailure");
1043
1044             waitUntilLeader(shard);
1045
1046             String transactionID = "tx1";
1047             final DOMStoreThreePhaseCommitCohort cohort = mock(DOMStoreThreePhaseCommitCohort.class, "cohort1");
1048             doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit();
1049             doReturn(Futures.immediateFailedFuture(new IllegalStateException("mock"))).when(cohort).preCommit();
1050
1051             ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() {
1052                 @Override
1053                 public DOMStoreThreePhaseCommitCohort decorate(String transactionID,
1054                         DOMStoreThreePhaseCommitCohort actual) {
1055                     return cohort;
1056                 }
1057             };
1058
1059             shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator);
1060
1061             FiniteDuration duration = duration("5 seconds");
1062
1063             // Send BatchedModifications to start and ready a transaction.
1064
1065             shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH,
1066                     ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef());
1067             expectMsgClass(duration, BatchedModificationsReply.class);
1068
1069             // Send the CanCommitTransaction message.
1070
1071             shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
1072             CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
1073                     expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
1074             assertEquals("Can commit", true, canCommitReply.getCanCommit());
1075
1076             // Send the CommitTransaction message. This should send back an error
1077             // for preCommit failure.
1078
1079             shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef());
1080             expectMsgClass(duration, akka.actor.Status.Failure.class);
1081
1082             InOrder inOrder = inOrder(cohort);
1083             inOrder.verify(cohort).canCommit();
1084             inOrder.verify(cohort).preCommit();
1085
1086             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
1087         }};
1088     }
1089
1090     @Test
1091     public void testCanCommitPhaseFailure() throws Throwable {
1092         new ShardTestKit(getSystem()) {{
1093             final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
1094                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1095                     "testCanCommitPhaseFailure");
1096
1097             waitUntilLeader(shard);
1098
1099             final FiniteDuration duration = duration("5 seconds");
1100
1101             String transactionID = "tx1";
1102             final DOMStoreThreePhaseCommitCohort cohort = mock(DOMStoreThreePhaseCommitCohort.class, "cohort1");
1103             doReturn(Futures.immediateFailedFuture(new IllegalStateException("mock"))).when(cohort).canCommit();
1104
1105             ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() {
1106                 @Override
1107                 public DOMStoreThreePhaseCommitCohort decorate(String transactionID,
1108                         DOMStoreThreePhaseCommitCohort actual) {
1109                     return cohort;
1110                 }
1111             };
1112
1113             shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator);
1114
1115             // Send BatchedModifications to start and ready a transaction.
1116
1117             shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH,
1118                     ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef());
1119             expectMsgClass(duration, BatchedModificationsReply.class);
1120
1121             // Send the CanCommitTransaction message.
1122
1123             shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
1124             expectMsgClass(duration, akka.actor.Status.Failure.class);
1125
1126             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
1127         }};
1128     }
1129
1130     @Test
1131     public void testAbortBeforeFinishCommit() throws Throwable {
1132         new ShardTestKit(getSystem()) {{
1133             final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
1134                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1135                     "testAbortBeforeFinishCommit");
1136
1137             waitUntilLeader(shard);
1138
1139             final FiniteDuration duration = duration("5 seconds");
1140             InMemoryDOMDataStore dataStore = shard.underlyingActor().getDataStore();
1141
1142             final String transactionID = "tx1";
1143             Function<DOMStoreThreePhaseCommitCohort,ListenableFuture<Void>> preCommit =
1144                           new Function<DOMStoreThreePhaseCommitCohort,ListenableFuture<Void>>() {
1145                 @Override
1146                 public ListenableFuture<Void> apply(final DOMStoreThreePhaseCommitCohort cohort) {
1147                     ListenableFuture<Void> preCommitFuture = cohort.preCommit();
1148
1149                     // Simulate an AbortTransaction message occurring during replication, after
1150                     // persisting and before finishing the commit to the in-memory store.
1151                     // We have no followers so due to optimizations in the RaftActor, it does not
1152                     // attempt replication and thus we can't send an AbortTransaction message b/c
1153                     // it would be processed too late after CommitTransaction completes. So we'll
1154                     // simulate an AbortTransaction message occurring during replication by calling
1155                     // the shard directly.
1156                     //
1157                     shard.underlyingActor().doAbortTransaction(transactionID, null);
1158
1159                     return preCommitFuture;
1160                 }
1161             };
1162
1163             shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH,
1164                     ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef());
1165             expectMsgClass(duration, BatchedModificationsReply.class);
1166
1167             shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
1168             CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
1169                     expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
1170             assertEquals("Can commit", true, canCommitReply.getCanCommit());
1171
1172             shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef());
1173             expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
1174
1175             NormalizedNode<?, ?> node = readStore(shard, TestModel.TEST_PATH);
1176
1177             // Since we're simulating an abort occurring during replication and before finish commit,
1178             // the data should still get written to the in-memory store since we've gotten past
1179             // canCommit and preCommit and persisted the data.
1180             assertNotNull(TestModel.TEST_QNAME.getLocalName() + " not found", node);
1181
1182             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
1183         }};
1184     }
1185
1186     @Test
1187     public void testTransactionCommitTimeout() throws Throwable {
1188         dataStoreContextBuilder.shardTransactionCommitTimeoutInSeconds(1);
1189
1190         new ShardTestKit(getSystem()) {{
1191             final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
1192                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1193                     "testTransactionCommitTimeout");
1194
1195             waitUntilLeader(shard);
1196
1197             final FiniteDuration duration = duration("5 seconds");
1198
1199             writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
1200             writeToStore(shard, TestModel.OUTER_LIST_PATH,
1201                     ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build());
1202
1203             // Create and ready the 1st Tx - will timeout
1204
1205             String transactionID1 = "tx1";
1206             shard.tell(newBatchedModifications(transactionID1, YangInstanceIdentifier.builder(
1207                     TestModel.OUTER_LIST_PATH).nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(),
1208                 ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1), true), getRef());
1209             expectMsgClass(duration, BatchedModificationsReply.class);
1210
1211             // Create and ready the 2nd Tx
1212
1213             String transactionID2 = "tx2";
1214             YangInstanceIdentifier listNodePath = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
1215                     .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2).build();
1216             shard.tell(newBatchedModifications(transactionID2, listNodePath,
1217                     ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2), true), getRef());
1218             expectMsgClass(duration, BatchedModificationsReply.class);
1219
1220             // canCommit 1st Tx. We don't send the commit so it should timeout.
1221
1222             shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef());
1223             expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS);
1224
1225             // canCommit the 2nd Tx - it should complete after the 1st Tx times out.
1226
1227             shard.tell(new CanCommitTransaction(transactionID2).toSerializable(), getRef());
1228             expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS);
1229
1230             // Commit the 2nd Tx.
1231
1232             shard.tell(new CommitTransaction(transactionID2).toSerializable(), getRef());
1233             expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
1234
1235             NormalizedNode<?, ?> node = readStore(shard, listNodePath);
1236             assertNotNull(listNodePath + " not found", node);
1237
1238             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
1239         }};
1240     }
1241
1242     @Test
1243     public void testTransactionCommitQueueCapacityExceeded() throws Throwable {
1244         dataStoreContextBuilder.shardTransactionCommitQueueCapacity(1);
1245
1246         new ShardTestKit(getSystem()) {{
1247             final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
1248                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1249                     "testTransactionCommitQueueCapacityExceeded");
1250
1251             waitUntilLeader(shard);
1252
1253             final FiniteDuration duration = duration("5 seconds");
1254
1255             String transactionID1 = "tx1";
1256             String transactionID2 = "tx2";
1257             String transactionID3 = "tx3";
1258
1259             // Send a BatchedModifications to start transactions and ready them.
1260
1261             shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH,
1262                     ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef());
1263             expectMsgClass(duration, BatchedModificationsReply.class);
1264
1265             shard.tell(newBatchedModifications(transactionID2,TestModel.OUTER_LIST_PATH,
1266                     ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), true), getRef());
1267             expectMsgClass(duration, BatchedModificationsReply.class);
1268
1269             shard.tell(newBatchedModifications(transactionID3, TestModel.TEST_PATH,
1270                     ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef());
1271             expectMsgClass(duration, BatchedModificationsReply.class);
1272
1273             // canCommit 1st Tx.
1274
1275             shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef());
1276             expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS);
1277
1278             // canCommit the 2nd Tx - it should get queued.
1279
1280             shard.tell(new CanCommitTransaction(transactionID2).toSerializable(), getRef());
1281
1282             // canCommit the 3rd Tx - should exceed queue capacity and fail.
1283
1284             shard.tell(new CanCommitTransaction(transactionID3).toSerializable(), getRef());
1285             expectMsgClass(duration, akka.actor.Status.Failure.class);
1286
1287             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
1288         }};
1289     }
1290
1291     @Test
1292     public void testCanCommitBeforeReadyFailure() throws Throwable {
1293         new ShardTestKit(getSystem()) {{
1294             final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
1295                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1296                     "testCanCommitBeforeReadyFailure");
1297
1298             shard.tell(new CanCommitTransaction("tx").toSerializable(), getRef());
1299             expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
1300
1301             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
1302         }};
1303     }
1304
1305     @Test
1306     public void testAbortTransaction() throws Throwable {
1307         new ShardTestKit(getSystem()) {{
1308             final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
1309                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1310                     "testAbortTransaction");
1311
1312             waitUntilLeader(shard);
1313
1314             // Setup 2 simulated transactions with mock cohorts. The first one will be aborted.
1315
1316             final String transactionID1 = "tx1";
1317             final DOMStoreThreePhaseCommitCohort cohort1 = mock(DOMStoreThreePhaseCommitCohort.class, "cohort1");
1318             doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort1).canCommit();
1319             doReturn(Futures.immediateFuture(null)).when(cohort1).abort();
1320
1321             final String transactionID2 = "tx2";
1322             final DOMStoreThreePhaseCommitCohort cohort2 = mock(DOMStoreThreePhaseCommitCohort.class, "cohort2");
1323             doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort2).canCommit();
1324
1325             FiniteDuration duration = duration("5 seconds");
1326             final Timeout timeout = new Timeout(duration);
1327
1328             ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() {
1329                 @Override
1330                 public DOMStoreThreePhaseCommitCohort decorate(String transactionID,
1331                         DOMStoreThreePhaseCommitCohort actual) {
1332                     return transactionID1.equals(transactionID) ? cohort1 : cohort2;
1333                 }
1334             };
1335
1336             shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator);
1337
1338             // Send BatchedModifications to start and ready each transaction.
1339
1340             shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH,
1341                     ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef());
1342             expectMsgClass(duration, BatchedModificationsReply.class);
1343
1344             shard.tell(newBatchedModifications(transactionID2, TestModel.TEST_PATH,
1345                     ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef());
1346             expectMsgClass(duration, BatchedModificationsReply.class);
1347
1348             // Send the CanCommitTransaction message for the first Tx.
1349
1350             shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef());
1351             CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
1352                     expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
1353             assertEquals("Can commit", true, canCommitReply.getCanCommit());
1354
1355             // Send the CanCommitTransaction message for the 2nd Tx. This should get queued and
1356             // processed after the first Tx completes.
1357
1358             Future<Object> canCommitFuture = Patterns.ask(shard,
1359                     new CanCommitTransaction(transactionID2).toSerializable(), timeout);
1360
1361             // Send the AbortTransaction message for the first Tx. This should trigger the 2nd
1362             // Tx to proceed.
1363
1364             shard.tell(new AbortTransaction(transactionID1).toSerializable(), getRef());
1365             expectMsgClass(duration, AbortTransactionReply.SERIALIZABLE_CLASS);
1366
1367             // Wait for the 2nd Tx to complete the canCommit phase.
1368
1369             Await.ready(canCommitFuture, duration);
1370
1371             InOrder inOrder = inOrder(cohort1, cohort2);
1372             inOrder.verify(cohort1).canCommit();
1373             inOrder.verify(cohort2).canCommit();
1374
1375             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
1376         }};
1377     }
1378
1379     @Test
1380     public void testCreateSnapshot() throws Exception {
1381         testCreateSnapshot(true, "testCreateSnapshot");
1382     }
1383
1384     @Test
1385     public void testCreateSnapshotWithNonPersistentData() throws Exception {
1386         testCreateSnapshot(false, "testCreateSnapshotWithNonPersistentData");
1387     }
1388
1389     @SuppressWarnings("serial")
1390     public void testCreateSnapshot(final boolean persistent, final String shardActorName) throws Exception{
1391
1392         final AtomicReference<Object> savedSnapshot = new AtomicReference<>();
1393         class TestPersistentDataProvider extends DelegatingPersistentDataProvider {
1394             TestPersistentDataProvider(DataPersistenceProvider delegate) {
1395                 super(delegate);
1396             }
1397
1398             @Override
1399             public void saveSnapshot(Object o) {
1400                 savedSnapshot.set(o);
1401                 super.saveSnapshot(o);
1402             }
1403         }
1404
1405         dataStoreContextBuilder.persistent(persistent);
1406
1407         new ShardTestKit(getSystem()) {{
1408             final AtomicReference<CountDownLatch> latch = new AtomicReference<>(new CountDownLatch(1));
1409
1410             class TestShard extends Shard {
1411
1412                 protected TestShard(ShardIdentifier name, Map<String, String> peerAddresses,
1413                                     DatastoreContext datastoreContext, SchemaContext schemaContext) {
1414                     super(name, peerAddresses, datastoreContext, schemaContext);
1415                     setPersistence(new TestPersistentDataProvider(super.persistence()));
1416                 }
1417
1418                 @Override
1419                 protected void commitSnapshot(final long sequenceNumber) {
1420                     super.commitSnapshot(sequenceNumber);
1421                     latch.get().countDown();
1422                 }
1423
1424                 @Override
1425                 public RaftActorContext getRaftActorContext() {
1426                     return super.getRaftActorContext();
1427                 }
1428             }
1429
1430             Creator<Shard> creator = new Creator<Shard>() {
1431                 @Override
1432                 public Shard create() throws Exception {
1433                     return new TestShard(shardID, Collections.<String,String>emptyMap(),
1434                             newDatastoreContext(), SCHEMA_CONTEXT);
1435                 }
1436             };
1437
1438             TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
1439                     Props.create(new DelegatingShardCreator(creator)), shardActorName);
1440
1441             waitUntilLeader(shard);
1442
1443             writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
1444
1445             NormalizedNode<?,?> expectedRoot = readStore(shard, YangInstanceIdentifier.builder().build());
1446
1447             // Trigger creation of a snapshot by ensuring
1448             RaftActorContext raftActorContext = ((TestShard) shard.underlyingActor()).getRaftActorContext();
1449             raftActorContext.getSnapshotManager().capture(mock(ReplicatedLogEntry.class), -1);
1450
1451             assertEquals("Snapshot saved", true, latch.get().await(5, TimeUnit.SECONDS));
1452
1453             assertTrue("Invalid saved snapshot " + savedSnapshot.get(),
1454                     savedSnapshot.get() instanceof Snapshot);
1455
1456             verifySnapshot((Snapshot)savedSnapshot.get(), expectedRoot);
1457
1458             latch.set(new CountDownLatch(1));
1459             savedSnapshot.set(null);
1460
1461             raftActorContext.getSnapshotManager().capture(mock(ReplicatedLogEntry.class), -1);
1462
1463             assertEquals("Snapshot saved", true, latch.get().await(5, TimeUnit.SECONDS));
1464
1465             assertTrue("Invalid saved snapshot " + savedSnapshot.get(),
1466                     savedSnapshot.get() instanceof Snapshot);
1467
1468             verifySnapshot((Snapshot)savedSnapshot.get(), expectedRoot);
1469
1470             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
1471         }
1472
1473         private void verifySnapshot(Snapshot snapshot, NormalizedNode<?,?> expectedRoot) {
1474
1475             NormalizedNode<?, ?> actual = SerializationUtils.deserializeNormalizedNode(snapshot.getState());
1476             assertEquals("Root node", expectedRoot, actual);
1477
1478         }};
1479     }
1480
1481     /**
1482      * This test simply verifies that the applySnapShot logic will work
1483      * @throws ReadFailedException
1484      */
1485     @Test
1486     public void testInMemoryDataStoreRestore() throws ReadFailedException {
1487         InMemoryDOMDataStore store = new InMemoryDOMDataStore("test", MoreExecutors.sameThreadExecutor());
1488
1489         store.onGlobalContextUpdated(SCHEMA_CONTEXT);
1490
1491         DOMStoreWriteTransaction putTransaction = store.newWriteOnlyTransaction();
1492         putTransaction.write(TestModel.TEST_PATH,
1493             ImmutableNodes.containerNode(TestModel.TEST_QNAME));
1494         commitTransaction(putTransaction);
1495
1496
1497         NormalizedNode<?, ?> expected = readStore(store);
1498
1499         DOMStoreWriteTransaction writeTransaction = store.newWriteOnlyTransaction();
1500
1501         writeTransaction.delete(YangInstanceIdentifier.builder().build());
1502         writeTransaction.write(YangInstanceIdentifier.builder().build(), expected);
1503
1504         commitTransaction(writeTransaction);
1505
1506         NormalizedNode<?, ?> actual = readStore(store);
1507
1508         assertEquals(expected, actual);
1509     }
1510
1511     @Test
1512     public void testRecoveryApplicable(){
1513
1514         final DatastoreContext persistentContext = DatastoreContext.newBuilder().
1515                 shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).persistent(true).build();
1516
1517         final Props persistentProps = Shard.props(shardID, Collections.<String, String>emptyMap(),
1518                 persistentContext, SCHEMA_CONTEXT);
1519
1520         final DatastoreContext nonPersistentContext = DatastoreContext.newBuilder().
1521                 shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).persistent(false).build();
1522
1523         final Props nonPersistentProps = Shard.props(shardID, Collections.<String, String>emptyMap(),
1524                 nonPersistentContext, SCHEMA_CONTEXT);
1525
1526         new ShardTestKit(getSystem()) {{
1527             TestActorRef<Shard> shard1 = TestActorRef.create(getSystem(),
1528                     persistentProps, "testPersistence1");
1529
1530             assertTrue("Recovery Applicable", shard1.underlyingActor().persistence().isRecoveryApplicable());
1531
1532             shard1.tell(PoisonPill.getInstance(), ActorRef.noSender());
1533
1534             TestActorRef<Shard> shard2 = TestActorRef.create(getSystem(),
1535                     nonPersistentProps, "testPersistence2");
1536
1537             assertFalse("Recovery Not Applicable", shard2.underlyingActor().persistence().isRecoveryApplicable());
1538
1539             shard2.tell(PoisonPill.getInstance(), ActorRef.noSender());
1540
1541         }};
1542
1543     }
1544
1545     @Test
1546     public void testOnDatastoreContext() {
1547         new ShardTestKit(getSystem()) {{
1548             dataStoreContextBuilder.persistent(true);
1549
1550             TestActorRef<Shard> shard = TestActorRef.create(getSystem(), newShardProps(), "testOnDatastoreContext");
1551
1552             assertEquals("isRecoveryApplicable", true,
1553                     shard.underlyingActor().persistence().isRecoveryApplicable());
1554
1555             waitUntilLeader(shard);
1556
1557             shard.tell(dataStoreContextBuilder.persistent(false).build(), ActorRef.noSender());
1558
1559             assertEquals("isRecoveryApplicable", false,
1560                     shard.underlyingActor().persistence().isRecoveryApplicable());
1561
1562             shard.tell(dataStoreContextBuilder.persistent(true).build(), ActorRef.noSender());
1563
1564             assertEquals("isRecoveryApplicable", true,
1565                     shard.underlyingActor().persistence().isRecoveryApplicable());
1566
1567             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
1568         }};
1569     }
1570
1571     @Test
1572     public void testRegisterRoleChangeListener() throws Exception {
1573         new ShardTestKit(getSystem()) {
1574             {
1575                 final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
1576                         newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1577                         "testRegisterRoleChangeListener");
1578
1579                 waitUntilLeader(shard);
1580
1581                 TestActorRef<MessageCollectorActor> listener =
1582                         TestActorRef.create(getSystem(), Props.create(MessageCollectorActor.class));
1583
1584                 shard.tell(new RegisterRoleChangeListener(), listener);
1585
1586                 // TODO: MessageCollectorActor exists as a test util in both the akka-raft and distributed-datastore
1587                 // projects. Need to move it to commons as a regular utility and then we can get rid of this arbitrary
1588                 // sleep.
1589                 Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
1590
1591                 List<Object> allMatching = MessageCollectorActor.getAllMatching(listener, RegisterRoleChangeListenerReply.class);
1592
1593                 assertEquals(1, allMatching.size());
1594             }
1595         };
1596     }
1597
1598     @Test
1599     public void testFollowerInitialSyncStatus() throws Exception {
1600         final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
1601                 newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
1602                 "testFollowerInitialSyncStatus");
1603
1604         shard.underlyingActor().onReceiveCommand(new FollowerInitialSyncUpStatus(false, "member-1-shard-inventory-operational"));
1605
1606         assertEquals(false, shard.underlyingActor().getShardMBean().getFollowerInitialSyncStatus());
1607
1608         shard.underlyingActor().onReceiveCommand(new FollowerInitialSyncUpStatus(true, "member-1-shard-inventory-operational"));
1609
1610         assertEquals(true, shard.underlyingActor().getShardMBean().getFollowerInitialSyncStatus());
1611
1612         shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
1613     }
1614
1615     private void commitTransaction(final DOMStoreWriteTransaction transaction) {
1616         DOMStoreThreePhaseCommitCohort commitCohort = transaction.ready();
1617         ListenableFuture<Void> future =
1618             commitCohort.preCommit();
1619         try {
1620             future.get();
1621             future = commitCohort.commit();
1622             future.get();
1623         } catch (InterruptedException | ExecutionException e) {
1624         }
1625     }
1626 }