Merge "Create transaction on the backend datastore only when neccessary"
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / TransactionProxy.java
index 715f48c3492156d1b14005462da2c26aacb1768c..e397ab501c064adf98c5ee6c2f6708f05eb6f2fe 100644 (file)
@@ -9,54 +9,44 @@
 package org.opendaylight.controller.cluster.datastore;
 
 import akka.actor.ActorSelection;
+import akka.dispatch.Mapper;
 import akka.dispatch.OnComplete;
-
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.FinalizablePhantomReference;
-import com.google.common.base.FinalizableReferenceQueue;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.CheckedFuture;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
-
-import org.opendaylight.controller.cluster.datastore.exceptions.PrimaryNotFoundException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import org.opendaylight.controller.cluster.datastore.compat.PreLithiumTransactionContextImpl;
 import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
-import org.opendaylight.controller.cluster.datastore.messages.CloseTransaction;
-import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
-import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
-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.MergeData;
-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.PrimaryShardInfo;
 import org.opendaylight.controller.cluster.datastore.shardstrategy.ShardStrategyFactory;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
+import org.opendaylight.controller.cluster.datastore.utils.NormalizedNodeAggregator;
 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
+import org.opendaylight.controller.sal.core.spi.data.AbstractDOMStoreTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
-import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
 import org.opendaylight.yangtools.util.concurrent.MappingCheckedFuture;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
-import scala.Function1;
 import scala.concurrent.Future;
-import scala.runtime.AbstractFunction1;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
+import scala.concurrent.Promise;
 
 /**
  * TransactionProxy acts as a proxy for one or more transactions that were created on a remote shard
@@ -70,20 +60,33 @@ import java.util.concurrent.atomic.AtomicLong;
  * shards will be executed.
  * </p>
  */
