Refactor Register*ListenerReply classes
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / ShardTest.java
index 25e37edf714cc8c6b2c6a6cf33ccc47b854c287f..ce2a43f015bbcb5ea67f791514f0eef02e7c19ae 100644 (file)
@@ -11,18 +11,17 @@ package org.opendaylight.controller.cluster.datastore;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.doReturn;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.inOrder;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.reset;
-import static org.mockito.Mockito.verify;
 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.actor.Status.Failure;
 import akka.dispatch.Dispatchers;
@@ -32,14 +31,13 @@ import akka.pattern.Patterns;
 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.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.base.Stopwatch;
+import com.google.common.base.Throwables;
 import com.google.common.util.concurrent.Uninterruptibles;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
@@ -49,6 +47,10 @@ 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.LocalHistoryIdentifier;
+import org.opendaylight.controller.cluster.access.concepts.MemberName;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
+import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
 import org.opendaylight.controller.cluster.datastore.messages.AbortTransaction;
@@ -57,68 +59,63 @@ import org.opendaylight.controller.cluster.datastore.messages.BatchedModificatio
 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.CloseDataTreeNotificationListenerRegistration;
+import org.opendaylight.controller.cluster.datastore.messages.CloseDataTreeNotificationListenerRegistrationReply;
 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.PeerAddressResolved;
 import org.opendaylight.controller.cluster.datastore.messages.ReadData;
 import org.opendaylight.controller.cluster.datastore.messages.ReadDataReply;
 import org.opendaylight.controller.cluster.datastore.messages.ReadyLocalTransaction;
 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.RegisterDataTreeChangeListener;
-import org.opendaylight.controller.cluster.datastore.messages.RegisterDataTreeChangeListenerReply;
+import org.opendaylight.controller.cluster.datastore.messages.RegisterDataTreeNotificationListenerReply;
 import org.opendaylight.controller.cluster.datastore.messages.ShardLeaderStateChanged;
 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.persisted.MetadataShardDataTreeSnapshot;
+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.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.ApplyJournalEntries;
-import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
 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;
 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.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;
-import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
 import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor;
 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.ReadFailedException;
-import org.opendaylight.controller.protobuff.messages.cohort3pc.ThreePhaseCommitCohortMessages;
-import org.opendaylight.controller.protobuff.messages.transaction.ShardTransactionMessages.CreateTransactionReply;
-import org.opendaylight.yangtools.yang.common.QName;
+import org.opendaylight.yangtools.concepts.Identifier;
 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.MapNode;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidate;
-import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidateNode;
-import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidateTip;
-import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidates;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException;
-import org.opendaylight.yangtools.yang.data.api.schema.tree.ModificationType;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.TipProducingDataTree;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType;
 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
 import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder;
@@ -128,41 +125,40 @@ import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
 public class ShardTest extends AbstractShardTest {
-    private static final QName CARS_QNAME = QName.create("urn:opendaylight:params:xml:ns:yang:controller:md:sal:dom:store:test:cars", "2014-03-13", "cars");
-
-    private static final String DUMMY_DATA = "Dummy data as snapshot sequence number is set to 0 in InMemorySnapshotStore and journal recovery seq number will start from 1";
+    private static final String DUMMY_DATA = "Dummy data as snapshot sequence number is set to 0 in "
+            + "InMemorySnapshotStore and journal recovery seq number will start from 1";
 
     @Test
     public void testRegisterChangeListener() throws Exception {
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps(),  "testRegisterChangeListener");
-
-            waitUntilLeader(shard);
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testRegisterChangeListener");
 
-            shard.tell(new UpdateSchemaContext(SchemaContextHelper.full()), ActorRef.noSender());
+                waitUntilLeader(shard);
 
-            final MockDataChangeListener listener = new MockDataChangeListener(1);
-            final ActorRef dclActor = getSystem().actorOf(DataChangeListener.props(listener),
-                    "testRegisterChangeListener-DataChangeListener");
+                shard.tell(new UpdateSchemaContext(SchemaContextHelper.full()), ActorRef.noSender());
 
-            shard.tell(new RegisterChangeListener(TestModel.TEST_PATH,
-                    dclActor, AsyncDataBroker.DataChangeScope.BASE, true), getRef());
+                final MockDataChangeListener listener = new MockDataChangeListener(1);
+                final ActorRef dclActor = actorFactory.createActor(DataChangeListener.props(listener,
+                        TestModel.TEST_PATH), "testRegisterChangeListener-DataChangeListener");
 
-            final RegisterChangeListenerReply reply = expectMsgClass(duration("3 seconds"),
-                    RegisterChangeListenerReply.class);
-            final String replyPath = reply.getListenerRegistrationPath().toString();
-            assertTrue("Incorrect reply path: " + replyPath, replyPath.matches(
-                    "akka:\\/\\/test\\/user\\/testRegisterChangeListener\\/\\$.*"));
+                shard.tell(new RegisterChangeListener(TestModel.TEST_PATH, dclActor,
+                        AsyncDataBroker.DataChangeScope.BASE, true), getRef());
 
-            final YangInstanceIdentifier path = TestModel.TEST_PATH;
-            writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+                final RegisterDataTreeNotificationListenerReply reply = expectMsgClass(duration("3 seconds"),
+                        RegisterDataTreeNotificationListenerReply.class);
+                final String replyPath = reply.getListenerRegistrationPath().toString();
+                assertTrue("Incorrect reply path: " + replyPath,
+                        replyPath.matches("akka:\\/\\/test\\/user\\/testRegisterChangeListener\\/\\$.*"));
 
-            listener.waitForChangeEvents(path);
+                final YangInstanceIdentifier path = TestModel.TEST_PATH;
+                writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
 
-            dclActor.tell(PoisonPill.getInstance(), ActorRef.noSender());
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+                listener.waitForChangeEvents(path);
+            }
+        };
     }
 
     @SuppressWarnings("serial")
@@ -171,394 +167,334 @@ public class ShardTest extends AbstractShardTest {
         // This test tests the timing window in which a change listener is registered before the
         // shard becomes the leader. We verify that the listener is registered and notified of the
         // existing data when the shard becomes the leader.
-        new ShardTestKit(getSystem()) {{
-            // For this test, we want to send the RegisterChangeListener message after the shard
-            // has recovered from persistence and before it becomes the leader. So we subclass
-            // Shard to override onReceiveCommand and, when the first ElectionTimeout is received,
-            // we know that the shard has been initialized to a follower and has started the
-            // election process. The following 2 CountDownLatches are used to coordinate the
-            // ElectionTimeout with the sending of the RegisterChangeListener message.
-            final CountDownLatch onFirstElectionTimeout = new CountDownLatch(1);
-            final CountDownLatch onChangeListenerRegistered = new CountDownLatch(1);
-            final Creator<Shard> creator = new Creator<Shard>() {
-                boolean firstElectionTimeout = true;
-
-                @Override
-                public Shard create() throws Exception {
-                    // 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(newShardBuilder()) {
-                        @Override
-                        public void onReceiveCommand(final Object message) throws Exception {
-                            if(message instanceof ElectionTimeout && firstElectionTimeout) {
-                                // Got the first ElectionTimeout. We don't forward it to the
-                                // base Shard yet until we've sent the RegisterChangeListener
-                                // message. So we signal the onFirstElectionTimeout latch to tell
-                                // the main thread to send the RegisterChangeListener message and
-                                // start a thread to wait on the onChangeListenerRegistered latch,
-                                // which the main thread signals after it has sent the message.
-                                // After the onChangeListenerRegistered is triggered, we send the
-                                // original ElectionTimeout message to proceed with the election.
-                                firstElectionTimeout = false;
-                                final ActorRef self = getSelf();
-                                new Thread() {
-                                    @Override
-                                    public void run() {
-                                        Uninterruptibles.awaitUninterruptibly(
-                                                onChangeListenerRegistered, 5, TimeUnit.SECONDS);
-                                        self.tell(message, self);
-                                    }
-                                }.start();
-
-                                onFirstElectionTimeout.countDown();
-                            } else {
-                                super.onReceiveCommand(message);
-                            }
-                        }
-                    };
-                }
-            };
-
-            final MockDataChangeListener listener = new MockDataChangeListener(1);
-            final ActorRef dclActor = getSystem().actorOf(DataChangeListener.props(listener),
-                    "testRegisterChangeListenerWhenNotLeaderInitially-DataChangeListener");
+        // For this test, we want to send the RegisterChangeListener message after the shard
+        // has recovered from persistence and before it becomes the leader. So we subclass
+        // Shard to override onReceiveCommand and, when the first ElectionTimeout is received,
+        // we know that the shard has been initialized to a follower and has started the
+        // election process. The following 2 CountDownLatches are used to coordinate the
+        // ElectionTimeout with the sending of the RegisterChangeListener message.
+        final CountDownLatch onFirstElectionTimeout = new CountDownLatch(1);
+        final CountDownLatch onChangeListenerRegistered = new CountDownLatch(1);
+        final Creator<Shard> creator = new Creator<Shard>() {
+            boolean firstElectionTimeout = true;
 
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testRegisterChangeListenerWhenNotLeaderInitially");
-
-            // Write initial data into the in-memory store.
-            final YangInstanceIdentifier path = TestModel.TEST_PATH;
-            writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
-
-            // Wait until the shard receives the first ElectionTimeout message.
-            assertEquals("Got first ElectionTimeout", true,
-                    onFirstElectionTimeout.await(5, TimeUnit.SECONDS));
-
-            // Now send the RegisterChangeListener and wait for the reply.
-            shard.tell(new RegisterChangeListener(path, dclActor,
-                    AsyncDataBroker.DataChangeScope.SUBTREE, false), getRef());
+            @Override
+            public Shard create() throws Exception {
+                // 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(newShardBuilder()) {
+                    @Override
+                    public void handleCommand(final Object message) {
+                        if (message instanceof ElectionTimeout && firstElectionTimeout) {
+                            // Got the first ElectionTimeout. We don't forward it to the
+                            // base Shard yet until we've sent the RegisterChangeListener
+                            // message. So we signal the onFirstElectionTimeout latch to tell
+                            // the main thread to send the RegisterChangeListener message and
+                            // start a thread to wait on the onChangeListenerRegistered latch,
+                            // which the main thread signals after it has sent the message.
+                            // After the onChangeListenerRegistered is triggered, we send the
+                            // original ElectionTimeout message to proceed with the election.
+                            firstElectionTimeout = false;
+                            final ActorRef self = getSelf();
+                            new Thread() {
+                                @Override
+                                public void run() {
+                                    Uninterruptibles.awaitUninterruptibly(
+                                            onChangeListenerRegistered, 5, TimeUnit.SECONDS);
+                                    self.tell(message, self);
+                                }
+                            }.start();
 
-            final RegisterChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
-                    RegisterChangeListenerReply.class);
-            assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
+                            onFirstElectionTimeout.countDown();
+                        } else {
+                            super.handleCommand(message);
+                        }
+                    }
+                };
+            }
+        };
 
-            // Sanity check - verify the shard is not the leader yet.
-            shard.tell(new FindLeader(), getRef());
-            final FindLeaderReply findLeadeReply =
-                    expectMsgClass(duration("5 seconds"), FindLeaderReply.class);
-            assertNull("Expected the shard not to be the leader", findLeadeReply.getLeaderActor());
+        setupInMemorySnapshotStore();
 
-            // Signal the onChangeListenerRegistered latch to tell the thread above to proceed
-            // with the election process.
-            onChangeListenerRegistered.countDown();
+        final YangInstanceIdentifier path = TestModel.TEST_PATH;
+        final MockDataChangeListener listener = new MockDataChangeListener(1);
+        final ActorRef dclActor = actorFactory.createActor(DataChangeListener.props(listener, path),
+                "testRegisterChangeListenerWhenNotLeaderInitially-DataChangeListener");
 
-            // Wait for the shard to become the leader and notify our listener with the existing
-            // data in the store.
-            listener.waitForChangeEvents(path);
+        final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
+                "testRegisterChangeListenerWhenNotLeaderInitially");
 
-            dclActor.tell(PoisonPill.getInstance(), ActorRef.noSender());
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+        new ShardTestKit(getSystem()) {
+            {
+                // Wait until the shard receives the first ElectionTimeout
+                // message.
+                assertEquals("Got first ElectionTimeout", true, onFirstElectionTimeout.await(5, TimeUnit.SECONDS));
+
+                // Now send the RegisterChangeListener and wait for the reply.
+                shard.tell(new RegisterChangeListener(path, dclActor, AsyncDataBroker.DataChangeScope.SUBTREE, false),
+                        getRef());
+
+                final RegisterDataTreeNotificationListenerReply reply = expectMsgClass(duration("5 seconds"),
+                        RegisterDataTreeNotificationListenerReply.class);
+                assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
+
+                // Sanity check - verify the shard is not the leader yet.
+                shard.tell(FindLeader.INSTANCE, getRef());
+                final FindLeaderReply findLeadeReply = expectMsgClass(duration("5 seconds"), FindLeaderReply.class);
+                assertFalse("Expected the shard not to be the leader", findLeadeReply.getLeaderActor().isPresent());
+
+                // Signal the onChangeListenerRegistered latch to tell the
+                // thread above to proceed
+                // with the election process.
+                onChangeListenerRegistered.countDown();
+
+                // Wait for the shard to become the leader and notify our
+                // listener with the existing
+                // data in the store.
+                listener.waitForChangeEvents(path);
+            }
+        };
     }
 
     @Test
     public void testRegisterDataTreeChangeListener() throws Exception {
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps(), "testRegisterDataTreeChangeListener");
-
-            waitUntilLeader(shard);
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testRegisterDataTreeChangeListener");
 
-            shard.tell(new UpdateSchemaContext(SchemaContextHelper.full()), ActorRef.noSender());
+                waitUntilLeader(shard);
 
-            final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
-            final ActorRef dclActor = getSystem().actorOf(DataTreeChangeListenerActor.props(listener),
-                    "testRegisterDataTreeChangeListener-DataTreeChangeListener");
+                shard.tell(new UpdateSchemaContext(SchemaContextHelper.full()), ActorRef.noSender());
 
-            shard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor, false), getRef());
+                final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
+                final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener,
+                        TestModel.TEST_PATH), "testRegisterDataTreeChangeListener-DataTreeChangeListener");
 
-            final RegisterDataTreeChangeListenerReply reply = expectMsgClass(duration("3 seconds"),
-                    RegisterDataTreeChangeListenerReply.class);
-            final String replyPath = reply.getListenerRegistrationPath().toString();
-            assertTrue("Incorrect reply path: " + replyPath, replyPath.matches(
-                    "akka:\\/\\/test\\/user\\/testRegisterDataTreeChangeListener\\/\\$.*"));
+                shard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor, false), getRef());
 
-            final YangInstanceIdentifier path = TestModel.TEST_PATH;
-            writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+                final RegisterDataTreeNotificationListenerReply reply = expectMsgClass(duration("3 seconds"),
+                        RegisterDataTreeNotificationListenerReply.class);
+                final String replyPath = reply.getListenerRegistrationPath().toString();
+                assertTrue("Incorrect reply path: " + replyPath,
+                        replyPath.matches("akka:\\/\\/test\\/user\\/testRegisterDataTreeChangeListener\\/\\$.*"));
 
-            listener.waitForChangeEvents();
+                final YangInstanceIdentifier path = TestModel.TEST_PATH;
+                writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
 
-            dclActor.tell(PoisonPill.getInstance(), ActorRef.noSender());
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+                listener.waitForChangeEvents();
+            }
+        };
     }
 
     @SuppressWarnings("serial")
     @Test
     public void testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration() throws Exception {
-        new ShardTestKit(getSystem()) {{
-            final CountDownLatch onFirstElectionTimeout = new CountDownLatch(1);
-            final CountDownLatch onChangeListenerRegistered = new CountDownLatch(1);
-            final Creator<Shard> creator = new Creator<Shard>() {
-                boolean firstElectionTimeout = true;
-
-                @Override
-                public Shard create() throws Exception {
-                    return new Shard(Shard.builder().id(shardID).datastoreContext(
-                            dataStoreContextBuilder.persistent(false).build()).schemaContext(SCHEMA_CONTEXT)) {
-                        @Override
-                        public void onReceiveCommand(final Object message) throws Exception {
-                            if(message instanceof ElectionTimeout && firstElectionTimeout) {
-                                firstElectionTimeout = false;
-                                final ActorRef self = getSelf();
-                                new Thread() {
-                                    @Override
-                                    public void run() {
-                                        Uninterruptibles.awaitUninterruptibly(
-                                                onChangeListenerRegistered, 5, TimeUnit.SECONDS);
-                                        self.tell(message, self);
-                                    }
-                                }.start();
-
-                                onFirstElectionTimeout.countDown();
-                            } else {
-                                super.onReceiveCommand(message);
-                            }
-                        }
-                    };
-                }
-            };
+        final CountDownLatch onFirstElectionTimeout = new CountDownLatch(1);
+        final CountDownLatch onChangeListenerRegistered = new CountDownLatch(1);
+        final Creator<Shard> creator = new Creator<Shard>() {
+            boolean firstElectionTimeout = true;
 
-            final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
-            final ActorRef dclActor = getSystem().actorOf(DataTreeChangeListenerActor.props(listener),
-                    "testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration-DataChangeListener");
+            @Override
+            public Shard create() throws Exception {
+                return new Shard(newShardBuilder()) {
+                    @Override
+                    public void handleCommand(final Object message) {
+                        if (message instanceof ElectionTimeout && firstElectionTimeout) {
+                            firstElectionTimeout = false;
+                            final ActorRef self = getSelf();
+                            new Thread() {
+                                @Override
+                                public void run() {
+                                    Uninterruptibles.awaitUninterruptibly(
+                                            onChangeListenerRegistered, 5, TimeUnit.SECONDS);
+                                    self.tell(message, self);
+                                }
+                            }.start();
 
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration");
+                            onFirstElectionTimeout.countDown();
+                        } else {
+                            super.handleCommand(message);
+                        }
+                    }
+                };
+            }
+        };
 
-            final YangInstanceIdentifier path = TestModel.TEST_PATH;
-            writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+        setupInMemorySnapshotStore();
 
-            assertEquals("Got first ElectionTimeout", true,
-                onFirstElectionTimeout.await(5, TimeUnit.SECONDS));
+        final YangInstanceIdentifier path = TestModel.TEST_PATH;
+        final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
+        final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener, path),
+                "testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration-DataChangeListener");
 
-            shard.tell(new RegisterDataTreeChangeListener(path, dclActor, false), getRef());
-            final RegisterDataTreeChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
-                RegisterDataTreeChangeListenerReply.class);
-            assertNotNull("getListenerRegistratioznPath", reply.getListenerRegistrationPath());
+        final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                Props.create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
+                "testDataTreeChangeListenerNotifiedWhenNotTheLeaderOnRegistration");
 
