2bd389eda32a07c82a9b5a7818cf9db8749ba3cf
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / DistributedDataStoreRemotingIntegrationTest.java
1 /*
2  * Copyright (c) 2015, 2017 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.Patterns;
27 import akka.testkit.JavaTestKit;
28 import com.google.common.base.Optional;
29 import com.google.common.base.Stopwatch;
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.Collection;
40 import java.util.Collections;
41 import java.util.LinkedList;
42 import java.util.List;
43 import java.util.concurrent.ExecutionException;
44 import java.util.concurrent.TimeUnit;
45 import java.util.concurrent.atomic.AtomicLong;
46 import org.junit.After;
47 import org.junit.Assume;
48 import org.junit.Before;
49 import org.junit.Test;
50 import org.junit.runner.RunWith;
51 import org.junit.runners.Parameterized;
52 import org.junit.runners.Parameterized.Parameter;
53 import org.junit.runners.Parameterized.Parameters;
54 import org.mockito.Mockito;
55 import org.mockito.stubbing.Answer;
56 import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
57 import org.opendaylight.controller.cluster.databroker.ClientBackedDataStore;
58 import org.opendaylight.controller.cluster.databroker.ConcurrentDOMDataBroker;
59 import org.opendaylight.controller.cluster.datastore.DatastoreContext.Builder;
60 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
61 import org.opendaylight.controller.cluster.datastore.exceptions.ShardLeaderNotRespondingException;
62 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
63 import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction;
64 import org.opendaylight.controller.cluster.datastore.messages.GetShardDataTree;
65 import org.opendaylight.controller.cluster.datastore.messages.ReadyLocalTransaction;
66 import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
67 import org.opendaylight.controller.cluster.datastore.modification.MergeModification;
68 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
69 import org.opendaylight.controller.cluster.datastore.persisted.MetadataShardDataTreeSnapshot;
70 import org.opendaylight.controller.cluster.datastore.persisted.ShardSnapshotState;
71 import org.opendaylight.controller.cluster.raft.client.messages.Shutdown;
72 import org.opendaylight.controller.cluster.raft.persisted.ApplyJournalEntries;
73 import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
74 import org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy;
75 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
76 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
77 import org.opendaylight.controller.md.cluster.datastore.model.CarsModel;
78 import org.opendaylight.controller.md.cluster.datastore.model.PeopleModel;
79 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
80 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
81 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
82 import org.opendaylight.controller.md.sal.common.api.data.TransactionChainListener;
83 import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
84 import org.opendaylight.controller.md.sal.dom.api.DOMDataWriteTransaction;
85 import org.opendaylight.controller.md.sal.dom.api.DOMTransactionChain;
86 import org.opendaylight.controller.sal.core.spi.data.DOMStore;
87 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
88 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
89 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
90 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
91 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
92 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
93 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
94 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
95 import org.opendaylight.yangtools.yang.data.api.schema.MapNode;
96 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
97 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
98 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification;
99 import org.opendaylight.yangtools.yang.data.api.schema.tree.TipProducingDataTree;
100 import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType;
101 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
102 import org.opendaylight.yangtools.yang.data.impl.schema.builder.api.CollectionNodeBuilder;
103 import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder;
104 import org.opendaylight.yangtools.yang.data.impl.schema.tree.InMemoryDataTreeFactory;
105 import scala.concurrent.Await;
106 import scala.concurrent.Future;
107 import scala.concurrent.duration.FiniteDuration;
108
109 /**
110  * End-to-end distributed data store tests that exercise remote shards and transactions.
111  *
112  * @author Thomas Pantelis
113  */
114 @RunWith(Parameterized.class)
115 public class DistributedDataStoreRemotingIntegrationTest extends AbstractTest {
116
117     @Parameters(name = "{0}")
118     public static Collection<Object[]> data() {
119         return Arrays.asList(new Object[][] {
120                 { DistributedDataStore.class, 7}, { ClientBackedDataStore.class, 60 }
121         });
122     }
123
124     @Parameter(0)
125     public Class<? extends AbstractDataStore> testParameter;
126     @Parameter(1)
127     public int commitTimeout;
128
129     private static final String[] CARS_AND_PEOPLE = {"cars", "people"};
130     private static final String[] CARS = {"cars"};
131
132     private static final Address MEMBER_1_ADDRESS = AddressFromURIString.parse(
133             "akka://cluster-test@127.0.0.1:2558");
134     private static final Address MEMBER_2_ADDRESS = AddressFromURIString.parse(
135             "akka://cluster-test@127.0.0.1:2559");
136
137     private static final String MODULE_SHARDS_CARS_ONLY_1_2 = "module-shards-cars-member-1-and-2.conf";
138     private static final String MODULE_SHARDS_CARS_PEOPLE_1_2 = "module-shards-member1-and-2.conf";
139     private static final String MODULE_SHARDS_CARS_PEOPLE_1_2_3 = "module-shards-member1-and-2-and-3.conf";
140
141     private ActorSystem leaderSystem;
142     private ActorSystem followerSystem;
143     private ActorSystem follower2System;
144
145     private final DatastoreContext.Builder leaderDatastoreContextBuilder =
146             DatastoreContext.newBuilder().shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(2);
147
148     private final DatastoreContext.Builder followerDatastoreContextBuilder =
149             DatastoreContext.newBuilder().shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5)
150                 .customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName());
151     private final TransactionIdentifier tx1 = nextTransactionId();
152     private final TransactionIdentifier tx2 = nextTransactionId();
153
154     private AbstractDataStore followerDistributedDataStore;
155     private AbstractDataStore leaderDistributedDataStore;
156     private IntegrationTestKit followerTestKit;
157     private IntegrationTestKit leaderTestKit;
158
159     @Before
160     public void setUp() {
161         InMemoryJournal.clear();
162         InMemorySnapshotStore.clear();
163
164         leaderSystem = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member1"));
165         Cluster.get(leaderSystem).join(MEMBER_1_ADDRESS);
166
167         followerSystem = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member2"));
168         Cluster.get(followerSystem).join(MEMBER_1_ADDRESS);
169
170         follower2System = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member3"));
171         Cluster.get(follower2System).join(MEMBER_1_ADDRESS);
172     }
173
174     @After
175     public void tearDown() {
176         if (followerDistributedDataStore != null) {
177             leaderDistributedDataStore.close();
178         }
179         if (leaderDistributedDataStore != null) {
180             leaderDistributedDataStore.close();
181         }
182
183         JavaTestKit.shutdownActorSystem(leaderSystem);
184         JavaTestKit.shutdownActorSystem(followerSystem);
185         JavaTestKit.shutdownActorSystem(follower2System);
186
187         InMemoryJournal.clear();
188         InMemorySnapshotStore.clear();
189     }
190
191     private void initDatastoresWithCars(final String type) throws Exception {
192         initDatastores(type, MODULE_SHARDS_CARS_ONLY_1_2, CARS);
193     }
194
195     private void initDatastoresWithCarsAndPeople(final String type) throws Exception {
196         initDatastores(type, MODULE_SHARDS_CARS_PEOPLE_1_2, CARS_AND_PEOPLE);
197     }
198
199     private void initDatastores(final String type, final String moduleShardsConfig, final String[] shards)
200             throws Exception {
201         leaderTestKit = new IntegrationTestKit(leaderSystem, leaderDatastoreContextBuilder, commitTimeout);
202
203         leaderDistributedDataStore = leaderTestKit.setupAbstractDataStore(
204                 testParameter, type, moduleShardsConfig, false, shards);
205
206         followerTestKit = new IntegrationTestKit(followerSystem, followerDatastoreContextBuilder, commitTimeout);
207         followerDistributedDataStore = followerTestKit.setupAbstractDataStore(
208                 testParameter, type, moduleShardsConfig, false, shards);
209
210         leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(), shards);
211     }
212
213     private static void verifyCars(final DOMStoreReadTransaction readTx, final MapEntryNode... entries)
214             throws Exception {
215         final Optional<NormalizedNode<?, ?>> optional = readTx.read(CarsModel.CAR_LIST_PATH).get(5, TimeUnit.SECONDS);
216         assertEquals("isPresent", true, optional.isPresent());
217
218         final CollectionNodeBuilder<MapEntryNode, MapNode> listBuilder = ImmutableNodes.mapNodeBuilder(
219                 CarsModel.CAR_QNAME);
220         for (final NormalizedNode<?, ?> entry: entries) {
221             listBuilder.withChild((MapEntryNode) entry);
222         }
223
224         assertEquals("Car list node", listBuilder.build(), optional.get());
225     }
226
227     private static void verifyNode(final DOMStoreReadTransaction readTx, final YangInstanceIdentifier path,
228             final NormalizedNode<?, ?> expNode) throws Exception {
229         final Optional<NormalizedNode<?, ?>> optional = readTx.read(path).get(5, TimeUnit.SECONDS);
230         assertEquals("isPresent", true, optional.isPresent());
231         assertEquals("Data node", expNode, optional.get());
232     }
233
234     private static void verifyExists(final DOMStoreReadTransaction readTx, final YangInstanceIdentifier path)
235             throws Exception {
236         final Boolean exists = readTx.exists(path).get(5, TimeUnit.SECONDS);
237         assertEquals("exists", true, exists);
238     }
239
240     @Test
241     public void testWriteTransactionWithSingleShard() throws Exception {
242         final String testName = "testWriteTransactionWithSingleShard";
243         initDatastoresWithCars(testName);
244
245         final String followerCarShardName = "member-2-shard-cars-" + testName;
246
247         DOMStoreWriteTransaction writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
248         assertNotNull("newWriteOnlyTransaction returned null", writeTx);
249
250         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
251         writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
252
253         final MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
254         final YangInstanceIdentifier car1Path = CarsModel.newCarPath("optima");
255         writeTx.merge(car1Path, car1);
256
257         final MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(25000));
258         final YangInstanceIdentifier car2Path = CarsModel.newCarPath("sportage");
259         writeTx.merge(car2Path, car2);
260
261         followerTestKit.doCommit(writeTx.ready());
262
263         verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car1, car2);
264
265         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1, car2);
266
267         // Test delete
268
269         writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
270
271         writeTx.delete(car1Path);
272
273         followerTestKit.doCommit(writeTx.ready());
274
275         verifyExists(followerDistributedDataStore.newReadOnlyTransaction(), car2Path);
276
277         verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car2);
278
279         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car2);
280
281         // Re-instate the follower member 2 as a single-node to verify replication and recovery.
282
283         // The following is a bit tricky. Before we reinstate the follower we need to ensure it has persisted and
284         // applied and all the log entries from the leader. Since we've verified the car data above we know that
285         // all the transactions have been applied on the leader so we first read and capture its lastAppliedIndex.
286         final AtomicLong leaderLastAppliedIndex = new AtomicLong();
287         IntegrationTestKit.verifyShardState(leaderDistributedDataStore, CARS[0],
288             state -> leaderLastAppliedIndex.set(state.getLastApplied()));
289
290         // Now we need to make sure the follower has persisted the leader's lastAppliedIndex via ApplyJournalEntries.
291         // However we don't know exactly how many ApplyJournalEntries messages there will be as it can differ between
292         // the tell-based and ask-based front-ends. For ask-based there will be exactly 2 ApplyJournalEntries but
293         // tell-based persists additional payloads which could be replicated and applied in a batch resulting in
294         // either 2 or 3 ApplyJournalEntries. To handle this we read the follower's persisted ApplyJournalEntries
295         // until we find the one that encompasses the leader's lastAppliedIndex.
296         Stopwatch sw = Stopwatch.createStarted();
297         boolean done = false;
298         while (!done) {
299             final List<ApplyJournalEntries> entries = InMemoryJournal.get(followerCarShardName,
300                     ApplyJournalEntries.class);
301             for (ApplyJournalEntries aje: entries) {
302                 if (aje.getToIndex() >= leaderLastAppliedIndex.get()) {
303                     done = true;
304                     break;
305                 }
306             }
307
308             assertTrue("Follower did not persist ApplyJournalEntries containing leader's lastAppliedIndex "
309                     + leaderLastAppliedIndex + ". Entries persisted: " + entries, sw.elapsed(TimeUnit.SECONDS) <= 5);
310
311             Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
312         }
313
314         JavaTestKit.shutdownActorSystem(leaderSystem, null, Boolean.TRUE);
315         JavaTestKit.shutdownActorSystem(followerSystem, null, Boolean.TRUE);
316
317         final ActorSystem newSystem = newActorSystem("reinstated-member2", "Member2");
318
319         try (final AbstractDataStore member2Datastore = new IntegrationTestKit(newSystem, leaderDatastoreContextBuilder)
320                 .setupAbstractDataStore(testParameter, testName, "module-shards-member2", true, CARS)) {
321             verifyCars(member2Datastore.newReadOnlyTransaction(), car2);
322         }
323     }
324
325     @Test
326     public void testReadWriteTransactionWithSingleShard() throws Exception {
327         initDatastoresWithCars("testReadWriteTransactionWithSingleShard");
328
329         final DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
330         assertNotNull("newReadWriteTransaction returned null", rwTx);
331
332         rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
333         rwTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
334
335         final MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
336         rwTx.merge(CarsModel.newCarPath("optima"), car1);
337
338         verifyCars(rwTx, car1);
339
340         final MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(25000));
341         final YangInstanceIdentifier car2Path = CarsModel.newCarPath("sportage");
342         rwTx.merge(car2Path, car2);
343
344         verifyExists(rwTx, car2Path);
345
346         followerTestKit.doCommit(rwTx.ready());
347
348         verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car1, car2);
349     }
350
351     @Test
352     public void testWriteTransactionWithMultipleShards() throws Exception {
353         initDatastoresWithCarsAndPeople("testWriteTransactionWithMultipleShards");
354
355         final DOMStoreWriteTransaction writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
356         assertNotNull("newWriteOnlyTransaction returned null", writeTx);
357
358         final YangInstanceIdentifier carsPath = CarsModel.BASE_PATH;
359         final NormalizedNode<?, ?> carsNode = CarsModel.emptyContainer();
360         writeTx.write(carsPath, carsNode);
361
362         final YangInstanceIdentifier peoplePath = PeopleModel.BASE_PATH;
363         final NormalizedNode<?, ?> peopleNode = PeopleModel.emptyContainer();
364         writeTx.write(peoplePath, peopleNode);
365
366         followerTestKit.doCommit(writeTx.ready());
367
368         final DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
369
370         verifyNode(readTx, carsPath, carsNode);
371         verifyNode(readTx, peoplePath, peopleNode);
372     }
373
374     @Test
375     public void testReadWriteTransactionWithMultipleShards() throws Exception {
376         initDatastoresWithCarsAndPeople("testReadWriteTransactionWithMultipleShards");
377
378         final DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
379         assertNotNull("newReadWriteTransaction returned null", rwTx);
380
381         final YangInstanceIdentifier carsPath = CarsModel.BASE_PATH;
382         final NormalizedNode<?, ?> carsNode = CarsModel.emptyContainer();
383         rwTx.write(carsPath, carsNode);
384
385         final YangInstanceIdentifier peoplePath = PeopleModel.BASE_PATH;
386         final NormalizedNode<?, ?> peopleNode = PeopleModel.emptyContainer();
387         rwTx.write(peoplePath, peopleNode);
388
389         followerTestKit.doCommit(rwTx.ready());
390
391         final DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
392
393         verifyNode(readTx, carsPath, carsNode);
394         verifyNode(readTx, peoplePath, peopleNode);
395     }
396
397     @Test
398     public void testTransactionChainWithSingleShard() throws Exception {
399         initDatastoresWithCars("testTransactionChainWithSingleShard");
400
401         final DOMStoreTransactionChain txChain = followerDistributedDataStore.createTransactionChain();
402
403         // Add the top-level cars container with write-only.
404
405         final DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
406         assertNotNull("newWriteOnlyTransaction returned null", writeTx);
407
408         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
409
410         final DOMStoreThreePhaseCommitCohort writeTxReady = writeTx.ready();
411
412         // Verify the top-level cars container with read-only.
413
414         verifyNode(txChain.newReadOnlyTransaction(), CarsModel.BASE_PATH, CarsModel.emptyContainer());
415
416         // Perform car operations with read-write.
417
418         final DOMStoreReadWriteTransaction rwTx = txChain.newReadWriteTransaction();
419
420         verifyNode(rwTx, CarsModel.BASE_PATH, CarsModel.emptyContainer());
421
422         rwTx.merge(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
423
424         final MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
425         final YangInstanceIdentifier car1Path = CarsModel.newCarPath("optima");
426         rwTx.write(car1Path, car1);
427
428         verifyExists(rwTx, car1Path);
429
430         verifyCars(rwTx, car1);
431
432         final MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(25000));
433         rwTx.merge(CarsModel.newCarPath("sportage"), car2);
434
435         rwTx.delete(car1Path);
436
437         followerTestKit.doCommit(writeTxReady);
438
439         followerTestKit.doCommit(rwTx.ready());
440
441         txChain.close();
442
443         verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car2);
444     }
445
446     @Test
447     public void testTransactionChainWithMultipleShards() throws Exception {
448         initDatastoresWithCarsAndPeople("testTransactionChainWithMultipleShards");
449
450         final DOMStoreTransactionChain txChain = followerDistributedDataStore.createTransactionChain();
451
452         DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
453         assertNotNull("newWriteOnlyTransaction returned null", writeTx);
454
455         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
456         writeTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
457
458         writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
459         writeTx.write(PeopleModel.PERSON_LIST_PATH, PeopleModel.newPersonMapNode());
460
461         followerTestKit.doCommit(writeTx.ready());
462
463         final DOMStoreReadWriteTransaction readWriteTx = txChain.newReadWriteTransaction();
464
465         final MapEntryNode car = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
466         final YangInstanceIdentifier carPath = CarsModel.newCarPath("optima");
467         readWriteTx.write(carPath, car);
468
469         final MapEntryNode person = PeopleModel.newPersonEntry("jack");
470         final YangInstanceIdentifier personPath = PeopleModel.newPersonPath("jack");
471         readWriteTx.merge(personPath, person);
472
473         Optional<NormalizedNode<?, ?>> optional = readWriteTx.read(carPath).get(5, TimeUnit.SECONDS);
474         assertEquals("isPresent", true, optional.isPresent());
475         assertEquals("Data node", car, optional.get());
476
477         optional = readWriteTx.read(personPath).get(5, TimeUnit.SECONDS);
478         assertEquals("isPresent", true, optional.isPresent());
479         assertEquals("Data node", person, optional.get());
480
481         final DOMStoreThreePhaseCommitCohort cohort2 = readWriteTx.ready();
482
483         writeTx = txChain.newWriteOnlyTransaction();
484
485         writeTx.delete(personPath);
486
487         final DOMStoreThreePhaseCommitCohort cohort3 = writeTx.ready();
488
489         followerTestKit.doCommit(cohort2);
490         followerTestKit.doCommit(cohort3);
491
492         txChain.close();
493
494         final DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
495         verifyCars(readTx, car);
496
497         optional = readTx.read(personPath).get(5, TimeUnit.SECONDS);
498         assertEquals("isPresent", false, optional.isPresent());
499     }
500
501     @Test
502     public void testChainedTransactionFailureWithSingleShard() throws Exception {
503         initDatastoresWithCars("testChainedTransactionFailureWithSingleShard");
504
505         final ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
506                 ImmutableMap.<LogicalDatastoreType, DOMStore>builder().put(
507                         LogicalDatastoreType.CONFIGURATION, followerDistributedDataStore).build(),
508                         MoreExecutors.directExecutor());
509
510         final TransactionChainListener listener = Mockito.mock(TransactionChainListener.class);
511         final DOMTransactionChain txChain = broker.createTransactionChain(listener);
512
513         final DOMDataWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
514
515         final ContainerNode invalidData = ImmutableContainerNodeBuilder.create().withNodeIdentifier(
516                 new YangInstanceIdentifier.NodeIdentifier(CarsModel.BASE_QNAME))
517                     .withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build();
518
519         writeTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData);
520
521         try {
522             writeTx.submit().checkedGet(5, TimeUnit.SECONDS);
523             fail("Expected TransactionCommitFailedException");
524         } catch (final TransactionCommitFailedException e) {
525             // Expected
526         }
527
528         verify(listener, timeout(5000)).onTransactionChainFailed(eq(txChain), eq(writeTx), any(Throwable.class));
529
530         txChain.close();
531         broker.close();
532     }
533
534     @Test
535     public void testChainedTransactionFailureWithMultipleShards() throws Exception {
536         initDatastoresWithCarsAndPeople("testChainedTransactionFailureWithMultipleShards");
537
538         final ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
539                 ImmutableMap.<LogicalDatastoreType, DOMStore>builder().put(
540                         LogicalDatastoreType.CONFIGURATION, followerDistributedDataStore).build(),
541                         MoreExecutors.directExecutor());
542
543         final TransactionChainListener listener = Mockito.mock(TransactionChainListener.class);
544         final DOMTransactionChain txChain = broker.createTransactionChain(listener);
545
546         final DOMDataWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
547
548         writeTx.put(LogicalDatastoreType.CONFIGURATION, PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
549
550         final ContainerNode invalidData = ImmutableContainerNodeBuilder.create().withNodeIdentifier(
551                 new YangInstanceIdentifier.NodeIdentifier(CarsModel.BASE_QNAME))
552                     .withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build();
553
554         // Note that merge will validate the data and fail but put succeeds b/c deep validation is not
555         // done for put for performance reasons.
556         writeTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData);
557
558         try {
559             writeTx.submit().checkedGet(5, TimeUnit.SECONDS);
560             fail("Expected TransactionCommitFailedException");
561         } catch (final TransactionCommitFailedException e) {
562             // Expected
563         }
564
565         verify(listener, timeout(5000)).onTransactionChainFailed(eq(txChain), eq(writeTx), any(Throwable.class));
566
567         txChain.close();
568         broker.close();
569     }
570
571     @Test
572     public void testSingleShardTransactionsWithLeaderChanges() throws Exception {
573         final String testName = "testSingleShardTransactionsWithLeaderChanges";
574         initDatastoresWithCars(testName);
575
576         final String followerCarShardName = "member-2-shard-cars-" + testName;
577         InMemoryJournal.addWriteMessagesCompleteLatch(followerCarShardName, 1, ApplyJournalEntries.class);
578
579         // Write top-level car container from the follower so it uses a remote Tx.
580
581         DOMStoreWriteTransaction writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
582
583         writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
584         writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
585
586         followerTestKit.doCommit(writeTx.ready());
587
588         InMemoryJournal.waitForWriteMessagesComplete(followerCarShardName);
589
590         // Switch the leader to the follower
591
592         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder
593                 .shardElectionTimeoutFactor(1).customRaftPolicyImplementation(null));
594
595         JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
596         Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS);
597
598         followerTestKit.waitUntilNoLeader(followerDistributedDataStore.getActorContext(), CARS);
599
600         leaderSystem = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member1"));
601         Cluster.get(leaderSystem).join(MEMBER_2_ADDRESS);
602
603         final DatastoreContext.Builder newMember1Builder = DatastoreContext.newBuilder()
604                 .shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5);
605         IntegrationTestKit newMember1TestKit = new IntegrationTestKit(leaderSystem, newMember1Builder, commitTimeout);
606
607         try (AbstractDataStore ds =
608                 newMember1TestKit.setupAbstractDataStore(
609                         testParameter, testName, MODULE_SHARDS_CARS_ONLY_1_2, false, CARS)) {
610
611             followerTestKit.waitUntilLeader(followerDistributedDataStore.getActorContext(), CARS);
612
613             // Write a car entry to the new leader - should switch to local Tx
614
615             writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
616
617             MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
618             YangInstanceIdentifier car1Path = CarsModel.newCarPath("optima");
619             writeTx.merge(car1Path, car1);
620
621             followerTestKit.doCommit(writeTx.ready());
622
623             verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car1);
624         }
625     }
626
627     @SuppressWarnings("unchecked")
628     @Test
629     public void testReadyLocalTransactionForwardedToLeader() throws Exception {
630         initDatastoresWithCars("testReadyLocalTransactionForwardedToLeader");
631         followerTestKit.waitUntilLeader(followerDistributedDataStore.getActorContext(), "cars");
632
633         final Optional<ActorRef> carsFollowerShard = followerDistributedDataStore.getActorContext()
634                 .findLocalShard("cars");
635         assertEquals("Cars follower shard found", true, carsFollowerShard.isPresent());
636
637         final TipProducingDataTree dataTree = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL);
638         dataTree.setSchemaContext(SchemaContextHelper.full());
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         final MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
647         new WriteModification(CarsModel.newCarPath("optima"), car1).apply(modification);
648         modification.ready();
649
650         ReadyLocalTransaction readyLocal = new ReadyLocalTransaction(tx1 , modification, true);
651
652         carsFollowerShard.get().tell(readyLocal, followerTestKit.getRef());
653         Object resp = followerTestKit.expectMsgClass(Object.class);
654         if (resp instanceof akka.actor.Status.Failure) {
655             throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause());
656         }
657
658         assertEquals("Response type", CommitTransactionReply.class, resp.getClass());
659
660         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1);
661
662         // Send another tx without immediate commit.
663
664         modification = dataTree.takeSnapshot().newModification();
665         MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(30000));
666         new WriteModification(CarsModel.newCarPath("sportage"), car2).apply(modification);
667         modification.ready();
668
669         readyLocal = new ReadyLocalTransaction(tx2 , modification, false);
670
671         carsFollowerShard.get().tell(readyLocal, followerTestKit.getRef());
672         resp = followerTestKit.expectMsgClass(Object.class);
673         if (resp instanceof akka.actor.Status.Failure) {
674             throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause());
675         }
676
677         assertEquals("Response type", ReadyTransactionReply.class, resp.getClass());
678
679         final ActorSelection txActor = leaderDistributedDataStore.getActorContext().actorSelection(
680                 ((ReadyTransactionReply)resp).getCohortPath());
681
682         final Supplier<Short> versionSupplier = Mockito.mock(Supplier.class);
683         Mockito.doReturn(DataStoreVersions.CURRENT_VERSION).when(versionSupplier).get();
684         ThreePhaseCommitCohortProxy cohort = new ThreePhaseCommitCohortProxy(
685                 leaderDistributedDataStore.getActorContext(), Arrays.asList(
686                         new ThreePhaseCommitCohortProxy.CohortInfo(Futures.successful(txActor), versionSupplier)), tx2);
687         cohort.canCommit().get(5, TimeUnit.SECONDS);
688         cohort.preCommit().get(5, TimeUnit.SECONDS);
689         cohort.commit().get(5, TimeUnit.SECONDS);
690
691         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1, car2);
692     }
693
694     @SuppressWarnings("unchecked")
695     @Test
696     public void testForwardedReadyTransactionForwardedToLeader() throws Exception {
697         initDatastoresWithCars("testForwardedReadyTransactionForwardedToLeader");
698         followerTestKit.waitUntilLeader(followerDistributedDataStore.getActorContext(), "cars");
699
700         final Optional<ActorRef> carsFollowerShard = followerDistributedDataStore.getActorContext()
701                 .findLocalShard("cars");
702         assertEquals("Cars follower shard found", true, carsFollowerShard.isPresent());
703
704         carsFollowerShard.get().tell(GetShardDataTree.INSTANCE, followerTestKit.getRef());
705         final DataTree dataTree = followerTestKit.expectMsgClass(DataTree.class);
706
707         // Send a tx with immediate commit.
708
709         DataTreeModification modification = dataTree.takeSnapshot().newModification();
710         new WriteModification(CarsModel.BASE_PATH, CarsModel.emptyContainer()).apply(modification);
711         new MergeModification(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode()).apply(modification);
712
713         final MapEntryNode car1 = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
714         new WriteModification(CarsModel.newCarPath("optima"), car1).apply(modification);
715
716         ForwardedReadyTransaction forwardedReady = new ForwardedReadyTransaction(tx1,
717                 DataStoreVersions.CURRENT_VERSION, new ReadWriteShardDataTreeTransaction(
718                         Mockito.mock(ShardDataTreeTransactionParent.class), tx1, modification), true);
719
720         carsFollowerShard.get().tell(forwardedReady, followerTestKit.getRef());
721         Object resp = followerTestKit.expectMsgClass(Object.class);
722         if (resp instanceof akka.actor.Status.Failure) {
723             throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause());
724         }
725
726         assertEquals("Response type", CommitTransactionReply.class, resp.getClass());
727
728         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1);
729
730         // Send another tx without immediate commit.
731
732         modification = dataTree.takeSnapshot().newModification();
733         MapEntryNode car2 = CarsModel.newCarEntry("sportage", BigInteger.valueOf(30000));
734         new WriteModification(CarsModel.newCarPath("sportage"), car2).apply(modification);
735
736         forwardedReady = new ForwardedReadyTransaction(tx2,
737                 DataStoreVersions.CURRENT_VERSION, new ReadWriteShardDataTreeTransaction(
738                         Mockito.mock(ShardDataTreeTransactionParent.class), tx2, modification), false);
739
740         carsFollowerShard.get().tell(forwardedReady, followerTestKit.getRef());
741         resp = followerTestKit.expectMsgClass(Object.class);
742         if (resp instanceof akka.actor.Status.Failure) {
743             throw new AssertionError("Unexpected failure response", ((akka.actor.Status.Failure)resp).cause());
744         }
745
746         assertEquals("Response type", ReadyTransactionReply.class, resp.getClass());
747
748         ActorSelection txActor = leaderDistributedDataStore.getActorContext().actorSelection(
749                 ((ReadyTransactionReply)resp).getCohortPath());
750
751         final Supplier<Short> versionSupplier = Mockito.mock(Supplier.class);
752         Mockito.doReturn(DataStoreVersions.CURRENT_VERSION).when(versionSupplier).get();
753         final ThreePhaseCommitCohortProxy cohort = new ThreePhaseCommitCohortProxy(
754                 leaderDistributedDataStore.getActorContext(), Arrays.asList(
755                         new ThreePhaseCommitCohortProxy.CohortInfo(Futures.successful(txActor), versionSupplier)), tx2);
756         cohort.canCommit().get(5, TimeUnit.SECONDS);
757         cohort.preCommit().get(5, TimeUnit.SECONDS);
758         cohort.commit().get(5, TimeUnit.SECONDS);
759
760         verifyCars(leaderDistributedDataStore.newReadOnlyTransaction(), car1, car2);
761     }
762
763     @Test
764     public void testTransactionForwardedToLeaderAfterRetry() throws Exception {
765         //TODO remove when test passes also for ClientBackedDataStore
766         Assume.assumeTrue(testParameter.equals(DistributedDataStore.class));
767         followerDatastoreContextBuilder.shardBatchedModificationCount(2);
768         leaderDatastoreContextBuilder.shardBatchedModificationCount(2);
769         initDatastoresWithCarsAndPeople("testTransactionForwardedToLeaderAfterRetry");
770
771         // Do an initial write to get the primary shard info cached.
772
773         final DOMStoreWriteTransaction initialWriteTx = followerDistributedDataStore.newWriteOnlyTransaction();
774         initialWriteTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
775         initialWriteTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
776         followerTestKit.doCommit(initialWriteTx.ready());
777
778         // Wait for the commit to be replicated to the follower.
779
780         MemberNode.verifyRaftState(followerDistributedDataStore, "cars",
781             raftState -> assertEquals("getLastApplied", 0, raftState.getLastApplied()));
782
783         MemberNode.verifyRaftState(followerDistributedDataStore, "people",
784             raftState -> assertEquals("getLastApplied", 0, raftState.getLastApplied()));
785
786         // Prepare, ready and canCommit a WO tx that writes to 2 shards. This will become the current tx in
787         // the leader shard.
788
789         final DOMStoreWriteTransaction writeTx1 = followerDistributedDataStore.newWriteOnlyTransaction();
790         writeTx1.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
791         writeTx1.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
792         final DOMStoreThreePhaseCommitCohort writeTx1Cohort = writeTx1.ready();
793         final ListenableFuture<Boolean> writeTx1CanCommit = writeTx1Cohort.canCommit();
794         writeTx1CanCommit.get(5, TimeUnit.SECONDS);
795
796         // Prepare and ready another WO tx that writes to 2 shards but don't canCommit yet. This will be queued
797         // in the leader shard.
798
799         final DOMStoreWriteTransaction writeTx2 = followerDistributedDataStore.newWriteOnlyTransaction();
800         final LinkedList<MapEntryNode> cars = new LinkedList<>();
801         int carIndex = 1;
802         cars.add(CarsModel.newCarEntry("car" + carIndex, BigInteger.valueOf(carIndex)));
803         writeTx2.write(CarsModel.newCarPath("car" + carIndex), cars.getLast());
804         carIndex++;
805         NormalizedNode<?, ?> people = PeopleModel.newPersonMapNode();
806         writeTx2.write(PeopleModel.PERSON_LIST_PATH, people);
807         final DOMStoreThreePhaseCommitCohort writeTx2Cohort = writeTx2.ready();
808
809         // Prepare another WO that writes to a single shard and thus will be directly committed on ready. This
810         // tx writes 5 cars so 2 BatchedModidifications messages will be sent initially and cached in the
811         // leader shard (with shardBatchedModificationCount set to 2). The 3rd BatchedModidifications will be
812         // sent on ready.
813
814         final DOMStoreWriteTransaction writeTx3 = followerDistributedDataStore.newWriteOnlyTransaction();
815         for (int i = 1; i <= 5; i++, carIndex++) {
816             cars.add(CarsModel.newCarEntry("car" + carIndex, BigInteger.valueOf(carIndex)));
817             writeTx3.write(CarsModel.newCarPath("car" + carIndex), cars.getLast());
818         }
819
820         // Prepare another WO that writes to a single shard. This will send a single BatchedModidifications
821         // message on ready.
822
823         final DOMStoreWriteTransaction writeTx4 = followerDistributedDataStore.newWriteOnlyTransaction();
824         cars.add(CarsModel.newCarEntry("car" + carIndex, BigInteger.valueOf(carIndex)));
825         writeTx4.write(CarsModel.newCarPath("car" + carIndex), cars.getLast());
826         carIndex++;
827
828         // Prepare a RW tx that will create a tx actor and send a ForwardedReadyTransaciton message to the
829         // leader shard on ready.
830
831         final DOMStoreReadWriteTransaction readWriteTx = followerDistributedDataStore.newReadWriteTransaction();
832         cars.add(CarsModel.newCarEntry("car" + carIndex, BigInteger.valueOf(carIndex)));
833         readWriteTx.write(CarsModel.newCarPath("car" + carIndex), cars.getLast());
834
835         IntegrationTestKit.verifyShardStats(leaderDistributedDataStore, "cars",
836             stats -> assertEquals("getReadWriteTransactionCount", 1, stats.getReadWriteTransactionCount()));
837
838         // Disable elections on the leader so it switches to follower.
839
840         sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder
841                 .customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName())
842                 .shardElectionTimeoutFactor(10));
843
844         Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS);
845         leaderTestKit.waitUntilNoLeader(leaderDistributedDataStore.getActorContext(), "cars");
846
847         // Submit all tx's - the messages should get queued for retry.
848
849         final ListenableFuture<Boolean> writeTx2CanCommit = writeTx2Cohort.canCommit();
850         final DOMStoreThreePhaseCommitCohort writeTx3Cohort = writeTx3.ready();
851         final DOMStoreThreePhaseCommitCohort writeTx4Cohort = writeTx4.ready();
852         final DOMStoreThreePhaseCommitCohort rwTxCohort = readWriteTx.ready();
853
854         // Enable elections on the other follower so it becomes the leader, at which point the
855         // tx's should get forwarded from the previous leader to the new leader to complete the commits.
856
857         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder
858                 .customRaftPolicyImplementation(null).shardElectionTimeoutFactor(1));
859
860         followerTestKit.doCommit(writeTx1CanCommit, writeTx1Cohort);
861         followerTestKit.doCommit(writeTx2CanCommit, writeTx2Cohort);
862         followerTestKit.doCommit(writeTx3Cohort);
863         followerTestKit.doCommit(writeTx4Cohort);
864         followerTestKit.doCommit(rwTxCohort);
865
866         DOMStoreReadTransaction readTx = leaderDistributedDataStore.newReadOnlyTransaction();
867         verifyCars(readTx, cars.toArray(new MapEntryNode[cars.size()]));
868         verifyNode(readTx, PeopleModel.PERSON_LIST_PATH, people);
869     }
870
871     @Test
872     public void testLeadershipTransferOnShutdown() throws Exception {
873         //TODO remove when test passes also for ClientBackedDataStore
874         Assume.assumeTrue(testParameter.equals(DistributedDataStore.class));
875         leaderDatastoreContextBuilder.shardBatchedModificationCount(1);
876         followerDatastoreContextBuilder.shardElectionTimeoutFactor(10).customRaftPolicyImplementation(null);
877         final String testName = "testLeadershipTransferOnShutdown";
878         initDatastores(testName, MODULE_SHARDS_CARS_PEOPLE_1_2_3, CARS_AND_PEOPLE);
879
880         final IntegrationTestKit follower2TestKit = new IntegrationTestKit(follower2System,
881                 DatastoreContext.newBuilderFrom(followerDatastoreContextBuilder.build()).operationTimeoutInMillis(100),
882                 commitTimeout);
883         try (AbstractDataStore follower2DistributedDataStore = follower2TestKit.setupAbstractDataStore(
884                 testParameter, testName, MODULE_SHARDS_CARS_PEOPLE_1_2_3, false)) {
885
886             // Create and submit a couple tx's so they're pending.
887
888             DOMStoreWriteTransaction writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
889             writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
890             writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
891             writeTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
892             final DOMStoreThreePhaseCommitCohort cohort1 = writeTx.ready();
893
894             IntegrationTestKit.verifyShardStats(leaderDistributedDataStore, "cars",
895                 stats -> assertEquals("getTxCohortCacheSize", 1, stats.getTxCohortCacheSize()));
896
897             writeTx = followerDistributedDataStore.newWriteOnlyTransaction();
898             final MapEntryNode car = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
899             writeTx.write(CarsModel.newCarPath("optima"), car);
900             final DOMStoreThreePhaseCommitCohort cohort2 = writeTx.ready();
901
902             IntegrationTestKit.verifyShardStats(leaderDistributedDataStore, "cars",
903                 stats -> assertEquals("getTxCohortCacheSize", 2, stats.getTxCohortCacheSize()));
904
905             // Gracefully stop the leader via a Shutdown message.
906
907             sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder
908                 .shardElectionTimeoutFactor(100));
909
910             final FiniteDuration duration = FiniteDuration.create(5, TimeUnit.SECONDS);
911             final Future<ActorRef> future = leaderDistributedDataStore.getActorContext().findLocalShardAsync("cars");
912             final ActorRef leaderActor = Await.result(future, duration);
913
914             final Future<Boolean> stopFuture = Patterns.gracefulStop(leaderActor, duration, Shutdown.INSTANCE);
915
916             // Commit the 2 transactions. They should finish and succeed.
917
918             followerTestKit.doCommit(cohort1);
919             followerTestKit.doCommit(cohort2);
920
921             // Wait for the leader actor stopped.
922
923             final Boolean stopped = Await.result(stopFuture, duration);
924             assertEquals("Stopped", Boolean.TRUE, stopped);
925
926             // Verify leadership was transferred by reading the committed data from the other nodes.
927
928             verifyCars(followerDistributedDataStore.newReadOnlyTransaction(), car);
929             verifyCars(follower2DistributedDataStore.newReadOnlyTransaction(), car);
930         }
931     }
932
933     @Test
934     public void testTransactionWithIsolatedLeader() throws Exception {
935         //TODO remove when test passes also for ClientBackedDataStore
936         Assume.assumeTrue(testParameter.equals(DistributedDataStore.class));
937         // Set the isolated leader check interval high so we can control the switch to IsolatedLeader.
938         leaderDatastoreContextBuilder.shardIsolatedLeaderCheckIntervalInMillis(10000000);
939         final String testName = "testTransactionWithIsolatedLeader";
940         initDatastoresWithCars(testName);
941
942         // Tx that is submitted after the follower is stopped but before the leader transitions to IsolatedLeader.
943         final DOMStoreWriteTransaction preIsolatedLeaderWriteTx = leaderDistributedDataStore.newWriteOnlyTransaction();
944         preIsolatedLeaderWriteTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
945
946         // Tx that is submitted after the leader transitions to IsolatedLeader.
947         final DOMStoreWriteTransaction noShardLeaderWriteTx = leaderDistributedDataStore.newWriteOnlyTransaction();
948         noShardLeaderWriteTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
949
950         // Tx that is submitted after the follower is reinstated.
951         final DOMStoreWriteTransaction successWriteTx = leaderDistributedDataStore.newWriteOnlyTransaction();
952         successWriteTx.merge(CarsModel.BASE_PATH, CarsModel.emptyContainer());
953
954         // Stop the follower
955         followerTestKit.watch(followerDistributedDataStore.getActorContext().getShardManager());
956         followerDistributedDataStore.close();
957         followerTestKit.expectTerminated(followerDistributedDataStore.getActorContext().getShardManager());
958
959         // Submit the preIsolatedLeaderWriteTx so it's pending
960         final DOMStoreThreePhaseCommitCohort preIsolatedLeaderTxCohort = preIsolatedLeaderWriteTx.ready();
961
962         // Change the isolated leader check interval low so it changes to IsolatedLeader.
963         sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder
964                 .shardIsolatedLeaderCheckIntervalInMillis(200));
965
966         MemberNode.verifyRaftState(leaderDistributedDataStore, "cars",
967             raftState -> assertEquals("getRaftState", "IsolatedLeader", raftState.getRaftState()));
968
969         try {
970             leaderTestKit.doCommit(noShardLeaderWriteTx.ready());
971             fail("Expected NoShardLeaderException");
972         } catch (final ExecutionException e) {
973             assertEquals("getCause", NoShardLeaderException.class, Throwables.getRootCause(e).getClass());
974         }
975
976         sendDatastoreContextUpdate(leaderDistributedDataStore, leaderDatastoreContextBuilder
977                 .shardElectionTimeoutFactor(100));
978
979         final DOMStoreThreePhaseCommitCohort successTxCohort = successWriteTx.ready();
980
981         followerDistributedDataStore = followerTestKit.setupAbstractDataStore(
982                 testParameter, testName, MODULE_SHARDS_CARS_ONLY_1_2, false, CARS);
983
984         leaderTestKit.doCommit(preIsolatedLeaderTxCohort);
985         leaderTestKit.doCommit(successTxCohort);
986     }
987
988     @Test
989     public void testTransactionWithShardLeaderNotResponding() throws Exception {
990         followerDatastoreContextBuilder.shardElectionTimeoutFactor(50);
991         initDatastoresWithCars("testTransactionWithShardLeaderNotResponding");
992
993         // Do an initial read to get the primary shard info cached.
994
995         final DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
996         readTx.read(CarsModel.BASE_PATH).checkedGet(5, TimeUnit.SECONDS);
997
998         // Shutdown the leader and try to create a new tx.
999
1000         JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
1001
1002         followerDatastoreContextBuilder.operationTimeoutInMillis(50).shardElectionTimeoutFactor(1);
1003         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder);
1004
1005         final DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
1006
1007         rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
1008
1009         try {
1010             followerTestKit.doCommit(rwTx.ready());
1011             fail("Exception expected");
1012         } catch (final ExecutionException e) {
1013             final String msg = "Unexpected exception: " + Throwables.getStackTraceAsString(e.getCause());
1014             assertTrue(msg, Throwables.getRootCause(e) instanceof NoShardLeaderException
1015                     || e.getCause() instanceof ShardLeaderNotRespondingException);
1016         }
1017     }
1018
1019     @Test
1020     public void testTransactionWithCreateTxFailureDueToNoLeader() throws Exception {
1021         initDatastoresWithCars("testTransactionWithCreateTxFailureDueToNoLeader");
1022
1023         // Do an initial read to get the primary shard info cached.
1024
1025         final DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
1026         readTx.read(CarsModel.BASE_PATH).checkedGet(5, TimeUnit.SECONDS);
1027
1028         // Shutdown the leader and try to create a new tx.
1029
1030         JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
1031
1032         Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS);
1033
1034         Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
1035
1036         sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder
1037                 .operationTimeoutInMillis(10).shardElectionTimeoutFactor(1).customRaftPolicyImplementation(null));
1038
1039         final DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
1040
1041         rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
1042
1043         try {
1044             followerTestKit.doCommit(rwTx.ready());
1045             fail("Exception expected");
1046         } catch (final ExecutionException e) {
1047             final String msg = "Expected instance of NoShardLeaderException, actual: \n"
1048                     + Throwables.getStackTraceAsString(e.getCause());
1049             assertTrue(msg, Throwables.getRootCause(e) instanceof NoShardLeaderException);
1050         }
1051     }
1052
1053     @Test
1054     public void testTransactionRetryWithInitialAskTimeoutExOnCreateTx() throws Exception {
1055         String testName = "testTransactionRetryWithInitialAskTimeoutExOnCreateTx";
1056         initDatastores(testName, MODULE_SHARDS_CARS_PEOPLE_1_2_3, CARS);
1057
1058         final DatastoreContext.Builder follower2DatastoreContextBuilder = DatastoreContext.newBuilder()
1059                 .shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5);
1060         final IntegrationTestKit follower2TestKit = new IntegrationTestKit(
1061                 follower2System, follower2DatastoreContextBuilder, commitTimeout);
1062
1063         try (AbstractDataStore ds =
1064                 follower2TestKit.setupAbstractDataStore(
1065                         testParameter, testName, MODULE_SHARDS_CARS_PEOPLE_1_2_3, false, CARS)) {
1066
1067             followerTestKit.waitForMembersUp("member-1", "member-3");
1068             follower2TestKit.waitForMembersUp("member-1", "member-2");
1069
1070             // Do an initial read to get the primary shard info cached.
1071
1072             final DOMStoreReadTransaction readTx = followerDistributedDataStore.newReadOnlyTransaction();
1073             readTx.read(CarsModel.BASE_PATH).checkedGet(5, TimeUnit.SECONDS);
1074
1075             // Shutdown the leader and try to create a new tx.
1076
1077             JavaTestKit.shutdownActorSystem(leaderSystem, null, true);
1078
1079             Cluster.get(followerSystem).leave(MEMBER_1_ADDRESS);
1080
1081             sendDatastoreContextUpdate(followerDistributedDataStore, followerDatastoreContextBuilder
1082                 .operationTimeoutInMillis(500).shardElectionTimeoutFactor(1).customRaftPolicyImplementation(null));
1083
1084             final DOMStoreReadWriteTransaction rwTx = followerDistributedDataStore.newReadWriteTransaction();
1085
1086             rwTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
1087
1088             followerTestKit.doCommit(rwTx.ready());
1089         }
1090     }
1091
1092     @Test
1093     public void testInstallSnapshot() throws Exception {
1094         final String testName = "testInstallSnapshot";
1095         final String leaderCarShardName = "member-1-shard-cars-" + testName;
1096         final String followerCarShardName = "member-2-shard-cars-" + testName;
1097
1098         // Setup a saved snapshot on the leader. The follower will startup with no data and the leader should
1099         // install a snapshot to sync the follower.
1100
1101         TipProducingDataTree tree = InMemoryDataTreeFactory.getInstance().create(TreeType.CONFIGURATION);
1102         tree.setSchemaContext(SchemaContextHelper.full());
1103
1104         final ContainerNode carsNode = CarsModel.newCarsNode(
1105                 CarsModel.newCarsMapNode(CarsModel.newCarEntry("optima", BigInteger.valueOf(20000))));
1106         AbstractShardTest.writeToStore(tree, CarsModel.BASE_PATH, carsNode);
1107
1108         final NormalizedNode<?, ?> snapshotRoot = AbstractShardTest.readStore(tree, YangInstanceIdentifier.EMPTY);
1109         final Snapshot initialSnapshot = Snapshot.create(
1110                 new ShardSnapshotState(new MetadataShardDataTreeSnapshot(snapshotRoot)),
1111                 Collections.emptyList(), 5, 1, 5, 1, 1, null, null);
1112         InMemorySnapshotStore.addSnapshot(leaderCarShardName, initialSnapshot);
1113
1114         InMemorySnapshotStore.addSnapshotSavedLatch(leaderCarShardName);
1115         InMemorySnapshotStore.addSnapshotSavedLatch(followerCarShardName);
1116
1117         initDatastoresWithCars(testName);
1118
1119         final Optional<NormalizedNode<?, ?>> readOptional = leaderDistributedDataStore.newReadOnlyTransaction().read(
1120                 CarsModel.BASE_PATH).checkedGet(5, TimeUnit.SECONDS);
1121         assertEquals("isPresent", true, readOptional.isPresent());
1122         assertEquals("Node", carsNode, readOptional.get());
1123
1124         verifySnapshot(InMemorySnapshotStore.waitForSavedSnapshot(leaderCarShardName, Snapshot.class),
1125                 initialSnapshot, snapshotRoot);
1126
1127         verifySnapshot(InMemorySnapshotStore.waitForSavedSnapshot(followerCarShardName, Snapshot.class),
1128                 initialSnapshot, snapshotRoot);
1129     }
1130
1131     private static void verifySnapshot(final Snapshot actual, final Snapshot expected,
1132                                        final NormalizedNode<?, ?> expRoot) {
1133         assertEquals("Snapshot getLastAppliedTerm", expected.getLastAppliedTerm(), actual.getLastAppliedTerm());
1134         assertEquals("Snapshot getLastAppliedIndex", expected.getLastAppliedIndex(), actual.getLastAppliedIndex());
1135         assertEquals("Snapshot getLastTerm", expected.getLastTerm(), actual.getLastTerm());
1136         assertEquals("Snapshot getLastIndex", expected.getLastIndex(), actual.getLastIndex());
1137         assertEquals("Snapshot state type", ShardSnapshotState.class, actual.getState().getClass());
1138         MetadataShardDataTreeSnapshot shardSnapshot =
1139                 (MetadataShardDataTreeSnapshot) ((ShardSnapshotState)actual.getState()).getSnapshot();
1140         assertEquals("Snapshot root node", expRoot, shardSnapshot.getRootNode().get());
1141     }
1142
1143     private static void sendDatastoreContextUpdate(final AbstractDataStore dataStore, final Builder builder) {
1144         final Builder newBuilder = DatastoreContext.newBuilderFrom(builder.build());
1145         final DatastoreContextFactory mockContextFactory = Mockito.mock(DatastoreContextFactory.class);
1146         final Answer<DatastoreContext> answer = invocation -> newBuilder.build();
1147         Mockito.doAnswer(answer).when(mockContextFactory).getBaseDatastoreContext();
1148         Mockito.doAnswer(answer).when(mockContextFactory).getShardDatastoreContext(Mockito.anyString());
1149         dataStore.onDatastoreContextUpdated(mockContextFactory);
1150     }
1151 }