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