BUG-5280: make sure all DistributedDataStore instances are shut down
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / DistributedDataStoreRemotingIntegrationTest.java
1 /*
2  * Copyright (c) 2015 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 import akka.actor.ActorRef;
19 import akka.actor.ActorSelection;
20 import akka.actor.ActorSystem;
21 import akka.actor.Address;
22 import akka.actor.AddressFromURIString;
23 import akka.cluster.Cluster;
24 import akka.dispatch.Futures;
25 import akka.pattern.AskTimeoutException;
26 import akka.pattern.Patterns;
27 import akka.testkit.JavaTestKit;
28 import com.google.common.base.Optional;
29 import com.google.common.base.Supplier;
30 import com.google.common.collect.ImmutableMap;
31 import com.google.common.util.concurrent.ListenableFuture;
32 import com.google.common.util.concurrent.MoreExecutors;
33 import com.google.common.util.concurrent.Uninterruptibles;
34 import com.typesafe.config.ConfigFactory;
35 import java.math.BigInteger;
36 import java.util.Arrays;
37 import java.util.LinkedList;
38 import java.util.concurrent.ExecutionException;
39 import java.util.concurrent.TimeUnit;
40 import org.junit.After;
41 import org.junit.Before;
42 import org.junit.Test;
43 import org.mockito.Mockito;
44 import org.mockito.invocation.InvocationOnMock;
45 import org.mockito.stubbing.Answer;
46 import org.opendaylight.controller.cluster.databroker.ConcurrentDOMDataBroker;
47 import org.opendaylight.controller.cluster.datastore.DatastoreContext.Builder;
48 import org.opendaylight.controller.cluster.datastore.IntegrationTestKit.ShardStatsVerifier;
49 import org.opendaylight.controller.cluster.datastore.MemberNode.RaftStateVerifier;
50 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
51 import org.opendaylight.controller.cluster.datastore.exceptions.ShardLeaderNotRespondingException;
52 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
53 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
54 import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction;
55 import org.opendaylight.controller.cluster.datastore.messages.GetShardDataTree;
56 import org.opendaylight.controller.cluster.datastore.messages.ReadyLocalTransaction;
57 import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
58 import org.opendaylight.controller.cluster.datastore.modification.MergeModification;
59 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
60 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
61 import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState;
62 import org.opendaylight.controller.cluster.raft.client.messages.Shutdown;
63 import org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy;
64 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
65 import org.opendaylight.controller.md.cluster.datastore.model.CarsModel;
66 import org.opendaylight.controller.md.cluster.datastore.model.PeopleModel;
67 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
68 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
69 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
70 import org.opendaylight.controller.md.sal.common.api.data.TransactionChainListener;
71 import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
72 import org.opendaylight.controller.md.sal.dom.api.DOMDataWriteTransaction;
73 import org.opendaylight.controller.md.sal.dom.api.DOMTransactionChain;
74 import org.opendaylight.controller.sal.core.spi.data.DOMStore;
75 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
76 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
77 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
78 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
79 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
80 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
81 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
82 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
83 import org.opendaylight.yangtools.yang.data.api.schema.MapNode;
84 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
85 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
86 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification;
87 import org.opendaylight.yangtools.yang.data.api.schema.tree.TipProducingDataTree;
88 import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType;
89 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
90 import org.opendaylight.yangtools.yang.data.impl.schema.builder.api.CollectionNodeBuilder;
91 import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder;
92 import org.opendaylight.yangtools.yang.data.impl.schema.tree.InMemoryDataTreeFactory;
93 import scala.concurrent.Await;
94 import scala.concurrent.Future;
95 import scala.concurrent.duration.FiniteDuration;
96
97 /**
98  * End-to-end distributed data store tests that exercise remote shards and transactions.
99  *
100  * @author Thomas Pantelis
101  */
102 public class DistributedDataStoreRemotingIntegrationTest {
103
104     private static final String[] CARS_AND_PEOPLE = {"cars", "people"};
105     private static final String[] CARS = {"cars"};
106
107     private static final Address MEMBER_1_ADDRESS = AddressFromURIString.parse("akka.tcp://cluster-test@127.0.0.1:2558");
108     private static final Address MEMBER_2_ADDRESS = AddressFromURIString.parse("akka.tcp://cluster-test@127.0.0.1:2559");
109
110     private static final String MODULE_SHARDS_CARS_ONLY_1_2 = "module-shards-cars-member-1-and-2.conf";
111     private static final String MODULE_SHARDS_CARS_PEOPLE_1_2 = "module-shards-member1-and-2.conf";
112     private static final String MODULE_SHARDS_CARS_PEOPLE_1_2_3 = "module-shards-member1-and-2-and-3.conf";
113
114     private ActorSystem leaderSystem;
115     private ActorSystem followerSystem;
116     private ActorSystem follower2System;
117
118     private final DatastoreContext.Builder leaderDatastoreContextBuilder =
119             DatastoreContext.newBuilder().shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(2);
120
121     private final DatastoreContext.Builder followerDatastoreContextBuilder =
122             DatastoreContext.newBuilder().shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5).
123                 customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName());
124
125     private DistributedDataStore followerDistributedDataStore;
126     private DistributedDataStore leaderDistributedDataStore;
127     private IntegrationTestKit followerTestKit;
128     private IntegrationTestKit leaderTestKit;
129
130     @Before
131     public void setUp() {
132         leaderSystem = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member1"));
133         Cluster.get(leaderSystem).join(MEMBER_1_ADDRESS);
134
135         followerSystem = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member2"));
136         Cluster.get(followerSystem).join(MEMBER_1_ADDRESS);
137
138         follower2System = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member3"));
139         Cluster.get(follower2System).join(MEMBER_1_ADDRESS);
140     }
141
142     @After
143     public void tearDown() {
144         JavaTestKit.shutdownActorSystem(leaderSystem);
145         JavaTestKit.shutdownActorSystem(followerSystem);
146         JavaTestKit.shutdownActorSystem(follower2System);
147     }
148
149     private void initDatastoresWithCars(String type) {
150         initDatastores(type, MODULE_SHARDS_CARS_ONLY_1_2, CARS);
151     }
152
153     private void initDatastoresWithCarsAndPeople(String type) {
154         initDatastores(type, MODULE_SHARDS_CARS_PEOPLE_1_2, CARS_AND_PEOPLE);
155     }
156
157     private void initDatastores(String type, String moduleShardsConfig, String[] shards) {
158         leaderTestKit = new IntegrationTestKit(leaderSystem, leaderDatastoreContextBuilder);
159
160         leaderDistributedDataStore = leaderTestKit.setupDistributedDataStore(type, moduleShardsConfig, false, shards);
161
162         followerTestKit = new IntegrationTestKit(followerSystem, followerDatastoreContextBuilder);
163         followerDistributedDataStore = followerTestKit.setupDistributedDataStore(type, moduleShardsConfig, false, shards);
164
165         leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(), shards);
166     }
167
168     private static void verifyCars(DOMStoreReadTransaction readTx, MapEntryNode... entries) throws Exception {
169         Optional<NormalizedNode<?, ?>> optional = readTx.read(CarsModel.CAR_LIST_PATH).get(5, TimeUnit.SECONDS);
170         assertEquals("isPresent", true, optional.isPresent());
171
172         CollectionNodeBuilder<MapEntryNode, MapNode> listBuilder = ImmutableNodes.mapNodeBuilder(CarsModel.CAR_QNAME);
173         for(NormalizedNode<?, ?> entry: entries) {
174             listBuilder.withChild((MapEntryNode) entry);
175         }
176
177         assertEquals("Car list node", listBuilder.build(), optional.get());
178     }
179
180     private static void verifyNode(DOMStoreReadTransaction readTx, YangInstanceIdentifier path, NormalizedNode<?, ?> expNode)
181             throws Exception {
182         Optional<NormalizedNode<?, ?>> optional = readTx.read(path).get(5, TimeUnit.SECONDS);
183         assertEquals("isPresent", true, optional.isPresent());
184         assertEquals("Data node", expNode, optional.get());
185     }
186
187     private static void verifyExists(DOMStoreReadTransaction readTx, YangInstanceIdentifier path) throws Exception {
188         Boolean exists = readTx.exists(path).get(5, TimeUnit.SECONDS);
189         assertEquals("exists", true, exists);
190     }
191
192     @Test
193     public void testWriteTransactionWithSingleShard() throws Exception {
194         String testName = "testWriteTransactionWithSingleShard";
195         initDatastoresWithCars(testName);
196
197         String followerCarShardName = "member-2-shard-cars-" + testName;
198         InMemoryJournal.addWriteMessagesCompleteLatch(followerCarShardName, 2, ApplyJournalEntries.class );
199
200         DOMStoreWriteTransaction writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
201         assertNotNull("newWriteOnlyTransaction returned null", writeTx);
202
203         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
204         writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
205
206         MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
207         YangInstanceIdentifier car1Path = CarsModel.newCarPath("optima");
208         writeTx.merge(car1Path, car1);
209
210         MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(25000));
211         YangInstanceIdentifier car2Path = CarsModel.newCarPath("sportage");
212         writeTx.merge(car2Path, car2);
213
214         followerTestKit.doCommit(writeTx.ready());
215
216         verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car1, car2);
217
218         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1, car2);
219
220         // Test delete
221
222         writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
223
224         writeTx.delete(car1Path);
225
226         followerTestKit.doCommit(writeTx.ready());
227
228         verifyExists(followerDistributedDataStore.newReadOnlyTransaction(), car2Path);
229
230         verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car2);
231
232         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car2);
233
234         // Re-instate the follower member 2 as a single-node to verify replication and recovery.
235
236         InMemoryJournal.waitForWriteMessagesComplete(followerCarShardName);
237
238         JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
239         JavaTestKit.shutdownActorSystem(followerSystem, null, true);
240
241         ActorSystem newSystem = ActorSystem.create("reinstated-member2", ConfigFactory.load().getConfig("Member2"));
242
243         try (DistributedDataStore member2Datastore = new IntegrationTestKit(newSystem, leaderDatastoreContextBuilder).
244                 setupDistributedDataStore(testName, "module-shards-member2", true, CARS_AND_PEOPLE)) {
245             verifyCars(member2Datastore.newReadOnlyTransaction(), car2);
246         }
247
248         JavaTestKit.shutdownActorSystem(newSystem);
249     }
250
251     @Test
252     public void testReadWriteTransactionWithSingleShard() throws Exception {
253         initDatastoresWithCars("testReadWriteTransactionWithSingleShard");
254
255         DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
256         assertNotNull("newReadWriteTransaction returned null", rwTx);
257
258         rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
259         rwTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
260
261         MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
262         rwTx.merge(CarsModel.newCarPath("optima"), car1);
263
264         verifyCars(rwTx, car1);
265
266         MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(25000));
267         YangInstanceIdentifier car2Path = CarsModel.newCarPath("sportage");
268         rwTx.merge(car2Path, car2);
269
270         verifyExists(rwTx, car2Path);
271
272         followerTestKit.doCommit(rwTx.ready());
273
274         verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car1, car2);
275     }
276
277     @Test
278     public void testWriteTransactionWithMultipleShards() throws Exception {
279         initDatastoresWithCarsAndPeople("testWriteTransactionWithMultipleShards");
280
281         DOMStoreWriteTransaction writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
282         assertNotNull("newWriteOnlyTransaction returned null", writeTx);
283
284         YangInstanceIdentifier carsPath = CarsModel.BASE_PATH;
285         NormalizedNode<?, ?> carsNode = CarsModel.emptyContainer();
286         writeTx.write(carsPath, carsNode);
287
288         YangInstanceIdentifier peoplePath = PeopleModel.BASE_PATH;
289         NormalizedNode<?, ?> peopleNode = PeopleModel.emptyContainer();
290         writeTx.write(peoplePath, peopleNode);
291
292         followerTestKit.doCommit(writeTx.ready());
293
294         DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
295
296         verifyNode(readTx, carsPath, carsNode);
297         verifyNode(readTx, peoplePath, peopleNode);
298     }
299
300     @Test
301     public void testReadWriteTransactionWithMultipleShards() throws Exception {
302         initDatastoresWithCarsAndPeople("testReadWriteTransactionWithMultipleShards");
303
304         DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
305         assertNotNull("newReadWriteTransaction returned null", rwTx);
306
307         YangInstanceIdentifier carsPath = CarsModel.BASE_PATH;
308         NormalizedNode<?, ?> carsNode = CarsModel.emptyContainer();
309         rwTx.write(carsPath, carsNode);
310
311         YangInstanceIdentifier peoplePath = PeopleModel.BASE_PATH;
312         NormalizedNode<?, ?> peopleNode = PeopleModel.emptyContainer();
313         rwTx.write(peoplePath, peopleNode);
314
315         followerTestKit.doCommit(rwTx.ready());
316
317         DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
318
319         verifyNode(readTx, carsPath, carsNode);
320         verifyNode(readTx, peoplePath, peopleNode);
321     }
322
323     @Test
324     public void testTransactionChainWithSingleShard() throws Exception {
325         initDatastoresWithCars("testTransactionChainWithSingleShard");
326
327         DOMStoreTransactionChain txChain = followerDistributedDataStore.createTransactionChain();
328
329         // Add the top-level cars container with write-only.
330
331         DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
332         assertNotNull("newWriteOnlyTransaction returned null", writeTx);
333
334         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
335
336         writeTx.ready();
337
338         // Verify the top-level cars container with read-only.
339
340         verifyNode(txChain.newReadOnlyTransaction(), CarsModel.BASE_PATH, CarsModel.emptyContainer());
341
342         // Perform car operations with read-write.
343
344         DOMStoreReadWriteTransaction rwTx = txChain.newReadWriteTransaction();
345
346         verifyNode(rwTx, CarsModel.BASE_PATH, CarsModel.emptyContainer());
347
348         rwTx.merge(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
349
350         MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
351         YangInstanceIdentifier car1Path = CarsModel.newCarPath("optima");
352         rwTx.write(car1Path, car1);
353
354         verifyExists(rwTx, car1Path);
355
356         verifyCars(rwTx, car1);
357
358         MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(25000));
359         rwTx.merge(CarsModel.newCarPath("sportage"), car2);
360
361         rwTx.delete(car1Path);
362
363         followerTestKit.doCommit(rwTx.ready());
364
365         txChain.close();
366
367         verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car2);
368     }
369
370     @Test
371     public void testTransactionChainWithMultipleShards() throws Exception{
372         initDatastoresWithCarsAndPeople("testTransactionChainWithMultipleShards");
373
374         DOMStoreTransactionChain txChain = followerDistributedDataStore.createTransactionChain();
375
376         DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
377         assertNotNull("newWriteOnlyTransaction returned null", writeTx);
378
379         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
380         writeTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
381
382         writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
383         writeTx.write(PeopleModel.PERSON_LIST_PATH, PeopleModel.newPersonMapNode());
384
385         followerTestKit.doCommit(writeTx.ready());
386
387         DOMStoreReadWriteTransaction readWriteTx = txChain.newReadWriteTransaction();
388
389         MapEntryNode car = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
390         YangInstanceIdentifier carPath = CarsModel.newCarPath("optima");
391         readWriteTx.write(carPath, car);
392
393         MapEntryNode person = PeopleModel.newPersonEntry("jack");
394         YangInstanceIdentifier personPath = PeopleModel.newPersonPath("jack");
395         readWriteTx.merge(personPath, person);
396
397         Optional<NormalizedNode<?, ?>> optional = readWriteTx.read(carPath).get(5, TimeUnit.SECONDS);
398         assertEquals("isPresent", true, optional.isPresent());
399         assertEquals("Data node", car, optional.get());
400
401         optional = readWriteTx.read(personPath).get(5, TimeUnit.SECONDS);
402         assertEquals("isPresent", true, optional.isPresent());
403         assertEquals("Data node", person, optional.get());
404
405         DOMStoreThreePhaseCommitCohort cohort2 = readWriteTx.ready();
406
407         writeTx = txChain.newWriteOnlyTransaction();
408
409         writeTx.delete(personPath);
410
411         DOMStoreThreePhaseCommitCohort cohort3 = writeTx.ready();
412
413         followerTestKit.doCommit(cohort2);
414         followerTestKit.doCommit(cohort3);
415
416         txChain.close();
417
418         DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
419         verifyCars(readTx, car);
420
421         optional = readTx.read(personPath).get(5, TimeUnit.SECONDS);
422         assertEquals("isPresent", false, optional.isPresent());
423     }
424
425     @Test
426     public void testChainedTransactionFailureWithSingleShard() throws Exception {
427         initDatastoresWithCars("testChainedTransactionFailureWithSingleShard");
428
429         ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
430                 ImmutableMap.<LogicalDatastoreType, DOMStore>builder().put(
431                         LogicalDatastoreType.CONFIGURATION, followerDistributedDataStore).build(),
432                         MoreExecutors.directExecutor());
433
434         TransactionChainListener listener = Mockito.mock(TransactionChainListener.class);
435         DOMTransactionChain txChain = broker.createTransactionChain(listener);
436
437         DOMDataWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
438
439         ContainerNode invalidData = ImmutableContainerNodeBuilder.create().withNodeIdentifier(
440                 new YangInstanceIdentifier.NodeIdentifier(CarsModel.BASE_QNAME)).
441                     withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build();
442
443         writeTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData);
444
445         try {
446             writeTx.submit().checkedGet(5, TimeUnit.SECONDS);
447             fail("Expected TransactionCommitFailedException");
448         } catch (TransactionCommitFailedException e) {
449             // Expected
450         }
451
452         verify(listener, timeout(5000)).onTransactionChainFailed(eq(txChain), eq(writeTx), any(Throwable.class));
453
454         txChain.close();
455         broker.close();
456     }
457
458     @Test
459     public void testChainedTransactionFailureWithMultipleShards() throws Exception {
460         initDatastoresWithCarsAndPeople("testChainedTransactionFailureWithMultipleShards");
461
462         ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
463                 ImmutableMap.<LogicalDatastoreType, DOMStore>builder().put(
464                         LogicalDatastoreType.CONFIGURATION, followerDistributedDataStore).build(),
465                         MoreExecutors.directExecutor());
466
467         TransactionChainListener listener = Mockito.mock(TransactionChainListener.class);
468         DOMTransactionChain txChain = broker.createTransactionChain(listener);
469
470         DOMDataWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
471
472         writeTx.put(LogicalDatastoreType.CONFIGURATION, PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
473
474         ContainerNode invalidData = ImmutableContainerNodeBuilder.create().withNodeIdentifier(
475                 new YangInstanceIdentifier.NodeIdentifier(CarsModel.BASE_QNAME)).
476                     withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build();
477
478         // Note that merge will validate the data and fail but put succeeds b/c deep validation is not
479         // done for put for performance reasons.
480         writeTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData);
481
482         try {
483             writeTx.submit().checkedGet(5, TimeUnit.SECONDS);
484             fail("Expected TransactionCommitFailedException");
485         } catch (TransactionCommitFailedException e) {
486             // Expected
487         }
488
489         verify(listener, timeout(5000)).onTransactionChainFailed(eq(txChain), eq(writeTx), any(Throwable.class));
490
491         txChain.close();
492         broker.close();
493     }
494
495     @Test
496     public void testSingleShardTransactionsWithLeaderChanges() throws Exception {
497         String testName = "testSingleShardTransactionsWithLeaderChanges";
498         initDatastoresWithCars(testName);
499
500         String followerCarShardName = "member-2-shard-cars-" + testName;
501         InMemoryJournal.addWriteMessagesCompleteLatch(followerCarShardName, 1, ApplyJournalEntries.class );
502
503         // Write top-level car container from the follower so it uses a remote Tx.
504
505         DOMStoreWriteTransaction writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
506
507         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
508         writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
509
510         followerTestKit.doCommit(writeTx.ready());
511
512         InMemoryJournal.waitForWriteMessagesComplete(followerCarShardName);
513
514         // Switch the leader to the follower
515
516         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder.
517                 shardElectionTimeoutFactor(1).customRaftPolicyImplementation(null));
518
519         JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
520
521         followerTestKit.waitUntilNoLeader(followerDistributedDataStore.getActorContext(), CARS);
522
523         leaderSystem = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member1"));
524         Cluster.get(leaderSystem).join(MEMBER_2_ADDRESS);
525
526         DatastoreContext.Builder newMember1Builder = DatastoreContext.newBuilder().
527                 shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5);
528         IntegrationTestKit newMember1TestKit = new IntegrationTestKit(leaderSystem, newMember1Builder);
529
530         try (DistributedDataStore ds =
531                 newMember1TestKit.setupDistributedDataStore(testName, MODULE_SHARDS_CARS_ONLY_1_2, false, CARS)) {
532
533             followerTestKit.waitUntilLeader(followerDistributedDataStore.getActorContext(), CARS);
534
535             // Write a car entry to the new leader - should switch to local Tx
536
537             writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
538
539             MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
540             YangInstanceIdentifier car1Path = CarsModel.newCarPath("optima");
541             writeTx.merge(car1Path, car1);
542
543             followerTestKit.doCommit(writeTx.ready());
544
545             verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car1);
546         }
547     }
548
549     @SuppressWarnings("unchecked")
550     @Test
551     public void testReadyLocalTransactionForwardedToLeader() throws Exception {
552         initDatastoresWithCars("testReadyLocalTransactionForwardedToLeader");
553         followerTestKit.waitUntilLeader(followerDistributedDataStore.getActorContext(), "cars");
554
555         Optional<ActorRef> carsFollowerShard = followerDistributedDataStore.getActorContext().findLocalShard("cars");
556         assertEquals("Cars follower shard found", true, carsFollowerShard.isPresent());
557
558         TipProducingDataTree dataTree = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL);
559         dataTree.setSchemaContext(SchemaContextHelper.full());
560
561         // Send a tx with immediate commit.
562
563         DataTreeModification modification = dataTree.takeSnapshot().newModification();
564         new WriteModification(CarsModel.BASE_PATH, CarsModel.emptyContainer()).apply(modification);
565         new MergeModification(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode()).apply(modification);
566
567         MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
568         new WriteModification(CarsModel.newCarPath("optima"), car1).apply(modification);
569         modification.ready();
570
571         ReadyLocalTransaction readyLocal = new ReadyLocalTransaction("tx-1" , modification, true);
572
573         carsFollowerShard.get().tell(readyLocal, followerTestKit.getRef());
574         Object resp = followerTestKit.expectMsgClass(Object.class);
575         if(resp instanceof akka.actor.Status.Failure) {
576             throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause());
577         }
578
579         assertEquals("Response type", CommitTransactionReply.class, resp.getClass());
580
581         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1);
582
583         // Send another tx without immediate commit.
584
585         modification = dataTree.takeSnapshot().newModification();
586         MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(30000));
587         new WriteModification(CarsModel.newCarPath("sportage"), car2).apply(modification);
588         modification.ready();
589
590         readyLocal = new ReadyLocalTransaction("tx-2" , modification, false);
591
592         carsFollowerShard.get().tell(readyLocal, followerTestKit.getRef());
593         resp = followerTestKit.expectMsgClass(Object.class);
594         if(resp instanceof akka.actor.Status.Failure) {
595             throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause());
596         }
597
598         assertEquals("Response type", ReadyTransactionReply.class, resp.getClass());
599
600         ActorSelection txActor = leaderDistributedDataStore.getActorContext().actorSelection(
601                 ((ReadyTransactionReply)resp).getCohortPath());
602
603         Supplier<Short> versionSupplier = Mockito.mock(Supplier.class);
604         Mockito.doReturn(DataStoreVersions.CURRENT_VERSION).when(versionSupplier).get();
605         ThreePhaseCommitCohortProxy cohort = new ThreePhaseCommitCohortProxy(
606                 leaderDistributedDataStore.getActorContext(), Arrays.asList(
607                         new ThreePhaseCommitCohortProxy.CohortInfo(Futures.successful(txActor), versionSupplier)), "tx-2");
608         cohort.canCommit().get(5, TimeUnit.SECONDS);
609         cohort.preCommit().get(5, TimeUnit.SECONDS);
610         cohort.commit().get(5, TimeUnit.SECONDS);
611
612         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1, car2);
613     }
614
615     @SuppressWarnings("unchecked")
616     @Test
617     public void testForwardedReadyTransactionForwardedToLeader() throws Exception {
618         initDatastoresWithCars("testForwardedReadyTransactionForwardedToLeader");
619         followerTestKit.waitUntilLeader(followerDistributedDataStore.getActorContext(), "cars");
620
621         Optional<ActorRef> carsFollowerShard = followerDistributedDataStore.getActorContext().findLocalShard("cars");
622         assertEquals("Cars follower shard found", true, carsFollowerShard.isPresent());
623
624         carsFollowerShard.get().tell(GetShardDataTree.INSTANCE, followerTestKit.getRef());
625         DataTree dataTree = followerTestKit.expectMsgClass(DataTree.class);
626
627         // Send a tx with immediate commit.
628
629         DataTreeModification modification = dataTree.takeSnapshot().newModification();
630         new WriteModification(CarsModel.BASE_PATH, CarsModel.emptyContainer()).apply(modification);
631         new MergeModification(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode()).apply(modification);
632
633         MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
634         new WriteModification(CarsModel.newCarPath("optima"), car1).apply(modification);
635
636         ForwardedReadyTransaction forwardedReady = new ForwardedReadyTransaction("tx-1",
637                 DataStoreVersions.CURRENT_VERSION, new ReadWriteShardDataTreeTransaction(
638                         Mockito.mock(ShardDataTreeTransactionParent.class), "tx-1", modification), true);
639
640         carsFollowerShard.get().tell(forwardedReady, followerTestKit.getRef());
641         Object resp = followerTestKit.expectMsgClass(Object.class);
642         if(resp instanceof akka.actor.Status.Failure) {
643             throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause());
644         }
645
646         assertEquals("Response type", CommitTransactionReply.class, resp.getClass());
647
648         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1);
649
650         // Send another tx without immediate commit.
651
652         modification = dataTree.takeSnapshot().newModification();
653         MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(30000));
654         new WriteModification(CarsModel.newCarPath("sportage"), car2).apply(modification);
655
656         forwardedReady = new ForwardedReadyTransaction("tx-2",
657                 DataStoreVersions.CURRENT_VERSION, new ReadWriteShardDataTreeTransaction(
658                         Mockito.mock(ShardDataTreeTransactionParent.class), "tx-2", modification), false);
659
660         carsFollowerShard.get().tell(forwardedReady, followerTestKit.getRef());
661         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", ReadyTransactionReply.class, resp.getClass());
667
668         ActorSelection txActor = leaderDistributedDataStore.getActorContext().actorSelection(
669                 ((ReadyTransactionReply)resp).getCohortPath());
670
671         Supplier<Short> versionSupplier = Mockito.mock(Supplier.class);
672         Mockito.doReturn(DataStoreVersions.CURRENT_VERSION).when(versionSupplier).get();
673         ThreePhaseCommitCohortProxy cohort = new ThreePhaseCommitCohortProxy(
674                 leaderDistributedDataStore.getActorContext(), Arrays.asList(
675                         new ThreePhaseCommitCohortProxy.CohortInfo(Futures.successful(txActor), versionSupplier)), "tx-2");
676         cohort.canCommit().get(5, TimeUnit.SECONDS);
677         cohort.preCommit().get(5, TimeUnit.SECONDS);
678         cohort.commit().get(5, TimeUnit.SECONDS);
679
680         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1, car2);
681     }
682
683     @Test
684     public void testTransactionForwardedToLeaderAfterRetry() throws Exception {
685         followerDatastoreContextBuilder.shardBatchedModificationCount(2);
686         leaderDatastoreContextBuilder.shardBatchedModificationCount(2);
687         initDatastoresWithCarsAndPeople("testTransactionForwardedToLeaderAfterRetry");
688
689         // Do an initial write to get the primary shard info cached.
690
691         DOMStoreWriteTransaction initialWriteTx = followerDistributedDataStore.newWriteOnlyTransaction();
692         initialWriteTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
693         initialWriteTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
694         followerTestKit.doCommit(initialWriteTx.ready());
695
696         // Wait for the commit to be replicated to the follower.
697
698         MemberNode.verifyRaftState(followerDistributedDataStore, "cars", new RaftStateVerifier() {
699             @Override
700             public void verify(OnDemandRaftState raftState) {
701                 assertEquals("getLastApplied", 0, raftState.getLastApplied());
702             }
703         });
704
705         // Prepare, ready and canCommit a WO tx that writes to 2 shards. This will become the current tx in
706         // the leader shard.
707
708         DOMStoreWriteTransaction writeTx1 = followerDistributedDataStore.newWriteOnlyTransaction();
709         writeTx1.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
710         writeTx1.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
711         DOMStoreThreePhaseCommitCohort writeTx1Cohort = writeTx1.ready();
712         ListenableFuture<Boolean> writeTx1CanCommit = writeTx1Cohort.canCommit();
713         writeTx1CanCommit.get(5, TimeUnit.SECONDS);
714
715         // Prepare and ready another WO tx that writes to 2 shards but don't canCommit yet. This will be queued
716         // in the leader shard.
717
718         DOMStoreWriteTransaction writeTx2 = followerDistributedDataStore.newWriteOnlyTransaction();
719         LinkedList<MapEntryNode> cars = new LinkedList<>();
720         int carIndex = 1;
721         cars.add(CarsModel.newCarEntry("car" + carIndex, BigInteger.valueOf(carIndex)));
722         writeTx2.write(CarsModel.newCarPath("car" + carIndex), cars.getLast());
723         carIndex++;
724         NormalizedNode<?, ?> people = PeopleModel.newPersonMapNode();
725         writeTx2.write(PeopleModel.PERSON_LIST_PATH, people);
726         DOMStoreThreePhaseCommitCohort writeTx2Cohort = writeTx2.ready();
727
728         // Prepare another WO that writes to a single shard and thus will be directly committed on ready. This
729         // tx writes 5 cars so 2 BatchedModidifications messages will be sent initially and cached in the
730         // leader shard (with shardBatchedModificationCount set to 2). The 3rd BatchedModidifications will be
731         // sent on ready.
732
733         DOMStoreWriteTransaction writeTx3 = followerDistributedDataStore.newWriteOnlyTransaction();
734         for(int i = 1; i <= 5; i++, carIndex++) {
735             cars.add(CarsModel.newCarEntry("car" + carIndex, BigInteger.valueOf(carIndex)));
736             writeTx3.write(CarsModel.newCarPath("car" + carIndex), cars.getLast());
737         }
738
739         // Prepare another WO that writes to a single shard. This will send a single BatchedModidifications
740         // message on ready.
741
742         DOMStoreWriteTransaction writeTx4 = followerDistributedDataStore.newWriteOnlyTransaction();
743         cars.add(CarsModel.newCarEntry("car" + carIndex, BigInteger.valueOf(carIndex)));
744         writeTx4.write(CarsModel.newCarPath("car" + carIndex), cars.getLast());
745         carIndex++;
746
747         // Prepare a RW tx that will create a tx actor and send a ForwardedReadyTransaciton message to the
748         // leader shard on ready.
749
750         DOMStoreReadWriteTransaction readWriteTx = followerDistributedDataStore.newReadWriteTransaction();
751         cars.add(CarsModel.newCarEntry("car" + carIndex, BigInteger.valueOf(carIndex)));
752         readWriteTx.write(CarsModel.newCarPath("car" + carIndex), cars.getLast());
753
754         IntegrationTestKit.verifyShardStats(leaderDistributedDataStore, "cars", new ShardStatsVerifier() {
755             @Override
756             public void verify(ShardStats stats) {
757                 assertEquals("getReadWriteTransactionCount", 1, stats.getReadWriteTransactionCount());
758             }
759         });
760
761         // Disable elections on the leader so it switches to follower.
762
763         sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder.
764                 customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName()).
765                 shardElectionTimeoutFactor(10));
766
767         leaderTestKit.waitUntilNoLeader(leaderDistributedDataStore.getActorContext(), "cars");
768
769         // Submit all tx's - the messages should get queued for retry.
770
771         ListenableFuture<Boolean> writeTx2CanCommit = writeTx2Cohort.canCommit();
772         DOMStoreThreePhaseCommitCohort writeTx3Cohort = writeTx3.ready();
773         DOMStoreThreePhaseCommitCohort writeTx4Cohort = writeTx4.ready();
774         DOMStoreThreePhaseCommitCohort rwTxCohort = readWriteTx.ready();
775
776         // Enable elections on the other follower so it becomes the leader, at which point the
777         // tx's should get forwarded from the previous leader to the new leader to complete the commits.
778
779         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder.
780                 customRaftPolicyImplementation(null).shardElectionTimeoutFactor(1));
781
782         followerTestKit.doCommit(writeTx1CanCommit, writeTx1Cohort);
783         followerTestKit.doCommit(writeTx2CanCommit, writeTx2Cohort);
784         followerTestKit.doCommit(writeTx3Cohort);
785         followerTestKit.doCommit(writeTx4Cohort);
786         followerTestKit.doCommit(rwTxCohort);
787
788         DOMStoreReadTransaction readTx = leaderDistributedDataStore.newReadOnlyTransaction();
789         verifyCars(readTx, cars.toArray(new MapEntryNode[cars.size()]));
790         verifyNode(readTx, PeopleModel.PERSON_LIST_PATH, people);
791     }
792
793     @Test
794     public void testLeadershipTransferOnShutdown() throws Exception {
795         leaderDatastoreContextBuilder.shardBatchedModificationCount(1);
796         followerDatastoreContextBuilder.shardElectionTimeoutFactor(10).customRaftPolicyImplementation(null);
797         String testName = "testLeadershipTransferOnShutdown";
798         initDatastores(testName, MODULE_SHARDS_CARS_PEOPLE_1_2_3, CARS_AND_PEOPLE);
799
800         IntegrationTestKit follower2TestKit = new IntegrationTestKit(follower2System, followerDatastoreContextBuilder);
801         try (DistributedDataStore follower2DistributedDataStore = follower2TestKit.setupDistributedDataStore(testName,
802                 MODULE_SHARDS_CARS_PEOPLE_1_2_3, false)) {
803
804             // Create and submit a couple tx's so they're pending.
805
806             DOMStoreWriteTransaction writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
807             writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
808             writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
809             writeTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
810             DOMStoreThreePhaseCommitCohort cohort1 = writeTx.ready();
811
812             IntegrationTestKit.verifyShardStats(leaderDistributedDataStore, "cars", new ShardStatsVerifier() {
813                 @Override
814                 public void verify(ShardStats stats) {
815                     assertEquals("getTxCohortCacheSize", 1, stats.getTxCohortCacheSize());
816                 }
817             });
818
819             writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
820             MapEntryNode car = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
821             writeTx.write(CarsModel.newCarPath("optima"), car);
822             DOMStoreThreePhaseCommitCohort cohort2 = writeTx.ready();
823
824             IntegrationTestKit.verifyShardStats(leaderDistributedDataStore, "cars", new ShardStatsVerifier() {
825                 @Override
826                 public void verify(ShardStats stats) {
827                     assertEquals("getTxCohortCacheSize", 2, stats.getTxCohortCacheSize());
828                 }
829             });
830
831             // Gracefully stop the leader via a Shutdown message.
832
833             sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder.
834                 shardElectionTimeoutFactor(100));
835
836             FiniteDuration duration = FiniteDuration.create(5, TimeUnit.SECONDS);
837             Future<ActorRef> future = leaderDistributedDataStore.getActorContext().findLocalShardAsync("cars");
838             ActorRef leaderActor = Await.result(future, duration);
839
840             Future<Boolean> stopFuture = Patterns.gracefulStop(leaderActor, duration, Shutdown.INSTANCE);
841
842             // Commit the 2 transactions. They should finish and succeed.
843
844             followerTestKit.doCommit(cohort1);
845             followerTestKit.doCommit(cohort2);
846
847             // Wait for the leader actor stopped.
848
849             Boolean stopped = Await.result(stopFuture, duration);
850             assertEquals("Stopped", Boolean.TRUE, stopped);
851
852             // Verify leadership was transferred by reading the committed data from the other nodes.
853
854             verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car);
855             verifyCars(follower2DistributedDataStore.newReadOnlyTransaction(), car);
856         }
857     }
858
859     @Test
860     public void testTransactionWithIsolatedLeader() throws Throwable {
861         leaderDatastoreContextBuilder.shardIsolatedLeaderCheckIntervalInMillis(200);
862         String testName = "testTransactionWithIsolatedLeader";
863         initDatastoresWithCars(testName);
864
865         DOMStoreWriteTransaction failWriteTx = leaderDistributedDataStore.newWriteOnlyTransaction();
866         failWriteTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
867
868         DOMStoreWriteTransaction successWriteTx = leaderDistributedDataStore.newWriteOnlyTransaction();
869         successWriteTx.merge(CarsModel.BASE_PATH, CarsModel.emptyContainer());
870
871         followerTestKit.watch(followerDistributedDataStore.getActorContext().getShardManager());
872         followerDistributedDataStore.close();
873         followerTestKit.expectTerminated(followerDistributedDataStore.getActorContext().getShardManager());
874
875         MemberNode.verifyRaftState(leaderDistributedDataStore, "cars", new RaftStateVerifier() {
876             @Override
877             public void verify(OnDemandRaftState raftState) {
878                 assertEquals("getRaftState", "IsolatedLeader", raftState.getRaftState());
879             }
880         });
881
882         try {
883             leaderTestKit.doCommit(failWriteTx.ready());
884             fail("Expected NoShardLeaderException");
885         } catch (ExecutionException e) {
886             assertEquals("getCause", NoShardLeaderException.class, e.getCause().getClass());
887         }
888
889         sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder.
890                 shardElectionTimeoutFactor(100));
891
892         DOMStoreThreePhaseCommitCohort writeTxCohort = successWriteTx.ready();
893
894         followerDistributedDataStore = followerTestKit.setupDistributedDataStore(testName,
895                 MODULE_SHARDS_CARS_ONLY_1_2, false, CARS);
896
897         leaderTestKit.doCommit(writeTxCohort);
898     }
899
900     @Test(expected=AskTimeoutException.class)
901     public void testTransactionWithShardLeaderNotResponding() throws Throwable {
902         initDatastoresWithCars("testTransactionWithShardLeaderNotResponding");
903
904         // Do an initial read to get the primary shard info cached.
905
906         DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
907         readTx.read(CarsModel.BASE_PATH).checkedGet(5, TimeUnit.SECONDS);
908
909         // Shutdown the leader and try to create a new tx.
910
911         JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
912
913         followerDatastoreContextBuilder.operationTimeoutInMillis(50).shardElectionTimeoutFactor(1);
914         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder);
915
916         DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
917
918         rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
919
920         try {
921             followerTestKit.doCommit(rwTx.ready());
922         } catch (ExecutionException e) {
923             assertTrue("Expected ShardLeaderNotRespondingException cause. Actual: " + e.getCause(),
924                     e.getCause() instanceof ShardLeaderNotRespondingException);
925             assertNotNull("Expected a nested cause", e.getCause().getCause());
926             throw e.getCause().getCause();
927         }
928     }
929
930     @Test(expected=NoShardLeaderException.class)
931     public void testTransactionWithCreateTxFailureDueToNoLeader() throws Throwable {
932         initDatastoresWithCars("testTransactionWithCreateTxFailureDueToNoLeader");
933
934         // Do an initial read to get the primary shard info cached.
935
936         DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
937         readTx.read(CarsModel.BASE_PATH).checkedGet(5, TimeUnit.SECONDS);
938
939         // Shutdown the leader and try to create a new tx.
940
941         JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
942
943         Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
944
945         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder.
946                 operationTimeoutInMillis(10).shardElectionTimeoutFactor(1).customRaftPolicyImplementation(null));
947
948         DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
949
950         rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
951
952         try {
953             followerTestKit.doCommit(rwTx.ready());
954         } catch (ExecutionException e) {
955             throw e.getCause();
956         }
957     }
958
959     @Test
960     public void testTransactionRetryWithInitialAskTimeoutExOnCreateTx() throws Exception {
961         String testName = "testTransactionRetryWithInitialAskTimeoutExOnCreateTx";
962         initDatastores(testName, MODULE_SHARDS_CARS_PEOPLE_1_2_3, CARS);
963
964         DatastoreContext.Builder follower2DatastoreContextBuilder = DatastoreContext.newBuilder().
965                 shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5);
966         IntegrationTestKit follower2TestKit = new IntegrationTestKit(follower2System, follower2DatastoreContextBuilder);
967
968         try (DistributedDataStore ds =
969                 follower2TestKit.setupDistributedDataStore(testName, MODULE_SHARDS_CARS_PEOPLE_1_2_3, false, CARS)) {
970
971             followerTestKit.waitForMembersUp("member-1", "member-3");
972             follower2TestKit.waitForMembersUp("member-1", "member-2");
973
974             // Do an initial read to get the primary shard info cached.
975
976             DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
977             readTx.read(CarsModel.BASE_PATH).checkedGet(5, TimeUnit.SECONDS);
978
979             // Shutdown the leader and try to create a new tx.
980
981             JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
982
983             sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder.
984                 operationTimeoutInMillis(500).shardElectionTimeoutFactor(1).customRaftPolicyImplementation(null));
985
986             DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
987
988             rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
989
990             followerTestKit.doCommit(rwTx.ready());
991         }
992     }
993
994     private static void sendDatastoreContextUpdate(DistributedDataStore dataStore, final Builder builder) {
995         final Builder newBuilder = DatastoreContext.newBuilderFrom(builder.build());
996         DatastoreContextFactory mockContextFactory = Mockito.mock(DatastoreContextFactory.class);
997         Answer<DatastoreContext> answer = new Answer<DatastoreContext>() {
998             @Override
999             public DatastoreContext answer(InvocationOnMock invocation) {
1000                 return newBuilder.build();
1001             }
1002         };
1003         Mockito.doAnswer(answer).when(mockContextFactory).getBaseDatastoreContext();
1004         Mockito.doAnswer(answer).when(mockContextFactory).getShardDatastoreContext(Mockito.anyString());
1005         dataStore.onDatastoreContextUpdated(mockContextFactory);
1006     }
1007 }