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