Handle leader change on ForwardedReadyTransaction in Shard
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / DistributedDataStoreRemotingIntegrationTest.java
1 /*
2  * Copyright (c) 2015 Brocade Communications Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.controller.cluster.datastore;
9
10 import static org.junit.Assert.assertEquals;
11 import static org.junit.Assert.assertNotNull;
12 import static org.junit.Assert.assertTrue;
13 import static org.junit.Assert.fail;
14 import static org.mockito.Matchers.any;
15 import static org.mockito.Matchers.eq;
16 import static org.mockito.Mockito.timeout;
17 import static org.mockito.Mockito.verify;
18 import akka.actor.ActorRef;
19 import akka.actor.ActorSelection;
20 import akka.actor.ActorSystem;
21 import akka.actor.Address;
22 import akka.actor.AddressFromURIString;
23 import akka.cluster.Cluster;
24 import akka.dispatch.Futures;
25 import akka.pattern.AskTimeoutException;
26 import akka.testkit.JavaTestKit;
27 import com.google.common.base.Optional;
28 import com.google.common.collect.ImmutableMap;
29 import com.google.common.util.concurrent.MoreExecutors;
30 import com.google.common.util.concurrent.Uninterruptibles;
31 import com.typesafe.config.ConfigFactory;
32 import java.math.BigInteger;
33 import java.util.Arrays;
34 import java.util.concurrent.ExecutionException;
35 import java.util.concurrent.TimeUnit;
36 import org.junit.After;
37 import org.junit.Before;
38 import org.junit.Test;
39 import org.mockito.Mockito;
40 import org.mockito.invocation.InvocationOnMock;
41 import org.mockito.stubbing.Answer;
42 import org.opendaylight.controller.cluster.datastore.DatastoreContext.Builder;
43 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
44 import org.opendaylight.controller.cluster.datastore.exceptions.ShardLeaderNotRespondingException;
45 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
46 import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction;
47 import org.opendaylight.controller.cluster.datastore.messages.GetShardDataTree;
48 import org.opendaylight.controller.cluster.datastore.messages.ReadyLocalTransaction;
49 import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
50 import org.opendaylight.controller.cluster.datastore.modification.MergeModification;
51 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
52 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
53 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
54 import org.opendaylight.controller.md.cluster.datastore.model.CarsModel;
55 import org.opendaylight.controller.md.cluster.datastore.model.PeopleModel;
56 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
57 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
58 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
59 import org.opendaylight.controller.md.sal.common.api.data.TransactionChainListener;
60 import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
61 import org.opendaylight.controller.md.sal.dom.api.DOMDataWriteTransaction;
62 import org.opendaylight.controller.md.sal.dom.api.DOMTransactionChain;
63 import org.opendaylight.controller.sal.core.spi.data.DOMStore;
64 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
65 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
66 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
67 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
68 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
69 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
70 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
71 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
72 import org.opendaylight.yangtools.yang.data.api.schema.MapNode;
73 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
74 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
75 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification;
76 import org.opendaylight.yangtools.yang.data.api.schema.tree.TipProducingDataTree;
77 import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType;
78 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
79 import org.opendaylight.yangtools.yang.data.impl.schema.builder.api.CollectionNodeBuilder;
80 import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder;
81 import org.opendaylight.yangtools.yang.data.impl.schema.tree.InMemoryDataTreeFactory;
82
83 /**
84  * End-to-end distributed data store tests that exercise remote shards and transactions.
85  *
86  * @author Thomas Pantelis
87  */
88 public class DistributedDataStoreRemotingIntegrationTest {
89
90     private static final String[] SHARD_NAMES = {"cars", "people"};
91
92     private static final Address MEMBER_1_ADDRESS = AddressFromURIString.parse("akka.tcp://cluster-test@127.0.0.1:2558");
93     private static final Address MEMBER_2_ADDRESS = AddressFromURIString.parse("akka.tcp://cluster-test@127.0.0.1:2559");
94
95     private static final String MODULE_SHARDS_CONFIG_2 = "module-shards-member1-and-2.conf";
96     private static final String MODULE_SHARDS_CONFIG_3 = "module-shards-member1-and-2-and-3.conf";
97
98     private ActorSystem leaderSystem;
99     private ActorSystem followerSystem;
100     private ActorSystem follower2System;
101
102     private final DatastoreContext.Builder leaderDatastoreContextBuilder =
103             DatastoreContext.newBuilder().shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(1);
104
105     private final DatastoreContext.Builder followerDatastoreContextBuilder =
106             DatastoreContext.newBuilder().shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5);
107
108     private DistributedDataStore followerDistributedDataStore;
109     private DistributedDataStore leaderDistributedDataStore;
110     private IntegrationTestKit followerTestKit;
111     private IntegrationTestKit leaderTestKit;
112
113     @Before
114     public void setUp() {
115         leaderSystem = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member1"));
116         Cluster.get(leaderSystem).join(MEMBER_1_ADDRESS);
117
118         followerSystem = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member2"));
119         Cluster.get(followerSystem).join(MEMBER_1_ADDRESS);
120
121         follower2System = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member3"));
122         Cluster.get(follower2System).join(MEMBER_1_ADDRESS);
123     }
124
125     @After
126     public void tearDown() {
127         JavaTestKit.shutdownActorSystem(leaderSystem);
128         JavaTestKit.shutdownActorSystem(followerSystem);
129         JavaTestKit.shutdownActorSystem(follower2System);
130     }
131
132     private void initDatastores(String type) {
133         initDatastores(type, MODULE_SHARDS_CONFIG_2);
134     }
135
136     private void initDatastores(String type, String moduleShardsConfig) {
137         leaderTestKit = new IntegrationTestKit(leaderSystem, leaderDatastoreContextBuilder);
138
139         leaderDistributedDataStore = leaderTestKit.setupDistributedDataStore(type, moduleShardsConfig, false, SHARD_NAMES);
140
141         followerTestKit = new IntegrationTestKit(followerSystem, followerDatastoreContextBuilder);
142         followerDistributedDataStore = followerTestKit.setupDistributedDataStore(type, moduleShardsConfig, false, SHARD_NAMES);
143
144         leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(), SHARD_NAMES);
145     }
146
147     private static void verifyCars(DOMStoreReadTransaction readTx, MapEntryNode... entries) throws Exception {
148         Optional<NormalizedNode<?, ?>> optional = readTx.read(CarsModel.CAR_LIST_PATH).get(5, TimeUnit.SECONDS);
149         assertEquals("isPresent", true, optional.isPresent());
150
151         CollectionNodeBuilder<MapEntryNode, MapNode> listBuilder = ImmutableNodes.mapNodeBuilder(CarsModel.CAR_QNAME);
152         for(NormalizedNode<?, ?> entry: entries) {
153             listBuilder.withChild((MapEntryNode) entry);
154         }
155
156         assertEquals("Car list node", listBuilder.build(), optional.get());
157     }
158
159     private static void verifyNode(DOMStoreReadTransaction readTx, YangInstanceIdentifier path, NormalizedNode<?, ?> expNode)
160             throws Exception {
161         Optional<NormalizedNode<?, ?>> optional = readTx.read(path).get(5, TimeUnit.SECONDS);
162         assertEquals("isPresent", true, optional.isPresent());
163         assertEquals("Data node", expNode, optional.get());
164     }
165
166     private static void verifyExists(DOMStoreReadTransaction readTx, YangInstanceIdentifier path) throws Exception {
167         Boolean exists = readTx.exists(path).get(5, TimeUnit.SECONDS);
168         assertEquals("exists", true, exists);
169     }
170
171     @Test
172     public void testWriteTransactionWithSingleShard() throws Exception {
173         String testName = "testWriteTransactionWithSingleShard";
174         initDatastores(testName);
175
176         String followerCarShardName = "member-2-shard-cars-" + testName;
177         InMemoryJournal.addWriteMessagesCompleteLatch(followerCarShardName, 2, ApplyJournalEntries.class );
178
179         DOMStoreWriteTransaction writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
180         assertNotNull("newWriteOnlyTransaction returned null", writeTx);
181
182         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
183         writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
184
185         MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
186         YangInstanceIdentifier car1Path = CarsModel.newCarPath("optima");
187         writeTx.merge(car1Path, car1);
188
189         MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(25000));
190         YangInstanceIdentifier car2Path = CarsModel.newCarPath("sportage");
191         writeTx.merge(car2Path, car2);
192
193         followerTestKit.doCommit(writeTx.ready());
194
195         verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car1, car2);
196
197         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1, car2);
198
199         // Test delete
200
201         writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
202
203         writeTx.delete(car1Path);
204
205         followerTestKit.doCommit(writeTx.ready());
206
207         verifyExists(followerDistributedDataStore.newReadOnlyTransaction(), car2Path);
208
209         verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car2);
210
211         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car2);
212
213         // Re-instate the follower member 2 as a single-node to verify replication and recovery.
214
215         InMemoryJournal.waitForWriteMessagesComplete(followerCarShardName);
216
217         JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
218         JavaTestKit.shutdownActorSystem(followerSystem, null, true);
219
220         ActorSystem newSystem = ActorSystem.create("reinstated-member2", ConfigFactory.load().getConfig("Member2"));
221
222         DistributedDataStore member2Datastore = new IntegrationTestKit(newSystem, leaderDatastoreContextBuilder).
223                 setupDistributedDataStore(testName, "module-shards-member2", true, SHARD_NAMES);
224
225         verifyCars(member2Datastore.newReadOnlyTransaction(), car2);
226
227         JavaTestKit.shutdownActorSystem(newSystem);
228     }
229
230     @Test
231     public void testReadWriteTransactionWithSingleShard() throws Exception {
232         initDatastores("testReadWriteTransactionWithSingleShard");
233
234         DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
235         assertNotNull("newReadWriteTransaction returned null", rwTx);
236
237         rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
238         rwTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
239
240         MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
241         rwTx.merge(CarsModel.newCarPath("optima"), car1);
242
243         verifyCars(rwTx, car1);
244
245         MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(25000));
246         YangInstanceIdentifier car2Path = CarsModel.newCarPath("sportage");
247         rwTx.merge(car2Path, car2);
248
249         verifyExists(rwTx, car2Path);
250
251         followerTestKit.doCommit(rwTx.ready());
252
253         verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car1, car2);
254     }
255
256     @Test
257     public void testWriteTransactionWithMultipleShards() throws Exception {
258         initDatastores("testWriteTransactionWithMultipleShards");
259
260         DOMStoreWriteTransaction writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
261         assertNotNull("newWriteOnlyTransaction returned null", writeTx);
262
263         YangInstanceIdentifier carsPath = CarsModel.BASE_PATH;
264         NormalizedNode<?, ?> carsNode = CarsModel.emptyContainer();
265         writeTx.write(carsPath, carsNode);
266
267         YangInstanceIdentifier peoplePath = PeopleModel.BASE_PATH;
268         NormalizedNode<?, ?> peopleNode = PeopleModel.emptyContainer();
269         writeTx.write(peoplePath, peopleNode);
270
271         followerTestKit.doCommit(writeTx.ready());
272
273         DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
274
275         verifyNode(readTx, carsPath, carsNode);
276         verifyNode(readTx, peoplePath, peopleNode);
277     }
278
279     @Test
280     public void testReadWriteTransactionWithMultipleShards() throws Exception {
281         initDatastores("testReadWriteTransactionWithMultipleShards");
282
283         DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
284         assertNotNull("newReadWriteTransaction returned null", rwTx);
285
286         YangInstanceIdentifier carsPath = CarsModel.BASE_PATH;
287         NormalizedNode<?, ?> carsNode = CarsModel.emptyContainer();
288         rwTx.write(carsPath, carsNode);
289
290         YangInstanceIdentifier peoplePath = PeopleModel.BASE_PATH;
291         NormalizedNode<?, ?> peopleNode = PeopleModel.emptyContainer();
292         rwTx.write(peoplePath, peopleNode);
293
294         followerTestKit.doCommit(rwTx.ready());
295
296         DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
297
298         verifyNode(readTx, carsPath, carsNode);
299         verifyNode(readTx, peoplePath, peopleNode);
300     }
301
302     @Test
303     public void testTransactionChainWithSingleShard() throws Exception {
304         initDatastores("testTransactionChainWithSingleShard");
305
306         DOMStoreTransactionChain txChain = followerDistributedDataStore.createTransactionChain();
307
308         // Add the top-level cars container with write-only.
309
310         DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
311         assertNotNull("newWriteOnlyTransaction returned null", writeTx);
312
313         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
314
315         writeTx.ready();
316
317         // Verify the top-level cars container with read-only.
318
319         verifyNode(txChain.newReadOnlyTransaction(), CarsModel.BASE_PATH, CarsModel.emptyContainer());
320
321         // Perform car operations with read-write.
322
323         DOMStoreReadWriteTransaction rwTx = txChain.newReadWriteTransaction();
324
325         verifyNode(rwTx, CarsModel.BASE_PATH, CarsModel.emptyContainer());
326
327         rwTx.merge(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
328
329         MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
330         YangInstanceIdentifier car1Path = CarsModel.newCarPath("optima");
331         rwTx.write(car1Path, car1);
332
333         verifyExists(rwTx, car1Path);
334
335         verifyCars(rwTx, car1);
336
337         MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(25000));
338         rwTx.merge(CarsModel.newCarPath("sportage"), car2);
339
340         rwTx.delete(car1Path);
341
342         followerTestKit.doCommit(rwTx.ready());
343
344         txChain.close();
345
346         verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car2);
347     }
348
349     @Test
350     public void testTransactionChainWithMultipleShards() throws Exception{
351         initDatastores("testTransactionChainWithMultipleShards");
352
353         DOMStoreTransactionChain txChain = followerDistributedDataStore.createTransactionChain();
354
355         DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
356         assertNotNull("newWriteOnlyTransaction returned null", writeTx);
357
358         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
359         writeTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
360
361         writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
362         writeTx.write(PeopleModel.PERSON_LIST_PATH, PeopleModel.newPersonMapNode());
363
364         followerTestKit.doCommit(writeTx.ready());
365
366         DOMStoreReadWriteTransaction readWriteTx = txChain.newReadWriteTransaction();
367
368         MapEntryNode car = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
369         YangInstanceIdentifier carPath = CarsModel.newCarPath("optima");
370         readWriteTx.write(carPath, car);
371
372         MapEntryNode person = PeopleModel.newPersonEntry("jack");
373         YangInstanceIdentifier personPath = PeopleModel.newPersonPath("jack");
374         readWriteTx.merge(personPath, person);
375
376         Optional<NormalizedNode<?, ?>> optional = readWriteTx.read(carPath).get(5, TimeUnit.SECONDS);
377         assertEquals("isPresent", true, optional.isPresent());
378         assertEquals("Data node", car, optional.get());
379
380         optional = readWriteTx.read(personPath).get(5, TimeUnit.SECONDS);
381         assertEquals("isPresent", true, optional.isPresent());
382         assertEquals("Data node", person, optional.get());
383
384         DOMStoreThreePhaseCommitCohort cohort2 = readWriteTx.ready();
385
386         writeTx = txChain.newWriteOnlyTransaction();
387
388         writeTx.delete(personPath);
389
390         DOMStoreThreePhaseCommitCohort cohort3 = writeTx.ready();
391
392         followerTestKit.doCommit(cohort2);
393         followerTestKit.doCommit(cohort3);
394
395         txChain.close();
396
397         DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
398         verifyCars(readTx, car);
399
400         optional = readTx.read(personPath).get(5, TimeUnit.SECONDS);
401         assertEquals("isPresent", false, optional.isPresent());
402     }
403
404     @Test
405     public void testChainedTransactionFailureWithSingleShard() throws Exception {
406         initDatastores("testChainedTransactionFailureWithSingleShard");
407
408         ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
409                 ImmutableMap.<LogicalDatastoreType, DOMStore>builder().put(
410                         LogicalDatastoreType.CONFIGURATION, followerDistributedDataStore).build(),
411                         MoreExecutors.directExecutor());
412
413         TransactionChainListener listener = Mockito.mock(TransactionChainListener.class);
414         DOMTransactionChain txChain = broker.createTransactionChain(listener);
415
416         DOMDataWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
417
418         ContainerNode invalidData = ImmutableContainerNodeBuilder.create().withNodeIdentifier(
419                 new YangInstanceIdentifier.NodeIdentifier(CarsModel.BASE_QNAME)).
420                     withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build();
421
422         writeTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData);
423
424         try {
425             writeTx.submit().checkedGet(5, TimeUnit.SECONDS);
426             fail("Expected TransactionCommitFailedException");
427         } catch (TransactionCommitFailedException e) {
428             // Expected
429         }
430
431         verify(listener, timeout(5000)).onTransactionChainFailed(eq(txChain), eq(writeTx), any(Throwable.class));
432
433         txChain.close();
434         broker.close();
435     }
436
437     @Test
438     public void testChainedTransactionFailureWithMultipleShards() throws Exception {
439         initDatastores("testChainedTransactionFailureWithMultipleShards");
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         writeTx.put(LogicalDatastoreType.CONFIGURATION, PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
452
453         ContainerNode invalidData = ImmutableContainerNodeBuilder.create().withNodeIdentifier(
454                 new YangInstanceIdentifier.NodeIdentifier(CarsModel.BASE_QNAME)).
455                     withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build();
456
457         // Note that merge will validate the data and fail but put succeeds b/c deep validation is not
458         // done for put for performance reasons.
459         writeTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData);
460
461         try {
462             writeTx.submit().checkedGet(5, TimeUnit.SECONDS);
463             fail("Expected TransactionCommitFailedException");
464         } catch (TransactionCommitFailedException e) {
465             // Expected
466         }
467
468         verify(listener, timeout(5000)).onTransactionChainFailed(eq(txChain), eq(writeTx), any(Throwable.class));
469
470         txChain.close();
471         broker.close();
472     }
473
474     @Test
475     public void testSingleShardTransactionsWithLeaderChanges() throws Exception {
476         String testName = "testSingleShardTransactionsWithLeaderChanges";
477         initDatastores(testName);
478
479         String followerCarShardName = "member-2-shard-cars-" + testName;
480         InMemoryJournal.addWriteMessagesCompleteLatch(followerCarShardName, 1, ApplyJournalEntries.class );
481
482         // Write top-level car container from the follower so it uses a remote Tx.
483
484         DOMStoreWriteTransaction writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
485
486         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
487         writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
488
489         followerTestKit.doCommit(writeTx.ready());
490
491         InMemoryJournal.waitForWriteMessagesComplete(followerCarShardName);
492
493         // Switch the leader to the follower
494
495         followerDatastoreContextBuilder.shardElectionTimeoutFactor(1);
496         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder);
497
498         JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
499
500         followerTestKit.waitUntilNoLeader(followerDistributedDataStore.getActorContext(), SHARD_NAMES);
501
502         leaderSystem = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member1"));
503         Cluster.get(leaderSystem).join(MEMBER_2_ADDRESS);
504
505         DatastoreContext.Builder newMember1Builder = DatastoreContext.newBuilder().
506                 shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5);
507         IntegrationTestKit newMember1TestKit = new IntegrationTestKit(leaderSystem, newMember1Builder);
508         newMember1TestKit.setupDistributedDataStore(testName, MODULE_SHARDS_CONFIG_2, false, SHARD_NAMES);
509
510         followerTestKit.waitUntilLeader(followerDistributedDataStore.getActorContext(), SHARD_NAMES);
511
512         // Write a car entry to the new leader - should switch to local Tx
513
514         writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
515
516         MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
517         YangInstanceIdentifier car1Path = CarsModel.newCarPath("optima");
518         writeTx.merge(car1Path, car1);
519
520         followerTestKit.doCommit(writeTx.ready());
521
522         verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car1);
523     }
524
525     @Test
526     public void testReadyLocalTransactionForwardedToLeader() throws Exception {
527         initDatastores("testReadyLocalTransactionForwardedToLeader");
528         followerTestKit.waitUntilLeader(followerDistributedDataStore.getActorContext(), "cars");
529
530         Optional<ActorRef> carsFollowerShard = followerDistributedDataStore.getActorContext().findLocalShard("cars");
531         assertEquals("Cars follower shard found", true, carsFollowerShard.isPresent());
532
533         TipProducingDataTree dataTree = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL);
534         dataTree.setSchemaContext(SchemaContextHelper.full());
535
536         // Send a tx with immediate commit.
537
538         DataTreeModification modification = dataTree.takeSnapshot().newModification();
539         new WriteModification(CarsModel.BASE_PATH, CarsModel.emptyContainer()).apply(modification);
540         new MergeModification(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode()).apply(modification);
541
542         MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
543         new WriteModification(CarsModel.newCarPath("optima"), car1).apply(modification);
544         modification.ready();
545
546         ReadyLocalTransaction readyLocal = new ReadyLocalTransaction("tx-1" , modification, true);
547
548         carsFollowerShard.get().tell(readyLocal, followerTestKit.getRef());
549         Object resp = followerTestKit.expectMsgClass(Object.class);
550         if(resp instanceof akka.actor.Status.Failure) {
551             throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause());
552         }
553
554         assertEquals("Response type", CommitTransactionReply.SERIALIZABLE_CLASS, resp.getClass());
555
556         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1);
557
558         // Send another tx without immediate commit.
559
560         modification = dataTree.takeSnapshot().newModification();
561         MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(30000));
562         new WriteModification(CarsModel.newCarPath("sportage"), car2).apply(modification);
563         modification.ready();
564
565         readyLocal = new ReadyLocalTransaction("tx-2" , modification, false);
566
567         carsFollowerShard.get().tell(readyLocal, followerTestKit.getRef());
568         resp = followerTestKit.expectMsgClass(Object.class);
569         if(resp instanceof akka.actor.Status.Failure) {
570             throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause());
571         }
572
573         assertEquals("Response type", ReadyTransactionReply.class, resp.getClass());
574
575         ActorSelection txActor = leaderDistributedDataStore.getActorContext().actorSelection(
576                 ((ReadyTransactionReply)resp).getCohortPath());
577
578         ThreePhaseCommitCohortProxy cohort = new ThreePhaseCommitCohortProxy(
579                 leaderDistributedDataStore.getActorContext(), Arrays.asList(Futures.successful(txActor)), "tx-2");
580         cohort.canCommit().get(5, TimeUnit.SECONDS);
581         cohort.preCommit().get(5, TimeUnit.SECONDS);
582         cohort.commit().get(5, TimeUnit.SECONDS);
583
584         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1, car2);
585     }
586
587     @Test
588     public void testForwardedReadyTransactionForwardedToLeader() throws Exception {
589         initDatastores("testForwardedReadyTransactionForwardedToLeader");
590         followerTestKit.waitUntilLeader(followerDistributedDataStore.getActorContext(), "cars");
591
592         Optional<ActorRef> carsFollowerShard = followerDistributedDataStore.getActorContext().findLocalShard("cars");
593         assertEquals("Cars follower shard found", true, carsFollowerShard.isPresent());
594
595         carsFollowerShard.get().tell(GetShardDataTree.INSTANCE, followerTestKit.getRef());
596         DataTree dataTree = followerTestKit.expectMsgClass(DataTree.class);
597
598         // Send a tx with immediate commit.
599
600         DataTreeModification modification = dataTree.takeSnapshot().newModification();
601         new WriteModification(CarsModel.BASE_PATH, CarsModel.emptyContainer()).apply(modification);
602         new MergeModification(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode()).apply(modification);
603
604         MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
605         new WriteModification(CarsModel.newCarPath("optima"), car1).apply(modification);
606
607         ForwardedReadyTransaction forwardedReady = new ForwardedReadyTransaction("tx-1",
608                 DataStoreVersions.CURRENT_VERSION, new ReadWriteShardDataTreeTransaction(
609                         Mockito.mock(ShardDataTreeTransactionParent.class), "tx-1", modification), true, true);
610
611         carsFollowerShard.get().tell(forwardedReady, followerTestKit.getRef());
612         Object resp = followerTestKit.expectMsgClass(Object.class);
613         if(resp instanceof akka.actor.Status.Failure) {
614             throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause());
615         }
616
617         assertEquals("Response type", CommitTransactionReply.SERIALIZABLE_CLASS, resp.getClass());
618
619         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1);
620
621         // Send another tx without immediate commit.
622
623         modification = dataTree.takeSnapshot().newModification();
624         MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(30000));
625         new WriteModification(CarsModel.newCarPath("sportage"), car2).apply(modification);
626
627         forwardedReady = new ForwardedReadyTransaction("tx-2",
628                 DataStoreVersions.CURRENT_VERSION, new ReadWriteShardDataTreeTransaction(
629                         Mockito.mock(ShardDataTreeTransactionParent.class), "tx-2", modification), true, false);
630
631         carsFollowerShard.get().tell(forwardedReady, followerTestKit.getRef());
632         resp = followerTestKit.expectMsgClass(Object.class);
633         if(resp instanceof akka.actor.Status.Failure) {
634             throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause());
635         }
636
637         assertEquals("Response type", ReadyTransactionReply.class, resp.getClass());
638
639         ActorSelection txActor = leaderDistributedDataStore.getActorContext().actorSelection(
640                 ((ReadyTransactionReply)resp).getCohortPath());
641
642         ThreePhaseCommitCohortProxy cohort = new ThreePhaseCommitCohortProxy(
643                 leaderDistributedDataStore.getActorContext(), Arrays.asList(Futures.successful(txActor)), "tx-2");
644         cohort.canCommit().get(5, TimeUnit.SECONDS);
645         cohort.preCommit().get(5, TimeUnit.SECONDS);
646         cohort.commit().get(5, TimeUnit.SECONDS);
647
648         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1, car2);
649     }
650
651     @Test(expected=NoShardLeaderException.class)
652     public void testTransactionWithIsolatedLeader() throws Throwable {
653         leaderDatastoreContextBuilder.shardIsolatedLeaderCheckIntervalInMillis(300);
654         String testName = "testTransactionWithIsolatedLeader";
655         initDatastores(testName);
656
657         JavaTestKit.shutdownActorSystem(followerSystem, null, true);
658
659         Uninterruptibles.sleepUninterruptibly(leaderDistributedDataStore.getActorContext().getDatastoreContext()
660                 .getShardRaftConfig().getElectionTimeOutInterval().toMillis() * 3, TimeUnit.MILLISECONDS);
661
662         DOMStoreWriteTransaction writeTx = leaderDistributedDataStore.newWriteOnlyTransaction();
663         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
664
665         try {
666             followerTestKit.doCommit(writeTx.ready());
667         } catch (ExecutionException e) {
668             throw e.getCause();
669         }
670     }
671
672     @Test(expected=AskTimeoutException.class)
673     public void testTransactionWithShardLeaderNotResponding() throws Throwable {
674         followerDatastoreContextBuilder.shardElectionTimeoutFactor(30);
675         initDatastores("testTransactionWithShardLeaderNotResponding");
676
677         // Do an initial read to get the primary shard info cached.
678
679         DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
680         readTx.read(CarsModel.BASE_PATH).checkedGet(5, TimeUnit.SECONDS);
681
682         // Shutdown the leader and try to create a new tx.
683
684         JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
685
686         followerDatastoreContextBuilder.operationTimeoutInMillis(50).shardElectionTimeoutFactor(1);
687         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder);
688
689         DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
690
691         rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
692
693         try {
694             followerTestKit.doCommit(rwTx.ready());
695         } catch (ExecutionException e) {
696             assertTrue("Expected ShardLeaderNotRespondingException cause. Actual: " + e.getCause(),
697                     e.getCause() instanceof ShardLeaderNotRespondingException);
698             assertNotNull("Expected a nested cause", e.getCause().getCause());
699             throw e.getCause().getCause();
700         }
701     }
702
703     @Test(expected=NoShardLeaderException.class)
704     public void testTransactionWithCreateTxFailureDueToNoLeader() throws Throwable {
705         initDatastores("testTransactionWithCreateTxFailureDueToNoLeader");
706
707         // Do an initial read to get the primary shard info cached.
708
709         DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
710         readTx.read(CarsModel.BASE_PATH).checkedGet(5, TimeUnit.SECONDS);
711
712         // Shutdown the leader and try to create a new tx.
713
714         JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
715
716         Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
717
718         followerDatastoreContextBuilder.operationTimeoutInMillis(10).shardElectionTimeoutFactor(1);
719         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder);
720
721         DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
722
723         rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
724
725         try {
726             followerTestKit.doCommit(rwTx.ready());
727         } catch (ExecutionException e) {
728             throw e.getCause();
729         }
730     }
731
732     @Test
733     public void testTransactionRetryWithInitialAskTimeoutExOnCreateTx() throws Exception {
734         followerDatastoreContextBuilder.shardElectionTimeoutFactor(30);
735         String testName = "testTransactionRetryWithInitialAskTimeoutExOnCreateTx";
736         initDatastores(testName, MODULE_SHARDS_CONFIG_3);
737
738         DatastoreContext.Builder follower2DatastoreContextBuilder = DatastoreContext.newBuilder().
739                 shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5);
740         IntegrationTestKit follower2TestKit = new IntegrationTestKit(follower2System, follower2DatastoreContextBuilder);
741         follower2TestKit.setupDistributedDataStore(testName, MODULE_SHARDS_CONFIG_3, false, SHARD_NAMES);
742
743         // Do an initial read to get the primary shard info cached.
744
745         DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
746         readTx.read(CarsModel.BASE_PATH).checkedGet(5, TimeUnit.SECONDS);
747
748         // Shutdown the leader and try to create a new tx.
749
750         JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
751
752         followerDatastoreContextBuilder.operationTimeoutInMillis(500);
753         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder);
754
755         DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
756
757         rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
758
759         followerTestKit.doCommit(rwTx.ready());
760     }
761
762     private static void sendDatastoreContextUpdate(DistributedDataStore dataStore, final Builder builder) {
763         DatastoreContextFactory mockContextFactory = Mockito.mock(DatastoreContextFactory.class);
764         Answer<DatastoreContext> answer = new Answer<DatastoreContext>() {
765             @Override
766             public DatastoreContext answer(InvocationOnMock invocation) {
767                 return builder.build();
768             }
769         };
770         Mockito.doAnswer(answer).when(mockContextFactory).getBaseDatastoreContext();
771         Mockito.doAnswer(answer).when(mockContextFactory).getShardDatastoreContext(Mockito.anyString());
772         dataStore.onDatastoreContextUpdated(mockContextFactory);
773     }
774 }