Merge changes I114cbac1,I45c2e7cd
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / ShardTest.java
index 94b9698abf3a06f41527a83e5cec2461b2658266..e3b82df1743e75c433cec193d54a2cbfbd696319 100644 (file)
@@ -5,26 +5,23 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.inOrder;
 import static org.mockito.Mockito.mock;
 import static org.opendaylight.controller.cluster.datastore.DataStoreVersions.CURRENT_VERSION;
 import akka.actor.ActorRef;
+import akka.actor.ActorSelection;
 import akka.actor.PoisonPill;
 import akka.actor.Props;
 import akka.dispatch.Dispatchers;
 import akka.dispatch.OnComplete;
 import akka.japi.Creator;
-import akka.japi.Procedure;
 import akka.pattern.Patterns;
-import akka.persistence.SnapshotSelectionCriteria;
+import akka.persistence.SaveSnapshotSuccess;
 import akka.testkit.TestActorRef;
 import akka.util.Timeout;
 import com.google.common.base.Function;
 import com.google.common.base.Optional;
-import com.google.common.util.concurrent.CheckedFuture;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.MoreExecutors;
@@ -32,24 +29,23 @@ import com.google.common.util.concurrent.Uninterruptibles;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
 import org.mockito.InOrder;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
 import org.opendaylight.controller.cluster.DataPersistenceProvider;
-import org.opendaylight.controller.cluster.datastore.DatastoreContext.Builder;
+import org.opendaylight.controller.cluster.DelegatingPersistentDataProvider;
 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.AbortTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.AbortTransactionReply;
+import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
+import org.opendaylight.controller.cluster.datastore.messages.BatchedModificationsReply;
 import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransactionReply;
 import org.opendaylight.controller.cluster.datastore.messages.CommitTransaction;
@@ -57,47 +53,49 @@ import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionR
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
 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;
 import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListener;
 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListenerReply;
 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
+import org.opendaylight.controller.cluster.datastore.modification.DeleteModification;
 import org.opendaylight.controller.cluster.datastore.modification.MergeModification;
 import org.opendaylight.controller.cluster.datastore.modification.Modification;
 import org.opendaylight.controller.cluster.datastore.modification.ModificationPayload;
 import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
-import org.opendaylight.controller.cluster.datastore.node.NormalizedNodeToNodeCodec;
-import org.opendaylight.controller.cluster.datastore.utils.InMemoryJournal;
-import org.opendaylight.controller.cluster.datastore.utils.InMemorySnapshotStore;
+import org.opendaylight.controller.cluster.datastore.utils.MessageCollectorActor;
 import org.opendaylight.controller.cluster.datastore.utils.MockDataChangeListener;
 import org.opendaylight.controller.cluster.datastore.utils.SerializationUtils;
+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.ReplicatedLogImplEntry;
 import org.opendaylight.controller.cluster.raft.Snapshot;
-import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
 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.CaptureSnapshot;
 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.protobuff.client.messages.CompositeModificationByteStringPayload;
-import org.opendaylight.controller.cluster.raft.protobuff.client.messages.CompositeModificationPayload;
+import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
+import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
 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.AsyncDataChangeEvent;
-import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener;
 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore;
 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStoreFactory;
-import org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages;
+import org.opendaylight.controller.protobuff.messages.cohort3pc.ThreePhaseCommitCohortMessages;
 import org.opendaylight.controller.protobuff.messages.transaction.ShardTransactionMessages.CreateTransactionReply;
-import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.PathArgument;
+import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
 import org.opendaylight.yangtools.yang.data.api.schema.DataContainerChild;
 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
@@ -107,40 +105,7 @@ import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
-public class ShardTest extends AbstractActorTest {
-
-    private static final SchemaContext SCHEMA_CONTEXT = TestModel.createTestContext();
-
-    private static final AtomicInteger NEXT_SHARD_NUM = new AtomicInteger();
-
-    private final ShardIdentifier shardID = ShardIdentifier.builder().memberName("member-1")
-            .shardName("inventory").type("config" + NEXT_SHARD_NUM.getAndIncrement()).build();
-
-    private final Builder dataStoreContextBuilder = DatastoreContext.newBuilder().
-            shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).
-            shardHeartbeatIntervalInMillis(100);
-
-    @Before
-    public void setUp() {
-        Builder newBuilder = DatastoreContext.newBuilder();
-        InMemorySnapshotStore.clear();
-        InMemoryJournal.clear();
-    }
-
-    @After
-    public void tearDown() {
-        InMemorySnapshotStore.clear();
-        InMemoryJournal.clear();
-    }
-
-    private DatastoreContext newDatastoreContext() {
-        return dataStoreContextBuilder.build();
-    }
-
-    private Props newShardProps() {
-        return Shard.props(shardID, Collections.<ShardIdentifier,String>emptyMap(),
-                newDatastoreContext(), SCHEMA_CONTEXT);
-    }
+public class ShardTest extends AbstractShardTest {
 
     @Test
     public void testRegisterChangeListener() throws Exception {
@@ -157,7 +122,7 @@ public class ShardTest extends AbstractActorTest {
                     "testRegisterChangeListener-DataChangeListener");
 
             shard.tell(new RegisterChangeListener(TestModel.TEST_PATH,
-                    dclActor.path(), AsyncDataBroker.DataChangeScope.BASE), getRef());
+                    dclActor, AsyncDataBroker.DataChangeScope.BASE), getRef());
 
             RegisterChangeListenerReply reply = expectMsgClass(duration("3 seconds"),
                     RegisterChangeListenerReply.class);