-            shard.tell(new FindLeader(), getRef());
-            final FindLeaderReply findLeadeReply =
-                    expectMsgClass(duration("5 seconds"), FindLeaderReply.class);
-            assertNull("Expected the shard not to be the leader", findLeadeReply.getLeaderActor());
+        new ShardTestKit(getSystem()) {
+            {
+                assertEquals("Got first ElectionTimeout", true, onFirstElectionTimeout.await(5, TimeUnit.SECONDS));
 
-            writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+                shard.tell(new RegisterDataTreeChangeListener(path, dclActor, false), getRef());
+                final RegisterDataTreeNotificationListenerReply reply = expectMsgClass(duration("5 seconds"),
+                        RegisterDataTreeNotificationListenerReply.class);
+                assertNotNull("getListenerRegistratioznPath", reply.getListenerRegistrationPath());
 
-            onChangeListenerRegistered.countDown();
+                shard.tell(FindLeader.INSTANCE, getRef());
+                final FindLeaderReply findLeadeReply = expectMsgClass(duration("5 seconds"), FindLeaderReply.class);
+                assertFalse("Expected the shard not to be the leader", findLeadeReply.getLeaderActor().isPresent());
 
-            // TODO: investigate why we do not receive data chage events
-            listener.waitForChangeEvents();
+                onChangeListenerRegistered.countDown();
 
-            dclActor.tell(PoisonPill.getInstance(), ActorRef.noSender());
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+                // TODO: investigate why we do not receive data chage events
+                listener.waitForChangeEvents();
+            }
+        };
     }
 
     @Test
-    public void testCreateTransaction(){
-        new ShardTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(newShardProps(), "testCreateTransaction");
-
-            waitUntilLeader(shard);
+    public void testCreateTransaction() {
+        new ShardTestKit(getSystem()) {
+            {
+                final ActorRef shard = actorFactory.createActor(newShardProps(), "testCreateTransaction");
 
-            shard.tell(new UpdateSchemaContext(TestModel.createTestContext()), getRef());
+                waitUntilLeader(shard);
 
-            shard.tell(new CreateTransaction("txn-1",
-                    TransactionType.READ_ONLY.ordinal() ).toSerializable(), getRef());
+                shard.tell(new UpdateSchemaContext(TestModel.createTestContext()), getRef());
 
-            final CreateTransactionReply reply = expectMsgClass(duration("3 seconds"),
-                    CreateTransactionReply.class);
+                shard.tell(new CreateTransaction(nextTransactionId(), TransactionType.READ_ONLY.ordinal(),
+                        DataStoreVersions.CURRENT_VERSION).toSerializable(), getRef());
 
-            final String path = reply.getTransactionActorPath().toString();
-            assertTrue("Unexpected transaction path " + path,
-                    path.contains("akka://test/user/testCreateTransaction/shard-txn-1"));
+                final CreateTransactionReply reply = expectMsgClass(duration("3 seconds"),
+                        CreateTransactionReply.class);
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+                final String path = reply.getTransactionPath().toString();
+                assertTrue("Unexpected transaction path " + path, path.startsWith(String.format(
+                        "akka://test/user/testCreateTransaction/shard-%s-%s:ShardTransactionTest@0:",
+                            shardID.getShardName(), shardID.getMemberName().getName())));
+            }
+        };
     }
 
     @Test
-    public void testCreateTransactionOnChain(){
-        new ShardTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(newShardProps(), "testCreateTransactionOnChain");
-
-            waitUntilLeader(shard);
+    public void testCreateTransactionOnChain() {
+        new ShardTestKit(getSystem()) {
+            {
+                final ActorRef shard = actorFactory.createActor(newShardProps(), "testCreateTransactionOnChain");
 
-            shard.tell(new CreateTransaction("txn-1",
-                    TransactionType.READ_ONLY.ordinal() , "foobar").toSerializable(),
-                    getRef());
+                waitUntilLeader(shard);
 
-            final CreateTransactionReply reply = expectMsgClass(duration("3 seconds"),
-                    CreateTransactionReply.class);
+                shard.tell(new CreateTransaction(nextTransactionId(), TransactionType.READ_ONLY.ordinal(),
+                        DataStoreVersions.CURRENT_VERSION).toSerializable(), getRef());
 
-            final String path = reply.getTransactionActorPath().toString();
-            assertTrue("Unexpected transaction path " + path,
-                    path.contains("akka://test/user/testCreateTransactionOnChain/shard-txn-1"));
+                final CreateTransactionReply reply = expectMsgClass(duration("3 seconds"),
+                        CreateTransactionReply.class);
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+                final String path = reply.getTransactionPath().toString();
+                assertTrue("Unexpected transaction path " + path, path.startsWith(String.format(
+                        "akka://test/user/testCreateTransactionOnChain/shard-%s-%s:ShardTransactionTest@0:",
+                        shardID.getShardName(), shardID.getMemberName().getName())));
+            }
+        };
     }
 
-    @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();
-                    }
-                }
+        new ShardTestKit(getSystem()) {
+            {
+                final 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.tell(new PeerAddressResolved(peerID.toString(), address), ActorRef.noSender());
+
+                shard.tell(GetOnDemandRaftState.INSTANCE, getRef());
+                final OnDemandRaftState state = expectMsgClass(OnDemandRaftState.class);
+                assertEquals("getPeerAddress", address, state.getPeerAddresses().get(peerID.toString()));
             }
-
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    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));
-
-            final String address = "akka://foobar";
-            shard.underlyingActor().onReceiveCommand(new PeerAddressResolved(shardID.toString(), address));
-
-            assertEquals("getPeerAddress", address,
-                ((TestShard) shard.underlyingActor()).getPeerAddress(shardID.toString()));
-
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+        };
     }
 
     @Test
     public void testApplySnapshot() throws Exception {
 
-        ShardTestKit testkit = new ShardTestKit(getSystem());
-
-        final TestActorRef<Shard> shard = TestActorRef.create(getSystem(), newShardProps(),
-                "testApplySnapshot");
+        final TestActorRef<Shard> shard = actorFactory.createTestActor(newShardProps()
+                .withDispatcher(Dispatchers.DefaultDispatcherId()), "testApplySnapshot");
 
-        testkit.waitUntilLeader(shard);
+        ShardTestKit.waitUntilLeader(shard);
 
         final DataTree store = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL);
         store.setSchemaContext(SCHEMA_CONTEXT);
 
         final ContainerNode container = ImmutableContainerNodeBuilder.create().withNodeIdentifier(
-                new YangInstanceIdentifier.NodeIdentifier(TestModel.TEST_QNAME)).
-                    withChild(ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).addChild(
+                new YangInstanceIdentifier.NodeIdentifier(TestModel.TEST_QNAME))
+                    .withChild(ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).addChild(
                         ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 1)).build()).build();
 
         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),
-                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.underlyingActor().getRaftActorSnapshotCohort().applySnapshot(snapshot.getState());
+        shard.tell(new ApplySnapshot(snapshot), ActorRef.noSender());
 
-        final NormalizedNode<?,?> actual = readStore(shard, root);
+        final Stopwatch sw = Stopwatch.createStarted();
+        while (sw.elapsed(TimeUnit.SECONDS) <= 5) {
+            Uninterruptibles.sleepUninterruptibly(75, TimeUnit.MILLISECONDS);
 
-        assertEquals("Root node", expected, actual);
+            try {
+                assertEquals("Root node", expected, readStore(shard, root));
+                return;
+            } catch (final AssertionError e) {
+                // try again
+            }
+        }
 
-        shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
+        fail("Snapshot was not applied");
     }
 
     @Test
     public void testApplyState() throws Exception {
+        final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), "testApplyState");
 
-        ShardTestKit testkit = new ShardTestKit(getSystem());
-
-        final TestActorRef<Shard> shard = TestActorRef.create(getSystem(), newShardProps(), "testApplyState");
-
-        testkit.waitUntilLeader(shard);
-
-        final NormalizedNode<?, ?> node = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
-
-        final ApplyState applyState = new ApplyState(null, "test", new ReplicatedLogImplEntry(1, 2,
-                newModificationPayload(new WriteModification(TestModel.TEST_PATH, node))));
-
-        shard.underlyingActor().onReceiveCommand(applyState);
-
-        final NormalizedNode<?,?> actual = readStore(shard, TestModel.TEST_PATH);
-        assertEquals("Applied state", node, actual);
-
-        shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-    }
-
-    @Test
-    public void testApplyStateWithCandidatePayload() throws Exception {
-
-        ShardTestKit testkit = new ShardTestKit(getSystem());
-
-        final TestActorRef<Shard> shard = TestActorRef.create(getSystem(), newShardProps(), "testApplyState");
-
-        testkit.waitUntilLeader(shard);
+        ShardTestKit.waitUntilLeader(shard);
 
-        final NormalizedNode<?, ?> node = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
-        final DataTreeCandidate candidate = DataTreeCandidates.fromNormalizedNode(TestModel.TEST_PATH, node);
-
-        final ApplyState applyState = new ApplyState(null, "test", new ReplicatedLogImplEntry(1, 2,
-                DataTreeCandidatePayload.create(candidate)));
-
-        shard.underlyingActor().onReceiveCommand(applyState);
-
-        final NormalizedNode<?,?> actual = readStore(shard, TestModel.TEST_PATH);
-        assertEquals("Applied state", node, actual);
-
-        shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-    }
+        final DataTree store = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL);
+        store.setSchemaContext(SCHEMA_CONTEXT);
 
-    DataTree setupInMemorySnapshotStore() throws DataValidationFailedException {
-        final DataTree testStore = InMemoryDataTreeFactory.getInstance().create(TreeType.OPERATIONAL);
-        testStore.setSchemaContext(SCHEMA_CONTEXT);
+        final DataTreeModification writeMod = store.takeSnapshot().newModification();
+        final ContainerNode node = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
+        writeMod.write(TestModel.TEST_PATH, node);
+        writeMod.ready();
 
-        writeToStore(testStore, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+        final TransactionIdentifier tx = nextTransactionId();
+        shard.underlyingActor().applyState(null, null, payloadForModification(store, writeMod, tx));
 
-        final NormalizedNode<?, ?> root = readStore(testStore, YangInstanceIdentifier.builder().build());
+        final Stopwatch sw = Stopwatch.createStarted();
+        while (sw.elapsed(TimeUnit.SECONDS) <= 5) {
+            Uninterruptibles.sleepUninterruptibly(75, TimeUnit.MILLISECONDS);
 
-        InMemorySnapshotStore.addSnapshot(shardID.toString(), Snapshot.create(
-                SerializationUtils.serializeNormalizedNode(root),
-                Collections.<ReplicatedLogEntry>emptyList(), 0, 1, -1, -1));
-        return testStore;
-    }
+            final NormalizedNode<?,?> actual = readStore(shard, TestModel.TEST_PATH);
+            if (actual != null) {
+                assertEquals("Applied state", node, actual);
+                return;
+            }
+        }
 
-    private static DataTreeCandidatePayload payloadForModification(final DataTree source, final DataTreeModification mod) throws DataValidationFailedException {
-        source.validate(mod);
-        final DataTreeCandidate candidate = source.prepare(mod);
-        source.commit(candidate);
-        return DataTreeCandidatePayload.create(candidate);
+        fail("State was not applied");
     }
 
     @Test
@@ -572,7 +508,8 @@ public class ShardTest extends AbstractShardTest {
         InMemoryJournal.addEntry(shardID.toString(), 0, DUMMY_DATA);
 
         // Set up the InMemoryJournal.
-        InMemoryJournal.addEntry(shardID.toString(), 1, new ReplicatedLogImplEntry(0, 1, payloadForModification(source, writeMod)));
+        InMemoryJournal.addEntry(shardID.toString(), 1, new SimpleReplicatedLogEntry(0, 1,
+            payloadForModification(source, writeMod, nextTransactionId())));
 
         final int nListEntries = 16;
         final Set<Integer> listEntryKeys = new HashSet<>();
@@ -587,8 +524,9 @@ public class ShardTest extends AbstractShardTest {
             final DataTreeModification mod = source.takeSnapshot().newModification();
             mod.merge(path, ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i));
             mod.ready();
-            InMemoryJournal.addEntry(shardID.toString(), i+1, new ReplicatedLogImplEntry(i, 1,
-                payloadForModification(source, mod)));
+
+            InMemoryJournal.addEntry(shardID.toString(), i + 1, new SimpleReplicatedLogEntry(i, 1,
+                payloadForModification(source, mod, nextTransactionId())));
         }
 
         InMemoryJournal.addEntry(shardID.toString(), nListEntries + 2,
@@ -598,1670 +536,1358 @@ public class ShardTest extends AbstractShardTest {
     }
 
     @Test
-    public void testModicationRecovery() throws Exception {
+    @SuppressWarnings("checkstyle:IllegalCatch")
+    public void testConcurrentThreePhaseCommits() throws Exception {
+        final AtomicReference<Throwable> caughtEx = new AtomicReference<>();
+        final CountDownLatch commitLatch = new CountDownLatch(2);
 
-        // Set up the InMemorySnapshotStore.
-        setupInMemorySnapshotStore();
+        final long timeoutSec = 5;
+        final FiniteDuration duration = FiniteDuration.create(timeoutSec, TimeUnit.SECONDS);
+        final Timeout timeout = new Timeout(duration);
 
-        // Set up the InMemoryJournal.
+        final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                "testConcurrentThreePhaseCommits");
 
-        InMemoryJournal.addEntry(shardID.toString(), 0, DUMMY_DATA);
+        class OnFutureComplete extends OnComplete<Object> {
+            private final Class<?> expRespType;
 
-        InMemoryJournal.addEntry(shardID.toString(), 1, new ReplicatedLogImplEntry(0, 1, newModificationPayload(
-            new WriteModification(TestModel.OUTER_LIST_PATH,
-                ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build()))));
+            OnFutureComplete(final Class<?> expRespType) {
+                this.expRespType = expRespType;
+            }
 
-        final int nListEntries = 16;
-        final Set<Integer> listEntryKeys = new HashSet<>();
+            @Override
+            public void onComplete(final Throwable error, final Object resp) {
+                if (error != null) {
+                    caughtEx.set(new AssertionError(getClass().getSimpleName() + " failure", error));
+                } else {
+                    try {
+                        assertEquals("Commit response type", expRespType, resp.getClass());
+                        onSuccess(resp);
+                    } catch (final Exception e) {
+                        caughtEx.set(e);
+                    }
+                }
+            }
 
-        // Add some ModificationPayload entries
-        for(int i = 1; i <= nListEntries; i++) {
-            listEntryKeys.add(Integer.valueOf(i));
-            final YangInstanceIdentifier path = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
-                    .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i).build();
-            final Modification mod = new MergeModification(path,
-                    ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i));
-            InMemoryJournal.addEntry(shardID.toString(), i + 1, new ReplicatedLogImplEntry(i, 1,
-                    newModificationPayload(mod)));
+            void onSuccess(final Object resp) throws Exception {
+            }
         }
 
-        InMemoryJournal.addEntry(shardID.toString(), nListEntries + 2,
-                new ApplyJournalEntries(nListEntries));
-
-        testRecovery(listEntryKeys);
-    }
+        class OnCommitFutureComplete extends OnFutureComplete {
+            OnCommitFutureComplete() {
+                super(CommitTransactionReply.class);
+            }
 
-    private static ModificationPayload newModificationPayload(final Modification... mods) throws IOException {
-        final MutableCompositeModification compMod = new MutableCompositeModification();
-        for(final Modification mod: mods) {
-            compMod.addModification(mod);
+            @Override
+            public void onComplete(final Throwable error, final Object resp) {
+                super.onComplete(error, resp);
+                commitLatch.countDown();
+            }
         }
 
-        return new ModificationPayload(compMod);
-    }
-
-    @Test
-    public void testConcurrentThreePhaseCommits() throws Throwable {
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testConcurrentThreePhaseCommits");
-
-            waitUntilLeader(shard);
-
-         // Setup 3 simulated transactions with mock cohorts backed by real cohorts.
-
-            final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
-
-            final String transactionID1 = "tx1";
-            final MutableCompositeModification modification1 = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore,
-                    TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1);
+        class OnCanCommitFutureComplete extends OnFutureComplete {
+            private final TransactionIdentifier transactionID;
 
-            final String transactionID2 = "tx2";
-            final MutableCompositeModification modification2 = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore,
-                    TestModel.OUTER_LIST_PATH,
-                    ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(),
-                    modification2);
-
-            final String transactionID3 = "tx3";
-            final MutableCompositeModification modification3 = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort3 = setupMockWriteTransaction("cohort3", dataStore,
-                    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),
-                    modification3);
-
-            final long timeoutSec = 5;
-            final FiniteDuration duration = FiniteDuration.create(timeoutSec, TimeUnit.SECONDS);
-            final Timeout timeout = new Timeout(duration);
-
-            shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
-            final ReadyTransactionReply readyReply = ReadyTransactionReply.fromSerializable(
-                    expectMsgClass(duration, ReadyTransactionReply.class));
-            assertEquals("Cohort path", shard.path().toString(), readyReply.getCohortPath());
-
-            // Send the CanCommitTransaction message for the first Tx.
-
-            shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef());
-            final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
-                    expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
-            assertEquals("Can commit", true, canCommitReply.getCanCommit());
+            OnCanCommitFutureComplete(final TransactionIdentifier transactionID) {
+                super(CanCommitTransactionReply.class);
+                this.transactionID = transactionID;
+            }
 
-            shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+            @Override
+            void onSuccess(final Object resp) throws Exception {
+                final CanCommitTransactionReply canCommitReply =
+                        CanCommitTransactionReply.fromSerializable(resp);
+                assertEquals("Can commit", true, canCommitReply.getCanCommit());
 
-            shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort3, transactionID3, modification3), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+                final Future<Object> commitFuture = Patterns.ask(shard,
+                        new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), timeout);
+                commitFuture.onComplete(new OnCommitFutureComplete(), getSystem().dispatcher());
+            }
+        }
 
