Remove CompositeModification field in ShardWriteTransaction
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / test / java / org / opendaylight / controller / cluster / datastore / ShardTransactionTest.java
index c9335f378a8662e7b7ceb488961fb49588d50baf..314f497a2377174f9857e003eaca4c588eebb609 100644 (file)
@@ -1,3 +1,11 @@
+/*
+ * 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;
@@ -11,14 +19,10 @@ import akka.actor.Status.Failure;
 import akka.actor.Terminated;
 import akka.testkit.JavaTestKit;
 import akka.testkit.TestActorRef;
-import com.google.common.util.concurrent.MoreExecutors;
-import java.util.Collections;
 import java.util.concurrent.TimeUnit;
-import org.junit.Before;
 import org.junit.Test;
 import org.mockito.InOrder;
 import org.mockito.Mockito;
-import org.opendaylight.controller.cluster.datastore.ShardWriteTransaction.GetCompositeModificationReply;
 import org.opendaylight.controller.cluster.datastore.exceptions.UnknownMessageException;
 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
@@ -26,6 +30,7 @@ import org.opendaylight.controller.cluster.datastore.messages.BatchedModificatio
 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.CreateSnapshot;
 import org.opendaylight.controller.cluster.datastore.messages.DataExists;
 import org.opendaylight.controller.cluster.datastore.messages.DataExistsReply;
@@ -39,23 +44,20 @@ 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.datastore.node.NormalizedNodeToNodeCodec;
 import org.opendaylight.controller.cluster.datastore.node.NormalizedNodeToNodeCodec.Encoded;
 import org.opendaylight.controller.cluster.datastore.utils.SerializationUtils;
 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
 import org.opendaylight.controller.md.cluster.datastore.model.TestModel;
-import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore;
 import org.opendaylight.controller.protobuff.messages.transaction.ShardTransactionMessages;
-import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransaction;
-import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
 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.api.schema.tree.TreeType;
 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
 import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
@@ -63,6 +65,9 @@ import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 public class ShardTransactionTest extends AbstractActorTest {
 
     private static final SchemaContext testSchemaContext = TestModel.createTestContext();
+    private static final TransactionType RO = TransactionType.READ_ONLY;
+    private static final TransactionType RW = TransactionType.READ_WRITE;
+    private static final TransactionType WO = TransactionType.WRITE_ONLY;
 
     private static final ShardIdentifier SHARD_IDENTIFIER =
         ShardIdentifier.builder().memberName("member-1")
@@ -72,46 +77,50 @@ public class ShardTransactionTest extends AbstractActorTest {
 
     private final ShardStats shardStats = new ShardStats(SHARD_IDENTIFIER.toString(), "DataStore");
 
-    private final InMemoryDOMDataStore store =
-            new InMemoryDOMDataStore("OPER", MoreExecutors.sameThreadExecutor());
+    private final ShardDataTree store = new ShardDataTree(testSchemaContext, TreeType.OPERATIONAL);
 
-    @Before
-    public void setup() {
-        store.onGlobalContextUpdated(testSchemaContext);
-    }
+    private int txCounter = 0;
 
-    private ActorRef createShard(){
-        return getSystem().actorOf(Shard.props(SHARD_IDENTIFIER,
-            Collections.<String, String>emptyMap(), datastoreContext, TestModel.createTestContext()));
+    private ActorRef createShard() {
+        return getSystem().actorOf(Shard.builder().id(SHARD_IDENTIFIER).datastoreContext(datastoreContext).
+                schemaContext(TestModel.createTestContext()).props());
     }
 
-    private ActorRef newTransactionActor(DOMStoreTransaction transaction, String name) {
-        return newTransactionActor(transaction, name, DataStoreVersions.CURRENT_VERSION);
+    private ActorRef newTransactionActor(TransactionType type, AbstractShardDataTreeTransaction<?> transaction, String name) {
+        return newTransactionActor(type, transaction, name, DataStoreVersions.CURRENT_VERSION);
     }
 
-    private ActorRef newTransactionActor(DOMStoreTransaction transaction, String name, short version) {
-        return newTransactionActor(transaction, null, name, version);
+    private ActorRef newTransactionActor(TransactionType type, AbstractShardDataTreeTransaction<?> transaction, String name, short version) {
+        return newTransactionActor(type, transaction, null, name, version);
     }
 
-    private ActorRef newTransactionActor(DOMStoreTransaction transaction, ActorRef shard, String name) {
-        return newTransactionActor(transaction, null, name, DataStoreVersions.CURRENT_VERSION);
+    private ActorRef newTransactionActor(TransactionType type, AbstractShardDataTreeTransaction<?> transaction, ActorRef shard, String name) {
+        return newTransactionActor(type, transaction, null, name, DataStoreVersions.CURRENT_VERSION);
     }
 
-    private ActorRef newTransactionActor(DOMStoreTransaction transaction, ActorRef shard, String name,
+    private ActorRef newTransactionActor(TransactionType type, AbstractShardDataTreeTransaction<?> transaction, ActorRef shard, String name,
             short version) {
-        Props props = ShardTransaction.props(transaction, shard != null ? shard : createShard(),
+        Props props = ShardTransaction.props(type, transaction, shard != null ? shard : createShard(),
                 datastoreContext, shardStats, "txn", version);
         return getSystem().actorOf(props, name);
     }
 
+    private ReadOnlyShardDataTreeTransaction readOnlyTransaction() {
+        return store.newReadOnlyTransaction("test-ro-" + String.valueOf(txCounter++), null);
+    }
+
+    private ReadWriteShardDataTreeTransaction readWriteTransaction() {
+        return store.newReadWriteTransaction("test-rw-" + String.valueOf(txCounter++), null);
+    }
+
     @Test
     public void testOnReceiveReadData() throws Exception {
         new JavaTestKit(getSystem()) {{
             final ActorRef shard = createShard();
 
-            testOnReceiveReadData(newTransactionActor(store.newReadOnlyTransaction(), shard, "testReadDataRO"));
+            testOnReceiveReadData(newTransactionActor(RO, readOnlyTransaction(), shard, "testReadDataRO"));
 
-            testOnReceiveReadData(newTransactionActor(store.newReadWriteTransaction(), shard, "testReadDataRW"));
+            testOnReceiveReadData(newTransactionActor(RW, readWriteTransaction(), shard, "testReadDataRW"));
         }
 
         private void testOnReceiveReadData(final ActorRef transaction) {
@@ -139,10 +148,10 @@ public class ShardTransactionTest extends AbstractActorTest {
             final ActorRef shard = createShard();
 
             testOnReceiveReadDataWhenDataNotFound(newTransactionActor(
-                    store.newReadOnlyTransaction(), shard, "testReadDataWhenDataNotFoundRO"));
+                    RO, readOnlyTransaction(), shard, "testReadDataWhenDataNotFoundRO"));
 
             testOnReceiveReadDataWhenDataNotFound(newTransactionActor(
-                    store.newReadWriteTransaction(), shard, "testReadDataWhenDataNotFoundRW"));
+                    RW, readWriteTransaction(), shard, "testReadDataWhenDataNotFoundRW"));
         }
 
         private void testOnReceiveReadDataWhenDataNotFound(final ActorRef transaction) {
@@ -166,7 +175,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testOnReceiveReadDataHeliumR1() throws Exception {
         new JavaTestKit(getSystem()) {{
-            ActorRef transaction = newTransactionActor(store.newReadOnlyTransaction(),
+            ActorRef transaction = newTransactionActor(RO, readOnlyTransaction(),
                     "testOnReceiveReadDataHeliumR1", DataStoreVersions.HELIUM_1_VERSION);
 
             transaction.tell(new ReadData(YangInstanceIdentifier.builder().build()).toSerializable(),
@@ -184,10 +193,10 @@ public class ShardTransactionTest extends AbstractActorTest {
         new JavaTestKit(getSystem()) {{
             final ActorRef shard = createShard();
 
-            testOnReceiveDataExistsPositive(newTransactionActor(store.newReadOnlyTransaction(), shard,
+            testOnReceiveDataExistsPositive(newTransactionActor(RO, readOnlyTransaction(), shard,
                     "testDataExistsPositiveRO"));
 
-            testOnReceiveDataExistsPositive(newTransactionActor(store.newReadWriteTransaction(), shard,
+            testOnReceiveDataExistsPositive(newTransactionActor(RW, readWriteTransaction(), shard,
                     "testDataExistsPositiveRW"));
         }
 
@@ -214,10 +223,10 @@ public class ShardTransactionTest extends AbstractActorTest {
         new JavaTestKit(getSystem()) {{
             final ActorRef shard = createShard();
 
-            testOnReceiveDataExistsNegative(newTransactionActor(store.newReadOnlyTransaction(), shard,
+            testOnReceiveDataExistsNegative(newTransactionActor(RO, readOnlyTransaction(), shard,
                     "testDataExistsNegativeRO"));
 
-            testOnReceiveDataExistsNegative(newTransactionActor(store.newReadWriteTransaction(), shard,
+            testOnReceiveDataExistsNegative(newTransactionActor(RW, readWriteTransaction(), shard,
                     "testDataExistsNegativeRW"));
         }
 
@@ -238,23 +247,10 @@ public class ShardTransactionTest extends AbstractActorTest {
         }};
     }
 
-    private void assertModification(final ActorRef subject,
-        final Class<? extends Modification> modificationType) {
-        new JavaTestKit(getSystem()) {{
-            subject.tell(new ShardWriteTransaction.GetCompositedModification(), getRef());
-
-            CompositeModification compositeModification = expectMsgClass(duration("3 seconds"),
-                    GetCompositeModificationReply.class).getModification();
-
-            assertTrue(compositeModification.getModifications().size() == 1);
-            assertEquals(modificationType, compositeModification.getModifications().get(0).getClass());
-        }};
-    }
-
     @Test
-    public void testOnReceiveWriteData() throws Exception {
+    public void testOnReceiveWriteData() {
         new JavaTestKit(getSystem()) {{
-            final ActorRef transaction = newTransactionActor(store.newWriteOnlyTransaction(),
+            final ActorRef transaction = newTransactionActor(WO, readWriteTransaction(),
                     "testOnReceiveWriteData");
 
             transaction.tell(new WriteData(TestModel.TEST_PATH,
@@ -263,8 +259,6 @@ public class ShardTransactionTest extends AbstractActorTest {
 
             expectMsgClass(duration("5 seconds"), ShardTransactionMessages.WriteDataReply.class);
 
-            assertModification(transaction, WriteModification.class);
-
             // unserialized write
             transaction.tell(new WriteData(TestModel.TEST_PATH,
                 ImmutableNodes.containerNode(TestModel.TEST_QNAME), DataStoreVersions.CURRENT_VERSION),
@@ -275,9 +269,9 @@ public class ShardTransactionTest extends AbstractActorTest {
     }
 
     @Test
-    public void testOnReceiveHeliumR1WriteData() throws Exception {
+    public void testOnReceiveHeliumR1WriteData() {
         new JavaTestKit(getSystem()) {{
-            final ActorRef transaction = newTransactionActor(store.newWriteOnlyTransaction(),
+            final ActorRef transaction = newTransactionActor(WO, readWriteTransaction(),
                     "testOnReceiveHeliumR1WriteData", DataStoreVersions.HELIUM_1_VERSION);
 
             Encoded encoded = new NormalizedNodeToNodeCodec(null).encode(TestModel.TEST_PATH,
@@ -289,15 +283,13 @@ public class ShardTransactionTest extends AbstractActorTest {
             transaction.tell(serialized, getRef());
 
             expectMsgClass(duration("5 seconds"), ShardTransactionMessages.WriteDataReply.class);
-
-            assertModification(transaction, WriteModification.class);
         }};
     }
 
     @Test
-    public void testOnReceiveMergeData() throws Exception {
+    public void testOnReceiveMergeData() {
         new JavaTestKit(getSystem()) {{
-            final ActorRef transaction = newTransactionActor(store.newReadWriteTransaction(),
+            final ActorRef transaction = newTransactionActor(RW, readWriteTransaction(),
                     "testMergeData");
 
             transaction.tell(new MergeData(TestModel.TEST_PATH,
@@ -306,8 +298,6 @@ public class ShardTransactionTest extends AbstractActorTest {
 
             expectMsgClass(duration("5 seconds"), ShardTransactionMessages.MergeDataReply.class);
 
-            assertModification(transaction, MergeModification.class);
-
             //unserialized merge
             transaction.tell(new MergeData(TestModel.TEST_PATH,
                 ImmutableNodes.containerNode(TestModel.TEST_QNAME), DataStoreVersions.CURRENT_VERSION),
@@ -320,7 +310,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testOnReceiveHeliumR1MergeData() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef transaction = newTransactionActor(store.newWriteOnlyTransaction(),
+            final ActorRef transaction = newTransactionActor(WO, readWriteTransaction(),
                     "testOnReceiveHeliumR1MergeData", DataStoreVersions.HELIUM_1_VERSION);
 
             Encoded encoded = new NormalizedNodeToNodeCodec(null).encode(TestModel.TEST_PATH,
@@ -332,15 +322,13 @@ public class ShardTransactionTest extends AbstractActorTest {
             transaction.tell(serialized, getRef());
 
             expectMsgClass(duration("5 seconds"), ShardTransactionMessages.MergeDataReply.class);
-
-            assertModification(transaction, MergeModification.class);
         }};
     }
 
     @Test
     public void testOnReceiveDeleteData() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef transaction = newTransactionActor(store.newWriteOnlyTransaction(),
+            final ActorRef transaction = newTransactionActor(WO, readWriteTransaction(),
                     "testDeleteData");
 
             transaction.tell(new DeleteData(TestModel.TEST_PATH, DataStoreVersions.HELIUM_2_VERSION).
@@ -348,8 +336,6 @@ public class ShardTransactionTest extends AbstractActorTest {
 
             expectMsgClass(duration("5 seconds"), ShardTransactionMessages.DeleteDataReply.class);
 
-            assertModification(transaction, DeleteModification.class);
-
             //unserialized
             transaction.tell(new DeleteData(TestModel.TEST_PATH, DataStoreVersions.CURRENT_VERSION), getRef());
 
@@ -361,8 +347,10 @@ public class ShardTransactionTest extends AbstractActorTest {
     public void testOnReceiveBatchedModifications() throws Exception {
         new JavaTestKit(getSystem()) {{
 
-            DOMStoreWriteTransaction mockWriteTx = Mockito.mock(DOMStoreWriteTransaction.class);
-            final ActorRef transaction = newTransactionActor(mockWriteTx, "testOnReceiveBatchedModifications");
+            ShardDataTreeTransactionParent parent = Mockito.mock(ShardDataTreeTransactionParent.class);
+            DataTreeModification mockModification = Mockito.mock(DataTreeModification.class);
+            ReadWriteShardDataTreeTransaction mockWriteTx = new ReadWriteShardDataTreeTransaction(parent, "id", mockModification);
+            final ActorRef transaction = newTransactionActor(RW, mockWriteTx, "testOnReceiveBatchedModifications");
 
             YangInstanceIdentifier writePath = TestModel.TEST_PATH;
             NormalizedNode<?, ?> writeData = ImmutableContainerNodeBuilder.create().withNodeIdentifier(
@@ -385,38 +373,19 @@ public class ShardTransactionTest extends AbstractActorTest {
             BatchedModificationsReply reply = expectMsgClass(duration("5 seconds"), BatchedModificationsReply.class);
             assertEquals("getNumBatched", 3, reply.getNumBatched());
 
-            JavaTestKit verification = new JavaTestKit(getSystem());
-            transaction.tell(new ShardWriteTransaction.GetCompositedModification(), verification.getRef());
-
-            CompositeModification compositeModification = verification.expectMsgClass(duration("5 seconds"),
-                        GetCompositeModificationReply.class).getModification();
-
-            assertEquals("CompositeModification size", 3, compositeModification.getModifications().size());
-
-            WriteModification write = (WriteModification)compositeModification.getModifications().get(0);
-            assertEquals("getPath", writePath, write.getPath());
-            assertEquals("getData", writeData, write.getData());
-
-            MergeModification merge = (MergeModification)compositeModification.getModifications().get(1);
-            assertEquals("getPath", mergePath, merge.getPath());
-            assertEquals("getData", mergeData, merge.getData());
-
-            DeleteModification delete = (DeleteModification)compositeModification.getModifications().get(2);
-            assertEquals("getPath", deletePath, delete.getPath());
-
-            InOrder inOrder = Mockito.inOrder(mockWriteTx);
-            inOrder.verify(mockWriteTx).write(writePath, writeData);
-            inOrder.verify(mockWriteTx).merge(mergePath, mergeData);
-            inOrder.verify(mockWriteTx).delete(deletePath);
+            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 testOnReceiveBatchedModificationsReady() throws Exception {
+    public void testOnReceiveBatchedModificationsReadyWithoutImmediateCommit() throws Exception {
         new JavaTestKit(getSystem()) {{
 
-            final ActorRef transaction = newTransactionActor(store.newWriteOnlyTransaction(),
-                    "testOnReceiveBatchedModificationsReady");
+            final ActorRef transaction = newTransactionActor(WO, readWriteTransaction(),
+                    "testOnReceiveBatchedModificationsReadyWithoutImmediateCommit");
 
             JavaTestKit watcher = new JavaTestKit(getSystem());
             watcher.watch(transaction);
@@ -443,12 +412,41 @@ public class ShardTransactionTest extends AbstractActorTest {
         }};
     }
 
+    @Test
+    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("tx1", DataStoreVersions.CURRENT_VERSION, null);
+            batched.addModification(new WriteModification(writePath, writeData));
+            batched.setReady(true);
+            batched.setDoCommitOnReady(true);
+            batched.setTotalMessagesSent(1);
+
+            transaction.tell(batched, getRef());
+            expectMsgClass(duration("5 seconds"), CommitTransactionReply.SERIALIZABLE_CLASS);
+            watcher.expectMsgClass(duration("5 seconds"), Terminated.class);
+        }};
+    }
+
     @Test(expected=TestException.class)
     public void testOnReceiveBatchedModificationsFailure() throws Throwable {
         new JavaTestKit(getSystem()) {{
 
-            DOMStoreWriteTransaction mockWriteTx = Mockito.mock(DOMStoreWriteTransaction.class);
-            final ActorRef transaction = newTransactionActor(mockWriteTx,
+            ShardDataTreeTransactionParent parent = Mockito.mock(ShardDataTreeTransactionParent.class);
+            DataTreeModification mockModification = Mockito.mock(DataTreeModification.class);
+            ReadWriteShardDataTreeTransaction mockWriteTx = new ReadWriteShardDataTreeTransaction(parent, "id", mockModification);
+            final ActorRef transaction = newTransactionActor(RW, mockWriteTx,
                     "testOnReceiveBatchedModificationsFailure");
 
             JavaTestKit watcher = new JavaTestKit(getSystem());
@@ -457,7 +455,7 @@ public class ShardTransactionTest extends AbstractActorTest {
             YangInstanceIdentifier path = TestModel.TEST_PATH;
             ContainerNode node = ImmutableNodes.containerNode(TestModel.TEST_QNAME);
 
-            doThrow(new TestException()).when(mockWriteTx).write(path, node);
+            doThrow(new TestException()).when(mockModification).write(path, node);
 
             BatchedModifications batched = new BatchedModifications("tx1", DataStoreVersions.CURRENT_VERSION, null);
             batched.addModification(new WriteModification(path, node));
@@ -483,7 +481,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     public void testOnReceiveBatchedModificationsReadyWithIncorrectTotalMessageCount() throws Throwable {
         new JavaTestKit(getSystem()) {{
 
-            final ActorRef transaction = newTransactionActor(store.newWriteOnlyTransaction(),
+            final ActorRef transaction = newTransactionActor(WO, readWriteTransaction(),
                     "testOnReceiveBatchedModificationsReadyWithIncorrectTotalMessageCount");
 
             JavaTestKit watcher = new JavaTestKit(getSystem());
@@ -507,7 +505,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testOnReceivePreLithiumReadyTransaction() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef transaction = newTransactionActor(store.newReadWriteTransaction(),
+            final ActorRef transaction = newTransactionActor(RW, readWriteTransaction(),
                     "testReadyTransaction", DataStoreVersions.HELIUM_2_VERSION);
 
             JavaTestKit watcher = new JavaTestKit(getSystem());
@@ -521,7 +519,7 @@ public class ShardTransactionTest extends AbstractActorTest {
 
         // test
         new JavaTestKit(getSystem()) {{
-            final ActorRef transaction = newTransactionActor(store.newReadWriteTransaction(),
+            final ActorRef transaction = newTransactionActor(RW, readWriteTransaction(),
                     "testReadyTransaction2", DataStoreVersions.HELIUM_2_VERSION);
 
             JavaTestKit watcher = new JavaTestKit(getSystem());
@@ -537,13 +535,13 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testOnReceiveCreateSnapshot() throws Exception {
         new JavaTestKit(getSystem()) {{
-            ShardTest.writeToStore(store, TestModel.TEST_PATH,
+            ShardTest.writeToStore(store.getDataTree(), TestModel.TEST_PATH,
                     ImmutableNodes.containerNode(TestModel.TEST_QNAME));
 
-            NormalizedNode<?,?> expectedRoot = ShardTest.readStore(store,
+            NormalizedNode<?,?> expectedRoot = ShardTest.readStore(store.getDataTree(),
                     YangInstanceIdentifier.builder().build());
 
-            final ActorRef transaction = newTransactionActor(store.newReadOnlyTransaction(),
+            final ActorRef transaction = newTransactionActor(TransactionType.READ_ONLY, readOnlyTransaction(),
                     "testOnReceiveCreateSnapshot");
 
             watch(transaction);
@@ -566,7 +564,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testReadWriteTxOnReceiveCloseTransaction() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef transaction = newTransactionActor(store.newReadWriteTransaction(),
+            final ActorRef transaction = newTransactionActor(RW, readWriteTransaction(),
                     "testReadWriteTxOnReceiveCloseTransaction");
 
             watch(transaction);
@@ -581,7 +579,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testWriteOnlyTxOnReceiveCloseTransaction() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef transaction = newTransactionActor(store.newWriteOnlyTransaction(),
+            final ActorRef transaction = newTransactionActor(WO, readWriteTransaction(),
                     "testWriteTxOnReceiveCloseTransaction");
 
             watch(transaction);
@@ -596,7 +594,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test
     public void testReadOnlyTxOnReceiveCloseTransaction() throws Exception {
         new JavaTestKit(getSystem()) {{
-            final ActorRef transaction = newTransactionActor(store.newReadOnlyTransaction(),
+            final ActorRef transaction = newTransactionActor(TransactionType.READ_ONLY, readOnlyTransaction(),
                     "testReadOnlyTxOnReceiveCloseTransaction");
 
             watch(transaction);
@@ -610,7 +608,7 @@ public class ShardTransactionTest extends AbstractActorTest {
     @Test(expected=UnknownMessageException.class)
     public void testNegativePerformingWriteOperationOnReadTransaction() throws Exception {
         final ActorRef shard = createShard();
-        final Props props = ShardTransaction.props(store.newReadOnlyTransaction(), shard,
+        final Props props = ShardTransaction.props(TransactionType.READ_ONLY, readOnlyTransaction(), shard,
                 datastoreContext, shardStats, "txn", DataStoreVersions.CURRENT_VERSION);
         final TestActorRef<ShardTransaction> transaction = TestActorRef.apply(props,getSystem());
 
@@ -625,7 +623,7 @@ public class ShardTransactionTest extends AbstractActorTest {
                 500, TimeUnit.MILLISECONDS).build();
 
         new JavaTestKit(getSystem()) {{
-            final ActorRef transaction = newTransactionActor(store.newReadWriteTransaction(),
+            final ActorRef transaction = newTransactionActor(RW, readWriteTransaction(),
                     "testShardTransactionInactivity");
 
             watch(transaction);