-public class TransactionProxy implements DOMStoreReadWriteTransaction {
+public class TransactionProxy extends AbstractDOMStoreTransaction<TransactionIdentifier> implements DOMStoreReadWriteTransaction {
 
-    private final TransactionChainProxy transactionChainProxy;
+    public static enum TransactionType {
+        READ_ONLY,
+        WRITE_ONLY,
+        READ_WRITE;
 
+        // Cache all values
+        private static final TransactionType[] VALUES = values();
 
+        public static TransactionType fromInt(final int type) {
+            try {
+                return VALUES[type];
+            } catch (IndexOutOfBoundsException e) {
+                throw new IllegalArgumentException("In TransactionType enum value " + type, e);
+            }
+        }
+    }
 
-    public enum TransactionType {
-        READ_ONLY,
-        WRITE_ONLY,
-        READ_WRITE
+    private static enum TransactionState {
+        OPEN,
+        READY,
+        CLOSED,
     }
 
-    static Function1<Throwable, Throwable> SAME_FAILURE_TRANSFORMER = new AbstractFunction1<
-                                                                          Throwable, Throwable>() {
+    static final Mapper<Throwable, Throwable> SAME_FAILURE_TRANSFORMER =
+                                                              new Mapper<Throwable, Throwable>() {
         @Override
         public Throwable apply(Throwable failure) {
             return failure;
@@ -92,76 +95,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
     private static final AtomicLong counter = new AtomicLong();
 
-    private static final Logger
-        LOG = LoggerFactory.getLogger(TransactionProxy.class);
-
-
-    /**
-     * Used to enqueue the PhantomReferences for read-only TransactionProxy instances. The
-     * FinalizableReferenceQueue is safe to use statically in an OSGi environment as it uses some
-     * trickery to clean up its internal thread when the bundle is unloaded.
-     */
-    private static final FinalizableReferenceQueue phantomReferenceQueue =
-                                                                  new FinalizableReferenceQueue();
-
-    /**
-     * This stores the TransactionProxyCleanupPhantomReference instances statically, This is
-     * necessary because PhantomReferences need a hard reference so they're not garbage collected.
-     * Once finalized, the TransactionProxyCleanupPhantomReference removes itself from this map
-     * and thus becomes eligible for garbage collection.
-     */
-    private static final Map<TransactionProxyCleanupPhantomReference,
-                             TransactionProxyCleanupPhantomReference> phantomReferenceCache =
-                                                                        new ConcurrentHashMap<>();
-
-    /**
-     * A PhantomReference that closes remote transactions for a TransactionProxy when it's
-     * garbage collected. This is used for read-only transactions as they're not explicitly closed
-     * by clients. So the only way to detect that a transaction is no longer in use and it's safe
-     * to clean up is when it's garbage collected. It's inexact as to when an instance will be GC'ed
-     * but TransactionProxy instances should generally be short-lived enough to avoid being moved
-     * to the old generation space and thus should be cleaned up in a timely manner as the GC
-     * runs on the young generation (eden, swap1...) space much more frequently.
-     */
-    private static class TransactionProxyCleanupPhantomReference
-                                           extends FinalizablePhantomReference<TransactionProxy> {
-
-        private final List<ActorSelection> remoteTransactionActors;
-        private final AtomicBoolean remoteTransactionActorsMB;
-        private final ActorContext actorContext;
-        private final TransactionIdentifier identifier;
-
-        protected TransactionProxyCleanupPhantomReference(TransactionProxy referent) {
-            super(referent, phantomReferenceQueue);
-
-            // Note we need to cache the relevant fields from the TransactionProxy as we can't
-            // have a hard reference to the TransactionProxy instance itself.
-
-            remoteTransactionActors = referent.remoteTransactionActors;
-            remoteTransactionActorsMB = referent.remoteTransactionActorsMB;
-            actorContext = referent.actorContext;
-            identifier = referent.identifier;
-        }
-
-        @Override
-        public void finalizeReferent() {
-            LOG.trace("Cleaning up {} Tx actors for TransactionProxy {}",
-                    remoteTransactionActors.size(), identifier);
-
-            phantomReferenceCache.remove(this);
-
-            // Access the memory barrier volatile to ensure all previous updates to the
-            // remoteTransactionActors list are visible to this thread.
-
-            if(remoteTransactionActorsMB.get()) {
-                for(ActorSelection actor : remoteTransactionActors) {
-                    LOG.trace("Sending CloseTransaction to {}", actor);
-                    actorContext.sendOperationAsync(actor,
-                        new CloseTransaction().toSerializable());
-                }
-            }
-        }
-    }
+    private static final Logger LOG = LoggerFactory.getLogger(TransactionProxy.class);
 
     /**
      * Stores the remote Tx actors for each requested data store path to be used by the
@@ -170,697 +104,465 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
      * remoteTransactionActors list so they will be visible to the thread accessing the
      * PhantomReference.
      */
-    private List<ActorSelection> remoteTransactionActors;
-    private AtomicBoolean remoteTransactionActorsMB;
+    List<ActorSelection> remoteTransactionActors;
+    volatile AtomicBoolean remoteTransactionActorsMB;
 
-    private final Map<String, TransactionContext> remoteTransactionPaths = new HashMap<>();
+    /**
+     * Stores the create transaction results per shard.
+     */
+    private final Map<String, TransactionFutureCallback> txFutureCallbackMap = new HashMap<>();
 
     private final TransactionType transactionType;
-    private final ActorContext actorContext;
-    private final TransactionIdentifier identifier;
+    final ActorContext actorContext;
+    private final String transactionChainId;
     private final SchemaContext schemaContext;
-    private boolean inReadyState;
-
-    public TransactionProxy(ActorContext actorContext, TransactionType transactionType) {
-        this(actorContext, transactionType, null);
-    }
+    private TransactionState state = TransactionState.OPEN;
 
-    @VisibleForTesting
-    List<Future<Object>> getRecordedOperationFutures() {
-        List<Future<Object>> recordedOperationFutures = Lists.newArrayList();
-        for(TransactionContext transactionContext : remoteTransactionPaths.values()) {
-            recordedOperationFutures.addAll(transactionContext.getRecordedOperationFutures());
-        }
+    private volatile boolean initialized;
+    private Semaphore operationLimiter;
+    private OperationCompleter operationCompleter;
 
-        return recordedOperationFutures;
+    public TransactionProxy(ActorContext actorContext, TransactionType transactionType) {
+        this(actorContext, transactionType, "");
     }
 
-    public TransactionProxy(ActorContext actorContext, TransactionType transactionType, TransactionChainProxy transactionChainProxy) {
+    public TransactionProxy(ActorContext actorContext, TransactionType transactionType, String transactionChainId) {
+        super(createIdentifier(actorContext));
         this.actorContext = Preconditions.checkNotNull(actorContext,
             "actorContext should not be null");
         this.transactionType = Preconditions.checkNotNull(transactionType,
             "transactionType should not be null");
         this.schemaContext = Preconditions.checkNotNull(actorContext.getSchemaContext(),
             "schemaContext should not be null");
-        this.transactionChainProxy = transactionChainProxy;
+        this.transactionChainId = transactionChainId;
+
+        LOG.debug("Created txn {} of type {} on chain {}", getIdentifier(), transactionType, transactionChainId);
+    }
 
+    private static TransactionIdentifier createIdentifier(ActorContext actorContext) {
         String memberName = actorContext.getCurrentMemberName();
-        if(memberName == null){
+        if (memberName == null) {
             memberName = "UNKNOWN-MEMBER";
         }
 
-        this.identifier = TransactionIdentifier.builder().memberName(memberName).counter(
-            counter.getAndIncrement()).build();
+        return new TransactionIdentifier(memberName, counter.getAndIncrement());
+    }
 
-        if(transactionType == TransactionType.READ_ONLY) {
-            // Read-only Tx's aren't explicitly closed by the client so we create a PhantomReference
-            // to close the remote Tx's when this instance is no longer in use and is garbage
-            // collected.
+    @VisibleForTesting
+    boolean hasTransactionContext() {
+        for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
+            TransactionContext transactionContext = txFutureCallback.getTransactionContext();
+            if(transactionContext != null) {
+                return true;
+            }
+        }
 
-            remoteTransactionActors = Lists.newArrayList();
-            remoteTransactionActorsMB = new AtomicBoolean();
+        return false;
+    }
 
-            TransactionProxyCleanupPhantomReference cleanup =
-                new TransactionProxyCleanupPhantomReference(this);
-            phantomReferenceCache.put(cleanup, cleanup);
-        }
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("Created txn {} of type {}", identifier, transactionType);
-        }
+    private static boolean isRootPath(YangInstanceIdentifier path) {
+        return !path.getPathArguments().iterator().hasNext();
     }
 
     @Override
-    public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> read(
-            final YangInstanceIdentifier path) {
+    public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> read(final YangInstanceIdentifier path) {
 
         Preconditions.checkState(transactionType != TransactionType.WRITE_ONLY,
                 "Read operation on write-only transaction is not allowed");
 
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("Tx {} read {}", identifier, path);
-        }
-        createTransactionIfMissing(actorContext, path);
+        LOG.debug("Tx {} read {}", getIdentifier(), path);
 
-        return transactionContext(path).readData(path);
-    }
+        final SettableFuture<Optional<NormalizedNode<?, ?>>> proxyFuture = SettableFuture.create();
 
-    @Override
-    public CheckedFuture<Boolean, ReadFailedException> exists(YangInstanceIdentifier path) {
+        if(isRootPath(path)){
+            readAllData(path, proxyFuture);
+        } else {
+            throttleOperation();
 
-        Preconditions.checkState(transactionType != TransactionType.WRITE_ONLY,
-                "Exists operation on write-only transaction is not allowed");
+            TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
+            txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
+                @Override
+                public void invoke(TransactionContext transactionContext) {
+                    transactionContext.readData(path, proxyFuture);
+                }
+            });
 
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("Tx {} exists {}", identifier, path);
         }
-        createTransactionIfMissing(actorContext, path);
-
-        return transactionContext(path).dataExists(path);
-    }
 
-    private void checkModificationState() {
-        Preconditions.checkState(transactionType != TransactionType.READ_ONLY,
-                "Modification operation on read-only transaction is not allowed");
-        Preconditions.checkState(!inReadyState,
-                "Transaction is sealed - further modifications are not allowed");
+        return MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
     }
 
-    @Override
-    public void write(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
+    private void readAllData(final YangInstanceIdentifier path,
+                             final SettableFuture<Optional<NormalizedNode<?, ?>>> proxyFuture) {
+        Set<String> allShardNames = actorContext.getConfiguration().getAllShardNames();
+        List<SettableFuture<Optional<NormalizedNode<?, ?>>>> futures = new ArrayList<>(allShardNames.size());
 
-        checkModificationState();
+        for(String shardName : allShardNames){
+            final SettableFuture<Optional<NormalizedNode<?, ?>>> subProxyFuture = SettableFuture.create();
 
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("Tx {} write {}", identifier, path);
-        }
-        createTransactionIfMissing(actorContext, path);
-
-        transactionContext(path).writeData(path, data);
-    }
+            throttleOperation();
 
-    @Override
-    public void merge(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-
-        checkModificationState();
+            TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(shardName);
+            txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
+                @Override
+                public void invoke(TransactionContext transactionContext) {
+                    transactionContext.readData(path, subProxyFuture);
+                }
+            });
 
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("Tx {} merge {}", identifier, path);
+            futures.add(subProxyFuture);
         }
-        createTransactionIfMissing(actorContext, path);
 
-        transactionContext(path).mergeData(path, data);
-    }
+        final ListenableFuture<List<Optional<NormalizedNode<?, ?>>>> future = Futures.allAsList(futures);
 
-    @Override
-    public void delete(YangInstanceIdentifier path) {
-
-        checkModificationState();
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("Tx {} delete {}", identifier, path);
-        }
-        createTransactionIfMissing(actorContext, path);
-
-        transactionContext(path).deleteData(path);
+        future.addListener(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    proxyFuture.set(NormalizedNodeAggregator.aggregate(YangInstanceIdentifier.builder().build(),
+                            future.get(), actorContext.getSchemaContext()));
+                } catch (DataValidationFailedException | InterruptedException | ExecutionException e) {
+                    proxyFuture.setException(e);
+                }
+            }
+        }, actorContext.getActorSystem().dispatcher());
     }
 
     @Override
