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