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