-            // Send the CanCommitTransaction message for the next 2 Tx's. These should get queued and
-            // processed after the first Tx completes.
+        new ShardTestKit(getSystem()) {
+            {
+                waitUntilLeader(shard);
 
-            final Future<Object> canCommitFuture1 = Patterns.ask(shard,
-                    new CanCommitTransaction(transactionID2).toSerializable(), timeout);
+                final TransactionIdentifier transactionID1 = nextTransactionId();
+                final TransactionIdentifier transactionID2 = nextTransactionId();
+                final TransactionIdentifier transactionID3 = nextTransactionId();
+
+                final Map<TransactionIdentifier, CapturingShardDataTreeCohort> cohortMap = setupCohortDecorator(
+                        shard.underlyingActor(), transactionID1, transactionID2, transactionID3);
+                final CapturingShardDataTreeCohort cohort1 = cohortMap.get(transactionID1);
+                final CapturingShardDataTreeCohort cohort2 = cohortMap.get(transactionID2);
+                final CapturingShardDataTreeCohort cohort3 = cohortMap.get(transactionID3);
+
+                shard.tell(prepareBatchedModifications(transactionID1, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), false), getRef());
+                final ReadyTransactionReply readyReply = ReadyTransactionReply
+                        .fromSerializable(expectMsgClass(duration, ReadyTransactionReply.class));
+                assertEquals("Cohort path", shard.path().toString(), readyReply.getCohortPath());
+                // Send the CanCommitTransaction message for the first Tx.
+
+                shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply
+                        .fromSerializable(expectMsgClass(duration, CanCommitTransactionReply.class));
+                assertEquals("Can commit", true, canCommitReply.getCanCommit());
 
-            final Future<Object> canCommitFuture2 = Patterns.ask(shard,
-                    new CanCommitTransaction(transactionID3).toSerializable(), timeout);
+                // Ready 2 more Tx's.
 
-            // Send the CommitTransaction message for the first Tx. After it completes, it should
-            // trigger the 2nd Tx to proceed which should in turn then trigger the 3rd.
+                shard.tell(prepareBatchedModifications(transactionID2, TestModel.OUTER_LIST_PATH,
+                        ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), false), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            shard.tell(new CommitTransaction(transactionID1).toSerializable(), getRef());
-            expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
+                shard.tell(
+                        prepareBatchedModifications(transactionID3,
+                                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), false),
+                        getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            // Wait for the next 2 Tx's to complete.
+                // Send the CanCommitTransaction message for the next 2 Tx's.
+                // These should get queued and
+                // processed after the first Tx completes.
 
-            final AtomicReference<Throwable> caughtEx = new AtomicReference<>();
-            final CountDownLatch commitLatch = new CountDownLatch(2);
+                final Future<Object> canCommitFuture1 = Patterns.ask(shard,
+                        new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), timeout);
 
-            class OnFutureComplete extends OnComplete<Object> {
-                private final Class<?> expRespType;
+                final Future<Object> canCommitFuture2 = Patterns.ask(shard,
+                        new CanCommitTransaction(transactionID3, CURRENT_VERSION).toSerializable(), timeout);
 
-                OnFutureComplete(final Class<?> expRespType) {
-                    this.expRespType = expRespType;
-                }
+                // Send the CommitTransaction message for the first Tx. After it
+                // completes, it should
+                // trigger the 2nd Tx to proceed which should in turn then
+                // trigger the 3rd.
 
-                @Override
-                public void onComplete(final Throwable error, final Object resp) {
-                    if(error != null) {
-                        caughtEx.set(new AssertionError(getClass().getSimpleName() + " failure", error));
-                    } else {
-                        try {
-                            assertEquals("Commit response type", expRespType, resp.getClass());
-                            onSuccess(resp);
-                        } catch (final Exception e) {
-                            caughtEx.set(e);
-                        }
-                    }
-                }
+                shard.tell(new CommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, CommitTransactionReply.class);
 
-                void onSuccess(final Object resp) throws Exception {
-                }
-            }
+                // Wait for the next 2 Tx's to complete.
 
-            class OnCommitFutureComplete extends OnFutureComplete {
-                OnCommitFutureComplete() {
-                    super(CommitTransactionReply.SERIALIZABLE_CLASS);
-                }
+                canCommitFuture1.onComplete(new OnCanCommitFutureComplete(transactionID2), getSystem().dispatcher());
 
-                @Override
-                public void onComplete(final Throwable error, final Object resp) {
-                    super.onComplete(error, resp);
-                    commitLatch.countDown();
-                }
-            }
+                canCommitFuture2.onComplete(new OnCanCommitFutureComplete(transactionID3), getSystem().dispatcher());
 
-            class OnCanCommitFutureComplete extends OnFutureComplete {
-                private final String transactionID;
+                final boolean done = commitLatch.await(timeoutSec, TimeUnit.SECONDS);
 
-                OnCanCommitFutureComplete(final String transactionID) {
-                    super(CanCommitTransactionReply.SERIALIZABLE_CLASS);
-                    this.transactionID = transactionID;
+                if (caughtEx.get() != null) {
+                    Throwables.propagateIfInstanceOf(caughtEx.get(), Exception.class);
+                    Throwables.propagate(caughtEx.get());
                 }
 
-                @Override
-                void onSuccess(final Object resp) throws Exception {
-                    final CanCommitTransactionReply canCommitReply =
-                            CanCommitTransactionReply.fromSerializable(resp);
-                    assertEquals("Can commit", true, canCommitReply.getCanCommit());
+                assertEquals("Commits complete", true, done);
 
-                    final Future<Object> commitFuture = Patterns.ask(shard,
-                            new CommitTransaction(transactionID).toSerializable(), timeout);
-                    commitFuture.onComplete(new OnCommitFutureComplete(), getSystem().dispatcher());
-                }
-            }
-
-            canCommitFuture1.onComplete(new OnCanCommitFutureComplete(transactionID2),
-                    getSystem().dispatcher());
+//                final InOrder inOrder = inOrder(cohort1.getCanCommit(), cohort1.getPreCommit(), cohort1.getCommit(),
+//                        cohort2.getCanCommit(), cohort2.getPreCommit(), cohort2.getCommit(), cohort3.getCanCommit(),
+//                        cohort3.getPreCommit(), cohort3.getCommit());
+//                inOrder.verify(cohort1.getCanCommit()).onSuccess(any(Void.class));
+//                inOrder.verify(cohort1.getPreCommit()).onSuccess(any(DataTreeCandidate.class));
+//                inOrder.verify(cohort2.getCanCommit()).onSuccess(any(Void.class));
+//                inOrder.verify(cohort2.getPreCommit()).onSuccess(any(DataTreeCandidate.class));
+//                inOrder.verify(cohort3.getCanCommit()).onSuccess(any(Void.class));
+//                inOrder.verify(cohort3.getPreCommit()).onSuccess(any(DataTreeCandidate.class));
+//                inOrder.verify(cohort1.getCommit()).onSuccess(any(UnsignedLong.class));
+//                inOrder.verify(cohort2.getCommit()).onSuccess(any(UnsignedLong.class));
+//                inOrder.verify(cohort3.getCommit()).onSuccess(any(UnsignedLong.class));
 
-            canCommitFuture2.onComplete(new OnCanCommitFutureComplete(transactionID3),
-                    getSystem().dispatcher());
+                // Verify data in the data store.
 
-            final boolean done = commitLatch.await(timeoutSec, TimeUnit.SECONDS);
+                verifyOuterListEntry(shard, 1);
 
-            if(caughtEx.get() != null) {
-                throw caughtEx.get();
+                verifyLastApplied(shard, 2);
             }
-
-            assertEquals("Commits complete", true, done);
-
-            final InOrder inOrder = inOrder(cohort1, cohort2, cohort3);
-            inOrder.verify(cohort1).canCommit();
-            inOrder.verify(cohort1).preCommit();
-            inOrder.verify(cohort1).commit();
-            inOrder.verify(cohort2).canCommit();
-            inOrder.verify(cohort2).preCommit();
-            inOrder.verify(cohort2).commit();
-            inOrder.verify(cohort3).canCommit();
-            inOrder.verify(cohort3).preCommit();
-            inOrder.verify(cohort3).commit();
-
-            // Verify data in the data store.
-
-            verifyOuterListEntry(shard, 1);
-
-            verifyLastApplied(shard, 2);
-
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
-    }
-
-    private static BatchedModifications newBatchedModifications(final String transactionID, final YangInstanceIdentifier path,
-            final NormalizedNode<?, ?> data, final boolean ready, final boolean doCommitOnReady, final int messagesSent) {
-        return newBatchedModifications(transactionID, null, path, data, ready, doCommitOnReady, messagesSent);
-    }
-
-    private static BatchedModifications newBatchedModifications(final String transactionID, final String transactionChainID,
-            final YangInstanceIdentifier path, final NormalizedNode<?, ?> data, final boolean ready, final boolean doCommitOnReady,
-            final int messagesSent) {
-        final BatchedModifications batched = new BatchedModifications(transactionID, CURRENT_VERSION, transactionChainID);
-        batched.addModification(new WriteModification(path, data));
-        batched.setReady(ready);
-        batched.setDoCommitOnReady(doCommitOnReady);
-        batched.setTotalMessagesSent(messagesSent);
-        return batched;
+        };
     }
 
     @Test
-    public void testBatchedModificationsWithNoCommitOnReady() throws Throwable {
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testBatchedModificationsWithNoCommitOnReady");
-
-            waitUntilLeader(shard);
-
-            final String transactionID = "tx";
-            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();
-                }
-            };
-
-            shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator);
+    public void testBatchedModificationsWithNoCommitOnReady() throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testBatchedModificationsWithNoCommitOnReady");
 
-            // Send a BatchedModifications to start a transaction.
+                waitUntilLeader(shard);
 
-            shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH,
-                    ImmutableNodes.containerNode(TestModel.TEST_QNAME), false, false, 1), getRef());
-            expectMsgClass(duration, BatchedModificationsReply.class);
+                final TransactionIdentifier transactionID = nextTransactionId();
+                final FiniteDuration duration = duration("5 seconds");
 
-            // Send a couple more BatchedModifications.
+                // Send a BatchedModifications to start a transaction.
 
-            shard.tell(newBatchedModifications(transactionID, TestModel.OUTER_LIST_PATH,
-                    ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), false, false, 2), getRef());
-            expectMsgClass(duration, BatchedModificationsReply.class);
+                shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), false, false, 1), 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, false, 3), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+                // Send a couple more BatchedModifications.
 
-            // Send the CanCommitTransaction message.
+                shard.tell(
+                        newBatchedModifications(transactionID, TestModel.OUTER_LIST_PATH,
+                                ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), false, false, 2),
+                        getRef());
+                expectMsgClass(duration, BatchedModificationsReply.class);
 
-            shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
-            final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
-                    expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
-            assertEquals("Can commit", true, canCommitReply.getCanCommit());
+                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, false, 3),
+                        getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            // Send the CanCommitTransaction message.
+                // Send the CanCommitTransaction message.
 
-            shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef());
-            expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
+                shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
+                final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply
+                        .fromSerializable(expectMsgClass(duration, CanCommitTransactionReply.class));
+                assertEquals("Can commit", true, canCommitReply.getCanCommit());
 
-            final InOrder inOrder = inOrder(mockCohort.get());
-            inOrder.verify(mockCohort.get()).canCommit();
-            inOrder.verify(mockCohort.get()).preCommit();
-            inOrder.verify(mockCohort.get()).commit();
+                // Send the CommitTransaction message.
 
-            // Verify data in the data store.
+                shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, CommitTransactionReply.class);
 
-            verifyOuterListEntry(shard, 1);
+                // Verify data in the data store.
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+                verifyOuterListEntry(shard, 1);
+            }
+        };
     }
 
     @Test
-    public void testBatchedModificationsWithCommitOnReady() throws Throwable {
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testBatchedModificationsWithCommitOnReady");
-
-            waitUntilLeader(shard);
-
-            final String transactionID = "tx";
-            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();
-                }
-            };
-
-            shard.underlyingActor().getCommitCoordinator().setCohortDecorator(cohortDecorator);
+    public void testBatchedModificationsWithCommitOnReady() throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testBatchedModificationsWithCommitOnReady");
 
-            // Send a BatchedModifications to start a transaction.
+                waitUntilLeader(shard);
 
-            shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH,
-                    ImmutableNodes.containerNode(TestModel.TEST_QNAME), false, false, 1), getRef());
-            expectMsgClass(duration, BatchedModificationsReply.class);
+                final TransactionIdentifier transactionID = nextTransactionId();
+                final FiniteDuration duration = duration("5 seconds");
 
-            // Send a couple more BatchedModifications.
+                // Send a BatchedModifications to start a transaction.
 
-            shard.tell(newBatchedModifications(transactionID, TestModel.OUTER_LIST_PATH,
-                ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), false, false, 2), getRef());
-            expectMsgClass(duration, BatchedModificationsReply.class);
+                shard.tell(newBatchedModifications(transactionID, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), false, false, 1), 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, true, 3), getRef());
+                // Send a couple more BatchedModifications.
 
-            expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
+                shard.tell(newBatchedModifications(transactionID, TestModel.OUTER_LIST_PATH,
+                                ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), false, false, 2),
+                        getRef());
+                expectMsgClass(duration, BatchedModificationsReply.class);
 
-            final InOrder inOrder = inOrder(mockCohort.get());
-            inOrder.verify(mockCohort.get()).canCommit();
-            inOrder.verify(mockCohort.get()).preCommit();
-            inOrder.verify(mockCohort.get()).commit();
+                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, true, 3),
+                        getRef());
 
-            // Verify data in the data store.
+                expectMsgClass(duration, CommitTransactionReply.class);
 
-            verifyOuterListEntry(shard, 1);
+                // Verify data in the data store.
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+                verifyOuterListEntry(shard, 1);
+            }
+        };
     }
 
-    @Test(expected=IllegalStateException.class)
-    public void testBatchedModificationsReadyWithIncorrectTotalMessageCount() throws Throwable {
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testBatchedModificationsReadyWithIncorrectTotalMessageCount");
-
-            waitUntilLeader(shard);
+    @Test(expected = IllegalStateException.class)
+    public void testBatchedModificationsReadyWithIncorrectTotalMessageCount() throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testBatchedModificationsReadyWithIncorrectTotalMessageCount");
 
-            final String transactionID = "tx1";
-            final BatchedModifications batched = new BatchedModifications(transactionID, DataStoreVersions.CURRENT_VERSION, null);
-            batched.setReady(true);
-            batched.setTotalMessagesSent(2);
+                waitUntilLeader(shard);
 
-            shard.tell(batched, getRef());
+                final TransactionIdentifier transactionID = nextTransactionId();
+                final BatchedModifications batched = new BatchedModifications(transactionID,
+                        DataStoreVersions.CURRENT_VERSION);
+                batched.setReady(true);
+                batched.setTotalMessagesSent(2);
 
-            final Failure failure = expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
+                shard.tell(batched, getRef());
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
+                final Failure failure = expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
 
-            if(failure != null) {
-                throw failure.cause();
+                if (failure != null) {
+                    Throwables.propagateIfInstanceOf(failure.cause(), Exception.class);
+                    Throwables.propagate(failure.cause());
+                }
             }
-        }};
+        };
     }
 
     @Test
-    public void testBatchedModificationsWithOperationFailure() throws Throwable {
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testBatchedModificationsWithOperationFailure");
-
-            waitUntilLeader(shard);
-
-            // Test merge with invalid data. An exception should occur when the merge is applied. Note that
-            // write will not validate the children for performance reasons.
+    public void testBatchedModificationsWithOperationFailure() throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testBatchedModificationsWithOperationFailure");
 
-            String transactionID = "tx1";
+                waitUntilLeader(shard);
 
-            ContainerNode invalidData = ImmutableContainerNodeBuilder.create().withNodeIdentifier(
-                    new YangInstanceIdentifier.NodeIdentifier(TestModel.TEST_QNAME)).
-                        withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build();
+                // Test merge with invalid data. An exception should occur when
+                // the merge is applied. Note that
+                // write will not validate the children for performance reasons.
 
-            BatchedModifications batched = new BatchedModifications(transactionID, CURRENT_VERSION, null);
-            batched.addModification(new MergeModification(TestModel.TEST_PATH, invalidData));
-            shard.tell(batched, getRef());
-            Failure failure = expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
+                final TransactionIdentifier transactionID = nextTransactionId();
 
-            Throwable cause = failure.cause();
+                final ContainerNode invalidData = ImmutableContainerNodeBuilder.create()
+                        .withNodeIdentifier(new YangInstanceIdentifier.NodeIdentifier(TestModel.TEST_QNAME))
+                        .withChild(ImmutableNodes.leafNode(TestModel.JUNK_QNAME, "junk")).build();
 
-            batched = new BatchedModifications(transactionID, DataStoreVersions.CURRENT_VERSION, null);
-            batched.setReady(true);
-            batched.setTotalMessagesSent(2);
+                BatchedModifications batched = new BatchedModifications(transactionID, CURRENT_VERSION);
+                batched.addModification(new MergeModification(TestModel.TEST_PATH, invalidData));
+                shard.tell(batched, getRef());
+                Failure failure = expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
 
-            shard.tell(batched, getRef());
+                final Throwable cause = failure.cause();
 
-            failure = expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
-            assertEquals("Failure cause", cause, failure.cause());
+                batched = new BatchedModifications(transactionID, DataStoreVersions.CURRENT_VERSION);
+                batched.setReady(true);
+                batched.setTotalMessagesSent(2);
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
-    }
+                shard.tell(batched, getRef());
 
-    @SuppressWarnings("unchecked")
-    private static void verifyOuterListEntry(final TestActorRef<Shard> shard, final Object expIDValue) throws Exception {
-        final 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);
-        final Object entry = ((Iterable<Object>)outerList.getValue()).iterator().next();
-        assertTrue(TestModel.OUTER_LIST_QNAME.getLocalName() + " entry is not MapEntryNode",
-                entry instanceof MapEntryNode);
-        final MapEntryNode mapEntry = (MapEntryNode)entry;
-        final 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", expIDValue, idLeaf.get().getValue());
+                failure = expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
+                assertEquals("Failure cause", cause, failure.cause());
+            }
+        };
     }
 
     @Test
