BUG-2138: DistributedShardListeners support for nested shards
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / sharding / DistributedShardedDOMDataTreeTest.java
index 1e29bf638e7a67998ad896d3ee9cce9919f26bbf..91435bed5c1efd8076ee52b59ff91ffc3f506049 100644 (file)
@@ -8,8 +8,18 @@
 
 package org.opendaylight.controller.cluster.sharding;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+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.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.timeout;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.opendaylight.controller.cluster.datastore.IntegrationTestKit.findLocalShard;
+import static org.opendaylight.controller.cluster.datastore.IntegrationTestKit.waitUntilShardIsDown;
 
 import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
@@ -17,93 +27,117 @@ import akka.actor.Address;
 import akka.actor.AddressFromURIString;
 import akka.cluster.Cluster;
 import akka.testkit.JavaTestKit;
+import com.google.common.base.Optional;
 import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.CheckedFuture;
 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.Set;
+import java.util.concurrent.CompletionStage;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+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.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.messages.FindLocalShard;
-import org.opendaylight.controller.cluster.datastore.messages.FindPrimary;
-import org.opendaylight.controller.cluster.datastore.messages.LocalPrimaryShardFound;
-import org.opendaylight.controller.cluster.datastore.messages.LocalShardFound;
-import org.opendaylight.controller.cluster.datastore.messages.RemotePrimaryShardFound;
 import org.opendaylight.controller.cluster.datastore.utils.ClusterUtils;
-import org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy;
 import org.opendaylight.controller.cluster.sharding.DistributedShardFactory.DistributedShardRegistration;
 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;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeProducer;
-import org.opendaylight.mdsal.dom.api.DOMDataTreeShardingConflictException;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteCursor;
-import org.opendaylight.mdsal.dom.broker.ShardedDOMDataTree;
+import org.opendaylight.yangtools.yang.common.QName;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.NodeIdentifier;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.NodeIdentifierWithPredicates;
+import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
+import org.opendaylight.yangtools.yang.data.api.schema.LeafNode;
+import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
+import org.opendaylight.yangtools.yang.data.api.schema.MapNode;
+import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidate;
+import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
+import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder;
 import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableLeafNodeBuilder;
+import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableMapNodeBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @Ignore("distributed-data is broken needs to be removed")
 public class DistributedShardedDOMDataTreeTest extends AbstractTest {
 
+    private static final Logger LOG = LoggerFactory.getLogger(DistributedShardedDOMDataTreeRemotingTest.class);
+
     private static final Address MEMBER_1_ADDRESS =
             AddressFromURIString.parse("akka.tcp://cluster-test@127.0.0.1:2558");
 
     private static final DOMDataTreeIdentifier TEST_ID =
             new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, TestModel.TEST_PATH);
 
-    private ShardedDOMDataTree shardedDOMDataTree = new ShardedDOMDataTree();
+    private static final DOMDataTreeIdentifier INNER_LIST_ID =
+            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 ActorSystem leaderSystem;
-    private ActorSystem followerSystem;
-
 
     private final Builder leaderDatastoreContextBuilder =
-            DatastoreContext.newBuilder().shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(2);
+            DatastoreContext.newBuilder()
+                    .shardHeartbeatIntervalInMillis(100)
+                    .shardElectionTimeoutFactor(2)
+                    .logicalStoreType(
+                            org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType.CONFIGURATION);
 
-    private final DatastoreContext.Builder followerDatastoreContextBuilder =
-            DatastoreContext.newBuilder().shardHeartbeatIntervalInMillis(100).shardElectionTimeoutFactor(5)
-                    .customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName());
-
-    private DistributedDataStore followerDistributedDataStore;
     private DistributedDataStore leaderDistributedDataStore;
-    private IntegrationTestKit followerTestKit;
     private IntegrationTestKit leaderTestKit;
 
     private DistributedShardedDOMDataTree leaderShardFactory;