-    public DOMStoreThreePhaseCommitCohort ready() {
+    public CheckedFuture<Boolean, ReadFailedException> exists(final YangInstanceIdentifier path) {
 
-        checkModificationState();
+        Preconditions.checkState(transactionType != TransactionType.WRITE_ONLY,
+                "Exists operation on write-only transaction is not allowed");
 
-        inReadyState = true;
+        LOG.debug("Tx {} exists {}", getIdentifier(), path);
 
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("Tx {} Trying to get {} transactions ready for commit", identifier,
-                remoteTransactionPaths.size());
-        }
-        List<Future<ActorSelection>> cohortFutures = Lists.newArrayList();
+        throttleOperation();
 
-        for(TransactionContext transactionContext : remoteTransactionPaths.values()) {
+        final SettableFuture<Boolean> proxyFuture = SettableFuture.create();
 
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} Readying transaction for shard {}", identifier,
-                    transactionContext.getShardName());
+        TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
+        txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
+            @Override
+            public void invoke(TransactionContext transactionContext) {
+                transactionContext.dataExists(path, proxyFuture);
             }
-            cohortFutures.add(transactionContext.readyTransaction());
-        }
-
-        if(transactionChainProxy != null){
-            transactionChainProxy.onTransactionReady(cohortFutures);
-        }
+        });
 
-        return new ThreePhaseCommitCohortProxy(actorContext, cohortFutures,
-                identifier.toString());
+        return MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
     }
 
