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