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