-    @Override
-    public Object getIdentifier() {
-        return this.identifier;
-    }
-
-    @Override
-    public void close() {
-        for(TransactionContext transactionContext : remoteTransactionPaths.values()) {
-            transactionContext.closeTransaction();
-        }
-
-        remoteTransactionPaths.clear();
-
-        if(transactionType == TransactionType.READ_ONLY) {
-            remoteTransactionActors.clear();
-            remoteTransactionActorsMB.set(true);
-        }
-    }
-
-    private TransactionContext transactionContext(YangInstanceIdentifier path){
-        String shardName = shardNameFromIdentifier(path);
-        return remoteTransactionPaths.get(shardName);
+    private void checkModificationState() {
+        Preconditions.checkState(transactionType != TransactionType.READ_ONLY,
+                "Modification operation on read-only transaction is not allowed");
+        Preconditions.checkState(state == TransactionState.OPEN,
+                "Transaction is sealed - further modifications are not allowed");
     }
 
-    private String shardNameFromIdentifier(YangInstanceIdentifier path){
-        return ShardStrategyFactory.getStrategy(path).findShard(path);
+    private void throttleOperation() {
+        throttleOperation(1);
     }
 
-    private void createTransactionIfMissing(ActorContext actorContext,
-        YangInstanceIdentifier path) {
+    private void throttleOperation(int acquirePermits) {
+        if(!initialized) {
+            // Note : Currently mailbox-capacity comes from akka.conf and not from the config-subsystem
+            operationLimiter = new Semaphore(actorContext.getTransactionOutstandingOperationLimit());
+            operationCompleter = new OperationCompleter(operationLimiter);
 
-        if(transactionChainProxy != null){
-            transactionChainProxy.waitTillCurrentTransactionReady();
-        }
-
-        String shardName = ShardStrategyFactory.getStrategy(path).findShard(path);
-
-        TransactionContext transactionContext =
-            remoteTransactionPaths.get(shardName);
-
-        if (transactionContext != null) {
-            // A transaction already exists with that shard
-            return;
+            // Make sure we write this last because it's volatile and will also publish the non-volatile writes
+            // above as well so they'll be visible to other threads.
+            initialized = true;
         }
 
         try {
-            Optional<ActorSelection> primaryShard = actorContext.findPrimaryShard(shardName);
-            if (!primaryShard.isPresent()) {
-                throw new PrimaryNotFoundException("Primary could not be found for shard " + shardName);
+            if(!operationLimiter.tryAcquire(acquirePermits,
+                    actorContext.getDatastoreContext().getOperationTimeoutInSeconds(), TimeUnit.SECONDS)){
+                LOG.warn("Failed to acquire operation permit for transaction {}", getIdentifier());
             }
-
-            Object response = actorContext.executeOperation(primaryShard.get(),
-                new CreateTransaction(identifier.toString(), this.transactionType.ordinal(),
-                    getTransactionChainId()).toSerializable());
-            if (response.getClass().equals(CreateTransactionReply.SERIALIZABLE_CLASS)) {
-                CreateTransactionReply reply =
-                    CreateTransactionReply.fromSerializable(response);
-
-                String transactionPath = reply.getTransactionPath();
-
-                if(LOG.isDebugEnabled()) {
-                    LOG.debug("Tx {} Received transaction path = {}", identifier, transactionPath);
-                }
-                ActorSelection transactionActor = actorContext.actorSelection(transactionPath);
-
-                if (transactionType == TransactionType.READ_ONLY) {
-                    // Add the actor to the remoteTransactionActors list for access by the
-                    // cleanup PhantonReference.
-                    remoteTransactionActors.add(transactionActor);
-
-                    // Write to the memory barrier volatile to publish the above update to the
-                    // remoteTransactionActors list for thread visibility.
-                    remoteTransactionActorsMB.set(true);
-                }
-
-                // TxActor is always created where the leader of the shard is.
-                // Check if TxActor is created in the same node
-                boolean isTxActorLocal = actorContext.isLocalPath(transactionPath);
-
-                transactionContext = new TransactionContextImpl(shardName, transactionPath,
-                    transactionActor, identifier, actorContext, schemaContext, isTxActorLocal);
-
-                remoteTransactionPaths.put(shardName, transactionContext);
-            } else {
-                throw new IllegalArgumentException(String.format(
-                    "Invalid reply type {} for CreateTransaction", response.getClass()));
-            }
-        } catch (Exception e) {
+        } catch (InterruptedException e) {
             if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} Creating NoOpTransaction because of : {}", identifier, e.getMessage());
+                LOG.debug("Interrupted when trying to acquire operation permit for transaction " + getIdentifier().toString(), e);
+            } else {
+                LOG.warn("Interrupted when trying to acquire operation permit for transaction {}", getIdentifier());
             }
-            remoteTransactionPaths
-                .put(shardName, new NoOpTransactionContext(shardName, e, identifier));
         }
     }
 
-    public String getTransactionChainId() {
-        if(transactionChainProxy == null){
-            return "";
-        }
-        return transactionChainProxy.getTransactionChainId();
+    final void ensureInitializied() {
+        Preconditions.checkState(initialized, "Transaction %s was not propertly initialized.", getIdentifier());
     }
 
+    @Override
+    public void write(final YangInstanceIdentifier path, final NormalizedNode<?, ?> data) {
 
-    private interface TransactionContext {
-        String getShardName();
+        checkModificationState();
 
-        void closeTransaction();
+        LOG.debug("Tx {} write {}", getIdentifier(), path);
 
-        Future<ActorSelection> readyTransaction();
+        throttleOperation();
 
-        void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
+        TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
+        txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
+            @Override
+            public void invoke(TransactionContext transactionContext) {
+                transactionContext.writeData(path, data);
+            }
+        });
+    }
 
-        void deleteData(YangInstanceIdentifier path);
+    @Override
+    public void merge(final YangInstanceIdentifier path, final NormalizedNode<?, ?> data) {
 
-        void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
+        checkModificationState();
 
-        CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
-                final YangInstanceIdentifier path);
+        LOG.debug("Tx {} merge {}", getIdentifier(), path);
 
-        CheckedFuture<Boolean, ReadFailedException> dataExists(YangInstanceIdentifier path);
+        throttleOperation();
 
-        List<Future<Object>> getRecordedOperationFutures();
+        TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
+        txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
+            @Override
+            public void invoke(TransactionContext transactionContext) {
+                transactionContext.mergeData(path, data);
+            }
+        });
     }
 