-    private DistributedShardedDOMDataTree followerShardFactory;
+
+    @Captor
+    private ArgumentCaptor<Collection<DataTreeCandidate>> captorForChanges;
+    @Captor
+    private ArgumentCaptor<Map<DOMDataTreeIdentifier, NormalizedNode<?, ?>>> captorForSubtrees;
+
+    private ActorSystemProvider leaderSystemProvider;
 
     @Before
     public void setUp() {
-        shardedDOMDataTree = new ShardedDOMDataTree();
+        MockitoAnnotations.initMocks(this);
 
         leaderSystem = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member1"));
         Cluster.get(leaderSystem).join(MEMBER_1_ADDRESS);
 
-        followerSystem = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member2"));
-        Cluster.get(followerSystem).join(MEMBER_1_ADDRESS);
+        leaderSystemProvider = Mockito.mock(ActorSystemProvider.class);
+        doReturn(leaderSystem).when(leaderSystemProvider).getActorSystem();
     }
 
     @After
     public void tearDown() {
-        if (followerDistributedDataStore != null) {
-            leaderDistributedDataStore.close();
-        }
         if (leaderDistributedDataStore != null) {
             leaderDistributedDataStore.close();
         }
 
         JavaTestKit.shutdownActorSystem(leaderSystem);
-        JavaTestKit.shutdownActorSystem(followerSystem);
     }
 
     private void initEmptyDatastore(final String type) {
@@ -112,124 +146,350 @@ public class DistributedShardedDOMDataTreeTest extends AbstractTest {
         leaderDistributedDataStore =
                 leaderTestKit.setupDistributedDataStoreWithoutConfig(type, SchemaContextHelper.full());
 
-        followerTestKit = new IntegrationTestKit(followerSystem, followerDatastoreContextBuilder);
-        followerDistributedDataStore =
-                followerTestKit.setupDistributedDataStoreWithoutConfig(type, SchemaContextHelper.full());
 
-        leaderShardFactory = new DistributedShardedDOMDataTree(leaderSystem,
-                Mockito.mock(DistributedDataStore.class),
+        leaderShardFactory = new DistributedShardedDOMDataTree(leaderSystemProvider,
+                leaderDistributedDataStore,
                 leaderDistributedDataStore);
+    }
+
+
+    @Test
+    public void testWritesIntoDefaultShard() throws Exception {
+        initEmptyDatastore("config");
+
+        final DOMDataTreeIdentifier configRoot =
+                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));
+        Assert.assertNotNull(cursor);
+
+        final ContainerNode test =
+                ImmutableContainerNodeBuilder.create()
+                        .withNodeIdentifier(new NodeIdentifier(TestModel.TEST_QNAME)).build();
+
+        cursor.write(test.getIdentifier(), test);
+        cursor.close();
 
-        followerShardFactory = new DistributedShardedDOMDataTree(followerSystem,
-                Mockito.mock(DistributedDataStore.class),
-                followerDistributedDataStore);
+        tx.submit().checkedGet();
     }
 
     @Test
