Bump upstreams
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / AbstractDistributedDataStoreIntegrationTest.java
1 /*
2  * Copyright (c) 2014, 2017 Cisco Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.controller.cluster.datastore;
9
10 import static org.awaitility.Awaitility.await;
11 import static org.hamcrest.CoreMatchers.instanceOf;
12 import static org.hamcrest.MatcherAssert.assertThat;
13 import static org.junit.Assert.assertEquals;
14 import static org.junit.Assert.assertFalse;
15 import static org.junit.Assert.assertNotNull;
16 import static org.junit.Assert.assertThrows;
17 import static org.junit.Assert.assertTrue;
18 import static org.mockito.ArgumentMatchers.any;
19 import static org.mockito.Mockito.mock;
20 import static org.mockito.Mockito.timeout;
21 import static org.mockito.Mockito.verify;
22
23 import akka.actor.ActorSystem;
24 import com.google.common.base.Throwables;
25 import com.google.common.collect.ImmutableMap;
26 import com.google.common.util.concurrent.FluentFuture;
27 import com.google.common.util.concurrent.FutureCallback;
28 import com.google.common.util.concurrent.ListenableFuture;
29 import com.google.common.util.concurrent.MoreExecutors;
30 import com.google.common.util.concurrent.Uninterruptibles;
31 import java.util.ArrayList;
32 import java.util.Arrays;
33 import java.util.Collection;
34 import java.util.Collections;
35 import java.util.List;
36 import java.util.Optional;
37 import java.util.concurrent.CountDownLatch;
38 import java.util.concurrent.ExecutionException;
39 import java.util.concurrent.TimeUnit;
40 import java.util.concurrent.atomic.AtomicReference;
41 import org.junit.Ignore;
42 import org.junit.Test;
43 import org.junit.runners.Parameterized.Parameter;
44 import org.opendaylight.controller.cluster.access.client.RequestTimeoutException;
45 import org.opendaylight.controller.cluster.databroker.ConcurrentDOMDataBroker;
46 import org.opendaylight.controller.cluster.datastore.TestShard.RequestFrontendMetadata;
47 import org.opendaylight.controller.cluster.datastore.messages.FindLocalShard;
48 import org.opendaylight.controller.cluster.datastore.messages.LocalShardFound;
49 import org.opendaylight.controller.cluster.datastore.persisted.DatastoreSnapshot;
50 import org.opendaylight.controller.cluster.datastore.persisted.FrontendShardDataTreeSnapshotMetadata;
51 import org.opendaylight.controller.cluster.datastore.persisted.MetadataShardDataTreeSnapshot;
52 import org.opendaylight.controller.cluster.datastore.persisted.ShardSnapshotState;
53 import org.opendaylight.controller.cluster.datastore.utils.MockDataTreeChangeListener;
54 import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
55 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
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.mdsal.common.api.LogicalDatastoreType;
61 import org.opendaylight.mdsal.dom.api.DOMDataTreeReadWriteTransaction;
62 import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteTransaction;
63 import org.opendaylight.mdsal.dom.api.DOMTransactionChain;
64 import org.opendaylight.mdsal.dom.api.DOMTransactionChainClosedException;
65 import org.opendaylight.mdsal.dom.spi.store.DOMStore;
66 import org.opendaylight.mdsal.dom.spi.store.DOMStoreReadTransaction;
67 import org.opendaylight.mdsal.dom.spi.store.DOMStoreReadWriteTransaction;
68 import org.opendaylight.mdsal.dom.spi.store.DOMStoreThreePhaseCommitCohort;
69 import org.opendaylight.mdsal.dom.spi.store.DOMStoreTransactionChain;
70 import org.opendaylight.mdsal.dom.spi.store.DOMStoreWriteTransaction;
71 import org.opendaylight.yangtools.yang.common.Uint64;
72 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
73 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.NodeIdentifier;
74 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
75 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
76 import org.opendaylight.yangtools.yang.data.api.schema.MapNode;
77 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
78 import org.opendaylight.yangtools.yang.data.impl.schema.Builders;
79 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
80 import org.opendaylight.yangtools.yang.data.tree.api.DataTree;
81 import org.opendaylight.yangtools.yang.data.tree.api.DataTreeConfiguration;
82 import org.opendaylight.yangtools.yang.data.tree.impl.di.InMemoryDataTreeFactory;
83 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
84
85 public abstract class AbstractDistributedDataStoreIntegrationTest {
86
87     @Parameter
88     public Class<? extends AbstractDataStore> testParameter;
89
90     protected ActorSystem system;
91
92     protected final DatastoreContext.Builder datastoreContextBuilder = DatastoreContext.newBuilder()
93             .shardHeartbeatIntervalInMillis(100);
94
95     protected ActorSystem getSystem() {
96         return system;
97     }
98
99     @Test
100     public void testWriteTransactionWithSingleShard() throws Exception {
101         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
102         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
103             testParameter, "transactionIntegrationTest", "test-1")) {
104
105             testKit.testWriteTransaction(dataStore, TestModel.TEST_PATH,
106                 ImmutableNodes.containerNode(TestModel.TEST_QNAME));
107
108             testKit.testWriteTransaction(dataStore, TestModel.OUTER_LIST_PATH,
109                 ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME)
110                 .withChild(ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 42))
111                 .build());
112         }
113     }
114
115     @Test
116     public void testWriteTransactionWithMultipleShards() throws Exception {
117         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
118         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
119             testParameter, "testWriteTransactionWithMultipleShards", "cars-1", "people-1")) {
120
121             DOMStoreWriteTransaction writeTx = dataStore.newWriteOnlyTransaction();
122             assertNotNull("newWriteOnlyTransaction returned null", writeTx);
123
124             writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
125             writeTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
126
127             testKit.doCommit(writeTx.ready());
128
129             writeTx = dataStore.newWriteOnlyTransaction();
130
131             writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
132             writeTx.write(PeopleModel.PERSON_LIST_PATH, PeopleModel.newPersonMapNode());
133
134             testKit.doCommit(writeTx.ready());
135
136             writeTx = dataStore.newWriteOnlyTransaction();
137
138             final MapEntryNode car = CarsModel.newCarEntry("optima", Uint64.valueOf(20000));
139             final YangInstanceIdentifier carPath = CarsModel.newCarPath("optima");
140             writeTx.write(carPath, car);
141
142             final MapEntryNode person = PeopleModel.newPersonEntry("jack");
143             final YangInstanceIdentifier personPath = PeopleModel.newPersonPath("jack");
144             writeTx.write(personPath, person);
145
146             testKit.doCommit(writeTx.ready());
147
148             // Verify the data in the store
149             final DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
150
151             assertEquals(Optional.of(car), readTx.read(carPath).get(5, TimeUnit.SECONDS));
152             assertEquals(Optional.of(person), readTx.read(personPath).get(5, TimeUnit.SECONDS));
153         }
154     }
155
156     @Test
157     public void testReadWriteTransactionWithSingleShard() throws Exception {
158         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
159         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
160             testParameter, "testReadWriteTransactionWithSingleShard", "test-1")) {
161
162             // 1. Create a read-write Tx
163             final DOMStoreReadWriteTransaction readWriteTx = dataStore.newReadWriteTransaction();
164             assertNotNull("newReadWriteTransaction returned null", readWriteTx);
165
166             // 2. Write some data
167             final YangInstanceIdentifier nodePath = TestModel.TEST_PATH;
168             final NormalizedNode nodeToWrite = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
169             readWriteTx.write(nodePath, nodeToWrite);
170
171             // 3. Read the data from Tx
172             final Boolean exists = readWriteTx.exists(nodePath).get(5, TimeUnit.SECONDS);
173             assertEquals("exists", Boolean.TRUE, exists);
174
175             assertEquals(Optional.of(nodeToWrite), readWriteTx.read(nodePath).get(5, TimeUnit.SECONDS));
176
177             // 4. Ready the Tx for commit
178             final DOMStoreThreePhaseCommitCohort cohort = readWriteTx.ready();
179
180             // 5. Commit the Tx
181             testKit.doCommit(cohort);
182
183             // 6. Verify the data in the store
184             final DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
185
186             assertEquals(Optional.of(nodeToWrite), readTx.read(nodePath).get(5, TimeUnit.SECONDS));
187         }
188     }
189
190     @Test
191     public void testReadWriteTransactionWithMultipleShards() throws Exception {
192         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
193         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
194             testParameter, "testReadWriteTransactionWithMultipleShards", "cars-1", "people-1")) {
195
196             DOMStoreReadWriteTransaction readWriteTx = dataStore.newReadWriteTransaction();
197             assertNotNull("newReadWriteTransaction returned null", readWriteTx);
198
199             readWriteTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
200             readWriteTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
201
202             testKit.doCommit(readWriteTx.ready());
203
204             readWriteTx = dataStore.newReadWriteTransaction();
205
206             readWriteTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
207             readWriteTx.write(PeopleModel.PERSON_LIST_PATH, PeopleModel.newPersonMapNode());
208
209             testKit.doCommit(readWriteTx.ready());
210
211             readWriteTx = dataStore.newReadWriteTransaction();
212
213             final MapEntryNode car = CarsModel.newCarEntry("optima", Uint64.valueOf(20000));
214             final YangInstanceIdentifier carPath = CarsModel.newCarPath("optima");
215             readWriteTx.write(carPath, car);
216
217             final MapEntryNode person = PeopleModel.newPersonEntry("jack");
218             final YangInstanceIdentifier personPath = PeopleModel.newPersonPath("jack");
219             readWriteTx.write(personPath, person);
220
221             final Boolean exists = readWriteTx.exists(carPath).get(5, TimeUnit.SECONDS);
222             assertEquals("exists", Boolean.TRUE, exists);
223
224             assertEquals("Data node", Optional.of(car), readWriteTx.read(carPath).get(5, TimeUnit.SECONDS));
225
226             testKit.doCommit(readWriteTx.ready());
227
228             // Verify the data in the store
229             DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
230
231             assertEquals(Optional.of(car), readTx.read(carPath).get(5, TimeUnit.SECONDS));
232             assertEquals(Optional.of(person), readTx.read(personPath).get(5, TimeUnit.SECONDS));
233         }
234     }
235
236     @Test
237     public void testSingleTransactionsWritesInQuickSuccession() throws Exception {
238         final var testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
239         try (var dataStore = testKit.setupAbstractDataStore(testParameter,
240                 "testSingleTransactionsWritesInQuickSuccession", "cars-1")) {
241
242             final var txChain = dataStore.createTransactionChain();
243
244             var writeTx = txChain.newWriteOnlyTransaction();
245             writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
246             writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
247             testKit.doCommit(writeTx.ready());
248
249             int numCars = 5;
250             for (int i = 0; i < numCars; i++) {
251                 writeTx = txChain.newWriteOnlyTransaction();
252                 writeTx.write(CarsModel.newCarPath("car" + i), CarsModel.newCarEntry("car" + i, Uint64.valueOf(20000)));
253
254                 testKit.doCommit(writeTx.ready());
255
256                 try (var tx = txChain.newReadOnlyTransaction()) {
257                     tx.read(CarsModel.BASE_PATH).get();
258                 }
259             }
260
261             // wait to let the shard catch up with purged
262             await("transaction state propagation").atMost(5, TimeUnit.SECONDS)
263                 .pollInterval(500, TimeUnit.MILLISECONDS)
264                 .untilAsserted(() -> {
265                     // verify frontend metadata has no holes in purged transactions causing overtime memory leak
266                     final var localShard = dataStore.getActorUtils().findLocalShard("cars-1") .orElseThrow();
267                     final var frontendMetadata = (FrontendShardDataTreeSnapshotMetadata) dataStore.getActorUtils()
268                             .executeOperation(localShard, new RequestFrontendMetadata());
269
270                     final var clientMeta = frontendMetadata.getClients().get(0);
271                     final var iterator = clientMeta.getCurrentHistories().iterator();
272                     var metadata = iterator.next();
273                     while (iterator.hasNext() && metadata.getHistoryId() != 1) {
274                         metadata = iterator.next();
275                     }
276                     assertEquals("[[0..10]]", metadata.getPurgedTransactions().ranges().toString());
277                 });
278
279             final var body = txChain.newReadOnlyTransaction().read(CarsModel.CAR_LIST_PATH)
280                 .get(5, TimeUnit.SECONDS)
281                 .orElseThrow()
282                 .body();
283             assertThat(body, instanceOf(Collection.class));
284             assertEquals("# cars", numCars, ((Collection<?>) body).size());
285         }
286     }
287
288     @SuppressWarnings("checkstyle:IllegalCatch")
289     private void testTransactionCommitFailureWithNoShardLeader(final boolean writeOnly, final String testName)
290             throws Exception {
291         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
292         final String shardName = "default";
293
294         // We don't want the shard to become the leader so prevent shard
295         // elections.
296         datastoreContextBuilder.customRaftPolicyImplementation(
297                 "org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy");
298
299         // The ShardManager uses the election timeout for FindPrimary so
300         // reset it low so it will timeout quickly.
301         datastoreContextBuilder.shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(1)
302         .shardInitializationTimeout(200, TimeUnit.MILLISECONDS).frontendRequestTimeoutInSeconds(2);
303
304         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(testParameter, testName, false, shardName)) {
305
306             final Object result = dataStore.getActorUtils().executeOperation(
307                 dataStore.getActorUtils().getShardManager(), new FindLocalShard(shardName, true));
308             assertTrue("Expected LocalShardFound. Actual: " + result, result instanceof LocalShardFound);
309
310             // Create the write Tx.
311             DOMStoreWriteTransaction writeTxToClose = null;
312             try {
313                 writeTxToClose = writeOnly ? dataStore.newWriteOnlyTransaction()
314                         : dataStore.newReadWriteTransaction();
315                 final DOMStoreWriteTransaction writeTx = writeTxToClose;
316                 assertNotNull("newReadWriteTransaction returned null", writeTx);
317
318                 // Do some modifications and ready the Tx on a separate
319                 // thread.
320                 final AtomicReference<DOMStoreThreePhaseCommitCohort> txCohort = new AtomicReference<>();
321                 final AtomicReference<Exception> caughtEx = new AtomicReference<>();
322                 final CountDownLatch txReady = new CountDownLatch(1);
323                 final Thread txThread = new Thread(() -> {
324                     try {
325                         writeTx.write(TestModel.JUNK_PATH,
326                             ImmutableNodes.containerNode(TestModel.JUNK_QNAME));
327
328                         txCohort.set(writeTx.ready());
329                     } catch (Exception e) {
330                         caughtEx.set(e);
331                     } finally {
332                         txReady.countDown();
333                     }
334                 });
335
336                 txThread.start();
337
338                 // Wait for the Tx operations to complete.
339                 boolean done = Uninterruptibles.awaitUninterruptibly(txReady, 5, TimeUnit.SECONDS);
340                 if (caughtEx.get() != null) {
341                     throw caughtEx.get();
342                 }
343
344                 assertTrue("Tx ready", done);
345
346                 // Wait for the commit to complete. Since no shard
347                 // leader was elected in time, the Tx
348                 // should have timed out and throw an appropriate
349                 // exception cause.
350                 final var ex = assertThrows(ExecutionException.class,
351                     () -> txCohort.get().canCommit().get(10, TimeUnit.SECONDS));
352                 assertTrue("Unexpected exception: " + Throwables.getStackTraceAsString(ex.getCause()),
353                     Throwables.getRootCause(ex) instanceof RequestTimeoutException);
354             } finally {
355                 try {
356                     if (writeTxToClose != null) {
357                         writeTxToClose.close();
358                     }
359                 } catch (Exception e) {
360                     // FIXME TransactionProxy.close throws IllegalStateException:
361                     // Transaction is ready, it cannot be closed
362                 }
363             }
364         }
365     }
366
367     @Test
368     public void testWriteOnlyTransactionCommitFailureWithNoShardLeader() throws Exception {
369         datastoreContextBuilder.writeOnlyTransactionOptimizationsEnabled(true);
370         testTransactionCommitFailureWithNoShardLeader(true, "testWriteOnlyTransactionCommitFailureWithNoShardLeader");
371     }
372
373     @Test
374     public void testReadWriteTransactionCommitFailureWithNoShardLeader() throws Exception {
375         testTransactionCommitFailureWithNoShardLeader(false, "testReadWriteTransactionCommitFailureWithNoShardLeader");
376     }
377
378     @Test
379     public void testTransactionAbort() throws Exception {
380         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
381         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
382             testParameter, "transactionAbortIntegrationTest", "test-1")) {
383
384             final DOMStoreWriteTransaction writeTx = dataStore.newWriteOnlyTransaction();
385             assertNotNull("newWriteOnlyTransaction returned null", writeTx);
386
387             writeTx.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
388
389             final DOMStoreThreePhaseCommitCohort cohort = writeTx.ready();
390
391             cohort.canCommit().get(5, TimeUnit.SECONDS);
392
393             cohort.abort().get(5, TimeUnit.SECONDS);
394
395             testKit.testWriteTransaction(dataStore, TestModel.TEST_PATH,
396                 ImmutableNodes.containerNode(TestModel.TEST_QNAME));
397         }
398     }
399
400     @Test
401     @SuppressWarnings("checkstyle:IllegalCatch")
402     public void testTransactionChainWithSingleShard() throws Exception {
403         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
404         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
405             testParameter, "testTransactionChainWithSingleShard", "test-1")) {
406
407             // 1. Create a Tx chain and write-only Tx
408             final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
409
410             final DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
411             assertNotNull("newWriteOnlyTransaction returned null", writeTx);
412
413             // 2. Write some data
414             final NormalizedNode testNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
415             writeTx.write(TestModel.TEST_PATH, testNode);
416
417             // 3. Ready the Tx for commit
418             final DOMStoreThreePhaseCommitCohort cohort1 = writeTx.ready();
419
420             // 4. Commit the Tx on another thread that first waits for
421             // the second read Tx.
422             final CountDownLatch continueCommit1 = new CountDownLatch(1);
423             final CountDownLatch commit1Done = new CountDownLatch(1);
424             final AtomicReference<Exception> commit1Error = new AtomicReference<>();
425             new Thread(() -> {
426                 try {
427                     continueCommit1.await();
428                     testKit.doCommit(cohort1);
429                 } catch (Exception e) {
430                     commit1Error.set(e);
431                 } finally {
432                     commit1Done.countDown();
433                 }
434             }).start();
435
436             // 5. Create a new read Tx from the chain to read and verify
437             // the data from the first
438             // Tx is visible after being readied.
439             DOMStoreReadTransaction readTx = txChain.newReadOnlyTransaction();
440             assertEquals(Optional.of(testNode), readTx.read(TestModel.TEST_PATH).get(5, TimeUnit.SECONDS));
441
442             // 6. Create a new RW Tx from the chain, write more data,
443             // and ready it
444             final DOMStoreReadWriteTransaction rwTx = txChain.newReadWriteTransaction();
445             final MapNode outerNode = ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME)
446                     .withChild(ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 42))
447                     .build();
448             rwTx.write(TestModel.OUTER_LIST_PATH, outerNode);
449
450             final DOMStoreThreePhaseCommitCohort cohort2 = rwTx.ready();
451
452             // 7. Create a new read Tx from the chain to read the data
453             // from the last RW Tx to
454             // verify it is visible.
455             readTx = txChain.newReadWriteTransaction();
456             assertEquals(Optional.of(outerNode), readTx.read(TestModel.OUTER_LIST_PATH).get(5, TimeUnit.SECONDS));
457
458             // 8. Wait for the 2 commits to complete and close the
459             // chain.
460             continueCommit1.countDown();
461             Uninterruptibles.awaitUninterruptibly(commit1Done, 5, TimeUnit.SECONDS);
462
463             if (commit1Error.get() != null) {
464                 throw commit1Error.get();
465             }
466
467             testKit.doCommit(cohort2);
468
469             txChain.close();
470
471             // 9. Create a new read Tx from the data store and verify
472             // committed data.
473             readTx = dataStore.newReadOnlyTransaction();
474             assertEquals(Optional.of(outerNode), readTx.read(TestModel.OUTER_LIST_PATH).get(5, TimeUnit.SECONDS));
475         }
476     }
477
478     @Test
479     public void testTransactionChainWithMultipleShards() throws Exception {
480         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
481         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
482             testParameter, "testTransactionChainWithMultipleShards", "cars-1", "people-1")) {
483
484             final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
485
486             DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
487             assertNotNull("newWriteOnlyTransaction returned null", writeTx);
488
489             writeTx.write(CarsModel.BASE_PATH, CarsModel.emptyContainer());
490             writeTx.write(PeopleModel.BASE_PATH, PeopleModel.emptyContainer());
491
492             writeTx.write(CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
493             writeTx.write(PeopleModel.PERSON_LIST_PATH, PeopleModel.newPersonMapNode());
494
495             final DOMStoreThreePhaseCommitCohort cohort1 = writeTx.ready();
496
497             final DOMStoreReadWriteTransaction readWriteTx = txChain.newReadWriteTransaction();
498
499             final MapEntryNode car = CarsModel.newCarEntry("optima", Uint64.valueOf(20000));
500             final YangInstanceIdentifier carPath = CarsModel.newCarPath("optima");
501             readWriteTx.write(carPath, car);
502
503             final MapEntryNode person = PeopleModel.newPersonEntry("jack");
504             final YangInstanceIdentifier personPath = PeopleModel.newPersonPath("jack");
505             readWriteTx.merge(personPath, person);
506
507             assertEquals(Optional.of(car), readWriteTx.read(carPath).get(5, TimeUnit.SECONDS));
508             assertEquals(Optional.of(person), readWriteTx.read(personPath).get(5, TimeUnit.SECONDS));
509
510             final DOMStoreThreePhaseCommitCohort cohort2 = readWriteTx.ready();
511
512             writeTx = txChain.newWriteOnlyTransaction();
513
514             writeTx.delete(carPath);
515
516             final DOMStoreThreePhaseCommitCohort cohort3 = writeTx.ready();
517
518             final ListenableFuture<Boolean> canCommit1 = cohort1.canCommit();
519             final ListenableFuture<Boolean> canCommit2 = cohort2.canCommit();
520
521             testKit.doCommit(canCommit1, cohort1);
522             testKit.doCommit(canCommit2, cohort2);
523             testKit.doCommit(cohort3);
524
525             txChain.close();
526
527             final DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
528
529             assertEquals(Optional.empty(), readTx.read(carPath).get(5, TimeUnit.SECONDS));
530             assertEquals(Optional.of(person), readTx.read(personPath).get(5, TimeUnit.SECONDS));
531         }
532     }
533
534     @Test
535     public void testCreateChainedTransactionsInQuickSuccession() throws Exception {
536         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
537         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
538             testParameter, "testCreateChainedTransactionsInQuickSuccession", "cars-1")) {
539
540             final ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
541                 ImmutableMap.<LogicalDatastoreType, DOMStore>builder()
542                 .put(LogicalDatastoreType.CONFIGURATION, dataStore).build(),
543                 MoreExecutors.directExecutor());
544
545             DOMTransactionChain txChain = broker.createTransactionChain();
546
547             final List<ListenableFuture<?>> futures = new ArrayList<>();
548
549             final DOMDataTreeWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
550             writeTx.put(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, CarsModel.emptyContainer());
551             writeTx.put(LogicalDatastoreType.CONFIGURATION, CarsModel.CAR_LIST_PATH, CarsModel.newCarMapNode());
552             futures.add(writeTx.commit());
553
554             int numCars = 100;
555             for (int i = 0; i < numCars; i++) {
556                 final DOMDataTreeReadWriteTransaction rwTx = txChain.newReadWriteTransaction();
557
558                 rwTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.newCarPath("car" + i),
559                     CarsModel.newCarEntry("car" + i, Uint64.valueOf(20000)));
560
561                 futures.add(rwTx.commit());
562             }
563
564             for (final ListenableFuture<?> f : futures) {
565                 f.get(5, TimeUnit.SECONDS);
566             }
567
568             final Optional<NormalizedNode> optional = txChain.newReadOnlyTransaction()
569                     .read(LogicalDatastoreType.CONFIGURATION, CarsModel.CAR_LIST_PATH).get(5, TimeUnit.SECONDS);
570             assertTrue("isPresent", optional.isPresent());
571             assertEquals("# cars", numCars, ((Collection<?>) optional.orElseThrow().body()).size());
572
573             txChain.close();
574
575             broker.close();
576         }
577     }
578
579     @Test
580     public void testCreateChainedTransactionAfterEmptyTxReadied() throws Exception {
581         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
582         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
583             testParameter, "testCreateChainedTransactionAfterEmptyTxReadied", "test-1")) {
584
585             final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
586
587             final DOMStoreReadWriteTransaction rwTx1 = txChain.newReadWriteTransaction();
588
589             rwTx1.ready();
590
591             final DOMStoreReadWriteTransaction rwTx2 = txChain.newReadWriteTransaction();
592
593             final Optional<NormalizedNode> optional = rwTx2.read(TestModel.TEST_PATH).get(5, TimeUnit.SECONDS);
594             assertFalse("isPresent", optional.isPresent());
595
596             txChain.close();
597         }
598     }
599
600     @Test
601     public void testCreateChainedTransactionWhenPreviousNotReady() throws Exception {
602         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
603         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
604             testParameter, "testCreateChainedTransactionWhenPreviousNotReady", "test-1")) {
605
606             final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
607
608             final DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
609             assertNotNull("newWriteOnlyTransaction returned null", writeTx);
610
611             writeTx.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
612
613             // Try to create another Tx of each type - each should fail
614             // b/c the previous Tx wasn't
615             // readied.
616             testKit.assertExceptionOnTxChainCreates(txChain, IllegalStateException.class);
617         }
618     }
619
620     @Test
621     public void testCreateChainedTransactionAfterClose() throws Exception {
622         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
623         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
624             testParameter, "testCreateChainedTransactionAfterClose", "test-1")) {
625
626             final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
627             txChain.close();
628
629             // Try to create another Tx of each type - should fail b/c
630             // the previous Tx was closed.
631             testKit.assertExceptionOnTxChainCreates(txChain, DOMTransactionChainClosedException.class);
632         }
633     }
634
635     @Test
636     public void testChainWithReadOnlyTxAfterPreviousReady() throws Exception {
637         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
638         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
639             testParameter, "testChainWithReadOnlyTxAfterPreviousReady", "test-1")) {
640
641             final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
642
643             // Create a write tx and submit.
644             final DOMStoreWriteTransaction writeTx = txChain.newWriteOnlyTransaction();
645             writeTx.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
646             final DOMStoreThreePhaseCommitCohort cohort1 = writeTx.ready();
647
648             // Create read-only tx's and issue a read.
649             FluentFuture<Optional<NormalizedNode>> readFuture1 = txChain
650                     .newReadOnlyTransaction().read(TestModel.TEST_PATH);
651
652             FluentFuture<Optional<NormalizedNode>> readFuture2 = txChain
653                     .newReadOnlyTransaction().read(TestModel.TEST_PATH);
654
655             // Create another write tx and issue the write.
656             DOMStoreWriteTransaction writeTx2 = txChain.newWriteOnlyTransaction();
657             writeTx2.write(TestModel.OUTER_LIST_PATH,
658                 ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME)
659                 .withChild(ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 42))
660                 .build());
661
662             // Ensure the reads succeed.
663
664             assertTrue("isPresent", readFuture1.get(5, TimeUnit.SECONDS).isPresent());
665             assertTrue("isPresent", readFuture2.get(5, TimeUnit.SECONDS).isPresent());
666
667             // Ensure the writes succeed.
668             DOMStoreThreePhaseCommitCohort cohort2 = writeTx2.ready();
669
670             testKit.doCommit(cohort1);
671             testKit.doCommit(cohort2);
672
673             assertTrue("isPresent", txChain.newReadOnlyTransaction().read(TestModel.OUTER_LIST_PATH)
674                 .get(5, TimeUnit.SECONDS).isPresent());
675         }
676     }
677
678     @Test
679     public void testChainedTransactionFailureWithSingleShard() throws Exception {
680         final var testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
681         try (var dataStore = testKit.setupAbstractDataStore(testParameter,
682                 "testChainedTransactionFailureWithSingleShard", "cars-1")) {
683
684             final var broker = new ConcurrentDOMDataBroker(
685                 ImmutableMap.<LogicalDatastoreType, DOMStore>builder()
686                 .put(LogicalDatastoreType.CONFIGURATION, dataStore).build(),
687                 MoreExecutors.directExecutor());
688
689             final var listener = mock(FutureCallback.class);
690             final var txChain = broker.createTransactionChain();
691             txChain.addCallback(listener);
692
693             final var writeTx = txChain.newReadWriteTransaction();
694
695             writeTx.put(LogicalDatastoreType.CONFIGURATION, PeopleModel.BASE_PATH,
696                 PeopleModel.emptyContainer());
697
698             final var invalidData = Builders.containerBuilder()
699                     .withNodeIdentifier(new NodeIdentifier(CarsModel.BASE_QNAME))
700                     .withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk"))
701                     .build();
702
703             writeTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData);
704
705             assertThrows(ExecutionException.class, () -> writeTx.commit().get(5, TimeUnit.SECONDS));
706
707             verify(listener, timeout(5000)).onFailure(any());
708
709             txChain.close();
710             broker.close();
711         }
712     }
713
714     @Test
715     public void testChainedTransactionFailureWithMultipleShards() throws Exception {
716         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
717         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
718             testParameter, "testChainedTransactionFailureWithMultipleShards", "cars-1", "people-1")) {
719
720             final ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
721                 ImmutableMap.<LogicalDatastoreType, DOMStore>builder()
722                 .put(LogicalDatastoreType.CONFIGURATION, dataStore).build(),
723                 MoreExecutors.directExecutor());
724
725             final var listener = mock(FutureCallback.class);
726             final DOMTransactionChain txChain = broker.createTransactionChain();
727             txChain.addCallback(listener);
728
729             final DOMDataTreeWriteTransaction writeTx = txChain.newReadWriteTransaction();
730
731             writeTx.put(LogicalDatastoreType.CONFIGURATION, PeopleModel.BASE_PATH,
732                 PeopleModel.emptyContainer());
733
734             final ContainerNode invalidData = Builders.containerBuilder()
735                 .withNodeIdentifier(new NodeIdentifier(CarsModel.BASE_QNAME))
736                 .withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk"))
737                 .build();
738
739             writeTx.merge(LogicalDatastoreType.CONFIGURATION, CarsModel.BASE_PATH, invalidData);
740
741             // Note that merge will validate the data and fail but put
742             // succeeds b/c deep validation is not
743             // done for put for performance reasons.
744             assertThrows(ExecutionException.class, () -> writeTx.commit().get(5, TimeUnit.SECONDS));
745
746             verify(listener, timeout(5000)).onFailure(any());
747
748             txChain.close();
749             broker.close();
750         }
751     }
752
753     @Test
754     public void testDataTreeChangeListenerRegistration() throws Exception {
755         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
756         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
757             testParameter, "testDataTreeChangeListenerRegistration", "test-1")) {
758
759             testKit.testWriteTransaction(dataStore, TestModel.TEST_PATH,
760                 ImmutableNodes.containerNode(TestModel.TEST_QNAME));
761
762             final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
763
764             final var listenerReg = dataStore.registerTreeChangeListener(TestModel.TEST_PATH, listener);
765
766             assertNotNull("registerTreeChangeListener returned null", listenerReg);
767
768             IntegrationTestKit.verifyShardState(dataStore, "test-1",
769                 state -> assertEquals("getTreeChangeListenerActors", 1,
770                         state.getTreeChangeListenerActors().size()));
771
772             // Wait for the initial notification
773             listener.waitForChangeEvents(TestModel.TEST_PATH);
774             listener.reset(2);
775
776             // Write 2 updates.
777             testKit.testWriteTransaction(dataStore, TestModel.OUTER_LIST_PATH,
778                 ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME)
779                 .withChild(ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 42))
780                 .build());
781
782             YangInstanceIdentifier listPath = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
783                     .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build();
784             testKit.testWriteTransaction(dataStore, listPath,
785                 ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1));
786
787             // Wait for the 2 updates.
788             listener.waitForChangeEvents(TestModel.OUTER_LIST_PATH, listPath);
789             listenerReg.close();
790
791             IntegrationTestKit.verifyShardState(dataStore, "test-1",
792                 state -> assertEquals("getTreeChangeListenerActors", 0,
793                     state.getTreeChangeListenerActors().size()));
794
795             testKit.testWriteTransaction(dataStore,
796                 YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
797                 .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2).build(),
798                 ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2));
799
800             listener.expectNoMoreChanges("Received unexpected change after close");
801         }
802     }
803
804     @Test
805     public void testRestoreFromDatastoreSnapshot() throws Exception {
806         final IntegrationTestKit testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder);
807         final String name = "transactionIntegrationTest";
808
809         final ContainerNode carsNode = CarsModel.newCarsNode(
810             CarsModel.newCarsMapNode(CarsModel.newCarEntry("optima", Uint64.valueOf(20000)),
811                 CarsModel.newCarEntry("sportage", Uint64.valueOf(30000))));
812
813         DataTree dataTree = new InMemoryDataTreeFactory().create(
814             DataTreeConfiguration.DEFAULT_OPERATIONAL, SchemaContextHelper.full());
815         AbstractShardTest.writeToStore(dataTree, CarsModel.BASE_PATH, carsNode);
816         NormalizedNode root = AbstractShardTest.readStore(dataTree, YangInstanceIdentifier.of());
817
818         final Snapshot carsSnapshot = Snapshot.create(
819             new ShardSnapshotState(new MetadataShardDataTreeSnapshot(root)),
820             Collections.emptyList(), 2, 1, 2, 1, 1, "member-1", null);
821
822         dataTree = new InMemoryDataTreeFactory().create(DataTreeConfiguration.DEFAULT_OPERATIONAL,
823             SchemaContextHelper.full());
824
825         final NormalizedNode peopleNode = PeopleModel.create();
826         AbstractShardTest.writeToStore(dataTree, PeopleModel.BASE_PATH, peopleNode);
827
828         root = AbstractShardTest.readStore(dataTree, YangInstanceIdentifier.of());
829
830         final Snapshot peopleSnapshot = Snapshot.create(
831             new ShardSnapshotState(new MetadataShardDataTreeSnapshot(root)),
832             Collections.emptyList(), 2, 1, 2, 1, 1, "member-1", null);
833
834         testKit.restoreFromSnapshot = new DatastoreSnapshot(name, null, Arrays.asList(
835             new DatastoreSnapshot.ShardSnapshot("cars", carsSnapshot),
836             new DatastoreSnapshot.ShardSnapshot("people", peopleSnapshot)));
837
838         try (AbstractDataStore dataStore = testKit.setupAbstractDataStore(
839             testParameter, name, "module-shards-member1.conf", true, "cars", "people")) {
840
841             final DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
842
843             // two reads
844             assertEquals(Optional.of(carsNode), readTx.read(CarsModel.BASE_PATH).get(5, TimeUnit.SECONDS));
845             assertEquals(Optional.of(peopleNode), readTx.read(PeopleModel.BASE_PATH).get(5, TimeUnit.SECONDS));
846         }
847     }
848
849     @Test
850     @Ignore("ClientBackedDatastore does not have stable indexes/term, the snapshot index seems to fluctuate")
851     // FIXME: re-enable this test
852     public void testSnapshotOnRootOverwrite() throws Exception {
853         final var testKit = new IntegrationTestKit(getSystem(), datastoreContextBuilder.snapshotOnRootOverwrite(true));
854         try (var dataStore = testKit.setupAbstractDataStore(
855                 testParameter, "testRootOverwrite", "module-shards-default-cars-member1.conf",
856                 true, "cars", "default")) {
857
858             final var rootNode = Builders.containerBuilder()
859                 .withNodeIdentifier(NodeIdentifier.create(SchemaContext.NAME))
860                 .withChild(CarsModel.create())
861                 .build();
862
863             testKit.testWriteTransaction(dataStore, YangInstanceIdentifier.of(), rootNode);
864             IntegrationTestKit.verifyShardState(dataStore, "cars",
865                 state -> assertEquals(1, state.getSnapshotIndex()));
866
867             // root has been written expect snapshot at index 0
868             verifySnapshot("member-1-shard-cars-testRootOverwrite", 1, 1);
869
870             for (int i = 0; i < 10; i++) {
871                 testKit.testWriteTransaction(dataStore, CarsModel.newCarPath("car " + i),
872                     CarsModel.newCarEntry("car " + i, Uint64.ONE));
873             }
874
875             // fake snapshot causes the snapshotIndex to move
876             IntegrationTestKit.verifyShardState(dataStore, "cars",
877                 state -> assertEquals(10, state.getSnapshotIndex()));
878
879             // however the real snapshot still has not changed and was taken at index 0
880             verifySnapshot("member-1-shard-cars-testRootOverwrite", 1, 1);
881
882             // root overwrite so expect a snapshot
883             testKit.testWriteTransaction(dataStore, YangInstanceIdentifier.of(), rootNode);
884
885             // this was a real snapshot so everything should be in it(1 + 10 + 1)
886             IntegrationTestKit.verifyShardState(dataStore, "cars",
887                 state -> assertEquals(12, state.getSnapshotIndex()));
888
889             verifySnapshot("member-1-shard-cars-testRootOverwrite", 12, 1);
890         }
891     }
892
893     private static void verifySnapshot(final String persistenceId, final long lastAppliedIndex,
894             final long lastAppliedTerm) {
895         await().atMost(5, TimeUnit.SECONDS).untilAsserted(() -> {
896                 List<Snapshot> snap = InMemorySnapshotStore.getSnapshots(persistenceId, Snapshot.class);
897                 assertEquals(1, snap.size());
898                 assertEquals(lastAppliedIndex, snap.get(0).getLastAppliedIndex());
899                 assertEquals(lastAppliedTerm, snap.get(0).getLastAppliedTerm());
900             }
901         );
902     }
903 }