Deprecate DOMDataTreeProducer-related classes
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / sharding / DistributedShardedDOMDataTreeTest.java
index 9841ca01250f27ef1ca0ede6fa5a28e5dd12e351..602e2cc4077099728827640d3575a7ee15938a90 100644 (file)
@@ -10,8 +10,9 @@ package org.opendaylight.controller.cluster.sharding;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
-import static org.mockito.Matchers.anyCollection;
-import static org.mockito.Matchers.anyMap;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.anyCollection;
+import static org.mockito.ArgumentMatchers.anyMap;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
@@ -25,17 +26,22 @@ import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
 import akka.actor.Address;
 import akka.actor.AddressFromURIString;
+import akka.actor.Props;
 import akka.cluster.Cluster;
-import akka.testkit.JavaTestKit;
-import com.google.common.base.Optional;
+import akka.testkit.javadsl.TestKit;
 import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.CheckedFuture;
+import com.google.common.util.concurrent.FluentFuture;
+import com.google.common.util.concurrent.ListenableFuture;
 import com.typesafe.config.ConfigFactory;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletionStage;
+import java.util.concurrent.TimeUnit;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -46,17 +52,25 @@ import org.mockito.Captor;
 import org.mockito.Mockito;
 import org.mockito.MockitoAnnotations;
 import org.opendaylight.controller.cluster.ActorSystemProvider;
+import org.opendaylight.controller.cluster.access.concepts.MemberName;
+import org.opendaylight.controller.cluster.databroker.actors.dds.ClientLocalHistory;
+import org.opendaylight.controller.cluster.databroker.actors.dds.ClientTransaction;
+import org.opendaylight.controller.cluster.databroker.actors.dds.DataStoreClient;
+import org.opendaylight.controller.cluster.databroker.actors.dds.SimpleDataStoreClientActor;
 import org.opendaylight.controller.cluster.datastore.AbstractTest;
 import org.opendaylight.controller.cluster.datastore.DatastoreContext;
 import org.opendaylight.controller.cluster.datastore.DatastoreContext.Builder;
 import org.opendaylight.controller.cluster.datastore.DistributedDataStore;
 import org.opendaylight.controller.cluster.datastore.IntegrationTestKit;
+import org.opendaylight.controller.cluster.datastore.utils.ActorUtils;
 import org.opendaylight.controller.cluster.datastore.utils.ClusterUtils;
