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