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