Use YangInstanceIdentifier.EMPTY
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / ShardTest.java
index f639c8db49c659921f1f472284eb8812ad95b485..994c91ca89e318d36d9b032928fb5843004189ca 100644 (file)
@@ -13,6 +13,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.inOrder;
 import static org.mockito.Mockito.mock;
@@ -31,6 +32,7 @@ import akka.persistence.SaveSnapshotSuccess;
 import akka.testkit.TestActorRef;
 import akka.util.Timeout;
 import com.google.common.base.Function;
+import com.google.common.base.Stopwatch;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.Uninterruptibles;
@@ -45,6 +47,7 @@ import org.junit.Test;
 import org.mockito.InOrder;
 import org.opendaylight.controller.cluster.DataPersistenceProvider;
 import org.opendaylight.controller.cluster.DelegatingPersistentDataProvider;
+import org.opendaylight.controller.cluster.access.concepts.MemberName;
 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;
@@ -88,6 +91,8 @@ import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus;
 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
+import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftState;
+import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState;
 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
 import org.opendaylight.controller.cluster.raft.messages.ServerRemoved;
 import org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy;
@@ -97,6 +102,7 @@ import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelpe
 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataBroker;
 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
+import org.opendaylight.yangtools.util.StringIdentifier;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
 import org.opendaylight.yangtools.yang.data.api.schema.MapNode;
@@ -381,48 +387,20 @@ public class ShardTest extends AbstractShardTest {
         }};
     }
 
-    @SuppressWarnings("serial")
     @Test
     public void testPeerAddressResolved() throws Exception {
         new ShardTestKit(getSystem()) {{
-            final CountDownLatch recoveryComplete = new CountDownLatch(1);
-            class TestShard extends Shard {
-                TestShard() {
-                    super(Shard.builder().id(shardID).datastoreContext(newDatastoreContext()).
-                            peerAddresses(Collections.<String, String>singletonMap(shardID.toString(), null)).
-                            schemaContext(SCHEMA_CONTEXT));
-                }
-
-                String getPeerAddress(String id) {
-                    return getRaftActorContext().getPeerAddress(id);
-                }
-
-                @Override
-                protected void onRecoveryComplete() {
-                    try {
-                        super.onRecoveryComplete();
-                    } finally {
-                        recoveryComplete.countDown();
-                    }
-                }
-            }
-
-            final TestActorRef<Shard> shard = actorFactory.createTestActor(
-                    Props.create(new DelegatingShardCreator(new Creator<Shard>() {
-                        @Override
-                        public TestShard create() throws Exception {
-                            return new TestShard();
-                        }
-                    })), "testPeerAddressResolved");
-
-            assertEquals("Recovery complete", true,
-                Uninterruptibles.awaitUninterruptibly(recoveryComplete, 5, TimeUnit.SECONDS));
+            ShardIdentifier peerID = ShardIdentifier.create("inventory", MemberName.forName("member-2"), "config");
+            final TestActorRef<Shard> shard = actorFactory.createTestActor(newShardBuilder().
+                    peerAddresses(Collections.<String, String>singletonMap(peerID.toString(), null)).props().
+                        withDispatcher(Dispatchers.DefaultDispatcherId()), "testPeerAddressResolved");
 
             final String address = "akka://foobar";
-            shard.underlyingActor().onReceiveCommand(new PeerAddressResolved(shardID.toString(), address));
+            shard.tell(new PeerAddressResolved(peerID.toString(), address), ActorRef.noSender());
 
-            assertEquals("getPeerAddress", address,
-                ((TestShard) shard.underlyingActor()).getPeerAddress(shardID.toString()));
+            shard.tell(GetOnDemandRaftState.INSTANCE, getRef());
+            OnDemandRaftState state = expectMsgClass(OnDemandRaftState.class);
+            assertEquals("getPeerAddress", address, state.getPeerAddresses().get(peerID.toString()));
         }};
     }
 
