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