Add unit test for FrontedMetadata memory leaks
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / DistributedDataStoreRemotingIntegrationTest.java
1 /*
2  * Copyright (c) 2015, 2017 Brocade Communications Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.controller.cluster.datastore;
9
10 import static org.awaitility.Awaitility.await;
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.assertTrue;
15 import static org.junit.Assert.fail;
16 import static org.mockito.ArgumentMatchers.any;
17 import static org.mockito.ArgumentMatchers.eq;
18 import static org.mockito.Mockito.timeout;
19 import static org.mockito.Mockito.verify;
20
21 import akka.actor.ActorRef;
22 import akka.actor.ActorSelection;
23 import akka.actor.ActorSystem;
24 import akka.actor.Address;
25 import akka.actor.AddressFromURIString;
26 import akka.cluster.Cluster;
27 import akka.dispatch.Futures;
28 import akka.pattern.Patterns;
29 import akka.testkit.javadsl.TestKit;
30 import com.google.common.base.Stopwatch;
31 import com.google.common.base.Supplier;
32 import com.google.common.base.Throwables;
33 import com.google.common.collect.ImmutableMap;
34 import com.google.common.collect.Range;
35 import com.google.common.primitives.UnsignedLong;
36 import com.google.common.util.concurrent.ListenableFuture;
37 import com.google.common.util.concurrent.MoreExecutors;
38 import com.google.common.util.concurrent.Uninterruptibles;
39 import com.typesafe.config.ConfigFactory;
40 import java.math.BigInteger;
41 import java.util.Arrays;
42 import java.util.Collection;
43 import java.util.Collections;
44 import java.util.Iterator;
45 import java.util.LinkedList;
46 import java.util.List;
47 import java.util.Optional;
48 import java.util.Set;
49 import java.util.concurrent.ExecutionException;
50 import java.util.concurrent.TimeUnit;
51 import java.util.concurrent.atomic.AtomicLong;
52 import org.junit.After;
53 import org.junit.Assume;
54 import org.junit.Before;
55 import org.junit.Ignore;
56 import org.junit.Test;
57 import org.junit.runner.RunWith;
58 import org.junit.runners.Parameterized;
59 import org.junit.runners.Parameterized.Parameter;
60 import org.junit.runners.Parameterized.Parameters;
61 import org.mockito.Mockito;
62 import org.mockito.stubbing.Answer;
63 import org.opendaylight.controller.cluster.access.client.RequestTimeoutException;
64 import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
65 import org.opendaylight.controller.cluster.databroker.ClientBackedDataStore;
66 import org.opendaylight.controller.cluster.databroker.ConcurrentDOMDataBroker;
67 import org.opendaylight.controller.cluster.databroker.TestClientBackedDataStore;
68 import org.opendaylight.controller.cluster.datastore.DatastoreContext.Builder;
69 import org.opendaylight.controller.cluster.datastore.TestShard.RequestFrontendMetadata;
70 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
71 import org.opendaylight.controller.cluster.datastore.exceptions.ShardLeaderNotRespondingException;
72 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
73 import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction;
74 import org.opendaylight.controller.cluster.datastore.messages.GetShardDataTree;
75 import org.opendaylight.controller.cluster.datastore.messages.ReadyLocalTransaction;
76 import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
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.FrontendHistoryMetadata;
80 import org.opendaylight.controller.cluster.datastore.persisted.FrontendShardDataTreeSnapshotMetadata;
81 import org.opendaylight.controller.cluster.datastore.persisted.MetadataShardDataTreeSnapshot;
82 import org.opendaylight.controller.cluster.datastore.persisted.ShardSnapshotState;
83 import org.opendaylight.controller.cluster.raft.base.messages.TimeoutNow;
84 import org.opendaylight.controller.cluster.raft.client.messages.Shutdown;
85 import org.opendaylight.controller.cluster.raft.persisted.ApplyJournalEntries;
86 import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
87 import org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy;
88 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
89 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
90 import org.opendaylight.controller.md.cluster.datastore.model.CarsModel;
91 import org.opendaylight.controller.md.cluster.datastore.model.PeopleModel;
92 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
93 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
94 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
95 import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteTransaction;
96 import org.opendaylight.mdsal.dom.api.DOMTransactionChain;
97 import org.opendaylight.mdsal.dom.api.DOMTransactionChainListener;
98 import org.opendaylight.mdsal.dom.spi.store.DOMStore;
99 import org.opendaylight.mdsal.dom.spi.store.DOMStoreReadTransaction;
100 import org.opendaylight.mdsal.dom.spi.store.DOMStoreReadWriteTransaction;
101 import org.opendaylight.mdsal.dom.spi.store.DOMStoreThreePhaseCommitCohort;
102 import org.opendaylight.mdsal.dom.spi.store.DOMStoreTransactionChain;
103 import org.opendaylight.mdsal.dom.spi.store.DOMStoreWriteTransaction;
104 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
105 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
106 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
107 import org.opendaylight.yangtools.yang.data.api.schema.MapNode;
108 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
109 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
110 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeConfiguration;
111 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification;
112 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
113 import org.opendaylight.yangtools.yang.data.impl.schema.builder.api.CollectionNodeBuilder;
114 import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder;
115 import org.opendaylight.yangtools.yang.data.impl.schema.tree.InMemoryDataTreeFactory;
116 import scala.concurrent.Await;
117 import scala.concurrent.Future;
118 import scala.concurrent.duration.FiniteDuration;
119
120 /**
121  * End-to-end distributed data store tests that exercise remote shards and transactions.
122  *
123  * @author Thomas Pantelis
124  */
125 @RunWith(Parameterized.class)
126 public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest {
127
128     @Parameters(name = "{0}")
129     public static Collection<Object[]> data() {
130         return Arrays.asList(new Object[][] {
131                 { TestDistributedDataStore.class, 7}, { TestClientBackedDataStore.class, 12 }
132         });
133     }
134
135     @Parameter(0)
136     public Class<? extends AbstractDataStore> testParameter;
137     @Parameter(1)
138     public int commitTimeout;
139
140     private static final String[] CARS_AND_PEOPLE = {"cars", "people"};
141     private static final String[] CARS = {"cars"};
142
143     private static final Address MEMBER_1_ADDRESS = AddressFromURIString.parse(
144             "akka://cluster-test@127.0.0.1:2558");
145     private static final Address MEMBER_2_ADDRESS = AddressFromURIString.parse(
146             "akka://cluster-test@127.0.0.1:2559");
147
148     private static final String MODULE_SHARDS_CARS_ONLY_1_2 = "module-shards-cars-member-1-and-2.conf";
149     private static final String MODULE_SHARDS_CARS_PEOPLE_1_2 = "module-shards-member1-and-2.conf";
150     private static final String MODULE_SHARDS_CARS_PEOPLE_1_2_3 = "module-shards-member1-and-2-and-3.conf";
151     private static final String MODULE_SHARDS_CARS_1_2_3 = "module-shards-cars-member-1-and-2-and-3.conf";
152
153     private ActorSystem leaderSystem;
154     private ActorSystem followerSystem;
155     private ActorSystem follower2System;
156
157     private final DatastoreContext.Builder leaderDatastoreContextBuilder =
158             DatastoreContext.newBuilder().shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(2);
159
160     private final DatastoreContext.Builder followerDatastoreContextBuilder =
161             DatastoreContext.newBuilder().shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5)
162                 .customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName());
163     private final TransactionIdentifier tx1 = nextTransactionId();
164     private final TransactionIdentifier tx2 = nextTransactionId();
165
166     private AbstractDataStore followerDistributedDataStore;
167     private AbstractDataStore leaderDistributedDataStore;
168     private IntegrationTestKit followerTestKit;
169     private IntegrationTestKit leaderTestKit;
170
171     @Before
172     public void setUp() {
173         InMemoryJournal.clear();
174         InMemorySnapshotStore.clear();
175
176         leaderSystem = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member1"));
177         Cluster.get(leaderSystem).join(MEMBER_1_ADDRESS);
178
179         followerSystem = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member2"));
180         Cluster.get(followerSystem).join(MEMBER_1_ADDRESS);
181
182         follower2System = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member3"));
183         Cluster.get(follower2System).join(MEMBER_1_ADDRESS);
184     }
185
186     @After
187     public void tearDown() {
188         if (followerDistributedDataStore != null) {
189             leaderDistributedDataStore.close();
190         }
191         if (leaderDistributedDataStore != null) {
192             leaderDistributedDataStore.close();
193         }
194
195         TestKit.shutdownActorSystem(leaderSystem);
196         TestKit.shutdownActorSystem(followerSystem);
197         TestKit.shutdownActorSystem(follower2System);
198
199         InMemoryJournal.clear();
200         InMemorySnapshotStore.clear();
201     }
202
203     private void initDatastoresWithCars(final String type) throws Exception {
204         initDatastores(type, MODULE_SHARDS_CARS_ONLY_1_2, CARS);
205     }
206
207     private void initDatastoresWithCarsAndPeople(final String type) throws Exception {
208         initDatastores(type, MODULE_SHARDS_CARS_PEOPLE_1_2, CARS_AND_PEOPLE);
209     }
210
211     private void initDatastores(final String type, final String moduleShardsConfig, final String[] shards)
212             throws Exception {
213         leaderTestKit = new IntegrationTestKit(leaderSystem, leaderDatastoreContextBuilder, commitTimeout);
214
215         leaderDistributedDataStore = leaderTestKit.setupAbstractDataStore(
216                 testParameter, type, moduleShardsConfig, false, shards);
217
218         followerTestKit = new IntegrationTestKit(followerSystem, followerDatastoreContextBuilder, commitTimeout);
219         followerDistributedDataStore = followerTestKit.setupAbstractDataStore(
220                 testParameter, type, moduleShardsConfig, false, shards);
221
222         leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorUtils(), shards);
223
224         leaderTestKit.waitForMembersUp("member-2");
225         followerTestKit.waitForMembersUp("member-1");
226     }
227
228     private static void verifyCars(final DOMStoreReadTransaction readTx, final MapEntryNode... entries)
229             throws Exception {
230         final Optional<NormalizedNode<?, ?>> optional = readTx.read(CarsModel.CAR_LIST_PATH).get(5, TimeUnit.SECONDS);
231         assertTrue("isPresent", optional.isPresent());
232
233         final CollectionNodeBuilder<MapEntryNode, MapNode> listBuilder = ImmutableNodes.mapNodeBuilder(
234                 CarsModel.CAR_QNAME);
235         for (final NormalizedNode<?, ?> entry: entries) {
236             listBuilder.withChild((MapEntryNode) entry);
237         }
238
239         assertEquals("Car list node", listBuilder.build(), optional.get());
240     }
241
242     private static void verifyNode(final DOMStoreReadTransaction readTx, final YangInstanceIdentifier path,
243             final NormalizedNode<?, ?> expNode) throws Exception {
244         final Optional<NormalizedNode<?, ?>> optional = readTx.read(path).get(5, TimeUnit.SECONDS);
245         assertTrue("isPresent", optional.isPresent());
246         assertEquals("Data node", expNode, optional.get());
247     }
248
249     private static void verifyExists(final DOMStoreReadTransaction readTx, final YangInstanceIdentifier path)
250             throws Exception {
251         final Boolean exists = readTx.exists(path).get(5, TimeUnit.SECONDS);
252         assertEquals("exists", Boolean.TRUE, exists);
253     }
254
255     @Test
256     public void testWriteTransactionWithSingleShard() throws Exception {
257         final String testName = "testWriteTransactionWithSingleShard";
258         initDatastoresWithCars(testName);
259
260         final String followerCarShardName = "member-2-shard-cars-" + testName;
261
262         DOMStoreWriteTransaction writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
263         assertNotNull("newWriteOnlyTransaction returned null", writeTx);
264
265         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
266         writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
267
268         final MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
269         final YangInstanceIdentifier car1Path = CarsModel.newCarPath("optima");
270         writeTx.merge(car1Path, car1);
271
272         final MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(25000));
273         final YangInstanceIdentifier car2Path = CarsModel.newCarPath("sportage");
274         writeTx.merge(car2Path, car2);
275
276         followerTestKit.doCommit(writeTx.ready());
277
278         verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car1, car2);
279
280         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1, car2);
281
282         // Test delete
283
284         writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
285
286         writeTx.delete(car1Path);
287
288         followerTestKit.doCommit(writeTx.ready());
289
290         verifyExists(followerDistributedDataStore.newReadOnlyTransaction(), car2Path);
291
292         verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car2);
293
294         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car2);
295
296         // Re-instate the follower member 2 as a single-node to verify replication and recovery.
297
298         // The following is a bit tricky. Before we reinstate the follower we need to ensure it has persisted and
299         // applied and all the log entries from the leader. Since we've verified the car data above we know that
300         // all the transactions have been applied on the leader so we first read and capture its lastAppliedIndex.
301         final AtomicLong leaderLastAppliedIndex = new AtomicLong();
302         IntegrationTestKit.verifyShardState(leaderDistributedDataStore, CARS[0],
303             state -> leaderLastAppliedIndex.set(state.getLastApplied()));
304
305         // Now we need to make sure the follower has persisted the leader's lastAppliedIndex via ApplyJournalEntries.
306         // However we don't know exactly how many ApplyJournalEntries messages there will be as it can differ between
307         // the tell-based and ask-based front-ends. For ask-based there will be exactly 2 ApplyJournalEntries but
308         // tell-based persists additional payloads which could be replicated and applied in a batch resulting in
309         // either 2 or 3 ApplyJournalEntries. To handle this we read the follower's persisted ApplyJournalEntries
310         // until we find the one that encompasses the leader's lastAppliedIndex.
311         Stopwatch sw = Stopwatch.createStarted();
312         boolean done = false;
313         while (!done) {
314             final List<ApplyJournalEntries> entries = InMemoryJournal.get(followerCarShardName,
315                     ApplyJournalEntries.class);
316             for (ApplyJournalEntries aje: entries) {
317                 if (aje.getToIndex() >= leaderLastAppliedIndex.get()) {
318                     done = true;
319                     break;
320                 }
321             }
322
323             assertTrue("Follower did not persist ApplyJournalEntries containing leader's lastAppliedIndex "
324                     + leaderLastAppliedIndex + ". Entries persisted: " + entries, sw.elapsed(TimeUnit.SECONDS) <= 5);
325
326             Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
327         }
328
329         TestKit.shutdownActorSystem(leaderSystem, true);
330         TestKit.shutdownActorSystem(followerSystem, true);
331
332         final ActorSystem newSystem = newActorSystem("reinstated-member2", "Member2");
333
334         try (AbstractDataStore member2Datastore = new IntegrationTestKit(newSystem, leaderDatastoreContextBuilder,
335                 commitTimeout)
336                 .setupAbstractDataStore(testParameter, testName, "module-shards-member2", true, CARS)) {
337             verifyCars(member2Datastore.newReadOnlyTransaction(), car2);
338         }
339     }
340
341     @Test
342     public void testSingleTransactionsWritesInQuickSuccession() throws Exception {
343         final String testName = "testWriteTransactionWithSingleShard";
344         initDatastoresWithCars(testName);
345
346         final DOMStoreTransactionChain txChain = followerDistributedDataStore.createTransactionChain();
347
348         DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
349         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
350         writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
351         followerTestKit.doCommit(writeTx.ready());
352
353         int numCars = 5;
354         for (int i = 0; i < numCars; i++) {
355             writeTx = txChain.newWriteOnlyTransaction();
356             writeTx.write(CarsModel.newCarPath("car" + i),
357                     CarsModel.newCarEntry("car" + i, BigInteger.valueOf(20000)));
358
359             followerTestKit.doCommit(writeTx.ready());
360
361             DOMStoreReadTransaction domStoreReadTransaction = txChain.newReadOnlyTransaction();
362             domStoreReadTransaction.read(CarsModel.BASE_PATH).get();
363
364             domStoreReadTransaction.close();
365         }
366
367         // wait to let the shard catch up with purged
368         await("Range set leak test").atMost(5, TimeUnit.SECONDS)
369                 .pollInterval(500, TimeUnit.MILLISECONDS)
370                 .untilAsserted(() -> {
371                     Optional<ActorRef> localShard =
372                             leaderDistributedDataStore.getActorUtils().findLocalShard("cars");
373                     FrontendShardDataTreeSnapshotMetadata frontendMetadata =
374                             (FrontendShardDataTreeSnapshotMetadata) leaderDistributedDataStore.getActorUtils()
375                                     .executeOperation(localShard.get(), new RequestFrontendMetadata());
376
377                     if (leaderDistributedDataStore.getActorUtils().getDatastoreContext().isUseTellBasedProtocol()) {
378                         Iterator<FrontendHistoryMetadata> iterator =
379                                 frontendMetadata.getClients().get(0).getCurrentHistories().iterator();
380                         FrontendHistoryMetadata metadata = iterator.next();
381                         while (iterator.hasNext() && metadata.getHistoryId() != 1) {
382                             metadata = iterator.next();
383                         }
384
385                         assertEquals(0, metadata.getClosedTransactions().size());
386                         assertEquals(Range.closedOpen(UnsignedLong.valueOf(0), UnsignedLong.valueOf(11)),
387                                 metadata.getPurgedTransactions().asRanges().iterator().next());
388                     } else {
389                         // ask based should track no metadata
390                         assertTrue(frontendMetadata.getClients().get(0).getCurrentHistories().isEmpty());
391                     }
392                 });
393
394         final Optional<NormalizedNode<?, ?>> optional = txChain.newReadOnlyTransaction()
395                 .read(CarsModel.CAR_LIST_PATH).get(5, TimeUnit.SECONDS);
396         assertTrue("isPresent", optional.isPresent());
397         assertEquals("# cars", numCars, ((Collection<?>) optional.get().getValue()).size());
398     }
399
400     @Test
401     @Ignore("Flushes out tell based leak needs to be handled separately")
402     public void testCloseTransactionMetadataLeak() throws Exception {
403         // Ask based frontend seems to have some issues with back to back close
404         Assume.assumeTrue(testParameter.isAssignableFrom(TestClientBackedDataStore.class));
405
406         final String testName = "testWriteTransactionWithSingleShard";
407         initDatastoresWithCars(testName);
408
409         final DOMStoreTransactionChain txChain = followerDistributedDataStore.createTransactionChain();
410
411         DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
412         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
413         writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
414         followerTestKit.doCommit(writeTx.ready());
415
416         int numCars = 5;
417         for (int i = 0; i < numCars; i++) {
418             writeTx = txChain.newWriteOnlyTransaction();
419             writeTx.close();
420
421             DOMStoreReadTransaction domStoreReadTransaction = txChain.newReadOnlyTransaction();
422             domStoreReadTransaction.read(CarsModel.BASE_PATH).get();
423
424             domStoreReadTransaction.close();
425         }
426
427         writeTx = txChain.newWriteOnlyTransaction();
428         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
429         writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
430         followerTestKit.doCommit(writeTx.ready());
431
432         // wait to let the shard catch up with purged
433         await("Close transaction purge leak test.").atMost(5, TimeUnit.SECONDS)
434                 .pollInterval(500, TimeUnit.MILLISECONDS)
435                 .untilAsserted(() -> {
436                     Optional<ActorRef> localShard =
437                             leaderDistributedDataStore.getActorUtils().findLocalShard("cars");
438                     FrontendShardDataTreeSnapshotMetadata frontendMetadata =
439                             (FrontendShardDataTreeSnapshotMetadata) leaderDistributedDataStore.getActorUtils()
440                                     .executeOperation(localShard.get(), new RequestFrontendMetadata());
441
442                     if (leaderDistributedDataStore.getActorUtils().getDatastoreContext().isUseTellBasedProtocol()) {
443                         Iterator<FrontendHistoryMetadata> iterator =
444                                 frontendMetadata.getClients().get(0).getCurrentHistories().iterator();
445                         FrontendHistoryMetadata metadata = iterator.next();
446                         while (iterator.hasNext() && metadata.getHistoryId() != 1) {
447                             metadata = iterator.next();
448                         }
449
450                         Set<Range<UnsignedLong>> ranges = metadata.getPurgedTransactions().asRanges();
451
452                         assertEquals(0, metadata.getClosedTransactions().size());
453                         assertEquals(1, ranges.size());
454                     } else {
455                         // ask based should track no metadata
456                         assertTrue(frontendMetadata.getClients().get(0).getCurrentHistories().isEmpty());
457                     }
458                 });
459
460         final Optional<NormalizedNode<?, ?>> optional = txChain.newReadOnlyTransaction()
461                 .read(CarsModel.CAR_LIST_PATH).get(5, TimeUnit.SECONDS);
462         assertTrue("isPresent", optional.isPresent());
463         assertEquals("# cars", numCars, ((Collection<?>) optional.get().getValue()).size());
464     }
465
466     @Test
467     public void testReadWriteTransactionWithSingleShard() throws Exception {
468         initDatastoresWithCars("testReadWriteTransactionWithSingleShard");
469
470         final DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
471         assertNotNull("newReadWriteTransaction returned null", rwTx);
472
473         rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
474         rwTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
475
476         final MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
477         rwTx.merge(CarsModel.newCarPath("optima"), car1);
478
479         verifyCars(rwTx, car1);
480
481         final MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(25000));
482         final YangInstanceIdentifier car2Path = CarsModel.newCarPath("sportage");
483         rwTx.merge(car2Path, car2);
484
485         verifyExists(rwTx, car2Path);
486
487         followerTestKit.doCommit(rwTx.ready());
488
489         verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car1, car2);
490     }
491
492     @Test
493     public void testWriteTransactionWithMultipleShards() throws Exception {
494         initDatastoresWithCarsAndPeople("testWriteTransactionWithMultipleShards");
495
496         final DOMStoreWriteTransaction writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
497         assertNotNull("newWriteOnlyTransaction returned null", writeTx);
498
499         final YangInstanceIdentifier carsPath = CarsModel.BASE_PATH;
500         final NormalizedNode<?, ?> carsNode = CarsModel.emptyContainer();
501         writeTx.write(carsPath, carsNode);
502
503         final YangInstanceIdentifier peoplePath = PeopleModel.BASE_PATH;
504         final NormalizedNode<?, ?> peopleNode = PeopleModel.emptyContainer();
505         writeTx.write(peoplePath, peopleNode);
506
507         followerTestKit.doCommit(writeTx.ready());
508
509         final DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
510
511         verifyNode(readTx, carsPath, carsNode);
512         verifyNode(readTx, peoplePath, peopleNode);
513     }
514
515     @Test
516     public void testReadWriteTransactionWithMultipleShards() throws Exception {
517         initDatastoresWithCarsAndPeople("testReadWriteTransactionWithMultipleShards");
518
519         final DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
520         assertNotNull("newReadWriteTransaction returned null", rwTx);
521
522         final YangInstanceIdentifier carsPath = CarsModel.BASE_PATH;
523         final NormalizedNode<?, ?> carsNode = CarsModel.emptyContainer();
524         rwTx.write(carsPath, carsNode);
525
526         final YangInstanceIdentifier peoplePath = PeopleModel.BASE_PATH;
527         final NormalizedNode<?, ?> peopleNode = PeopleModel.emptyContainer();
528         rwTx.write(peoplePath, peopleNode);
529
530         followerTestKit.doCommit(rwTx.ready());
531
532         final DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
533
534         verifyNode(readTx, carsPath, carsNode);
535         verifyNode(readTx, peoplePath, peopleNode);
536     }
537
538     @Test
539     public void testTransactionChainWithSingleShard() throws Exception {
540         initDatastoresWithCars("testTransactionChainWithSingleShard");
541
542         final DOMStoreTransactionChain txChain = followerDistributedDataStore.createTransactionChain();
543
544         // Add the top-level cars container with write-only.
545
546         final DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
547         assertNotNull("newWriteOnlyTransaction returned null", writeTx);
548
549         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
550
551         final DOMStoreThreePhaseCommitCohort writeTxReady = writeTx.ready();
552
553         // Verify the top-level cars container with read-only.
554
555         verifyNode(txChain.newReadOnlyTransaction(), CarsModel.BASE_PATH, CarsModel.emptyContainer());
556
557         // Perform car operations with read-write.
558
559         final DOMStoreReadWriteTransaction rwTx = txChain.newReadWriteTransaction();
560
561         verifyNode(rwTx, CarsModel.BASE_PATH, CarsModel.emptyContainer());
562
563         rwTx.merge(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
564
565         final MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
566         final YangInstanceIdentifier car1Path = CarsModel.newCarPath("optima");
567         rwTx.write(car1Path, car1);
568
569         verifyExists(rwTx, car1Path);
570
571         verifyCars(rwTx, car1);
572
573         final MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(25000));
574         rwTx.merge(CarsModel.newCarPath("sportage"), car2);
575
576         rwTx.delete(car1Path);
577
578         followerTestKit.doCommit(writeTxReady);
579
580         followerTestKit.doCommit(rwTx.ready());
581
582         txChain.close();
583
584         verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car2);
585     }
586
587     @Test
588     public void testTransactionChainWithMultipleShards() throws Exception {
589         initDatastoresWithCarsAndPeople("testTransactionChainWithMultipleShards");
590
591         final DOMStoreTransactionChain txChain = followerDistributedDataStore.createTransactionChain();
592
593         DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
594         assertNotNull("newWriteOnlyTransaction returned null", writeTx);
595
596         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
597         writeTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
598
599         writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
600         writeTx.write(PeopleModel.PERSON_LIST_PATH, PeopleModel.newPersonMapNode());
601
602         followerTestKit.doCommit(writeTx.ready());
603
604         final DOMStoreReadWriteTransaction readWriteTx = txChain.newReadWriteTransaction();
605
606         final MapEntryNode car = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
607         final YangInstanceIdentifier carPath = CarsModel.newCarPath("optima");
608         readWriteTx.write(carPath, car);
609
610         final MapEntryNode person = PeopleModel.newPersonEntry("jack");
611         final YangInstanceIdentifier personPath = PeopleModel.newPersonPath("jack");
612         readWriteTx.merge(personPath, person);
613
614         Optional<NormalizedNode<?, ?>> optional = readWriteTx.read(carPath).get(5, TimeUnit.SECONDS);
615         assertTrue("isPresent", optional.isPresent());
616         assertEquals("Data node", car, optional.get());
617
618         optional = readWriteTx.read(personPath).get(5, TimeUnit.SECONDS);
619         assertTrue("isPresent", optional.isPresent());
620         assertEquals("Data node", person, optional.get());
621
622         final DOMStoreThreePhaseCommitCohort cohort2 = readWriteTx.ready();
623
624         writeTx = txChain.newWriteOnlyTransaction();
625
626         writeTx.delete(personPath);
627
628         final DOMStoreThreePhaseCommitCohort cohort3 = writeTx.ready();
629
630         followerTestKit.doCommit(cohort2);
631         followerTestKit.doCommit(cohort3);
632
633         txChain.close();
634
635         final DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
636         verifyCars(readTx, car);
637
638         optional = readTx.read(personPath).get(5, TimeUnit.SECONDS);
639         assertFalse("isPresent", optional.isPresent());
640     }
641
642     @Test
643     public void testChainedTransactionFailureWithSingleShard() throws Exception {
644         initDatastoresWithCars("testChainedTransactionFailureWithSingleShard");
645
646         final ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
647                 ImmutableMap.<LogicalDatastoreType, DOMStore>builder().put(
648                         LogicalDatastoreType.CONFIGURATION, followerDistributedDataStore).build(),
649                         MoreExecutors.directExecutor());
650
651         final DOMTransactionChainListener listener = Mockito.mock(DOMTransactionChainListener.class);
652         final DOMTransactionChain txChain = broker.createTransactionChain(listener);
653
654         final DOMDataTreeWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
655
656         final ContainerNode invalidData = ImmutableContainerNodeBuilder.create().withNodeIdentifier(
657                 new YangInstanceIdentifier.NodeIdentifier(CarsModel.BASE_QNAME))
658                     .withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build();
659
660         writeTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData);
661
662         try {
663             writeTx.commit().get(5, TimeUnit.SECONDS);
664             fail("Expected TransactionCommitFailedException");
665         } catch (final ExecutionException e) {
666             // Expected
667         }
668
669         verify(listener, timeout(5000)).onTransactionChainFailed(eq(txChain), eq(writeTx), any(Throwable.class));
670
671         txChain.close();
672         broker.close();
673     }
674
675     @Test
676     public void testChainedTransactionFailureWithMultipleShards() throws Exception {
677         initDatastoresWithCarsAndPeople("testChainedTransactionFailureWithMultipleShards");
678
679         final ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
680                 ImmutableMap.<LogicalDatastoreType, DOMStore>builder().put(
681                         LogicalDatastoreType.CONFIGURATION, followerDistributedDataStore).build(),
682                         MoreExecutors.directExecutor());
683
684         final DOMTransactionChainListener listener = Mockito.mock(DOMTransactionChainListener.class);
685         final DOMTransactionChain txChain = broker.createTransactionChain(listener);
686
687         final DOMDataTreeWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
688
689         writeTx.put(LogicalDatastoreType.CONFIGURATION, PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
690
691         final ContainerNode invalidData = ImmutableContainerNodeBuilder.create().withNodeIdentifier(
692                 new YangInstanceIdentifier.NodeIdentifier(CarsModel.BASE_QNAME))
693                     .withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build();
694
695         // Note that merge will validate the data and fail but put succeeds b/c deep validation is not
696         // done for put for performance reasons.
697         writeTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData);
698
699         try {
700             writeTx.commit().get(5, TimeUnit.SECONDS);
701             fail("Expected TransactionCommitFailedException");
702         } catch (final ExecutionException e) {
703             // Expected
704         }
705
706         verify(listener, timeout(5000)).onTransactionChainFailed(eq(txChain), eq(writeTx), any(Throwable.class));
707
708         txChain.close();
709         broker.close();
710     }
711
712     @Test
713     public void testSingleShardTransactionsWithLeaderChanges() throws Exception {
714         followerDatastoreContextBuilder.backendAlivenessTimerIntervalInSeconds(2);
715         final String testName = "testSingleShardTransactionsWithLeaderChanges";
716         initDatastoresWithCars(testName);
717
718         final String followerCarShardName = "member-2-shard-cars-" + testName;
719         InMemoryJournal.addWriteMessagesCompleteLatch(followerCarShardName, 1, ApplyJournalEntries.class);
720
721         // Write top-level car container from the follower so it uses a remote Tx.
722
723         DOMStoreWriteTransaction writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
724
725         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
726         writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
727
728         followerTestKit.doCommit(writeTx.ready());
729
730         InMemoryJournal.waitForWriteMessagesComplete(followerCarShardName);
731
732         // Switch the leader to the follower
733
734         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder
735                 .shardElectionTimeoutFactor(1).customRaftPolicyImplementation(null));
736
737         TestKit.shutdownActorSystem(leaderSystem, true);
738         Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS);
739
740         followerTestKit.waitUntilNoLeader(followerDistributedDataStore.getActorUtils(), CARS);
741
742         leaderSystem = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member1"));
743         Cluster.get(leaderSystem).join(MEMBER_2_ADDRESS);
744
745         final DatastoreContext.Builder newMember1Builder = DatastoreContext.newBuilder()
746                 .shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5);
747         IntegrationTestKit newMember1TestKit = new IntegrationTestKit(leaderSystem, newMember1Builder, commitTimeout);
748
749         try (AbstractDataStore ds =
750                 newMember1TestKit.setupAbstractDataStore(
751                         testParameter, testName, MODULE_SHARDS_CARS_ONLY_1_2, false, CARS)) {
752
753             followerTestKit.waitUntilLeader(followerDistributedDataStore.getActorUtils(), CARS);
754
755             // Write a car entry to the new leader - should switch to local Tx
756
757             writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
758
759             MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
760             YangInstanceIdentifier car1Path = CarsModel.newCarPath("optima");
761             writeTx.merge(car1Path, car1);
762
763             followerTestKit.doCommit(writeTx.ready());
764
765             verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car1);
766         }
767     }
768
769     @SuppressWarnings("unchecked")
770     @Test
771     public void testReadyLocalTransactionForwardedToLeader() throws Exception {
772         initDatastoresWithCars("testReadyLocalTransactionForwardedToLeader");
773         followerTestKit.waitUntilLeader(followerDistributedDataStore.getActorUtils(), "cars");
774
775         final Optional<ActorRef> carsFollowerShard =
776                 followerDistributedDataStore.getActorUtils().findLocalShard("cars");
777         assertTrue("Cars follower shard found", carsFollowerShard.isPresent());
778
779         final DataTree dataTree = new InMemoryDataTreeFactory().create(
780             DataTreeConfiguration.DEFAULT_OPERATIONAL, SchemaContextHelper.full());
781
782         // Send a tx with immediate commit.
783
784         DataTreeModification modification = dataTree.takeSnapshot().newModification();
785         new WriteModification(CarsModel.BASE_PATH, CarsModel.emptyContainer()).apply(modification);
786         new MergeModification(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode()).apply(modification);
787
788         final MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
789         new WriteModification(CarsModel.newCarPath("optima"), car1).apply(modification);
790         modification.ready();
791
792         ReadyLocalTransaction readyLocal = new ReadyLocalTransaction(tx1 , modification, true, Optional.empty());
793
794         carsFollowerShard.get().tell(readyLocal, followerTestKit.getRef());
795         Object resp = followerTestKit.expectMsgClass(Object.class);
796         if (resp instanceof akka.actor.Status.Failure) {
797             throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause());
798         }
799
800         assertEquals("Response type", CommitTransactionReply.class, resp.getClass());
801
802         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1);
803
804         // Send another tx without immediate commit.
805
806         modification = dataTree.takeSnapshot().newModification();
807         MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(30000));
808         new WriteModification(CarsModel.newCarPath("sportage"), car2).apply(modification);
809         modification.ready();
810
811         readyLocal = new ReadyLocalTransaction(tx2 , modification, false, Optional.empty());
812
813         carsFollowerShard.get().tell(readyLocal, followerTestKit.getRef());
814         resp = followerTestKit.expectMsgClass(Object.class);
815         if (resp instanceof akka.actor.Status.Failure) {
816             throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause());
817         }
818
819         assertEquals("Response type", ReadyTransactionReply.class, resp.getClass());
820
821         final ActorSelection txActor = leaderDistributedDataStore.getActorUtils().actorSelection(
822                 ((ReadyTransactionReply)resp).getCohortPath());
823
824         final Supplier<Short> versionSupplier = Mockito.mock(Supplier.class);
825         Mockito.doReturn(DataStoreVersions.CURRENT_VERSION).when(versionSupplier).get();
826         ThreePhaseCommitCohortProxy cohort = new ThreePhaseCommitCohortProxy(
827                 leaderDistributedDataStore.getActorUtils(), Arrays.asList(
828                         new ThreePhaseCommitCohortProxy.CohortInfo(Futures.successful(txActor), versionSupplier)), tx2);
829         cohort.canCommit().get(5, TimeUnit.SECONDS);
830         cohort.preCommit().get(5, TimeUnit.SECONDS);
831         cohort.commit().get(5, TimeUnit.SECONDS);
832
833         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1, car2);
834     }
835
836     @SuppressWarnings("unchecked")
837     @Test
838     public void testForwardedReadyTransactionForwardedToLeader() throws Exception {
839         initDatastoresWithCars("testForwardedReadyTransactionForwardedToLeader");
840         followerTestKit.waitUntilLeader(followerDistributedDataStore.getActorUtils(), "cars");
841
842         final Optional<ActorRef> carsFollowerShard =
843                 followerDistributedDataStore.getActorUtils().findLocalShard("cars");
844         assertTrue("Cars follower shard found", carsFollowerShard.isPresent());
845
846         carsFollowerShard.get().tell(GetShardDataTree.INSTANCE, followerTestKit.getRef());
847         final DataTree dataTree = followerTestKit.expectMsgClass(DataTree.class);
848
849         // Send a tx with immediate commit.
850
851         DataTreeModification modification = dataTree.takeSnapshot().newModification();
852         new WriteModification(CarsModel.BASE_PATH, CarsModel.emptyContainer()).apply(modification);
853         new MergeModification(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode()).apply(modification);
854
855         final MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
856         new WriteModification(CarsModel.newCarPath("optima"), car1).apply(modification);
857
858         ForwardedReadyTransaction forwardedReady = new ForwardedReadyTransaction(tx1,
859                 DataStoreVersions.CURRENT_VERSION, new ReadWriteShardDataTreeTransaction(
860                         Mockito.mock(ShardDataTreeTransactionParent.class), tx1, modification), true,
861                 Optional.empty());
862
863         carsFollowerShard.get().tell(forwardedReady, followerTestKit.getRef());
864         Object resp = followerTestKit.expectMsgClass(Object.class);
865         if (resp instanceof akka.actor.Status.Failure) {
866             throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause());
867         }
868
869         assertEquals("Response type", CommitTransactionReply.class, resp.getClass());
870
871         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1);
872
873         // Send another tx without immediate commit.
874
875         modification = dataTree.takeSnapshot().newModification();
876         MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(30000));
877         new WriteModification(CarsModel.newCarPath("sportage"), car2).apply(modification);
878
879         forwardedReady = new ForwardedReadyTransaction(tx2,
880                 DataStoreVersions.CURRENT_VERSION, new ReadWriteShardDataTreeTransaction(
881                         Mockito.mock(ShardDataTreeTransactionParent.class), tx2, modification), false,
882                 Optional.empty());
883
884         carsFollowerShard.get().tell(forwardedReady, followerTestKit.getRef());
885         resp = followerTestKit.expectMsgClass(Object.class);
886         if (resp instanceof akka.actor.Status.Failure) {
887             throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause());
888         }
889
890         assertEquals("Response type", ReadyTransactionReply.class, resp.getClass());
891
892         ActorSelection txActor = leaderDistributedDataStore.getActorUtils().actorSelection(
893                 ((ReadyTransactionReply)resp).getCohortPath());
894
895         final Supplier<Short> versionSupplier = Mockito.mock(Supplier.class);
896         Mockito.doReturn(DataStoreVersions.CURRENT_VERSION).when(versionSupplier).get();
897         final ThreePhaseCommitCohortProxy cohort = new ThreePhaseCommitCohortProxy(
898                 leaderDistributedDataStore.getActorUtils(), Arrays.asList(
899                         new ThreePhaseCommitCohortProxy.CohortInfo(Futures.successful(txActor), versionSupplier)), tx2);
900         cohort.canCommit().get(5, TimeUnit.SECONDS);
901         cohort.preCommit().get(5, TimeUnit.SECONDS);
902         cohort.commit().get(5, TimeUnit.SECONDS);
903
904         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1, car2);
905     }
906
907     @Test
908     public void testTransactionForwardedToLeaderAfterRetry() throws Exception {
909         // FIXME: remove when test passes also for ClientBackedDataStore
910         Assume.assumeTrue(DistributedDataStore.class.isAssignableFrom(testParameter));
911         followerDatastoreContextBuilder.shardBatchedModificationCount(2);
912         leaderDatastoreContextBuilder.shardBatchedModificationCount(2);
913         initDatastoresWithCarsAndPeople("testTransactionForwardedToLeaderAfterRetry");
914
915         // Do an initial write to get the primary shard info cached.
916
917         final DOMStoreWriteTransaction initialWriteTx = followerDistributedDataStore.newWriteOnlyTransaction();
918         initialWriteTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
919         initialWriteTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
920         followerTestKit.doCommit(initialWriteTx.ready());
921
922         // Wait for the commit to be replicated to the follower.
923
924         MemberNode.verifyRaftState(followerDistributedDataStore, "cars",
925             raftState -> assertEquals("getLastApplied", 1, raftState.getLastApplied()));
926
927         MemberNode.verifyRaftState(followerDistributedDataStore, "people",
928             raftState -> assertEquals("getLastApplied", 1, raftState.getLastApplied()));
929
930         // Prepare, ready and canCommit a WO tx that writes to 2 shards. This will become the current tx in
931         // the leader shard.
932
933         final DOMStoreWriteTransaction writeTx1 = followerDistributedDataStore.newWriteOnlyTransaction();
934         writeTx1.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
935         writeTx1.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
936         final DOMStoreThreePhaseCommitCohort writeTx1Cohort = writeTx1.ready();
937         final ListenableFuture<Boolean> writeTx1CanCommit = writeTx1Cohort.canCommit();
938         writeTx1CanCommit.get(5, TimeUnit.SECONDS);
939
940         // Prepare and ready another WO tx that writes to 2 shards but don't canCommit yet. This will be queued
941         // in the leader shard.
942
943         final DOMStoreWriteTransaction writeTx2 = followerDistributedDataStore.newWriteOnlyTransaction();
944         final LinkedList<MapEntryNode> cars = new LinkedList<>();
945         int carIndex = 1;
946         cars.add(CarsModel.newCarEntry("car" + carIndex, BigInteger.valueOf(carIndex)));
947         writeTx2.write(CarsModel.newCarPath("car" + carIndex), cars.getLast());
948         carIndex++;
949         NormalizedNode<?, ?> people = ImmutableNodes.mapNodeBuilder(PeopleModel.PERSON_QNAME)
950                 .withChild(PeopleModel.newPersonEntry("Dude")).build();
951         writeTx2.write(PeopleModel.PERSON_LIST_PATH, people);
952         final DOMStoreThreePhaseCommitCohort writeTx2Cohort = writeTx2.ready();
953
954         // Prepare another WO that writes to a single shard and thus will be directly committed on ready. This
955         // tx writes 5 cars so 2 BatchedModidifications messages will be sent initially and cached in the
956         // leader shard (with shardBatchedModificationCount set to 2). The 3rd BatchedModidifications will be
957         // sent on ready.
958
959         final DOMStoreWriteTransaction writeTx3 = followerDistributedDataStore.newWriteOnlyTransaction();
960         for (int i = 1; i <= 5; i++, carIndex++) {
961             cars.add(CarsModel.newCarEntry("car" + carIndex, BigInteger.valueOf(carIndex)));
962             writeTx3.write(CarsModel.newCarPath("car" + carIndex), cars.getLast());
963         }
964
965         // Prepare another WO that writes to a single shard. This will send a single BatchedModidifications
966         // message on ready.
967
968         final DOMStoreWriteTransaction writeTx4 = followerDistributedDataStore.newWriteOnlyTransaction();
969         cars.add(CarsModel.newCarEntry("car" + carIndex, BigInteger.valueOf(carIndex)));
970         writeTx4.write(CarsModel.newCarPath("car" + carIndex), cars.getLast());
971         carIndex++;
972
973         // Prepare a RW tx that will create a tx actor and send a ForwardedReadyTransaciton message to the
974         // leader shard on ready.
975
976         final DOMStoreReadWriteTransaction readWriteTx = followerDistributedDataStore.newReadWriteTransaction();
977         cars.add(CarsModel.newCarEntry("car" + carIndex, BigInteger.valueOf(carIndex)));
978         readWriteTx.write(CarsModel.newCarPath("car" + carIndex), cars.getLast());
979
980         IntegrationTestKit.verifyShardStats(leaderDistributedDataStore, "cars",
981             stats -> assertEquals("getReadWriteTransactionCount", 5, stats.getReadWriteTransactionCount()));
982
983         // Disable elections on the leader so it switches to follower.
984
985         sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder
986                 .customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName())
987                 .shardElectionTimeoutFactor(10));
988
989         leaderTestKit.waitUntilNoLeader(leaderDistributedDataStore.getActorUtils(), "cars");
990
991         // Submit all tx's - the messages should get queued for retry.
992
993         final ListenableFuture<Boolean> writeTx2CanCommit = writeTx2Cohort.canCommit();
994         final DOMStoreThreePhaseCommitCohort writeTx3Cohort = writeTx3.ready();
995         final DOMStoreThreePhaseCommitCohort writeTx4Cohort = writeTx4.ready();
996         final DOMStoreThreePhaseCommitCohort rwTxCohort = readWriteTx.ready();
997
998         // Enable elections on the other follower so it becomes the leader, at which point the
999         // tx's should get forwarded from the previous leader to the new leader to complete the commits.
1000
1001         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder
1002                 .customRaftPolicyImplementation(null).shardElectionTimeoutFactor(1));
1003         IntegrationTestKit.findLocalShard(followerDistributedDataStore.getActorUtils(), "cars")
1004                 .tell(TimeoutNow.INSTANCE, ActorRef.noSender());
1005         IntegrationTestKit.findLocalShard(followerDistributedDataStore.getActorUtils(), "people")
1006                 .tell(TimeoutNow.INSTANCE, ActorRef.noSender());
1007
1008         followerTestKit.doCommit(writeTx1CanCommit, writeTx1Cohort);
1009         followerTestKit.doCommit(writeTx2CanCommit, writeTx2Cohort);
1010         followerTestKit.doCommit(writeTx3Cohort);
1011         followerTestKit.doCommit(writeTx4Cohort);
1012         followerTestKit.doCommit(rwTxCohort);
1013
1014         DOMStoreReadTransaction readTx = leaderDistributedDataStore.newReadOnlyTransaction();
1015         verifyCars(readTx, cars.toArray(new MapEntryNode[cars.size()]));
1016         verifyNode(readTx, PeopleModel.PERSON_LIST_PATH, people);
1017     }
1018
1019     @Test
1020     public void testLeadershipTransferOnShutdown() throws Exception {
1021         // FIXME: remove when test passes also for ClientBackedDataStore
1022         Assume.assumeTrue(DistributedDataStore.class.isAssignableFrom(testParameter));
1023         leaderDatastoreContextBuilder.shardBatchedModificationCount(1);
1024         followerDatastoreContextBuilder.shardElectionTimeoutFactor(10).customRaftPolicyImplementation(null);
1025         final String testName = "testLeadershipTransferOnShutdown";
1026         initDatastores(testName, MODULE_SHARDS_CARS_PEOPLE_1_2_3, CARS_AND_PEOPLE);
1027
1028         final IntegrationTestKit follower2TestKit = new IntegrationTestKit(follower2System,
1029                 DatastoreContext.newBuilderFrom(followerDatastoreContextBuilder.build()).operationTimeoutInMillis(500),
1030                 commitTimeout);
1031         try (AbstractDataStore follower2DistributedDataStore = follower2TestKit.setupAbstractDataStore(
1032                 testParameter, testName, MODULE_SHARDS_CARS_PEOPLE_1_2_3, false)) {
1033
1034             followerTestKit.waitForMembersUp("member-3");
1035             follower2TestKit.waitForMembersUp("member-1", "member-2");
1036
1037             // Create and submit a couple tx's so they're pending.
1038
1039             DOMStoreWriteTransaction writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
1040             writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
1041             writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
1042             writeTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
1043             final DOMStoreThreePhaseCommitCohort cohort1 = writeTx.ready();
1044
1045             IntegrationTestKit.verifyShardStats(leaderDistributedDataStore, "cars",
1046                 stats -> assertEquals("getTxCohortCacheSize", 1, stats.getTxCohortCacheSize()));
1047
1048             writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
1049             final MapEntryNode car = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
1050             writeTx.write(CarsModel.newCarPath("optima"), car);
1051             final DOMStoreThreePhaseCommitCohort cohort2 = writeTx.ready();
1052
1053             IntegrationTestKit.verifyShardStats(leaderDistributedDataStore, "cars",
1054                 stats -> assertEquals("getTxCohortCacheSize", 2, stats.getTxCohortCacheSize()));
1055
1056             // Gracefully stop the leader via a Shutdown message.
1057
1058             sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder
1059                 .shardElectionTimeoutFactor(100));
1060
1061             final FiniteDuration duration = FiniteDuration.create(5, TimeUnit.SECONDS);
1062             final Future<ActorRef> future = leaderDistributedDataStore.getActorUtils().findLocalShardAsync("cars");
1063             final ActorRef leaderActor = Await.result(future, duration);
1064
1065             final Future<Boolean> stopFuture = Patterns.gracefulStop(leaderActor, duration, Shutdown.INSTANCE);
1066
1067             // Commit the 2 transactions. They should finish and succeed.
1068
1069             followerTestKit.doCommit(cohort1);
1070             followerTestKit.doCommit(cohort2);
1071
1072             // Wait for the leader actor stopped.
1073
1074             final Boolean stopped = Await.result(stopFuture, duration);
1075             assertEquals("Stopped", Boolean.TRUE, stopped);
1076
1077             // Verify leadership was transferred by reading the committed data from the other nodes.
1078
1079             verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car);
1080             verifyCars(follower2DistributedDataStore.newReadOnlyTransaction(), car);
1081         }
1082     }
1083
1084     @Test
1085     public void testTransactionWithIsolatedLeader() throws Exception {
1086         // FIXME: remove when test passes also for ClientBackedDataStore
1087         Assume.assumeTrue(DistributedDataStore.class.isAssignableFrom(testParameter));
1088         // Set the isolated leader check interval high so we can control the switch to IsolatedLeader.
1089         leaderDatastoreContextBuilder.shardIsolatedLeaderCheckIntervalInMillis(10000000);
1090         final String testName = "testTransactionWithIsolatedLeader";
1091         initDatastoresWithCars(testName);
1092
1093         // Tx that is submitted after the follower is stopped but before the leader transitions to IsolatedLeader.
1094         final DOMStoreWriteTransaction preIsolatedLeaderWriteTx = leaderDistributedDataStore.newWriteOnlyTransaction();
1095         preIsolatedLeaderWriteTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
1096
1097         // Tx that is submitted after the leader transitions to IsolatedLeader.
1098         final DOMStoreWriteTransaction noShardLeaderWriteTx = leaderDistributedDataStore.newWriteOnlyTransaction();
1099         noShardLeaderWriteTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
1100
1101         // Tx that is submitted after the follower is reinstated.
1102         final DOMStoreWriteTransaction successWriteTx = leaderDistributedDataStore.newWriteOnlyTransaction();
1103         successWriteTx.merge(CarsModel.BASE_PATH, CarsModel.emptyContainer());
1104
1105         // Stop the follower
1106         followerTestKit.watch(followerDistributedDataStore.getActorUtils().getShardManager());
1107         followerDistributedDataStore.close();
1108         followerTestKit.expectTerminated(followerDistributedDataStore.getActorUtils().getShardManager());
1109
1110         // Submit the preIsolatedLeaderWriteTx so it's pending
1111         final DOMStoreThreePhaseCommitCohort preIsolatedLeaderTxCohort = preIsolatedLeaderWriteTx.ready();
1112
1113         // Change the isolated leader check interval low so it changes to IsolatedLeader.
1114         sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder
1115                 .shardIsolatedLeaderCheckIntervalInMillis(200));
1116
1117         MemberNode.verifyRaftState(leaderDistributedDataStore, "cars",
1118             raftState -> assertEquals("getRaftState", "IsolatedLeader", raftState.getRaftState()));
1119
1120         try {
1121             leaderTestKit.doCommit(noShardLeaderWriteTx.ready());
1122             fail("Expected NoShardLeaderException");
1123         } catch (final ExecutionException e) {
1124             assertEquals("getCause", NoShardLeaderException.class, Throwables.getRootCause(e).getClass());
1125         }
1126
1127         sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder
1128                 .shardElectionTimeoutFactor(100));
1129
1130         final DOMStoreThreePhaseCommitCohort successTxCohort = successWriteTx.ready();
1131
1132         followerDistributedDataStore = followerTestKit.setupAbstractDataStore(
1133                 testParameter, testName, MODULE_SHARDS_CARS_ONLY_1_2, false, CARS);
1134
1135         leaderTestKit.doCommit(preIsolatedLeaderTxCohort);
1136         leaderTestKit.doCommit(successTxCohort);
1137     }
1138
1139     @Test
1140     public void testTransactionWithShardLeaderNotResponding() throws Exception {
1141         followerDatastoreContextBuilder.frontendRequestTimeoutInSeconds(2);
1142         followerDatastoreContextBuilder.shardElectionTimeoutFactor(50);
1143         initDatastoresWithCars("testTransactionWithShardLeaderNotResponding");
1144
1145         // Do an initial read to get the primary shard info cached.
1146
1147         final DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
1148         readTx.read(CarsModel.BASE_PATH).get(5, TimeUnit.SECONDS);
1149
1150         // Shutdown the leader and try to create a new tx.
1151
1152         TestKit.shutdownActorSystem(leaderSystem, true);
1153
1154         followerDatastoreContextBuilder.operationTimeoutInMillis(50).shardElectionTimeoutFactor(1);
1155         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder);
1156
1157         final DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
1158
1159         rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
1160
1161         try {
1162             followerTestKit.doCommit(rwTx.ready());
1163             fail("Exception expected");
1164         } catch (final ExecutionException e) {
1165             final String msg = "Unexpected exception: " + Throwables.getStackTraceAsString(e.getCause());
1166             if (DistributedDataStore.class.isAssignableFrom(testParameter)) {
1167                 assertTrue(msg, Throwables.getRootCause(e) instanceof NoShardLeaderException
1168                         || e.getCause() instanceof ShardLeaderNotRespondingException);
1169             } else {
1170                 assertTrue(msg, Throwables.getRootCause(e) instanceof RequestTimeoutException);
1171             }
1172         }
1173     }
1174
1175     @Test
1176     public void testTransactionWithCreateTxFailureDueToNoLeader() throws Exception {
1177         followerDatastoreContextBuilder.frontendRequestTimeoutInSeconds(2);
1178         initDatastoresWithCars("testTransactionWithCreateTxFailureDueToNoLeader");
1179
1180         // Do an initial read to get the primary shard info cached.
1181
1182         final DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
1183         readTx.read(CarsModel.BASE_PATH).get(5, TimeUnit.SECONDS);
1184
1185         // Shutdown the leader and try to create a new tx.
1186
1187         TestKit.shutdownActorSystem(leaderSystem, true);
1188
1189         Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS);
1190
1191         Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
1192
1193         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder
1194                 .operationTimeoutInMillis(10).shardElectionTimeoutFactor(1).customRaftPolicyImplementation(null));
1195
1196         final DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
1197
1198         rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
1199
1200         try {
1201             followerTestKit.doCommit(rwTx.ready());
1202             fail("Exception expected");
1203         } catch (final ExecutionException e) {
1204             final String msg = "Unexpected exception: " + Throwables.getStackTraceAsString(e.getCause());
1205             if (DistributedDataStore.class.isAssignableFrom(testParameter)) {
1206                 assertTrue(msg, Throwables.getRootCause(e) instanceof NoShardLeaderException);
1207             } else {
1208                 assertTrue(msg, Throwables.getRootCause(e) instanceof RequestTimeoutException);
1209             }
1210         }
1211     }
1212
1213     @Test
1214     public void testTransactionRetryWithInitialAskTimeoutExOnCreateTx() throws Exception {
1215         followerDatastoreContextBuilder.backendAlivenessTimerIntervalInSeconds(2);
1216         String testName = "testTransactionRetryWithInitialAskTimeoutExOnCreateTx";
1217         initDatastores(testName, MODULE_SHARDS_CARS_1_2_3, CARS);
1218
1219         final DatastoreContext.Builder follower2DatastoreContextBuilder = DatastoreContext.newBuilder()
1220                 .shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(10);
1221         final IntegrationTestKit follower2TestKit = new IntegrationTestKit(
1222                 follower2System, follower2DatastoreContextBuilder, commitTimeout);
1223
1224         try (AbstractDataStore ds =
1225                 follower2TestKit.setupAbstractDataStore(
1226                         testParameter, testName, MODULE_SHARDS_CARS_1_2_3, false, CARS)) {
1227
1228             followerTestKit.waitForMembersUp("member-1", "member-3");
1229             follower2TestKit.waitForMembersUp("member-1", "member-2");
1230
1231             // Do an initial read to get the primary shard info cached.
1232
1233             final DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
1234             readTx.read(CarsModel.BASE_PATH).get(5, TimeUnit.SECONDS);
1235
1236             // Shutdown the leader and try to create a new tx.
1237
1238             TestKit.shutdownActorSystem(leaderSystem, true);
1239
1240             Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS);
1241
1242             sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder
1243                 .operationTimeoutInMillis(500).shardElectionTimeoutFactor(5).customRaftPolicyImplementation(null));
1244
1245             final DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
1246
1247             rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
1248
1249             followerTestKit.doCommit(rwTx.ready());
1250         }
1251     }
1252
1253     @Test
1254     public void testInstallSnapshot() throws Exception {
1255         final String testName = "testInstallSnapshot";
1256         final String leaderCarShardName = "member-1-shard-cars-" + testName;
1257         final String followerCarShardName = "member-2-shard-cars-" + testName;
1258
1259         // Setup a saved snapshot on the leader. The follower will startup with no data and the leader should
1260         // install a snapshot to sync the follower.
1261
1262         DataTree tree = new InMemoryDataTreeFactory().create(DataTreeConfiguration.DEFAULT_CONFIGURATION,
1263             SchemaContextHelper.full());
1264
1265         final ContainerNode carsNode = CarsModel.newCarsNode(
1266                 CarsModel.newCarsMapNode(CarsModel.newCarEntry("optima", BigInteger.valueOf(20000))));
1267         AbstractShardTest.writeToStore(tree, CarsModel.BASE_PATH, carsNode);
1268
1269         final NormalizedNode<?, ?> snapshotRoot = AbstractShardTest.readStore(tree, YangInstanceIdentifier.EMPTY);
1270         final Snapshot initialSnapshot = Snapshot.create(
1271                 new ShardSnapshotState(new MetadataShardDataTreeSnapshot(snapshotRoot)),
1272                 Collections.emptyList(), 5, 1, 5, 1, 1, null, null);
1273         InMemorySnapshotStore.addSnapshot(leaderCarShardName, initialSnapshot);
1274
1275         InMemorySnapshotStore.addSnapshotSavedLatch(leaderCarShardName);
1276         InMemorySnapshotStore.addSnapshotSavedLatch(followerCarShardName);
1277
1278         initDatastoresWithCars(testName);
1279
1280         final Optional<NormalizedNode<?, ?>> readOptional = leaderDistributedDataStore.newReadOnlyTransaction().read(
1281                 CarsModel.BASE_PATH).get(5, TimeUnit.SECONDS);
1282         assertTrue("isPresent", readOptional.isPresent());
1283         assertEquals("Node", carsNode, readOptional.get());
1284
1285         verifySnapshot(InMemorySnapshotStore.waitForSavedSnapshot(leaderCarShardName, Snapshot.class),
1286                 initialSnapshot, snapshotRoot);
1287
1288         verifySnapshot(InMemorySnapshotStore.waitForSavedSnapshot(followerCarShardName, Snapshot.class),
1289                 initialSnapshot, snapshotRoot);
1290     }
1291
1292     @Test
1293     public void testReadWriteMessageSlicing() throws Exception {
1294         // The slicing is only implemented for tell-based protocol
1295         Assume.assumeTrue(ClientBackedDataStore.class.isAssignableFrom(testParameter));
1296
1297         leaderDatastoreContextBuilder.maximumMessageSliceSize(100);
1298         followerDatastoreContextBuilder.maximumMessageSliceSize(100);
1299         initDatastoresWithCars("testLargeReadReplySlicing");
1300
1301         final DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
1302
1303         final NormalizedNode<?, ?> carsNode = CarsModel.create();
1304         rwTx.write(CarsModel.BASE_PATH, carsNode);
1305
1306         verifyNode(rwTx, CarsModel.BASE_PATH, carsNode);
1307     }
1308
1309     private static void verifySnapshot(final Snapshot actual, final Snapshot expected,
1310                                        final NormalizedNode<?, ?> expRoot) {
1311         assertEquals("Snapshot getLastAppliedTerm", expected.getLastAppliedTerm(), actual.getLastAppliedTerm());
1312         assertEquals("Snapshot getLastAppliedIndex", expected.getLastAppliedIndex(), actual.getLastAppliedIndex());
1313         assertEquals("Snapshot getLastTerm", expected.getLastTerm(), actual.getLastTerm());
1314         assertEquals("Snapshot getLastIndex", expected.getLastIndex(), actual.getLastIndex());
1315         assertEquals("Snapshot state type", ShardSnapshotState.class, actual.getState().getClass());
1316         MetadataShardDataTreeSnapshot shardSnapshot =
1317                 (MetadataShardDataTreeSnapshot) ((ShardSnapshotState)actual.getState()).getSnapshot();
1318         assertEquals("Snapshot root node", expRoot, shardSnapshot.getRootNode().get());
1319     }
1320
1321     private static void sendDatastoreContextUpdate(final AbstractDataStore dataStore, final Builder builder) {
1322         final Builder newBuilder = DatastoreContext.newBuilderFrom(builder.build());
1323         final DatastoreContextFactory mockContextFactory = Mockito.mock(DatastoreContextFactory.class);
1324         final Answer<DatastoreContext> answer = invocation -> newBuilder.build();
1325         Mockito.doAnswer(answer).when(mockContextFactory).getBaseDatastoreContext();
1326         Mockito.doAnswer(answer).when(mockContextFactory).getShardDatastoreContext(Mockito.anyString());
1327         dataStore.onDatastoreContextUpdated(mockContextFactory);
1328     }
1329 }