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