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