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