-    private static abstract class AbstractTransactionContext implements TransactionContext {
+    @Override
+    public void delete(final YangInstanceIdentifier path) {
 
-        protected final TransactionIdentifier identifier;
-        protected final String shardName;
-        protected final List<Future<Object>> recordedOperationFutures = Lists.newArrayList();
+        checkModificationState();
 
-        AbstractTransactionContext(String shardName, TransactionIdentifier identifier) {
-            this.shardName = shardName;
-            this.identifier = identifier;
-        }
+        LOG.debug("Tx {} delete {}", getIdentifier(), path);
 
-        @Override
-        public String getShardName() {
-            return shardName;
-        }
+        throttleOperation();
 
-        @Override
-        public List<Future<Object>> getRecordedOperationFutures() {
-            return recordedOperationFutures;
-        }
+        TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
+        txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
+            @Override
+            public void invoke(TransactionContext transactionContext) {
+                transactionContext.deleteData(path);
+            }
+        });
     }
 
-    private static class TransactionContextImpl extends AbstractTransactionContext {
-        private final Logger LOG = LoggerFactory.getLogger(TransactionContextImpl.class);
-
-        private final ActorContext actorContext;
-        private final SchemaContext schemaContext;
-        private final String actorPath;
-        private final ActorSelection actor;
-        private final boolean isTxActorLocal;
-
-        private TransactionContextImpl(String shardName, String actorPath,
-                ActorSelection actor, TransactionIdentifier identifier, ActorContext actorContext,
-                SchemaContext schemaContext, boolean isTxActorLocal) {
-            super(shardName, identifier);
-            this.actorPath = actorPath;
-            this.actor = actor;
-            this.actorContext = actorContext;
-            this.schemaContext = schemaContext;
-            this.isTxActorLocal = isTxActorLocal;
+    private boolean seal(final TransactionState newState) {
+        if (state == TransactionState.OPEN) {
+            state = newState;
+            return true;
+        } else {
+            return false;
         }
+    }
 
-        private ActorSelection getActor() {
-            return actor;
-        }
+    @Override
+    public AbstractThreePhaseCommitCohort<?> ready() {
+        Preconditions.checkState(transactionType != TransactionType.READ_ONLY,
+                "Read-only transactions cannot be readied");
 
-        @Override
-        public void closeTransaction() {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} closeTransaction called", identifier);
-            }
-            actorContext.sendOperationAsync(getActor(), new CloseTransaction().toSerializable());
-        }
+        final boolean success = seal(TransactionState.READY);
+        Preconditions.checkState(success, "Transaction %s is %s, it cannot be readied", getIdentifier(), state);
 
-        @Override
-        public Future<ActorSelection> readyTransaction() {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} readyTransaction called with {} previous recorded operations pending",
-                    identifier, recordedOperationFutures.size());
-            }
-            // Send the ReadyTransaction message to the Tx actor.
+        LOG.debug("Tx {} Readying {} transactions for commit", getIdentifier(),
+                    txFutureCallbackMap.size());
 
-            ReadyTransaction readyTransaction = new ReadyTransaction();
-            final Future<Object> replyFuture = actorContext.executeOperationAsync(getActor(),
-                isTxActorLocal ? readyTransaction : readyTransaction.toSerializable());
+        if (txFutureCallbackMap.isEmpty()) {
+            TransactionRateLimitingCallback.adjustRateLimitForUnusedTransaction(actorContext);
+            return NoOpDOMStoreThreePhaseCommitCohort.INSTANCE;
+        }
 
-            // Combine all the previously recorded put/merge/delete operation reply Futures and the
-            // ReadyTransactionReply Future into one Future. If any one fails then the combined
-            // Future will fail. We need all prior operations and the ready operation to succeed
-            // in order to attempt commit.
+        throttleOperation(txFutureCallbackMap.size());
 
-            List<Future<Object>> futureList =
-                    Lists.newArrayListWithCapacity(recordedOperationFutures.size() + 1);
-            futureList.addAll(recordedOperationFutures);
-            futureList.add(replyFuture);
+        final boolean isSingleShard = txFutureCallbackMap.size() == 1;
+        return isSingleShard ? createSingleCommitCohort() : createMultiCommitCohort();
+    }
 
-            Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(futureList,
-                    actorContext.getActorSystem().dispatcher());
+    @SuppressWarnings({ "rawtypes", "unchecked" })
+    private AbstractThreePhaseCommitCohort<Object> createSingleCommitCohort() {
+        TransactionFutureCallback txFutureCallback = txFutureCallbackMap.values().iterator().next();
 
-            // Transform the combined Future into a Future that returns the cohort actor path from
-            // the ReadyTransactionReply. That's the end result of the ready operation.
+        LOG.debug("Tx {} Readying transaction for shard {} on chain {}", getIdentifier(),
+                txFutureCallback.getShardName(), transactionChainId);
 
-            return combinedFutures.transform(new AbstractFunction1<Iterable<Object>, ActorSelection>() {
+        final OperationCallback.Reference operationCallbackRef =
+                new OperationCallback.Reference(OperationCallback.NO_OP_CALLBACK);
+        final TransactionContext transactionContext = txFutureCallback.getTransactionContext();
+        final Future future;
+        if (transactionContext != null) {
+            // avoid the creation of a promise and a TransactionOperation
+            future = getReadyOrDirectCommitFuture(transactionContext, operationCallbackRef);
+        } else {
+            final Promise promise = akka.dispatch.Futures.promise();
+            txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
                 @Override
-                public ActorSelection apply(Iterable<Object> notUsed) {
-                    if(LOG.isDebugEnabled()) {
-                        LOG.debug("Tx {} readyTransaction: pending recorded operations succeeded",
-                            identifier);
-                    }
-                    // At this point all the Futures succeeded and we need to extract the cohort
-                    // actor path from the ReadyTransactionReply. For the recorded operations, they
-                    // don't return any data so we're only interested that they completed
-                    // successfully. We could be paranoid and verify the correct reply types but
-                    // that really should never happen so it's not worth the overhead of
-                    // de-serializing each reply.
-
-                    // Note the Future get call here won't block as it's complete.
-                    Object serializedReadyReply = replyFuture.value().get().get();
-                    if (serializedReadyReply instanceof ReadyTransactionReply) {
-                        return actorContext.actorSelection(((ReadyTransactionReply)serializedReadyReply).getCohortPath());
-
-                    } else if(serializedReadyReply.getClass().equals(ReadyTransactionReply.SERIALIZABLE_CLASS)) {
-                        ReadyTransactionReply reply = ReadyTransactionReply.fromSerializable(serializedReadyReply);
-                        return actorContext.actorSelection(reply.getCohortPath());
-
-                    } else {
-                        // Throwing an exception here will fail the Future.
-                        throw new IllegalArgumentException(String.format("Invalid reply type {}",
-                                serializedReadyReply.getClass()));
-                    }
+                public void invoke(TransactionContext transactionContext) {
+                    promise.completeWith(getReadyOrDirectCommitFuture(transactionContext, operationCallbackRef));
                 }
-            }, SAME_FAILURE_TRANSFORMER, actorContext.getActorSystem().dispatcher());
-        }
-
-        @Override
-        public void deleteData(YangInstanceIdentifier path) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} deleteData called path = {}", identifier, path);
-            }
-
-            DeleteData deleteData = new DeleteData(path);
-            recordedOperationFutures.add(actorContext.executeOperationAsync(getActor(),
-                isTxActorLocal ? deleteData : deleteData.toSerializable()));
+            });
+            future = promise.future();
         }
 
