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