Bug 2486: Get testAbortBeforeFinishCommit working again
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / ShardTest.java
index cd8a65844756b51820c5f6fb93dfe62a53071d31..ed842b2021475b3fef53d95244a37e03ac053d6b 100644 (file)
@@ -1,5 +1,16 @@
 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.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.inOrder;
+import static org.mockito.Mockito.mock;
+import static org.opendaylight.controller.cluster.datastore.messages.CreateTransaction.CURRENT_VERSION;
 import akka.actor.ActorRef;
 import akka.actor.PoisonPill;
 import akka.actor.Props;
@@ -16,12 +27,23 @@ import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.MoreExecutors;
 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.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.InOrder;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
+import org.opendaylight.controller.cluster.datastore.DatastoreContext.Builder;
 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.AbortTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.AbortTransactionReply;
@@ -54,6 +76,7 @@ import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
+import org.opendaylight.controller.cluster.raft.protobuff.client.messages.CompositeModificationByteStringPayload;
 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.CompositeModificationPayload;
 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
 import org.opendaylight.controller.md.cluster.datastore.model.SchemaContextHelper;
@@ -80,29 +103,6 @@ import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
-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.ExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
-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.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.inOrder;
-import static org.mockito.Mockito.mock;
-
-
 public class ShardTest extends AbstractActorTest {
 
     private static final SchemaContext SCHEMA_CONTEXT = TestModel.createTestContext();
@@ -112,12 +112,13 @@ public class ShardTest extends AbstractActorTest {
     private final ShardIdentifier shardID = ShardIdentifier.builder().memberName("member-1")
             .shardName("inventory").type("config" + NEXT_SHARD_NUM.getAndIncrement()).build();
 
-    private DatastoreContext dataStoreContext = DatastoreContext.newBuilder().
+    private final Builder dataStoreContextBuilder = DatastoreContext.newBuilder().
             shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).
-            shardHeartbeatIntervalInMillis(100).build();
+            shardHeartbeatIntervalInMillis(100);
 
     @Before
     public void setUp() {
+        Builder newBuilder = DatastoreContext.newBuilder();
         InMemorySnapshotStore.clear();
         InMemoryJournal.clear();
     }
@@ -128,9 +129,13 @@ public class ShardTest extends AbstractActorTest {
         InMemoryJournal.clear();
     }
 
+    private DatastoreContext newDatastoreContext() {
+        return dataStoreContextBuilder.build();
+    }
+
     private Props newShardProps() {
         return Shard.props(shardID, Collections.<ShardIdentifier,String>emptyMap(),
-                dataStoreContext, SCHEMA_CONTEXT);
+                newDatastoreContext(), SCHEMA_CONTEXT);
     }
 
     @Test
