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