@@ -195,8 +160,12 @@ public class ShardTest extends AbstractActorTest {
 
                 @Override
                 public Shard create() throws Exception {
-                    return new Shard(shardID, Collections.<ShardIdentifier,String>emptyMap(),
-                            newDatastoreContext(), SCHEMA_CONTEXT) {
+                    // Use a non persistent provider because this test actually invokes persist on the journal
+                    // this will cause all other messages to not be queued properly after that.
+                    // The basic issue is that you cannot use TestActorRef with a persistent actor (at least when
+                    // it does do a persist)
+                    return new Shard(shardID, Collections.<String,String>emptyMap(),
+                            dataStoreContextBuilder.persistent(false).build(), SCHEMA_CONTEXT) {
                         @Override
                         public void onReceiveCommand(final Object message) throws Exception {
                             if(message instanceof ElectionTimeout && firstElectionTimeout) {
@@ -245,7 +214,7 @@ public class ShardTest extends AbstractActorTest {
                     onFirstElectionTimeout.await(5, TimeUnit.SECONDS));
 
             // Now send the RegisterChangeListener and wait for the reply.
-            shard.tell(new RegisterChangeListener(path, dclActor.path(),
+            shard.tell(new RegisterChangeListener(path, dclActor,
                     AsyncDataBroker.DataChangeScope.SUBTREE), getRef());
 
             RegisterChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
@@ -323,7 +292,7 @@ public class ShardTest extends AbstractActorTest {
             final CountDownLatch recoveryComplete = new CountDownLatch(1);
             class TestShard extends Shard {
                 TestShard() {
-                    super(shardID, Collections.<ShardIdentifier, String>singletonMap(shardID, null),
+                    super(shardID, Collections.<String, String>singletonMap(shardID.toString(), null),
                             newDatastoreContext(), SCHEMA_CONTEXT);
                 }
 
@@ -354,7 +323,7 @@ public class ShardTest extends AbstractActorTest {
                     Uninterruptibles.awaitUninterruptibly(recoveryComplete, 5, TimeUnit.SECONDS));
 
             String address = "akka://foobar";
-            shard.underlyingActor().onReceiveCommand(new PeerAddressResolved(shardID, address));
+            shard.underlyingActor().onReceiveCommand(new PeerAddressResolved(shardID.toString(), address));
 
             assertEquals("getPeerAddresses", address,
                     ((TestShard)shard.underlyingActor()).getPeerAddresses().get(shardID.toString()));
@@ -389,36 +358,6 @@ public class ShardTest extends AbstractActorTest {
         shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
     }
 
-    @Test
-    public void testApplyHelium2VersionSnapshot() throws Exception {
-        TestActorRef<Shard> shard = TestActorRef.create(getSystem(), newShardProps(),
-                "testApplySnapshot");
-
-        NormalizedNodeToNodeCodec codec = new NormalizedNodeToNodeCodec(SCHEMA_CONTEXT);
-
-        InMemoryDOMDataStore store = new InMemoryDOMDataStore("OPER", MoreExecutors.sameThreadExecutor());
-        store.onGlobalContextUpdated(SCHEMA_CONTEXT);
-
-        writeToStore(store, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
-
-        YangInstanceIdentifier root = YangInstanceIdentifier.builder().build();
-        NormalizedNode<?,?> expected = readStore(store, root);
-
-        NormalizedNodeMessages.Container encode = codec.encode(expected);
-
-        ApplySnapshot applySnapshot = new ApplySnapshot(Snapshot.create(
-                encode.getNormalizedNode().toByteString().toByteArray(),
-                Collections.<ReplicatedLogEntry>emptyList(), 1, 2, 3, 4));
-
-        shard.underlyingActor().onReceiveCommand(applySnapshot);
-
-        NormalizedNode<?,?> actual = readStore(shard, root);
-
-        assertEquals("Root node", expected, actual);
-
-        shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-    }
-
     @Test
     public void testApplyState() throws Exception {
 
@@ -437,24 +376,6 @@ public class ShardTest extends AbstractActorTest {
         shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
     }
 
-    @Test
-    public void testApplyStateLegacy() throws Exception {
-
-        TestActorRef<Shard> shard = TestActorRef.create(getSystem(), newShardProps(), "testApplyStateLegacy");
-
-        NormalizedNode<?, ?> node = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
-
-        ApplyState applyState = new ApplyState(null, "test", new ReplicatedLogImplEntry(1, 2,
-                newLegacyByteStringPayload(new WriteModification(TestModel.TEST_PATH, node))));
-
-        shard.underlyingActor().onReceiveCommand(applyState);
-
-        NormalizedNode<?,?> actual = readStore(shard, TestModel.TEST_PATH);
-        assertEquals("Applied state", node, actual);
-
-        shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-    }
-
     @Test
     public void testRecovery() throws Exception {
 
@@ -473,7 +394,7 @@ public class ShardTest extends AbstractActorTest {
 
         // Set up the InMemoryJournal.
 
-        InMemoryJournal.addEntry(shardID.toString(), 0, new ReplicatedLogImplEntry(0, 1, newLegacyPayload(
+        InMemoryJournal.addEntry(shardID.toString(), 0, new ReplicatedLogImplEntry(0, 1, newModificationPayload(
                   new WriteModification(TestModel.OUTER_LIST_PATH,
                           ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build()))));
 
@@ -492,146 +413,11 @@ public class ShardTest extends AbstractActorTest {
         }
 
         InMemoryJournal.addEntry(shardID.toString(), nListEntries + 1,
-                new ApplyLogEntries(nListEntries));
+                new ApplyJournalEntries(nListEntries));
 
         testRecovery(listEntryKeys);
     }
 
-    @Test
-    public void testHelium2VersionRecovery() throws Exception {
-
-        // Set up the InMemorySnapshotStore.
-
-        InMemoryDOMDataStore testStore = InMemoryDOMDataStoreFactory.create("Test", null, null);
-        testStore.onGlobalContextUpdated(SCHEMA_CONTEXT);
-
-        writeToStore(testStore, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
-
-        NormalizedNode<?, ?> root = readStore(testStore, YangInstanceIdentifier.builder().build());
-
-        InMemorySnapshotStore.addSnapshot(shardID.toString(), Snapshot.create(
-                new NormalizedNodeToNodeCodec(SCHEMA_CONTEXT).encode(root).
-                                getNormalizedNode().toByteString().toByteArray(),
-                                Collections.<ReplicatedLogEntry>emptyList(), 0, 1, -1, -1));
-
-        // Set up the InMemoryJournal.
-
-        InMemoryJournal.addEntry(shardID.toString(), 0, new ReplicatedLogImplEntry(0, 1, newLegacyPayload(
-                  new WriteModification(TestModel.OUTER_LIST_PATH,
-                          ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build()))));
-
-        int nListEntries = 16;
-        Set<Integer> listEntryKeys = new HashSet<>();
-        int i = 1;
-
-        // Add some CompositeModificationPayload entries
-        for(; i <= 8; i++) {
-            listEntryKeys.add(Integer.valueOf(i));
-            YangInstanceIdentifier path = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
-                    .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i).build();
-            Modification mod = new MergeModification(path,
-                    ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i));
-            InMemoryJournal.addEntry(shardID.toString(), i, new ReplicatedLogImplEntry(i, 1,
-                    newLegacyPayload(mod)));
-        }
-
-        // Add some CompositeModificationByteStringPayload entries
-        for(; i <= nListEntries; i++) {
-            listEntryKeys.add(Integer.valueOf(i));
-            YangInstanceIdentifier path = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
-                    .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i).build();
-            Modification mod = new MergeModification(path,
-                    ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i));
-            InMemoryJournal.addEntry(shardID.toString(), i, new ReplicatedLogImplEntry(i, 1,
-                    newLegacyByteStringPayload(mod)));
-        }
-
-        InMemoryJournal.addEntry(shardID.toString(), nListEntries + 1, new ApplyLogEntries(nListEntries));
-
-        testRecovery(listEntryKeys);
-    }
-
-    private void testRecovery(Set<Integer> listEntryKeys) throws Exception {
-        // Create the actor and wait for recovery complete.
-
-        int nListEntries = listEntryKeys.size();
-
-        final CountDownLatch recoveryComplete = new CountDownLatch(1);
-
-        @SuppressWarnings("serial")
-        Creator<Shard> creator = new Creator<Shard>() {
-            @Override
-            public Shard create() throws Exception {
-                return new Shard(shardID, Collections.<ShardIdentifier,String>emptyMap(),
-                        newDatastoreContext(), SCHEMA_CONTEXT) {
-                    @Override
-                    protected void onRecoveryComplete() {
-                        try {
-                            super.onRecoveryComplete();
-                        } finally {
-                            recoveryComplete.countDown();
-                        }
-                    }
-                };
-            }
-        };
-
-        TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                Props.create(new DelegatingShardCreator(creator)), "testRecovery");
-
-        assertEquals("Recovery complete", true, recoveryComplete.await(5, TimeUnit.SECONDS));
-
-        // Verify data in the data store.
-
-        NormalizedNode<?, ?> outerList = readStore(shard, TestModel.OUTER_LIST_PATH);
-        assertNotNull(TestModel.OUTER_LIST_QNAME.getLocalName() + " not found", outerList);
-        assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " value is not Iterable",
-                outerList.getValue() instanceof Iterable);
-        for(Object entry: (Iterable<?>) outerList.getValue()) {
-            assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " entry is not MapEntryNode",
-                    entry instanceof MapEntryNode);
-            MapEntryNode mapEntry = (MapEntryNode)entry;
-            Optional<DataContainerChild<? extends PathArgument, ?>> idLeaf =
-                    mapEntry.getChild(new YangInstanceIdentifier.NodeIdentifier(TestModel.ID_QNAME));
-            assertTrue("Missing leaf " + TestModel.ID_QNAME.getLocalName(), idLeaf.isPresent());
-            Object value = idLeaf.get().getValue();
-            assertTrue("Unexpected value for leaf "+ TestModel.ID_QNAME.getLocalName() + ": " + value,
-                    listEntryKeys.remove(value));
-        }
-
-        if(!listEntryKeys.isEmpty()) {
-            fail("Missing " + TestModel.OUTER_LIST_QNAME.getLocalName() + " entries with keys: " +
-                    listEntryKeys);
-        }
-
-        assertEquals("Last log index", nListEntries,
-                shard.underlyingActor().getShardMBean().getLastLogIndex());
-        assertEquals("Commit index", nListEntries,
-                shard.underlyingActor().getShardMBean().getCommitIndex());
-        assertEquals("Last applied", nListEntries,
-                shard.underlyingActor().getShardMBean().getLastApplied());
-
-        shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-    }
-
-    private CompositeModificationPayload newLegacyPayload(final Modification... mods) {
-        MutableCompositeModification compMod = new MutableCompositeModification();
-        for(Modification mod: mods) {
-            compMod.addModification(mod);
-        }
-
-        return new CompositeModificationPayload(compMod.toSerializable());
-    }
-
-    private CompositeModificationByteStringPayload newLegacyByteStringPayload(final Modification... mods) {
-        MutableCompositeModification compMod = new MutableCompositeModification();
-        for(Modification mod: mods) {
-            compMod.addModification(mod);
-        }
-
-        return new CompositeModificationByteStringPayload(compMod.toSerializable());
-    }
-
     private ModificationPayload newModificationPayload(final Modification... mods) throws IOException {
         MutableCompositeModification compMod = new MutableCompositeModification();
         for(Modification mod: mods) {
@@ -641,59 +427,6 @@ public class ShardTest extends AbstractActorTest {
         return new ModificationPayload(compMod);
     }
 
-    private DOMStoreThreePhaseCommitCohort setupMockWriteTransaction(final String cohortName,
-            final InMemoryDOMDataStore dataStore, final YangInstanceIdentifier path, final NormalizedNode<?, ?> data,
-            final MutableCompositeModification modification) {
-        return setupMockWriteTransaction(cohortName, dataStore, path, data, modification, null);
-    }
-
-    private DOMStoreThreePhaseCommitCohort setupMockWriteTransaction(final String cohortName,
-            final InMemoryDOMDataStore dataStore, final YangInstanceIdentifier path, final NormalizedNode<?, ?> data,
-            final MutableCompositeModification modification,
-            final Function<DOMStoreThreePhaseCommitCohort,ListenableFuture<Void>> preCommit) {
-
-        DOMStoreWriteTransaction tx = dataStore.newWriteOnlyTransaction();
-        tx.write(path, data);
-        final DOMStoreThreePhaseCommitCohort realCohort = tx.ready();
-        DOMStoreThreePhaseCommitCohort cohort = mock(DOMStoreThreePhaseCommitCohort.class, cohortName);
-
-        doAnswer(new Answer<ListenableFuture<Boolean>>() {
-            @Override
-            public ListenableFuture<Boolean> answer(final InvocationOnMock invocation) {
-                return realCohort.canCommit();
-            }
-        }).when(cohort).canCommit();
-
-        doAnswer(new Answer<ListenableFuture<Void>>() {
-            @Override
-            public ListenableFuture<Void> answer(final InvocationOnMock invocation) throws Throwable {
-                if(preCommit != null) {
-                    return preCommit.apply(realCohort);
-                } else {
-                    return realCohort.preCommit();
-                }
-            }
-        }).when(cohort).preCommit();
-
-        doAnswer(new Answer<ListenableFuture<Void>>() {
-            @Override
-            public ListenableFuture<Void> answer(final InvocationOnMock invocation) throws Throwable {
-                return realCohort.commit();
-            }
-        }).when(cohort).commit();
-
-        doAnswer(new Answer<ListenableFuture<Void>>() {
-            @Override
-            public ListenableFuture<Void> answer(final InvocationOnMock invocation) throws Throwable {
-                return realCohort.abort();
-            }
-        }).when(cohort).abort();
-
-        modification.addModification(new WriteModification(path, data));
-
-        return cohort;
-    }
-
     @SuppressWarnings({ "unchecked" })
     @Test
     public void testConcurrentThreePhaseCommits() throws Throwable {
@@ -704,7 +437,7 @@ public class ShardTest extends AbstractActorTest {
 
             waitUntilLeader(shard);
 
-            // Setup 3 simulated transactions with mock cohorts backed by real cohorts.
+         // Setup 3 simulated transactions with mock cohorts backed by real cohorts.
 
             InMemoryDOMDataStore dataStore = shard.underlyingActor().getDataStore();
 
@@ -738,7 +471,7 @@ public class ShardTest extends AbstractActorTest {
             shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION,
                     cohort1, modification1, true), getRef());
             ReadyTransactionReply readyReply = ReadyTransactionReply.fromSerializable(
-                    expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS));
+                    expectMsgClass(duration, ReadyTransactionReply.class));
             assertEquals("Cohort path", shard.path().toString(), readyReply.getCohortPath());
 
             // Send the CanCommitTransaction message for the first Tx.
@@ -752,11 +485,11 @@ public class ShardTest extends AbstractActorTest {
 
             shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION,
                     cohort2, modification2, true), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
+            expectMsgClass(duration, ReadyTransactionReply.class);
 
             shard.tell(new ForwardedReadyTransaction(transactionID3, CURRENT_VERSION,
                     cohort3, modification3, true), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
+            expectMsgClass(duration, ReadyTransactionReply.class);
 
             // Send the CanCommitTransaction message for the next 2 Tx's. These should get queued and
             // processed after the first Tx completes.
@@ -875,22 +608,196 @@ public class ShardTest extends AbstractActorTest {
             assertTrue("Missing leaf " + TestModel.ID_QNAME.getLocalName(), idLeaf.isPresent());
             assertEquals(TestModel.ID_QNAME.getLocalName() + " value", 1, idLeaf.get().getValue());
 
-            verifyLastLogIndex(shard, 2);
+            verifyLastApplied(shard, 2);
 
             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
         }};
     }
 
-    private void verifyLastLogIndex(TestActorRef<Shard> shard, long expectedValue) {
-        for(int i = 0; i < 20 * 5; i++) {
-            long lastLogIndex = shard.underlyingActor().getShardMBean().getLastLogIndex();
-            if(lastLogIndex == expectedValue) {
-                break;
-            }
-            Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
-        }
+    private BatchedModifications newBatchedModifications(String transactionID, YangInstanceIdentifier path,
+            NormalizedNode<?, ?> data, boolean ready) {
+        return newBatchedModifications(transactionID, null, path, data, ready);
+    }
 
-        assertEquals("Last log index", expectedValue, shard.underlyingActor().getShardMBean().getLastLogIndex());
+    private BatchedModifications newBatchedModifications(String transactionID, String transactionChainID,
+            YangInstanceIdentifier path, NormalizedNode<?, ?> data, boolean ready) {
+        BatchedModifications batched = new BatchedModifications(transactionID, CURRENT_VERSION, transactionChainID);
+        batched.addModification(new WriteModification(path, data));
+        batched.setReady(ready);
+        return batched;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testMultipleBatchedModifications() throws Throwable {
+        new ShardTestKit(getSystem()) {{
+            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
+                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                    "testMultipleBatchedModifications");
+
+            waitUntilLeader(shard);
+
+            final String transactionID = "tx";
+            FiniteDuration duration = duration("5 seconds");
+
+            final AtomicReference<DOMStoreThreePhaseCommitCohort> mockCohort = new AtomicReference<>();
+            ShardCommitCoordinator.CohortDecorator cohortDecorator = new ShardCommitCoordinator.CohortDecorator() {
+                @Override
+                public DOMStoreThreePhaseCommitCohort decorate(String txID, DOMStoreThreePhaseCommitCohort actual) {
+                    if(mockCohort.get() == null) {
+                        mockCohort.set(createDelegatingMockCohort("cohort", actual));
+                    }
+
+                    return mockCohort.get();
+                }
+            };
+
+            shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator);
+
+            // Send a BatchedModifications to start a transaction.
+
+            shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH,
+                    ImmutableNodes.containerNode(TestModel.TEST_QNAME), false), getRef());
+            expectMsgClass(duration, BatchedModificationsReply.class);
+
+            // Send a couple more BatchedModifications.
+
+            shard.tell(newBatchedModifications(transactionID, TestModel.OUTER_LIST_PATH,
+                    ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), false), getRef());
+            expectMsgClass(duration, BatchedModificationsReply.class);
+
+            shard.tell(newBatchedModifications(transactionID, YangInstanceIdentifier.builder(
+                    TestModel.OUTER_LIST_PATH).nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1).build(),
+                    ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1), true), getRef());
+            expectMsgClass(duration, ReadyTransactionReply.class);
+
+            // Send the CanCommitTransaction message.
+
+            shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
+            CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
+                    expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
+            assertEquals("Can commit", true, canCommitReply.getCanCommit());
+
+            // Send the CanCommitTransaction message.
+
+            shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef());
+            expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
+
+            InOrder inOrder = inOrder(mockCohort.get());
+            inOrder.verify(mockCohort.get()).canCommit();
+            inOrder.verify(mockCohort.get()).preCommit();
+            inOrder.verify(mockCohort.get()).commit();
+
+            // Verify data in the data store.
+
+            NormalizedNode<?, ?> outerList = readStore(shard, TestModel.OUTER_LIST_PATH);
+            assertNotNull(TestModel.OUTER_LIST_QNAME.getLocalName() + " not found", outerList);
+            assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " value is not Iterable",
+                    outerList.getValue() instanceof Iterable);
+            Object entry = ((Iterable<Object>)outerList.getValue()).iterator().next();
+            assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " entry is not MapEntryNode",
+                       entry instanceof MapEntryNode);
+            MapEntryNode mapEntry = (MapEntryNode)entry;
+            Optional<DataContainerChild<? extends PathArgument, ?>> idLeaf =
+                    mapEntry.getChild(new YangInstanceIdentifier.NodeIdentifier(TestModel.ID_QNAME));
+            assertTrue("Missing leaf " + TestModel.ID_QNAME.getLocalName(), idLeaf.isPresent());
+            assertEquals(TestModel.ID_QNAME.getLocalName() + " value", 1, idLeaf.get().getValue());
+
+            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
+        }};
+    }
+
+    @Test
+    public void testBatchedModificationsOnTransactionChain() throws Throwable {
+        new ShardTestKit(getSystem()) {{
+            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
+                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                    "testBatchedModificationsOnTransactionChain");
+
+            waitUntilLeader(shard);
+
+            String transactionChainID = "txChain";
+            String transactionID1 = "tx1";
+            String transactionID2 = "tx2";
+
+            FiniteDuration duration = duration("5 seconds");
+
+            // Send a BatchedModifications to start a chained write transaction and ready it.
+
+            ContainerNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
+            YangInstanceIdentifier path = TestModel.TEST_PATH;
+            shard.tell(newBatchedModifications(transactionID1, transactionChainID, path,
+                    containerNode, true), getRef());
+            expectMsgClass(duration, ReadyTransactionReply.class);
+
+            // Create a read Tx on the same chain.
+
+            shard.tell(new CreateTransaction(transactionID2, TransactionProxy.TransactionType.READ_ONLY.ordinal() ,
+                    transactionChainID).toSerializable(), getRef());
+
+            CreateTransactionReply createReply = expectMsgClass(duration("3 seconds"), CreateTransactionReply.class);
+
+            getSystem().actorSelection(createReply.getTransactionActorPath()).tell(new ReadData(path), getRef());
+            ReadDataReply readReply = expectMsgClass(duration("3 seconds"), ReadDataReply.class);
+            assertEquals("Read node", containerNode, readReply.getNormalizedNode());
+
+            // Commit the write transaction.
+
+            shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef());
+            CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
+                    expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
+            assertEquals("Can commit", true, canCommitReply.getCanCommit());
+
+            shard.tell(new CommitTransaction(transactionID1).toSerializable(), getRef());
+            expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
+
+            // Verify data in the data store.
+
+            NormalizedNode<?, ?> actualNode = readStore(shard, path);
+            assertEquals("Stored node", containerNode, actualNode);
+
+            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
+        }};
+    }
+
+    @Test
+    public void testOnBatchedModificationsWhenNotLeader() {
+        final AtomicBoolean overrideLeaderCalls = new AtomicBoolean();
+        new ShardTestKit(getSystem()) {{
+            Creator<Shard> creator = new Creator<Shard>() {
+                @Override
+                public Shard create() throws Exception {
+                    return new Shard(shardID, Collections.<String,String>emptyMap(),
+                            newDatastoreContext(), SCHEMA_CONTEXT) {
+                        @Override
+                        protected boolean isLeader() {
+                            return overrideLeaderCalls.get() ? false : super.isLeader();
+                        }
+
+                        @Override
+                        protected ActorSelection getLeader() {
+                            return overrideLeaderCalls.get() ? getSystem().actorSelection(getRef().path()) :
+                                super.getLeader();
+                        }
+                    };
+                }
+            };
+
+            TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
+                    Props.create(new DelegatingShardCreator(creator)), "testOnBatchedModificationsWhenNotLeader");
+
+            waitUntilLeader(shard);
+
+            overrideLeaderCalls.set(true);
+
+            BatchedModifications batched = new BatchedModifications("tx", DataStoreVersions.CURRENT_VERSION, "");
+
+            shard.tell(batched, ActorRef.noSender());
+
+            expectMsgEquals(batched);
+
+            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
+        }};
     }
 
     @Test
