BUG-2138: Create DistributedShardFrontend
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / ShardTransactionTest.java
index 1dd824568a81820d21f50fd820d47eeef68c73e9..f8997106f62e6124f0aedfcf0bd6abd4b09966e6 100644 (file)
+/*
+ * Copyright (c) 2014, 2015 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+
 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.assertTrue;
+import static org.mockito.Mockito.doThrow;
+
 import akka.actor.ActorRef;
 import akka.actor.Props;
+import akka.actor.Status.Failure;
 import akka.actor.Terminated;
+import akka.dispatch.Dispatchers;
 import akka.testkit.JavaTestKit;
 import akka.testkit.TestActorRef;
-
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
-
-import org.junit.BeforeClass;
+import com.google.common.base.Throwables;
+import java.util.concurrent.TimeUnit;
+import org.junit.Before;
 import org.junit.Test;
-import org.opendaylight.controller.cluster.datastore.exceptions.UnknownMessageException;
+import org.mockito.InOrder;
+import org.mockito.Mockito;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
+import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
+import org.opendaylight.controller.cluster.datastore.messages.BatchedModificationsReply;
 import org.opendaylight.controller.cluster.datastore.messages.CloseTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CloseTransactionReply;
+import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
 import org.opendaylight.controller.cluster.datastore.messages.DataExists;
 import org.opendaylight.controller.cluster.datastore.messages.DataExistsReply;
-import org.opendaylight.controller.cluster.datastore.messages.DeleteData;
-import org.opendaylight.controller.cluster.datastore.messages.DeleteDataReply;
-import org.opendaylight.controller.cluster.datastore.messages.MergeData;
-import org.opendaylight.controller.cluster.datastore.messages.MergeDataReply;
 import org.opendaylight.controller.cluster.datastore.messages.ReadData;
 import org.opendaylight.controller.cluster.datastore.messages.ReadDataReply;
-import org.opendaylight.controller.cluster.datastore.messages.ReadyTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
-import org.opendaylight.controller.cluster.datastore.messages.WriteData;
-import org.opendaylight.controller.cluster.datastore.messages.WriteDataReply;
-import org.opendaylight.controller.cluster.datastore.modification.CompositeModification;
 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.WriteModification;
+import org.opendaylight.controller.cluster.raft.TestActorFactory;
 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
-import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore;
-import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStoreConfigProperties;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
+import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification;
 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
-import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder;
 
-import scala.concurrent.duration.Duration;
+public class ShardTransactionTest extends AbstractActorTest {
 
-import java.util.Collections;
-import java.util.concurrent.TimeUnit;
+    private static final TransactionType RO = TransactionType.READ_ONLY;
+    private static final TransactionType RW = TransactionType.READ_WRITE;
+    private static final TransactionType WO = TransactionType.WRITE_ONLY;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+    private static final ShardIdentifier SHARD_IDENTIFIER =
+        ShardIdentifier.create("inventory", MEMBER_NAME, "config");
 
-public class ShardTransactionTest extends AbstractActorTest {
-    private static ListeningExecutorService storeExecutor =
-        MoreExecutors.listeningDecorator(MoreExecutors.sameThreadExecutor());
+    private DatastoreContext datastoreContext = DatastoreContext.newBuilder().persistent(false).build();
 
-    private static final InMemoryDOMDataStore store =
-        new InMemoryDOMDataStore("OPER", storeExecutor, MoreExecutors.sameThreadExecutor());
+    private final TestActorFactory actorFactory = new TestActorFactory(getSystem());
 
-    private static final SchemaContext testSchemaContext = TestModel.createTestContext();
+    private TestActorRef<Shard> shard;
+    private ShardDataTree store;
 
-    private static final ShardIdentifier SHARD_IDENTIFIER =
-        ShardIdentifier.builder().memberName("member-1")
-            .shardName("inventory").type("config").build();
+    @Before
+    public void setUp() {
+        shard = actorFactory.createTestActor(Shard.builder().id(SHARD_IDENTIFIER).datastoreContext(datastoreContext)
+                .schemaContext(TestModel.createTestContext()).props()
+                .withDispatcher(Dispatchers.DefaultDispatcherId()));
+        ShardTestKit.waitUntilLeader(shard);
+        store = shard.underlyingActor().getDataStore();
+    }
 
-    private ShardContext shardContext = new ShardContext();
+    private ActorRef newTransactionActor(final TransactionType type,
+            final AbstractShardDataTreeTransaction<?> transaction, final String name) {
+        Props props = ShardTransaction.props(type, transaction, shard, datastoreContext,
+                shard.underlyingActor().getShardMBean());
+        return actorFactory.createActorNoVerify(props, name);
+    }
 
-    @BeforeClass
-    public static void staticSetup() {
-        store.onGlobalContextUpdated(testSchemaContext);
+    private ReadOnlyShardDataTreeTransaction readOnlyTransaction() {
+        return store.newReadOnlyTransaction(nextTransactionId());
+    }
+
+    private ReadWriteShardDataTreeTransaction readWriteTransaction() {
+        return store.newReadWriteTransaction(nextTransactionId());
     }
 
     @Test
     public void testOnReceiveReadData() throws Exception {
-        new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER,
-                    Collections.EMPTY_MAP, new ShardContext()));
-            final Props props = ShardTransaction.props(store.newReadOnlyTransaction(), shard,
-                    testSchemaContext, shardContext);
-            final ActorRef subject = getSystem().actorOf(props, "testReadData");
-
-            new Within(duration("1 seconds")) {
-                @Override
-                protected void run() {
-
-                    subject.tell(
-                        new ReadData(YangInstanceIdentifier.builder().build()).toSerializable(),
-                        getRef());
+        new JavaTestKit(getSystem()) {
+            {
+                testOnReceiveReadData(newTransactionActor(RO, readOnlyTransaction(), "testReadDataRO"));
 
-                    final String out = new ExpectMsg<String>(duration("1 seconds"), "match hint") {
-                        // do not put code outside this method, will run afterwards
-                        @Override
-                        protected String match(Object in) {
-                            if (in.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)) {
-                              if (ReadDataReply.fromSerializable(testSchemaContext,YangInstanceIdentifier.builder().build(), in)
-                                  .getNormalizedNode()!= null) {
-                                    return "match";
-                                }
-                                return null;
-                            } else {
-                                throw noMatch();
-                            }
-                        }
-                    }.get(); // this extracts the received message
-
-                    assertEquals("match", out);
-
-                    expectNoMsg();
-                }
+                testOnReceiveReadData(newTransactionActor(RW, readWriteTransaction(), "testReadDataRW"));
+            }
 
+            private void testOnReceiveReadData(final ActorRef transaction) {
+                transaction.tell(new ReadData(YangInstanceIdentifier.EMPTY, DataStoreVersions.CURRENT_VERSION),
+                        getRef());
+
+                ReadDataReply reply = expectMsgClass(duration("5 seconds"), ReadDataReply.class);
 
-            };
-        }};
+                assertNotNull(reply.getNormalizedNode());
+            }
+        };
     }
 
     @Test
     public void testOnReceiveReadDataWhenDataNotFound() throws Exception {
-        new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER,
-                    Collections.EMPTY_MAP, new ShardContext()));
-            final Props props = ShardTransaction.props( store.newReadOnlyTransaction(), shard,
-                    testSchemaContext, shardContext);
-            final ActorRef subject = getSystem().actorOf(props, "testReadDataWhenDataNotFound");
-
-            new Within(duration("1 seconds")) {
-                @Override
-                protected void run() {
-
-                    subject.tell(
-                        new ReadData(TestModel.TEST_PATH).toSerializable(),
-                        getRef());
+        new JavaTestKit(getSystem()) {
+            {
+                testOnReceiveReadDataWhenDataNotFound(
+                        newTransactionActor(RO, readOnlyTransaction(), "testReadDataWhenDataNotFoundRO"));
 
-                    final String out = new ExpectMsg<String>(duration("1 seconds"), "match hint") {
-                        // do not put code outside this method, will run afterwards
-                        @Override
-                        protected String match(Object in) {
-                            if (in.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)) {
-                                if (ReadDataReply.fromSerializable(testSchemaContext,TestModel.TEST_PATH, in)
-                                    .getNormalizedNode()
-                                    == null) {
-                                    return "match";
-                                }
-                                return null;
-                            } else {
-                                throw noMatch();
-                            }
-                        }
-                    }.get(); // this extracts the received message
-
-                    assertEquals("match", out);
-
-                    expectNoMsg();
-                }
+                testOnReceiveReadDataWhenDataNotFound(
+                        newTransactionActor(RW, readWriteTransaction(), "testReadDataWhenDataNotFoundRW"));
+            }
 
+            private void testOnReceiveReadDataWhenDataNotFound(final ActorRef transaction) {
+                transaction.tell(new ReadData(TestModel.TEST_PATH, DataStoreVersions.CURRENT_VERSION), getRef());
 
-            };
-        }};
+                ReadDataReply reply = expectMsgClass(duration("5 seconds"), ReadDataReply.class);
+
+                assertTrue(reply.getNormalizedNode() == null);
+            }
+        };
     }
 
     @Test
     public void testOnReceiveDataExistsPositive() throws Exception {
-        new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER,
-                    Collections.EMPTY_MAP, new ShardContext()));
-            final Props props = ShardTransaction.props(store.newReadOnlyTransaction(), shard,
-                    testSchemaContext, shardContext);
-            final ActorRef subject = getSystem().actorOf(props, "testDataExistsPositive");
-
-            new Within(duration("1 seconds")) {
-                @Override
-                protected void run() {
-
-                    subject.tell(
-                        new DataExists(YangInstanceIdentifier.builder().build()).toSerializable(),
-                        getRef());
+        new JavaTestKit(getSystem()) {
+            {
+                testOnReceiveDataExistsPositive(
+                        newTransactionActor(RO, readOnlyTransaction(), "testDataExistsPositiveRO"));
 
-                    final String out = new ExpectMsg<String>(duration("1 seconds"), "match hint") {
-                        // do not put code outside this method, will run afterwards
-                        @Override
-                        protected String match(Object in) {
-                            if (in.getClass().equals(DataExistsReply.SERIALIZABLE_CLASS)) {
-                                if (DataExistsReply.fromSerializable(in)
-                                    .exists()) {
-                                    return "match";
-                                }
-                                return null;
-                            } else {
-                                throw noMatch();
-                            }
-                        }
-                    }.get(); // this extracts the received message
-
-                    assertEquals("match", out);
-
-                    expectNoMsg();
-                }
+                testOnReceiveDataExistsPositive(
+                        newTransactionActor(RW, readWriteTransaction(), "testDataExistsPositiveRW"));
+            }
 
+            private void testOnReceiveDataExistsPositive(final ActorRef transaction) {
+                transaction.tell(new DataExists(YangInstanceIdentifier.EMPTY, DataStoreVersions.CURRENT_VERSION),
+                        getRef());
 
-            };
-        }};
+                DataExistsReply reply = expectMsgClass(duration("5 seconds"), DataExistsReply.class);
+
+                assertTrue(reply.exists());
+            }
+        };
     }
 
     @Test
     public void testOnReceiveDataExistsNegative() throws Exception {
-        new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER,
-                    Collections.EMPTY_MAP, new ShardContext()));
-            final Props props = ShardTransaction.props(store.newReadOnlyTransaction(), shard,
-                    testSchemaContext, shardContext);
-            final ActorRef subject = getSystem().actorOf(props, "testDataExistsNegative");
-
-            new Within(duration("1 seconds")) {
-                @Override
-                protected void run() {
-
-                    subject.tell(
-                        new DataExists(TestModel.TEST_PATH).toSerializable(),
-                        getRef());
+        new JavaTestKit(getSystem()) {
+            {
+                testOnReceiveDataExistsNegative(
+                        newTransactionActor(RO, readOnlyTransaction(), "testDataExistsNegativeRO"));
 
-                    final String out = new ExpectMsg<String>(duration("1 seconds"), "match hint") {
-                        // do not put code outside this method, will run afterwards
-                        @Override
-                        protected String match(Object in) {
-                            if (in.getClass().equals(DataExistsReply.SERIALIZABLE_CLASS)) {
-                                if (!DataExistsReply.fromSerializable(in)
-                                    .exists()) {
-                                    return "match";
-                                }
-                                return null;
-                            } else {
-                                throw noMatch();
-                            }
-                        }
-                    }.get(); // this extracts the received message
-
-                    assertEquals("match", out);
-
-                    expectNoMsg();
-                }
+                testOnReceiveDataExistsNegative(
+                        newTransactionActor(RW, readWriteTransaction(), "testDataExistsNegativeRW"));
+            }
 
+            private void testOnReceiveDataExistsNegative(final ActorRef transaction) {
+                transaction.tell(new DataExists(TestModel.TEST_PATH, DataStoreVersions.CURRENT_VERSION), getRef());
 
-            };
-        }};
+                DataExistsReply reply = expectMsgClass(duration("5 seconds"), DataExistsReply.class);
+
+                assertFalse(reply.exists());
+            }
+        };
     }
 
-    private void assertModification(final ActorRef subject,
-        final Class<? extends Modification> modificationType) {
-        new JavaTestKit(getSystem()) {{
-            new Within(duration("1 seconds")) {
-                @Override
-                protected void run() {
-                    subject
-                        .tell(new ShardTransaction.GetCompositedModification(),
-                            getRef());
-
-                    final CompositeModification compositeModification =
-                        new ExpectMsg<CompositeModification>(duration("1 seconds"), "match hint") {
-                            // do not put code outside this method, will run afterwards
-                            @Override
-                            protected CompositeModification match(Object in) {
-                                if (in instanceof ShardTransaction.GetCompositeModificationReply) {
-                                    return ((ShardTransaction.GetCompositeModificationReply) in)
-                                        .getModification();
-                                } else {
-                                    throw noMatch();
-                                }
-                            }
-                        }.get(); // this extracts the received message
-
-                    assertTrue(
-                        compositeModification.getModifications().size() == 1);
-                    assertEquals(modificationType,
-                        compositeModification.getModifications().get(0)
-                            .getClass());
+    @Test
+    public void testOnReceiveBatchedModifications() throws Exception {
+        new JavaTestKit(getSystem()) {
+            {
+                ShardDataTreeTransactionParent parent = Mockito.mock(ShardDataTreeTransactionParent.class);
+                DataTreeModification mockModification = Mockito.mock(DataTreeModification.class);
+                ReadWriteShardDataTreeTransaction mockWriteTx = new ReadWriteShardDataTreeTransaction(parent,
+                        nextTransactionId(), mockModification);
+                final ActorRef transaction = newTransactionActor(RW, mockWriteTx, "testOnReceiveBatchedModifications");
+
+                YangInstanceIdentifier writePath = TestModel.TEST_PATH;
+                NormalizedNode<?, ?> writeData = ImmutableContainerNodeBuilder.create()
+                        .withNodeIdentifier(new YangInstanceIdentifier.NodeIdentifier(TestModel.TEST_QNAME))
+                        .withChild(ImmutableNodes.leafNode(TestModel.DESC_QNAME, "foo")).build();
+
+                YangInstanceIdentifier mergePath = TestModel.OUTER_LIST_PATH;
+                NormalizedNode<?, ?> mergeData = ImmutableContainerNodeBuilder.create()
+                        .withNodeIdentifier(new YangInstanceIdentifier.NodeIdentifier(TestModel.OUTER_LIST_QNAME))
+                        .build();
+
+                YangInstanceIdentifier deletePath = TestModel.TEST_PATH;
+
+                BatchedModifications batched = new BatchedModifications(nextTransactionId(),
+                        DataStoreVersions.CURRENT_VERSION);
+                batched.addModification(new WriteModification(writePath, writeData));
+                batched.addModification(new MergeModification(mergePath, mergeData));
+                batched.addModification(new DeleteModification(deletePath));
+
+                transaction.tell(batched, getRef());
+
+                BatchedModificationsReply reply = expectMsgClass(duration("5 seconds"),
+                        BatchedModificationsReply.class);
+                assertEquals("getNumBatched", 3, reply.getNumBatched());
+
+                InOrder inOrder = Mockito.inOrder(mockModification);
+                inOrder.verify(mockModification).write(writePath, writeData);
+                inOrder.verify(mockModification).merge(mergePath, mergeData);
+                inOrder.verify(mockModification).delete(deletePath);
+            }
+        };
+    }
 
-                }
-            };
-        }};
+    @Test
+    public void testOnReceiveBatchedModificationsReadyWithoutImmediateCommit() throws Exception {
+        new JavaTestKit(getSystem()) {
+            {
+                final ActorRef transaction = newTransactionActor(WO, readWriteTransaction(),
+                        "testOnReceiveBatchedModificationsReadyWithoutImmediateCommit");
+
+                JavaTestKit watcher = new JavaTestKit(getSystem());
+                watcher.watch(transaction);
+
+                YangInstanceIdentifier writePath = TestModel.TEST_PATH;
+                NormalizedNode<?, ?> writeData = ImmutableContainerNodeBuilder.create()
+                        .withNodeIdentifier(new YangInstanceIdentifier.NodeIdentifier(TestModel.TEST_QNAME))
+                        .withChild(ImmutableNodes.leafNode(TestModel.DESC_QNAME, "foo")).build();
+
+                final TransactionIdentifier tx1 = nextTransactionId();
+                BatchedModifications batched = new BatchedModifications(tx1, DataStoreVersions.CURRENT_VERSION);
+                batched.addModification(new WriteModification(writePath, writeData));
+
+                transaction.tell(batched, getRef());
+                BatchedModificationsReply reply = expectMsgClass(duration("5 seconds"),
+                        BatchedModificationsReply.class);
+                assertEquals("getNumBatched", 1, reply.getNumBatched());
+
+                batched = new BatchedModifications(tx1, DataStoreVersions.CURRENT_VERSION);
+                batched.setReady(true);
+                batched.setTotalMessagesSent(2);
+
+                transaction.tell(batched, getRef());
+                expectMsgClass(duration("5 seconds"), ReadyTransactionReply.class);
+                watcher.expectMsgClass(duration("5 seconds"), Terminated.class);
+            }
+        };
     }
 
     @Test
-    public void testOnReceiveWriteData() throws Exception {
-        new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER,
-                    Collections.EMPTY_MAP, new ShardContext()));
-            final Props props = ShardTransaction.props(store.newWriteOnlyTransaction(), shard,
-                    testSchemaContext, shardContext);
-            final ActorRef subject =
-                getSystem().actorOf(props, "testWriteData");
-
-            new Within(duration("1 seconds")) {
-                @Override
-                protected void run() {
-
-                    subject.tell(new WriteData(TestModel.TEST_PATH,
-                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), TestModel.createTestContext()).toSerializable(),
-                        getRef());
+    public void testOnReceiveBatchedModificationsReadyWithImmediateCommit() throws Exception {
+        new JavaTestKit(getSystem()) {
+            {
+                final ActorRef transaction = newTransactionActor(WO, readWriteTransaction(),
+                        "testOnReceiveBatchedModificationsReadyWithImmediateCommit");
+
+                JavaTestKit watcher = new JavaTestKit(getSystem());
+                watcher.watch(transaction);
+
+                YangInstanceIdentifier writePath = TestModel.TEST_PATH;
+                NormalizedNode<?, ?> writeData = ImmutableContainerNodeBuilder.create()
+                        .withNodeIdentifier(new YangInstanceIdentifier.NodeIdentifier(TestModel.TEST_QNAME))
+                        .withChild(ImmutableNodes.leafNode(TestModel.DESC_QNAME, "foo")).build();
+
+                BatchedModifications batched = new BatchedModifications(nextTransactionId(),
+                        DataStoreVersions.CURRENT_VERSION);
+                batched.addModification(new WriteModification(writePath, writeData));
+                batched.setReady(true);
+                batched.setDoCommitOnReady(true);
+                batched.setTotalMessagesSent(1);
+
+                transaction.tell(batched, getRef());
+                expectMsgClass(duration("5 seconds"), CommitTransactionReply.class);
+                watcher.expectMsgClass(duration("5 seconds"), Terminated.class);
+            }
+        };
+    }
 
-                    final String out = new ExpectMsg<String>(duration("1 seconds"), "match hint") {
-                        // do not put code outside this method, will run afterwards
-                        @Override
-                        protected String match(Object in) {
-                            if (in.getClass().equals(WriteDataReply.SERIALIZABLE_CLASS)) {
-                                return "match";
-                            } else {
-                                throw noMatch();
-                            }
-                        }
-                    }.get(); // this extracts the received message
-
-                    assertEquals("match", out);
-
-                    assertModification(subject, WriteModification.class);
-                    expectNoMsg();
-                }
+    @Test(expected = TestException.class)
+    public void testOnReceiveBatchedModificationsFailure() throws Exception {
+        new JavaTestKit(getSystem()) {
+            {
 
+                ShardDataTreeTransactionParent parent = Mockito.mock(ShardDataTreeTransactionParent.class);
+                DataTreeModification mockModification = Mockito.mock(DataTreeModification.class);
+                ReadWriteShardDataTreeTransaction mockWriteTx = new ReadWriteShardDataTreeTransaction(parent,
+                        nextTransactionId(), mockModification);
+                final ActorRef transaction = newTransactionActor(RW, mockWriteTx,
+                        "testOnReceiveBatchedModificationsFailure");
 
-            };
-        }};
-    }
+                JavaTestKit watcher = new JavaTestKit(getSystem());
+                watcher.watch(transaction);
 
-    @Test
-    public void testOnReceiveMergeData() throws Exception {
-        new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER,
-                    Collections.EMPTY_MAP, new ShardContext()));
-            final Props props = ShardTransaction.props(store.newReadWriteTransaction(), shard,
-                    testSchemaContext, shardContext);
-            final ActorRef subject =
-                getSystem().actorOf(props, "testMergeData");
-
-            new Within(duration("1 seconds")) {
-                @Override
-                protected void run() {
-
-                    subject.tell(new MergeData(TestModel.TEST_PATH,
-                        ImmutableNodes.containerNode(TestModel.TEST_QNAME), testSchemaContext).toSerializable(),
-                        getRef());
+                YangInstanceIdentifier path = TestModel.TEST_PATH;
+                ContainerNode node = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
+
+                doThrow(new TestException()).when(mockModification).write(path, node);
 
-                    final String out = new ExpectMsg<String>(duration("500 milliseconds"), "match hint") {
-                        // do not put code outside this method, will run afterwards
-                        @Override
-                        protected String match(Object in) {
-                            if (in.getClass().equals(MergeDataReply.SERIALIZABLE_CLASS)) {
-                                return "match";
-                            } else {
-                                throw noMatch();
-                            }
-                        }
-                    }.get(); // this extracts the received message
+                final TransactionIdentifier tx1 = nextTransactionId();
+                BatchedModifications batched = new BatchedModifications(tx1, DataStoreVersions.CURRENT_VERSION);
+                batched.addModification(new WriteModification(path, node));
 
-                    assertEquals("match", out);
+                transaction.tell(batched, getRef());
+                expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
 
-                    assertModification(subject, MergeModification.class);
+                batched = new BatchedModifications(tx1, DataStoreVersions.CURRENT_VERSION);
+                batched.setReady(true);
+                batched.setTotalMessagesSent(2);
 
-                    expectNoMsg();
+                transaction.tell(batched, getRef());
+                Failure failure = expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
+                watcher.expectMsgClass(duration("5 seconds"), Terminated.class);
+
+                if (failure != null) {
+                    Throwables.propagateIfInstanceOf(failure.cause(), Exception.class);
+                    Throwables.propagate(failure.cause());
                 }
+            }
+        };
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void testOnReceiveBatchedModificationsReadyWithIncorrectTotalMessageCount() throws Exception {
+        new JavaTestKit(getSystem()) {
+            {
+
+                final ActorRef transaction = newTransactionActor(WO, readWriteTransaction(),
+                        "testOnReceiveBatchedModificationsReadyWithIncorrectTotalMessageCount");
 
+                JavaTestKit watcher = new JavaTestKit(getSystem());
+                watcher.watch(transaction);
 
-            };
-        }};
+                BatchedModifications batched = new BatchedModifications(nextTransactionId(),
+                        DataStoreVersions.CURRENT_VERSION);
+                batched.setReady(true);
+                batched.setTotalMessagesSent(2);
+
+                transaction.tell(batched, getRef());
+
+                Failure failure = expectMsgClass(duration("5 seconds"), akka.actor.Status.Failure.class);
+                watcher.expectMsgClass(duration("5 seconds"), Terminated.class);
+
+                if (failure != null) {
+                    Throwables.propagateIfInstanceOf(failure.cause(), Exception.class);
+                    Throwables.propagate(failure.cause());
+                }
+            }
+        };
     }
 
     @Test
-    public void testOnReceiveDeleteData() throws Exception {
-        new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER,
-                    Collections.EMPTY_MAP, new ShardContext()));
-            final Props props = ShardTransaction.props( store.newWriteOnlyTransaction(), shard,
-                    testSchemaContext, shardContext);
-            final ActorRef subject =
-                getSystem().actorOf(props, "testDeleteData");
-
-            new Within(duration("1 seconds")) {
-                @Override
-                protected void run() {
-
-                    subject.tell(new DeleteData(TestModel.TEST_PATH).toSerializable(), getRef());
-
-                    final String out = new ExpectMsg<String>(duration("1 seconds"), "match hint") {
-                        // do not put code outside this method, will run afterwards
-                        @Override
-                        protected String match(Object in) {
-                            if (in.getClass().equals(DeleteDataReply.SERIALIZABLE_CLASS)) {
-                                return "match";
-                            } else {
-                                throw noMatch();
-                            }
-                        }
-                    }.get(); // this extracts the received message
-
-                    assertEquals("match", out);
-
-                    assertModification(subject, DeleteModification.class);
-                    expectNoMsg();
-                }
+    public void testReadWriteTxOnReceiveCloseTransaction() throws Exception {
+        new JavaTestKit(getSystem()) {
+            {
+                final ActorRef transaction = newTransactionActor(RW, readWriteTransaction(),
+                        "testReadWriteTxOnReceiveCloseTransaction");
 
+                watch(transaction);
 
-            };
-        }};
-    }
+                transaction.tell(new CloseTransaction().toSerializable(), getRef());
 
+                expectMsgClass(duration("3 seconds"), CloseTransactionReply.class);
+                expectTerminated(duration("3 seconds"), transaction);
+            }
+        };
+    }
 
     @Test
-    public void testOnReceiveReadyTransaction() throws Exception {
-        new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER,
-                    Collections.EMPTY_MAP, new ShardContext()));
-            final Props props = ShardTransaction.props( store.newReadWriteTransaction(), shard,
-                    testSchemaContext, shardContext);
-            final ActorRef subject =
-                getSystem().actorOf(props, "testReadyTransaction");
-
-            new Within(duration("1 seconds")) {
-                @Override
-                protected void run() {
-
-                    subject.tell(new ReadyTransaction().toSerializable(), getRef());
-
-                    final String out = new ExpectMsg<String>(duration("1 seconds"), "match hint") {
-                        // do not put code outside this method, will run afterwards
-                        @Override
-                        protected String match(Object in) {
-                            if (in.getClass().equals(ReadyTransactionReply.SERIALIZABLE_CLASS)) {
-                                return "match";
-                            } else {
-                                throw noMatch();
-                            }
-                        }
-                    }.get(); // this extracts the received message
-
-                    assertEquals("match", out);
-
-                    expectNoMsg();
-                }
+    public void testWriteOnlyTxOnReceiveCloseTransaction() throws Exception {
+        new JavaTestKit(getSystem()) {
+            {
+                final ActorRef transaction = newTransactionActor(WO, readWriteTransaction(),
+                        "testWriteTxOnReceiveCloseTransaction");
 
+                watch(transaction);
 
-            };
-        }};
+                transaction.tell(new CloseTransaction().toSerializable(), getRef());
 
+                expectMsgClass(duration("3 seconds"), CloseTransactionReply.class);
+                expectTerminated(duration("3 seconds"), transaction);
+            }
+        };
     }
 
     @Test
-    public void testOnReceiveCloseTransaction() throws Exception {
-        new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER,
-                    Collections.EMPTY_MAP, new ShardContext()));
-            final Props props = ShardTransaction.props(store.newReadWriteTransaction(), shard,
-                    testSchemaContext, shardContext);
-            final ActorRef subject =
-                getSystem().actorOf(props, "testCloseTransaction");
-
-            watch(subject);
-
-            new Within(duration("6 seconds")) {
-                @Override
-                protected void run() {
-
-                    subject.tell(new CloseTransaction().toSerializable(), getRef());
-
-                    final String out = new ExpectMsg<String>(duration("3 seconds"), "match hint") {
-                        // do not put code outside this method, will run afterwards
-                        @Override
-                        protected String match(Object in) {
-                            System.out.println("!!!IN match 1: "+(in!=null?in.getClass():"NULL"));
-                            if (in.getClass().equals(CloseTransactionReply.SERIALIZABLE_CLASS)) {
-                                return "match";
-                            } else {
-                                throw noMatch();
-                            }
-                        }
-                    }.get(); // this extracts the received message
-
-                    assertEquals("match", out);
-
-                    final String termination = new ExpectMsg<String>(duration("3 seconds"), "match hint") {
-                        // do not put code outside this method, will run afterwards
-                        @Override
-                        protected String match(Object in) {
-                            System.out.println("!!!IN match 2: "+(in!=null?in.getClass():"NULL"));
-                            if (in instanceof Terminated) {
-                                return "match";
-                            } else {
-                                throw noMatch();
-                            }
-                        }
-                    }.get(); // this extracts the received message
-
-                    assertEquals("match", termination);
-                }
-            };
-        }};
-    }
+    public void testReadOnlyTxOnReceiveCloseTransaction() throws Exception {
+        new JavaTestKit(getSystem()) {
+            {
+                final ActorRef transaction = newTransactionActor(TransactionType.READ_ONLY, readOnlyTransaction(),
+                        "testReadOnlyTxOnReceiveCloseTransaction");
 
-    @Test(expected=UnknownMessageException.class)
-    public void testNegativePerformingWriteOperationOnReadTransaction() throws Exception {
-        final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER,
-                Collections.EMPTY_MAP, new ShardContext()));
-        final Props props = ShardTransaction.props(store.newReadOnlyTransaction(), shard,
-                testSchemaContext, shardContext);
-        final TestActorRef subject = TestActorRef.apply(props,getSystem());
+                watch(transaction);
 
-        subject.receive(new DeleteData(TestModel.TEST_PATH).toSerializable(), ActorRef.noSender());
+                transaction.tell(new CloseTransaction().toSerializable(), getRef());
+
+                expectMsgClass(duration("3 seconds"), Terminated.class);
+            }
+        };
     }
 
     @Test
     public void testShardTransactionInactivity() {
+        datastoreContext = DatastoreContext.newBuilder().shardTransactionIdleTimeout(
+                500, TimeUnit.MILLISECONDS).build();
 
-        shardContext = new ShardContext(InMemoryDOMDataStoreConfigProperties.getDefault(),
-                Duration.create(500, TimeUnit.MILLISECONDS));
-
-        new JavaTestKit(getSystem()) {{
-            final ActorRef shard = getSystem().actorOf(Shard.props(SHARD_IDENTIFIER,
-                    Collections.EMPTY_MAP, new ShardContext()));
-            final Props props = ShardTransaction.props(store.newReadWriteTransaction(), shard,
-                    testSchemaContext, shardContext);
-            final ActorRef subject =
-                getSystem().actorOf(props, "testShardTransactionInactivity");
-
-            watch(subject);
-
-            // The shard Tx actor should receive a ReceiveTimeout message and self-destruct.
-
-            final String termination = new ExpectMsg<String>(duration("3 seconds"), "match hint") {
-                // do not put code outside this method, will run afterwards
-                @Override
-                protected String match(Object in) {
-                    if (in instanceof Terminated) {
-                        return "match";
-                    } else {
-                        throw noMatch();
-                    }
-                }
-            }.get(); // this extracts the received message
+        new JavaTestKit(getSystem()) {
+            {
+                final ActorRef transaction = newTransactionActor(RW, readWriteTransaction(),
+                        "testShardTransactionInactivity");
+
+                watch(transaction);
+
+                expectMsgClass(duration("3 seconds"), Terminated.class);
+            }
+        };
+    }
 
-            assertEquals("match", termination);
-        }};
+    public static class TestException extends RuntimeException {
+        private static final long serialVersionUID = 1L;
     }
 }