-        @Override
-        public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} mergeData called path = {}", identifier, path);
-            }
-
-            MergeData mergeData = new MergeData(path, data, schemaContext);
-            recordedOperationFutures.add(actorContext.executeOperationAsync(getActor(),
-                isTxActorLocal ? mergeData : mergeData.toSerializable()));
-        }
-
-        @Override
-        public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} writeData called path = {}", identifier, path);
-            }
+        return new SingleCommitCohortProxy(actorContext, future, getIdentifier().toString(), operationCallbackRef);
+    }
 
-            WriteData writeData = new WriteData(path, data, schemaContext);
-            recordedOperationFutures.add(actorContext.executeOperationAsync(getActor(),
-                isTxActorLocal ? writeData : writeData.toSerializable()));
+    private Future<?> getReadyOrDirectCommitFuture(TransactionContext transactionContext,
+            OperationCallback.Reference operationCallbackRef) {
+        if(transactionContext.supportsDirectCommit()) {
+            TransactionRateLimitingCallback rateLimitingCallback = new TransactionRateLimitingCallback(actorContext);
+            operationCallbackRef.set(rateLimitingCallback);
+            rateLimitingCallback.run();
+            return transactionContext.directCommit();
+        } else {
+            return transactionContext.readyTransaction();
         }
+    }
 
-        @Override
-        public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
-                final YangInstanceIdentifier path) {
+    private AbstractThreePhaseCommitCohort<ActorSelection> createMultiCommitCohort() {
+        List<Future<ActorSelection>> cohortFutures = new ArrayList<>(txFutureCallbackMap.size());
+        for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
 
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} readData called path = {}", identifier, path);
-            }
-            final SettableFuture<Optional<NormalizedNode<?, ?>>> returnFuture = SettableFuture.create();
-
-            // If there were any previous recorded put/merge/delete operation reply Futures then we
-            // must wait for them to successfully complete. This is necessary to honor the read
-            // uncommitted semantics of the public API contract. If any one fails then fail the read.
+            LOG.debug("Tx {} Readying transaction for shard {} on chain {}", getIdentifier(),
+                        txFutureCallback.getShardName(), transactionChainId);
 
-            if(recordedOperationFutures.isEmpty()) {
-                finishReadData(path, returnFuture);
+            final TransactionContext transactionContext = txFutureCallback.getTransactionContext();
+            final Future<ActorSelection> future;
+            if (transactionContext != null) {
+                // avoid the creation of a promise and a TransactionOperation
+                future = transactionContext.readyTransaction();
             } else {
-                if(LOG.isDebugEnabled()) {
-                    LOG.debug("Tx {} readData: verifying {} previous recorded operations",
-                        identifier, recordedOperationFutures.size());
-                }
-                // Note: we make a copy of recordedOperationFutures to be on the safe side in case
-                // Futures#sequence accesses the passed List on a different thread, as
-                // recordedOperationFutures is not synchronized.
-
-                Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(
-                        Lists.newArrayList(recordedOperationFutures),
-                        actorContext.getActorSystem().dispatcher());
-
-                OnComplete<Iterable<Object>> onComplete = new OnComplete<Iterable<Object>>() {
+                final Promise<ActorSelection> promise = akka.dispatch.Futures.promise();
+                txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
                     @Override
-                    public void onComplete(Throwable failure, Iterable<Object> notUsed)
-                            throws Throwable {
-                        if(failure != null) {
-                            if(LOG.isDebugEnabled()) {
-                                LOG.debug("Tx {} readData: a recorded operation failed: {}",
-                                    identifier, failure);
-                            }
-                            returnFuture.setException(new ReadFailedException(
-                                    "The read could not be performed because a previous put, merge,"
-                                    + "or delete operation failed", failure));
-                        } else {
-                            finishReadData(path, returnFuture);
-                        }
+                    public void invoke(TransactionContext transactionContext) {
+                        promise.completeWith(transactionContext.readyTransaction());
                     }
-                };
-
-                combinedFutures.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
+                });
+                future = promise.future();
             }
 
-            return MappingCheckedFuture.create(returnFuture, ReadFailedException.MAPPER);
+            cohortFutures.add(future);
         }
 
