Partially enable testSingleTransactionsWritesInQuickSuccession()
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / AbstractDistributedDataStoreIntegrationTest.java
1 /*
2  * Copyright (c) 2014, 2017 Cisco 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.awaitility.Awaitility.await;
11 import static org.hamcrest.CoreMatchers.instanceOf;
12 import static org.hamcrest.MatcherAssert.assertThat;
13 import static org.junit.Assert.assertEquals;
14 import static org.junit.Assert.assertFalse;
15 import static org.junit.Assert.assertNotNull;
16 import static org.junit.Assert.assertTrue;
17 import static org.junit.Assert.fail;
18 import static org.junit.Assume.assumeFalse;
19 import static org.mockito.ArgumentMatchers.any;
20 import static org.mockito.ArgumentMatchers.eq;
21 import static org.mockito.Mockito.timeout;
22 import static org.mockito.Mockito.verify;
23
24 import akka.actor.ActorSystem;
25 import com.google.common.base.Throwables;
26 import com.google.common.collect.ImmutableMap;
27 import com.google.common.util.concurrent.FluentFuture;
28 import com.google.common.util.concurrent.ListenableFuture;
29 import com.google.common.util.concurrent.MoreExecutors;
30 import com.google.common.util.concurrent.Uninterruptibles;
31 import java.util.ArrayList;
32 import java.util.Arrays;
33 import java.util.Collection;
34 import java.util.Collections;
35 import java.util.List;
36 import java.util.Optional;
37 import java.util.concurrent.CountDownLatch;
38 import java.util.concurrent.ExecutionException;
39 import java.util.concurrent.TimeUnit;
40 import java.util.concurrent.atomic.AtomicReference;
41 import org.junit.Test;
42 import org.junit.runners.Parameterized.Parameter;
43 import org.mockito.Mockito;
44 import org.opendaylight.controller.cluster.access.client.RequestTimeoutException;
45 import org.opendaylight.controller.cluster.databroker.ConcurrentDOMDataBroker;
46 import org.opendaylight.controller.cluster.datastore.TestShard.RequestFrontendMetadata;
47 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
48 import org.opendaylight.controller.cluster.datastore.messages.FindLocalShard;
49 import org.opendaylight.controller.cluster.datastore.messages.LocalShardFound;
50 import org.opendaylight.controller.cluster.datastore.persisted.DatastoreSnapshot;
51 import org.opendaylight.controller.cluster.datastore.persisted.FrontendClientMetadata;
52 import org.opendaylight.controller.cluster.datastore.persisted.FrontendShardDataTreeSnapshotMetadata;
53 import org.opendaylight.controller.cluster.datastore.persisted.MetadataShardDataTreeSnapshot;
54 import org.opendaylight.controller.cluster.datastore.persisted.ShardSnapshotState;
55 import org.opendaylight.controller.cluster.datastore.utils.MockDataTreeChangeListener;
56 import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
57 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
58 import org.opendaylight.controller.md.cluster.datastore.model.CarsModel;
59 import org.opendaylight.controller.md.cluster.datastore.model.PeopleModel;
60 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
61 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
62 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
63 import org.opendaylight.mdsal.dom.api.DOMDataTreeReadWriteTransaction;
64 import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteTransaction;
65 import org.opendaylight.mdsal.dom.api.DOMTransactionChain;
66 import org.opendaylight.mdsal.dom.api.DOMTransactionChainClosedException;
67 import org.opendaylight.mdsal.dom.api.DOMTransactionChainListener;
68 import org.opendaylight.mdsal.dom.spi.store.DOMStore;
69 import org.opendaylight.mdsal.dom.spi.store.DOMStoreReadTransaction;
70 import org.opendaylight.mdsal.dom.spi.store.DOMStoreReadWriteTransaction;
71 import org.opendaylight.mdsal.dom.spi.store.DOMStoreThreePhaseCommitCohort;
72 import org.opendaylight.mdsal.dom.spi.store.DOMStoreTransactionChain;
73 import org.opendaylight.mdsal.dom.spi.store.DOMStoreWriteTransaction;
74 import org.opendaylight.yangtools.concepts.ListenerRegistration;
75 import org.opendaylight.yangtools.yang.common.Uint64;
76 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
77 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
78 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
79 import org.opendaylight.yangtools.yang.data.api.schema.MapNode;
80 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
81 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
82 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeConfiguration;
83 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
84 import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder;
85 import org.opendaylight.yangtools.yang.data.impl.schema.tree.InMemoryDataTreeFactory;
86 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
87
88 public abstract class AbstractDistributedDataStoreIntegrationTest {
89
90     @Parameter
91     public Class<? extends AbstractDataStore> testParameter;
92
93     protected ActorSystem system;
94
95     protected final DatastoreContext.Builder datastoreContextBuilder = DatastoreContext.newBuilder()
96             .shardHeartbeatIntervalInMillis(100);
97
98     protected ActorSystem getSystem() {
99         return system;
100     }
101
102     @Test
103     public void testWriteTransactionWithSingleShard() throws Exception {
104         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
105         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
106             testParameter, "transactionIntegrationTest", "test-1")) {
107
108             testKit.testWriteTransaction(dataStore, TestModel.TEST_PATH,
109                 ImmutableNodes.containerNode(TestModel.TEST_QNAME));
110
111             testKit.testWriteTransaction(dataStore, TestModel.OUTER_LIST_PATH,
112                 ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME)
113                 .withChild(ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 42))
114                 .build());
115         }
116     }
117
118     @Test
119     public void testWriteTransactionWithMultipleShards() throws Exception {
120         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
121         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
122             testParameter, "testWriteTransactionWithMultipleShards", "cars-1", "people-1")) {
123
124             DOMStoreWriteTransaction writeTx = dataStore.newWriteOnlyTransaction();
125             assertNotNull("newWriteOnlyTransaction returned null", writeTx);
126
127             writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
128             writeTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
129
130             testKit.doCommit(writeTx.ready());
131
132             writeTx = dataStore.newWriteOnlyTransaction();
133
134             writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
135             writeTx.write(PeopleModel.PERSON_LIST_PATH, PeopleModel.newPersonMapNode());
136
137             testKit.doCommit(writeTx.ready());
138
139             writeTx = dataStore.newWriteOnlyTransaction();
140
141             final MapEntryNode car = CarsModel.newCarEntry("optima", Uint64.valueOf(20000));
142             final YangInstanceIdentifier carPath = CarsModel.newCarPath("optima");
143             writeTx.write(carPath, car);
144
145             final MapEntryNode person = PeopleModel.newPersonEntry("jack");
146             final YangInstanceIdentifier personPath = PeopleModel.newPersonPath("jack");
147             writeTx.write(personPath, person);
148
149             testKit.doCommit(writeTx.ready());
150
151             // Verify the data in the store
152             final DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
153
154             Optional<NormalizedNode> optional = readTx.read(carPath).get(5, TimeUnit.SECONDS);
155             assertTrue("isPresent", optional.isPresent());
156             assertEquals("Data node", car, optional.get());
157
158             optional = readTx.read(personPath).get(5, TimeUnit.SECONDS);
159             assertTrue("isPresent", optional.isPresent());
160             assertEquals("Data node", person, optional.get());
161         }
162     }
163
164     @Test
165     public void testReadWriteTransactionWithSingleShard() throws Exception {
166         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
167         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
168             testParameter, "testReadWriteTransactionWithSingleShard", "test-1")) {
169
170             // 1. Create a read-write Tx
171             final DOMStoreReadWriteTransaction readWriteTx = dataStore.newReadWriteTransaction();
172             assertNotNull("newReadWriteTransaction returned null", readWriteTx);
173
174             // 2. Write some data
175             final YangInstanceIdentifier nodePath = TestModel.TEST_PATH;
176             final NormalizedNode nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
177             readWriteTx.write(nodePath, nodeToWrite);
178
179             // 3. Read the data from Tx
180             final Boolean exists = readWriteTx.exists(nodePath).get(5, TimeUnit.SECONDS);
181             assertEquals("exists", Boolean.TRUE, exists);
182
183             Optional<NormalizedNode> optional = readWriteTx.read(nodePath).get(5, TimeUnit.SECONDS);
184             assertTrue("isPresent", optional.isPresent());
185             assertEquals("Data node", nodeToWrite, optional.get());
186
187             // 4. Ready the Tx for commit
188             final DOMStoreThreePhaseCommitCohort cohort = readWriteTx.ready();
189
190             // 5. Commit the Tx
191             testKit.doCommit(cohort);
192
193             // 6. Verify the data in the store
194             final DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
195
196             optional = readTx.read(nodePath).get(5, TimeUnit.SECONDS);
197             assertTrue("isPresent", optional.isPresent());
198             assertEquals("Data node", nodeToWrite, optional.get());
199         }
200     }
201
202     @Test
203     public void testReadWriteTransactionWithMultipleShards() throws Exception {
204         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
205         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
206             testParameter, "testReadWriteTransactionWithMultipleShards", "cars-1", "people-1")) {
207
208             DOMStoreReadWriteTransaction readWriteTx = dataStore.newReadWriteTransaction();
209             assertNotNull("newReadWriteTransaction returned null", readWriteTx);
210
211             readWriteTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
212             readWriteTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
213
214             testKit.doCommit(readWriteTx.ready());
215
216             readWriteTx = dataStore.newReadWriteTransaction();
217
218             readWriteTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
219             readWriteTx.write(PeopleModel.PERSON_LIST_PATH, PeopleModel.newPersonMapNode());
220
221             testKit.doCommit(readWriteTx.ready());
222
223             readWriteTx = dataStore.newReadWriteTransaction();
224
225             final MapEntryNode car = CarsModel.newCarEntry("optima", Uint64.valueOf(20000));
226             final YangInstanceIdentifier carPath = CarsModel.newCarPath("optima");
227             readWriteTx.write(carPath, car);
228
229             final MapEntryNode person = PeopleModel.newPersonEntry("jack");
230             final YangInstanceIdentifier personPath = PeopleModel.newPersonPath("jack");
231             readWriteTx.write(personPath, person);
232
233             final Boolean exists = readWriteTx.exists(carPath).get(5, TimeUnit.SECONDS);
234             assertEquals("exists", Boolean.TRUE, exists);
235
236             Optional<NormalizedNode> optional = readWriteTx.read(carPath).get(5, TimeUnit.SECONDS);
237             assertTrue("isPresent", optional.isPresent());
238             assertEquals("Data node", car, optional.get());
239
240             testKit.doCommit(readWriteTx.ready());
241
242             // Verify the data in the store
243             DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
244
245             optional = readTx.read(carPath).get(5, TimeUnit.SECONDS);
246             assertTrue("isPresent", optional.isPresent());
247             assertEquals("Data node", car, optional.get());
248
249             optional = readTx.read(personPath).get(5, TimeUnit.SECONDS);
250             assertTrue("isPresent", optional.isPresent());
251             assertEquals("Data node", person, optional.get());
252         }
253     }
254
255     @Test
256     public void testSingleTransactionsWritesInQuickSuccession() throws Exception {
257         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
258         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
259             testParameter, "testSingleTransactionsWritesInQuickSuccession", "cars-1")) {
260
261             final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
262
263             DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
264             writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
265             writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
266             testKit.doCommit(writeTx.ready());
267
268             int numCars = 5;
269             for (int i = 0; i < numCars; i++) {
270                 writeTx = txChain.newWriteOnlyTransaction();
271                 writeTx.write(CarsModel.newCarPath("car" + i), CarsModel.newCarEntry("car" + i, Uint64.valueOf(20000)));
272
273                 testKit.doCommit(writeTx.ready());
274
275                 try (var tx = txChain.newReadOnlyTransaction()) {
276                     tx.read(CarsModel.BASE_PATH).get();
277                 }
278             }
279
280             // wait to let the shard catch up with purged
281             await("transaction state propagation").atMost(5, TimeUnit.SECONDS)
282                 .pollInterval(500, TimeUnit.MILLISECONDS)
283                 .untilAsserted(() -> {
284                     // verify frontend metadata has no holes in purged transactions causing overtime memory leak
285                     final var localShard = dataStore.getActorUtils().findLocalShard("cars-1") .orElseThrow();
286                     FrontendShardDataTreeSnapshotMetadata frontendMetadata =
287                         (FrontendShardDataTreeSnapshotMetadata) dataStore.getActorUtils()
288                             .executeOperation(localShard, new RequestFrontendMetadata());
289
290                     final var clientMeta = frontendMetadata.getClients().get(0);
291                     if (dataStore.getActorUtils().getDatastoreContext().isUseTellBasedProtocol()) {
292                         assertTellMetadata(clientMeta);
293                     } else {
294                         assertAskMetadata(clientMeta);
295                     }
296                 });
297
298             final var body = txChain.newReadOnlyTransaction().read(CarsModel.CAR_LIST_PATH)
299                 .get(5, TimeUnit.SECONDS)
300                 .orElseThrow()
301                 .body();
302             assertThat(body, instanceOf(Collection.class));
303             assertEquals("# cars", numCars, ((Collection<?>) body).size());
304         }
305     }
306
307     private static void assertAskMetadata(final FrontendClientMetadata clientMeta) {
308         // FIXME: needs to be enabled
309         assumeFalse(true);
310         // ask based should track no metadata
311         assertEquals(List.of(), clientMeta.getCurrentHistories());
312     }
313
314     private static void assertTellMetadata(final FrontendClientMetadata clientMeta) {
315         final var iterator = clientMeta.getCurrentHistories().iterator();
316         var metadata = iterator.next();
317         while (iterator.hasNext() && metadata.getHistoryId() != 1) {
318             metadata = iterator.next();
319         }
320         assertEquals("[[0..10]]", metadata.getPurgedTransactions().ranges().toString());
321     }
322
323     @SuppressWarnings("checkstyle:IllegalCatch")
324     private void testTransactionCommitFailureWithNoShardLeader(final boolean writeOnly, final String testName)
325             throws Exception {
326         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
327         final String shardName = "default";
328
329         // We don't want the shard to become the leader so prevent shard
330         // elections.
331         datastoreContextBuilder.customRaftPolicyImplementation(
332                 "org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy");
333
334         // The ShardManager uses the election timeout for FindPrimary so
335         // reset it low so it will timeout quickly.
336         datastoreContextBuilder.shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(1)
337         .shardInitializationTimeout(200, TimeUnit.MILLISECONDS).frontendRequestTimeoutInSeconds(2);
338
339         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(testParameter, testName, false, shardName)) {
340
341             final Object result = dataStore.getActorUtils().executeOperation(
342                 dataStore.getActorUtils().getShardManager(), new FindLocalShard(shardName, true));
343             assertTrue("Expected LocalShardFound. Actual: " + result, result instanceof LocalShardFound);
344
345             // Create the write Tx.
346             DOMStoreWriteTransaction writeTxToClose = null;
347             try {
348                 writeTxToClose = writeOnly ? dataStore.newWriteOnlyTransaction()
349                         : dataStore.newReadWriteTransaction();
350                 final DOMStoreWriteTransaction writeTx = writeTxToClose;
351                 assertNotNull("newReadWriteTransaction returned null", writeTx);
352
353                 // Do some modifications and ready the Tx on a separate
354                 // thread.
355                 final AtomicReference<DOMStoreThreePhaseCommitCohort> txCohort = new AtomicReference<>();
356                 final AtomicReference<Exception> caughtEx = new AtomicReference<>();
357                 final CountDownLatch txReady = new CountDownLatch(1);
358                 final Thread txThread = new Thread(() -> {
359                     try {
360                         writeTx.write(TestModel.JUNK_PATH,
361                             ImmutableNodes.containerNode(TestModel.JUNK_QNAME));
362
363                         txCohort.set(writeTx.ready());
364                     } catch (Exception e) {
365                         caughtEx.set(e);
366                     } finally {
367                         txReady.countDown();
368                     }
369                 });
370
371                 txThread.start();
372
373                 // Wait for the Tx operations to complete.
374                 boolean done = Uninterruptibles.awaitUninterruptibly(txReady, 5, TimeUnit.SECONDS);
375                 if (caughtEx.get() != null) {
376                     throw caughtEx.get();
377                 }
378
379                 assertTrue("Tx ready", done);
380
381                 // Wait for the commit to complete. Since no shard
382                 // leader was elected in time, the Tx
383                 // should have timed out and throw an appropriate
384                 // exception cause.
385                 try {
386                     txCohort.get().canCommit().get(10, TimeUnit.SECONDS);
387                     fail("Expected NoShardLeaderException");
388                 } catch (final ExecutionException e) {
389                     final String msg = "Unexpected exception: "
390                             + Throwables.getStackTraceAsString(e.getCause());
391                     if (DistributedDataStore.class.isAssignableFrom(testParameter)) {
392                         assertTrue(Throwables.getRootCause(e) instanceof NoShardLeaderException);
393                     } else {
394                         assertTrue(msg, Throwables.getRootCause(e) instanceof RequestTimeoutException);
395                     }
396                 }
397             } finally {
398                 try {
399                     if (writeTxToClose != null) {
400                         writeTxToClose.close();
401                     }
402                 } catch (Exception e) {
403                     // FIXME TransactionProxy.close throws IllegalStateException:
404                     // Transaction is ready, it cannot be closed
405                 }
406             }
407         }
408     }
409
410     @Test
411     public void testWriteOnlyTransactionCommitFailureWithNoShardLeader() throws Exception {
412         datastoreContextBuilder.writeOnlyTransactionOptimizationsEnabled(true);
413         testTransactionCommitFailureWithNoShardLeader(true, "testWriteOnlyTransactionCommitFailureWithNoShardLeader");
414     }
415
416     @Test
417     public void testReadWriteTransactionCommitFailureWithNoShardLeader() throws Exception {
418         testTransactionCommitFailureWithNoShardLeader(false, "testReadWriteTransactionCommitFailureWithNoShardLeader");
419     }
420
421     @Test
422     public void testTransactionAbort() throws Exception {
423         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
424         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
425             testParameter, "transactionAbortIntegrationTest", "test-1")) {
426
427             final DOMStoreWriteTransaction writeTx = dataStore.newWriteOnlyTransaction();
428             assertNotNull("newWriteOnlyTransaction returned null", writeTx);
429
430             writeTx.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
431
432             final DOMStoreThreePhaseCommitCohort cohort = writeTx.ready();
433
434             cohort.canCommit().get(5, TimeUnit.SECONDS);
435
436             cohort.abort().get(5, TimeUnit.SECONDS);
437
438             testKit.testWriteTransaction(dataStore, TestModel.TEST_PATH,
439                 ImmutableNodes.containerNode(TestModel.TEST_QNAME));
440         }
441     }
442
443     @Test
444     @SuppressWarnings("checkstyle:IllegalCatch")
445     public void testTransactionChainWithSingleShard() throws Exception {
446         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
447         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
448             testParameter, "testTransactionChainWithSingleShard", "test-1")) {
449
450             // 1. Create a Tx chain and write-only Tx
451             final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
452
453             final DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
454             assertNotNull("newWriteOnlyTransaction returned null", writeTx);
455
456             // 2. Write some data
457             final NormalizedNode testNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
458             writeTx.write(TestModel.TEST_PATH, testNode);
459
460             // 3. Ready the Tx for commit
461             final DOMStoreThreePhaseCommitCohort cohort1 = writeTx.ready();
462
463             // 4. Commit the Tx on another thread that first waits for
464             // the second read Tx.
465             final CountDownLatch continueCommit1 = new CountDownLatch(1);
466             final CountDownLatch commit1Done = new CountDownLatch(1);
467             final AtomicReference<Exception> commit1Error = new AtomicReference<>();
468             new Thread(() -> {
469                 try {
470                     continueCommit1.await();
471                     testKit.doCommit(cohort1);
472                 } catch (Exception e) {
473                     commit1Error.set(e);
474                 } finally {
475                     commit1Done.countDown();
476                 }
477             }).start();
478
479             // 5. Create a new read Tx from the chain to read and verify
480             // the data from the first
481             // Tx is visible after being readied.
482             DOMStoreReadTransaction readTx = txChain.newReadOnlyTransaction();
483             Optional<NormalizedNode> optional = readTx.read(TestModel.TEST_PATH).get(5, TimeUnit.SECONDS);
484             assertTrue("isPresent", optional.isPresent());
485             assertEquals("Data node", testNode, optional.get());
486
487             // 6. Create a new RW Tx from the chain, write more data,
488             // and ready it
489             final DOMStoreReadWriteTransaction rwTx = txChain.newReadWriteTransaction();
490             final MapNode outerNode = ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME)
491                     .withChild(ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 42))
492                     .build();
493             rwTx.write(TestModel.OUTER_LIST_PATH, outerNode);
494
495             final DOMStoreThreePhaseCommitCohort cohort2 = rwTx.ready();
496
497             // 7. Create a new read Tx from the chain to read the data
498             // from the last RW Tx to
499             // verify it is visible.
500             readTx = txChain.newReadWriteTransaction();
501             optional = readTx.read(TestModel.OUTER_LIST_PATH).get(5, TimeUnit.SECONDS);
502             assertTrue("isPresent", optional.isPresent());
503             assertEquals("Data node", outerNode, optional.get());
504
505             // 8. Wait for the 2 commits to complete and close the
506             // chain.
507             continueCommit1.countDown();
508             Uninterruptibles.awaitUninterruptibly(commit1Done, 5, TimeUnit.SECONDS);
509
510             if (commit1Error.get() != null) {
511                 throw commit1Error.get();
512             }
513
514             testKit.doCommit(cohort2);
515
516             txChain.close();
517
518             // 9. Create a new read Tx from the data store and verify
519             // committed data.
520             readTx = dataStore.newReadOnlyTransaction();
521             optional = readTx.read(TestModel.OUTER_LIST_PATH).get(5, TimeUnit.SECONDS);
522             assertTrue("isPresent", optional.isPresent());
523             assertEquals("Data node", outerNode, optional.get());
524         }
525     }
526
527     @Test
528     public void testTransactionChainWithMultipleShards() throws Exception {
529         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
530         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
531             testParameter, "testTransactionChainWithMultipleShards", "cars-1", "people-1")) {
532
533             final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
534
535             DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
536             assertNotNull("newWriteOnlyTransaction returned null", writeTx);
537
538             writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
539             writeTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
540
541             writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
542             writeTx.write(PeopleModel.PERSON_LIST_PATH, PeopleModel.newPersonMapNode());
543
544             final DOMStoreThreePhaseCommitCohort cohort1 = writeTx.ready();
545
546             final DOMStoreReadWriteTransaction readWriteTx = txChain.newReadWriteTransaction();
547
548             final MapEntryNode car = CarsModel.newCarEntry("optima", Uint64.valueOf(20000));
549             final YangInstanceIdentifier carPath = CarsModel.newCarPath("optima");
550             readWriteTx.write(carPath, car);
551
552             final MapEntryNode person = PeopleModel.newPersonEntry("jack");
553             final YangInstanceIdentifier personPath = PeopleModel.newPersonPath("jack");
554             readWriteTx.merge(personPath, person);
555
556             Optional<NormalizedNode> optional = readWriteTx.read(carPath).get(5, TimeUnit.SECONDS);
557             assertTrue("isPresent", optional.isPresent());
558             assertEquals("Data node", car, optional.get());
559
560             optional = readWriteTx.read(personPath).get(5, TimeUnit.SECONDS);
561             assertTrue("isPresent", optional.isPresent());
562             assertEquals("Data node", person, optional.get());
563
564             final DOMStoreThreePhaseCommitCohort cohort2 = readWriteTx.ready();
565
566             writeTx = txChain.newWriteOnlyTransaction();
567
568             writeTx.delete(carPath);
569
570             final DOMStoreThreePhaseCommitCohort cohort3 = writeTx.ready();
571
572             final ListenableFuture<Boolean> canCommit1 = cohort1.canCommit();
573             final ListenableFuture<Boolean> canCommit2 = cohort2.canCommit();
574
575             testKit.doCommit(canCommit1, cohort1);
576             testKit.doCommit(canCommit2, cohort2);
577             testKit.doCommit(cohort3);
578
579             txChain.close();
580
581             final DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
582
583             optional = readTx.read(carPath).get(5, TimeUnit.SECONDS);
584             assertFalse("isPresent", optional.isPresent());
585
586             optional = readTx.read(personPath).get(5, TimeUnit.SECONDS);
587             assertTrue("isPresent", optional.isPresent());
588             assertEquals("Data node", person, optional.get());
589         }
590     }
591
592     @Test
593     public void testCreateChainedTransactionsInQuickSuccession() throws Exception {
594         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
595         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
596             testParameter, "testCreateChainedTransactionsInQuickSuccession", "cars-1")) {
597
598             final ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
599                 ImmutableMap.<LogicalDatastoreType, DOMStore>builder()
600                 .put(LogicalDatastoreType.CONFIGURATION, dataStore).build(),
601                 MoreExecutors.directExecutor());
602
603             final DOMTransactionChainListener listener = Mockito.mock(DOMTransactionChainListener.class);
604             DOMTransactionChain txChain = broker.createTransactionChain(listener);
605
606             final List<ListenableFuture<?>> futures = new ArrayList<>();
607
608             final DOMDataTreeWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
609             writeTx.put(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, CarsModel.emptyContainer());
610             writeTx.put(LogicalDatastoreType.CONFIGURATION, CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
611             futures.add(writeTx.commit());
612
613             int numCars = 100;
614             for (int i = 0; i < numCars; i++) {
615                 final DOMDataTreeReadWriteTransaction rwTx = txChain.newReadWriteTransaction();
616
617                 rwTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.newCarPath("car" + i),
618                     CarsModel.newCarEntry("car" + i, Uint64.valueOf(20000)));
619
620                 futures.add(rwTx.commit());
621             }
622
623             for (final ListenableFuture<?> f : futures) {
624                 f.get(5, TimeUnit.SECONDS);
625             }
626
627             final Optional<NormalizedNode> optional = txChain.newReadOnlyTransaction()
628                     .read(LogicalDatastoreType.CONFIGURATION, CarsModel.CAR_LIST_PATH).get(5, TimeUnit.SECONDS);
629             assertTrue("isPresent", optional.isPresent());
630             assertEquals("# cars", numCars, ((Collection<?>) optional.get().body()).size());
631
632             txChain.close();
633
634             broker.close();
635         }
636     }
637
638     @Test
639     public void testCreateChainedTransactionAfterEmptyTxReadied() throws Exception {
640         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
641         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
642             testParameter, "testCreateChainedTransactionAfterEmptyTxReadied", "test-1")) {
643
644             final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
645
646             final DOMStoreReadWriteTransaction rwTx1 = txChain.newReadWriteTransaction();
647
648             rwTx1.ready();
649
650             final DOMStoreReadWriteTransaction rwTx2 = txChain.newReadWriteTransaction();
651
652             final Optional<NormalizedNode> optional = rwTx2.read(TestModel.TEST_PATH).get(5, TimeUnit.SECONDS);
653             assertFalse("isPresent", optional.isPresent());
654
655             txChain.close();
656         }
657     }
658
659     @Test
660     public void testCreateChainedTransactionWhenPreviousNotReady() throws Exception {
661         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
662         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
663             testParameter, "testCreateChainedTransactionWhenPreviousNotReady", "test-1")) {
664
665             final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
666
667             final DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
668             assertNotNull("newWriteOnlyTransaction returned null", writeTx);
669
670             writeTx.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
671
672             // Try to create another Tx of each type - each should fail
673             // b/c the previous Tx wasn't
674             // readied.
675             testKit.assertExceptionOnTxChainCreates(txChain, IllegalStateException.class);
676         }
677     }
678
679     @Test
680     public void testCreateChainedTransactionAfterClose() throws Exception {
681         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
682         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
683             testParameter, "testCreateChainedTransactionAfterClose", "test-1")) {
684
685             final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
686             txChain.close();
687
688             // Try to create another Tx of each type - should fail b/c
689             // the previous Tx was closed.
690             testKit.assertExceptionOnTxChainCreates(txChain, DOMTransactionChainClosedException.class);
691         }
692     }
693
694     @Test
695     public void testChainWithReadOnlyTxAfterPreviousReady() throws Exception {
696         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
697         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
698             testParameter, "testChainWithReadOnlyTxAfterPreviousReady", "test-1")) {
699
700             final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
701
702             // Create a write tx and submit.
703             final DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
704             writeTx.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
705             final DOMStoreThreePhaseCommitCohort cohort1 = writeTx.ready();
706
707             // Create read-only tx's and issue a read.
708             FluentFuture<Optional<NormalizedNode>> readFuture1 = txChain
709                     .newReadOnlyTransaction().read(TestModel.TEST_PATH);
710
711             FluentFuture<Optional<NormalizedNode>> readFuture2 = txChain
712                     .newReadOnlyTransaction().read(TestModel.TEST_PATH);
713
714             // Create another write tx and issue the write.
715             DOMStoreWriteTransaction writeTx2 = txChain.newWriteOnlyTransaction();
716             writeTx2.write(TestModel.OUTER_LIST_PATH,
717                 ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME)
718                 .withChild(ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 42))
719                 .build());
720
721             // Ensure the reads succeed.
722
723             assertTrue("isPresent", readFuture1.get(5, TimeUnit.SECONDS).isPresent());
724             assertTrue("isPresent", readFuture2.get(5, TimeUnit.SECONDS).isPresent());
725
726             // Ensure the writes succeed.
727             DOMStoreThreePhaseCommitCohort cohort2 = writeTx2.ready();
728
729             testKit.doCommit(cohort1);
730             testKit.doCommit(cohort2);
731
732             assertTrue("isPresent", txChain.newReadOnlyTransaction().read(TestModel.OUTER_LIST_PATH)
733                 .get(5, TimeUnit.SECONDS).isPresent());
734         }
735     }
736
737     @Test
738     public void testChainedTransactionFailureWithSingleShard() throws Exception {
739         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
740         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
741             testParameter, "testChainedTransactionFailureWithSingleShard", "cars-1")) {
742
743             final ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
744                 ImmutableMap.<LogicalDatastoreType, DOMStore>builder()
745                 .put(LogicalDatastoreType.CONFIGURATION, dataStore).build(),
746                 MoreExecutors.directExecutor());
747
748             final DOMTransactionChainListener listener = Mockito.mock(DOMTransactionChainListener.class);
749             final DOMTransactionChain txChain = broker.createTransactionChain(listener);
750
751             final DOMDataTreeReadWriteTransaction writeTx = txChain.newReadWriteTransaction();
752
753             writeTx.put(LogicalDatastoreType.CONFIGURATION, PeopleModel.BASE_PATH,
754                 PeopleModel.emptyContainer());
755
756             final ContainerNode invalidData = ImmutableContainerNodeBuilder.create()
757                     .withNodeIdentifier(new YangInstanceIdentifier.NodeIdentifier(CarsModel.BASE_QNAME))
758                     .withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build();
759
760             writeTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData);
761
762             try {
763                 writeTx.commit().get(5, TimeUnit.SECONDS);
764                 fail("Expected TransactionCommitFailedException");
765             } catch (final ExecutionException e) {
766                 // Expected
767             }
768
769             verify(listener, timeout(5000)).onTransactionChainFailed(eq(txChain), eq(writeTx),
770                 any(Throwable.class));
771
772             txChain.close();
773             broker.close();
774         }
775     }
776
777     @Test
778     public void testChainedTransactionFailureWithMultipleShards() throws Exception {
779         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
780         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
781             testParameter, "testChainedTransactionFailureWithMultipleShards", "cars-1", "people-1")) {
782
783             final ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
784                 ImmutableMap.<LogicalDatastoreType, DOMStore>builder()
785                 .put(LogicalDatastoreType.CONFIGURATION, dataStore).build(),
786                 MoreExecutors.directExecutor());
787
788             final DOMTransactionChainListener listener = Mockito.mock(DOMTransactionChainListener.class);
789             final DOMTransactionChain txChain = broker.createTransactionChain(listener);
790
791             final DOMDataTreeWriteTransaction writeTx = txChain.newReadWriteTransaction();
792
793             writeTx.put(LogicalDatastoreType.CONFIGURATION, PeopleModel.BASE_PATH,
794                 PeopleModel.emptyContainer());
795
796             final ContainerNode invalidData = ImmutableContainerNodeBuilder.create()
797                     .withNodeIdentifier(new YangInstanceIdentifier.NodeIdentifier(CarsModel.BASE_QNAME))
798                     .withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build();
799
800             writeTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData);
801
802             // Note that merge will validate the data and fail but put
803             // succeeds b/c deep validation is not
804             // done for put for performance reasons.
805             try {
806                 writeTx.commit().get(5, TimeUnit.SECONDS);
807                 fail("Expected TransactionCommitFailedException");
808             } catch (final ExecutionException e) {
809                 // Expected
810             }
811
812             verify(listener, timeout(5000)).onTransactionChainFailed(eq(txChain), eq(writeTx),
813                 any(Throwable.class));
814
815             txChain.close();
816             broker.close();
817         }
818     }
819
820     @Test
821     public void testDataTreeChangeListenerRegistration() throws Exception {
822         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
823         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
824             testParameter, "testDataTreeChangeListenerRegistration", "test-1")) {
825
826             testKit.testWriteTransaction(dataStore, TestModel.TEST_PATH,
827                 ImmutableNodes.containerNode(TestModel.TEST_QNAME));
828
829             final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
830
831             ListenerRegistration<MockDataTreeChangeListener> listenerReg = dataStore
832                     .registerTreeChangeListener(TestModel.TEST_PATH, listener);
833
834             assertNotNull("registerTreeChangeListener returned null", listenerReg);
835
836             IntegrationTestKit.verifyShardState(dataStore, "test-1",
837                 state -> assertEquals("getTreeChangeListenerActors", 1,
838                         state.getTreeChangeListenerActors().size()));
839
840             // Wait for the initial notification
841             listener.waitForChangeEvents(TestModel.TEST_PATH);
842             listener.reset(2);
843
844             // Write 2 updates.
845             testKit.testWriteTransaction(dataStore, TestModel.OUTER_LIST_PATH,
846                 ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME)
847                 .withChild(ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 42))
848                 .build());
849
850             YangInstanceIdentifier listPath = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
851                     .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build();
852             testKit.testWriteTransaction(dataStore, listPath,
853                 ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1));
854
855             // Wait for the 2 updates.
856             listener.waitForChangeEvents(TestModel.OUTER_LIST_PATH, listPath);
857             listenerReg.close();
858
859             IntegrationTestKit.verifyShardState(dataStore, "test-1",
860                 state -> assertEquals("getTreeChangeListenerActors", 0,
861                     state.getTreeChangeListenerActors().size()));
862
863             testKit.testWriteTransaction(dataStore,
864                 YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
865                 .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2).build(),
866                 ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2));
867
868             listener.expectNoMoreChanges("Received unexpected change after close");
869         }
870     }
871
872     @Test
873     public void testRestoreFromDatastoreSnapshot() throws Exception {
874         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
875         final String name = "transactionIntegrationTest";
876
877         final ContainerNode carsNode = CarsModel.newCarsNode(
878             CarsModel.newCarsMapNode(CarsModel.newCarEntry("optima", Uint64.valueOf(20000)),
879                 CarsModel.newCarEntry("sportage", Uint64.valueOf(30000))));
880
881         DataTree dataTree = new InMemoryDataTreeFactory().create(
882             DataTreeConfiguration.DEFAULT_OPERATIONAL, SchemaContextHelper.full());
883         AbstractShardTest.writeToStore(dataTree, CarsModel.BASE_PATH, carsNode);
884         NormalizedNode root = AbstractShardTest.readStore(dataTree, YangInstanceIdentifier.empty());
885
886         final Snapshot carsSnapshot = Snapshot.create(
887             new ShardSnapshotState(new MetadataShardDataTreeSnapshot(root)),
888             Collections.emptyList(), 2, 1, 2, 1, 1, "member-1", null);
889
890         dataTree = new InMemoryDataTreeFactory().create(DataTreeConfiguration.DEFAULT_OPERATIONAL,
891             SchemaContextHelper.full());
892
893         final NormalizedNode peopleNode = PeopleModel.create();
894         AbstractShardTest.writeToStore(dataTree, PeopleModel.BASE_PATH, peopleNode);
895
896         root = AbstractShardTest.readStore(dataTree, YangInstanceIdentifier.empty());
897
898         final Snapshot peopleSnapshot = Snapshot.create(
899             new ShardSnapshotState(new MetadataShardDataTreeSnapshot(root)),
900             Collections.emptyList(), 2, 1, 2, 1, 1, "member-1", null);
901
902         testKit.restoreFromSnapshot = new DatastoreSnapshot(name, null, Arrays.asList(
903             new DatastoreSnapshot.ShardSnapshot("cars", carsSnapshot),
904             new DatastoreSnapshot.ShardSnapshot("people", peopleSnapshot)));
905
906         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
907             testParameter, name, "module-shards-member1.conf", true, "cars", "people")) {
908
909             final DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
910
911             // two reads
912             Optional<NormalizedNode> optional = readTx.read(CarsModel.BASE_PATH).get(5, TimeUnit.SECONDS);
913             assertTrue("isPresent", optional.isPresent());
914             assertEquals("Data node", carsNode, optional.get());
915
916             optional = readTx.read(PeopleModel.BASE_PATH).get(5, TimeUnit.SECONDS);
917             assertTrue("isPresent", optional.isPresent());
918             assertEquals("Data node", peopleNode, optional.get());
919         }
920     }
921
922     @Test
923     public void testSnapshotOnRootOverwrite() throws Exception {
924         if (!DistributedDataStore.class.isAssignableFrom(testParameter)) {
925             // FIXME: ClientBackedDatastore does not have stable indexes/term, the snapshot index seems to fluctuate
926             return;
927         }
928
929         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(),
930                 datastoreContextBuilder.snapshotOnRootOverwrite(true));
931         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
932                 testParameter, "testRootOverwrite", "module-shards-default-cars-member1.conf",
933                 true, "cars", "default")) {
934
935             ContainerNode rootNode = ImmutableContainerNodeBuilder.create()
936                     .withNodeIdentifier(YangInstanceIdentifier.NodeIdentifier.create(SchemaContext.NAME))
937                     .withChild(CarsModel.create())
938                     .build();
939
940             testKit.testWriteTransaction(dataStore, YangInstanceIdentifier.empty(), rootNode);
941             IntegrationTestKit.verifyShardState(dataStore, "cars",
942                 state -> assertEquals(1, state.getSnapshotIndex()));
943
944             // root has been written expect snapshot at index 0
945             verifySnapshot("member-1-shard-cars-testRootOverwrite", 1, 1);
946
947             for (int i = 0; i < 10; i++) {
948                 testKit.testWriteTransaction(dataStore, CarsModel.newCarPath("car " + i),
949                     CarsModel.newCarEntry("car " + i, Uint64.ONE));
950             }
951
952             // fake snapshot causes the snapshotIndex to move
953             IntegrationTestKit.verifyShardState(dataStore, "cars",
954                 state -> assertEquals(10, state.getSnapshotIndex()));
955
956             // however the real snapshot still has not changed and was taken at index 0
957             verifySnapshot("member-1-shard-cars-testRootOverwrite", 1, 1);
958
959             // root overwrite so expect a snapshot
960             testKit.testWriteTransaction(dataStore, YangInstanceIdentifier.empty(), rootNode);
961
962             // this was a real snapshot so everything should be in it(1 + 10 + 1)
963             IntegrationTestKit.verifyShardState(dataStore, "cars",
964                 state -> assertEquals(12, state.getSnapshotIndex()));
965
966             verifySnapshot("member-1-shard-cars-testRootOverwrite", 12, 1);
967         }
968     }
969
970     private static void verifySnapshot(final String persistenceId, final long lastAppliedIndex,
971             final long lastAppliedTerm) {
972         await().atMost(5, TimeUnit.SECONDS).untilAsserted(() -> {
973                 List<Snapshot> snap = InMemorySnapshotStore.getSnapshots(persistenceId, Snapshot.class);
974                 assertEquals(1, snap.size());
975                 assertEquals(lastAppliedIndex, snap.get(0).getLastAppliedIndex());
976                 assertEquals(lastAppliedTerm, snap.get(0).getLastAppliedTerm());
977             }
978         );
979     }
980 }