@@ -187,7 +192,7 @@ public class ShardTest extends AbstractActorTest {
                 @Override
                 public Shard create() throws Exception {
                     return new Shard(shardID, Collections.<ShardIdentifier,String>emptyMap(),
-                            dataStoreContext, SCHEMA_CONTEXT) {
+                            newDatastoreContext(), SCHEMA_CONTEXT) {
                         @Override
                         public void onReceiveCommand(final Object message) throws Exception {
                             if(message instanceof ElectionTimeout && firstElectionTimeout) {
@@ -307,6 +312,7 @@ public class ShardTest extends AbstractActorTest {
         }};
     }
 
+    @SuppressWarnings("serial")
     @Test
     public void testPeerAddressResolved() throws Exception {
         new ShardTestKit(getSystem()) {{
@@ -314,7 +320,7 @@ public class ShardTest extends AbstractActorTest {
             class TestShard extends Shard {
                 TestShard() {
                     super(shardID, Collections.<ShardIdentifier, String>singletonMap(shardID, null),
-                            dataStoreContext, SCHEMA_CONTEXT);
+                            newDatastoreContext(), SCHEMA_CONTEXT);
                 }
 
                 Map<String, String> getPeerAddresses() {
@@ -433,9 +439,9 @@ public class ShardTest extends AbstractActorTest {
                           ImmutableNodes.mapNodeBuilder(TestModel.OUTER_LIST_QNAME).build(),
                           SCHEMA_CONTEXT))));
 
-        int nListEntries = 11;
+        int nListEntries = 16;
         Set<Integer> listEntryKeys = new HashSet<>();
-        for(int i = 1; i <= nListEntries; i++) {
+        for(int i = 1; i <= nListEntries-5; i++) {
             listEntryKeys.add(Integer.valueOf(i));
             YangInstanceIdentifier path = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
                     .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i).build();
@@ -446,6 +452,19 @@ public class ShardTest extends AbstractActorTest {
                     newPayload(mod)));
         }
 
+        // Add some of the new CompositeModificationByteStringPayload
+        for(int i = 11; i <= nListEntries; i++) {
+            listEntryKeys.add(Integer.valueOf(i));
+            YangInstanceIdentifier path = YangInstanceIdentifier.builder(TestModel.OUTER_LIST_PATH)
+                    .nodeWithKey(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i).build();
+            Modification mod = new MergeModification(path,
+                    ImmutableNodes.mapEntry(TestModel.OUTER_LIST_QNAME, TestModel.ID_QNAME, i),
+                    SCHEMA_CONTEXT);
+            InMemoryJournal.addEntry(shardID.toString(), i, new ReplicatedLogImplEntry(i, 1,
+                    newByteStringPayload(mod)));
+        }
+
+
         InMemoryJournal.addEntry(shardID.toString(), nListEntries + 1,
                 new ApplyLogEntries(nListEntries));
 
@@ -457,7 +476,7 @@ public class ShardTest extends AbstractActorTest {
             @Override
             public Shard create() throws Exception {
                 return new Shard(shardID, Collections.<ShardIdentifier,String>emptyMap(),
-                        dataStoreContext, SCHEMA_CONTEXT) {
+                        newDatastoreContext(), SCHEMA_CONTEXT) {
                     @Override
                     protected void onRecoveryComplete() {
                         try {
@@ -508,7 +527,7 @@ public class ShardTest extends AbstractActorTest {
         shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
     }
 
-    private CompositeModificationPayload newPayload(Modification... mods) {
+    private CompositeModificationPayload newPayload(final Modification... mods) {
         MutableCompositeModification compMod = new MutableCompositeModification();
         for(Modification mod: mods) {
             compMod.addModification(mod);
@@ -517,15 +536,25 @@ public class ShardTest extends AbstractActorTest {
         return new CompositeModificationPayload(compMod.toSerializable());
     }
 
-    private DOMStoreThreePhaseCommitCohort setupMockWriteTransaction(String cohortName,
-            InMemoryDOMDataStore dataStore, YangInstanceIdentifier path, NormalizedNode data,
-            MutableCompositeModification modification) {
+    private CompositeModificationByteStringPayload newByteStringPayload(final Modification... mods) {
+        MutableCompositeModification compMod = new MutableCompositeModification();
+        for(Modification mod: mods) {
+            compMod.addModification(mod);
+        }
+
+        return new CompositeModificationByteStringPayload(compMod.toSerializable());
+    }
+
+
+    private DOMStoreThreePhaseCommitCohort setupMockWriteTransaction(final String cohortName,
+            final InMemoryDOMDataStore dataStore, final YangInstanceIdentifier path, final NormalizedNode<?, ?> data,
+            final MutableCompositeModification modification) {
         return setupMockWriteTransaction(cohortName, dataStore, path, data, modification, null);
     }
 
-    private DOMStoreThreePhaseCommitCohort setupMockWriteTransaction(String cohortName,
-            InMemoryDOMDataStore dataStore, YangInstanceIdentifier path, NormalizedNode data,
-            MutableCompositeModification modification,
+    private DOMStoreThreePhaseCommitCohort setupMockWriteTransaction(final String cohortName,
+            final InMemoryDOMDataStore dataStore, final YangInstanceIdentifier path, final NormalizedNode<?, ?> data,
+            final MutableCompositeModification modification,
             final Function<DOMStoreThreePhaseCommitCohort,ListenableFuture<Void>> preCommit) {
 
         DOMStoreWriteTransaction tx = dataStore.newWriteOnlyTransaction();
@@ -535,14 +564,14 @@ public class ShardTest extends AbstractActorTest {
 
         doAnswer(new Answer<ListenableFuture<Boolean>>() {
             @Override
-            public ListenableFuture<Boolean> answer(InvocationOnMock invocation) {
+            public ListenableFuture<Boolean> answer(final InvocationOnMock invocation) {
                 return realCohort.canCommit();
             }
         }).when(cohort).canCommit();
 
         doAnswer(new Answer<ListenableFuture<Void>>() {
             @Override
-            public ListenableFuture<Void> answer(InvocationOnMock invocation) throws Throwable {
+            public ListenableFuture<Void> answer(final InvocationOnMock invocation) throws Throwable {
                 if(preCommit != null) {
                     return preCommit.apply(realCohort);
                 } else {
@@ -553,14 +582,14 @@ public class ShardTest extends AbstractActorTest {
 
         doAnswer(new Answer<ListenableFuture<Void>>() {
             @Override
-            public ListenableFuture<Void> answer(InvocationOnMock invocation) throws Throwable {
+            public ListenableFuture<Void> answer(final InvocationOnMock invocation) throws Throwable {
                 return realCohort.commit();
             }
         }).when(cohort).commit();
 
         doAnswer(new Answer<ListenableFuture<Void>>() {
             @Override
-            public ListenableFuture<Void> answer(InvocationOnMock invocation) throws Throwable {
+            public ListenableFuture<Void> answer(final InvocationOnMock invocation) throws Throwable {
                 return realCohort.abort();
             }
         }).when(cohort).abort();
@@ -611,7 +640,8 @@ public class ShardTest extends AbstractActorTest {
             // Simulate the ForwardedReadyTransaction message for the first Tx that would be sent
             // by the ShardTransaction.
 
-            shard.tell(new ForwardedReadyTransaction(transactionID1, cohort1, modification1, true), getRef());
+            shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION,
+                    cohort1, modification1, true), getRef());
             ReadyTransactionReply readyReply = ReadyTransactionReply.fromSerializable(
                     expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS));
             assertEquals("Cohort path", shard.path().toString(), readyReply.getCohortPath());
@@ -625,10 +655,12 @@ public class ShardTest extends AbstractActorTest {
 
             // Send the ForwardedReadyTransaction for the next 2 Tx's.
 
-            shard.tell(new ForwardedReadyTransaction(transactionID2, cohort2, modification2, true), getRef());
+            shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION,
+                    cohort2, modification2, true), getRef());
             expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
 
-            shard.tell(new ForwardedReadyTransaction(transactionID3, cohort3, modification3, true), getRef());
+            shard.tell(new ForwardedReadyTransaction(transactionID3, CURRENT_VERSION,
+                    cohort3, modification3, true), getRef());
             expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
 
             // Send the CanCommitTransaction message for the next 2 Tx's. These should get queued and
@@ -654,12 +686,12 @@ public class ShardTest extends AbstractActorTest {
             class OnFutureComplete extends OnComplete<Object> {
                 private final Class<?> expRespType;
 
-                OnFutureComplete(Class<?> expRespType) {
+                OnFutureComplete(final Class<?> expRespType) {
                     this.expRespType = expRespType;
                 }
 
                 @Override
-                public void onComplete(Throwable error, Object resp) {
+                public void onComplete(final Throwable error, final Object resp) {
                     if(error != null) {
                         caughtEx.set(new AssertionError(getClass().getSimpleName() + " failure", error));
                     } else {
@@ -672,7 +704,7 @@ public class ShardTest extends AbstractActorTest {
                     }
                 }
 
-                void onSuccess(Object resp) throws Exception {
+                void onSuccess(final Object resp) throws Exception {
                 }
             }
 
@@ -682,7 +714,7 @@ public class ShardTest extends AbstractActorTest {
                 }
 
                 @Override
-                public void onComplete(Throwable error, Object resp) {
+                public void onComplete(final Throwable error, final Object resp) {
                     super.onComplete(error, resp);
                     commitLatch.countDown();
                 }
@@ -691,13 +723,13 @@ public class ShardTest extends AbstractActorTest {
             class OnCanCommitFutureComplete extends OnFutureComplete {
                 private final String transactionID;
 
-                OnCanCommitFutureComplete(String transactionID) {
+                OnCanCommitFutureComplete(final String transactionID) {
                     super(CanCommitTransactionReply.SERIALIZABLE_CLASS);
                     this.transactionID = transactionID;
                 }
 
                 @Override
-                void onSuccess(Object resp) throws Exception {
+                void onSuccess(final Object resp) throws Exception {
                     CanCommitTransactionReply canCommitReply =
                             CanCommitTransactionReply.fromSerializable(resp);
                     assertEquals("Can commit", true, canCommitReply.getCanCommit());
@@ -748,15 +780,72 @@ public class ShardTest extends AbstractActorTest {
             assertTrue("Missing leaf " + TestModel.ID_QNAME.getLocalName(), idLeaf.isPresent());
             assertEquals(TestModel.ID_QNAME.getLocalName() + " value", 1, idLeaf.get().getValue());
 
-            for(int i = 0; i < 20 * 5; i++) {
-                long lastLogIndex = shard.underlyingActor().getShardMBean().getLastLogIndex();
-                if(lastLogIndex == 2) {
-                    break;
-                }
-                Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
+            verifyLastLogIndex(shard, 2);
+
+            shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
+        }};
+    }
+
+    private void verifyLastLogIndex(TestActorRef<Shard> shard, long expectedValue) {
+        for(int i = 0; i < 20 * 5; i++) {
+            long lastLogIndex = shard.underlyingActor().getShardMBean().getLastLogIndex();
+            if(lastLogIndex == expectedValue) {
+                break;
             }
+            Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
+        }
+
+        assertEquals("Last log index", expectedValue, shard.underlyingActor().getShardMBean().getLastLogIndex());
+    }
+
+    @Test
+    public void testCommitWithPersistenceDisabled() throws Throwable {
+        dataStoreContextBuilder.persistent(false);
+        new ShardTestKit(getSystem()) {{
+            final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
+                    newShardProps().withDispatcher(Dispatchers.DefaultDispatcherId()),
+                    "testCommitPhaseFailure");
+
+            waitUntilLeader(shard);
+
+            InMemoryDOMDataStore dataStore = shard.underlyingActor().getDataStore();
+
+            // Setup a simulated transactions with a mock cohort.
+
+            String transactionID = "tx";
+            MutableCompositeModification modification = new MutableCompositeModification();
+            NormalizedNode<?, ?> containerNode = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
+            DOMStoreThreePhaseCommitCohort cohort = setupMockWriteTransaction("cohort", dataStore,
+                    TestModel.TEST_PATH, containerNode, modification);
+
+            FiniteDuration duration = duration("5 seconds");
+
+            // Simulate the ForwardedReadyTransaction messages that would be sent
+            // by the ShardTransaction.
+
+            shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION,
+                    cohort, modification, true), getRef());
+            expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
+
+            // Send the CanCommitTransaction message.
+
+            shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
+            CanCommitTransactionReply canCommitReply = CanCommitTransactionReply.fromSerializable(
+                    expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
+            assertEquals("Can commit", true, canCommitReply.getCanCommit());
+
+            // Send the CanCommitTransaction message.
+
+            shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef());
+            expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
 
-            assertEquals("Last log index", 2, shard.underlyingActor().getShardMBean().getLastLogIndex());
+            InOrder inOrder = inOrder(cohort);
+            inOrder.verify(cohort).canCommit();
+            inOrder.verify(cohort).preCommit();
+            inOrder.verify(cohort).commit();
+
+            NormalizedNode<?, ?> actualNode = readStore(shard, TestModel.TEST_PATH);
+            assertEquals(TestModel.TEST_QNAME.getLocalName(), containerNode, actualNode);
 
             shard.tell(PoisonPill.getInstance(), ActorRef.noSender());
         }};
@@ -792,10 +881,12 @@ public class ShardTest extends AbstractActorTest {
             // Simulate the ForwardedReadyTransaction messages that would be sent
             // by the ShardTransaction.
 
-            shard.tell(new ForwardedReadyTransaction(transactionID1, cohort1, modification1, true), getRef());
+            shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION,
+                    cohort1, modification1, true), getRef());
             expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
 
-            shard.tell(new ForwardedReadyTransaction(transactionID2, cohort2, modification2, true), getRef());
+            shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION,
+                    cohort2, modification2, true), getRef());
             expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
 
             // Send the CanCommitTransaction message for the first Tx.
@@ -822,7 +913,7 @@ public class ShardTest extends AbstractActorTest {
             final CountDownLatch latch = new CountDownLatch(1);
             canCommitFuture.onComplete(new OnComplete<Object>() {
                 @Override
-                public void onComplete(Throwable t, Object resp) {
+                public void onComplete(final Throwable t, final Object resp) {
                     latch.countDown();
                 }
             }, getSystem().dispatcher());
@@ -859,7 +950,8 @@ public class ShardTest extends AbstractActorTest {
             // Simulate the ForwardedReadyTransaction messages that would be sent
             // by the ShardTransaction.
 
-            shard.tell(new ForwardedReadyTransaction(transactionID, cohort, modification, true), getRef());
+            shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION,
+                    cohort, modification, true), getRef());
             expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
 
             // Send the CanCommitTransaction message.
@@ -902,7 +994,8 @@ public class ShardTest extends AbstractActorTest {
             // Simulate the ForwardedReadyTransaction messages that would be sent
             // by the ShardTransaction.
 
-            shard.tell(new ForwardedReadyTransaction(transactionID, cohort, modification, true), getRef());
+            shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION,
+                    cohort, modification, true), getRef());
             expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
 
             // Send the CanCommitTransaction message.
@@ -924,26 +1017,24 @@ public class ShardTest extends AbstractActorTest {
             waitUntilLeader(shard);
 
             final FiniteDuration duration = duration("5 seconds");
-            final Timeout timeout = new Timeout(duration);
-
             InMemoryDOMDataStore dataStore = shard.underlyingActor().getDataStore();
 
             final String transactionID = "tx1";
-            final CountDownLatch abortComplete = new CountDownLatch(1);
             Function<DOMStoreThreePhaseCommitCohort,ListenableFuture<Void>> preCommit =
                           new Function<DOMStoreThreePhaseCommitCohort,ListenableFuture<Void>>() {
                 @Override
                 public ListenableFuture<Void> apply(final DOMStoreThreePhaseCommitCohort cohort) {
                     ListenableFuture<Void> preCommitFuture = cohort.preCommit();
 
-                    Future<Object> abortFuture = Patterns.ask(shard,
-                            new AbortTransaction(transactionID).toSerializable(), timeout);
-                    abortFuture.onComplete(new OnComplete<Object>() {
-                        @Override
-                        public void onComplete(Throwable e, Object resp) {
-                            abortComplete.countDown();
-                        }
-                    }, getSystem().dispatcher());
+                    // 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;
                 }
@@ -954,7 +1045,8 @@ public class ShardTest extends AbstractActorTest {
                     TestModel.TEST_PATH, ImmutableNodes.containerNode(TestModel.TEST_QNAME),
                     modification, preCommit);
 
-            shard.tell(new ForwardedReadyTransaction(transactionID, cohort, modification, true), getRef());
+            shard.tell(new ForwardedReadyTransaction(transactionID, CURRENT_VERSION,
+                    cohort, modification, true), getRef());
             expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
 
             shard.tell(new CanCommitTransaction(transactionID).toSerializable(), getRef());
@@ -962,14 +1054,14 @@ public class ShardTest extends AbstractActorTest {
                     expectMsgClass(duration, CanCommitTransactionReply.SERIALIZABLE_CLASS));
             assertEquals("Can commit", true, canCommitReply.getCanCommit());
 
-            Future<Object> commitFuture = Patterns.ask(shard,
-                    new CommitTransaction(transactionID).toSerializable(), timeout);
-
-            assertEquals("Abort complete", true, abortComplete.await(5, TimeUnit.SECONDS));
-
-            Await.result(commitFuture, duration);
+            shard.tell(new CommitTransaction(transactionID).toSerializable(), getRef());
+            expectMsgClass(duration, CommitTransactionReply.SERIALIZABLE_CLASS);
 
             NormalizedNode<?, ?> node = readStore(shard, TestModel.TEST_PATH);
+
+            // 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(PoisonPill.getInstance(), ActorRef.noSender());
@@ -978,7 +1070,7 @@ public class ShardTest extends AbstractActorTest {
 
     @Test
     public void testTransactionCommitTimeout() throws Throwable {
-        dataStoreContext = DatastoreContext.newBuilder().shardTransactionCommitTimeoutInSeconds(1).build();
+        dataStoreContextBuilder.shardTransactionCommitTimeoutInSeconds(1);
 
         new ShardTestKit(getSystem()) {{
             final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
@@ -1018,10 +1110,12 @@ public class ShardTest extends AbstractActorTest {
 
             // Ready the Tx's
 
-            shard.tell(new ForwardedReadyTransaction(transactionID1, cohort1, modification1, true), getRef());
+            shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION,
+                    cohort1, modification1, true), getRef());
             expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
 
-            shard.tell(new ForwardedReadyTransaction(transactionID2, cohort2, modification2, true), getRef());
+            shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION,
+                    cohort2, modification2, true), getRef());
             expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
 
             // canCommit 1st Tx. We don't send the commit so it should timeout.
@@ -1048,7 +1142,7 @@ public class ShardTest extends AbstractActorTest {
 
     @Test
     public void testTransactionCommitQueueCapacityExceeded() throws Throwable {
-        dataStoreContext = DatastoreContext.newBuilder().shardTransactionCommitQueueCapacity(1).build();
+        dataStoreContextBuilder.shardTransactionCommitQueueCapacity(1);
 
         new ShardTestKit(getSystem()) {{
             final TestActorRef<Shard> shard = TestActorRef.create(getSystem(),
@@ -1080,13 +1174,16 @@ public class ShardTest extends AbstractActorTest {
 
             // Ready the Tx's
 
-            shard.tell(new ForwardedReadyTransaction(transactionID1, cohort1, modification1, true), getRef());
+            shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION,
+                    cohort1, modification1, true), getRef());
             expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
 
-            shard.tell(new ForwardedReadyTransaction(transactionID2, cohort2, modification2, true), getRef());
+            shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION,
+                    cohort2, modification2, true), getRef());
             expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
 
-            shard.tell(new ForwardedReadyTransaction(transactionID3, cohort3, modification3, true), getRef());
+            shard.tell(new ForwardedReadyTransaction(transactionID3, CURRENT_VERSION,
+                    cohort3, modification3, true), getRef());
             expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
 
             // canCommit 1st Tx.
@@ -1149,10 +1246,12 @@ public class ShardTest extends AbstractActorTest {
             // Simulate the ForwardedReadyTransaction messages that would be sent
             // by the ShardTransaction.
 
-            shard.tell(new ForwardedReadyTransaction(transactionID1, cohort1, modification1, true), getRef());
+            shard.tell(new ForwardedReadyTransaction(transactionID1, CURRENT_VERSION,
+                    cohort1, modification1, true), getRef());
             expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
 
-            shard.tell(new ForwardedReadyTransaction(transactionID2, cohort2, modification2, true), getRef());
+            shard.tell(new ForwardedReadyTransaction(transactionID2, CURRENT_VERSION,
+                    cohort2, modification2, true), getRef());
             expectMsgClass(duration, ReadyTransactionReply.SERIALIZABLE_CLASS);
 
             // Send the CanCommitTransaction message for the first Tx.
@@ -1176,15 +1275,7 @@ public class ShardTest extends AbstractActorTest {
 
             // 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(Throwable t, Object resp) {
-                    latch.countDown();
-                }
-            }, getSystem().dispatcher());
-
-            assertEquals("2nd CanCommit complete", true, latch.await(5, TimeUnit.SECONDS));
+            Await.ready(canCommitFuture, duration);
 
             InOrder inOrder = inOrder(cohort1, cohort2);
             inOrder.verify(cohort1).canCommit();
@@ -1204,7 +1295,8 @@ public class ShardTest extends AbstractActorTest {
         testCreateSnapshot(false, "testCreateSnapshotWithNonPersistentData");
     }
 
-    public void testCreateSnapshot(boolean persistent, final String shardActorName) throws IOException, InterruptedException {
+    @SuppressWarnings("serial")
+    public void testCreateSnapshot(final boolean persistent, final String shardActorName) throws IOException, InterruptedException {
         final DatastoreContext dataStoreContext = DatastoreContext.newBuilder().
                 shardJournalRecoveryLogBatchSize(3).shardSnapshotBatchCount(5000).persistent(persistent).build();
 
@@ -1214,9 +1306,9 @@ public class ShardTest extends AbstractActorTest {
                 @Override
                 public Shard create() throws Exception {
                     return new Shard(shardID, Collections.<ShardIdentifier,String>emptyMap(),
-                            dataStoreContext, SCHEMA_CONTEXT) {
+                            newDatastoreContext(), SCHEMA_CONTEXT) {
                         @Override
-                        protected void commitSnapshot(long sequenceNumber) {
+                        protected void commitSnapshot(final long sequenceNumber) {
                             super.commitSnapshot(sequenceNumber);
                             latch.get().countDown();
                         }
@@ -1248,8 +1340,7 @@ public class ShardTest extends AbstractActorTest {
      */
     @Test
     public void testInMemoryDataStoreRestore() throws ReadFailedException {
-        InMemoryDOMDataStore store = new InMemoryDOMDataStore("test", MoreExecutors.listeningDecorator(
-            MoreExecutors.sameThreadExecutor()), MoreExecutors.sameThreadExecutor());
+        InMemoryDOMDataStore store = new InMemoryDOMDataStore("test", MoreExecutors.sameThreadExecutor());
 
         store.onGlobalContextUpdated(SCHEMA_CONTEXT);
 
@@ -1259,7 +1350,7 @@ public class ShardTest extends AbstractActorTest {
         commitTransaction(putTransaction);
 
 
-        NormalizedNode expected = readStore(store);
+        NormalizedNode<?, ?> expected = readStore(store);
 
         DOMStoreWriteTransaction writeTransaction = store.newWriteOnlyTransaction();
 
@@ -1268,10 +1359,9 @@ public class ShardTest extends AbstractActorTest {
 
         commitTransaction(writeTransaction);
 
-        NormalizedNode actual = readStore(store);
+        NormalizedNode<?, ?> actual = readStore(store);
 
         assertEquals(expected, actual);
-
     }
 
     @Test
@@ -1309,7 +1399,7 @@ public class ShardTest extends AbstractActorTest {
     }
 
 
-    private NormalizedNode readStore(InMemoryDOMDataStore store) throws ReadFailedException {
+    private NormalizedNode<?, ?> readStore(final InMemoryDOMDataStore store) throws ReadFailedException {
         DOMStoreReadTransaction transaction = store.newReadOnlyTransaction();
         CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> read =
             transaction.read(YangInstanceIdentifier.builder().build());
@@ -1323,7 +1413,7 @@ public class ShardTest extends AbstractActorTest {
         return normalizedNode;
     }
 
-    private void commitTransaction(DOMStoreWriteTransaction transaction) {
+    private void commitTransaction(final DOMStoreWriteTransaction transaction) {
         DOMStoreThreePhaseCommitCohort commitCohort = transaction.ready();
         ListenableFuture<Void> future =
             commitCohort.preCommit();
@@ -1339,13 +1429,13 @@ public class ShardTest extends AbstractActorTest {
         return new AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>() {
             @Override
             public void onDataChanged(
-                AsyncDataChangeEvent<YangInstanceIdentifier, NormalizedNode<?, ?>> change) {
+                final AsyncDataChangeEvent<YangInstanceIdentifier, NormalizedNode<?, ?>> change) {
 
             }
         };
     }
 
-    private NormalizedNode<?,?> readStore(TestActorRef<Shard> shard, YangInstanceIdentifier id)
+    static NormalizedNode<?,?> readStore(final TestActorRef<Shard> shard, final YangInstanceIdentifier id)
             throws ExecutionException, InterruptedException {
         DOMStoreReadTransaction transaction = shard.underlyingActor().getDataStore().newReadOnlyTransaction();
 
@@ -1360,7 +1450,7 @@ public class ShardTest extends AbstractActorTest {
         return node;
     }
 
-    private void writeToStore(TestActorRef<Shard> shard, YangInstanceIdentifier id, NormalizedNode<?,?> node)
+    private void writeToStore(final TestActorRef<Shard> shard, final YangInstanceIdentifier id, final NormalizedNode<?,?> node)
         throws ExecutionException, InterruptedException {
         DOMStoreWriteTransaction transaction = shard.underlyingActor().getDataStore().newWriteOnlyTransaction();
 
@@ -1371,10 +1461,11 @@ public class ShardTest extends AbstractActorTest {
         commitCohort.commit().get();
     }
 
+    @SuppressWarnings("serial")
     private static final class DelegatingShardCreator implements Creator<Shard> {
         private final Creator<Shard> delegate;
 
-        DelegatingShardCreator(Creator<Shard> delegate) {
+        DelegatingShardCreator(final Creator<Shard> delegate) {
             this.delegate = delegate;
         }