-    public void testBatchedModificationsOnTransactionChain() throws Throwable {
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testBatchedModificationsOnTransactionChain");
-
-            waitUntilLeader(shard);
+    public void testBatchedModificationsOnTransactionChain() throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testBatchedModificationsOnTransactionChain");
 
-            final String transactionChainID = "txChain";
-            final String transactionID1 = "tx1";
-            final String transactionID2 = "tx2";
+                waitUntilLeader(shard);
 
-            final FiniteDuration duration = duration("5 seconds");
+                final LocalHistoryIdentifier historyId = nextHistoryId();
+                final TransactionIdentifier transactionID1 = new TransactionIdentifier(historyId, 0);
+                final TransactionIdentifier transactionID2 = new TransactionIdentifier(historyId, 1);
 
-            // Send a BatchedModifications to start a chained write transaction and ready it.
+                final FiniteDuration duration = duration("5 seconds");
 
-            final ContainerNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
-            final YangInstanceIdentifier path = TestModel.TEST_PATH;
-            shard.tell(newBatchedModifications(transactionID1, transactionChainID, path,
-                    containerNode, true, false, 1), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+                // Send a BatchedModifications to start a chained write
+                // transaction and ready it.
 
-            // Create a read Tx on the same chain.
+                final ContainerNode containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
+                final YangInstanceIdentifier path = TestModel.TEST_PATH;
+                shard.tell(newBatchedModifications(transactionID1, path, containerNode, true, false, 1), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            shard.tell(new CreateTransaction(transactionID2, TransactionType.READ_ONLY.ordinal() ,
-                    transactionChainID).toSerializable(), getRef());
+                // Create a read Tx on the same chain.
 
-            final CreateTransactionReply createReply = expectMsgClass(duration("3 seconds"), CreateTransactionReply.class);
+                shard.tell(new CreateTransaction(transactionID2, TransactionType.READ_ONLY.ordinal(),
+                        DataStoreVersions.CURRENT_VERSION).toSerializable(), getRef());
 
-            getSystem().actorSelection(createReply.getTransactionActorPath()).tell(new ReadData(path), getRef());
-            final ReadDataReply readReply = expectMsgClass(duration("3 seconds"), ReadDataReply.class);
-            assertEquals("Read node", containerNode, readReply.getNormalizedNode());
+                final CreateTransactionReply createReply = expectMsgClass(duration("3 seconds"),
+                        CreateTransactionReply.class);
 
-            // Commit the write transaction.
+                getSystem().actorSelection(createReply.getTransactionPath())
+                        .tell(new ReadData(path, DataStoreVersions.CURRENT_VERSION), getRef());
+                final ReadDataReply readReply = expectMsgClass(duration("3 seconds"), ReadDataReply.class);
+                assertEquals("Read node", containerNode, readReply.getNormalizedNode());
 
-            shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef());
-            final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
-                    expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
-            assertEquals("Can commit", true, canCommitReply.getCanCommit());
+                // Commit the write transaction.
 
-            shard.tell(new CommitTransaction(transactionID1).toSerializable(), getRef());
-            expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
+                shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply
+                        .fromSerializable(expectMsgClass(duration, CanCommitTransactionReply.class));
+                assertEquals("Can commit", true, canCommitReply.getCanCommit());
 
-            // Verify data in the data store.
+                shard.tell(new CommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, CommitTransactionReply.class);
 
-            final NormalizedNode<?, ?> actualNode = readStore(shard, path);
-            assertEquals("Stored node", containerNode, actualNode);
+                // Verify data in the data store.
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+                final NormalizedNode<?, ?> actualNode = readStore(shard, path);
+                assertEquals("Stored node", containerNode, actualNode);
+            }
+        };
     }
 
     @Test
     public void testOnBatchedModificationsWhenNotLeader() {
         final AtomicBoolean overrideLeaderCalls = new AtomicBoolean();
-        new ShardTestKit(getSystem()) {{
-            final Creator<Shard> creator = new Creator<Shard>() {
-                private static final long serialVersionUID = 1L;
-
-                @Override
-                public Shard create() throws Exception {
-                    return new Shard(newShardBuilder()) {
-                        @Override
-                        protected boolean isLeader() {
-                            return overrideLeaderCalls.get() ? false : super.isLeader();
-                        }
-
-                        @Override
-                        protected ActorSelection getLeader() {
-                            return overrideLeaderCalls.get() ? getSystem().actorSelection(getRef().path()) :
-                                super.getLeader();
-                        }
-                    };
-                }
-            };
+        new ShardTestKit(getSystem()) {
+            {
+                final Creator<Shard> creator = new Creator<Shard>() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Shard create() throws Exception {
+                        return new Shard(newShardBuilder()) {
+                            @Override
+                            protected boolean isLeader() {
+                                return overrideLeaderCalls.get() ? false : super.isLeader();
+                            }
 
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    Props.create(new DelegatingShardCreator(creator)), "testOnBatchedModificationsWhenNotLeader");
+                            @Override
+                            public ActorSelection getLeader() {
+                                return overrideLeaderCalls.get() ? getSystem().actorSelection(getRef().path())
+                                        : super.getLeader();
+                            }
+                        };
+                    }
+                };
 
-            waitUntilLeader(shard);
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(Props
+                        .create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testOnBatchedModificationsWhenNotLeader");
 
-            overrideLeaderCalls.set(true);
+                waitUntilLeader(shard);
 
-            final BatchedModifications batched = new BatchedModifications("tx", DataStoreVersions.CURRENT_VERSION, "");
+                overrideLeaderCalls.set(true);
 
-            shard.tell(batched, ActorRef.noSender());
+                final BatchedModifications batched = new BatchedModifications(nextTransactionId(),
+                        DataStoreVersions.CURRENT_VERSION);
 
-            expectMsgEquals(batched);
+                shard.tell(batched, ActorRef.noSender());
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+                expectMsgEquals(batched);
+            }
+        };
     }
 
     @Test
-    public void testReadyWithImmediateCommit() throws Exception{
-        testReadyWithImmediateCommit(true);
-        testReadyWithImmediateCommit(false);
-    }
-
-    public void testReadyWithImmediateCommit(final boolean readWrite) throws Exception{
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testReadyWithImmediateCommit-" + readWrite);
-
-            waitUntilLeader(shard);
-
-            final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
-
-            final String transactionID = "tx1";
-            final MutableCompositeModification modification = new MutableCompositeModification();
-            final NormalizedNode<?, ?> containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
-            final ShardDataTreeCohort cohort = setupMockWriteTransaction("cohort", dataStore,
-                    TestModel.TEST_PATH, containerNode, modification);
-
-            final FiniteDuration duration = duration("5 seconds");
-
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification, true), getRef());
+    public void testTransactionMessagesWithNoLeader() {
+        new ShardTestKit(getSystem()) {
+            {
+                dataStoreContextBuilder.customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName())
+                        .shardHeartbeatIntervalInMillis(50).shardElectionTimeoutFactor(1);
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testTransactionMessagesWithNoLeader");
 
-            expectMsgClass(duration, ThreePhaseCommitCohortMessages.CommitTransactionReply.class);
+                waitUntilNoLeader(shard);
 
-            final InOrder inOrder = inOrder(cohort);
-            inOrder.verify(cohort).canCommit();
-            inOrder.verify(cohort).preCommit();
-            inOrder.verify(cohort).commit();
+                final TransactionIdentifier txId = nextTransactionId();
+                shard.tell(new BatchedModifications(txId, DataStoreVersions.CURRENT_VERSION), getRef());
+                Failure failure = expectMsgClass(Failure.class);
+                assertEquals("Failure cause type", NoShardLeaderException.class, failure.cause().getClass());
 
-            final NormalizedNode<?, ?> actualNode = readStore(shard, TestModel.TEST_PATH);
-            assertEquals(TestModel.TEST_QNAME.getLocalName(), containerNode, actualNode);
+                shard.tell(prepareForwardedReadyTransaction(shard, txId, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef());
+                failure = expectMsgClass(Failure.class);
+                assertEquals("Failure cause type", NoShardLeaderException.class, failure.cause().getClass());
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+                shard.tell(new ReadyLocalTransaction(txId, mock(DataTreeModification.class), true), getRef());
+                failure = expectMsgClass(Failure.class);
+                assertEquals("Failure cause type", NoShardLeaderException.class, failure.cause().getClass());
+            }
+        };
     }
 
     @Test
-    public void testReadyLocalTransactionWithImmediateCommit() throws Exception{
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testReadyLocalTransactionWithImmediateCommit");
-
-            waitUntilLeader(shard);
-
-            final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
-
-            final DataTreeModification modification = dataStore.newModification();
-
-            final ContainerNode writeData = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
-            new WriteModification(TestModel.TEST_PATH, writeData).apply(modification);
-            final MapNode mergeData = ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build();
-            new MergeModification(TestModel.OUTER_LIST_PATH, mergeData).apply(modification);
-
-            final String txId = "tx1";
-            modification.ready();
-            final ReadyLocalTransaction readyMessage = new ReadyLocalTransaction(txId, modification, true);
-
-            shard.tell(readyMessage, getRef());
-
-            expectMsgClass(CommitTransactionReply.SERIALIZABLE_CLASS);
-
-            final NormalizedNode<?, ?> actualNode = readStore(shard, TestModel.OUTER_LIST_PATH);
-            assertEquals(TestModel.OUTER_LIST_QNAME.getLocalName(), mergeData, actualNode);
-
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+    public void testReadyWithReadWriteImmediateCommit() throws Exception {
+        testReadyWithImmediateCommit(true);
     }
 
     @Test
-    public void testReadyLocalTransactionWithThreePhaseCommit() throws Exception{
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testReadyLocalTransactionWithThreePhaseCommit");
-
-            waitUntilLeader(shard);
-
-            final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
-
-            final DataTreeModification modification = dataStore.newModification();
-
-            final ContainerNode writeData = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
-            new WriteModification(TestModel.TEST_PATH, writeData).apply(modification);
-            final MapNode mergeData = ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build();
-            new MergeModification(TestModel.OUTER_LIST_PATH, mergeData).apply(modification);
-
-            final String txId = "tx1";
-                modification.ready();
-            final ReadyLocalTransaction readyMessage = new ReadyLocalTransaction(txId, modification, false);
-
-            shard.tell(readyMessage, getRef());
-
-            expectMsgClass(ReadyTransactionReply.class);
-
-            // Send the CanCommitTransaction message.
+    public void testReadyWithWriteOnlyImmediateCommit() throws Exception {
+        testReadyWithImmediateCommit(false);
+    }
 
-            shard.tell(new CanCommitTransaction(txId).toSerializable(), getRef());
-            final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
-                    expectMsgClass(CanCommitTransactionReply.SERIALIZABLE_CLASS));
-            assertEquals("Can commit", true, canCommitReply.getCanCommit());
+    private void testReadyWithImmediateCommit(final boolean readWrite) throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testReadyWithImmediateCommit-" + readWrite);
 
-            // Send the CanCommitTransaction message.
+                waitUntilLeader(shard);
 
-            shard.tell(new CommitTransaction(txId).toSerializable(), getRef());
-            expectMsgClass(CommitTransactionReply.SERIALIZABLE_CLASS);
+                final TransactionIdentifier transactionID = nextTransactionId();
+                final NormalizedNode<?, ?> containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
+                if (readWrite) {
+                    shard.tell(prepareForwardedReadyTransaction(shard, transactionID, TestModel.TEST_PATH,
+                            containerNode, true), getRef());
+                } else {
+                    shard.tell(prepareBatchedModifications(transactionID, TestModel.TEST_PATH, containerNode, true),
+                            getRef());
+                }
 
-            final NormalizedNode<?, ?> actualNode = readStore(shard, TestModel.OUTER_LIST_PATH);
-            assertEquals(TestModel.OUTER_LIST_QNAME.getLocalName(), mergeData, actualNode);
+                expectMsgClass(duration("5 seconds"), CommitTransactionReply.class);
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+                final NormalizedNode<?, ?> actualNode = readStore(shard, TestModel.TEST_PATH);
+                assertEquals(TestModel.TEST_QNAME.getLocalName(), containerNode, actualNode);
+            }
+        };
     }
 
     @Test
-    public void testCommitWithPersistenceDisabled() throws Throwable {
-        testCommitWithPersistenceDisabled(true);
-        testCommitWithPersistenceDisabled(false);
-    }
-
-    public void testCommitWithPersistenceDisabled(final boolean readWrite) throws Throwable {
-        dataStoreContextBuilder.persistent(false);
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testCommitWithPersistenceDisabled-" + readWrite);
-
-            waitUntilLeader(shard);
-
-            final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
-
-            // Setup a simulated transactions with a mock cohort.
-
-            final String transactionID = "tx";
-            final MutableCompositeModification modification = new MutableCompositeModification();
-            final NormalizedNode<?, ?> containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
-            final ShardDataTreeCohort cohort = setupMockWriteTransaction("cohort", dataStore,
-                TestModel.TEST_PATH, containerNode, modification);
-
-            final FiniteDuration duration = duration("5 seconds");
-
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+    public void testReadyLocalTransactionWithImmediateCommit() throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testReadyLocalTransactionWithImmediateCommit");
 
-            // Send the CanCommitTransaction message.
+                waitUntilLeader(shard);
 
-            shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
-            final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
-                    expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
-            assertEquals("Can commit", true, canCommitReply.getCanCommit());
+                final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
 
-            // Send the CanCommitTransaction message.
+                final DataTreeModification modification = dataStore.newModification();
 
-            shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef());
-            expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
+                final ContainerNode writeData = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
+                new WriteModification(TestModel.TEST_PATH, writeData).apply(modification);
+                final MapNode mergeData = ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build();
+                new MergeModification(TestModel.OUTER_LIST_PATH, mergeData).apply(modification);
 
-            final InOrder inOrder = inOrder(cohort);
-            inOrder.verify(cohort).canCommit();
-            inOrder.verify(cohort).preCommit();
-            inOrder.verify(cohort).commit();
+                final TransactionIdentifier txId = nextTransactionId();
+                modification.ready();
+                final ReadyLocalTransaction readyMessage = new ReadyLocalTransaction(txId, modification, true);
 
-            final NormalizedNode<?, ?> actualNode = readStore(shard, TestModel.TEST_PATH);
-            assertEquals(TestModel.TEST_QNAME.getLocalName(), containerNode, actualNode);
+                shard.tell(readyMessage, getRef());
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
-    }
+                expectMsgClass(CommitTransactionReply.class);
 
-    private static DataTreeCandidateTip mockCandidate(final String name) {
-        final DataTreeCandidateTip mockCandidate = mock(DataTreeCandidateTip.class, name);
-        final DataTreeCandidateNode mockCandidateNode = mock(DataTreeCandidateNode.class, name + "-node");
-        doReturn(ModificationType.WRITE).when(mockCandidateNode).getModificationType();
-        doReturn(Optional.of(ImmutableNodes.containerNode(CARS_QNAME))).when(mockCandidateNode).getDataAfter();
-        doReturn(YangInstanceIdentifier.builder().build()).when(mockCandidate).getRootPath();
-        doReturn(mockCandidateNode).when(mockCandidate).getRootNode();
-        return mockCandidate;
-    }
-
-    private static DataTreeCandidateTip mockUnmodifiedCandidate(final String name) {
-        final DataTreeCandidateTip mockCandidate = mock(DataTreeCandidateTip.class, name);
-        final DataTreeCandidateNode mockCandidateNode = mock(DataTreeCandidateNode.class, name + "-node");
-        doReturn(ModificationType.UNMODIFIED).when(mockCandidateNode).getModificationType();
-        doReturn(YangInstanceIdentifier.builder().build()).when(mockCandidate).getRootPath();
-        doReturn(mockCandidateNode).when(mockCandidate).getRootNode();
-        return mockCandidate;
+                final NormalizedNode<?, ?> actualNode = readStore(shard, TestModel.OUTER_LIST_PATH);
+                assertEquals(TestModel.OUTER_LIST_QNAME.getLocalName(), mergeData, actualNode);
+            }
+        };
     }
 
     @Test
-    public void testCommitWhenTransactionHasNoModifications() {
-        testCommitWhenTransactionHasNoModifications(true);
-        testCommitWhenTransactionHasNoModifications(false);
-    }
-
-    public 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
+    public void testReadyLocalTransactionWithThreePhaseCommit() throws Exception {
         new ShardTestKit(getSystem()) {
             {
-                final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
                         newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                        "testCommitWhenTransactionHasNoModifications-" + readWrite);
+                        "testReadyLocalTransactionWithThreePhaseCommit");
 
                 waitUntilLeader(shard);
 
-                final String transactionID = "tx1";
-                final MutableCompositeModification modification = new MutableCompositeModification();
-                final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.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();
-                doReturn(mockUnmodifiedCandidate("cohort1-candidate")).when(cohort).getCandidate();
+                final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
 
-                final FiniteDuration duration = duration("5 seconds");
+                final DataTreeModification modification = dataStore.newModification();
 
-                shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification), getRef());
-                expectMsgClass(duration, ReadyTransactionReply.class);
+                final ContainerNode writeData = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
+                new WriteModification(TestModel.TEST_PATH, writeData).apply(modification);
+                final MapNode mergeData = ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build();
+                new MergeModification(TestModel.OUTER_LIST_PATH, mergeData).apply(modification);
 
-                // Send the CanCommitTransaction message.
-
-                shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
-                final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
-                        expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
-                assertEquals("Can commit", true, canCommitReply.getCanCommit());
+                final TransactionIdentifier txId = nextTransactionId();
+                modification.ready();
+                final ReadyLocalTransaction readyMessage = new ReadyLocalTransaction(txId, modification, false);
 
-                shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef());
-                expectMsgClass(duration, ThreePhaseCommitCohortMessages.CommitTransactionReply.class);
+                shard.tell(readyMessage, getRef());
 
-                final InOrder inOrder = inOrder(cohort);
-                inOrder.verify(cohort).canCommit();
-                inOrder.verify(cohort).preCommit();
-                inOrder.verify(cohort).commit();
+                expectMsgClass(ReadyTransactionReply.class);
 
-                shard.tell(Shard.GET_SHARD_MBEAN_MESSAGE, getRef());
-                final ShardStats shardStats = expectMsgClass(duration, ShardStats.class);
+                // Send the CanCommitTransaction message.
 
-                // Use MBean for verification
-                // Committed transaction count should increase as usual
-                assertEquals(1,shardStats.getCommittedTransactionsCount());
+                shard.tell(new CanCommitTransaction(txId, CURRENT_VERSION).toSerializable(), getRef());
+                final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply
+                        .fromSerializable(expectMsgClass(CanCommitTransactionReply.class));
+                assertEquals("Can commit", true, canCommitReply.getCanCommit());
 
-                // Commit index should not advance because this does not go into the journal
-                assertEquals(-1, shardStats.getCommitIndex());
+                // Send the CanCommitTransaction message.
 
-                shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
+                shard.tell(new CommitTransaction(txId, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(CommitTransactionReply.class);
 
+                final NormalizedNode<?, ?> actualNode = readStore(shard, TestModel.OUTER_LIST_PATH);
+                assertEquals(TestModel.OUTER_LIST_QNAME.getLocalName(), mergeData, actualNode);
             }
         };
     }
 
     @Test
-    public void testCommitWhenTransactionHasModifications() {
-        testCommitWhenTransactionHasModifications(true);
-        testCommitWhenTransactionHasModifications(false);
-    }
-
-    public void testCommitWhenTransactionHasModifications(final boolean readWrite){
+    public void testReadWriteCommitWithPersistenceDisabled() throws Exception {
+        dataStoreContextBuilder.persistent(false);
         new ShardTestKit(getSystem()) {
             {
-                final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
                         newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                        "testCommitWhenTransactionHasModifications-" + readWrite);
+                        "testCommitWithPersistenceDisabled");
 
                 waitUntilLeader(shard);
 
-                final String transactionID = "tx1";
-                final MutableCompositeModification modification = new MutableCompositeModification();
-                modification.addModification(new DeleteModification(YangInstanceIdentifier.builder().build()));
-                final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.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();
-                doReturn(mockCandidate("cohort1-candidate")).when(cohort).getCandidate();
+                // Setup a simulated transactions with a mock cohort.
 
                 final FiniteDuration duration = duration("5 seconds");
 
-                shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification), getRef());
+                final TransactionIdentifier transactionID = nextTransactionId();
+                final NormalizedNode<?, ?> containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
+                shard.tell(prepareBatchedModifications(transactionID, TestModel.TEST_PATH, containerNode, false),
+                        getRef());
                 expectMsgClass(duration, ReadyTransactionReply.class);
 
                 // Send the CanCommitTransaction message.
 