-    public void testProducerRegistrations() throws Exception {
+    public void testSingleNodeWrites() throws Exception {
         initEmptyDatastore("config");
 
-        final DistributedShardRegistration shardRegistration =
-                leaderShardFactory.createDistributedShard(TEST_ID,
-                        Lists.newArrayList(AbstractTest.MEMBER_NAME, AbstractTest.MEMBER_2_NAME));
+        final DistributedShardRegistration shardRegistration = waitOnAsyncTask(
+                leaderShardFactory.createDistributedShard(TEST_ID, Lists.newArrayList(AbstractTest.MEMBER_NAME)),
+                DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
 
         leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
                 ClusterUtils.getCleanShardName(TEST_ID.getRootIdentifier()));
 
-        final ActorRef leaderShardManager = leaderDistributedDataStore.getActorContext().getShardManager();
+        final DOMDataTreeProducer producer = leaderShardFactory.createProducer(Collections.singleton(TEST_ID));
 
-        leaderShardManager.tell(
-                new FindLocalShard(ClusterUtils.getCleanShardName(TestModel.TEST_PATH), true), leaderTestKit.getRef());
-        leaderTestKit.expectMsgClass(JavaTestKit.duration("10 seconds"), LocalShardFound.class);
+        final DOMDataTreeCursorAwareTransaction tx = producer.createTransaction(true);
+        final DOMDataTreeWriteCursor cursor = tx.createCursor(TEST_ID);
+        Assert.assertNotNull(cursor);
+        final YangInstanceIdentifier nameId =
+                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();
+        cursor.write(nameId.getLastPathArgument(),
+                valueToCheck);
 
-        IntegrationTestKit.findLocalShard(followerDistributedDataStore.getActorContext(),
-                ClusterUtils.getCleanShardName(TEST_ID.getRootIdentifier()));
+        cursor.close();
+        LOG.debug("Got to pre submit");
 
-        leaderShardManager.tell(
-                new FindPrimary(ClusterUtils.getCleanShardName(TestModel.TEST_PATH), true), leaderTestKit.getRef());
-        leaderTestKit.expectMsgClass(JavaTestKit.duration("10 seconds"), LocalPrimaryShardFound.class);
+        tx.submit().checkedGet();
 
-        final ActorRef followerShardManager = followerDistributedDataStore.getActorContext().getShardManager();
-        followerShardManager.tell(
-                new FindPrimary(ClusterUtils.getCleanShardName(TestModel.TEST_PATH), true), followerTestKit.getRef());
-        followerTestKit.expectMsgClass(JavaTestKit.duration("10 seconds"), RemotePrimaryShardFound.class);
+        final DOMDataTreeListener mockedDataTreeListener = mock(DOMDataTreeListener.class);
+        doNothing().when(mockedDataTreeListener).onDataTreeChanged(anyCollection(), anyMap());
 
-        final DOMDataTreeProducer producer = leaderShardFactory.createProducer(Collections.singleton(TEST_ID));
-        try {
-            followerShardFactory.createProducer(Collections.singleton(TEST_ID));
-            fail("Producer should be already registered on the other node");
-        } catch (final IllegalArgumentException e) {
-            assertTrue(e.getMessage().contains("is attached to producer"));
-        }
+        leaderShardFactory.registerListener(mockedDataTreeListener, Collections.singletonList(TEST_ID),
+                true, Collections.emptyList());
 
-        producer.close();
+        verify(mockedDataTreeListener, timeout(1000).times(1)).onDataTreeChanged(captorForChanges.capture(),
+                captorForSubtrees.capture());
+        final List<Collection<DataTreeCandidate>> capturedValue = captorForChanges.getAllValues();
 
-        final DOMDataTreeProducer followerProducer =
-                followerShardFactory.createProducer(Collections.singleton(TEST_ID));
-        try {
-            leaderShardFactory.createProducer(Collections.singleton(TEST_ID));
-            fail("Producer should be already registered on the other node");
-        } catch (final IllegalArgumentException e) {
-            assertTrue(e.getMessage().contains("is attached to producer"));
-        }
+        final Optional<NormalizedNode<?, ?>> dataAfter =
+                capturedValue.get(0).iterator().next().getRootNode().getDataAfter();
+
+        final NormalizedNode<?,?> expected = ImmutableContainerNodeBuilder.create()
+                .withNodeIdentifier(new NodeIdentifier(TestModel.TEST_QNAME)).withChild(valueToCheck).build();
+        assertEquals(expected, dataAfter.get());
+
+        verifyNoMoreInteractions(mockedDataTreeListener);
 
-        followerProducer.close();
-        // try to create a shard on an already registered prefix on follower
-        try {
-            followerShardFactory.createDistributedShard(TEST_ID,
-                    Lists.newArrayList(AbstractTest.MEMBER_NAME, AbstractTest.MEMBER_2_NAME));
-            fail("This prefix already should have a shard registration that was forwarded from the other node");
-        } catch (final DOMDataTreeShardingConflictException e) {
-            assertTrue(e.getMessage().contains("is already occupied by shard"));
-        }
     }
 
     @Test
-    @Ignore("Needs some other stuff related to 5280")
-    public void testWriteIntoMultipleShards() throws Exception {
+    public void testMultipleWritesIntoSingleMapEntry() throws Exception {
         initEmptyDatastore("config");
 
-        final DistributedShardRegistration shardRegistration =
-                leaderShardFactory.createDistributedShard(
-                        TEST_ID,Lists.newArrayList(AbstractTest.MEMBER_NAME, AbstractTest.MEMBER_2_NAME));
+        final DistributedShardRegistration shardRegistration = waitOnAsyncTask(
+                leaderShardFactory.createDistributedShard(TEST_ID, Lists.newArrayList(AbstractTest.MEMBER_NAME)),
+                DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
 
         leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
                 ClusterUtils.getCleanShardName(TEST_ID.getRootIdentifier()));
 
+        LOG.warn("Got after waiting for nonleader");
         final ActorRef leaderShardManager = leaderDistributedDataStore.getActorContext().getShardManager();
 
-        new JavaTestKit(leaderSystem) {
-            {
-                leaderShardManager.tell(
-                        new FindLocalShard(ClusterUtils.getCleanShardName(TestModel.TEST_PATH), true), getRef());
-                expectMsgClass(duration("5 seconds"), LocalShardFound.class);
+        leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
+                ClusterUtils.getCleanShardName(TestModel.TEST_PATH));
 
-                final ActorRef followerShardManager = followerDistributedDataStore.getActorContext().getShardManager();
+        final YangInstanceIdentifier oid1 = getOuterListIdFor(0);
+        final DOMDataTreeIdentifier outerListPath = new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, oid1);
 
-                followerShardManager.tell(
-                        new FindLocalShard(ClusterUtils.getCleanShardName(TestModel.TEST_PATH), true), getRef());
-                expectMsgClass(duration("5 seconds"), LocalShardFound.class);
+        final DistributedShardRegistration outerListShardReg = waitOnAsyncTask(
+                leaderShardFactory.createDistributedShard(outerListPath, Lists.newArrayList(AbstractTest.MEMBER_NAME)),
+                DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
 
-                leaderDistributedDataStore.getActorContext().getShardManager().tell(
-                        new FindPrimary(ClusterUtils.getCleanShardName(TestModel.TEST_PATH), true), getRef());
-                expectMsgClass(duration("5 seconds"), LocalPrimaryShardFound.class);
+        leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
+                ClusterUtils.getCleanShardName(outerListPath.getRootIdentifier()));
+
+        final DOMDataTreeProducer shardProducer = leaderShardFactory.createProducer(
+                Collections.singletonList(outerListPath));
+
+        final DOMDataTreeCursorAwareTransaction tx = shardProducer.createTransaction(false);
+        final DOMDataTreeWriteCursor cursor =
+                tx.createCursor(new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, oid1));
+        assertNotNull(cursor);
+
+        MapNode innerList = ImmutableMapNodeBuilder
+                .create()
+                .withNodeIdentifier(new NodeIdentifier(TestModel.INNER_LIST_QNAME))
+                .build();
+
+        cursor.write(new NodeIdentifier(TestModel.INNER_LIST_QNAME), innerList);
+        cursor.close();
+        tx.submit().checkedGet();
+
+        final ArrayList<CheckedFuture<Void, TransactionCommitFailedException>> futures = new ArrayList<>();
+        for (int i = 0; i < 1000; i++) {
+            final Collection<MapEntryNode> innerListMapEntries = createInnerListMapEntries(1000, "run-" + i);
+            for (final MapEntryNode innerListMapEntry : innerListMapEntries) {
+                final DOMDataTreeCursorAwareTransaction tx1 = shardProducer.createTransaction(false);
+                final DOMDataTreeWriteCursor cursor1 = tx1.createCursor(
+                        new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION,
+                                oid1.node(new NodeIdentifier(TestModel.INNER_LIST_QNAME))));
+                cursor1.write(innerListMapEntry.getIdentifier(), innerListMapEntry);
+                cursor1.close();
+                futures.add(tx1.submit());
             }
-        };
+        }
 
