Speed up slow tell-based Distributed*IntegrationTest cases
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / DistributedDataStoreIntegrationTest.java
index 11379a0bf25efb88db839e2c54dfbcc9f32da784..bd9d68397047b93572885e6d528b8894fd570fa0 100644 (file)
@@ -1,5 +1,5 @@
 /*
- * Copyright (c) 2014, 2015 Cisco Systems, Inc. and others.  All rights reserved.
+ * Copyright (c) 2014, 2017 Cisco Systems, Inc. and others.  All rights reserved.
  *
  * This program and the accompanying materials are made available under the
  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
@@ -30,10 +30,7 @@ import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.Uninterruptibles;
 import com.typesafe.config.ConfigFactory;
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
-import java.io.ObjectOutputStream;
 import java.math.BigInteger;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -45,7 +42,6 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import org.junit.After;
-import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -53,6 +49,7 @@ import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameter;
 import org.junit.runners.Parameterized.Parameters;
 import org.mockito.Mockito;
+import org.opendaylight.controller.cluster.access.client.RequestTimeoutException;
 import org.opendaylight.controller.cluster.databroker.ClientBackedDataStore;
 import org.opendaylight.controller.cluster.databroker.ConcurrentDOMDataBroker;
 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
@@ -61,9 +58,9 @@ import org.opendaylight.controller.cluster.datastore.messages.FindLocalShard;
 import org.opendaylight.controller.cluster.datastore.messages.LocalShardFound;
 import org.opendaylight.controller.cluster.datastore.persisted.DatastoreSnapshot;
 import org.opendaylight.controller.cluster.datastore.persisted.MetadataShardDataTreeSnapshot;
-import org.opendaylight.controller.cluster.datastore.persisted.PayloadVersion;
 import org.opendaylight.controller.cluster.datastore.persisted.ShardSnapshotState;
 import org.opendaylight.controller.cluster.datastore.utils.MockDataChangeListener;
+import org.opendaylight.controller.cluster.datastore.utils.MockDataTreeChangeListener;
 import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
@@ -111,13 +108,15 @@ public class DistributedDataStoreIntegrationTest {
     @Parameter
     public Class<? extends AbstractDataStore> testParameter;
 
-    private static ActorSystem system;
+    private ActorSystem system;
 
     private final DatastoreContext.Builder datastoreContextBuilder = DatastoreContext.newBuilder()
             .shardHeartbeatIntervalInMillis(100);
 
     @Before
     public void setUp() throws IOException {
+        InMemorySnapshotStore.clear();
+        InMemoryJournal.clear();
         system = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member1"));
         Address member1Address = AddressFromURIString.parse("akka://cluster-test@127.0.0.1:2558");
         Cluster.get(system).join(member1Address);
@@ -125,7 +124,7 @@ public class DistributedDataStoreIntegrationTest {
 
     @After
     public void tearDown() throws IOException {
-        JavaTestKit.shutdownActorSystem(system);
+        JavaTestKit.shutdownActorSystem(system, null, Boolean.TRUE);
         system = null;
     }
 
@@ -137,7 +136,7 @@ public class DistributedDataStoreIntegrationTest {
     public void testWriteTransactionWithSingleShard() throws Exception {
         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
             {
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, "transactionIntegrationTest", "test-1")) {
 
                     testWriteTransaction(dataStore, TestModel.TEST_PATH,
@@ -154,7 +153,7 @@ public class DistributedDataStoreIntegrationTest {
     public void testWriteTransactionWithMultipleShards() throws Exception {
         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
             {
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, "testWriteTransactionWithMultipleShards", "cars-1", "people-1")) {
 
                     DOMStoreWriteTransaction writeTx = dataStore.newWriteOnlyTransaction();
@@ -203,7 +202,7 @@ public class DistributedDataStoreIntegrationTest {
     public void testReadWriteTransactionWithSingleShard() throws Exception {
         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
             {
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, "testReadWriteTransactionWithSingleShard", "test-1")) {
 
                     // 1. Create a read-write Tx
@@ -244,7 +243,7 @@ public class DistributedDataStoreIntegrationTest {
     public void testReadWriteTransactionWithMultipleShards() throws Exception {
         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
             {
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, "testReadWriteTransactionWithMultipleShards", "cars-1", "people-1")) {
 
                     DOMStoreReadWriteTransaction readWriteTx = dataStore.newReadWriteTransaction();
@@ -301,7 +300,7 @@ public class DistributedDataStoreIntegrationTest {
     public void testSingleTransactionsWritesInQuickSuccession() throws Exception {
         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
             {
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, "testSingleTransactionsWritesInQuickSuccession", "cars-1")) {
 
                     final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
@@ -344,7 +343,7 @@ public class DistributedDataStoreIntegrationTest {
                 final CountDownLatch blockRecoveryLatch = new CountDownLatch(1);
                 InMemoryJournal.addBlockReadMessagesLatch(persistentID, blockRecoveryLatch);
 
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, testName, false, shardName)) {
 
                     // Create the write Tx
@@ -361,28 +360,25 @@ public class DistributedDataStoreIntegrationTest {
                     final AtomicReference<DOMStoreThreePhaseCommitCohort> txCohort = new AtomicReference<>();
                     final AtomicReference<Exception> caughtEx = new AtomicReference<>();
                     final CountDownLatch txReady = new CountDownLatch(1);
-                    final Thread txThread = new Thread() {
-                        @Override
-                        public void run() {
-                            try {
-                                writeTx.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+                    final Thread txThread = new Thread(() -> {
+                        try {
+                            writeTx.write(TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
 
-                                writeTx.merge(TestModel.OUTER_LIST_PATH,
-                                        ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build());
+                            writeTx.merge(TestModel.OUTER_LIST_PATH,
+                                    ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build());
 
-                                writeTx.write(listEntryPath,
-                                        ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1));
+                            writeTx.write(listEntryPath,
+                                    ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1));
 
-                                writeTx.delete(listEntryPath);
+                            writeTx.delete(listEntryPath);
 
-                                txCohort.set(writeTx.ready());
-                            } catch (Exception e) {
-                                caughtEx.set(e);
-                            } finally {
-                                txReady.countDown();
-                            }
+                            txCohort.set(writeTx.ready());
+                        } catch (Exception e) {
+                            caughtEx.set(e);
+                        } finally {
+                            txReady.countDown();
                         }
-                    };
+                    });
 
                     txThread.start();
 
@@ -444,7 +440,7 @@ public class DistributedDataStoreIntegrationTest {
                 final CountDownLatch blockRecoveryLatch = new CountDownLatch(1);
                 InMemoryJournal.addBlockReadMessagesLatch(persistentID, blockRecoveryLatch);
 
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, testName, false, shardName)) {
 
                     // Create the read-write Tx
@@ -458,23 +454,20 @@ public class DistributedDataStoreIntegrationTest {
                             txReadFuture = new AtomicReference<>();
                     final AtomicReference<Exception> caughtEx = new AtomicReference<>();
                     final CountDownLatch txReadsDone = new CountDownLatch(1);
-                    final Thread txThread = new Thread() {
-                        @Override
-                        public void run() {
-                            try {
-                                readWriteTx.write(TestModel.TEST_PATH,
-                                        ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+                    final Thread txThread = new Thread(() -> {
+                        try {
+                            readWriteTx.write(TestModel.TEST_PATH,
+                                    ImmutableNodes.containerNode(TestModel.TEST_QNAME));
 
-                                txExistsFuture.set(readWriteTx.exists(TestModel.TEST_PATH));
+                            txExistsFuture.set(readWriteTx.exists(TestModel.TEST_PATH));
 
-                                txReadFuture.set(readWriteTx.read(TestModel.TEST_PATH));
-                            } catch (Exception e) {
-                                caughtEx.set(e);
-                            } finally {
-                                txReadsDone.countDown();
-                            }
+                            txReadFuture.set(readWriteTx.read(TestModel.TEST_PATH));
+                        } catch (Exception e) {
+                            caughtEx.set(e);
+                        } finally {
+                            txReadsDone.countDown();
                         }
-                    };
+                    });
 
                     txThread.start();
 
@@ -588,7 +581,7 @@ public class DistributedDataStoreIntegrationTest {
 
                 InMemoryJournal.addEntry(persistentID, 1, "Dummy data so akka will read from persistence");
 
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, testName, false, shardName)) {
 
                     // Create the read-write Tx
@@ -656,9 +649,9 @@ public class DistributedDataStoreIntegrationTest {
                 // The ShardManager uses the election timeout for FindPrimary so
                 // reset it low so it will timeout quickly.
                 datastoreContextBuilder.shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(1)
-                        .shardInitializationTimeout(200, TimeUnit.MILLISECONDS);
+                        .shardInitializationTimeout(200, TimeUnit.MILLISECONDS).frontendRequestTimeoutInSeconds(2);
 
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, testName, false, shardName)) {
 
                     final Object result = dataStore.getActorContext().executeOperation(
@@ -666,8 +659,11 @@ public class DistributedDataStoreIntegrationTest {
                     assertTrue("Expected LocalShardFound. Actual: " + result, result instanceof LocalShardFound);
 
                     // Create the write Tx.
-                    try (final DOMStoreWriteTransaction writeTx = writeOnly ? dataStore.newWriteOnlyTransaction()
-                            : dataStore.newReadWriteTransaction()) {
+                    DOMStoreWriteTransaction writeTxToClose = null;
+                    try {
+                        writeTxToClose = writeOnly ? dataStore.newWriteOnlyTransaction()
+                                : dataStore.newReadWriteTransaction();
+                        final DOMStoreWriteTransaction writeTx = writeTxToClose;
                         assertNotNull("newReadWriteTransaction returned null", writeTx);
 
                         // Do some modifications and ready the Tx on a separate
@@ -703,10 +699,25 @@ public class DistributedDataStoreIntegrationTest {
                         // should have timed out and throw an appropriate
                         // exception cause.
                         try {
-                            txCohort.get().canCommit().get(5, TimeUnit.SECONDS);
+                            txCohort.get().canCommit().get(10, TimeUnit.SECONDS);
                             fail("Expected NoShardLeaderException");
                         } catch (final ExecutionException e) {
-                            Throwables.propagate(Throwables.getRootCause(e));
+                            final String msg = "Unexpected exception: "
+                                    + Throwables.getStackTraceAsString(e.getCause());
+                            if (DistributedDataStore.class.equals(testParameter)) {
+                                assertTrue(Throwables.getRootCause(e) instanceof NoShardLeaderException);
+                            } else {
+                                assertTrue(msg, Throwables.getRootCause(e) instanceof RequestTimeoutException);
+                            }
+                        }
+                    } finally {
+                        try {
+                            if (writeTxToClose != null) {
+                                writeTxToClose.close();
+                            }
+                        } catch (Exception e) {
+                            // FIXME TransactionProxy.close throws IllegalStateException:
+                            // Transaction is ready, it cannot be closed
                         }
                     }
                 }
@@ -714,13 +725,13 @@ public class DistributedDataStoreIntegrationTest {
         };
     }
 
-    @Test(expected = NoShardLeaderException.class)
+    @Test
     public void testWriteOnlyTransactionCommitFailureWithNoShardLeader() throws Exception {
         datastoreContextBuilder.writeOnlyTransactionOptimizationsEnabled(true);
         testTransactionCommitFailureWithNoShardLeader(true, "testWriteOnlyTransactionCommitFailureWithNoShardLeader");
     }
 
-    @Test(expected = NoShardLeaderException.class)
+    @Test
     public void testReadWriteTransactionCommitFailureWithNoShardLeader() throws Exception {
         testTransactionCommitFailureWithNoShardLeader(false, "testReadWriteTransactionCommitFailureWithNoShardLeader");
     }
@@ -729,7 +740,7 @@ public class DistributedDataStoreIntegrationTest {
     public void testTransactionAbort() throws Exception {
         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
             {
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, "transactionAbortIntegrationTest", "test-1")) {
 
                     final DOMStoreWriteTransaction writeTx = dataStore.newWriteOnlyTransaction();
@@ -755,7 +766,7 @@ public class DistributedDataStoreIntegrationTest {
     public void testTransactionChainWithSingleShard() throws Exception {
         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
             {
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, "testTransactionChainWithSingleShard", "test-1")) {
 
                     // 1. Create a Tx chain and write-only Tx
@@ -839,7 +850,7 @@ public class DistributedDataStoreIntegrationTest {
     public void testTransactionChainWithMultipleShards() throws Exception {
         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
             {
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, "testTransactionChainWithMultipleShards", "cars-1", "people-1")) {
 
                     final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
@@ -907,7 +918,7 @@ public class DistributedDataStoreIntegrationTest {
     public void testCreateChainedTransactionsInQuickSuccession() throws Exception {
         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
             {
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, "testCreateChainedTransactionsInQuickSuccession", "cars-1")) {
 
                     final ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
@@ -956,7 +967,7 @@ public class DistributedDataStoreIntegrationTest {
     public void testCreateChainedTransactionAfterEmptyTxReadied() throws Exception {
         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
             {
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, "testCreateChainedTransactionAfterEmptyTxReadied", "test-1")) {
 
                     final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
@@ -981,7 +992,7 @@ public class DistributedDataStoreIntegrationTest {
     public void testCreateChainedTransactionWhenPreviousNotReady() throws Exception {
         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
             {
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, "testCreateChainedTransactionWhenPreviousNotReady", "test-1")) {
 
                     final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
@@ -1002,9 +1013,6 @@ public class DistributedDataStoreIntegrationTest {
 
     @Test
     public void testCreateChainedTransactionAfterClose() throws Exception {
-        //TODO remove when test passes also for ClientBackedDataStore
-        Assume.assumeTrue(testParameter.equals(DistributedDataStore.class));
-
         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
             {
                 try (AbstractDataStore dataStore = setupAbstractDataStore(
@@ -1023,12 +1031,9 @@ public class DistributedDataStoreIntegrationTest {
 
     @Test
     public void testChainWithReadOnlyTxAfterPreviousReady() throws Exception {
-        //TODO remove when test passes also for ClientBackedDataStore
-        Assume.assumeTrue(testParameter.equals(DistributedDataStore.class));
-
         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
             {
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, "testChainWithReadOnlyTxAfterPreviousReady", "test-1")) {
 
                     final DOMStoreTransactionChain txChain = dataStore.createTransactionChain();
@@ -1070,12 +1075,9 @@ public class DistributedDataStoreIntegrationTest {
 
     @Test
     public void testChainedTransactionFailureWithSingleShard() throws Exception {
-        //TODO remove when test passes also for ClientBackedDataStore
-        Assume.assumeTrue(testParameter.equals(DistributedDataStore.class));
-
         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
             {
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, "testChainedTransactionFailureWithSingleShard", "cars-1")) {
 
                     final ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
@@ -1116,12 +1118,9 @@ public class DistributedDataStoreIntegrationTest {
 
     @Test
     public void testChainedTransactionFailureWithMultipleShards() throws Exception {
-        //TODO remove when test passes also for ClientBackedDataStore
-        Assume.assumeTrue(testParameter.equals(DistributedDataStore.class));
-
         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
             {
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, "testChainedTransactionFailureWithMultipleShards", "cars-1", "people-1")) {
 
                     final ConcurrentDOMDataBroker broker = new ConcurrentDOMDataBroker(
@@ -1165,12 +1164,9 @@ public class DistributedDataStoreIntegrationTest {
 
     @Test
     public void testChangeListenerRegistration() throws Exception {
-        //TODO remove when test passes also for ClientBackedDataStore
-        Assume.assumeTrue(testParameter.equals(DistributedDataStore.class));
-
         new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
             {
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, "testChangeListenerRegistration", "test-1")) {
 
                     testWriteTransaction(dataStore, TestModel.TEST_PATH,
@@ -1183,6 +1179,63 @@ public class DistributedDataStoreIntegrationTest {
 
                     assertNotNull("registerChangeListener returned null", listenerReg);
 
+                    IntegrationTestKit.verifyShardState(dataStore, "test-1",
+                        state -> assertEquals("getDataChangeListenerActors", 1,
+                                state.getDataChangeListenerActors().size()));
+
+                    // Wait for the initial notification
+                    listener.waitForChangeEvents(TestModel.TEST_PATH);
+                    listener.reset(2);
+
+                    // Write 2 updates.
+                    testWriteTransaction(dataStore, TestModel.OUTER_LIST_PATH,
+                            ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build());
+
+                    YangInstanceIdentifier listPath = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
+                            .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build();
+                    testWriteTransaction(dataStore, listPath,
+                            ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1));
+
+                    // Wait for the 2 updates.
+                    listener.waitForChangeEvents(TestModel.OUTER_LIST_PATH, listPath);
+                    listenerReg.close();
+
+                    IntegrationTestKit.verifyShardState(dataStore, "test-1",
+                        state -> assertEquals("getDataChangeListenerActors", 0,
+                                state.getDataChangeListenerActors().size()));
+
+                    testWriteTransaction(dataStore,
+                            YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
+                                    .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2).build(),
+                            ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2));
+
+                    listener.expectNoMoreChanges("Received unexpected change after close");
+                }
+            }
+        };
+    }
+
+    @Test
+    public void testDataTreeChangeListenerRegistration() throws Exception {
+        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
+            {
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
+                        testParameter, "testDataTreeChangeListenerRegistration", "test-1")) {
+
+                    testWriteTransaction(dataStore, TestModel.TEST_PATH,
+                            ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+
+                    final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
+
+                    ListenerRegistration<MockDataTreeChangeListener> listenerReg = dataStore
+                            .registerTreeChangeListener(TestModel.TEST_PATH, listener);
+
+                    assertNotNull("registerTreeChangeListener returned null", listenerReg);
+
+                    IntegrationTestKit.verifyShardState(dataStore, "test-1",
+                        state -> assertEquals("getTreeChangeListenerActors", 1,
+                                state.getTreeChangeListenerActors().size()));
+
                     // Wait for the initial notification
                     listener.waitForChangeEvents(TestModel.TEST_PATH);
                     listener.reset(2);
@@ -1200,6 +1253,10 @@ public class DistributedDataStoreIntegrationTest {
                     listener.waitForChangeEvents(TestModel.OUTER_LIST_PATH, listPath);
                     listenerReg.close();
 
+                    IntegrationTestKit.verifyShardState(dataStore, "test-1",
+                        state -> assertEquals("getTreeChangeListenerActors", 0,
+                                state.getTreeChangeListenerActors().size()));
+
                     testWriteTransaction(dataStore,
                             YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
                                     .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2).build(),
@@ -1246,7 +1303,7 @@ public class DistributedDataStoreIntegrationTest {
                         new DatastoreSnapshot.ShardSnapshot("cars", carsSnapshot),
                         new DatastoreSnapshot.ShardSnapshot("people", peopleSnapshot)));
 
-                try (final AbstractDataStore dataStore = setupAbstractDataStore(
+                try (AbstractDataStore dataStore = setupAbstractDataStore(
                         testParameter, name, "module-shards-member1.conf", true, "cars", "people")) {
 
                     final DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
@@ -1263,48 +1320,4 @@ public class DistributedDataStoreIntegrationTest {
             }
         };
     }
-
-    @Test
-    @Deprecated
-    public void testRecoveryFromPreCarbonSnapshot() throws Exception {
-        new IntegrationTestKit(getSystem(), datastoreContextBuilder) {
-            {
-                final String name = "testRecoveryFromPreCarbonSnapshot";
-
-                final ContainerNode carsNode = CarsModel.newCarsNode(
-                        CarsModel.newCarsMapNode(CarsModel.newCarEntry("optima", BigInteger.valueOf(20000L)),
-                                CarsModel.newCarEntry("sportage", BigInteger.valueOf(30000L))));
-
-                DataTree dataTree = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL);
-                dataTree.setSchemaContext(SchemaContextHelper.full());
-                AbstractShardTest.writeToStore(dataTree, CarsModel.BASE_PATH, carsNode);
-                NormalizedNode<?, ?> root = AbstractShardTest.readStore(dataTree, YangInstanceIdentifier.EMPTY);
-
-                MetadataShardDataTreeSnapshot shardSnapshot = new MetadataShardDataTreeSnapshot(root);
-                final ByteArrayOutputStream bos = new ByteArrayOutputStream();
-                try (final DataOutputStream dos = new DataOutputStream(bos)) {
-                    PayloadVersion.BORON.writeTo(dos);
-                    try (ObjectOutputStream oos = new ObjectOutputStream(dos)) {
-                        oos.writeObject(shardSnapshot);
-                    }
-                }
-
-                final org.opendaylight.controller.cluster.raft.Snapshot snapshot =
-                        org.opendaylight.controller.cluster.raft.Snapshot.create(bos.toByteArray(),
-                                Collections.emptyList(), 2, 1, 2, 1, 1, "member-1", null);
-
-                InMemorySnapshotStore.addSnapshot("member-1-shard-cars-" + name, snapshot);
-
-                try (AbstractDataStore dataStore = setupAbstractDataStore(
-                        testParameter, name, "module-shards-member1.conf", true, "cars")) {
-
-                    DOMStoreReadTransaction readTx = dataStore.newReadOnlyTransaction();
-
-                    Optional<NormalizedNode<?, ?>> optional = readTx.read(CarsModel.BASE_PATH).get(5, TimeUnit.SECONDS);
-                    assertEquals("isPresent", true, optional.isPresent());
-                    assertEquals("Data node", carsNode, optional.get());
-                }
-            }
-        };
-    }
 }