Fix DistributesShardedDOMDataTree.ProxyProducer's getShardAccess
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / sharding / DistributedShardedDOMDataTreeTest.java
1 /*
2  * Copyright (c) 2016 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.sharding;
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.mockito.Matchers.anyCollection;
15 import static org.mockito.Matchers.anyMap;
16 import static org.mockito.Mockito.doNothing;
17 import static org.mockito.Mockito.doReturn;
18 import static org.mockito.Mockito.mock;
19 import static org.mockito.Mockito.timeout;
20 import static org.mockito.Mockito.verify;
21 import static org.mockito.Mockito.verifyNoMoreInteractions;
22 import static org.opendaylight.controller.cluster.datastore.IntegrationTestKit.findLocalShard;
23 import static org.opendaylight.controller.cluster.datastore.IntegrationTestKit.waitUntilShardIsDown;
24
25 import akka.actor.ActorRef;
26 import akka.actor.ActorSystem;
27 import akka.actor.Address;
28 import akka.actor.AddressFromURIString;
29 import akka.cluster.Cluster;
30 import akka.testkit.JavaTestKit;
31 import com.google.common.base.Optional;
32 import com.google.common.collect.Lists;
33 import com.google.common.util.concurrent.CheckedFuture;
34 import com.typesafe.config.ConfigFactory;
35 import java.util.ArrayList;
36 import java.util.Collection;
37 import java.util.Collections;
38 import java.util.List;
39 import java.util.Map;
40 import java.util.Set;
41 import java.util.concurrent.CompletionStage;
42 import org.junit.After;
43 import org.junit.Assert;
44 import org.junit.Before;
45 import org.junit.Ignore;
46 import org.junit.Test;
47 import org.mockito.ArgumentCaptor;
48 import org.mockito.Captor;
49 import org.mockito.Mockito;
50 import org.mockito.MockitoAnnotations;
51 import org.opendaylight.controller.cluster.ActorSystemProvider;
52 import org.opendaylight.controller.cluster.access.concepts.MemberName;
53 import org.opendaylight.controller.cluster.datastore.AbstractTest;
54 import org.opendaylight.controller.cluster.datastore.DatastoreContext;
55 import org.opendaylight.controller.cluster.datastore.DatastoreContext.Builder;
56 import org.opendaylight.controller.cluster.datastore.DistributedDataStore;
57 import org.opendaylight.controller.cluster.datastore.IntegrationTestKit;
58 import org.opendaylight.controller.cluster.datastore.utils.ClusterUtils;
59 import org.opendaylight.controller.cluster.dom.api.CDSDataTreeProducer;
60 import org.opendaylight.controller.cluster.dom.api.CDSShardAccess;
61 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
62 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
63 import org.opendaylight.controller.cluster.sharding.DistributedShardFactory.DistributedShardRegistration;
64 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
65 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
66 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
67 import org.opendaylight.mdsal.common.api.TransactionCommitFailedException;
68 import org.opendaylight.mdsal.dom.api.DOMDataTreeCursorAwareTransaction;
69 import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier;
70 import org.opendaylight.mdsal.dom.api.DOMDataTreeListener;
71 import org.opendaylight.mdsal.dom.api.DOMDataTreeProducer;
72 import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteCursor;
73 import org.opendaylight.yangtools.yang.common.QName;
74 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
75 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.NodeIdentifier;
76 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.NodeIdentifierWithPredicates;
77 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
78 import org.opendaylight.yangtools.yang.data.api.schema.LeafNode;
79 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
80 import org.opendaylight.yangtools.yang.data.api.schema.MapNode;
81 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
82 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidate;
83 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
84 import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder;
85 import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableLeafNodeBuilder;
86 import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableMapNodeBuilder;
87 import org.slf4j.Logger;
88 import org.slf4j.LoggerFactory;
89
90 public class DistributedShardedDOMDataTreeTest extends AbstractTest {
91
92     private static final Logger LOG = LoggerFactory.getLogger(DistributedShardedDOMDataTreeRemotingTest.class);
93
94     private static final Address MEMBER_1_ADDRESS =
95             AddressFromURIString.parse("akka.tcp://cluster-test@127.0.0.1:2558");
96
97     private static final DOMDataTreeIdentifier TEST_ID =
98             new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, TestModel.TEST_PATH);
99
100     private static final DOMDataTreeIdentifier INNER_LIST_ID =
101             new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION,
102                     YangInstanceIdentifier.create(getOuterListIdFor(0).getPathArguments())
103                             .node(TestModel.INNER_LIST_QNAME));
104     private static final Set<MemberName> SINGLE_MEMBER = Collections.singleton(AbstractTest.MEMBER_NAME);
105
106     private static final String MODULE_SHARDS_CONFIG = "module-shards-cars-member-1.conf";
107
108     private ActorSystem leaderSystem;
109
110     private final Builder leaderDatastoreContextBuilder =
111             DatastoreContext.newBuilder()
112                     .shardHeartbeatIntervalInMillis(100)
113                     .shardElectionTimeoutFactor(2)
114                     .logicalStoreType(
115                             org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType.CONFIGURATION);
116
117     private DistributedDataStore leaderDistributedDataStore;
118     private DistributedDataStore operDistributedDatastore;
119     private IntegrationTestKit leaderTestKit;
120
121     private DistributedShardedDOMDataTree leaderShardFactory;
122
123     @Captor
124     private ArgumentCaptor<Collection<DataTreeCandidate>> captorForChanges;
125     @Captor
126     private ArgumentCaptor<Map<DOMDataTreeIdentifier, NormalizedNode<?, ?>>> captorForSubtrees;
127
128     private ActorSystemProvider leaderSystemProvider;
129
130     @Before
131     public void setUp() {
132         MockitoAnnotations.initMocks(this);
133
134         InMemoryJournal.clear();
135         InMemorySnapshotStore.clear();
136
137         leaderSystem = ActorSystem.create("cluster-test", ConfigFactory.load().getConfig("Member1"));
138         Cluster.get(leaderSystem).join(MEMBER_1_ADDRESS);
139
140         leaderSystemProvider = Mockito.mock(ActorSystemProvider.class);
141         doReturn(leaderSystem).when(leaderSystemProvider).getActorSystem();
142     }
143
144     @After
145     public void tearDown() throws Exception {
146         if (leaderDistributedDataStore != null) {
147             leaderDistributedDataStore.close();
148         }
149
150         if (operDistributedDatastore != null) {
151             operDistributedDatastore.close();
152         }
153
154         JavaTestKit.shutdownActorSystem(leaderSystem);
155
156         InMemoryJournal.clear();
157         InMemorySnapshotStore.clear();
158     }
159
160     private void initEmptyDatastores() throws Exception {
161         leaderTestKit = new IntegrationTestKit(leaderSystem, leaderDatastoreContextBuilder);
162
163         leaderDistributedDataStore = leaderTestKit.setupDistributedDataStore(
164                 "config", MODULE_SHARDS_CONFIG, "empty-modules.conf", true,
165                 SchemaContextHelper.distributedShardedDOMDataTreeSchemaContext());
166
167         operDistributedDatastore = leaderTestKit.setupDistributedDataStore(
168                 "operational", MODULE_SHARDS_CONFIG, "empty-modules.conf",true,
169                 SchemaContextHelper.distributedShardedDOMDataTreeSchemaContext());
170
171         leaderShardFactory = new DistributedShardedDOMDataTree(leaderSystemProvider,
172                 operDistributedDatastore,
173                 leaderDistributedDataStore);
174
175         leaderShardFactory.init();
176     }
177
178
179     @Test
180     public void testWritesIntoDefaultShard() throws Exception {
181         initEmptyDatastores();
182
183         final DOMDataTreeIdentifier configRoot =
184                 new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, YangInstanceIdentifier.EMPTY);
185
186         final DOMDataTreeProducer producer = leaderShardFactory.createProducer(Collections.singleton(configRoot));
187
188         final DOMDataTreeCursorAwareTransaction tx = producer.createTransaction(true);
189         final DOMDataTreeWriteCursor cursor =
190                 tx.createCursor(new DOMDataTreeIdentifier(
191                         LogicalDatastoreType.CONFIGURATION, YangInstanceIdentifier.EMPTY));
192         Assert.assertNotNull(cursor);
193
194         final ContainerNode test =
195                 ImmutableContainerNodeBuilder.create()
196                         .withNodeIdentifier(new NodeIdentifier(TestModel.TEST_QNAME)).build();
197
198         cursor.write(test.getIdentifier(), test);
199         cursor.close();
200
201         tx.submit().checkedGet();
202     }
203
204     @Test
205     public void testSingleNodeWrites() throws Exception {
206         initEmptyDatastores();
207
208         final DistributedShardRegistration shardRegistration = waitOnAsyncTask(
209                 leaderShardFactory.createDistributedShard(TEST_ID, Lists.newArrayList(AbstractTest.MEMBER_NAME)),
210                 DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
211
212         leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
213                 ClusterUtils.getCleanShardName(TEST_ID.getRootIdentifier()));
214
215         final DOMDataTreeProducer producer = leaderShardFactory.createProducer(Collections.singleton(TEST_ID));
216
217         final DOMDataTreeCursorAwareTransaction tx = producer.createTransaction(true);
218         final DOMDataTreeWriteCursor cursor = tx.createCursor(TEST_ID);
219         Assert.assertNotNull(cursor);
220         final YangInstanceIdentifier nameId =
221                 YangInstanceIdentifier.builder(TestModel.TEST_PATH).node(TestModel.NAME_QNAME).build();
222         final LeafNode<String> valueToCheck = ImmutableLeafNodeBuilder.<String>create().withNodeIdentifier(
223                 new NodeIdentifier(TestModel.NAME_QNAME)).withValue("Test Value").build();
224         LOG.debug("Writing data {} at {}, cursor {}", nameId.getLastPathArgument(), valueToCheck, cursor);
225         cursor.write(nameId.getLastPathArgument(),
226                 valueToCheck);
227
228         cursor.close();
229         LOG.debug("Got to pre submit");
230
231         tx.submit().checkedGet();
232
233         final DOMDataTreeListener mockedDataTreeListener = mock(DOMDataTreeListener.class);
234         doNothing().when(mockedDataTreeListener).onDataTreeChanged(anyCollection(), anyMap());
235
236         leaderShardFactory.registerListener(mockedDataTreeListener, Collections.singletonList(TEST_ID),
237                 true, Collections.emptyList());
238
239         verify(mockedDataTreeListener, timeout(1000).times(1)).onDataTreeChanged(captorForChanges.capture(),
240                 captorForSubtrees.capture());
241         final List<Collection<DataTreeCandidate>> capturedValue = captorForChanges.getAllValues();
242
243         final Optional<NormalizedNode<?, ?>> dataAfter =
244                 capturedValue.get(0).iterator().next().getRootNode().getDataAfter();
245
246         final NormalizedNode<?,?> expected = ImmutableContainerNodeBuilder.create()
247                 .withNodeIdentifier(new NodeIdentifier(TestModel.TEST_QNAME)).withChild(valueToCheck).build();
248         assertEquals(expected, dataAfter.get());
249
250         verifyNoMoreInteractions(mockedDataTreeListener);
251
252         shardRegistration.close().toCompletableFuture().get();
253
254     }
255
256     @Test
257     public void testMultipleWritesIntoSingleMapEntry() throws Exception {
258         initEmptyDatastores();
259
260         final DistributedShardRegistration shardRegistration = waitOnAsyncTask(
261                 leaderShardFactory.createDistributedShard(TEST_ID, Lists.newArrayList(AbstractTest.MEMBER_NAME)),
262                 DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
263
264         leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
265                 ClusterUtils.getCleanShardName(TEST_ID.getRootIdentifier()));
266
267         LOG.warn("Got after waiting for nonleader");
268         final ActorRef leaderShardManager = leaderDistributedDataStore.getActorContext().getShardManager();
269
270         leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
271                 ClusterUtils.getCleanShardName(TestModel.TEST_PATH));
272
273         final YangInstanceIdentifier oid1 = getOuterListIdFor(0);
274         final DOMDataTreeIdentifier outerListPath = new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, oid1);
275
276         final DistributedShardRegistration outerListShardReg = waitOnAsyncTask(
277                 leaderShardFactory.createDistributedShard(outerListPath, Lists.newArrayList(AbstractTest.MEMBER_NAME)),
278                 DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
279
280         leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
281                 ClusterUtils.getCleanShardName(outerListPath.getRootIdentifier()));
282
283         final DOMDataTreeProducer shardProducer = leaderShardFactory.createProducer(
284                 Collections.singletonList(outerListPath));
285
286         final DOMDataTreeCursorAwareTransaction tx = shardProducer.createTransaction(false);
287         final DOMDataTreeWriteCursor cursor =
288                 tx.createCursor(new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, oid1));
289         assertNotNull(cursor);
290
291         MapNode innerList = ImmutableMapNodeBuilder
292                 .create()
293                 .withNodeIdentifier(new NodeIdentifier(TestModel.INNER_LIST_QNAME))
294                 .build();
295
296         cursor.write(new NodeIdentifier(TestModel.INNER_LIST_QNAME), innerList);
297         cursor.close();
298         tx.submit().checkedGet();
299
300         final ArrayList<CheckedFuture<Void, TransactionCommitFailedException>> futures = new ArrayList<>();
301         for (int i = 0; i < 1000; i++) {
302             final Collection<MapEntryNode> innerListMapEntries = createInnerListMapEntries(1000, "run-" + i);
303             for (final MapEntryNode innerListMapEntry : innerListMapEntries) {
304                 final DOMDataTreeCursorAwareTransaction tx1 = shardProducer.createTransaction(false);
305                 final DOMDataTreeWriteCursor cursor1 = tx1.createCursor(
306                         new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION,
307                                 oid1.node(new NodeIdentifier(TestModel.INNER_LIST_QNAME))));
308                 cursor1.write(innerListMapEntry.getIdentifier(), innerListMapEntry);
309                 cursor1.close();
310                 futures.add(tx1.submit());
311             }
312         }
313
314         futures.get(futures.size() - 1).checkedGet();
315
316         final DOMDataTreeListener mockedDataTreeListener = mock(DOMDataTreeListener.class);
317         doNothing().when(mockedDataTreeListener).onDataTreeChanged(anyCollection(), anyMap());
318
319         leaderShardFactory.registerListener(mockedDataTreeListener, Collections.singletonList(INNER_LIST_ID),
320                 true, Collections.emptyList());
321
322         verify(mockedDataTreeListener, timeout(1000).times(1)).onDataTreeChanged(captorForChanges.capture(),
323                 captorForSubtrees.capture());
324         verifyNoMoreInteractions(mockedDataTreeListener);
325         final List<Collection<DataTreeCandidate>> capturedValue = captorForChanges.getAllValues();
326
327         final NormalizedNode<?,?> expected =
328                 ImmutableMapNodeBuilder
329                         .create()
330                         .withNodeIdentifier(new NodeIdentifier(TestModel.INNER_LIST_QNAME))
331                                 // only the values from the last run should be present
332                         .withValue(createInnerListMapEntries(1000, "run-999"))
333                         .build();
334
335         assertEquals("List values dont match the expected values from the last run",
336                 expected, capturedValue.get(0).iterator().next().getRootNode().getDataAfter().get());
337
338     }
339
340     // top level shard at TEST element, with subshards on each outer-list map entry
341     @Test
342     @Ignore
343     public void testMultipleShardLevels() throws Exception {
344         initEmptyDatastores();
345
346         final DistributedShardRegistration testShardReg = waitOnAsyncTask(
347                 leaderShardFactory.createDistributedShard(TEST_ID, SINGLE_MEMBER),
348                 DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
349
350         final ArrayList<DistributedShardRegistration> registrations = new ArrayList<>();
351         final int listSize = 5;
352         for (int i = 0; i < listSize; i++) {
353             final YangInstanceIdentifier entryYID = getOuterListIdFor(i);
354             final CompletionStage<DistributedShardRegistration> future = leaderShardFactory.createDistributedShard(
355                     new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, entryYID), SINGLE_MEMBER);
356
357             registrations.add(waitOnAsyncTask(future, DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION));
358         }
359
360         final DOMDataTreeIdentifier rootId =
361                 new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, YangInstanceIdentifier.EMPTY);
362         final DOMDataTreeProducer producer = leaderShardFactory.createProducer(Collections.singletonList(
363                 rootId));
364
365         DOMDataTreeCursorAwareTransaction transaction = producer.createTransaction(false);
366
367         DOMDataTreeWriteCursor cursor = transaction.createCursor(rootId);
368         assertNotNull(cursor);
369
370         final MapNode outerList =
371                 ImmutableMapNodeBuilder.create()
372                         .withNodeIdentifier(new NodeIdentifier(TestModel.OUTER_LIST_QNAME)).build();
373
374         final ContainerNode testNode =
375                 ImmutableContainerNodeBuilder.create()
376                         .withNodeIdentifier(new NodeIdentifier(TestModel.TEST_QNAME))
377                         .withChild(outerList)
378                         .build();
379
380         cursor.write(testNode.getIdentifier(), testNode);
381
382         cursor.close();
383         transaction.submit().checkedGet();
384
385         final DOMDataTreeListener mockedDataTreeListener = mock(DOMDataTreeListener.class);
386         doNothing().when(mockedDataTreeListener).onDataTreeChanged(anyCollection(), anyMap());
387
388         final MapNode wholeList = ImmutableMapNodeBuilder.create(outerList)
389                 .withValue(createOuterEntries(listSize, "testing-values")).build();
390
391         transaction = producer.createTransaction(false);
392         cursor = transaction.createCursor(TEST_ID);
393         assertNotNull(cursor);
394
395         cursor.write(wholeList.getIdentifier(), wholeList);
396         cursor.close();
397
398         transaction.submit().checkedGet();
399
400         leaderShardFactory.registerListener(mockedDataTreeListener, Collections.singletonList(TEST_ID),
401                 true, Collections.emptyList());
402
403         verify(mockedDataTreeListener, timeout(35000).atLeast(2)).onDataTreeChanged(captorForChanges.capture(),
404                 captorForSubtrees.capture());
405         verifyNoMoreInteractions(mockedDataTreeListener);
406         final List<Map<DOMDataTreeIdentifier, NormalizedNode<?, ?>>> allSubtrees = captorForSubtrees.getAllValues();
407
408         final Map<DOMDataTreeIdentifier, NormalizedNode<?, ?>> lastSubtree = allSubtrees.get(allSubtrees.size() - 1);
409
410         final NormalizedNode<?, ?> actual = lastSubtree.get(TEST_ID);
411         assertNotNull(actual);
412
413         final NormalizedNode<?, ?> expected =
414                 ImmutableContainerNodeBuilder.create()
415                         .withNodeIdentifier(new NodeIdentifier(TestModel.TEST_QNAME))
416                         .withChild(ImmutableMapNodeBuilder.create(outerList)
417                                 .withValue(createOuterEntries(listSize, "testing-values")).build())
418                         .build();
419
420
421         for (final DistributedShardRegistration registration : registrations) {
422             waitOnAsyncTask(registration.close(), DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
423         }
424
425         waitOnAsyncTask(testShardReg.close(), DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
426
427         assertEquals(expected, actual);
428     }
429
430     @Test
431     public void testMultipleRegistrationsAtOnePrefix() throws Exception {
432         initEmptyDatastores();
433
434         for (int i = 0; i < 10; i++) {
435             LOG.debug("Round {}", i);
436             final DistributedShardRegistration reg1 = waitOnAsyncTask(leaderShardFactory.createDistributedShard(
437                     TEST_ID, Lists.newArrayList(AbstractTest.MEMBER_NAME)),
438                     DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
439
440             leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
441                     ClusterUtils.getCleanShardName(TestModel.TEST_PATH));
442
443             assertNotNull(findLocalShard(leaderDistributedDataStore.getActorContext(),
444                     ClusterUtils.getCleanShardName(TestModel.TEST_PATH)));
445
446             waitOnAsyncTask(reg1.close(), DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
447
448             waitUntilShardIsDown(leaderDistributedDataStore.getActorContext(),
449                     ClusterUtils.getCleanShardName(TestModel.TEST_PATH));
450         }
451     }
452
453     @Test
454     public void testCDSDataTreeProducer() throws Exception {
455         initEmptyDatastores();
456
457         final DistributedShardRegistration reg1 = waitOnAsyncTask(leaderShardFactory.createDistributedShard(
458                 TEST_ID, Lists.newArrayList(AbstractTest.MEMBER_NAME)),
459                 DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
460
461         leaderTestKit.waitUntilLeader(leaderDistributedDataStore.getActorContext(),
462                 ClusterUtils.getCleanShardName(TestModel.TEST_PATH));
463
464         assertNotNull(findLocalShard(leaderDistributedDataStore.getActorContext(),
465                 ClusterUtils.getCleanShardName(TestModel.TEST_PATH)));
466
467
468         final DOMDataTreeIdentifier configRoot =
469                 new DOMDataTreeIdentifier(LogicalDatastoreType.CONFIGURATION, YangInstanceIdentifier.EMPTY);
470         final DOMDataTreeProducer producer = leaderShardFactory.createProducer(Collections.singleton(configRoot));
471
472         assertTrue(producer instanceof CDSDataTreeProducer);
473
474         final CDSDataTreeProducer cdsProducer = (CDSDataTreeProducer) producer;
475         CDSShardAccess shardAccess = cdsProducer.getShardAccess(TEST_ID);
476         assertEquals(shardAccess.getShardIdentifier(), TEST_ID);
477
478         shardAccess = cdsProducer.getShardAccess(INNER_LIST_ID);
479         assertEquals(TEST_ID, shardAccess.getShardIdentifier());
480
481         shardAccess = cdsProducer.getShardAccess(configRoot);
482         assertEquals(configRoot, shardAccess.getShardIdentifier());
483
484         waitOnAsyncTask(reg1.close(), DistributedShardedDOMDataTree.SHARD_FUTURE_TIMEOUT_DURATION);
485     }
486
487     private static Collection<MapEntryNode> createOuterEntries(final int amount, final String valuePrefix) {
488         final Collection<MapEntryNode> ret = new ArrayList<>();
489         for (int i = 0; i < amount; i++) {
490             ret.add(ImmutableNodes.mapEntryBuilder()
491                     .withNodeIdentifier(new NodeIdentifierWithPredicates(TestModel.OUTER_LIST_QNAME,
492                             QName.create(TestModel.OUTER_LIST_QNAME, "id"), i))
493                     .withChild(ImmutableNodes
494                             .leafNode(QName.create(TestModel.OUTER_LIST_QNAME, "id"), i))
495                     .withChild(createWholeInnerList(amount, "outer id: " + i + " " + valuePrefix))
496                     .build());
497         }
498
499         return ret;
500     }
501
502     private static MapNode createWholeInnerList(final int amount, final String valuePrefix) {
503         return ImmutableMapNodeBuilder.create().withNodeIdentifier(new NodeIdentifier(TestModel.INNER_LIST_QNAME))
504                 .withValue(createInnerListMapEntries(amount, valuePrefix)).build();
505     }
506
507     private static Collection<MapEntryNode> createInnerListMapEntries(final int amount, final String valuePrefix) {
508         final Collection<MapEntryNode> ret = new ArrayList<>();
509         for (int i = 0; i < amount; i++) {
510             ret.add(ImmutableNodes.mapEntryBuilder()
511                     .withNodeIdentifier(new NodeIdentifierWithPredicates(TestModel.INNER_LIST_QNAME,
512                             QName.create(TestModel.INNER_LIST_QNAME, "name"), Integer.toString(i)))
513                     .withChild(ImmutableNodes
514                             .leafNode(QName.create(TestModel.INNER_LIST_QNAME, "value"), valuePrefix + "-" + i))
515                     .build());
516         }
517
518         return ret;
519     }
520
521     private static YangInstanceIdentifier getOuterListIdFor(final int id) {
522         return TestModel.OUTER_LIST_PATH.node(new NodeIdentifierWithPredicates(
523                 TestModel.OUTER_LIST_QNAME, QName.create(TestModel.OUTER_LIST_QNAME, "id"), id));
524     }
525 }