akka.actor.provider set to 'cluster'
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / ShardTest.java
index 46fdfacbe956497db88acba4e9d2de831aa6991e..7ac18e61dc146a6b27430c884864c39ffe8620d5 100644 (file)
@@ -52,7 +52,6 @@ import org.opendaylight.controller.cluster.access.concepts.MemberName;
 import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
-import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
 import org.opendaylight.controller.cluster.datastore.messages.AbortTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.AbortTransactionReply;
 import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
@@ -169,7 +168,7 @@ public class ShardTest extends AbstractShardTest {
     public void testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration() throws Exception {
         final CountDownLatch onFirstElectionTimeout = new CountDownLatch(1);
         final CountDownLatch onChangeListenerRegistered = new CountDownLatch(1);
-        final Creator<Shard> creator = new Creator<Shard>() {
+        final Creator<Shard> creator = new Creator<>() {
             boolean firstElectionTimeout = true;
 
             @Override
@@ -202,8 +201,8 @@ public class ShardTest extends AbstractShardTest {
         final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener, path),
                 "testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration-DataChangeListener");
 
-        final TestActorRef<Shard> shard = actorFactory.createTestActor(
-                Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
+        final TestActorRef<Shard> shard = actorFactory.createTestActor(Props.create(Shard.class,
+                new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
                 "testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration");
 
         final ShardTestKit testKit = new ShardTestKit(getSystem());
@@ -240,8 +239,8 @@ public class ShardTest extends AbstractShardTest {
             CreateTransactionReply.class);
 
         final String path = reply.getTransactionPath().toString();
-        assertTrue("Unexpected transaction path " + path, path.startsWith(String.format(
-            "akka://test/user/testCreateTransaction/shard-%s-%s:ShardTransactionTest@0:",
+        assertTrue("Unexpected transaction path " + path, path.contains(String.format(
+            "/user/testCreateTransaction/shard-%s-%s:ShardTransactionTest@0:",
             shardID.getShardName(), shardID.getMemberName().getName())));
     }
 
@@ -259,8 +258,8 @@ public class ShardTest extends AbstractShardTest {
             CreateTransactionReply.class);
 
         final String path = reply.getTransactionPath().toString();
-        assertTrue("Unexpected transaction path " + path, path.startsWith(String.format(
-            "akka://test/user/testCreateTransactionOnChain/shard-%s-%s:ShardTransactionTest@0:",
+        assertTrue("Unexpected transaction path " + path, path.contains(String.format(
+            "/user/testCreateTransactionOnChain/shard-%s-%s:ShardTransactionTest@0:",
             shardID.getShardName(), shardID.getMemberName().getName())));
     }
 
@@ -299,8 +298,8 @@ public class ShardTest extends AbstractShardTest {
 
         writeToStore(store, TestModel.TEST_PATH, container);
 
-        final YangInstanceIdentifier root = YangInstanceIdentifier.EMPTY;
-        final NormalizedNode<?,?> expected = readStore(store, root);
+        final YangInstanceIdentifier root = YangInstanceIdentifier.empty();
+        final NormalizedNode expected = readStore(store, root);
 
         final Snapshot snapshot = Snapshot.create(new ShardSnapshotState(new MetadataShardDataTreeSnapshot(expected)),
                 Collections.emptyList(), 1, 2, 3, 4, -1, null, null);
@@ -344,7 +343,7 @@ public class ShardTest extends AbstractShardTest {
         while (sw.elapsed(TimeUnit.SECONDS) <= 5) {
             Uninterruptibles.sleepUninterruptibly(75, TimeUnit.MILLISECONDS);
 
-            final NormalizedNode<?,?> actual = readStore(shard, TestModel.TEST_PATH);
+            final NormalizedNode actual = readStore(shard, TestModel.TEST_PATH);
             if (actual != null) {
                 assertEquals("Applied state", node, actual);
                 return;
@@ -373,7 +372,7 @@ public class ShardTest extends AbstractShardTest {
 
         // Add some ModificationPayload entries
         for (int i = 1; i <= nListEntries; i++) {
-            listEntryKeys.add(Integer.valueOf(i));
+            listEntryKeys.add(i);
 
             final YangInstanceIdentifier path = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
                     .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i).build();
@@ -389,7 +388,7 @@ public class ShardTest extends AbstractShardTest {
         InMemoryJournal.addEntry(shardID.toString(), nListEntries + 2,
             new ApplyJournalEntries(nListEntries));
 
-        testRecovery(listEntryKeys);
+        testRecovery(listEntryKeys, true);
     }
 
     @Test
@@ -479,7 +478,9 @@ public class ShardTest extends AbstractShardTest {
             ImmutableNodes.containerNode(TestModel.TEST_QNAME), false), testKit.getRef());
         final ReadyTransactionReply readyReply = ReadyTransactionReply
                 .fromSerializable(testKit.expectMsgClass(duration, ReadyTransactionReply.class));
-        assertEquals("Cohort path", shard.path().toString(), readyReply.getCohortPath());
+
+        String pathSuffix = shard.path().toString().replaceFirst("akka://test", "");
+        assertTrue("Cohort path", readyReply.getCohortPath().endsWith(pathSuffix));
         // Send the CanCommitTransaction message for the first Tx.
 
         shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), testKit.getRef());
@@ -749,7 +750,7 @@ public class ShardTest extends AbstractShardTest {
 
         // Verify data in the data store.
 
-        final NormalizedNode<?, ?> actualNode = readStore(shard, path);
+        final NormalizedNode actualNode = readStore(shard, path);
         assertEquals("Stored node", containerNode, actualNode);
     }
 
@@ -757,7 +758,7 @@ public class ShardTest extends AbstractShardTest {
     public void testOnBatchedModificationsWhenNotLeader() {
         final AtomicBoolean overrideLeaderCalls = new AtomicBoolean();
         final ShardTestKit testKit = new ShardTestKit(getSystem());
-        final Creator<Shard> creator = new Creator<Shard>() {
+        final Creator<Shard> creator = new Creator<>() {
             private static final long serialVersionUID = 1L;
 
             @Override
@@ -777,8 +778,8 @@ public class ShardTest extends AbstractShardTest {
             }
         };
 
-        final TestActorRef<Shard> shard = actorFactory.createTestActor(Props
-            .create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
+        final TestActorRef<Shard> shard = actorFactory.createTestActor(Props.create(Shard.class,
+            new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
             "testOnBatchedModificationsWhenNotLeader");
 
         ShardTestKit.waitUntilLeader(shard);
@@ -839,7 +840,7 @@ public class ShardTest extends AbstractShardTest {
         ShardTestKit.waitUntilLeader(shard);
 
         final TransactionIdentifier transactionID = nextTransactionId();
-        final NormalizedNode<?, ?> containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
+        final NormalizedNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
         if (readWrite) {
             shard.tell(prepareForwardedReadyTransaction(shard, transactionID, TestModel.TEST_PATH, containerNode, true),
                 testKit.getRef());
@@ -850,7 +851,7 @@ public class ShardTest extends AbstractShardTest {
 
         testKit.expectMsgClass(Duration.ofSeconds(5), CommitTransactionReply.class);
 
-        final NormalizedNode<?, ?> actualNode = readStore(shard, TestModel.TEST_PATH);
+        final NormalizedNode actualNode = readStore(shard, TestModel.TEST_PATH);
         assertEquals(TestModel.TEST_QNAME.getLocalName(), containerNode, actualNode);
     }
 
@@ -883,7 +884,7 @@ public class ShardTest extends AbstractShardTest {
 
         testKit.expectMsgClass(CommitTransactionReply.class);
 
-        final NormalizedNode<?, ?> actualNode = readStore(shard, TestModel.OUTER_LIST_PATH);
+        final NormalizedNode actualNode = readStore(shard, TestModel.OUTER_LIST_PATH);
         assertEquals(TestModel.OUTER_LIST_QNAME.getLocalName(), mergeData, actualNode);
     }
 
@@ -928,7 +929,7 @@ public class ShardTest extends AbstractShardTest {
         shard.tell(new CommitTransaction(txId, CURRENT_VERSION).toSerializable(), testKit.getRef());
         testKit.expectMsgClass(CommitTransactionReply.class);
 
-        final NormalizedNode<?, ?> actualNode = readStore(shard, TestModel.OUTER_LIST_PATH);
+        final NormalizedNode actualNode = readStore(shard, TestModel.OUTER_LIST_PATH);
         assertEquals(TestModel.OUTER_LIST_QNAME.getLocalName(), mergeData, actualNode);
     }
 
@@ -947,7 +948,7 @@ public class ShardTest extends AbstractShardTest {
         final Duration duration = Duration.ofSeconds(5);
 
         final TransactionIdentifier transactionID = nextTransactionId();
-        final NormalizedNode<?, ?> containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
+        final NormalizedNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
         shard.tell(prepareBatchedModifications(transactionID, TestModel.TEST_PATH, containerNode, false),
             testKit.getRef());
         testKit.expectMsgClass(duration, ReadyTransactionReply.class);
@@ -964,7 +965,7 @@ public class ShardTest extends AbstractShardTest {
         shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), testKit.getRef());
         testKit.expectMsgClass(duration, CommitTransactionReply.class);
 
-        final NormalizedNode<?, ?> actualNode = readStore(shard, TestModel.TEST_PATH);
+        final NormalizedNode actualNode = readStore(shard, TestModel.TEST_PATH);
         assertEquals(TestModel.TEST_QNAME.getLocalName(), containerNode, actualNode);
     }
 
@@ -1084,7 +1085,7 @@ public class ShardTest extends AbstractShardTest {
         // Wait for the 2nd Tx to complete the canCommit phase.
 
         final CountDownLatch latch = new CountDownLatch(1);
-        canCommitFuture.onComplete(new OnComplete<Object>() {
+        canCommitFuture.onComplete(new OnComplete<>() {
             @Override
             public void onComplete(final Throwable failure, final Object resp) {
                 latch.countDown();
@@ -1154,7 +1155,7 @@ public class ShardTest extends AbstractShardTest {
         // Wait for the 2nd Tx to complete the canCommit phase.
 
         final CountDownLatch latch = new CountDownLatch(1);
-        canCommitFuture.onComplete(new OnComplete<Object>() {
+        canCommitFuture.onComplete(new OnComplete<>() {
             @Override
             public void onComplete(final Throwable failure, final Object resp) {
                 latch.countDown();
@@ -1182,7 +1183,7 @@ public class ShardTest extends AbstractShardTest {
         final Duration duration = Duration.ofSeconds(5);
         final TransactionIdentifier transactionID1 = nextTransactionId();
 
-        doThrow(new DataValidationFailedException(YangInstanceIdentifier.EMPTY, "mock canCommit failure"))
+        doThrow(new DataValidationFailedException(YangInstanceIdentifier.empty(), "mock canCommit failure"))
         .doNothing().when(dataTree).validate(any(DataTreeModification.class));
 
         shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH,
@@ -1223,7 +1224,7 @@ public class ShardTest extends AbstractShardTest {
 
         ShardTestKit.waitUntilLeader(shard);
 
-        doThrow(new DataValidationFailedException(YangInstanceIdentifier.EMPTY, "mock canCommit failure"))
+        doThrow(new DataValidationFailedException(YangInstanceIdentifier.empty(), "mock canCommit failure"))
         .doNothing().when(dataTree).validate(any(DataTreeModification.class));
 
         final Duration duration = Duration.ofSeconds(5);
@@ -1272,8 +1273,8 @@ public class ShardTest extends AbstractShardTest {
             }
         };
 
-        final TestActorRef<Shard> shard = actorFactory.createTestActor(Props
-            .create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
+        final TestActorRef<Shard> shard = actorFactory.createTestActor(Props.create(Shard.class,
+            new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
             "testAbortWithCommitPending");
 
         ShardTestKit.waitUntilLeader(shard);
@@ -1292,7 +1293,7 @@ public class ShardTest extends AbstractShardTest {
         shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), testKit.getRef());
         testKit.expectMsgClass(duration, CommitTransactionReply.class);
 
-        final NormalizedNode<?, ?> node = readStore(shard, TestModel.TEST_PATH);
+        final NormalizedNode node = readStore(shard, TestModel.TEST_PATH);
 
         // Since we're simulating an abort occurring during replication
         // and before finish commit,
@@ -1360,7 +1361,7 @@ public class ShardTest extends AbstractShardTest {
         shard.tell(new CommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), testKit.getRef());
         testKit.expectMsgClass(duration, CommitTransactionReply.class);
 
-        final NormalizedNode<?, ?> node = readStore(shard, listNodePath);
+        final NormalizedNode node = readStore(shard, listNodePath);
         assertNotNull(listNodePath + " not found", node);
     }
 
@@ -1520,7 +1521,7 @@ public class ShardTest extends AbstractShardTest {
 
         testKit.expectMsgClass(duration, CommitTransactionReply.class);
 
-        final NormalizedNode<?, ?> node = readStore(shard, TestModel.TEST2_PATH);
+        final NormalizedNode node = readStore(shard, TestModel.TEST2_PATH);
         assertNotNull(TestModel.TEST2_PATH + " not found", node);
     }
 
@@ -1705,9 +1706,9 @@ public class ShardTest extends AbstractShardTest {
 
         dataStoreContextBuilder.persistent(persistent);
 
-        class TestShard extends Shard {
+        final class TestShard extends Shard {
 
-            protected TestShard(final AbstractBuilder<?, ?> builder) {
+            TestShard(final AbstractBuilder<?, ?> builder) {
                 super(builder);
                 setPersistence(new TestPersistentDataProvider(super.persistence()));
             }
@@ -1730,14 +1731,13 @@ public class ShardTest extends AbstractShardTest {
 
         final Creator<Shard> creator = () -> new TestShard(newShardBuilder());
 
-        final TestActorRef<Shard> shard = actorFactory.createTestActor(Props
-            .create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
-            shardActorName);
+        final TestActorRef<Shard> shard = actorFactory.createTestActor(Props.create(Shard.class,
+            new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()), shardActorName);
 
         ShardTestKit.waitUntilLeader(shard);
         writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
 
-        final NormalizedNode<?, ?> expectedRoot = readStore(shard, YangInstanceIdentifier.EMPTY);
+        final NormalizedNode expectedRoot = readStore(shard, YangInstanceIdentifier.empty());
 
         // Trigger creation of a snapshot by ensuring
         final RaftActorContext raftActorContext = ((TestShard) shard.underlyingActor()).getRaftActorContext();
@@ -1749,7 +1749,7 @@ public class ShardTest extends AbstractShardTest {
     }
 
     private static void awaitAndValidateSnapshot(final AtomicReference<CountDownLatch> latch,
-            final AtomicReference<Object> savedSnapshot, final NormalizedNode<?, ?> expectedRoot)
+            final AtomicReference<Object> savedSnapshot, final NormalizedNode expectedRoot)
                     throws InterruptedException {
         assertTrue("Snapshot saved", latch.get().await(5, TimeUnit.SECONDS));
 
@@ -1761,8 +1761,8 @@ public class ShardTest extends AbstractShardTest {
         savedSnapshot.set(null);
     }
 
-    private static void verifySnapshot(final Snapshot snapshot, final NormalizedNode<?, ?> expectedRoot) {
-        final NormalizedNode<?, ?> actual = ((ShardSnapshotState)snapshot.getState()).getSnapshot().getRootNode().get();
+    private static void verifySnapshot(final Snapshot snapshot, final NormalizedNode expectedRoot) {
+        final NormalizedNode actual = ((ShardSnapshotState)snapshot.getState()).getSnapshot().getRootNode().get();
         assertEquals("Root node", expectedRoot, actual);
     }
 
@@ -1780,16 +1780,16 @@ public class ShardTest extends AbstractShardTest {
         commitTransaction(store, putTransaction);
 
 
-        final NormalizedNode<?, ?> expected = readStore(store, YangInstanceIdentifier.EMPTY);
+        final NormalizedNode expected = readStore(store, YangInstanceIdentifier.empty());
 
         final DataTreeModification writeTransaction = store.takeSnapshot().newModification();
 
-        writeTransaction.delete(YangInstanceIdentifier.EMPTY);
-        writeTransaction.write(YangInstanceIdentifier.EMPTY, expected);
+        writeTransaction.delete(YangInstanceIdentifier.empty());
+        writeTransaction.write(YangInstanceIdentifier.empty(), expected);
 
         commitTransaction(store, writeTransaction);
 
-        final NormalizedNode<?, ?> actual = readStore(store, YangInstanceIdentifier.EMPTY);
+        final NormalizedNode actual = readStore(store, YangInstanceIdentifier.empty());
 
         assertEquals(expected, actual);
     }