-                shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
-                final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
-                        expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
+                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).toSerializable(), getRef());
-                expectMsgClass(duration, ThreePhaseCommitCohortMessages.CommitTransactionReply.class);
-
-                final InOrder inOrder = inOrder(cohort);
-                inOrder.verify(cohort).canCommit();
-                inOrder.verify(cohort).preCommit();
-                inOrder.verify(cohort).commit();
-
-                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 advance as we do not have an empty modification
-                assertEquals(0, shardStats.getCommitIndex());
+                // Send the CanCommitTransaction message.
 
-                shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
+                shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, CommitTransactionReply.class);
 
+                final NormalizedNode<?, ?> actualNode = readStore(shard, TestModel.TEST_PATH);
+                assertEquals(TestModel.TEST_QNAME.getLocalName(), containerNode, actualNode);
             }
         };
     }
 
     @Test
-    public void testCommitPhaseFailure() throws Throwable {
-        testCommitPhaseFailure(true);
-        testCommitPhaseFailure(false);
-    }
-
-    public void testCommitPhaseFailure(final boolean readWrite) throws Throwable {
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testCommitPhaseFailure-" + readWrite);
-
-            waitUntilLeader(shard);
-
-            // Setup 2 simulated transactions with mock cohorts. The first one fails in the
-            // commit phase.
-
-            final String transactionID1 = "tx1";
-            final MutableCompositeModification modification1 = new MutableCompositeModification();
-            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(mockCandidate("cohort1-candidate")).when(cohort1).getCandidate();
-
-            final String transactionID2 = "tx2";
-            final MutableCompositeModification modification2 = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort2 = mock(ShardDataTreeCohort.class, "cohort2");
-            doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort2).canCommit();
-
-            final FiniteDuration duration = duration("5 seconds");
-            final Timeout timeout = new Timeout(duration);
-
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
-
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
-
-            // Send the CanCommitTransaction message for the first Tx.
-
-            shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef());
-            final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
-                    expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
-            assertEquals("Can commit", true, canCommitReply.getCanCommit());
-
-            // Send the CanCommitTransaction message for the 2nd Tx. This should get queued and
-            // processed after the first Tx completes.
-
-            final Future<Object> canCommitFuture = Patterns.ask(shard,
-                    new CanCommitTransaction(transactionID2).toSerializable(), timeout);
-
-            // Send the CommitTransaction message for the first Tx. This should send back an error
-            // and trigger the 2nd Tx to proceed.
-
-            shard.tell(new CommitTransaction(transactionID1).toSerializable(), getRef());
-            expectMsgClass(duration, akka.actor.Status.Failure.class);
-
-            // Wait for the 2nd Tx to complete the canCommit phase.
-
-            final CountDownLatch latch = new CountDownLatch(1);
-            canCommitFuture.onComplete(new OnComplete<Object>() {
-                @Override
-                public void onComplete(final Throwable t, final Object resp) {
-                    latch.countDown();
-                }
-            }, getSystem().dispatcher());
-
-            assertEquals("2nd CanCommit complete", true, latch.await(5, TimeUnit.SECONDS));
-
-            final InOrder inOrder = inOrder(cohort1, cohort2);
-            inOrder.verify(cohort1).canCommit();
-            inOrder.verify(cohort1).preCommit();
-            inOrder.verify(cohort1).commit();
-            inOrder.verify(cohort2).canCommit();
-
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
-    }
-
-    @Test
-    public void testPreCommitPhaseFailure() throws Throwable {
-        testPreCommitPhaseFailure(true);
-        testPreCommitPhaseFailure(false);
-    }
-
-    public void testPreCommitPhaseFailure(final boolean readWrite) throws Throwable {
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testPreCommitPhaseFailure-" + readWrite);
-
-            waitUntilLeader(shard);
-
-            final String transactionID1 = "tx1";
-            final MutableCompositeModification modification1 = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort1 = mock(ShardDataTreeCohort.class, "cohort1");
-            doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort1).canCommit();
-            doReturn(Futures.immediateFailedFuture(new IllegalStateException("mock"))).when(cohort1).preCommit();
-
-            final String transactionID2 = "tx2";
-            final MutableCompositeModification modification2 = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort2 = mock(ShardDataTreeCohort.class, "cohort2");
-            doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort2).canCommit();
-
-            final FiniteDuration duration = duration("5 seconds");
-            final Timeout timeout = new Timeout(duration);
-
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
-
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
-
-            // Send the CanCommitTransaction message for the first Tx.
-
-            shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef());
-            final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
-                expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
-            assertEquals("Can commit", true, canCommitReply.getCanCommit());
-
-            // Send the CanCommitTransaction message for the 2nd Tx. This should get queued and
-            // processed after the first Tx completes.
-
-            final Future<Object> canCommitFuture = Patterns.ask(shard,
-                    new CanCommitTransaction(transactionID2).toSerializable(), timeout);
-
-            // Send the CommitTransaction message for the first Tx. This should send back an error
-            // and trigger the 2nd Tx to proceed.
-
-            shard.tell(new CommitTransaction(transactionID1).toSerializable(), getRef());
-            expectMsgClass(duration, akka.actor.Status.Failure.class);
-
-            // Wait for the 2nd Tx to complete the canCommit phase.
-
-            final CountDownLatch latch = new CountDownLatch(1);
-            canCommitFuture.onComplete(new OnComplete<Object>() {
-                @Override
-                public void onComplete(final Throwable t, final Object resp) {
-                    latch.countDown();
-                }
-            }, getSystem().dispatcher());
-
-            assertEquals("2nd CanCommit complete", true, latch.await(5, TimeUnit.SECONDS));
-
-            final InOrder inOrder = inOrder(cohort1, cohort2);
-            inOrder.verify(cohort1).canCommit();
-            inOrder.verify(cohort1).preCommit();
-            inOrder.verify(cohort2).canCommit();
-
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
-    }
-
-    @Test
-    public void testCanCommitPhaseFailure() throws Throwable {
-        testCanCommitPhaseFailure(true);
-        testCanCommitPhaseFailure(false);
-    }
-
-    public void testCanCommitPhaseFailure(final boolean readWrite) throws Throwable {
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testCanCommitPhaseFailure-" + readWrite);
-
-            waitUntilLeader(shard);
-
-            final FiniteDuration duration = duration("5 seconds");
-
-            final String transactionID1 = "tx1";
-            final MutableCompositeModification modification = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1");
-            doReturn(Futures.immediateFailedFuture(new IllegalStateException("mock"))).when(cohort).canCommit();
-
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID1, modification), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
-
-            // Send the CanCommitTransaction message.
-
-            shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef());
-            expectMsgClass(duration, akka.actor.Status.Failure.class);
-
-            // Send another can commit to ensure the failed one got cleaned up.
-
-            reset(cohort);
-
-            final String transactionID2 = "tx2";
-            doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit();
-
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID2, modification), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
-
-            shard.tell(new CanCommitTransaction(transactionID2).toSerializable(), getRef());
-            final CanCommitTransactionReply reply = CanCommitTransactionReply.fromSerializable(
-                    expectMsgClass(CanCommitTransactionReply.SERIALIZABLE_CLASS));
-            assertEquals("getCanCommit", true, reply.getCanCommit());
-
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+    public void testReadWriteCommitWhenTransactionHasModifications() throws Exception {
+        testCommitWhenTransactionHasModifications(true);
     }
 
     @Test
-    public void testCanCommitPhaseFalseResponse() throws Throwable {
-        testCanCommitPhaseFalseResponse(true);
-        testCanCommitPhaseFalseResponse(false);
+    public void testWriteOnlyCommitWhenTransactionHasModifications() throws Exception {
+        testCommitWhenTransactionHasModifications(false);
     }
 
-    public void testCanCommitPhaseFalseResponse(final boolean readWrite) throws Throwable {
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testCanCommitPhaseFalseResponse-" + readWrite);
-
-            waitUntilLeader(shard);
-
-            final FiniteDuration duration = duration("5 seconds");
+    private void testCommitWhenTransactionHasModifications(final boolean readWrite) throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final TipProducingDataTree dataTree = createDelegatingMockDataTree();
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardBuilder().dataTree(dataTree).props().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testCommitWhenTransactionHasModifications-" + readWrite);
 
-            final String transactionID1 = "tx1";
-            final MutableCompositeModification modification = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1");
-            doReturn(Futures.immediateFuture(Boolean.FALSE)).when(cohort).canCommit();
+                waitUntilLeader(shard);
 
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID1, modification), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+                final FiniteDuration duration = duration("5 seconds");
+                final TransactionIdentifier transactionID = nextTransactionId();
+
+                if (readWrite) {
+                    shard.tell(prepareForwardedReadyTransaction(shard, transactionID, TestModel.TEST_PATH,
+                            ImmutableNodes.containerNode(TestModel.TEST_QNAME), false), getRef());
+                } else {
+                    shard.tell(prepareBatchedModifications(transactionID, TestModel.TEST_PATH,
+                            ImmutableNodes.containerNode(TestModel.TEST_QNAME), false), getRef());
+                }
 
-            // Send the CanCommitTransaction message.
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef());
-            CanCommitTransactionReply reply = CanCommitTransactionReply.fromSerializable(
-                    expectMsgClass(CanCommitTransactionReply.SERIALIZABLE_CLASS));
-            assertEquals("getCanCommit", false, reply.getCanCommit());
+                // Send the CanCommitTransaction message.
 
-            // Send another can commit to ensure the failed one got cleaned up.
+                shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
+                final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply
+                        .fromSerializable(expectMsgClass(duration, CanCommitTransactionReply.class));
+                assertEquals("Can commit", true, canCommitReply.getCanCommit());
 
-            reset(cohort);
+                shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, CommitTransactionReply.class);
 
-            final String transactionID2 = "tx2";
-            doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit();
+                final InOrder inOrder = inOrder(dataTree);
+                inOrder.verify(dataTree).validate(any(DataTreeModification.class));
+                inOrder.verify(dataTree).prepare(any(DataTreeModification.class));
+                inOrder.verify(dataTree).commit(any(DataTreeCandidate.class));
 
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID2, modification), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+                shard.tell(Shard.GET_SHARD_MBEAN_MESSAGE, getRef());
+                final ShardStats shardStats = expectMsgClass(duration, ShardStats.class);
 
-            shard.tell(new CanCommitTransaction(transactionID2).toSerializable(), getRef());
-            reply = CanCommitTransactionReply.fromSerializable(
-                    expectMsgClass(CanCommitTransactionReply.SERIALIZABLE_CLASS));
-            assertEquals("getCanCommit", true, reply.getCanCommit());
+                // Use MBean for verification
+                // Committed transaction count should increase as usual
+                assertEquals(1, shardStats.getCommittedTransactionsCount());
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+                // Commit index should advance as we do not have an empty
+                // modification
+                assertEquals(0, shardStats.getCommitIndex());
+            }
+        };
     }
 
     @Test
-    public void testImmediateCommitWithCanCommitPhaseFailure() throws Throwable {
-        testImmediateCommitWithCanCommitPhaseFailure(true);
-        testImmediateCommitWithCanCommitPhaseFailure(false);
-    }
+    public void testCommitPhaseFailure() throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final TipProducingDataTree dataTree = createDelegatingMockDataTree();
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardBuilder().dataTree(dataTree).props().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testCommitPhaseFailure");
 
-    public void testImmediateCommitWithCanCommitPhaseFailure(final boolean readWrite) throws Throwable {
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testImmediateCommitWithCanCommitPhaseFailure-" + readWrite);
+                waitUntilLeader(shard);
 
-            waitUntilLeader(shard);
+                final FiniteDuration duration = duration("5 seconds");
+                final Timeout timeout = new Timeout(duration);
 
-            final FiniteDuration duration = duration("5 seconds");
+                // Setup 2 simulated transactions with mock cohorts. The first
+                // one fails in the
+                // commit phase.
 
-            final String transactionID1 = "tx1";
-            final MutableCompositeModification modification = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1");
-            doReturn(Futures.immediateFailedFuture(new IllegalStateException("mock"))).when(cohort).canCommit();
+                doThrow(new RuntimeException("mock commit failure")).when(dataTree)
+                        .commit(any(DataTreeCandidate.class));
 
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID1, modification, true), getRef());
+                final TransactionIdentifier transactionID1 = nextTransactionId();
+                shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            expectMsgClass(duration, akka.actor.Status.Failure.class);
+                final TransactionIdentifier transactionID2 = nextTransactionId();
+                shard.tell(newBatchedModifications(transactionID2, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            // Send another can commit to ensure the failed one got cleaned up.
+                // Send the CanCommitTransaction message for the first Tx.
 
-            reset(cohort);
+                shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply
+                        .fromSerializable(expectMsgClass(duration, CanCommitTransactionReply.class));
+                assertEquals("Can commit", true, canCommitReply.getCanCommit());
 
-            final String transactionID2 = "tx2";
-            doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit();
-            doReturn(Futures.immediateFuture(null)).when(cohort).preCommit();
-            doReturn(Futures.immediateFuture(null)).when(cohort).commit();
-            final DataTreeCandidateTip candidate = mock(DataTreeCandidateTip.class);
-            final DataTreeCandidateNode candidateRoot = mock(DataTreeCandidateNode.class);
-            doReturn(ModificationType.UNMODIFIED).when(candidateRoot).getModificationType();
-            doReturn(candidateRoot).when(candidate).getRootNode();
-            doReturn(candidate).when(cohort).getCandidate();
+                // Send the CanCommitTransaction message for the 2nd Tx. This
+                // should get queued and
+                // processed after the first Tx completes.
 
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID2, modification, true), getRef());
+                final Future<Object> canCommitFuture = Patterns.ask(shard,
+                        new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), timeout);
 
-            expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
+                // Send the CommitTransaction message for the first Tx. This
+                // should send back an error
+                // and trigger the 2nd Tx to proceed.
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
-    }
+                shard.tell(new CommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, akka.actor.Status.Failure.class);
 
-    @Test
-    public void testImmediateCommitWithCanCommitPhaseFalseResponse() throws Throwable {
-        testImmediateCommitWithCanCommitPhaseFalseResponse(true);
-        testImmediateCommitWithCanCommitPhaseFalseResponse(false);
-    }
+                // Wait for the 2nd Tx to complete the canCommit phase.
 
-    public void testImmediateCommitWithCanCommitPhaseFalseResponse(final boolean readWrite) throws Throwable {
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testImmediateCommitWithCanCommitPhaseFalseResponse-" + readWrite);
+                final CountDownLatch latch = new CountDownLatch(1);
+                canCommitFuture.onComplete(new OnComplete<Object>() {
+                    @Override
+                    public void onComplete(final Throwable failure, final Object resp) {
+                        latch.countDown();
+                    }
+                }, getSystem().dispatcher());
 
-            waitUntilLeader(shard);
+                assertEquals("2nd CanCommit complete", true, latch.await(5, TimeUnit.SECONDS));
 
-            final FiniteDuration duration = duration("5 seconds");
+                final InOrder inOrder = inOrder(dataTree);
+                inOrder.verify(dataTree).validate(any(DataTreeModification.class));
+                inOrder.verify(dataTree).prepare(any(DataTreeModification.class));
 
-            final String transactionID = "tx1";
-            final MutableCompositeModification modification = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort1");
-            doReturn(Futures.immediateFuture(Boolean.FALSE)).when(cohort).canCommit();
+                // FIXME: this invocation is done on the result of validate(). To test it, we need to make sure mock
+                //        validate performs wrapping and we capture that mock
+                // inOrder.verify(dataTree).validate(any(DataTreeModification.class));
 
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification, true), getRef());
+                inOrder.verify(dataTree).commit(any(DataTreeCandidate.class));
+            }
+        };
+    }
 
-            expectMsgClass(duration, akka.actor.Status.Failure.class);
+    @Test
+    public void testPreCommitPhaseFailure() throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final TipProducingDataTree dataTree = createDelegatingMockDataTree();
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardBuilder().dataTree(dataTree).props().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testPreCommitPhaseFailure");
 
-            // Send another can commit to ensure the failed one got cleaned up.
+                waitUntilLeader(shard);
 
-            reset(cohort);
+                final FiniteDuration duration = duration("5 seconds");
+                final Timeout timeout = new Timeout(duration);
 
-            final String transactionID2 = "tx2";
-            doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort).canCommit();
-            doReturn(Futures.immediateFuture(null)).when(cohort).preCommit();
-            doReturn(Futures.immediateFuture(null)).when(cohort).commit();
-            final DataTreeCandidateTip candidate = mock(DataTreeCandidateTip.class);
-            final DataTreeCandidateNode candidateRoot = mock(DataTreeCandidateNode.class);
-            doReturn(ModificationType.UNMODIFIED).when(candidateRoot).getModificationType();
-            doReturn(candidateRoot).when(candidate).getRootNode();
-            doReturn(candidate).when(cohort).getCandidate();
+                doThrow(new RuntimeException("mock preCommit failure")).when(dataTree)
+                        .prepare(any(DataTreeModification.class));
 
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID2, modification, true), getRef());
+                final TransactionIdentifier transactionID1 = nextTransactionId();
+                shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
+                final TransactionIdentifier transactionID2 = nextTransactionId();
+                shard.tell(newBatchedModifications(transactionID2, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
-    }
+                // Send the CanCommitTransaction message for the first Tx.
 
-    @Test
-    public void testAbortBeforeFinishCommit() throws Throwable {
-        testAbortBeforeFinishCommit(true);
-        testAbortBeforeFinishCommit(false);
-    }
+                shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply
+                        .fromSerializable(expectMsgClass(duration, CanCommitTransactionReply.class));
+                assertEquals("Can commit", true, canCommitReply.getCanCommit());
 
-    public void testAbortBeforeFinishCommit(final boolean readWrite) throws Throwable {
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testAbortBeforeFinishCommit-" + readWrite);
-
-            waitUntilLeader(shard);
-
-            final FiniteDuration duration = duration("5 seconds");
-            final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
-
-            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;
-                }
-            };
+                // Send the CanCommitTransaction message for the 2nd Tx. This
+                // should get queued and
+                // processed after the first Tx completes.
 
-            final MutableCompositeModification modification = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort = setupMockWriteTransaction("cohort1", dataStore,
-                    TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME),
-                    modification, preCommit);
+                final Future<Object> canCommitFuture = Patterns.ask(shard,
+                        new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), timeout);
 
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+                // Send the CommitTransaction message for the first Tx. This
+                // should send back an error
+                // and trigger the 2nd Tx to proceed.
 
-            shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
-            final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
-                expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
-            assertEquals("Can commit", true, canCommitReply.getCanCommit());
+                shard.tell(new CommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, akka.actor.Status.Failure.class);
 
-            shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef());
-            expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
+                // Wait for the 2nd Tx to complete the canCommit phase.
 
