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