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