-        final DOMDataTreeProducer producer = leaderShardFactory.createProducer(Collections.singleton(TEST_ID));
+        futures.get(futures.size() - 1).checkedGet();
 
-        final DOMDataTreeCursorAwareTransaction tx = producer.createTransaction(true);
-        final DOMDataTreeWriteCursor cursor = tx.createCursor(TEST_ID);
-        Assert.assertNotNull(cursor);
-        final YangInstanceIdentifier nameId =
-                YangInstanceIdentifier.builder(TestModel.TEST_PATH).node(TestModel.NAME_QNAME).build();
-        cursor.write(nameId.getLastPathArgument(),
-                ImmutableLeafNodeBuilder.<String>create()
-                        .withNodeIdentifier(new NodeIdentifier(TestModel.NAME_QNAME)).withValue("Test Value").build());
+        final DOMDataTreeListener mockedDataTreeListener = mock(DOMDataTreeListener.class);
+        doNothing().when(mockedDataTreeListener).onDataTreeChanged(anyCollection(), anyMap());
+
+        leaderShardFactory.registerListener(mockedDataTreeListener, Collections.singletonList(INNER_LIST_ID),
+                true, Collections.emptyList());
+
+        verify(mockedDataTreeListener, timeout(1000).times(1)).onDataTreeChanged(captorForChanges.capture(),
+                captorForSubtrees.capture());
+        verifyNoMoreInteractions(mockedDataTreeListener);
+        final List<Collection<DataTreeCandidate>> capturedValue = captorForChanges.getAllValues();
+
+        final NormalizedNode<?,?> expected =
+                ImmutableMapNodeBuilder
+                        .create()
+                        .withNodeIdentifier(new NodeIdentifier(TestModel.INNER_LIST_QNAME))
+                                // only the values from the last run should be present
+                        .withValue(createInnerListMapEntries(1000, "run-999"))
+                        .build();
+
+        assertEquals("List values dont match the expected values from the last run",
+                expected, capturedValue.get(0).iterator().next().getRootNode().getDataAfter().get());
+
+    }
+
+    // top level shard at TEST element, with subshards on each outer-list map entry
+    @Test
+    public void testMultipleShardLevels() throws Exception {
+        initEmptyDatastore("config");
+
+        final DistributedShardRegistration testShardId = 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));
+        }
+
+        final DOMDataTreeIdentifier rootId =
+                new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, YangInstanceIdentifier.EMPTY);
+        final DOMDataTreeProducer producer = leaderShardFactory.createProducer(Collections.singletonList(
+                rootId));
+
+        DOMDataTreeCursorAwareTransaction transaction = producer.createTransaction(false);
+
+        DOMDataTreeWriteCursor cursor = transaction.createCursor(rootId);
+        assertNotNull(cursor);
+
+        final MapNode outerList =
+                ImmutableMapNodeBuilder.create()
+                        .withNodeIdentifier(new NodeIdentifier(TestModel.OUTER_LIST_QNAME)).build();
+
+        final ContainerNode testNode =
+                ImmutableContainerNodeBuilder.create()
+                        .withNodeIdentifier(new NodeIdentifier(TestModel.TEST_QNAME))
+                        .withChild(outerList)
+                        .build();
+
+        cursor.write(testNode.getIdentifier(), testNode);
+
+        cursor.close();
+        transaction.submit().checkedGet();
+
+        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();
-        tx.submit();
 