-        private void finishReadData(final YangInstanceIdentifier path,
-                final SettableFuture<Optional<NormalizedNode<?, ?>>> returnFuture) {
+        return new ThreePhaseCommitCohortProxy(actorContext, cohortFutures, getIdentifier().toString());
+    }
 
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} finishReadData called path = {}", identifier, path);
+    @Override
+    public void close() {
+        if (!seal(TransactionState.CLOSED)) {
+            if (state == TransactionState.CLOSED) {
+                // Idempotent no-op as per AutoCloseable recommendation
+                return;
             }
-            OnComplete<Object> onComplete = new OnComplete<Object>() {
-                @Override
-                public void onComplete(Throwable failure, Object readResponse) throws Throwable {
-                    if(failure != null) {
-                        if(LOG.isDebugEnabled()) {
-                            LOG.debug("Tx {} read operation failed: {}", identifier, failure);
-                        }
-                        returnFuture.setException(new ReadFailedException(
-                                "Error reading data for path " + path, failure));
 
-                    } else {
-                        if(LOG.isDebugEnabled()) {
-                            LOG.debug("Tx {} read operation succeeded", identifier, failure);
-                        }
-
-                        if (readResponse instanceof ReadDataReply) {
-                            ReadDataReply reply = (ReadDataReply) readResponse;
-                            returnFuture.set(Optional.<NormalizedNode<?, ?>>fromNullable(reply.getNormalizedNode()));
-
-                        } else if (readResponse.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)) {
-                            ReadDataReply reply = ReadDataReply.fromSerializable(schemaContext, path, readResponse);
-                            returnFuture.set(Optional.<NormalizedNode<?, ?>>fromNullable(reply.getNormalizedNode()));
-
-                        } else {
-                            returnFuture.setException(new ReadFailedException(
-                                "Invalid response reading data for path " + path));
-                        }
-                    }
+            throw new IllegalStateException(String.format("Transaction %s is ready, it cannot be closed",
+                getIdentifier()));
+        }
+
+        for (TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
+            txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
+                @Override
+                public void invoke(TransactionContext transactionContext) {
+                    transactionContext.closeTransaction();
                 }
-            };
+            });
+        }
 
-            ReadData readData = new ReadData(path);
-            Future<Object> readFuture = actorContext.executeOperationAsync(getActor(),
-                isTxActorLocal ? readData : readData.toSerializable());
+        txFutureCallbackMap.clear();
 
-            readFuture.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
+        if(remoteTransactionActorsMB != null) {
+            remoteTransactionActors.clear();
+            remoteTransactionActorsMB.set(true);
         }
+    }
 
-        @Override
-        public CheckedFuture<Boolean, ReadFailedException> dataExists(
-                final YangInstanceIdentifier path) {
+    private String shardNameFromIdentifier(YangInstanceIdentifier path){
+        return ShardStrategyFactory.getStrategy(path).findShard(path);
+    }
 
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} dataExists called path = {}", identifier, path);
-            }
-            final SettableFuture<Boolean> returnFuture = SettableFuture.create();
+    protected Future<PrimaryShardInfo> sendFindPrimaryShardAsync(String shardName) {
+        return actorContext.findPrimaryShardAsync(shardName);
+    }
+
+    final TransactionType getTransactionType() {
+        return transactionType;
+    }
 
-            // If there were any previous recorded put/merge/delete operation reply Futures then we
-            // must wait for them to successfully complete. This is necessary to honor the read
-            // uncommitted semantics of the public API contract. If any one fails then fail this
-            // request.
+    final Semaphore getOperationLimiter() {
+        return operationLimiter;
+    }
 
