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