-            final NormalizedNode<?, ?> node = readStore(shard, TestModel.TEST_PATH);
+                final CountDownLatch latch = new CountDownLatch(1);
+                canCommitFuture.onComplete(new OnComplete<Object>() {
+                    @Override
+                    public void onComplete(final Throwable failure, final Object resp) {
+                        latch.countDown();
+                    }
+                }, getSystem().dispatcher());
 
-            // Since we're simulating an abort occurring during replication and before finish commit,
-            // the data should still get written to the in-memory store since we've gotten past
-            // canCommit and preCommit and persisted the data.
-            assertNotNull(TestModel.TEST_QNAME.getLocalName() + " not found", node);
+                assertEquals("2nd CanCommit complete", true, latch.await(5, TimeUnit.SECONDS));
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+                final InOrder inOrder = inOrder(dataTree);
+                inOrder.verify(dataTree).validate(any(DataTreeModification.class));
+                inOrder.verify(dataTree).prepare(any(DataTreeModification.class));
+                inOrder.verify(dataTree).validate(any(DataTreeModification.class));
+            }
+        };
     }
 
     @Test
-    public void testTransactionCommitTimeout() throws Throwable {
-        testTransactionCommitTimeout(true);
-        testTransactionCommitTimeout(false);
-    }
-
-    public void testTransactionCommitTimeout(final boolean readWrite) throws Throwable {
-        dataStoreContextBuilder.shardTransactionCommitTimeoutInSeconds(1);
-
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testTransactionCommitTimeout-" + readWrite);
-
-            waitUntilLeader(shard);
+    public void testCanCommitPhaseFailure() throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final TipProducingDataTree dataTree = createDelegatingMockDataTree();
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardBuilder().dataTree(dataTree).props().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testCanCommitPhaseFailure");
 
-            final FiniteDuration duration = duration("5 seconds");
+                waitUntilLeader(shard);
 
-            final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
+                final FiniteDuration duration = duration("5 seconds");
+                final TransactionIdentifier transactionID1 = nextTransactionId();
 
-            writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
-            writeToStore(shard, TestModel.OUTER_LIST_PATH,
-                    ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build());
+                doThrow(new DataValidationFailedException(YangInstanceIdentifier.EMPTY, "mock canCommit failure"))
+                        .doNothing().when(dataTree).validate(any(DataTreeModification.class));
 
-            // Create 1st Tx - will timeout
+                shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            final String transactionID1 = "tx1";
-            final MutableCompositeModification modification1 = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore,
-                    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),
-                    modification1);
+                // Send the CanCommitTransaction message.
 
-            // Create 2nd Tx
+                shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, akka.actor.Status.Failure.class);
 
-            final String transactionID2 = "tx3";
-            final MutableCompositeModification modification2 = new MutableCompositeModification();
-            final YangInstanceIdentifier listNodePath = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
-                .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2).build();
-            final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort3", dataStore,
-                    listNodePath,
-                    ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2),
-                    modification2);
+                // Send another can commit to ensure the failed one got cleaned
+                // up.
 
-            // Ready the Tx's
+                final TransactionIdentifier transactionID2 = nextTransactionId();
+                shard.tell(newBatchedModifications(transactionID2, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+                shard.tell(new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef());
+                final CanCommitTransactionReply reply = CanCommitTransactionReply
+                        .fromSerializable(expectMsgClass(CanCommitTransactionReply.class));
+                assertEquals("getCanCommit", true, reply.getCanCommit());
+            }
+        };
+    }
 
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+    @Test
+    public void testImmediateCommitWithCanCommitPhaseFailure() throws Exception {
+        testImmediateCommitWithCanCommitPhaseFailure(true);
+        testImmediateCommitWithCanCommitPhaseFailure(false);
+    }
 
-            // canCommit 1st Tx. We don't send the commit so it should timeout.
+    private void testImmediateCommitWithCanCommitPhaseFailure(final boolean readWrite) throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final TipProducingDataTree dataTree = createDelegatingMockDataTree();
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardBuilder().dataTree(dataTree).props().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testImmediateCommitWithCanCommitPhaseFailure-" + readWrite);
 
-            shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef());
-            expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS);
+                waitUntilLeader(shard);
 
-            // canCommit the 2nd Tx - it should complete after the 1st Tx times out.
+                doThrow(new DataValidationFailedException(YangInstanceIdentifier.EMPTY, "mock canCommit failure"))
+                        .doNothing().when(dataTree).validate(any(DataTreeModification.class));
 
-            shard.tell(new CanCommitTransaction(transactionID2).toSerializable(), getRef());
-            expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS);
+                final FiniteDuration duration = duration("5 seconds");
 
-            // Try to commit the 1st Tx - should fail as it's not the current Tx.
+                final TransactionIdentifier transactionID1 = nextTransactionId();
 
-            shard.tell(new CommitTransaction(transactionID1).toSerializable(), getRef());
-            expectMsgClass(duration, akka.actor.Status.Failure.class);
+                if (readWrite) {
+                    shard.tell(prepareForwardedReadyTransaction(shard, transactionID1, TestModel.TEST_PATH,
+                            ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef());
+                } else {
+                    shard.tell(prepareBatchedModifications(transactionID1, TestModel.TEST_PATH,
+                            ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef());
+                }
 
-            // Commit the 2nd Tx.
+                expectMsgClass(duration, akka.actor.Status.Failure.class);
 
-            shard.tell(new CommitTransaction(transactionID2).toSerializable(), getRef());
-            expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
+                // Send another can commit to ensure the failed one got cleaned
+                // up.
 
-            final NormalizedNode<?, ?> node = readStore(shard, listNodePath);
-            assertNotNull(listNodePath + " not found", node);
+                final TransactionIdentifier transactionID2 = nextTransactionId();
+                if (readWrite) {
+                    shard.tell(prepareForwardedReadyTransaction(shard, transactionID2, TestModel.TEST_PATH,
+                            ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef());
+                } else {
+                    shard.tell(prepareBatchedModifications(transactionID2, TestModel.TEST_PATH,
+                            ImmutableNodes.containerNode(TestModel.TEST_QNAME), true), getRef());
+                }
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+                expectMsgClass(duration, CommitTransactionReply.class);
+            }
+        };
     }
 
     @Test
-    public void testTransactionCommitQueueCapacityExceeded() throws Throwable {
-        dataStoreContextBuilder.shardTransactionCommitQueueCapacity(2);
+    public void testAbortWithCommitPending() throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final Creator<Shard> creator = () -> new Shard(newShardBuilder()) {
+                    @Override
+                    void persistPayload(final Identifier id, final Payload payload,
+                            final boolean batchHint) {
+                        // Simulate an AbortTransaction message occurring during
+                        // replication, after
+                        // persisting and before finishing the commit to the
+                        // in-memory store.
+
+                        doAbortTransaction(id, null);
+                        super.persistPayload(id, payload, batchHint);
+                    }
+                };
+
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(Props
+                        .create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testAbortWithCommitPending");
 
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testTransactionCommitQueueCapacityExceeded");
+                waitUntilLeader(shard);
 
-            waitUntilLeader(shard);
+                final FiniteDuration duration = duration("5 seconds");
 
-            final FiniteDuration duration = duration("5 seconds");
+                final TransactionIdentifier transactionID = nextTransactionId();
 
-            final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
+                shard.tell(prepareBatchedModifications(transactionID, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), false), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            final String transactionID1 = "tx1";
-            final MutableCompositeModification modification1 = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore,
-                    TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1);
+                shard.tell(new CanCommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, CanCommitTransactionReply.class);
 
-            final String transactionID2 = "tx2";
-            final MutableCompositeModification modification2 = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore,
-                    TestModel.OUTER_LIST_PATH,
-                    ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(),
-                    modification2);
+                shard.tell(new CommitTransaction(transactionID, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, CommitTransactionReply.class);
 
-            final String transactionID3 = "tx3";
-            final MutableCompositeModification modification3 = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort3 = setupMockWriteTransaction("cohort3", dataStore,
-                    TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification3);
+                final NormalizedNode<?, ?> node = readStore(shard, TestModel.TEST_PATH);
 
-            // Ready the Tx's
+                // Since we're simulating an abort occurring during replication
+                // and before finish commit,
+                // the data should still get written to the in-memory store
+                // since we've gotten past
+                // canCommit and preCommit and persisted the data.
+                assertNotNull(TestModel.TEST_QNAME.getLocalName() + " not found", node);
+            }
+        };
+    }
 
-            shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+    @Test
+    public void testTransactionCommitTimeout() throws Exception {
+        dataStoreContextBuilder.shardTransactionCommitTimeoutInSeconds(1);
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testTransactionCommitTimeout");
 
-            shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+                waitUntilLeader(shard);
 
-            // The 3rd Tx should exceed queue capacity and fail.
+                final FiniteDuration duration = duration("5 seconds");
 
-            shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort3, transactionID3, modification3), getRef());
-            expectMsgClass(duration, akka.actor.Status.Failure.class);
+                writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+                writeToStore(shard, TestModel.OUTER_LIST_PATH,
+                        ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build());
 
-            // canCommit 1st Tx.
+                // Ready 2 Tx's - the first will timeout
 
-            shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef());
-            expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS);
+                final TransactionIdentifier transactionID1 = nextTransactionId();
+                shard.tell(
+                        prepareBatchedModifications(transactionID1,
+                                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), false),
+                        getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            // canCommit the 2nd Tx - it should get queued.
+                final TransactionIdentifier transactionID2 = nextTransactionId();
+                final YangInstanceIdentifier listNodePath = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
+                        .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2).build();
+                shard.tell(
+                        prepareBatchedModifications(transactionID2, listNodePath,
+                                ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, 2), false),
+                        getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            shard.tell(new CanCommitTransaction(transactionID2).toSerializable(), getRef());
+                // canCommit 1st Tx. We don't send the commit so it should
+                // timeout.
 
-            // canCommit the 3rd Tx - should exceed queue capacity and fail.
+                shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, CanCommitTransactionReply.class);
 
-            shard.tell(new CanCommitTransaction(transactionID3).toSerializable(), getRef());
-            expectMsgClass(duration, akka.actor.Status.Failure.class);
+                // canCommit the 2nd Tx - it should complete after the 1st Tx
+                // times out.
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
-    }
+                shard.tell(new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, CanCommitTransactionReply.class);
 
-    @Test
-    public void testTransactionCommitWithPriorExpiredCohortEntries() throws Throwable {
-        dataStoreContextBuilder.shardCommitQueueExpiryTimeoutInMillis(1300).shardTransactionCommitTimeoutInSeconds(1);
+                // Try to commit the 1st Tx - should fail as it's not the
+                // current Tx.
 
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testTransactionCommitWithPriorExpiredCohortEntries");
+                shard.tell(new CommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, akka.actor.Status.Failure.class);
 
-            waitUntilLeader(shard);
+                // Commit the 2nd Tx.
 
-            final FiniteDuration duration = duration("5 seconds");
+                shard.tell(new CommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, CommitTransactionReply.class);
 
-            final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
+                final NormalizedNode<?, ?> node = readStore(shard, listNodePath);
+                assertNotNull(listNodePath + " not found", node);
+            }
+        };
+    }
 
-            final String transactionID1 = "tx1";
-            final MutableCompositeModification modification1 = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore,
-                    TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1);
+//    @Test
+//    @Ignore
+//    public void testTransactionCommitQueueCapacityExceeded() throws Throwable {
+//        dataStoreContextBuilder.shardTransactionCommitQueueCapacity(2);
+//
+//        new ShardTestKit(getSystem()) {{
+//            final TestActorRef<Shard> shard = actorFactory.createTestActor(
+//                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+//                    "testTransactionCommitQueueCapacityExceeded");
+//
+//            waitUntilLeader(shard);
+//
+//            final FiniteDuration duration = duration("5 seconds");
+//
+//            final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
+//
+//            final TransactionIdentifier transactionID1 = nextTransactionId();
+//            final MutableCompositeModification modification1 = new MutableCompositeModification();
+//            final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore,
+//                    TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), transactionID1,
+//                    modification1);
+//
+//            final TransactionIdentifier transactionID2 = nextTransactionId();
+//            final MutableCompositeModification modification2 = new MutableCompositeModification();
+//            final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore,
+//                    TestModel.OUTER_LIST_PATH,
+//                    ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), transactionID2,
+//                    modification2);
+//
+//            final TransactionIdentifier transactionID3 = nextTransactionId();
+//            final MutableCompositeModification modification3 = new MutableCompositeModification();
+//            final ShardDataTreeCohort cohort3 = setupMockWriteTransaction("cohort3", dataStore,
+//                    TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), transactionID3,
+//                    modification3);
+//
+//            // Ready the Tx's
+//
+//            shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1,
+//                    modification1), getRef());
+//            expectMsgClass(duration, ReadyTransactionReply.class);
+//
+//            shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2,
+//                    modification2), getRef());
+//            expectMsgClass(duration, ReadyTransactionReply.class);
+//
+//            // The 3rd Tx should exceed queue capacity and fail.
+//
+//            shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort3, transactionID3,
+//                    modification3), getRef());
+//            expectMsgClass(duration, akka.actor.Status.Failure.class);
+//
+//            // canCommit 1st Tx.
+//
+//            shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+//            expectMsgClass(duration, CanCommitTransactionReply.class);
+//
+//            // canCommit the 2nd Tx - it should get queued.
+//
+//            shard.tell(new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef());
+//
+//            // canCommit the 3rd Tx - should exceed queue capacity and fail.
+//
+//            shard.tell(new CanCommitTransaction(transactionID3, CURRENT_VERSION).toSerializable(), getRef());
+//            expectMsgClass(duration, akka.actor.Status.Failure.class);
+//        }};
+//    }
 
-            shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+    @Test
+    public void testTransactionCommitWithPriorExpiredCohortEntries() throws Exception {
+        dataStoreContextBuilder.shardTransactionCommitTimeoutInSeconds(1);
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testTransactionCommitWithPriorExpiredCohortEntries");
 
-            final String transactionID2 = "tx2";
-            final MutableCompositeModification modification2 = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore,
-                    TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification2);
+                waitUntilLeader(shard);
 
-            shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+                final FiniteDuration duration = duration("5 seconds");
 
-            final String transactionID3 = "tx3";
-            final MutableCompositeModification modification3 = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort3 = setupMockWriteTransaction("cohort3", dataStore,
-                    TestModel.TEST2_PATH, ImmutableNodes.containerNode(TestModel.TEST2_QNAME), modification3);
+                final TransactionIdentifier transactionID1 = nextTransactionId();
+                shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort3, transactionID3, modification3), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+                final TransactionIdentifier transactionID2 = nextTransactionId();
+                shard.tell(newBatchedModifications(transactionID2, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            // All Tx's are readied. We'll send canCommit for the last one but not the others. The others
-            // should expire from the queue and the last one should be processed.
+                final TransactionIdentifier transactionID3 = nextTransactionId();
+                shard.tell(newBatchedModifications(transactionID3, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            shard.tell(new CanCommitTransaction(transactionID3).toSerializable(), getRef());
-            expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS);
+                // All Tx's are readied. We'll send canCommit for the last one
+                // but not the others. The others
+                // should expire from the queue and the last one should be
+                // processed.
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+                shard.tell(new CanCommitTransaction(transactionID3, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, CanCommitTransactionReply.class);
+            }
+        };
     }
 
     @Test
-    public void testTransactionCommitWithSubsequentExpiredCohortEntry() throws Throwable {
-        dataStoreContextBuilder.shardCommitQueueExpiryTimeoutInMillis(1300).shardTransactionCommitTimeoutInSeconds(1);
-
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testTransactionCommitWithSubsequentExpiredCohortEntry");
-
-            waitUntilLeader(shard);
-
-            final FiniteDuration duration = duration("5 seconds");
+    public void testTransactionCommitWithSubsequentExpiredCohortEntry() throws Exception {
+        dataStoreContextBuilder.shardTransactionCommitTimeoutInSeconds(1);
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testTransactionCommitWithSubsequentExpiredCohortEntry");
 
-            final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
+                waitUntilLeader(shard);
 
-            final String transactionID1 = "tx1";
-            final MutableCompositeModification modification1 = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort1 = setupMockWriteTransaction("cohort1", dataStore,
-                    TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification1);
+                final FiniteDuration duration = duration("5 seconds");
 
-            shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+                final ShardDataTree dataStore = shard.underlyingActor().getDataStore();
 
-            // CanCommit the first one so it's the current in-progress CohortEntry.
+                final TransactionIdentifier transactionID1 = nextTransactionId();
+                shard.tell(prepareBatchedModifications(transactionID1, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), false), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef());
-            expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS);
+                // CanCommit the first Tx so it's the current in-progress Tx.
 
-            // Ready the second Tx.
+                shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, CanCommitTransactionReply.class);
 
-            final String transactionID2 = "tx2";
-            final MutableCompositeModification modification2 = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort2 = setupMockWriteTransaction("cohort2", dataStore,
-                    TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME), modification2);
+                // Ready the second Tx.
 
-            shard.tell(prepareReadyTransactionMessage(false, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+                final TransactionIdentifier transactionID2 = nextTransactionId();
+                shard.tell(prepareBatchedModifications(transactionID2, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), false), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            // Ready the third Tx.
+                // Ready the third Tx.
 
-            final String transactionID3 = "tx3";
-            final DataTreeModification modification3 = dataStore.newModification();
-            new WriteModification(TestModel.TEST2_PATH, ImmutableNodes.containerNode(TestModel.TEST2_QNAME))
-                    .apply(modification3);
+                final TransactionIdentifier transactionID3 = nextTransactionId();
+                final DataTreeModification modification3 = dataStore.newModification();
+                new WriteModification(TestModel.TEST2_PATH, ImmutableNodes.containerNode(TestModel.TEST2_QNAME))
+                        .apply(modification3);
                 modification3.ready();
-            final ReadyLocalTransaction readyMessage = new ReadyLocalTransaction(transactionID3, modification3, true);
-
-            shard.tell(readyMessage, getRef());
+                final ReadyLocalTransaction readyMessage = new ReadyLocalTransaction(transactionID3, modification3,
+                        true);
+                shard.tell(readyMessage, getRef());
 
-            // Commit the first Tx. After completing, the second should expire from the queue and the third
-            // Tx committed.
+                // Commit the first Tx. After completing, the second should
+                // expire from the queue and the third
+                // Tx committed.
 
-            shard.tell(new CommitTransaction(transactionID1).toSerializable(), getRef());
-            expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
+                shard.tell(new CommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, CommitTransactionReply.class);
 
-            // Expect commit reply from the third Tx.
+                // Expect commit reply from the third Tx.
 
-            expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
+                expectMsgClass(duration, CommitTransactionReply.class);
 
-            final NormalizedNode<?, ?> node = readStore(shard, TestModel.TEST2_PATH);
-            assertNotNull(TestModel.TEST2_PATH + " not found", node);
-
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+                final NormalizedNode<?, ?> node = readStore(shard, TestModel.TEST2_PATH);
+                assertNotNull(TestModel.TEST2_PATH + " not found", node);
+            }
+        };
     }
 
     @Test
