Remove deprecated Snapshot and related code
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / DistributedDataStoreIntegrationTest.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
9 package org.opendaylight.controller.cluster.datastore;
10
11 import static org.junit.Assert.assertEquals;
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.Matchers.any;
16 import static org.mockito.Matchers.eq;
17 import static org.mockito.Mockito.timeout;
18 import static org.mockito.Mockito.verify;
19
20 import akka.actor.ActorSystem;
21 import akka.actor.Address;
22 import akka.actor.AddressFromURIString;
23 import akka.cluster.Cluster;
24 import akka.testkit.JavaTestKit;
25 import com.google.common.base.Optional;
26 import com.google.common.base.Throwables;
27 import com.google.common.collect.ImmutableMap;
28 import com.google.common.util.concurrent.CheckedFuture;
29 import com.google.common.util.concurrent.ListenableFuture;
30 import com.google.common.util.concurrent.MoreExecutors;
31 import com.google.common.util.concurrent.Uninterruptibles;
32 import com.typesafe.config.ConfigFactory;
33 import java.io.IOException;
34 import java.math.BigInteger;
35 import java.util.ArrayList;
36 import java.util.Arrays;
37 import java.util.Collection;
38 import java.util.Collections;
39 import java.util.List;
40 import java.util.concurrent.CountDownLatch;
41 import java.util.concurrent.ExecutionException;
42 import java.util.concurrent.TimeUnit;
43 import java.util.concurrent.atomic.AtomicReference;
44 import org.junit.After;
45 import org.junit.Before;
46 import org.junit.Test;
47 import org.junit.runner.RunWith;
48 import org.junit.runners.Parameterized;
49 import org.junit.runners.Parameterized.Parameter;
50 import org.junit.runners.Parameterized.Parameters;
51 import org.mockito.Mockito;
52 import org.opendaylight.controller.cluster.databroker.ClientBackedDataStore;
53 import org.opendaylight.controller.cluster.databroker.ConcurrentDOMDataBroker;
54 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
55 import org.opendaylight.controller.cluster.datastore.exceptions.NotInitializedException;
56 import org.opendaylight.controller.cluster.datastore.messages.FindLocalShard;
57 import org.opendaylight.controller.cluster.datastore.messages.LocalShardFound;
58 import org.opendaylight.controller.cluster.datastore.persisted.DatastoreSnapshot;
59 import org.opendaylight.controller.cluster.datastore.persisted.MetadataShardDataTreeSnapshot;
60 import org.opendaylight.controller.cluster.datastore.persisted.ShardSnapshotState;
61 import org.opendaylight.controller.cluster.datastore.utils.MockDataChangeListener;
62 import org.opendaylight.controller.cluster.datastore.utils.MockDataTreeChangeListener;
63 import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
64 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
65 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
66 import org.opendaylight.controller.md.cluster.datastore.model.CarsModel;
67 import org.opendaylight.controller.md.cluster.datastore.model.PeopleModel;
68 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
69 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
70 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataBroker.DataChangeScope;
71 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
72 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
73 import org.opendaylight.controller.md.sal.common.api.data.TransactionChainClosedException;
74 import org.opendaylight.controller.md.sal.common.api.data.TransactionChainListener;
75 import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
76 import org.opendaylight.controller.md.sal.dom.api.DOMDataReadWriteTransaction;
77 import org.opendaylight.controller.md.sal.dom.api.DOMDataWriteTransaction;
78 import org.opendaylight.controller.md.sal.dom.api.DOMTransactionChain;
79 import org.opendaylight.controller.sal.core.spi.data.DOMStore;
80 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
81 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
82 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
83 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
84 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
85 import org.opendaylight.yangtools.concepts.ListenerRegistration;
86 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
87 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
88 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
89 import org.opendaylight.yangtools.yang.data.api.schema.MapNode;
90 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
91 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
92 import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType;
93 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
94 import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder;
95 import org.opendaylight.yangtools.yang.data.impl.schema.tree.InMemoryDataTreeFactory;
96
97 @RunWith(Parameterized.class)
98 public class DistributedDataStoreIntegrationTest {
99
100     @Parameters(name = "{0}")
101     public static Collection<Object[]> data() {
102         return Arrays.asList(new Object[][] {
103                 { DistributedDataStore.class }, { ClientBackedDataStore.class }
104         });
105     }
106
107     @Parameter
108     public Class<? extends AbstractDataStore> testParameter;
109
110     private ActorSystem system;
111
112     private final DatastoreContext.Builder datastoreContextBuilder = DatastoreContext.newBuilder()
113             .shardHeartbeatIntervalInMillis(100);
114
115     @Before
116     public void setUp() throws IOException {
117         InMemorySnapshotStore.clear();
118         InMemoryJournal.clear();
119         system = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member1"));
120         Address member1Address = AddressFromURIString.parse("akka://cluster-test@127.0.0.1:2558");
121         Cluster.get(system).join(member1Address);
122     }
123
124     @After
125     public void tearDown() throws IOException {
126         JavaTestKit.shutdownActorSystem(system, null, Boolean.TRUE);
127         system = null;
128     }
129
130     protected ActorSystem getSystem() {
131         return system;
132     }
133
134     @Test
135     public void testWriteTransactionWithSingleShard() throws Exception {
136         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
137             {
138                 try (AbstractDataStore dataStore = setupAbstractDataStore(
139                         testParameter, "transactionIntegrationTest", "test-1")) {
140
141                     testWriteTransaction(dataStore, TestModel.TEST_PATH,
142                             ImmutableNodes.containerNode(TestModel.TEST_QNAME));
143
144                     testWriteTransaction(dataStore, TestModel.OUTER_LIST_PATH,
145                             ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build());
146                 }
147             }
148         };
149     }
150
151     @Test
152     public void testWriteTransactionWithMultipleShards() throws Exception {
153         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
154             {
155                 try (AbstractDataStore dataStore = setupAbstractDataStore(
156                         testParameter, "testWriteTransactionWithMultipleShards", "cars-1", "people-1")) {
157
158                     DOMStoreWriteTransaction writeTx = dataStore.newWriteOnlyTransaction();
159                     assertNotNull("newWriteOnlyTransaction returned null", writeTx);
160
161                     writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
162                     writeTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
163
164                     doCommit(writeTx.ready());
165
166                     writeTx = dataStore.newWriteOnlyTransaction();
167
168                     writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
169                     writeTx.write(PeopleModel.PERSON_LIST_PATH, PeopleModel.newPersonMapNode());
170
171                     doCommit(writeTx.ready());
172
173                     writeTx = dataStore.newWriteOnlyTransaction();
174
175                     final MapEntryNode car = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
176                     final YangInstanceIdentifier carPath = CarsModel.newCarPath("optima");
177                     writeTx.write(carPath, car);
178
179                     final MapEntryNode person = PeopleModel.newPersonEntry("jack");
180                     final YangInstanceIdentifier personPath = PeopleModel.newPersonPath("jack");
181                     writeTx.write(personPath, person);
182
183                     doCommit(writeTx.ready());
184
185                     // Verify the data in the store
186                     final DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
187
188                     Optional<NormalizedNode<?, ?>> optional = readTx.read(carPath).get(5, TimeUnit.SECONDS);
189                     assertEquals("isPresent", true, optional.isPresent());
190                     assertEquals("Data node", car, optional.get());
191
192                     optional = readTx.read(personPath).get(5, TimeUnit.SECONDS);
193                     assertEquals("isPresent", true, optional.isPresent());
194                     assertEquals("Data node", person, optional.get());
195                 }
196             }
197         };
198     }
199
200     @Test
201     public void testReadWriteTransactionWithSingleShard() throws Exception {
202         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
203             {
204                 try (AbstractDataStore dataStore = setupAbstractDataStore(
205                         testParameter, "testReadWriteTransactionWithSingleShard", "test-1")) {
206
207                     // 1. Create a read-write Tx
208                     final DOMStoreReadWriteTransaction readWriteTx = dataStore.newReadWriteTransaction();
209                     assertNotNull("newReadWriteTransaction returned null", readWriteTx);
210
211                     // 2. Write some data
212                     final YangInstanceIdentifier nodePath = TestModel.TEST_PATH;
213                     final NormalizedNode<?, ?> nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
214                     readWriteTx.write(nodePath, nodeToWrite);
215
216                     // 3. Read the data from Tx
217                     final Boolean exists = readWriteTx.exists(nodePath).checkedGet(5, TimeUnit.SECONDS);
218                     assertEquals("exists", true, exists);
219
220                     Optional<NormalizedNode<?, ?>> optional = readWriteTx.read(nodePath).get(5, TimeUnit.SECONDS);
221                     assertEquals("isPresent", true, optional.isPresent());
222                     assertEquals("Data node", nodeToWrite, optional.get());
223
224                     // 4. Ready the Tx for commit
225                     final DOMStoreThreePhaseCommitCohort cohort = readWriteTx.ready();
226
227                     // 5. Commit the Tx
228                     doCommit(cohort);
229
230                     // 6. Verify the data in the store
231                     final DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
232
233                     optional = readTx.read(nodePath).get(5, TimeUnit.SECONDS);
234                     assertEquals("isPresent", true, optional.isPresent());
235                     assertEquals("Data node", nodeToWrite, optional.get());
236                 }
237             }
238         };
239     }
240
241     @Test
242     public void testReadWriteTransactionWithMultipleShards() throws Exception {
243         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
244             {
245                 try (AbstractDataStore dataStore = setupAbstractDataStore(
246                         testParameter, "testReadWriteTransactionWithMultipleShards", "cars-1", "people-1")) {
247
248                     DOMStoreReadWriteTransaction readWriteTx = dataStore.newReadWriteTransaction();
249                     assertNotNull("newReadWriteTransaction returned null", readWriteTx);
250
251                     readWriteTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
252                     readWriteTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
253
254                     doCommit(readWriteTx.ready());
255
256                     readWriteTx = dataStore.newReadWriteTransaction();
257
258                     readWriteTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
259                     readWriteTx.write(PeopleModel.PERSON_LIST_PATH, PeopleModel.newPersonMapNode());
260
261                     doCommit(readWriteTx.ready());
262
263                     readWriteTx = dataStore.newReadWriteTransaction();
264
265                     final MapEntryNode car = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
266                     final YangInstanceIdentifier carPath = CarsModel.newCarPath("optima");
267                     readWriteTx.write(carPath, car);
268
269                     final MapEntryNode person = PeopleModel.newPersonEntry("jack");
270                     final YangInstanceIdentifier personPath = PeopleModel.newPersonPath("jack");
271                     readWriteTx.write(personPath, person);
272
273                     final Boolean exists = readWriteTx.exists(carPath).checkedGet(5, TimeUnit.SECONDS);
274                     assertEquals("exists", true, exists);
275
276                     Optional<NormalizedNode<?, ?>> optional = readWriteTx.read(carPath).get(5, TimeUnit.SECONDS);
277                     assertEquals("isPresent", true, optional.isPresent());
278                     assertEquals("Data node", car, optional.get());
279
280                     doCommit(readWriteTx.ready());
281
282                     // Verify the data in the store
283                     DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
284
285                     optional = readTx.read(carPath).get(5, TimeUnit.SECONDS);
286                     assertEquals("isPresent", true, optional.isPresent());
287                     assertEquals("Data node", car, optional.get());
288
289                     optional = readTx.read(personPath).get(5, TimeUnit.SECONDS);
290                     assertEquals("isPresent", true, optional.isPresent());
291                     assertEquals("Data node", person, optional.get());
292
293                 }
294             }
295         };
296     }
297
298     @Test
299     public void testSingleTransactionsWritesInQuickSuccession() throws Exception {
300         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
301             {
302                 try (AbstractDataStore dataStore = setupAbstractDataStore(
303                         testParameter, "testSingleTransactionsWritesInQuickSuccession", "cars-1")) {
304
305                     final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
306
307                     DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
308                     writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
309                     writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
310                     doCommit(writeTx.ready());
311
312                     writeTx = txChain.newWriteOnlyTransaction();
313
314                     int numCars = 5;
315                     for (int i = 0; i < numCars; i++) {
316                         writeTx.write(CarsModel.newCarPath("car" + i),
317                                 CarsModel.newCarEntry("car" + i, BigInteger.valueOf(20000)));
318                     }
319
320                     doCommit(writeTx.ready());
321
322                     final Optional<NormalizedNode<?, ?>> optional = txChain.newReadOnlyTransaction()
323                             .read(CarsModel.CAR_LIST_PATH).get(5, TimeUnit.SECONDS);
324                     assertEquals("isPresent", true, optional.isPresent());
325                     assertEquals("# cars", numCars, ((Collection<?>) optional.get().getValue()).size());
326                 }
327             }
328         };
329     }
330
331     @SuppressWarnings("checkstyle:IllegalCatch")
332     private void testTransactionWritesWithShardNotInitiallyReady(final String testName, final boolean writeOnly)
333             throws Exception {
334         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
335             {
336                 final String shardName = "test-1";
337
338                 // Setup the InMemoryJournal to block shard recovery to ensure
339                 // the shard isn't
340                 // initialized until we create and submit the write the Tx.
341                 final String persistentID = String.format("member-1-shard-%s-%s", shardName, testName);
342                 final CountDownLatch blockRecoveryLatch = new CountDownLatch(1);
343                 InMemoryJournal.addBlockReadMessagesLatch(persistentID, blockRecoveryLatch);
344
345                 try (AbstractDataStore dataStore = setupAbstractDataStore(
346                         testParameter, testName, false, shardName)) {
347
348                     // Create the write Tx
349                     final DOMStoreWriteTransaction writeTx = writeOnly ? dataStore.newWriteOnlyTransaction()
350                             : dataStore.newReadWriteTransaction();
351                     assertNotNull("newReadWriteTransaction returned null", writeTx);
352
353                     // Do some modification operations and ready the Tx on a
354                     // separate thread.
355                     final YangInstanceIdentifier listEntryPath = YangInstanceIdentifier
356                             .builder(TestModel.OUTER_LIST_PATH)
357                             .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build();
358
359                     final AtomicReference<DOMStoreThreePhaseCommitCohort> txCohort = new AtomicReference<>();
360                     final AtomicReference<Exception> caughtEx = new AtomicReference<>();
361                     final CountDownLatch txReady = new CountDownLatch(1);
362                     final Thread txThread = new Thread(() -> {
363                         try {
364                             writeTx.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
365
366                             writeTx.merge(TestModel.OUTER_LIST_PATH,
367                                     ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build());
368
369                             writeTx.write(listEntryPath,
370                                     ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1));
371
372                             writeTx.delete(listEntryPath);
373
374                             txCohort.set(writeTx.ready());
375                         } catch (Exception e) {
376                             caughtEx.set(e);
377                         } finally {
378                             txReady.countDown();
379                         }
380                     });
381
382                     txThread.start();
383
384                     // Wait for the Tx operations to complete.
385                     final boolean done = Uninterruptibles.awaitUninterruptibly(txReady, 5, TimeUnit.SECONDS);
386                     if (caughtEx.get() != null) {
387                         throw caughtEx.get();
388                     }
389
390                     assertEquals("Tx ready", true, done);
391
392                     // At this point the Tx operations should be waiting for the
393                     // shard to initialize so
394                     // trigger the latch to let the shard recovery to continue.
395                     blockRecoveryLatch.countDown();
396
397                     // Wait for the Tx commit to complete.
398                     doCommit(txCohort.get());
399
400                     // Verify the data in the store
401                     final DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
402
403                     Optional<NormalizedNode<?, ?>> optional = readTx.read(TestModel.TEST_PATH).get(5, TimeUnit.SECONDS);
404                     assertEquals("isPresent", true, optional.isPresent());
405
406                     optional = readTx.read(TestModel.OUTER_LIST_PATH).get(5, TimeUnit.SECONDS);
407                     assertEquals("isPresent", true, optional.isPresent());
408
409                     optional = readTx.read(listEntryPath).get(5, TimeUnit.SECONDS);
410                     assertEquals("isPresent", false, optional.isPresent());
411                 }
412             }
413         };
414     }
415
416     @Test
417     public void testWriteOnlyTransactionWithShardNotInitiallyReady() throws Exception {
418         datastoreContextBuilder.writeOnlyTransactionOptimizationsEnabled(true);
419         testTransactionWritesWithShardNotInitiallyReady("testWriteOnlyTransactionWithShardNotInitiallyReady", true);
420     }
421
422     @Test
423     public void testReadWriteTransactionWithShardNotInitiallyReady() throws Exception {
424         testTransactionWritesWithShardNotInitiallyReady("testReadWriteTransactionWithShardNotInitiallyReady", false);
425     }
426
427     @Test
428     @SuppressWarnings("checkstyle:IllegalCatch")
429     public void testTransactionReadsWithShardNotInitiallyReady() throws Exception {
430         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
431             {
432                 final String testName = "testTransactionReadsWithShardNotInitiallyReady";
433                 final String shardName = "test-1";
434
435                 // Setup the InMemoryJournal to block shard recovery to ensure
436                 // the shard isn't
437                 // initialized until we create the Tx.
438                 final String persistentID = String.format("member-1-shard-%s-%s", shardName, testName);
439                 final CountDownLatch blockRecoveryLatch = new CountDownLatch(1);
440                 InMemoryJournal.addBlockReadMessagesLatch(persistentID, blockRecoveryLatch);
441
442                 try (AbstractDataStore dataStore = setupAbstractDataStore(
443                         testParameter, testName, false, shardName)) {
444
445                     // Create the read-write Tx
446                     final DOMStoreReadWriteTransaction readWriteTx = dataStore.newReadWriteTransaction();
447                     assertNotNull("newReadWriteTransaction returned null", readWriteTx);
448
449                     // Do some reads on the Tx on a separate thread.
450                     final AtomicReference<CheckedFuture<Boolean, ReadFailedException>> txExistsFuture =
451                             new AtomicReference<>();
452                     final AtomicReference<CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException>>
453                             txReadFuture = new AtomicReference<>();
454                     final AtomicReference<Exception> caughtEx = new AtomicReference<>();
455                     final CountDownLatch txReadsDone = new CountDownLatch(1);
456                     final Thread txThread = new Thread(() -> {
457                         try {
458                             readWriteTx.write(TestModel.TEST_PATH,
459                                     ImmutableNodes.containerNode(TestModel.TEST_QNAME));
460
461                             txExistsFuture.set(readWriteTx.exists(TestModel.TEST_PATH));
462
463                             txReadFuture.set(readWriteTx.read(TestModel.TEST_PATH));
464                         } catch (Exception e) {
465                             caughtEx.set(e);
466                         } finally {
467                             txReadsDone.countDown();
468                         }
469                     });
470
471                     txThread.start();
472
473                     // Wait for the Tx operations to complete.
474                     boolean done = Uninterruptibles.awaitUninterruptibly(txReadsDone, 5, TimeUnit.SECONDS);
475                     if (caughtEx.get() != null) {
476                         throw caughtEx.get();
477                     }
478
479                     assertEquals("Tx reads done", true, done);
480
481                     // At this point the Tx operations should be waiting for the
482                     // shard to initialize so
483                     // trigger the latch to let the shard recovery to continue.
484                     blockRecoveryLatch.countDown();
485
486                     // Wait for the reads to complete and verify.
487                     assertEquals("exists", true, txExistsFuture.get().checkedGet(5, TimeUnit.SECONDS));
488                     assertEquals("read", true, txReadFuture.get().checkedGet(5, TimeUnit.SECONDS).isPresent());
489
490                     readWriteTx.close();
491                 }
492             }
493         };
494     }
495
496     @Test(expected = NotInitializedException.class)
497     @SuppressWarnings("checkstyle:IllegalCatch")
498     public void testTransactionCommitFailureWithShardNotInitialized() throws Exception {
499         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
500             {
501                 final String testName = "testTransactionCommitFailureWithShardNotInitialized";
502                 final String shardName = "test-1";
503
504                 // Set the shard initialization timeout low for the test.
505                 datastoreContextBuilder.shardInitializationTimeout(300, TimeUnit.MILLISECONDS);
506
507                 // Setup the InMemoryJournal to block shard recovery
508                 // indefinitely.
509                 final String persistentID = String.format("member-1-shard-%s-%s", shardName, testName);
510                 final CountDownLatch blockRecoveryLatch = new CountDownLatch(1);
511                 InMemoryJournal.addBlockReadMessagesLatch(persistentID, blockRecoveryLatch);
512
513                 InMemoryJournal.addEntry(persistentID, 1, "Dummy data so akka will read from persistence");
514
515                 final AbstractDataStore dataStore =
516                         setupAbstractDataStore(testParameter, testName, false, shardName);
517
518                 // Create the write Tx
519                 final DOMStoreWriteTransaction writeTx = dataStore.newWriteOnlyTransaction();
520                 assertNotNull("newReadWriteTransaction returned null", writeTx);
521
522                 // Do some modifications and ready the Tx on a separate
523                 // thread.
524                 final AtomicReference<DOMStoreThreePhaseCommitCohort> txCohort = new AtomicReference<>();
525                 final AtomicReference<Exception> caughtEx = new AtomicReference<>();
526                 final CountDownLatch txReady = new CountDownLatch(1);
527                 final Thread txThread = new Thread(() -> {
528                     try {
529                         writeTx.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
530
531                         txCohort.set(writeTx.ready());
532                     } catch (Exception e) {
533                         caughtEx.set(e);
534                     } finally {
535                         txReady.countDown();
536                     }
537                 });
538
539                 txThread.start();
540
541                 // Wait for the Tx operations to complete.
542                 boolean done = Uninterruptibles.awaitUninterruptibly(txReady, 5, TimeUnit.SECONDS);
543                 if (caughtEx.get() != null) {
544                     throw caughtEx.get();
545                 }
546
547                 assertEquals("Tx ready", true, done);
548
549                 // Wait for the commit to complete. Since the shard never
550                 // initialized, the Tx should
551                 // have timed out and throw an appropriate exception cause.
552                 try {
553                     txCohort.get().canCommit().get(5, TimeUnit.SECONDS);
554                     fail("Expected NotInitializedException");
555                 } catch (final Exception e) {
556                     Throwables.propagate(Throwables.getRootCause(e));
557                 } finally {
558                     blockRecoveryLatch.countDown();
559                 }
560             }
561         };
562     }
563
564     @Test(expected = NotInitializedException.class)
565     @SuppressWarnings("checkstyle:IllegalCatch")
566     public void testTransactionReadFailureWithShardNotInitialized() throws Exception {
567         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
568             {
569                 final String testName = "testTransactionReadFailureWithShardNotInitialized";
570                 final String shardName = "test-1";
571
572                 // Set the shard initialization timeout low for the test.
573                 datastoreContextBuilder.shardInitializationTimeout(300, TimeUnit.MILLISECONDS);
574
575                 // Setup the InMemoryJournal to block shard recovery
576                 // indefinitely.
577                 final String persistentID = String.format("member-1-shard-%s-%s", shardName, testName);
578                 final CountDownLatch blockRecoveryLatch = new CountDownLatch(1);
579                 InMemoryJournal.addBlockReadMessagesLatch(persistentID, blockRecoveryLatch);
580
581                 InMemoryJournal.addEntry(persistentID, 1, "Dummy data so akka will read from persistence");
582
583                 try (AbstractDataStore dataStore = setupAbstractDataStore(
584                         testParameter, testName, false, shardName)) {
585
586                     // Create the read-write Tx
587                     final DOMStoreReadWriteTransaction readWriteTx = dataStore.newReadWriteTransaction();
588                     assertNotNull("newReadWriteTransaction returned null", readWriteTx);
589
590                     // Do a read on the Tx on a separate thread.
591                     final AtomicReference<CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException>>
592                             txReadFuture = new AtomicReference<>();
593                     final AtomicReference<Exception> caughtEx = new AtomicReference<>();
594                     final CountDownLatch txReadDone = new CountDownLatch(1);
595                     final Thread txThread = new Thread(() -> {
596                         try {
597                             readWriteTx.write(TestModel.TEST_PATH,
598                                     ImmutableNodes.containerNode(TestModel.TEST_QNAME));
599
600                             txReadFuture.set(readWriteTx.read(TestModel.TEST_PATH));
601
602                             readWriteTx.close();
603                         } catch (Exception e) {
604                             caughtEx.set(e);
605                         } finally {
606                             txReadDone.countDown();
607                         }
608                     });
609
610                     txThread.start();
611
612                     // Wait for the Tx operations to complete.
613                     boolean done = Uninterruptibles.awaitUninterruptibly(txReadDone, 5, TimeUnit.SECONDS);
614                     if (caughtEx.get() != null) {
615                         throw caughtEx.get();
616                     }
617
618                     assertEquals("Tx read done", true, done);
619
620                     // Wait for the read to complete. Since the shard never
621                     // initialized, the Tx should
622                     // have timed out and throw an appropriate exception cause.
623                     try {
624                         txReadFuture.get().checkedGet(5, TimeUnit.SECONDS);
625                         fail("Expected NotInitializedException");
626                     } catch (final ReadFailedException e) {
627                         Throwables.propagate(Throwables.getRootCause(e));
628                     } finally {
629                         blockRecoveryLatch.countDown();
630                     }
631                 }
632             }
633         };
634     }
635
636     @SuppressWarnings("checkstyle:IllegalCatch")
637     private void testTransactionCommitFailureWithNoShardLeader(final boolean writeOnly, final String testName)
638             throws Exception {
639         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
640             {
641                 final String shardName = "default";
642
643                 // We don't want the shard to become the leader so prevent shard
644                 // elections.
645                 datastoreContextBuilder.customRaftPolicyImplementation(
646                         "org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy");
647
648                 // The ShardManager uses the election timeout for FindPrimary so
649                 // reset it low so it will timeout quickly.
650                 datastoreContextBuilder.shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(1)
651                         .shardInitializationTimeout(200, TimeUnit.MILLISECONDS);
652
653                 try (AbstractDataStore dataStore = setupAbstractDataStore(
654                         testParameter, testName, false, shardName)) {
655
656                     final Object result = dataStore.getActorContext().executeOperation(
657                             dataStore.getActorContext().getShardManager(), new FindLocalShard(shardName, true));
658                     assertTrue("Expected LocalShardFound. Actual: " + result, result instanceof LocalShardFound);
659
660                     // Create the write Tx.
661                     try (DOMStoreWriteTransaction writeTx = writeOnly ? dataStore.newWriteOnlyTransaction()
662                             : dataStore.newReadWriteTransaction()) {
663                         assertNotNull("newReadWriteTransaction returned null", writeTx);
664
665                         // Do some modifications and ready the Tx on a separate
666                         // thread.
667                         final AtomicReference<DOMStoreThreePhaseCommitCohort> txCohort = new AtomicReference<>();
668                         final AtomicReference<Exception> caughtEx = new AtomicReference<>();
669                         final CountDownLatch txReady = new CountDownLatch(1);
670                         final Thread txThread = new Thread(() -> {
671                             try {
672                                 writeTx.write(TestModel.JUNK_PATH,
673                                         ImmutableNodes.containerNode(TestModel.JUNK_QNAME));
674
675                                 txCohort.set(writeTx.ready());
676                             } catch (Exception e) {
677                                 caughtEx.set(e);
678                             } finally {
679                                 txReady.countDown();
680                             }
681                         });
682
683                         txThread.start();
684
685                         // Wait for the Tx operations to complete.
686                         boolean done = Uninterruptibles.awaitUninterruptibly(txReady, 5, TimeUnit.SECONDS);
687                         if (caughtEx.get() != null) {
688                             throw caughtEx.get();
689                         }
690
691                         assertEquals("Tx ready", true, done);
692
693                         // Wait for the commit to complete. Since no shard
694                         // leader was elected in time, the Tx
695                         // should have timed out and throw an appropriate
696                         // exception cause.
697                         try {
698                             txCohort.get().canCommit().get(5, TimeUnit.SECONDS);
699                             fail("Expected NoShardLeaderException");
700                         } catch (final ExecutionException e) {
701                             Throwables.propagate(Throwables.getRootCause(e));
702                         }
703                     }
704                 }
705             }
706         };
707     }
708
709     @Test(expected = NoShardLeaderException.class)
710     public void testWriteOnlyTransactionCommitFailureWithNoShardLeader() throws Exception {
711         datastoreContextBuilder.writeOnlyTransactionOptimizationsEnabled(true);
712         testTransactionCommitFailureWithNoShardLeader(true, "testWriteOnlyTransactionCommitFailureWithNoShardLeader");
713     }
714
715     @Test(expected = NoShardLeaderException.class)
716     public void testReadWriteTransactionCommitFailureWithNoShardLeader() throws Exception {
717         testTransactionCommitFailureWithNoShardLeader(false, "testReadWriteTransactionCommitFailureWithNoShardLeader");
718     }
719
720     @Test
721     public void testTransactionAbort() throws Exception {
722         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
723             {
724                 try (AbstractDataStore dataStore = setupAbstractDataStore(
725                         testParameter, "transactionAbortIntegrationTest", "test-1")) {
726
727                     final DOMStoreWriteTransaction writeTx = dataStore.newWriteOnlyTransaction();
728                     assertNotNull("newWriteOnlyTransaction returned null", writeTx);
729
730                     writeTx.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
731
732                     final DOMStoreThreePhaseCommitCohort cohort = writeTx.ready();
733
734                     cohort.canCommit().get(5, TimeUnit.SECONDS);
735
736                     cohort.abort().get(5, TimeUnit.SECONDS);
737
738                     testWriteTransaction(dataStore, TestModel.TEST_PATH,
739                             ImmutableNodes.containerNode(TestModel.TEST_QNAME));
740                 }
741             }
742         };
743     }
744
745     @Test
746     @SuppressWarnings("checkstyle:IllegalCatch")
747     public void testTransactionChainWithSingleShard() throws Exception {
748         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
749             {
750                 try (AbstractDataStore dataStore = setupAbstractDataStore(
751                         testParameter, "testTransactionChainWithSingleShard", "test-1")) {
752
753                     // 1. Create a Tx chain and write-only Tx
754                     final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
755
756                     final DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
757                     assertNotNull("newWriteOnlyTransaction returned null", writeTx);
758
759                     // 2. Write some data
760                     final NormalizedNode<?, ?> testNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
761                     writeTx.write(TestModel.TEST_PATH, testNode);
762
763                     // 3. Ready the Tx for commit
764                     final DOMStoreThreePhaseCommitCohort cohort1 = writeTx.ready();
765
766                     // 4. Commit the Tx on another thread that first waits for
767                     // the second read Tx.
768                     final CountDownLatch continueCommit1 = new CountDownLatch(1);
769                     final CountDownLatch commit1Done = new CountDownLatch(1);
770                     final AtomicReference<Exception> commit1Error = new AtomicReference<>();
771                     new Thread(() -> {
772                         try {
773                             continueCommit1.await();
774                             doCommit(cohort1);
775                         } catch (Exception e) {
776                             commit1Error.set(e);
777                         } finally {
778                             commit1Done.countDown();
779                         }
780                     }).start();
781
782                     // 5. Create a new read Tx from the chain to read and verify
783                     // the data from the first
784                     // Tx is visible after being readied.
785                     DOMStoreReadTransaction readTx = txChain.newReadOnlyTransaction();
786                     Optional<NormalizedNode<?, ?>> optional = readTx.read(TestModel.TEST_PATH).get(5, TimeUnit.SECONDS);
787                     assertEquals("isPresent", true, optional.isPresent());
788                     assertEquals("Data node", testNode, optional.get());
789
790                     // 6. Create a new RW Tx from the chain, write more data,
791                     // and ready it
792                     final DOMStoreReadWriteTransaction rwTx = txChain.newReadWriteTransaction();
793                     final MapNode outerNode = ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build();
794                     rwTx.write(TestModel.OUTER_LIST_PATH, outerNode);
795
796                     final DOMStoreThreePhaseCommitCohort cohort2 = rwTx.ready();
797
798                     // 7. Create a new read Tx from the chain to read the data
799                     // from the last RW Tx to
800                     // verify it is visible.
801                     readTx = txChain.newReadWriteTransaction();
802                     optional = readTx.read(TestModel.OUTER_LIST_PATH).get(5, TimeUnit.SECONDS);
803                     assertEquals("isPresent", true, optional.isPresent());
804                     assertEquals("Data node", outerNode, optional.get());
805
806                     // 8. Wait for the 2 commits to complete and close the
807                     // chain.
808                     continueCommit1.countDown();
809                     Uninterruptibles.awaitUninterruptibly(commit1Done, 5, TimeUnit.SECONDS);
810
811                     if (commit1Error.get() != null) {
812                         throw commit1Error.get();
813                     }
814
815                     doCommit(cohort2);
816
817                     txChain.close();
818
819                     // 9. Create a new read Tx from the data store and verify
820                     // committed data.
821                     readTx = dataStore.newReadOnlyTransaction();
822                     optional = readTx.read(TestModel.OUTER_LIST_PATH).get(5, TimeUnit.SECONDS);
823                     assertEquals("isPresent", true, optional.isPresent());
824                     assertEquals("Data node", outerNode, optional.get());
825                 }
826             }
827         };
828     }
829
830     @Test
831     public void testTransactionChainWithMultipleShards() throws Exception {
832         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
833             {
834                 try (AbstractDataStore dataStore = setupAbstractDataStore(
835                         testParameter, "testTransactionChainWithMultipleShards", "cars-1", "people-1")) {
836
837                     final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
838
839                     DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
840                     assertNotNull("newWriteOnlyTransaction returned null", writeTx);
841
842                     writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
843                     writeTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
844
845                     writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
846                     writeTx.write(PeopleModel.PERSON_LIST_PATH, PeopleModel.newPersonMapNode());
847
848                     final DOMStoreThreePhaseCommitCohort cohort1 = writeTx.ready();
849
850                     final DOMStoreReadWriteTransaction readWriteTx = txChain.newReadWriteTransaction();
851
852                     final MapEntryNode car = CarsModel.newCarEntry("optima", BigInteger.valueOf(20000));
853                     final YangInstanceIdentifier carPath = CarsModel.newCarPath("optima");
854                     readWriteTx.write(carPath, car);
855
856                     final MapEntryNode person = PeopleModel.newPersonEntry("jack");
857                     final YangInstanceIdentifier personPath = PeopleModel.newPersonPath("jack");
858                     readWriteTx.merge(personPath, person);
859
860                     Optional<NormalizedNode<?, ?>> optional = readWriteTx.read(carPath).get(5, TimeUnit.SECONDS);
861                     assertEquals("isPresent", true, optional.isPresent());
862                     assertEquals("Data node", car, optional.get());
863
864                     optional = readWriteTx.read(personPath).get(5, TimeUnit.SECONDS);
865                     assertEquals("isPresent", true, optional.isPresent());
866                     assertEquals("Data node", person, optional.get());
867
868                     final DOMStoreThreePhaseCommitCohort cohort2 = readWriteTx.ready();
869
870                     writeTx = txChain.newWriteOnlyTransaction();
871
872                     writeTx.delete(carPath);
873
874                     final DOMStoreThreePhaseCommitCohort cohort3 = writeTx.ready();
875
876                     final ListenableFuture<Boolean> canCommit1 = cohort1.canCommit();
877                     final ListenableFuture<Boolean> canCommit2 = cohort2.canCommit();
878
879                     doCommit(canCommit1, cohort1);
880                     doCommit(canCommit2, cohort2);
881                     doCommit(cohort3);
882
883                     txChain.close();
884
885                     final DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
886
887                     optional = readTx.read(carPath).get(5, TimeUnit.SECONDS);
888                     assertEquals("isPresent", false, optional.isPresent());
889
890                     optional = readTx.read(personPath).get(5, TimeUnit.SECONDS);
891                     assertEquals("isPresent", true, optional.isPresent());
892                     assertEquals("Data node", person, optional.get());
893                 }
894             }
895         };
896     }
897
898     @Test
899     public void testCreateChainedTransactionsInQuickSuccession() throws Exception {
900         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
901             {
902                 try (AbstractDataStore dataStore = setupAbstractDataStore(
903                         testParameter, "testCreateChainedTransactionsInQuickSuccession", "cars-1")) {
904
905                     final ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
906                             ImmutableMap.<LogicalDatastoreType, DOMStore>builder()
907                                     .put(LogicalDatastoreType.CONFIGURATION, dataStore).build(),
908                             MoreExecutors.directExecutor());
909
910                     final TransactionChainListener listener = Mockito.mock(TransactionChainListener.class);
911                     DOMTransactionChain txChain = broker.createTransactionChain(listener);
912
913                     final List<CheckedFuture<Void, TransactionCommitFailedException>> futures = new ArrayList<>();
914
915                     final DOMDataWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
916                     writeTx.put(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, CarsModel.emptyContainer());
917                     writeTx.put(LogicalDatastoreType.CONFIGURATION, CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
918                     futures.add(writeTx.submit());
919
920                     int numCars = 100;
921                     for (int i = 0; i < numCars; i++) {
922                         final DOMDataReadWriteTransaction rwTx = txChain.newReadWriteTransaction();
923
924                         rwTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.newCarPath("car" + i),
925                                 CarsModel.newCarEntry("car" + i, BigInteger.valueOf(20000)));
926
927                         futures.add(rwTx.submit());
928                     }
929
930                     for (final CheckedFuture<Void, TransactionCommitFailedException> f : futures) {
931                         f.checkedGet();
932                     }
933
934                     final Optional<NormalizedNode<?, ?>> optional = txChain.newReadOnlyTransaction()
935                             .read(LogicalDatastoreType.CONFIGURATION, CarsModel.CAR_LIST_PATH).get(5, TimeUnit.SECONDS);
936                     assertEquals("isPresent", true, optional.isPresent());
937                     assertEquals("# cars", numCars, ((Collection<?>) optional.get().getValue()).size());
938
939                     txChain.close();
940
941                     broker.close();
942                 }
943             }
944         };
945     }
946
947     @Test
948     public void testCreateChainedTransactionAfterEmptyTxReadied() throws Exception {
949         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
950             {
951                 try (AbstractDataStore dataStore = setupAbstractDataStore(
952                         testParameter, "testCreateChainedTransactionAfterEmptyTxReadied", "test-1")) {
953
954                     final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
955
956                     final DOMStoreReadWriteTransaction rwTx1 = txChain.newReadWriteTransaction();
957
958                     rwTx1.ready();
959
960                     final DOMStoreReadWriteTransaction rwTx2 = txChain.newReadWriteTransaction();
961
962                     final Optional<NormalizedNode<?, ?>> optional = rwTx2.read(TestModel.TEST_PATH).get(
963                             5, TimeUnit.SECONDS);
964                     assertEquals("isPresent", false, optional.isPresent());
965
966                     txChain.close();
967                 }
968             }
969         };
970     }
971
972     @Test
973     public void testCreateChainedTransactionWhenPreviousNotReady() throws Exception {
974         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
975             {
976                 try (AbstractDataStore dataStore = setupAbstractDataStore(
977                         testParameter, "testCreateChainedTransactionWhenPreviousNotReady", "test-1")) {
978
979                     final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
980
981                     final DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
982                     assertNotNull("newWriteOnlyTransaction returned null", writeTx);
983
984                     writeTx.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
985
986                     // Try to create another Tx of each type - each should fail
987                     // b/c the previous Tx wasn't
988                     // readied.
989                     assertExceptionOnTxChainCreates(txChain, IllegalStateException.class);
990                 }
991             }
992         };
993     }
994
995     @Test
996     public void testCreateChainedTransactionAfterClose() throws Exception {
997         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
998             {
999                 try (AbstractDataStore dataStore = setupAbstractDataStore(
1000                         testParameter, "testCreateChainedTransactionAfterClose", "test-1")) {
1001
1002                     final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
1003                     txChain.close();
1004
1005                     // Try to create another Tx of each type - should fail b/c
1006                     // the previous Tx was closed.
1007                     assertExceptionOnTxChainCreates(txChain, TransactionChainClosedException.class);
1008                 }
1009             }
1010         };
1011     }
1012
1013     @Test
1014     public void testChainWithReadOnlyTxAfterPreviousReady() throws Exception {
1015         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
1016             {
1017                 try (AbstractDataStore dataStore = setupAbstractDataStore(
1018                         testParameter, "testChainWithReadOnlyTxAfterPreviousReady", "test-1")) {
1019
1020                     final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
1021
1022                     // Create a write tx and submit.
1023                     final DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
1024                     writeTx.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
1025                     final DOMStoreThreePhaseCommitCohort cohort1 = writeTx.ready();
1026
1027                     // Create read-only tx's and issue a read.
1028                     CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readFuture1 = txChain
1029                             .newReadOnlyTransaction().read(TestModel.TEST_PATH);
1030
1031                     CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readFuture2 = txChain
1032                             .newReadOnlyTransaction().read(TestModel.TEST_PATH);
1033
1034                     // Create another write tx and issue the write.
1035                     DOMStoreWriteTransaction writeTx2 = txChain.newWriteOnlyTransaction();
1036                     writeTx2.write(TestModel.OUTER_LIST_PATH,
1037                             ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build());
1038
1039                     // Ensure the reads succeed.
1040
1041                     assertEquals("isPresent", true, readFuture1.checkedGet(5, TimeUnit.SECONDS).isPresent());
1042                     assertEquals("isPresent", true, readFuture2.checkedGet(5, TimeUnit.SECONDS).isPresent());
1043
1044                     // Ensure the writes succeed.
1045                     DOMStoreThreePhaseCommitCohort cohort2 = writeTx2.ready();
1046
1047                     doCommit(cohort1);
1048                     doCommit(cohort2);
1049
1050                     assertEquals("isPresent", true, txChain.newReadOnlyTransaction().read(TestModel.OUTER_LIST_PATH)
1051                             .checkedGet(5, TimeUnit.SECONDS).isPresent());
1052                 }
1053             }
1054         };
1055     }
1056
1057     @Test
1058     public void testChainedTransactionFailureWithSingleShard() throws Exception {
1059         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
1060             {
1061                 try (AbstractDataStore dataStore = setupAbstractDataStore(
1062                         testParameter, "testChainedTransactionFailureWithSingleShard", "cars-1")) {
1063
1064                     final ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
1065                             ImmutableMap.<LogicalDatastoreType, DOMStore>builder()
1066                                     .put(LogicalDatastoreType.CONFIGURATION, dataStore).build(),
1067                             MoreExecutors.directExecutor());
1068
1069                     final TransactionChainListener listener = Mockito.mock(TransactionChainListener.class);
1070                     final DOMTransactionChain txChain = broker.createTransactionChain(listener);
1071
1072                     final DOMDataReadWriteTransaction writeTx = txChain.newReadWriteTransaction();
1073
1074                     writeTx.put(LogicalDatastoreType.CONFIGURATION, PeopleModel.BASE_PATH,
1075                             PeopleModel.emptyContainer());
1076
1077                     final ContainerNode invalidData = ImmutableContainerNodeBuilder.create()
1078                             .withNodeIdentifier(new YangInstanceIdentifier.NodeIdentifier(CarsModel.BASE_QNAME))
1079                             .withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build();
1080
1081                     writeTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData);
1082
1083                     try {
1084                         writeTx.submit().checkedGet(5, TimeUnit.SECONDS);
1085                         fail("Expected TransactionCommitFailedException");
1086                     } catch (final TransactionCommitFailedException e) {
1087                         // Expected
1088                     }
1089
1090                     verify(listener, timeout(5000)).onTransactionChainFailed(eq(txChain), eq(writeTx),
1091                             any(Throwable.class));
1092
1093                     txChain.close();
1094                     broker.close();
1095                 }
1096             }
1097         };
1098     }
1099
1100     @Test
1101     public void testChainedTransactionFailureWithMultipleShards() throws Exception {
1102         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
1103             {
1104                 try (AbstractDataStore dataStore = setupAbstractDataStore(
1105                         testParameter, "testChainedTransactionFailureWithMultipleShards", "cars-1", "people-1")) {
1106
1107                     final ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
1108                             ImmutableMap.<LogicalDatastoreType, DOMStore>builder()
1109                                     .put(LogicalDatastoreType.CONFIGURATION, dataStore).build(),
1110                             MoreExecutors.directExecutor());
1111
1112                     final TransactionChainListener listener = Mockito.mock(TransactionChainListener.class);
1113                     final DOMTransactionChain txChain = broker.createTransactionChain(listener);
1114
1115                     final DOMDataReadWriteTransaction writeTx = txChain.newReadWriteTransaction();
1116
1117                     writeTx.put(LogicalDatastoreType.CONFIGURATION, PeopleModel.BASE_PATH,
1118                             PeopleModel.emptyContainer());
1119
1120                     final ContainerNode invalidData = ImmutableContainerNodeBuilder.create()
1121                             .withNodeIdentifier(new YangInstanceIdentifier.NodeIdentifier(CarsModel.BASE_QNAME))
1122                             .withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build();
1123
1124                     writeTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData);
1125
1126                     // Note that merge will validate the data and fail but put
1127                     // succeeds b/c deep validation is not
1128                     // done for put for performance reasons.
1129                     try {
1130                         writeTx.submit().checkedGet(5, TimeUnit.SECONDS);
1131                         fail("Expected TransactionCommitFailedException");
1132                     } catch (final TransactionCommitFailedException e) {
1133                         // Expected
1134                     }
1135
1136                     verify(listener, timeout(5000)).onTransactionChainFailed(eq(txChain), eq(writeTx),
1137                             any(Throwable.class));
1138
1139                     txChain.close();
1140                     broker.close();
1141                 }
1142             }
1143         };
1144     }
1145
1146     @Test
1147     public void testChangeListenerRegistration() throws Exception {
1148         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
1149             {
1150                 try (AbstractDataStore dataStore = setupAbstractDataStore(
1151                         testParameter, "testChangeListenerRegistration", "test-1")) {
1152
1153                     testWriteTransaction(dataStore, TestModel.TEST_PATH,
1154                             ImmutableNodes.containerNode(TestModel.TEST_QNAME));
1155
1156                     final MockDataChangeListener listener = new MockDataChangeListener(1);
1157
1158                     final ListenerRegistration<MockDataChangeListener> listenerReg = dataStore
1159                             .registerChangeListener(TestModel.TEST_PATH, listener, DataChangeScope.SUBTREE);
1160
1161                     assertNotNull("registerChangeListener returned null", listenerReg);
1162
1163                     IntegrationTestKit.verifyShardState(dataStore, "test-1",
1164                         state -> assertEquals("getDataChangeListenerActors", 1,
1165                                 state.getDataChangeListenerActors().size()));
1166
1167                     // Wait for the initial notification
1168                     listener.waitForChangeEvents(TestModel.TEST_PATH);
1169                     listener.reset(2);
1170
1171                     // Write 2 updates.
1172                     testWriteTransaction(dataStore, TestModel.OUTER_LIST_PATH,
1173                             ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build());
1174
1175                     YangInstanceIdentifier listPath = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
1176                             .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build();
1177                     testWriteTransaction(dataStore, listPath,
1178                             ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1));
1179
1180                     // Wait for the 2 updates.
1181                     listener.waitForChangeEvents(TestModel.OUTER_LIST_PATH, listPath);
1182                     listenerReg.close();
1183
1184                     IntegrationTestKit.verifyShardState(dataStore, "test-1",
1185                         state -> assertEquals("getDataChangeListenerActors", 0,
1186                                 state.getDataChangeListenerActors().size()));
1187
1188                     testWriteTransaction(dataStore,
1189                             YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
1190                                     .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2).build(),
1191                             ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2));
1192
1193                     listener.expectNoMoreChanges("Received unexpected change after close");
1194                 }
1195             }
1196         };
1197     }
1198
1199     @Test
1200     public void testDataTreeChangeListenerRegistration() throws Exception {
1201         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
1202             {
1203                 try (AbstractDataStore dataStore = setupAbstractDataStore(
1204                         testParameter, "testDataTreeChangeListenerRegistration", "test-1")) {
1205
1206                     testWriteTransaction(dataStore, TestModel.TEST_PATH,
1207                             ImmutableNodes.containerNode(TestModel.TEST_QNAME));
1208
1209                     final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
1210
1211                     ListenerRegistration<MockDataTreeChangeListener> listenerReg = dataStore
1212                             .registerTreeChangeListener(TestModel.TEST_PATH, listener);
1213
1214                     assertNotNull("registerTreeChangeListener returned null", listenerReg);
1215
1216                     IntegrationTestKit.verifyShardState(dataStore, "test-1",
1217                         state -> assertEquals("getTreeChangeListenerActors", 1,
1218                                 state.getTreeChangeListenerActors().size()));
1219
1220                     // Wait for the initial notification
1221                     listener.waitForChangeEvents(TestModel.TEST_PATH);
1222                     listener.reset(2);
1223
1224                     // Write 2 updates.
1225                     testWriteTransaction(dataStore, TestModel.OUTER_LIST_PATH,
1226                             ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build());
1227
1228                     YangInstanceIdentifier listPath = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
1229                             .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build();
1230                     testWriteTransaction(dataStore, listPath,
1231                             ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1));
1232
1233                     // Wait for the 2 updates.
1234                     listener.waitForChangeEvents(TestModel.OUTER_LIST_PATH, listPath);
1235                     listenerReg.close();
1236
1237                     IntegrationTestKit.verifyShardState(dataStore, "test-1",
1238                         state -> assertEquals("getTreeChangeListenerActors", 0,
1239                                 state.getTreeChangeListenerActors().size()));
1240
1241                     testWriteTransaction(dataStore,
1242                             YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
1243                                     .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2).build(),
1244                             ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2));
1245
1246                     listener.expectNoMoreChanges("Received unexpected change after close");
1247                 }
1248             }
1249         };
1250     }
1251
1252     @Test
1253     public void testRestoreFromDatastoreSnapshot() throws Exception {
1254         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
1255             {
1256                 final String name = "transactionIntegrationTest";
1257
1258                 final ContainerNode carsNode = CarsModel.newCarsNode(
1259                         CarsModel.newCarsMapNode(CarsModel.newCarEntry("optima", BigInteger.valueOf(20000L)),
1260                                 CarsModel.newCarEntry("sportage", BigInteger.valueOf(30000L))));
1261
1262                 DataTree dataTree = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL);
1263                 dataTree.setSchemaContext(SchemaContextHelper.full());
1264                 AbstractShardTest.writeToStore(dataTree, CarsModel.BASE_PATH, carsNode);
1265                 NormalizedNode<?, ?> root = AbstractShardTest.readStore(dataTree, YangInstanceIdentifier.EMPTY);
1266
1267                 final Snapshot carsSnapshot = Snapshot.create(
1268                         new ShardSnapshotState(new MetadataShardDataTreeSnapshot(root)),
1269                         Collections.emptyList(), 2, 1, 2, 1, 1, "member-1", null);
1270
1271                 dataTree = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL);
1272                 dataTree.setSchemaContext(SchemaContextHelper.full());
1273
1274                 final NormalizedNode<?, ?> peopleNode = PeopleModel.create();
1275                 AbstractShardTest.writeToStore(dataTree, PeopleModel.BASE_PATH, peopleNode);
1276
1277                 root = AbstractShardTest.readStore(dataTree, YangInstanceIdentifier.EMPTY);
1278
1279                 final Snapshot peopleSnapshot = Snapshot.create(
1280                         new ShardSnapshotState(new MetadataShardDataTreeSnapshot(root)),
1281                         Collections.emptyList(), 2, 1, 2, 1, 1, "member-1", null);
1282
1283                 restoreFromSnapshot = new DatastoreSnapshot(name, null, Arrays.asList(
1284                         new DatastoreSnapshot.ShardSnapshot("cars", carsSnapshot),
1285                         new DatastoreSnapshot.ShardSnapshot("people", peopleSnapshot)));
1286
1287                 try (AbstractDataStore dataStore = setupAbstractDataStore(
1288                         testParameter, name, "module-shards-member1.conf", true, "cars", "people")) {
1289
1290                     final DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
1291
1292                     // two reads
1293                     Optional<NormalizedNode<?, ?>> optional = readTx.read(CarsModel.BASE_PATH).get(5, TimeUnit.SECONDS);
1294                     assertEquals("isPresent", true, optional.isPresent());
1295                     assertEquals("Data node", carsNode, optional.get());
1296
1297                     optional = readTx.read(PeopleModel.BASE_PATH).get(5, TimeUnit.SECONDS);
1298                     assertEquals("isPresent", true, optional.isPresent());
1299                     assertEquals("Data node", peopleNode, optional.get());
1300                 }
1301             }
1302         };
1303     }
1304 }