+        transaction.submit().checkedGet();
+
+        leaderShardFactory.registerListener(mockedDataTreeListener, Collections.singletonList(TEST_ID),
+                true, Collections.emptyList());
+
+        // need 6 invocations, first initial thats from the parent shard, and then each individual subshard
+        verify(mockedDataTreeListener, timeout(10000).times(6)).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();
+
+        assertEquals(expected, actual);
+    }
+
+    @Test
+    public void testDistributedData() throws Exception {
+        initEmptyDatastore("config");
+
+        waitOnAsyncTask(
+                leaderShardFactory.createDistributedShard(TEST_ID, Lists.newArrayList(AbstractTest.MEMBER_NAME)),
+                DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
+
+        waitOnAsyncTask(
+                leaderShardFactory.createDistributedShard(
+                        new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, TestModel.OUTER_CONTAINER_PATH),
+                        Lists.newArrayList(AbstractTest.MEMBER_NAME)),
+                DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
+
+        waitOnAsyncTask(
+                leaderShardFactory.createDistributedShard(
+                        new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, TestModel.INNER_LIST_PATH),
+                        Lists.newArrayList(AbstractTest.MEMBER_NAME)),
+                DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
+
+        waitOnAsyncTask(
+                leaderShardFactory.createDistributedShard(
+                        new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, TestModel.JUNK_PATH),
+                        Lists.newArrayList(AbstractTest.MEMBER_NAME)),
+                DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
+
+        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));
+
+    }
+
+    @Test
+    public void testMultipleRegistrationsAtOnePrefix() throws Exception {
+        initEmptyDatastore("config");
+
+        for (int i = 0; i < 10; i++) {
+            LOG.debug("Round {}", i);
+            final DistributedShardRegistration reg1 = waitOnAsyncTask(leaderShardFactory.createDistributedShard(
+                    TEST_ID, Lists.newArrayList(AbstractTest.MEMBER_NAME)),
+                    DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
+
+            leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
+                    ClusterUtils.getCleanShardName(TestModel.TEST_PATH));
+
+            assertNotNull(findLocalShard(leaderDistributedDataStore.getActorContext(),
+                    ClusterUtils.getCleanShardName(TestModel.TEST_PATH)));
+
+            waitOnAsyncTask(reg1.close(), DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
+
+            waitUntilShardIsDown(leaderDistributedDataStore.getActorContext(),
+                    ClusterUtils.getCleanShardName(TestModel.TEST_PATH));
+        }
+    }
+
+    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(new NodeIdentifierWithPredicates(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(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());
+        }
+
+        return ret;
+    }
 
+    private static YangInstanceIdentifier getOuterListIdFor(final int id) {
+        return TestModel.OUTER_LIST_PATH.node(new NodeIdentifierWithPredicates(
+                TestModel.OUTER_LIST_QNAME, QName.create(TestModel.OUTER_LIST_QNAME, "id"), id));
     }
 }