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