-    public void testCanCommitBeforeReadyFailure() throws Throwable {
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testCanCommitBeforeReadyFailure");
-
-            shard.tell(new CanCommitTransaction("tx").toSerializable(), getRef());
-            expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
+    public void testCanCommitBeforeReadyFailure() throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        "testCanCommitBeforeReadyFailure");
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+                shard.tell(new CanCommitTransaction(nextTransactionId(), CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
+            }
+        };
     }
 
     @Test
-    public void testAbortCurrentTransaction() throws Throwable {
-        testAbortCurrentTransaction(true);
-        testAbortCurrentTransaction(false);
-    }
+    public void testAbortAfterCanCommit() throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), "testAbortAfterCanCommit");
 
-    public void testAbortCurrentTransaction(final boolean readWrite) throws Throwable {
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    "testAbortCurrentTransaction-" + readWrite);
+                waitUntilLeader(shard);
 
-            waitUntilLeader(shard);
+                final FiniteDuration duration = duration("5 seconds");
+                final Timeout timeout = new Timeout(duration);
 
-            // Setup 2 simulated transactions with mock cohorts. The first one will be aborted.
+                // Ready 2 transactions - the first one will be aborted.
 
-            final String transactionID1 = "tx1";
-            final MutableCompositeModification modification1 = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort1 = mock(ShardDataTreeCohort.class, "cohort1");
-            doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort1).canCommit();
-            doReturn(Futures.immediateFuture(null)).when(cohort1).abort();
+                final TransactionIdentifier transactionID1 = nextTransactionId();
+                shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            final String transactionID2 = "tx2";
-            final MutableCompositeModification modification2 = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort2 = mock(ShardDataTreeCohort.class, "cohort2");
-            doReturn(Futures.immediateFuture(Boolean.TRUE)).when(cohort2).canCommit();
+                final TransactionIdentifier transactionID2 = nextTransactionId();
+                shard.tell(newBatchedModifications(transactionID2, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            final FiniteDuration duration = duration("5 seconds");
-            final Timeout timeout = new Timeout(duration);
+                // Send the CanCommitTransaction message for the first Tx.
 
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort1, transactionID1, modification1), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+                shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                CanCommitTransactionReply canCommitReply = CanCommitTransactionReply
+                        .fromSerializable(expectMsgClass(duration, CanCommitTransactionReply.class));
+                assertEquals("Can commit", true, canCommitReply.getCanCommit());
 
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort2, transactionID2, modification2), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+                // Send the CanCommitTransaction message for the 2nd Tx. This
+                // should get queued and
+                // processed after the first Tx completes.
 
-            // Send the CanCommitTransaction message for the first Tx.
+                final Future<Object> canCommitFuture = Patterns.ask(shard,
+                        new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), timeout);
 
-            shard.tell(new CanCommitTransaction(transactionID1).toSerializable(), getRef());
-            final CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
-                    expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
-            assertEquals("Can commit", true, canCommitReply.getCanCommit());
+                // Send the AbortTransaction message for the first Tx. This
+                // should trigger the 2nd
+                // Tx to proceed.
 
-            // Send the CanCommitTransaction message for the 2nd Tx. This should get queued and
-            // processed after the first Tx completes.
+                shard.tell(new AbortTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, AbortTransactionReply.class);
 
-            final Future<Object> canCommitFuture = Patterns.ask(shard,
-                    new CanCommitTransaction(transactionID2).toSerializable(), timeout);
+                // Wait for the 2nd Tx to complete the canCommit phase.
 
-            // Send the AbortTransaction message for the first Tx. This should trigger the 2nd
-            // Tx to proceed.
+                canCommitReply = (CanCommitTransactionReply) Await.result(canCommitFuture, duration);
+                assertEquals("Can commit", true, canCommitReply.getCanCommit());
+            }
+        };
+    }
 
-            shard.tell(new AbortTransaction(transactionID1).toSerializable(), getRef());
-            expectMsgClass(duration, AbortTransactionReply.SERIALIZABLE_CLASS);
+    @Test
+    public void testAbortAfterReady() throws Exception {
+        dataStoreContextBuilder.shardTransactionCommitTimeoutInSeconds(1);
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), "testAbortAfterReady");
 
-            // Wait for the 2nd Tx to complete the canCommit phase.
+                waitUntilLeader(shard);
 
-            Await.ready(canCommitFuture, duration);
+                final FiniteDuration duration = duration("5 seconds");
 
-            final InOrder inOrder = inOrder(cohort1, cohort2);
-            inOrder.verify(cohort1).canCommit();
-            inOrder.verify(cohort2).canCommit();
+                // Ready a tx.
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
-    }
+                final TransactionIdentifier transactionID1 = nextTransactionId();
+                shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-    @Test
-    public void testAbortQueuedTransaction() throws Throwable {
-        testAbortQueuedTransaction(true);
-        testAbortQueuedTransaction(false);
-    }
+                // Send the AbortTransaction message.
 
-    public void testAbortQueuedTransaction(final boolean readWrite) throws Throwable {
-        dataStoreContextBuilder.shardTransactionCommitTimeoutInSeconds(1);
-        new ShardTestKit(getSystem()) {{
-            final AtomicReference<CountDownLatch> cleaupCheckLatch = new AtomicReference<>();
-            @SuppressWarnings("serial")
-            final Creator<Shard> creator = new Creator<Shard>() {
-                @Override
-                public Shard create() throws Exception {
-                    return new Shard(newShardBuilder()) {
-                        @Override
-                        public void onReceiveCommand(final Object message) throws Exception {
-                            super.onReceiveCommand(message);
-                            if(message.equals(TX_COMMIT_TIMEOUT_CHECK_MESSAGE)) {
-                                if(cleaupCheckLatch.get() != null) {
-                                    cleaupCheckLatch.get().countDown();
-                                }
-                            }
-                        }
-                    };
-                }
-            };
+                shard.tell(new AbortTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, AbortTransactionReply.class);
 
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    Props.create(new DelegatingShardCreator(creator)).withDispatcher(
-                            Dispatchers.DefaultDispatcherId()), "testAbortQueuedTransaction-" + readWrite);
+                assertEquals("getPendingTxCommitQueueSize", 0, shard.underlyingActor().getPendingTxCommitQueueSize());
 
-            waitUntilLeader(shard);
+                // Now send CanCommitTransaction - should fail.
 
-            final String transactionID = "tx1";
+                shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                final Throwable failure = expectMsgClass(duration, akka.actor.Status.Failure.class).cause();
+                assertTrue("Failure type", failure instanceof IllegalStateException);
 
-            final MutableCompositeModification modification = new MutableCompositeModification();
-            final ShardDataTreeCohort cohort = mock(ShardDataTreeCohort.class, "cohort");
-            doReturn(Futures.immediateFuture(null)).when(cohort).abort();
+                // Ready and CanCommit another and verify success.
 
-            final FiniteDuration duration = duration("5 seconds");
+                final TransactionIdentifier transactionID2 = nextTransactionId();
+                shard.tell(newBatchedModifications(transactionID2, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            // Ready the tx.
+                shard.tell(new CanCommitTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, CanCommitTransactionReply.class);
+            }
+        };
+    }
 
-            shard.tell(prepareReadyTransactionMessage(readWrite, shard.underlyingActor(), cohort, transactionID, modification), getRef());
-            expectMsgClass(duration, ReadyTransactionReply.class);
+    @Test
+    public void testAbortQueuedTransaction() throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()), "testAbortAfterReady");
 
-            assertEquals("getPendingTxCommitQueueSize", 1, shard.underlyingActor().getPendingTxCommitQueueSize());
+                waitUntilLeader(shard);
 
-            // Send the AbortTransaction message.
+                final FiniteDuration duration = duration("5 seconds");
 
-            shard.tell(new AbortTransaction(transactionID).toSerializable(), getRef());
-            expectMsgClass(duration, AbortTransactionReply.SERIALIZABLE_CLASS);
+                // Ready 3 tx's.
 