@@ -899,7 +806,7 @@ public class ShardTest extends AbstractActorTest {
         new ShardTestKit(getSystem()) {{
             final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
                     newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testCommitPhaseFailure");
+                    "testCommitWithPersistenceDisabled");
 
             waitUntilLeader(shard);
 
@@ -920,7 +827,7 @@ public class ShardTest extends AbstractActorTest {
 
             shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION,
                     cohort, modification, true), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
+            expectMsgClass(duration, ReadyTransactionReply.class);
 
             // Send the CanCommitTransaction message.
 
@@ -946,6 +853,117 @@ public class ShardTest extends AbstractActorTest {
         }};
     }
 
+    @Test
+    public void testCommitWhenTransactionHasNoModifications(){
+        // 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 = TestActorRef.create(getSystem(),
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testCommitWhenTransactionHasNoModifications");
+
+                waitUntilLeader(shard);
+
+                String transactionID = "tx1";
+                MutableCompositeModification modification = new MutableCompositeModification();
+                DOMStoreThreePhaseCommitCohort cohort = mock(DOMStoreThreePhaseCommitCohort.class, "cohort1");
+                doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit();
+                doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).preCommit();
+                doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).commit();
+
+                FiniteDuration duration = duration("5 seconds");
+
+                // Simulate the ForwardedReadyTransaction messages that would be sent
+                // by the ShardTransaction.
+
+                shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION,
+                        cohort, modification, true), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
+
+                // Send the CanCommitTransaction message.
+
+                shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
+                CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
+                        expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
+                assertEquals("Can commit", true, canCommitReply.getCanCommit());
+
+                shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef());
+                expectMsgClass(duration, ThreePhaseCommitCohortMessages.CommitTransactionReply.class);
+
+                InOrder inOrder = inOrder(cohort);
+                inOrder.verify(cohort).canCommit();
+                inOrder.verify(cohort).preCommit();
+                inOrder.verify(cohort).commit();
+
+                // Use MBean for verification
+                // Committed transaction count should increase as usual
+                assertEquals(1,shard.underlyingActor().getShardMBean().getCommittedTransactionsCount());
+
+                // Commit index should not advance because this does not go into the journal
+                assertEquals(-1, shard.underlyingActor().getShardMBean().getCommitIndex());
+
+                shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
+
+            }
+        };
+    }
+
+    @Test
+    public void testCommitWhenTransactionHasModifications(){
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testCommitWhenTransactionHasModifications");
+
+                waitUntilLeader(shard);
+
+                String transactionID = "tx1";
+                MutableCompositeModification modification = new MutableCompositeModification();
+                modification.addModification(new DeleteModification(YangInstanceIdentifier.builder().build()));
+                DOMStoreThreePhaseCommitCohort cohort = mock(DOMStoreThreePhaseCommitCohort.class, "cohort1");
+                doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit();
+                doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).preCommit();
+                doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).commit();
+
+                FiniteDuration duration = duration("5 seconds");
+
+                // Simulate the ForwardedReadyTransaction messages that would be sent
+                // by the ShardTransaction.
+
+                shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION,
+                        cohort, modification, true), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
+
+                // Send the CanCommitTransaction message.
+
+                shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
+                CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
+                        expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
+                assertEquals("Can commit", true, canCommitReply.getCanCommit());
+
+                shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef());
+                expectMsgClass(duration, ThreePhaseCommitCohortMessages.CommitTransactionReply.class);
+
+                InOrder inOrder = inOrder(cohort);
+                inOrder.verify(cohort).canCommit();
+                inOrder.verify(cohort).preCommit();
+                inOrder.verify(cohort).commit();
+
+                // Use MBean for verification
+                // Committed transaction count should increase as usual
+                assertEquals(1, shard.underlyingActor().getShardMBean().getCommittedTransactionsCount());
+
+                // Commit index should advance as we do not have an empty modification
+                assertEquals(0, shard.underlyingActor().getShardMBean().getCommitIndex());
+
+                shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
+
+            }
+        };
+    }
+
     @Test
     public void testCommitPhaseFailure() throws Throwable {
         new ShardTestKit(getSystem()) {{
@@ -955,7 +973,7 @@ public class ShardTest extends AbstractActorTest {
 
             waitUntilLeader(shard);
 
-            // Setup 2 simulated transactions with mock cohorts. The first one fails in the
+         // Setup 2 simulated transactions with mock cohorts. The first one fails in the
             // commit phase.
 
             String transactionID1 = "tx1";
@@ -978,11 +996,11 @@ public class ShardTest extends AbstractActorTest {
 
             shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION,
                     cohort1, modification1, true), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
+            expectMsgClass(duration, ReadyTransactionReply.class);
 
             shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION,
                     cohort2, modification2, true), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
+            expectMsgClass(duration, ReadyTransactionReply.class);
 
             // Send the CanCommitTransaction message for the first Tx.
 
@@ -1047,7 +1065,7 @@ public class ShardTest extends AbstractActorTest {
 
             shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION,
                     cohort, modification, true), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
+            expectMsgClass(duration, ReadyTransactionReply.class);
 
             // Send the CanCommitTransaction message.
 
@@ -1091,7 +1109,7 @@ public class ShardTest extends AbstractActorTest {
 
             shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION,
                     cohort, modification, true), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
+            expectMsgClass(duration, ReadyTransactionReply.class);
 
             // Send the CanCommitTransaction message.
 
@@ -1142,7 +1160,7 @@ public class ShardTest extends AbstractActorTest {
 
             shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION,
                     cohort, modification, true), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
+            expectMsgClass(duration, ReadyTransactionReply.class);
 
             shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
             CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
@@ -1207,11 +1225,11 @@ public class ShardTest extends AbstractActorTest {
 
             shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION,
                     cohort1, modification1, true), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
+            expectMsgClass(duration, ReadyTransactionReply.class);
 
             shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION,
                     cohort2, modification2, true), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
+            expectMsgClass(duration, ReadyTransactionReply.class);
 
             // canCommit 1st Tx. We don't send the commit so it should timeout.
 
@@ -1271,15 +1289,15 @@ public class ShardTest extends AbstractActorTest {
 
             shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION,
                     cohort1, modification1, true), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
+            expectMsgClass(duration, ReadyTransactionReply.class);
 
             shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION,
                     cohort2, modification2, true), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
+            expectMsgClass(duration, ReadyTransactionReply.class);
 
             shard.tell(new ForwardedReadyTransaction(transactionID3, CURRENT_VERSION,
                     cohort3, modification3, true), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
+            expectMsgClass(duration, ReadyTransactionReply.class);
 
             // canCommit 1st Tx.
 
@@ -1343,11 +1361,11 @@ public class ShardTest extends AbstractActorTest {
 
             shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION,
                     cohort1, modification1, true), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
+            expectMsgClass(duration, ReadyTransactionReply.class);
 
             shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION,
                     cohort2, modification2, true), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
+            expectMsgClass(duration, ReadyTransactionReply.class);
 
             // Send the CanCommitTransaction message for the first Tx.
 
@@ -1393,68 +1411,52 @@ public class ShardTest extends AbstractActorTest {
     @SuppressWarnings("serial")
     public void testCreateSnapshot(final boolean persistent, final String shardActorName) throws Exception{
 
-        final AtomicReference<Object> savedSnapshot = new AtomicReference<>();
-        class DelegatingPersistentDataProvider implements DataPersistenceProvider {
-            DataPersistenceProvider delegate;
-
-            DelegatingPersistentDataProvider(DataPersistenceProvider delegate) {
-                this.delegate = delegate;
-            }
-
-            @Override
-            public boolean isRecoveryApplicable() {
-                return delegate.isRecoveryApplicable();
-            }
+        final AtomicReference<CountDownLatch> latch = new AtomicReference<>(new CountDownLatch(1));
 
-            @Override
-            public <T> void persist(T o, Procedure<T> procedure) {
-                delegate.persist(o, procedure);
+        final AtomicReference<Object> savedSnapshot = new AtomicReference<>();
+        class TestPersistentDataProvider extends DelegatingPersistentDataProvider {
+            TestPersistentDataProvider(DataPersistenceProvider delegate) {
+                super(delegate);
             }
 
             @Override
             public void saveSnapshot(Object o) {
                 savedSnapshot.set(o);
-                delegate.saveSnapshot(o);
-            }
-
-            @Override
-            public void deleteSnapshots(SnapshotSelectionCriteria criteria) {
-                delegate.deleteSnapshots(criteria);
-            }
-
-            @Override
-            public void deleteMessages(long sequenceNumber) {
-                delegate.deleteMessages(sequenceNumber);
+                super.saveSnapshot(o);
             }
         }
 
         dataStoreContextBuilder.persistent(persistent);
 
         new ShardTestKit(getSystem()) {{
-            final AtomicReference<CountDownLatch> latch = new AtomicReference<>(new CountDownLatch(1));
-            Creator<Shard> creator = new Creator<Shard>() {
-                @Override
-                public Shard create() throws Exception {
-                    return new Shard(shardID, Collections.<ShardIdentifier,String>emptyMap(),
-                            newDatastoreContext(), SCHEMA_CONTEXT) {
+            class TestShard extends Shard {
 
-                        DelegatingPersistentDataProvider delegating;
+                protected TestShard(ShardIdentifier name, Map<String, String> peerAddresses,
+                                    DatastoreContext datastoreContext, SchemaContext schemaContext) {
+                    super(name, peerAddresses, datastoreContext, schemaContext);
+                    setPersistence(new TestPersistentDataProvider(super.persistence()));
+                }
 
-                        @Override
-                        protected DataPersistenceProvider persistence() {
-                            if(delegating == null) {
-                                delegating = new DelegatingPersistentDataProvider(super.persistence());
-                            }
+                @Override
+                public void handleCommand(Object message) {
+                    super.handleCommand(message);
 
-                            return delegating;
-                        }
+                    if (message instanceof SaveSnapshotSuccess || message.equals("commit_snapshot")) {
+                        latch.get().countDown();
+                    }
+                }
 
-                        @Override
-                        protected void commitSnapshot(final long sequenceNumber) {
-                            super.commitSnapshot(sequenceNumber);
-                            latch.get().countDown();
-                        }
-                    };
+                @Override
+                public RaftActorContext getRaftActorContext() {
+                    return super.getRaftActorContext();
+                }
+            }
+
+            Creator<Shard> creator = new Creator<Shard>() {
+                @Override
+                public Shard create() throws Exception {
+                    return new TestShard(shardID, Collections.<String,String>emptyMap(),
+                            newDatastoreContext(), SCHEMA_CONTEXT);
                 }
             };
 
@@ -1467,8 +1469,9 @@ public class ShardTest extends AbstractActorTest {
 
             NormalizedNode<?,?> expectedRoot = readStore(shard, YangInstanceIdentifier.builder().build());
 
-            CaptureSnapshot capture = new CaptureSnapshot(-1, -1, -1, -1);
-            shard.tell(capture, getRef());
+            // Trigger creation of a snapshot by ensuring
+            RaftActorContext raftActorContext = ((TestShard) shard.underlyingActor()).getRaftActorContext();
+            raftActorContext.getSnapshotManager().capture(mock(ReplicatedLogEntry.class), -1);
 
             assertEquals("Snapshot saved", true, latch.get().await(5, TimeUnit.SECONDS));
 
@@ -1480,7 +1483,7 @@ public class ShardTest extends AbstractActorTest {
             latch.set(new CountDownLatch(1));
             savedSnapshot.set(null);
 
-            shard.tell(capture, getRef());
+            raftActorContext.getSnapshotManager().capture(mock(ReplicatedLogEntry.class), -1);
 
             assertEquals("Snapshot saved", true, latch.get().await(5, TimeUnit.SECONDS));
 
@@ -1536,27 +1539,27 @@ public class ShardTest extends AbstractActorTest {
         final DatastoreContext persistentContext = DatastoreContext.newBuilder().
                 shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).persistent(true).build();
 
-        final Props persistentProps = Shard.props(shardID, Collections.<ShardIdentifier, String>emptyMap(),
+        final Props persistentProps = Shard.props(shardID, Collections.<String, String>emptyMap(),
                 persistentContext, SCHEMA_CONTEXT);
 
         final DatastoreContext nonPersistentContext = DatastoreContext.newBuilder().
                 shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).persistent(false).build();
 
-        final Props nonPersistentProps = Shard.props(shardID, Collections.<ShardIdentifier, String>emptyMap(),
+        final Props nonPersistentProps = Shard.props(shardID, Collections.<String, String>emptyMap(),
                 nonPersistentContext, SCHEMA_CONTEXT);
 
         new ShardTestKit(getSystem()) {{
             TestActorRef<Shard> shard1 = TestActorRef.create(getSystem(),
                     persistentProps, "testPersistence1");
 
-            assertTrue("Recovery Applicable", shard1.underlyingActor().getDataPersistenceProvider().isRecoveryApplicable());
+            assertTrue("Recovery Applicable", shard1.underlyingActor().persistence().isRecoveryApplicable());
 
             shard1.tell(PoisonPill.getInstance(), ActorRef.noSender());
 
             TestActorRef<Shard> shard2 = TestActorRef.create(getSystem(),
                     nonPersistentProps, "testPersistence2");
 
-            assertFalse("Recovery Not Applicable", shard2.underlyingActor().getDataPersistenceProvider().isRecoveryApplicable());
+            assertFalse("Recovery Not Applicable", shard2.underlyingActor().persistence().isRecoveryApplicable());
 
             shard2.tell(PoisonPill.getInstance(), ActorRef.noSender());
 
@@ -1564,90 +1567,85 @@ public class ShardTest extends AbstractActorTest {
 
     }
 
+    @Test
+    public void testOnDatastoreContext() {
+        new ShardTestKit(getSystem()) {{
+            dataStoreContextBuilder.persistent(true);
 
-    private NormalizedNode<?, ?> readStore(final InMemoryDOMDataStore store) throws ReadFailedException {
-        DOMStoreReadTransaction transaction = store.newReadOnlyTransaction();
-        CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> read =
-            transaction.read(YangInstanceIdentifier.builder().build());
+            TestActorRef<Shard> shard = TestActorRef.create(getSystem(), newShardProps(), "testOnDatastoreContext");
 
-        Optional<NormalizedNode<?, ?>> optional = read.checkedGet();
+            assertEquals("isRecoveryApplicable", true,
+                    shard.underlyingActor().persistence().isRecoveryApplicable());
 
-        NormalizedNode<?, ?> normalizedNode = optional.get();
+            waitUntilLeader(shard);
 
-        transaction.close();
+            shard.tell(dataStoreContextBuilder.persistent(false).build(), ActorRef.noSender());
 
-        return normalizedNode;
-    }
+            assertEquals("isRecoveryApplicable", false,
+                    shard.underlyingActor().persistence().isRecoveryApplicable());
 
-    private void commitTransaction(final DOMStoreWriteTransaction transaction) {
-        DOMStoreThreePhaseCommitCohort commitCohort = transaction.ready();
-        ListenableFuture<Void> future =
-            commitCohort.preCommit();
-        try {
-            future.get();
-            future = commitCohort.commit();
-            future.get();
-        } catch (InterruptedException | ExecutionException e) {
-        }
-    }
+            shard.tell(dataStoreContextBuilder.persistent(true).build(), ActorRef.noSender());
 
-    private AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>> noOpDataChangeListener() {
-        return new AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>() {
-            @Override
-            public void onDataChanged(
-                final AsyncDataChangeEvent<YangInstanceIdentifier, NormalizedNode<?, ?>> change) {
+            assertEquals("isRecoveryApplicable", true,
+                    shard.underlyingActor().persistence().isRecoveryApplicable());
 
-            }
-        };
+            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
+        }};
     }
 
-    static NormalizedNode<?,?> readStore(final TestActorRef<Shard> shard, final YangInstanceIdentifier id)
-            throws ExecutionException, InterruptedException {
-        return readStore(shard.underlyingActor().getDataStore(), id);
-    }
+    @Test
+    public void testRegisterRoleChangeListener() throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testRegisterRoleChangeListener");
 
-    public static NormalizedNode<?,?> readStore(final InMemoryDOMDataStore store, final YangInstanceIdentifier id)
-            throws ExecutionException, InterruptedException {
-        DOMStoreReadTransaction transaction = store.newReadOnlyTransaction();
+                waitUntilLeader(shard);
 
-        CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> future =
-            transaction.read(id);
+                TestActorRef<MessageCollectorActor> listener =
+                        TestActorRef.create(getSystem(), Props.create(MessageCollectorActor.class));
 
-        Optional<NormalizedNode<?, ?>> optional = future.get();
-        NormalizedNode<?, ?> node = optional.isPresent()? optional.get() : null;
+                shard.tell(new RegisterRoleChangeListener(), listener);
 
-        transaction.close();
+                // TODO: MessageCollectorActor exists as a test util in both the akka-raft and distributed-datastore
+                // projects. Need to move it to commons as a regular utility and then we can get rid of this arbitrary
+                // sleep.
+                Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 
-        return node;
-    }
+                List<Object> allMatching = MessageCollectorActor.getAllMatching(listener, RegisterRoleChangeListenerReply.class);
 
-    static void writeToStore(final TestActorRef<Shard> shard, final YangInstanceIdentifier id,
-            final NormalizedNode<?,?> node) throws ExecutionException, InterruptedException {
-        writeToStore(shard.underlyingActor().getDataStore(), id, node);
+                assertEquals(1, allMatching.size());
+            }
+        };
     }
 
-    public static void writeToStore(final InMemoryDOMDataStore store, final YangInstanceIdentifier id,
-            final NormalizedNode<?,?> node) throws ExecutionException, InterruptedException {
-        DOMStoreWriteTransaction transaction = store.newWriteOnlyTransaction();
+    @Test
+    public void testFollowerInitialSyncStatus() throws Exception {
+        final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
+                newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                "testFollowerInitialSyncStatus");
 
-        transaction.write(id, node);
+        shard.underlyingActor().onReceiveCommand(new FollowerInitialSyncUpStatus(false, "member-1-shard-inventory-operational"));
 
-        DOMStoreThreePhaseCommitCohort commitCohort = transaction.ready();
-        commitCohort.preCommit().get();
-        commitCohort.commit().get();
-    }
+        assertEquals(false, shard.underlyingActor().getShardMBean().getFollowerInitialSyncStatus());
 
-    @SuppressWarnings("serial")
-    private static final class DelegatingShardCreator implements Creator<Shard> {
-        private final Creator<Shard> delegate;
+        shard.underlyingActor().onReceiveCommand(new FollowerInitialSyncUpStatus(true, "member-1-shard-inventory-operational"));
 
-        DelegatingShardCreator(final Creator<Shard> delegate) {
-            this.delegate = delegate;
-        }
+        assertEquals(true, shard.underlyingActor().getShardMBean().getFollowerInitialSyncStatus());
 
-        @Override
-        public Shard create() throws Exception {
-            return delegate.create();
+        shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
+    }
+
+    private void commitTransaction(final DOMStoreWriteTransaction transaction) {
+        DOMStoreThreePhaseCommitCohort commitCohort = transaction.ready();
+        ListenableFuture<Void> future =
+            commitCohort.preCommit();
+        try {
+            future.get();
+            future = commitCohort.commit();
+            future.get();
+        } catch (InterruptedException | ExecutionException e) {
         }
     }
 }