-            if(recordedOperationFutures.isEmpty()) {
-                finishDataExists(path, returnFuture);
-            } else {
-                if(LOG.isDebugEnabled()) {
-                    LOG.debug("Tx {} dataExists: verifying {} previous recorded operations",
-                        identifier, recordedOperationFutures.size());
-                }
-                // Note: we make a copy of recordedOperationFutures to be on the safe side in case
-                // Futures#sequence accesses the passed List on a different thread, as
-                // recordedOperationFutures is not synchronized.
-
-                Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(
-                        Lists.newArrayList(recordedOperationFutures),
-                        actorContext.getActorSystem().dispatcher());
-                OnComplete<Iterable<Object>> onComplete = new OnComplete<Iterable<Object>>() {
-                    @Override
-                    public void onComplete(Throwable failure, Iterable<Object> notUsed)
-                            throws Throwable {
-                        if(failure != null) {
-                            if(LOG.isDebugEnabled()) {
-                                LOG.debug("Tx {} dataExists: a recorded operation failed: {}",
-                                    identifier, failure);
-                            }
-                            returnFuture.setException(new ReadFailedException(
-                                    "The data exists could not be performed because a previous "
-                                    + "put, merge, or delete operation failed", failure));
-                        } else {
-                            finishDataExists(path, returnFuture);
-                        }
-                    }
-                };
+    private TransactionFutureCallback getOrCreateTxFutureCallback(YangInstanceIdentifier path) {
+        String shardName = shardNameFromIdentifier(path);
+        return getOrCreateTxFutureCallback(shardName);
+    }
 
-                combinedFutures.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
-            }
+    private TransactionFutureCallback getOrCreateTxFutureCallback(String shardName) {
+        TransactionFutureCallback txFutureCallback = txFutureCallbackMap.get(shardName);
+        if(txFutureCallback == null) {
+            Future<PrimaryShardInfo> findPrimaryFuture = sendFindPrimaryShardAsync(shardName);
 
-            return MappingCheckedFuture.create(returnFuture, ReadFailedException.MAPPER);
-        }
+            final TransactionFutureCallback newTxFutureCallback = new TransactionFutureCallback(this, shardName);
 
-        private void finishDataExists(final YangInstanceIdentifier path,
-                final SettableFuture<Boolean> returnFuture) {
+            txFutureCallback = newTxFutureCallback;
+            txFutureCallbackMap.put(shardName, txFutureCallback);
 
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} finishDataExists called path = {}", identifier, path);
-            }
-            OnComplete<Object> onComplete = new OnComplete<Object>() {
+            findPrimaryFuture.onComplete(new OnComplete<PrimaryShardInfo>() {
                 @Override
-                public void onComplete(Throwable failure, Object response) throws Throwable {
+                public void onComplete(Throwable failure, PrimaryShardInfo primaryShardInfo) {
                     if(failure != null) {
-                        if(LOG.isDebugEnabled()) {
-                            LOG.debug("Tx {} dataExists operation failed: {}", identifier, failure);
-                        }
-                        returnFuture.setException(new ReadFailedException(
-                                "Error checking data exists for path " + path, failure));
+                        newTxFutureCallback.createTransactionContext(failure, null);
                     } else {
-                        if(LOG.isDebugEnabled()) {
-                            LOG.debug("Tx {} dataExists operation succeeded", identifier, failure);
-                        }
-
-                        if (response instanceof DataExistsReply) {
-                            returnFuture.set(Boolean.valueOf(((DataExistsReply) response).exists()));
-
-                        } else if (response.getClass().equals(DataExistsReply.SERIALIZABLE_CLASS)) {
-                            returnFuture.set(Boolean.valueOf(DataExistsReply.fromSerializable(response).exists()));
-
-                        } else {
-                            returnFuture.setException(new ReadFailedException(
-                                    "Invalid response checking exists for path " + path));
-                        }
+                        newTxFutureCallback.setPrimaryShard(primaryShardInfo.getPrimaryShardActor());
                     }
                 }
-            };
-
-            DataExists dataExists = new DataExists(path);
-            Future<Object> future = actorContext.executeOperationAsync(getActor(),
-                isTxActorLocal ? dataExists : dataExists.toSerializable());
-
-            future.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
+            }, actorContext.getClientDispatcher());
         }
-    }
 
-    private static class NoOpTransactionContext extends AbstractTransactionContext {
+        return txFutureCallback;
+    }
 
-        private final Logger LOG = LoggerFactory.getLogger(NoOpTransactionContext.class);
+    String getTransactionChainId() {
+        return transactionChainId;
+    }
 
-        private final Exception failure;
+    protected ActorContext getActorContext() {
+        return actorContext;
+    }
 
-        public NoOpTransactionContext(String shardName, Exception failure,
-                TransactionIdentifier identifier){
-            super(shardName, identifier);
-            this.failure = failure;
-        }
+    TransactionContext createValidTransactionContext(ActorSelection transactionActor,
+            String transactionPath, short remoteTransactionVersion) {
 
-        @Override
-        public void closeTransaction() {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("NoOpTransactionContext {} closeTransaction called", identifier);
-            }
-        }
+        if (transactionType == TransactionType.READ_ONLY) {
+            // Read-only Tx's aren't explicitly closed by the client so we create a PhantomReference
+            // to close the remote Tx's when this instance is no longer in use and is garbage
+            // collected.
 
-        @Override
-        public Future<ActorSelection> readyTransaction() {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} readyTransaction called", identifier);
-            }
-            return akka.dispatch.Futures.failed(failure);
-        }
+            if(remoteTransactionActorsMB == null) {
+                remoteTransactionActors = Lists.newArrayList();
+                remoteTransactionActorsMB = new AtomicBoolean();
 
-        @Override
-        public void deleteData(YangInstanceIdentifier path) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} deleteData called path = {}", identifier, path);
+                TransactionProxyCleanupPhantomReference.track(TransactionProxy.this);
             }
-        }
 
-        @Override
-        public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} mergeData called path = {}", identifier, path);
-            }
-        }
+            // Add the actor to the remoteTransactionActors list for access by the
+            // cleanup PhantonReference.
+            remoteTransactionActors.add(transactionActor);
 
-        @Override
-        public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} writeData called path = {}", identifier, path);
-            }
+            // Write to the memory barrier volatile to publish the above update to the
+            // remoteTransactionActors list for thread visibility.
+            remoteTransactionActorsMB.set(true);
         }
 
-        @Override
-        public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
-            YangInstanceIdentifier path) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} readData called path = {}", identifier, path);
-            }
-            return Futures.immediateFailedCheckedFuture(new ReadFailedException(
-                    "Error reading data for path " + path, failure));
-        }
+        // TxActor is always created where the leader of the shard is.
+        // Check if TxActor is created in the same node
+        boolean isTxActorLocal = actorContext.isPathLocal(transactionPath);
 
-        @Override
-        public CheckedFuture<Boolean, ReadFailedException> dataExists(
-            YangInstanceIdentifier path) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} dataExists called path = {}", identifier, path);
-            }
-            return Futures.immediateFailedCheckedFuture(new ReadFailedException(
-                    "Error checking exists for path " + path, failure));
+        if(remoteTransactionVersion < DataStoreVersions.LITHIUM_VERSION) {
+            return new PreLithiumTransactionContextImpl(transactionPath, transactionActor, getIdentifier(),
+                    transactionChainId, actorContext, schemaContext, isTxActorLocal, remoteTransactionVersion,
+                    operationCompleter);
+        } else {
+            return new TransactionContextImpl(transactionActor, getIdentifier(), transactionChainId,
+                    actorContext, schemaContext, isTxActorLocal, remoteTransactionVersion, operationCompleter);
         }
     }
 }