@@ -443,7 +421,7 @@ public class ShardTest extends AbstractShardTest {
 
         writeToStore(store, TestModel.TEST_PATH, container);
 
-        final YangInstanceIdentifier root = YangInstanceIdentifier.builder().build();
+        final YangInstanceIdentifier root = YangInstanceIdentifier.EMPTY;
         final NormalizedNode<?,?> expected = readStore(store, root);
 
         final Snapshot snapshot = Snapshot.create(SerializationUtils.serializeNormalizedNode(expected),
@@ -468,13 +446,23 @@ public class ShardTest extends AbstractShardTest {
         writeMod.write(TestModel.TEST_PATH, node);
         writeMod.ready();
 
-        final ApplyState applyState = new ApplyState(null, "test", new ReplicatedLogImplEntry(1, 2,
-                payloadForModification(source, writeMod)));
+        final ApplyState applyState = new ApplyState(null, new StringIdentifier("test"),
+            new ReplicatedLogImplEntry(1, 2, payloadForModification(source, writeMod)));
+
+        shard.tell(applyState, shard);
 
-        shard.underlyingActor().onReceiveCommand(applyState);
+        Stopwatch sw = Stopwatch.createStarted();
+        while(sw.elapsed(TimeUnit.SECONDS) <= 5) {
+            Uninterruptibles.sleepUninterruptibly(75, TimeUnit.MILLISECONDS);
 
-        final NormalizedNode<?,?> actual = readStore(shard, TestModel.TEST_PATH);
-        assertEquals("Applied state", node, actual);
+            final NormalizedNode<?,?> actual = readStore(shard, TestModel.TEST_PATH);
+            if(actual != null) {
+                assertEquals("Applied state", node, actual);
+                return;
+            }
+        }
+
+        fail("State was not applied");
     }
 
     @Test
@@ -691,15 +679,12 @@ public class ShardTest extends AbstractShardTest {
             final FiniteDuration duration = duration("5 seconds");
 
             final AtomicReference<ShardDataTreeCohort> mockCohort = new AtomicReference<>();
-            final ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() {
-                @Override
-                public ShardDataTreeCohort decorate(final String txID, final ShardDataTreeCohort actual) {
-                    if(mockCohort.get() == null) {
-                        mockCohort.set(createDelegatingMockCohort("cohort", actual));
-                    }
-
-                    return mockCohort.get();
+            final ShardCommitCoordinator.CohortDecorator cohortDecorator = (txID, actual) -> {
+                if(mockCohort.get() == null) {
+                    mockCohort.set(createDelegatingMockCohort("cohort", actual));
                 }
+
+                return mockCohort.get();
             };
 
             shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator);
@@ -757,15 +742,12 @@ public class ShardTest extends AbstractShardTest {
             final FiniteDuration duration = duration("5 seconds");
 
             final AtomicReference<ShardDataTreeCohort> mockCohort = new AtomicReference<>();
-            final ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() {
-                @Override
-                public ShardDataTreeCohort decorate(final String txID, final ShardDataTreeCohort actual) {
-                    if(mockCohort.get() == null) {
-                        mockCohort.set(createDelegatingMockCohort("cohort", actual));
-                    }
-
-                    return mockCohort.get();
+            final ShardCommitCoordinator.CohortDecorator cohortDecorator = (txID, actual) -> {
+                if(mockCohort.get() == null) {
+                    mockCohort.set(createDelegatingMockCohort("cohort", actual));
                 }
+
+                return mockCohort.get();
             };
 
             shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator);
@@ -1231,7 +1213,7 @@ public class ShardTest extends AbstractShardTest {
 
                 final String transactionID = "tx1";
                 final MutableCompositeModification modification = new MutableCompositeModification();
-                modification.addModification(new DeleteModification(YangInstanceIdentifier.builder().build()));
+                modification.addModification(new DeleteModification(YangInstanceIdentifier.EMPTY));
                 final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1");
                 doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit();
                 doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).preCommit();
@@ -1293,7 +1275,7 @@ public class ShardTest extends AbstractShardTest {
             final ShardDataTreeCohort cohort1 = mock(ShardDataTreeCohort.class, "cohort1");
             doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort1).canCommit();
             doReturn(Futures.immediateFuture(null)).when(cohort1).preCommit();
-            doReturn(Futures.immediateFailedFuture(new IllegalStateException("mock"))).when(cohort1).commit();
+            doReturn(Futures.immediateFailedFuture(new RuntimeException("mock"))).when(cohort1).commit();
             doReturn(mockCandidate("cohort1-candidate")).when(cohort1).getCandidate();
 
             final String transactionID2 = "tx2";
@@ -1552,6 +1534,7 @@ public class ShardTest extends AbstractShardTest {
             final DataTreeCandidateNode candidateRoot = mock(DataTreeCandidateNode.class);
             doReturn(ModificationType.UNMODIFIED).when(candidateRoot).getModificationType();
             doReturn(candidateRoot).when(candidate).getRootNode();
+            doReturn(YangInstanceIdentifier.EMPTY).when(candidate).getRootPath();
             doReturn(candidate).when(cohort).getCandidate();
 
             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID2, modification, true), getRef());
@@ -1597,6 +1580,7 @@ public class ShardTest extends AbstractShardTest {
             final DataTreeCandidateNode candidateRoot = mock(DataTreeCandidateNode.class);
             doReturn(ModificationType.UNMODIFIED).when(candidateRoot).getModificationType();
             doReturn(candidateRoot).when(candidate).getRootNode();
+            doReturn(YangInstanceIdentifier.EMPTY).when(candidate).getRootPath();
             doReturn(candidate).when(cohort).getCandidate();
 
             shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID2, modification, true), getRef());
@@ -1624,23 +1608,20 @@ public class ShardTest extends AbstractShardTest {
 
             final String transactionID = "tx1";
             final Function<ShardDataTreeCohort, ListenableFuture<Void>> preCommit =
-                          new Function<ShardDataTreeCohort, ListenableFuture<Void>>() {
-                @Override
-                public ListenableFuture<Void> apply(final ShardDataTreeCohort cohort) {
-                    final ListenableFuture<Void> preCommitFuture = cohort.preCommit();
-
-                    // Simulate an AbortTransaction message occurring during replication, after
-                    // persisting and before finishing the commit to the in-memory store.
-                    // We have no followers so due to optimizations in the RaftActor, it does not
-                    // attempt replication and thus we can't send an AbortTransaction message b/c
-                    // it would be processed too late after CommitTransaction completes. So we'll
-                    // simulate an AbortTransaction message occurring during replication by calling
-                    // the shard directly.
-                    //
-                    shard.underlyingActor().doAbortTransaction(transactionID, null);
-
-                    return preCommitFuture;
-                }
+                          cohort -> {
+                final ListenableFuture<Void> preCommitFuture = cohort.preCommit();
+
+                // Simulate an AbortTransaction message occurring during replication, after
+                // persisting and before finishing the commit to the in-memory store.
+                // We have no followers so due to optimizations in the RaftActor, it does not
+                // attempt replication and thus we can't send an AbortTransaction message b/c
+                // it would be processed too late after CommitTransaction completes. So we'll
+                // simulate an AbortTransaction message occurring during replication by calling
+                // the shard directly.
+                //
+                shard.underlyingActor().doAbortTransaction(transactionID, null);
+
+                return preCommitFuture;
             };
 
             final MutableCompositeModification modification = new MutableCompositeModification();
@@ -2006,20 +1987,15 @@ public class ShardTest extends AbstractShardTest {
         new ShardTestKit(getSystem()) {{
             final AtomicReference<CountDownLatch> cleaupCheckLatch = new AtomicReference<>();
             @SuppressWarnings("serial")
-            final Creator<Shard> creator = new Creator<Shard>() {
+            final Creator<Shard> creator = () -> new Shard(newShardBuilder()) {
                 @Override
-                public Shard create() throws Exception {
-                    return new Shard(newShardBuilder()) {
-                        @Override
-                        public void handleCommand(final Object message) {
-                            super.handleCommand(message);
-                            if(message.equals(TX_COMMIT_TIMEOUT_CHECK_MESSAGE)) {
-                                if(cleaupCheckLatch.get() != null) {
-                                    cleaupCheckLatch.get().countDown();
-                                }
-                            }
+                public void handleCommand(final Object message) {
+                    super.handleCommand(message);
+                    if(TX_COMMIT_TIMEOUT_CHECK_MESSAGE.equals(message)) {
+                        if(cleaupCheckLatch.get() != null) {
+                            cleaupCheckLatch.get().countDown();
                         }
-                    };
+                    }
                 }
             };
 
@@ -2110,7 +2086,8 @@ public class ShardTest extends AbstractShardTest {
                 public void handleCommand(final Object message) {
                     super.handleCommand(message);
 
-                    if (message instanceof SaveSnapshotSuccess || message.equals("commit_snapshot")) {
+                    // XXX:  commit_snapshot equality check references RaftActorSnapshotMessageSupport.COMMIT_SNAPSHOT
+                    if (message instanceof SaveSnapshotSuccess || "commit_snapshot".equals(message.toString())) {
                         latch.get().countDown();
                     }
                 }
@@ -2121,12 +2098,7 @@ public class ShardTest extends AbstractShardTest {
                 }
             }
 
-            final Creator<Shard> creator = new Creator<Shard>() {
-                @Override
-                public Shard create() throws Exception {
-                    return new TestShard(newShardBuilder());
-                }
-            };
+            final Creator<Shard> creator = () -> new TestShard(newShardBuilder());
 
             final TestActorRef<Shard> shard = actorFactory.createTestActor(
                     Props.create(new DelegatingShardCreator(creator)), shardActorName);
@@ -2134,7 +2106,7 @@ public class ShardTest extends AbstractShardTest {
             waitUntilLeader(shard);
             writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
 
-            final NormalizedNode<?,?> expectedRoot = readStore(shard, YangInstanceIdentifier.builder().build());
+            final NormalizedNode<?,?> expectedRoot = readStore(shard, YangInstanceIdentifier.EMPTY);
 
             // Trigger creation of a snapshot by ensuring
             final RaftActorContext raftActorContext = ((TestShard) shard.underlyingActor()).getRaftActorContext();
@@ -2181,16 +2153,16 @@ public class ShardTest extends AbstractShardTest {
         commitTransaction(store, putTransaction);
 
 
-        final NormalizedNode<?, ?> expected = readStore(store, YangInstanceIdentifier.builder().build());
+        final NormalizedNode<?, ?> expected = readStore(store, YangInstanceIdentifier.EMPTY);
 
         final DataTreeModification writeTransaction = store.takeSnapshot().newModification();
 
-        writeTransaction.delete(YangInstanceIdentifier.builder().build());
-        writeTransaction.write(YangInstanceIdentifier.builder().build(), expected);
+        writeTransaction.delete(YangInstanceIdentifier.EMPTY);
+        writeTransaction.write(YangInstanceIdentifier.EMPTY, expected);
 
         commitTransaction(store, writeTransaction);
 
-        final NormalizedNode<?, ?> actual = readStore(store, YangInstanceIdentifier.builder().build());
+        final NormalizedNode<?, ?> actual = readStore(store, YangInstanceIdentifier.EMPTY);
 
         assertEquals(expected, actual);
     }
@@ -2289,11 +2261,11 @@ public class ShardTest extends AbstractShardTest {
                 newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
                 "testFollowerInitialSyncStatus");
 
-        shard.underlyingActor().onReceiveCommand(new FollowerInitialSyncUpStatus(false, "member-1-shard-inventory-operational"));
+        shard.underlyingActor().handleNonRaftCommand(new FollowerInitialSyncUpStatus(false, "member-1-shard-inventory-operational"));
 
         assertEquals(false, shard.underlyingActor().getShardMBean().getFollowerInitialSyncStatus());
 
-        shard.underlyingActor().onReceiveCommand(new FollowerInitialSyncUpStatus(true, "member-1-shard-inventory-operational"));
+        shard.underlyingActor().handleNonRaftCommand(new FollowerInitialSyncUpStatus(true, "member-1-shard-inventory-operational"));
 
         assertEquals(true, shard.underlyingActor().getShardMBean().getFollowerInitialSyncStatus());
     }
@@ -2335,11 +2307,11 @@ public class ShardTest extends AbstractShardTest {
     public void testClusteredDataChangeListenerRegistration() throws Exception {
         new ShardTestKit(getSystem()) {{
             String testName = "testClusteredDataChangeListenerRegistration";
-            final ShardIdentifier followerShardID = ShardIdentifier.builder().memberName(
-                    actorFactory.generateActorId(testName + "-follower")).shardName("inventory").type("config").build();
+            final ShardIdentifier followerShardID = ShardIdentifier.create("inventory",
+                    MemberName.forName(actorFactory.generateActorId(testName + "-follower")), "config");
 
-            final ShardIdentifier leaderShardID = ShardIdentifier.builder().memberName(
-                    actorFactory.generateActorId(testName + "-leader")).shardName("inventory").type("config").build();
+            final ShardIdentifier leaderShardID = ShardIdentifier.create("inventory",
+                    MemberName.forName(actorFactory.generateActorId(testName + "-leader")), "config");
 
             final TestActorRef<Shard> followerShard = actorFactory.createTestActor(
                     Shard.builder().id(followerShardID).
@@ -2411,11 +2383,11 @@ public class ShardTest extends AbstractShardTest {
     public void testClusteredDataTreeChangeListenerRegistration() throws Exception {
         new ShardTestKit(getSystem()) {{
             String testName = "testClusteredDataTreeChangeListenerRegistration";
-            final ShardIdentifier followerShardID = ShardIdentifier.builder().memberName(
-                    actorFactory.generateActorId(testName + "-follower")).shardName("inventory").type("config").build();
+            final ShardIdentifier followerShardID = ShardIdentifier.create("inventory",
+                    MemberName.forName(actorFactory.generateActorId(testName + "-follower")), "config");
 
-            final ShardIdentifier leaderShardID = ShardIdentifier.builder().memberName(
-                    actorFactory.generateActorId(testName + "-leader")).shardName("inventory").type("config").build();
+            final ShardIdentifier leaderShardID = ShardIdentifier.create("inventory",
+                    MemberName.forName(actorFactory.generateActorId(testName + "-leader")), "config");
 
             final TestActorRef<Shard> followerShard = actorFactory.createTestActor(
                     Shard.builder().id(followerShardID).