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