-import org.opendaylight.controller.cluster.sharding.DistributedShardFactory.DistributedShardRegistration;
+import org.opendaylight.controller.cluster.dom.api.CDSDataTreeProducer;
+import org.opendaylight.controller.cluster.dom.api.CDSShardAccess;
+import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
+import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
-import org.opendaylight.mdsal.common.api.TransactionCommitFailedException;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeCursorAwareTransaction;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeListener;
@@ -79,7 +93,7 @@ import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableMa
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-@Ignore("distributed-data is broken needs to be removed")
+@Deprecated(forRemoval = true)
 public class DistributedShardedDOMDataTreeTest extends AbstractTest {
 
     private static final Logger LOG = LoggerFactory.getLogger(DistributedShardedDOMDataTreeRemotingTest.class);
@@ -94,6 +108,9 @@ public class DistributedShardedDOMDataTreeTest extends AbstractTest {
             new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION,
                     YangInstanceIdentifier.create(getOuterListIdFor(0).getPathArguments())
                             .node(TestModel.INNER_LIST_QNAME));
+    private static final Set<MemberName> SINGLE_MEMBER = Collections.singleton(AbstractTest.MEMBER_NAME);
+
+    private static final String MODULE_SHARDS_CONFIG = "module-shards-default-member-1.conf";
 
     private ActorSystem leaderSystem;
 
@@ -101,10 +118,10 @@ public class DistributedShardedDOMDataTreeTest extends AbstractTest {
             DatastoreContext.newBuilder()
                     .shardHeartbeatIntervalInMillis(100)
                     .shardElectionTimeoutFactor(2)
-                    .logicalStoreType(
-                            org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType.CONFIGURATION);
+                    .logicalStoreType(LogicalDatastoreType.CONFIGURATION);
 
     private DistributedDataStore leaderDistributedDataStore;
+    private DistributedDataStore operDistributedDatastore;
     private IntegrationTestKit leaderTestKit;
 
     private DistributedShardedDOMDataTree leaderShardFactory;
@@ -120,6 +137,9 @@ public class DistributedShardedDOMDataTreeTest extends AbstractTest {
     public void setUp() {
         MockitoAnnotations.initMocks(this);
 
+        InMemoryJournal.clear();
+        InMemorySnapshotStore.clear();
+
         leaderSystem = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member1"));
         Cluster.get(leaderSystem).join(MEMBER_1_ADDRESS);
 
@@ -133,35 +153,48 @@ public class DistributedShardedDOMDataTreeTest extends AbstractTest {
             leaderDistributedDataStore.close();
         }
 
-        JavaTestKit.shutdownActorSystem(leaderSystem);
+        if (operDistributedDatastore != null) {
+            operDistributedDatastore.close();
+        }
+
+        TestKit.shutdownActorSystem(leaderSystem);
+
+        InMemoryJournal.clear();
+        InMemorySnapshotStore.clear();
     }
 
-    private void initEmptyDatastore(final String type) {
+    private void initEmptyDatastores() throws Exception {
         leaderTestKit = new IntegrationTestKit(leaderSystem, leaderDatastoreContextBuilder);
 
-        leaderDistributedDataStore =
-                leaderTestKit.setupDistributedDataStoreWithoutConfig(type, SchemaContextHelper.full());
+        leaderDistributedDataStore = leaderTestKit.setupDistributedDataStore(
+                "config", MODULE_SHARDS_CONFIG, "empty-modules.conf", true,
+                SchemaContextHelper.distributedShardedDOMDataTreeSchemaContext());
 
+        operDistributedDatastore = leaderTestKit.setupDistributedDataStore(
+                "operational", MODULE_SHARDS_CONFIG, "empty-modules.conf",true,
+                SchemaContextHelper.distributedShardedDOMDataTreeSchemaContext());
 
         leaderShardFactory = new DistributedShardedDOMDataTree(leaderSystemProvider,
-                leaderDistributedDataStore,
+                operDistributedDatastore,
                 leaderDistributedDataStore);
+
+        leaderShardFactory.init();
     }
 
 
     @Test
     public void testWritesIntoDefaultShard() throws Exception {
-        initEmptyDatastore("config");
+        initEmptyDatastores();
 
         final DOMDataTreeIdentifier configRoot =
-                new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, YangInstanceIdentifier.EMPTY);
+                new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, YangInstanceIdentifier.empty());
 
         final DOMDataTreeProducer producer = leaderShardFactory.createProducer(Collections.singleton(configRoot));
 
         final DOMDataTreeCursorAwareTransaction tx = producer.createTransaction(true);
         final DOMDataTreeWriteCursor cursor =
                 tx.createCursor(new DOMDataTreeIdentifier(
-                        LogicalDatastoreType.CONFIGURATION, YangInstanceIdentifier.EMPTY));
+                        LogicalDatastoreType.CONFIGURATION, YangInstanceIdentifier.empty()));
         Assert.assertNotNull(cursor);
 
         final ContainerNode test =
@@ -171,18 +204,18 @@ public class DistributedShardedDOMDataTreeTest extends AbstractTest {
         cursor.write(test.getIdentifier(), test);
         cursor.close();
 
-        tx.submit().checkedGet();
+        tx.commit().get();
     }
 
     @Test
-    public void testSingleNodeWrites() throws Exception {
-        initEmptyDatastore("config");
+    public void testSingleNodeWritesAndRead() throws Exception {
+        initEmptyDatastores();
 
         final DistributedShardRegistration shardRegistration = waitOnAsyncTask(
                 leaderShardFactory.createDistributedShard(TEST_ID, Lists.newArrayList(AbstractTest.MEMBER_NAME)),
                 DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
 
-        leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
+        leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorUtils(),
                 ClusterUtils.getCleanShardName(TEST_ID.getRootIdentifier()));
 
         final DOMDataTreeProducer producer = leaderShardFactory.createProducer(Collections.singleton(TEST_ID));
@@ -194,13 +227,14 @@ public class DistributedShardedDOMDataTreeTest extends AbstractTest {
                 YangInstanceIdentifier.builder(TestModel.TEST_PATH).node(TestModel.NAME_QNAME).build();
         final LeafNode<String> valueToCheck = ImmutableLeafNodeBuilder.<String>create().withNodeIdentifier(
                 new NodeIdentifier(TestModel.NAME_QNAME)).withValue("Test Value").build();
+        LOG.debug("Writing data {} at {}, cursor {}", nameId.getLastPathArgument(), valueToCheck, cursor);
         cursor.write(nameId.getLastPathArgument(),
                 valueToCheck);
 
         cursor.close();
         LOG.debug("Got to pre submit");
 
-        tx.submit().checkedGet();
+        tx.commit().get();
 
         final DOMDataTreeListener mockedDataTreeListener = mock(DOMDataTreeListener.class);
         doNothing().when(mockedDataTreeListener).onDataTreeChanged(anyCollection(), anyMap());
@@ -221,23 +255,45 @@ public class DistributedShardedDOMDataTreeTest extends AbstractTest {
 
         verifyNoMoreInteractions(mockedDataTreeListener);
 
+        final String shardName = ClusterUtils.getCleanShardName(TEST_ID.getRootIdentifier());
+        LOG.debug("Creating distributed datastore client for shard {}", shardName);
+
+        final ActorUtils actorUtils = leaderDistributedDataStore.getActorUtils();
+        final Props distributedDataStoreClientProps =
+                SimpleDataStoreClientActor.props(actorUtils.getCurrentMemberName(), "Shard-" + shardName, actorUtils,
+                    shardName);
+
+        final ActorRef clientActor = leaderSystem.actorOf(distributedDataStoreClientProps);
+        final DataStoreClient distributedDataStoreClient = SimpleDataStoreClientActor
+                    .getDistributedDataStoreClient(clientActor, 30, TimeUnit.SECONDS);
+
+        final ClientLocalHistory localHistory = distributedDataStoreClient.createLocalHistory();
+        final ClientTransaction tx2 = localHistory.createTransaction();
+        final FluentFuture<Optional<NormalizedNode<?, ?>>> read = tx2.read(YangInstanceIdentifier.empty());
+
+        final Optional<NormalizedNode<?, ?>> optional = read.get();
+        tx2.abort();
+        localHistory.close();
+
+        shardRegistration.close().toCompletableFuture().get();
+
     }
 
     @Test
     public void testMultipleWritesIntoSingleMapEntry() throws Exception {
-        initEmptyDatastore("config");
+        initEmptyDatastores();
 
         final DistributedShardRegistration shardRegistration = waitOnAsyncTask(
                 leaderShardFactory.createDistributedShard(TEST_ID, Lists.newArrayList(AbstractTest.MEMBER_NAME)),
                 DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
 
-        leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
+        leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorUtils(),
                 ClusterUtils.getCleanShardName(TEST_ID.getRootIdentifier()));
 
         LOG.warn("Got after waiting for nonleader");
-        final ActorRef leaderShardManager = leaderDistributedDataStore.getActorContext().getShardManager();
+        final ActorRef leaderShardManager = leaderDistributedDataStore.getActorUtils().getShardManager();
 
-        leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
+        leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorUtils(),
                 ClusterUtils.getCleanShardName(TestModel.TEST_PATH));
 
         final YangInstanceIdentifier oid1 = getOuterListIdFor(0);
@@ -247,7 +303,7 @@ public class DistributedShardedDOMDataTreeTest extends AbstractTest {
                 leaderShardFactory.createDistributedShard(outerListPath, Lists.newArrayList(AbstractTest.MEMBER_NAME)),
                 DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
 
-        leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
+        leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorUtils(),
                 ClusterUtils.getCleanShardName(outerListPath.getRootIdentifier()));
 
         final DOMDataTreeProducer shardProducer = leaderShardFactory.createProducer(
@@ -265,9 +321,9 @@ public class DistributedShardedDOMDataTreeTest extends AbstractTest {
 
         cursor.write(new NodeIdentifier(TestModel.INNER_LIST_QNAME), innerList);
         cursor.close();
-        tx.submit().checkedGet();
+        tx.commit().get();
 
-        final ArrayList<CheckedFuture<Void, TransactionCommitFailedException>> futures = new ArrayList<>();
+        final ArrayList<ListenableFuture<?>> futures = new ArrayList<>();
         for (int i = 0; i < 1000; i++) {
             final Collection<MapEntryNode> innerListMapEntries = createInnerListMapEntries(1000, "run-" + i);
             for (final MapEntryNode innerListMapEntry : innerListMapEntries) {
@@ -277,11 +333,11 @@ public class DistributedShardedDOMDataTreeTest extends AbstractTest {
                                 oid1.node(new NodeIdentifier(TestModel.INNER_LIST_QNAME))));
                 cursor1.write(innerListMapEntry.getIdentifier(), innerListMapEntry);
                 cursor1.close();
-                futures.add(tx1.submit());
+                futures.add(tx1.commit());
             }
         }
 
-        futures.get(futures.size() - 1).checkedGet();
+        futures.get(futures.size() - 1).get();
 
         final DOMDataTreeListener mockedDataTreeListener = mock(DOMDataTreeListener.class);
         doNothing().when(mockedDataTreeListener).onDataTreeChanged(anyCollection(), anyMap());
@@ -307,60 +363,99 @@ public class DistributedShardedDOMDataTreeTest extends AbstractTest {
 
     }
 
-    private static Collection<MapEntryNode> createInnerListMapEntries(final int amount, final String valuePrefix) {
-        final Collection<MapEntryNode> ret = new ArrayList<>();
-        for (int i = 0; i < amount; i++) {
-            ret.add(ImmutableNodes.mapEntryBuilder()
-                    .withNodeIdentifier(new NodeIdentifierWithPredicates(TestModel.INNER_LIST_QNAME,
-                            QName.create(TestModel.INNER_LIST_QNAME, "name"), Integer.toString(i)))
-                    .withChild(ImmutableNodes
-                            .leafNode(QName.create(TestModel.INNER_LIST_QNAME, "value"), valuePrefix + "-" + i))
-                    .build());
+    // top level shard at TEST element, with subshards on each outer-list map entry
+    @Test
+    @Ignore
+    public void testMultipleShardLevels() throws Exception {
+        initEmptyDatastores();
+
+        final DistributedShardRegistration testShardReg = waitOnAsyncTask(
+                leaderShardFactory.createDistributedShard(TEST_ID, SINGLE_MEMBER),
+                DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
+
+        final ArrayList<DistributedShardRegistration> registrations = new ArrayList<>();
+        final int listSize = 5;
+        for (int i = 0; i < listSize; i++) {
+            final YangInstanceIdentifier entryYID = getOuterListIdFor(i);
+            final CompletionStage<DistributedShardRegistration> future = leaderShardFactory.createDistributedShard(
+                    new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, entryYID), SINGLE_MEMBER);
+
+            registrations.add(waitOnAsyncTask(future, DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION));
         }
 
-        return ret;
-    }
+        final DOMDataTreeIdentifier rootId =
+                new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, YangInstanceIdentifier.empty());
+        final DOMDataTreeProducer producer = leaderShardFactory.createProducer(Collections.singletonList(
+                rootId));
 
-    @Test
-    public void testDistributedData() throws Exception {
-        initEmptyDatastore("config");
+        DOMDataTreeCursorAwareTransaction transaction = producer.createTransaction(false);
 
-        waitOnAsyncTask(
-                leaderShardFactory.createDistributedShard(TEST_ID, Lists.newArrayList(AbstractTest.MEMBER_NAME)),
-                DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
+        DOMDataTreeWriteCursor cursor = transaction.createCursor(rootId);
+        assertNotNull(cursor);
 
-        waitOnAsyncTask(
-                leaderShardFactory.createDistributedShard(
-                        new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, TestModel.OUTER_CONTAINER_PATH),
-                        Lists.newArrayList(AbstractTest.MEMBER_NAME)),
-                DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
+        final MapNode outerList =
+                ImmutableMapNodeBuilder.create()
+                        .withNodeIdentifier(new NodeIdentifier(TestModel.OUTER_LIST_QNAME)).build();
 
-        waitOnAsyncTask(
-                leaderShardFactory.createDistributedShard(
-                        new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, TestModel.INNER_LIST_PATH),
-                        Lists.newArrayList(AbstractTest.MEMBER_NAME)),
-                DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
+        final ContainerNode testNode =
+                ImmutableContainerNodeBuilder.create()
+                        .withNodeIdentifier(new NodeIdentifier(TestModel.TEST_QNAME))
+                        .withChild(outerList)
+                        .build();
 
-        waitOnAsyncTask(
-                leaderShardFactory.createDistributedShard(
-                        new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, TestModel.JUNK_PATH),
-                        Lists.newArrayList(AbstractTest.MEMBER_NAME)),
-                DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
+        cursor.write(testNode.getIdentifier(), testNode);
+
+        cursor.close();
+        transaction.commit().get();
+
+        final DOMDataTreeListener mockedDataTreeListener = mock(DOMDataTreeListener.class);
+        doNothing().when(mockedDataTreeListener).onDataTreeChanged(anyCollection(), anyMap());
+
+        final MapNode wholeList = ImmutableMapNodeBuilder.create(outerList)
+                .withValue(createOuterEntries(listSize, "testing-values")).build();
+
+        transaction = producer.createTransaction(false);
+        cursor = transaction.createCursor(TEST_ID);
+        assertNotNull(cursor);
+
+        cursor.write(wholeList.getIdentifier(), wholeList);
+        cursor.close();
+
+        transaction.commit().get();
+
+        leaderShardFactory.registerListener(mockedDataTreeListener, Collections.singletonList(TEST_ID),
+                true, Collections.emptyList());
+
+        verify(mockedDataTreeListener, timeout(35000).atLeast(2)).onDataTreeChanged(captorForChanges.capture(),
+                captorForSubtrees.capture());
+        verifyNoMoreInteractions(mockedDataTreeListener);
+        final List<Map<DOMDataTreeIdentifier, NormalizedNode<?, ?>>> allSubtrees = captorForSubtrees.getAllValues();
+
+        final Map<DOMDataTreeIdentifier, NormalizedNode<?, ?>> lastSubtree = allSubtrees.get(allSubtrees.size() - 1);
+
+        final NormalizedNode<?, ?> actual = lastSubtree.get(TEST_ID);
+        assertNotNull(actual);
+
+        final NormalizedNode<?, ?> expected =
+                ImmutableContainerNodeBuilder.create()
+                        .withNodeIdentifier(new NodeIdentifier(TestModel.TEST_QNAME))
+                        .withChild(ImmutableMapNodeBuilder.create(outerList)
+                                .withValue(createOuterEntries(listSize, "testing-values")).build())
+                        .build();
 
-        leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
-                ClusterUtils.getCleanShardName(TestModel.TEST_PATH));
-        leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
-                ClusterUtils.getCleanShardName(TestModel.OUTER_CONTAINER_PATH));
-        leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
-                ClusterUtils.getCleanShardName(TestModel.INNER_LIST_PATH));
-        leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
-                ClusterUtils.getCleanShardName(TestModel.JUNK_PATH));
 
+        for (final DistributedShardRegistration registration : registrations) {
+            waitOnAsyncTask(registration.close(), DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
+        }
+
+        waitOnAsyncTask(testShardReg.close(), DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
+
+        assertEquals(expected, actual);
     }
 
     @Test
     public void testMultipleRegistrationsAtOnePrefix() throws Exception {
-        initEmptyDatastore("config");
+        initEmptyDatastores();
 
         for (int i = 0; i < 10; i++) {
             LOG.debug("Round {}", i);
@@ -368,21 +463,89 @@ public class DistributedShardedDOMDataTreeTest extends AbstractTest {
                     TEST_ID, Lists.newArrayList(AbstractTest.MEMBER_NAME)),
                     DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
 
-            leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
+            leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorUtils(),
                     ClusterUtils.getCleanShardName(TestModel.TEST_PATH));
 
-            assertNotNull(findLocalShard(leaderDistributedDataStore.getActorContext(),
+            assertNotNull(findLocalShard(leaderDistributedDataStore.getActorUtils(),
                     ClusterUtils.getCleanShardName(TestModel.TEST_PATH)));
 
             waitOnAsyncTask(reg1.close(), DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
 
-            waitUntilShardIsDown(leaderDistributedDataStore.getActorContext(),
+            waitUntilShardIsDown(leaderDistributedDataStore.getActorUtils(),
                     ClusterUtils.getCleanShardName(TestModel.TEST_PATH));
         }
     }
 
+    @Test
+    public void testCDSDataTreeProducer() throws Exception {
+        initEmptyDatastores();
+
+        final DistributedShardRegistration reg1 = waitOnAsyncTask(leaderShardFactory.createDistributedShard(
+                TEST_ID, Lists.newArrayList(AbstractTest.MEMBER_NAME)),
+                DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
+
+        leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorUtils(),
+                ClusterUtils.getCleanShardName(TestModel.TEST_PATH));
+
+        assertNotNull(findLocalShard(leaderDistributedDataStore.getActorUtils(),
+                ClusterUtils.getCleanShardName(TestModel.TEST_PATH)));
+
+
+        final DOMDataTreeIdentifier configRoot =
+                new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, YangInstanceIdentifier.empty());
+        final DOMDataTreeProducer producer = leaderShardFactory.createProducer(Collections.singleton(configRoot));
+
+        assertTrue(producer instanceof CDSDataTreeProducer);
+
+        final CDSDataTreeProducer cdsProducer = (CDSDataTreeProducer) producer;
+        CDSShardAccess shardAccess = cdsProducer.getShardAccess(TEST_ID);
+        assertEquals(shardAccess.getShardIdentifier(), TEST_ID);
+
+        shardAccess = cdsProducer.getShardAccess(INNER_LIST_ID);
+        assertEquals(TEST_ID, shardAccess.getShardIdentifier());
+
+        shardAccess = cdsProducer.getShardAccess(configRoot);
+        assertEquals(configRoot, shardAccess.getShardIdentifier());
+
+        waitOnAsyncTask(reg1.close(), DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
+    }
+
+    private static Collection<MapEntryNode> createOuterEntries(final int amount, final String valuePrefix) {
+        final Collection<MapEntryNode> ret = new ArrayList<>();
+        for (int i = 0; i < amount; i++) {
+            ret.add(ImmutableNodes.mapEntryBuilder()
+                    .withNodeIdentifier(NodeIdentifierWithPredicates.of(TestModel.OUTER_LIST_QNAME,
+                            QName.create(TestModel.OUTER_LIST_QNAME, "id"), i))
+                    .withChild(ImmutableNodes
+                            .leafNode(QName.create(TestModel.OUTER_LIST_QNAME, "id"), i))
+                    .withChild(createWholeInnerList(amount, "outer id: " + i + " " + valuePrefix))
+                    .build());
+        }
+
+        return ret;
+    }
+
+    private static MapNode createWholeInnerList(final int amount, final String valuePrefix) {
+        return ImmutableMapNodeBuilder.create().withNodeIdentifier(new NodeIdentifier(TestModel.INNER_LIST_QNAME))
+                .withValue(createInnerListMapEntries(amount, valuePrefix)).build();
+    }
+
+    private static Collection<MapEntryNode> createInnerListMapEntries(final int amount, final String valuePrefix) {
+        final Collection<MapEntryNode> ret = new ArrayList<>();
+        for (int i = 0; i < amount; i++) {
+            ret.add(ImmutableNodes.mapEntryBuilder()
+                    .withNodeIdentifier(NodeIdentifierWithPredicates.of(TestModel.INNER_LIST_QNAME,
+                            QName.create(TestModel.INNER_LIST_QNAME, "name"), Integer.toString(i)))
+                    .withChild(ImmutableNodes
+                            .leafNode(QName.create(TestModel.INNER_LIST_QNAME, "value"), valuePrefix + "-" + i))
+                    .build());
+        }
+
+        return ret;
+    }
+
     private static YangInstanceIdentifier getOuterListIdFor(final int id) {
-        return TestModel.OUTER_LIST_PATH.node(new NodeIdentifierWithPredicates(
+        return TestModel.OUTER_LIST_PATH.node(NodeIdentifierWithPredicates.of(
                 TestModel.OUTER_LIST_QNAME, QName.create(TestModel.OUTER_LIST_QNAME, "id"), id));
     }
 }