Bug 7521: Convert Snapshot to store a State instance
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / ShardTest.java
index b1e31380a4af6b0eae41e0471e27a60c1d36d722..2edbff2380b5ba96b7622a97a5a453cb30976898 100644 (file)
@@ -63,7 +63,6 @@ import org.opendaylight.controller.cluster.datastore.messages.CommitTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
-import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
 import org.opendaylight.controller.cluster.datastore.messages.ReadData;
 import org.opendaylight.controller.cluster.datastore.messages.ReadDataReply;
@@ -76,19 +75,16 @@ import org.opendaylight.controller.cluster.datastore.messages.RegisterDataTreeCh
 import org.opendaylight.controller.cluster.datastore.messages.ShardLeaderStateChanged;
 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
 import org.opendaylight.controller.cluster.datastore.modification.MergeModification;
-import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
 import org.opendaylight.controller.cluster.datastore.persisted.MetadataShardDataTreeSnapshot;
-import org.opendaylight.controller.cluster.datastore.persisted.ShardDataTreeSnapshot;
+import org.opendaylight.controller.cluster.datastore.persisted.ShardSnapshotState;
 import org.opendaylight.controller.cluster.datastore.utils.MockDataChangeListener;
 import org.opendaylight.controller.cluster.datastore.utils.MockDataTreeChangeListener;
 import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListener;
 import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListenerReply;
 import org.opendaylight.controller.cluster.raft.RaftActorContext;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
-import org.opendaylight.controller.cluster.raft.Snapshot;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
-import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus;
 import org.opendaylight.controller.cluster.raft.base.messages.TimeoutNow;
@@ -100,6 +96,7 @@ import org.opendaylight.controller.cluster.raft.messages.RequestVote;
 import org.opendaylight.controller.cluster.raft.messages.ServerRemoved;
 import org.opendaylight.controller.cluster.raft.persisted.ApplyJournalEntries;
 import org.opendaylight.controller.cluster.raft.persisted.SimpleReplicatedLogEntry;
+import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
 import org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy;
 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
@@ -445,8 +442,9 @@ public class ShardTest extends AbstractShardTest {
         final YangInstanceIdentifier root = YangInstanceIdentifier.EMPTY;
         final NormalizedNode<?,?> expected = readStore(store, root);
 
-        final Snapshot snapshot = Snapshot.create(new MetadataShardDataTreeSnapshot(expected).serialize(),
-                Collections.<ReplicatedLogEntry>emptyList(), 1, 2, 3, 4);
+        final Snapshot snapshot = Snapshot.create(
+                new ShardSnapshotState(new MetadataShardDataTreeSnapshot(expected)),
+                Collections.<ReplicatedLogEntry>emptyList(), 1, 2, 3, 4, -1, null, null);
 
         shard.tell(new ApplySnapshot(snapshot), ActorRef.noSender());
 
@@ -474,13 +472,6 @@ public class ShardTest extends AbstractShardTest {
 
         final DataTree store = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL);
         store.setSchemaContext(SCHEMA_CONTEXT);
-        writeToStore(store, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
-
-        final NormalizedNode<?, ?> root = readStore(store, YangInstanceIdentifier.EMPTY);
-        final Snapshot snapshot = Snapshot.create(new MetadataShardDataTreeSnapshot(root).serialize(),
-                Collections.<ReplicatedLogEntry>emptyList(), 1, 2, 3, 4);
-
-        shard.tell(new ApplySnapshot(snapshot), ActorRef.noSender());
 
         final DataTreeModification writeMod = store.takeSnapshot().newModification();
         final ContainerNode node = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
@@ -488,10 +479,7 @@ public class ShardTest extends AbstractShardTest {
         writeMod.ready();
 
         final TransactionIdentifier tx = nextTransactionId();
-        final ApplyState applyState = new ApplyState(null, tx,
-                new SimpleReplicatedLogEntry(1, 2, payloadForModification(store, writeMod, tx)));
-
-        shard.tell(applyState, shard);
+        shard.underlyingActor().applyState(null, null, payloadForModification(store, writeMod, tx));
 
         final Stopwatch sw = Stopwatch.createStarted();
         while (sw.elapsed(TimeUnit.SECONDS) <= 5) {
@@ -1154,67 +1142,6 @@ public class ShardTest extends AbstractShardTest {
         };
     }
 
-    @Test
-    public void testReadWriteCommitWhenTransactionHasNoModifications() {
-        testCommitWhenTransactionHasNoModifications(true);
-    }
-
-    @Test
-    public void testWriteOnlyCommitWhenTransactionHasNoModifications() {
-        testCommitWhenTransactionHasNoModifications(false);
-    }
-
-    private void testCommitWhenTransactionHasNoModifications(final boolean readWrite) {
-        // Note that persistence is enabled which would normally result in the
-        // entry getting written to the journal
-        // but here that need not happen
-        new ShardTestKit(getSystem()) {
-            {
-                final TestActorRef<Shard> shard = actorFactory.createTestActor(
-                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                        "testCommitWhenTransactionHasNoModifications-" + readWrite);
-
-                waitUntilLeader(shard);
-
-                final TransactionIdentifier transactionID = nextTransactionId();
-
-                final FiniteDuration duration = duration("5 seconds");
-
-                if (readWrite) {
-                    final ReadWriteShardDataTreeTransaction rwTx = shard.underlyingActor().getDataStore()
-                            .newReadWriteTransaction(transactionID);
-                    shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION, rwTx, false), getRef());
-                } else {
-                    shard.tell(prepareBatchedModifications(transactionID, new MutableCompositeModification()),
-                            getRef());
-                }
-
-                expectMsgClass(duration, ReadyTransactionReply.class);
-
-                // Send the CanCommitTransaction message.
-
-                shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
-                final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply
-                        .fromSerializable(expectMsgClass(duration, CanCommitTransactionReply.class));
-                assertEquals("Can commit", true, canCommitReply.getCanCommit());
-
-                shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
-                expectMsgClass(duration, CommitTransactionReply.class);
-
-                shard.tell(Shard.GET_SHARD_MBEAN_MESSAGE, getRef());
-                final ShardStats shardStats = expectMsgClass(duration, ShardStats.class);
-
-                // Use MBean for verification
-                // Committed transaction count should increase as usual
-                assertEquals(1, shardStats.getCommittedTransactionsCount());
-
-                // Commit index should not advance because this does not go into
-                // the journal
-                assertEquals(-1, shardStats.getCommitIndex());
-            }
-        };
-    }
-
     @Test
     public void testReadWriteCommitWhenTransactionHasModifications() throws Exception {
         testCommitWhenTransactionHasModifications(true);
@@ -2049,8 +1976,8 @@ public class ShardTest extends AbstractShardTest {
 
             private void verifySnapshot(final Snapshot snapshot, final NormalizedNode<?, ?> expectedRoot)
                     throws IOException {
-                final NormalizedNode<?, ?> actual = ShardDataTreeSnapshot.deserialize(snapshot.getState()).getRootNode()
-                        .get();
+                final NormalizedNode<?, ?> actual = ((ShardSnapshotState)snapshot.getState()).getSnapshot()
+                        .getRootNode().get();
                 assertEquals("Root node", expectedRoot, actual);
             }
         };