-            verify(cohort).abort();
+                final TransactionIdentifier transactionID1 = nextTransactionId();
+                shard.tell(newBatchedModifications(transactionID1, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            // Verify the tx cohort is removed from queue at the cleanup check interval.
+                final TransactionIdentifier transactionID2 = nextTransactionId();
+                shard.tell(newBatchedModifications(transactionID2, TestModel.TEST_PATH,
+                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), true, false, 1), getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            cleaupCheckLatch.set(new CountDownLatch(1));
-            assertEquals("TX_COMMIT_TIMEOUT_CHECK_MESSAGE received", true,
-                    cleaupCheckLatch.get().await(5, TimeUnit.SECONDS));
+                final TransactionIdentifier transactionID3 = nextTransactionId();
+                shard.tell(
+                        newBatchedModifications(transactionID3, TestModel.OUTER_LIST_PATH,
+                                ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(), true, false, 1),
+                        getRef());
+                expectMsgClass(duration, ReadyTransactionReply.class);
 
-            assertEquals("getPendingTxCommitQueueSize", 0, shard.underlyingActor().getPendingTxCommitQueueSize());
+                // Abort the second tx while it's queued.
 
-            // Now send CanCommitTransaction - should fail.
+                shard.tell(new AbortTransaction(transactionID2, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, AbortTransactionReply.class);
 
-            shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
+                // Commit the other 2.
 
-            Throwable failure = expectMsgClass(duration, akka.actor.Status.Failure.class).cause();
-            assertTrue("Failure type", failure instanceof IllegalStateException);
+                shard.tell(new CanCommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, CanCommitTransactionReply.class);
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
-    }
+                shard.tell(new CommitTransaction(transactionID1, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, CommitTransactionReply.class);
 
-    @Test
-    public void testCreateSnapshot() throws Exception {
-        testCreateSnapshot(true, "testCreateSnapshot");
+                shard.tell(new CanCommitTransaction(transactionID3, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, CanCommitTransactionReply.class);
+
+                shard.tell(new CommitTransaction(transactionID3, CURRENT_VERSION).toSerializable(), getRef());
+                expectMsgClass(duration, CommitTransactionReply.class);
+
+                assertEquals("getPendingTxCommitQueueSize", 0, shard.underlyingActor().getPendingTxCommitQueueSize());
+            }
+        };
     }
 
     @Test
@@ -2269,9 +1895,12 @@ public class ShardTest extends AbstractShardTest {
         testCreateSnapshot(false, "testCreateSnapshotWithNonPersistentData");
     }
 
-    @SuppressWarnings("serial")
-    public void testCreateSnapshot(final boolean persistent, final String shardActorName) throws Exception{
+    @Test
+    public void testCreateSnapshot() throws Exception {
+        testCreateSnapshot(true, "testCreateSnapshot");
+    }
 
+    private void testCreateSnapshot(final boolean persistent, final String shardActorName) throws Exception {
         final AtomicReference<CountDownLatch> latch = new AtomicReference<>(new CountDownLatch(1));
 
         final AtomicReference<Object> savedSnapshot = new AtomicReference<>();
@@ -2281,90 +1910,82 @@ public class ShardTest extends AbstractShardTest {
             }
 
             @Override
-            public void saveSnapshot(final Object o) {
-                savedSnapshot.set(o);
-                super.saveSnapshot(o);
+            public void saveSnapshot(final Object obj) {
+                savedSnapshot.set(obj);
+                super.saveSnapshot(obj);
             }
         }
 
         dataStoreContextBuilder.persistent(persistent);
 
-        new ShardTestKit(getSystem()) {{
-            class TestShard extends Shard {
-
-                protected TestShard(AbstractBuilder<?, ?> builder) {
-                    super(builder);
-                    setPersistence(new TestPersistentDataProvider(super.persistence()));
-                }
+        class TestShard extends Shard {
 
-                @Override
-                public void handleCommand(final Object message) {
-                    super.handleCommand(message);
+            protected TestShard(final AbstractBuilder<?, ?> builder) {
+                super(builder);
+                setPersistence(new TestPersistentDataProvider(super.persistence()));
+            }
 
-                    if (message instanceof SaveSnapshotSuccess || message.equals("commit_snapshot")) {
-                        latch.get().countDown();
-                    }
-                }
+            @Override
+            public void handleCommand(final Object message) {
+                super.handleCommand(message);
 
-                @Override
-                public RaftActorContext getRaftActorContext() {
-                    return super.getRaftActorContext();
+                // XXX:  commit_snapshot equality check references RaftActorSnapshotMessageSupport.COMMIT_SNAPSHOT
+                if (message instanceof SaveSnapshotSuccess || "commit_snapshot".equals(message.toString())) {
+                    latch.get().countDown();
                 }
             }
 
-            final Creator<Shard> creator = new Creator<Shard>() {
-                @Override
-                public Shard create() throws Exception {
-                    return new TestShard(newShardBuilder());
-                }
-            };
+            @Override
+            public RaftActorContext getRaftActorContext() {
+                return super.getRaftActorContext();
+            }
+        }
 
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
-                    Props.create(new DelegatingShardCreator(creator)), shardActorName);
+        new ShardTestKit(getSystem()) {
+            {
+                final Creator<Shard> creator = () -> new TestShard(newShardBuilder());
 
-            waitUntilLeader(shard);
-            writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(Props
+                        .create(new DelegatingShardCreator(creator)).withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        shardActorName);
 
-            final NormalizedNode<?,?> expectedRoot = readStore(shard, YangInstanceIdentifier.builder().build());
+                waitUntilLeader(shard);
+                writeToStore(shard, TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
 
-            // Trigger creation of a snapshot by ensuring
-            final RaftActorContext raftActorContext = ((TestShard) shard.underlyingActor()).getRaftActorContext();
-            raftActorContext.getSnapshotManager().capture(mock(ReplicatedLogEntry.class), -1);
-            awaitAndValidateSnapshot(expectedRoot);
+                final NormalizedNode<?, ?> expectedRoot = readStore(shard, YangInstanceIdentifier.EMPTY);
 
-            raftActorContext.getSnapshotManager().capture(mock(ReplicatedLogEntry.class), -1);
-            awaitAndValidateSnapshot(expectedRoot);
+                // Trigger creation of a snapshot by ensuring
+                final RaftActorContext raftActorContext = ((TestShard) shard.underlyingActor()).getRaftActorContext();
+                raftActorContext.getSnapshotManager().capture(mock(ReplicatedLogEntry.class), -1);
+                awaitAndValidateSnapshot(expectedRoot);
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }
+                raftActorContext.getSnapshotManager().capture(mock(ReplicatedLogEntry.class), -1);
+                awaitAndValidateSnapshot(expectedRoot);
+            }
 
-            private void awaitAndValidateSnapshot(NormalizedNode<?,?> expectedRoot
-                                              ) throws InterruptedException {
-                System.out.println("Inside awaitAndValidateSnapshot {}" + savedSnapshot.get());
+            private void awaitAndValidateSnapshot(final NormalizedNode<?, ?> expectedRoot)
+                    throws InterruptedException, IOException {
                 assertEquals("Snapshot saved", true, latch.get().await(5, TimeUnit.SECONDS));
 
-                assertTrue("Invalid saved snapshot " + savedSnapshot.get(),
-                        savedSnapshot.get() instanceof Snapshot);
+                assertTrue("Invalid saved snapshot " + savedSnapshot.get(), savedSnapshot.get() instanceof Snapshot);
 
-                verifySnapshot((Snapshot)savedSnapshot.get(), expectedRoot);
+                verifySnapshot((Snapshot) savedSnapshot.get(), expectedRoot);
 
                 latch.set(new CountDownLatch(1));
                 savedSnapshot.set(null);
             }
 
-            private void verifySnapshot(final Snapshot snapshot, final NormalizedNode<?,?> expectedRoot) {
-
-                final NormalizedNode<?, ?> actual = SerializationUtils.deserializeNormalizedNode(snapshot.getState());
+            private void verifySnapshot(final Snapshot snapshot, final NormalizedNode<?, ?> expectedRoot)
+                    throws IOException {
+                final NormalizedNode<?, ?> actual = ((ShardSnapshotState)snapshot.getState()).getSnapshot()
+                        .getRootNode().get();
                 assertEquals("Root node", expectedRoot, actual);
-
-           }
+            }
         };
     }
 
     /**
-     * This test simply verifies that the applySnapShot logic will work
-     * @throws ReadFailedException
-     * @throws DataValidationFailedException
+     * This test simply verifies that the applySnapShot logic will work.
      */
     @Test
     public void testInMemoryDataTreeRestore() throws ReadFailedException, DataValidationFailedException {
@@ -2377,85 +1998,80 @@ 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);
     }
 
     @Test
-    public void testRecoveryApplicable(){
-
-        final DatastoreContext persistentContext = DatastoreContext.newBuilder().
-                shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).persistent(true).build();
-
-        final Props persistentProps = Shard.builder().id(shardID).datastoreContext(persistentContext).
-                schemaContext(SCHEMA_CONTEXT).props();
-
-        final DatastoreContext nonPersistentContext = DatastoreContext.newBuilder().
-                shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).persistent(false).build();
+    public void testRecoveryApplicable() {
 
-        final Props nonPersistentProps = Shard.builder().id(shardID).datastoreContext(nonPersistentContext).
-                schemaContext(SCHEMA_CONTEXT).props();
+        final DatastoreContext persistentContext = DatastoreContext.newBuilder()
+                .shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).persistent(true).build();
 
-        new ShardTestKit(getSystem()) {{
-            final TestActorRef<Shard> shard1 = TestActorRef.create(getSystem(),
-                    persistentProps, "testPersistence1");
+        final Props persistentProps = Shard.builder().id(shardID).datastoreContext(persistentContext)
+                .schemaContext(SCHEMA_CONTEXT).props();
 
-            assertTrue("Recovery Applicable", shard1.underlyingActor().persistence().isRecoveryApplicable());
+        final DatastoreContext nonPersistentContext = DatastoreContext.newBuilder()
+                .shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).persistent(false).build();
 
-            shard1.tell(PoisonPill.getInstance(), ActorRef.noSender());
+        final Props nonPersistentProps = Shard.builder().id(shardID).datastoreContext(nonPersistentContext)
+                .schemaContext(SCHEMA_CONTEXT).props();
 
-            final TestActorRef<Shard> shard2 = TestActorRef.create(getSystem(),
-                    nonPersistentProps, "testPersistence2");
-
-            assertFalse("Recovery Not Applicable", shard2.underlyingActor().persistence().isRecoveryApplicable());
+        new ShardTestKit(getSystem()) {
+            {
+                final TestActorRef<Shard> shard1 = actorFactory.createTestActor(persistentProps, "testPersistence1");
 
-            shard2.tell(PoisonPill.getInstance(), ActorRef.noSender());
+                assertTrue("Recovery Applicable", shard1.underlyingActor().persistence().isRecoveryApplicable());
 
-        }};
+                final TestActorRef<Shard> shard2 = actorFactory.createTestActor(nonPersistentProps, "testPersistence2");
 
+                assertFalse("Recovery Not Applicable", shard2.underlyingActor().persistence().isRecoveryApplicable());
+            }
+        };
     }
 
     @Test
     public void testOnDatastoreContext() {
-        new ShardTestKit(getSystem()) {{
-            dataStoreContextBuilder.persistent(true);
-
-            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(), newShardProps(), "testOnDatastoreContext");
+        new ShardTestKit(getSystem()) {
+            {
+                dataStoreContextBuilder.persistent(true);
 
-            assertEquals("isRecoveryApplicable", true,
-                    shard.underlyingActor().persistence().isRecoveryApplicable());
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(newShardProps(),
+                        "testOnDatastoreContext");
 
-            waitUntilLeader(shard);
+                assertEquals("isRecoveryApplicable", true,
+                        shard.underlyingActor().persistence().isRecoveryApplicable());
 
-            shard.tell(dataStoreContextBuilder.persistent(false).build(), ActorRef.noSender());
+                waitUntilLeader(shard);
 
-            assertEquals("isRecoveryApplicable", false,
-                shard.underlyingActor().persistence().isRecoveryApplicable());
+                shard.tell(dataStoreContextBuilder.persistent(false).build(), ActorRef.noSender());
 
-            shard.tell(dataStoreContextBuilder.persistent(true).build(), ActorRef.noSender());
+                assertEquals("isRecoveryApplicable", false,
+                        shard.underlyingActor().persistence().isRecoveryApplicable());
 
-            assertEquals("isRecoveryApplicable", true,
-                shard.underlyingActor().persistence().isRecoveryApplicable());
+                shard.tell(dataStoreContextBuilder.persistent(true).build(), ActorRef.noSender());
 
-            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-        }};
+                assertEquals("isRecoveryApplicable", true,
+                        shard.underlyingActor().persistence().isRecoveryApplicable());
+            }
+        };
     }
 
     @Test
     public void testRegisterRoleChangeListener() throws Exception {
         new ShardTestKit(getSystem()) {
             {
-                final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
                         newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
                         "testRegisterRoleChangeListener");
 
@@ -2485,186 +2101,231 @@ public class ShardTest extends AbstractShardTest {
                         ShardLeaderStateChanged.class);
                 assertEquals("getLocalShardDataTree present", false,
                         leaderStateChanged.getLocalShardDataTree().isPresent());
-
-                shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
             }
         };
     }
 
     @Test
     public void testFollowerInitialSyncStatus() throws Exception {
-        final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
+        final TestActorRef<Shard> shard = actorFactory.createTestActor(
                 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());
-
-        shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
-    }
-
-    private static void commitTransaction(final DataTree store, final DataTreeModification modification) throws DataValidationFailedException {
-        modification.ready();
-        store.validate(modification);
-        store.commit(store.prepare(modification));
     }
 
     @Test
-    public void testClusteredDataChangeListenerDelayedRegistration() throws Exception {
-        new ShardTestKit(getSystem()) {{
-            String testName = "testClusteredDataChangeListenerDelayedRegistration";
-            dataStoreContextBuilder.shardElectionTimeoutFactor(1000);
-
-            final MockDataChangeListener listener = new MockDataChangeListener(1);
-            final ActorRef dclActor = actorFactory.createActor(DataChangeListener.props(listener),
-                    actorFactory.generateActorId(testName + "-DataChangeListener"));
+    public void testClusteredDataChangeListenerWithDelayedRegistration() throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final String testName = "testClusteredDataChangeListenerWithDelayedRegistration";
+                dataStoreContextBuilder.shardElectionTimeoutFactor(1000)
+                        .customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName());
 
-            final TestActorRef<Shard> shard = actorFactory.createTestActor(
-                    newShardBuilder().props().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    actorFactory.generateActorId(testName + "-shard"));
+                final YangInstanceIdentifier path = TestModel.TEST_PATH;
+                final MockDataChangeListener listener = new MockDataChangeListener(1);
+                final ActorRef dclActor = actorFactory.createActor(DataChangeListener.props(listener, path),
+                        actorFactory.generateActorId(testName + "-DataChangeListener"));
 
-            waitUntilNoLeader(shard);
+                setupInMemorySnapshotStore();
 
-            final YangInstanceIdentifier path = TestModel.TEST_PATH;
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardBuilder().props().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        actorFactory.generateActorId(testName + "-shard"));
 
-            shard.tell(new RegisterChangeListener(path, dclActor, AsyncDataBroker.DataChangeScope.BASE, true), getRef());
-            final RegisterChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
-                RegisterChangeListenerReply.class);
-            assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
+                waitUntilNoLeader(shard);
 
-            writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+                shard.tell(new RegisterChangeListener(path, dclActor, AsyncDataBroker.DataChangeScope.BASE, true),
+                        getRef());
+                final RegisterDataTreeNotificationListenerReply reply = expectMsgClass(duration("5 seconds"),
+                        RegisterDataTreeNotificationListenerReply.class);
+                assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
 
-            shard.tell(new ElectionTimeout(), ActorRef.noSender());
+                shard.tell(DatastoreContext.newBuilderFrom(dataStoreContextBuilder.build())
+                        .customRaftPolicyImplementation(null).build(), ActorRef.noSender());
 
-            listener.waitForChangeEvents();
-        }};
+                listener.waitForChangeEvents();
+            }
+        };
     }
 
     @Test
     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 leaderShardID = ShardIdentifier.builder().memberName(
-                    actorFactory.generateActorId(testName + "-leader")).shardName("inventory").type("config").build();
-
-            final TestActorRef<Shard> followerShard = actorFactory.createTestActor(
-                    Shard.builder().id(followerShardID).
-                        datastoreContext(dataStoreContextBuilder.shardElectionTimeoutFactor(1000).build()).
-                        peerAddresses(Collections.singletonMap(leaderShardID.toString(),
-                            "akka://test/user/" + leaderShardID.toString())).schemaContext(SCHEMA_CONTEXT).props().
-                    withDispatcher(Dispatchers.DefaultDispatcherId()), followerShardID.toString());
-
-            final TestActorRef<Shard> leaderShard = actorFactory.createTestActor(
-                    Shard.builder().id(leaderShardID).datastoreContext(newDatastoreContext()).
-                        peerAddresses(Collections.singletonMap(followerShardID.toString(),
-                            "akka://test/user/" + followerShardID.toString())).schemaContext(SCHEMA_CONTEXT).props().
-                    withDispatcher(Dispatchers.DefaultDispatcherId()), leaderShardID.toString());
-
-            leaderShard.tell(new ElectionTimeout(), ActorRef.noSender());
-            String leaderPath = waitUntilLeader(followerShard);
-            assertEquals("Shard leader path", leaderShard.path().toString(), leaderPath);
-
-            final YangInstanceIdentifier path = TestModel.TEST_PATH;
-            final MockDataChangeListener listener = new MockDataChangeListener(1);
-            final ActorRef dclActor = actorFactory.createActor(DataChangeListener.props(listener),
-                    actorFactory.generateActorId(testName + "-DataChangeListener"));
-
-            followerShard.tell(new RegisterChangeListener(path, dclActor, AsyncDataBroker.DataChangeScope.BASE, true), getRef());
-            final RegisterChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
-                RegisterChangeListenerReply.class);
-            assertNotNull("getListenerRegistratioznPath", reply.getListenerRegistrationPath());
-
-            writeToStore(followerShard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
-
-            listener.waitForChangeEvents();
-        }};
+        new ShardTestKit(getSystem()) {
+            {
+                final String testName = "testClusteredDataChangeListenerRegistration";
+                final ShardIdentifier followerShardID = ShardIdentifier.create("inventory",
+                        MemberName.forName(actorFactory.generateActorId(testName + "-follower")), "config");
+
+                final ShardIdentifier leaderShardID = ShardIdentifier.create("inventory",
+                        MemberName.forName(actorFactory.generateActorId(testName + "-leader")), "config");
+
+                final TestActorRef<Shard> followerShard = actorFactory
+                        .createTestActor(Shard.builder().id(followerShardID)
+                                .datastoreContext(dataStoreContextBuilder.shardElectionTimeoutFactor(1000).build())
+                                .peerAddresses(Collections.singletonMap(leaderShardID.toString(),
+                                        "akka://test/user/" + leaderShardID.toString()))
+                                .schemaContext(SCHEMA_CONTEXT).props()
+                                .withDispatcher(Dispatchers.DefaultDispatcherId()), followerShardID.toString());
+
+                final TestActorRef<Shard> leaderShard = actorFactory
+                        .createTestActor(Shard.builder().id(leaderShardID).datastoreContext(newDatastoreContext())
+                                .peerAddresses(Collections.singletonMap(followerShardID.toString(),
+                                        "akka://test/user/" + followerShardID.toString()))
+                                .schemaContext(SCHEMA_CONTEXT).props()
+                                .withDispatcher(Dispatchers.DefaultDispatcherId()), leaderShardID.toString());
+
+                leaderShard.tell(TimeoutNow.INSTANCE, ActorRef.noSender());
+                final String leaderPath = waitUntilLeader(followerShard);
+                assertEquals("Shard leader path", leaderShard.path().toString(), leaderPath);
+
+                final YangInstanceIdentifier path = TestModel.TEST_PATH;
+                final MockDataChangeListener listener = new MockDataChangeListener(1);
+                final ActorRef dclActor = actorFactory.createActor(DataChangeListener.props(listener, path),
+                        actorFactory.generateActorId(testName + "-DataChangeListener"));
+
+                followerShard.tell(
+                        new RegisterChangeListener(path, dclActor, AsyncDataBroker.DataChangeScope.BASE, true),
+                        getRef());
+                final RegisterDataTreeNotificationListenerReply reply = expectMsgClass(duration("5 seconds"),
+                        RegisterDataTreeNotificationListenerReply.class);
+                assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
+
+                writeToStore(followerShard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+
+                listener.waitForChangeEvents();
+            }
+        };
+    }
+
+    @Test
+    public void testClusteredDataTreeChangeListenerWithDelayedRegistration() throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final String testName = "testClusteredDataTreeChangeListenerWithDelayedRegistration";
+                dataStoreContextBuilder.shardElectionTimeoutFactor(1000)
+                        .customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName());
+
+                final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
+                final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener,
+                        TestModel.TEST_PATH), actorFactory.generateActorId(testName + "-DataTreeChangeListener"));
+
+                setupInMemorySnapshotStore();
+
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardBuilder().props().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        actorFactory.generateActorId(testName + "-shard"));
+
+                waitUntilNoLeader(shard);
+
+                shard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor, true), getRef());
+                final RegisterDataTreeNotificationListenerReply reply = expectMsgClass(duration("5 seconds"),
+                        RegisterDataTreeNotificationListenerReply.class);
+                assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
+
+                shard.tell(DatastoreContext.newBuilderFrom(dataStoreContextBuilder.build())
+                        .customRaftPolicyImplementation(null).build(), ActorRef.noSender());
+
+                listener.waitForChangeEvents();
+            }
+        };
     }
 
     @Test
-    public void testClusteredDataTreeChangeListenerDelayedRegistration() throws Exception {
-        new ShardTestKit(getSystem()) {{
-            String testName = "testClusteredDataTreeChangeListenerDelayedRegistration";
-            dataStoreContextBuilder.shardElectionTimeoutFactor(1000);
+    public void testClusteredDataTreeChangeListenerWithDelayedRegistrationClosed() throws Exception {
+        new ShardTestKit(getSystem()) {
+            {
+                final String testName = "testClusteredDataTreeChangeListenerWithDelayedRegistrationClosed";
+                dataStoreContextBuilder.shardElectionTimeoutFactor(1000)
+                        .customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName());
 
-            final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
-            final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener),
-                    actorFactory.generateActorId(testName + "-DataTreeChangeListener"));
+                final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(0);
+                final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener,
+                        TestModel.TEST_PATH), actorFactory.generateActorId(testName + "-DataTreeChangeListener"));
 
-            final TestActorRef<Shard> shard = actorFactory.createTestActor(
-                    newShardBuilder().props().withDispatcher(Dispatchers.DefaultDispatcherId()),
-                    actorFactory.generateActorId(testName + "-shard"));
+                setupInMemorySnapshotStore();
 
-            waitUntilNoLeader(shard);
+                final TestActorRef<Shard> shard = actorFactory.createTestActor(
+                        newShardBuilder().props().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                        actorFactory.generateActorId(testName + "-shard"));
 
-            final YangInstanceIdentifier path = TestModel.TEST_PATH;
+                waitUntilNoLeader(shard);
 
-            shard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor, true), getRef());
-            final RegisterDataTreeChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
-                    RegisterDataTreeChangeListenerReply.class);
-            assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
+                shard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor, true), getRef());
+                final RegisterDataTreeNotificationListenerReply reply = expectMsgClass(duration("5 seconds"),
+                        RegisterDataTreeNotificationListenerReply.class);
+                assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
 
-            writeToStore(shard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+                final ActorSelection regActor = getSystem().actorSelection(reply.getListenerRegistrationPath());
+                regActor.tell(CloseDataTreeNotificationListenerRegistration.getInstance(), getRef());
+                expectMsgClass(CloseDataTreeNotificationListenerRegistrationReply.class);
 
-            shard.tell(new ElectionTimeout(), ActorRef.noSender());
+                shard.tell(DatastoreContext.newBuilderFrom(dataStoreContextBuilder.build())
+                        .customRaftPolicyImplementation(null).build(), ActorRef.noSender());
 
-            listener.waitForChangeEvents();
-        }};
+                listener.expectNoMoreChanges("Received unexpected change after close");
+            }
+        };
     }
 
     @Test
     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 leaderShardID = ShardIdentifier.builder().memberName(
-                    actorFactory.generateActorId(testName + "-leader")).shardName("inventory").type("config").build();
-
-            final TestActorRef<Shard> followerShard = actorFactory.createTestActor(
-                    Shard.builder().id(followerShardID).
-                        datastoreContext(dataStoreContextBuilder.shardElectionTimeoutFactor(1000).build()).
-                        peerAddresses(Collections.singletonMap(leaderShardID.toString(),
-                            "akka://test/user/" + leaderShardID.toString())).schemaContext(SCHEMA_CONTEXT).props().
-                    withDispatcher(Dispatchers.DefaultDispatcherId()), followerShardID.toString());
-
-            final TestActorRef<Shard> leaderShard = actorFactory.createTestActor(
-                    Shard.builder().id(leaderShardID).datastoreContext(newDatastoreContext()).
-                        peerAddresses(Collections.singletonMap(followerShardID.toString(),
-                            "akka://test/user/" + followerShardID.toString())).schemaContext(SCHEMA_CONTEXT).props().
-                    withDispatcher(Dispatchers.DefaultDispatcherId()), leaderShardID.toString());
-
-            leaderShard.tell(new ElectionTimeout(), ActorRef.noSender());
-            String leaderPath = waitUntilLeader(followerShard);
-            assertEquals("Shard leader path", leaderShard.path().toString(), leaderPath);
-
-            final YangInstanceIdentifier path = TestModel.TEST_PATH;
-            final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
-            final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener),
-                    actorFactory.generateActorId(testName + "-DataTreeChangeListener"));
-
-            followerShard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor, true), getRef());
-            final RegisterDataTreeChangeListenerReply reply = expectMsgClass(duration("5 seconds"),
-                    RegisterDataTreeChangeListenerReply.class);
-            assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
-
-            writeToStore(followerShard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
-
-            listener.waitForChangeEvents();
-        }};
+        new ShardTestKit(getSystem()) {
+            {
+                final String testName = "testClusteredDataTreeChangeListenerRegistration";
+                final ShardIdentifier followerShardID = ShardIdentifier.create("inventory",
+                        MemberName.forName(actorFactory.generateActorId(testName + "-follower")), "config");
+
+                final ShardIdentifier leaderShardID = ShardIdentifier.create("inventory",
+                        MemberName.forName(actorFactory.generateActorId(testName + "-leader")), "config");
+
+                final TestActorRef<Shard> followerShard = actorFactory
+                        .createTestActor(Shard.builder().id(followerShardID)
+                                .datastoreContext(dataStoreContextBuilder.shardElectionTimeoutFactor(1000).build())
+                                .peerAddresses(Collections.singletonMap(leaderShardID.toString(),
+                                        "akka://test/user/" + leaderShardID.toString()))
+                                .schemaContext(SCHEMA_CONTEXT).props()
+                                .withDispatcher(Dispatchers.DefaultDispatcherId()), followerShardID.toString());
+
+                final TestActorRef<Shard> leaderShard = actorFactory
+                        .createTestActor(Shard.builder().id(leaderShardID).datastoreContext(newDatastoreContext())
+                                .peerAddresses(Collections.singletonMap(followerShardID.toString(),
+                                        "akka://test/user/" + followerShardID.toString()))
+                                .schemaContext(SCHEMA_CONTEXT).props()
+                                .withDispatcher(Dispatchers.DefaultDispatcherId()), leaderShardID.toString());
+
+                leaderShard.tell(TimeoutNow.INSTANCE, ActorRef.noSender());
+                final String leaderPath = waitUntilLeader(followerShard);
+                assertEquals("Shard leader path", leaderShard.path().toString(), leaderPath);
+
+                final YangInstanceIdentifier path = TestModel.TEST_PATH;
+                final MockDataTreeChangeListener listener = new MockDataTreeChangeListener(1);
+                final ActorRef dclActor = actorFactory.createActor(DataTreeChangeListenerActor.props(listener, path),
+                        actorFactory.generateActorId(testName + "-DataTreeChangeListener"));
+
+                followerShard.tell(new RegisterDataTreeChangeListener(TestModel.TEST_PATH, dclActor, true), getRef());
+                final RegisterDataTreeNotificationListenerReply reply = expectMsgClass(duration("5 seconds"),
+                        RegisterDataTreeNotificationListenerReply.class);
+                assertNotNull("getListenerRegistrationPath", reply.getListenerRegistrationPath());
+
+                writeToStore(followerShard, path, ImmutableNodes.containerNode(TestModel.TEST_QNAME));
+
+                listener.waitForChangeEvents();
+            }
+        };
     }
 
     @Test
     public void testServerRemoved() throws Exception {
-        final TestActorRef<MessageCollectorActor> parent = TestActorRef.create(getSystem(), MessageCollectorActor.props());
+        final TestActorRef<MessageCollectorActor> parent = actorFactory.createTestActor(MessageCollectorActor.props());
 
         final ActorRef shard = parent.underlyingActor().context().actorOf(
                 newShardBuilder().props().withDispatcher(Dispatchers.DefaultDispatcherId()),
@@ -2673,7 +2334,5 @@ public class ShardTest extends AbstractShardTest {
         shard.tell(new ServerRemoved("test"), ActorRef.noSender());
 
         MessageCollectorActor.expectFirstMatching(parent, ServerRemoved.class);
-
     }
-
 }