Refactor TransactionProxy
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / TransactionProxy.java
index ffb1ab7c55064ecf7683e1857800b256c9c82a1d..5bc53442aeff04aa43c299a49890b3ecfe70b974 100644 (file)
@@ -18,23 +18,23 @@ 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.FutureCallback;
-import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.SettableFuture;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import javax.annotation.concurrent.GuardedBy;
 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
 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.shardstrategy.ShardStrategyFactory;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
@@ -50,15 +50,6 @@ import scala.concurrent.Future;
 import scala.concurrent.Promise;
 import scala.concurrent.duration.FiniteDuration;
 
-import javax.annotation.concurrent.GuardedBy;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-
 /**
  * TransactionProxy acts as a proxy for one or more transactions that were created on a remote shard
  * <p>
@@ -157,8 +148,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             if(remoteTransactionActorsMB.get()) {
                 for(ActorSelection actor : remoteTransactionActors) {
                     LOG.trace("Sending CloseTransaction to {}", actor);
-                    actorContext.sendOperationAsync(actor,
-                            new CloseTransaction().toSerializable());
+                    actorContext.sendOperationAsync(actor, CloseTransaction.INSTANCE.toSerializable());
                 }
             }
         }
@@ -182,23 +172,25 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
     private final TransactionType transactionType;
     private final ActorContext actorContext;
     private final TransactionIdentifier identifier;
-    private final TransactionChainProxy transactionChainProxy;
+    private final String transactionChainId;
     private final SchemaContext schemaContext;
     private boolean inReadyState;
+    private final Semaphore operationLimiter;
+    private final OperationCompleter operationCompleter;
 
     public TransactionProxy(ActorContext actorContext, TransactionType transactionType) {
-        this(actorContext, transactionType, null);
+        this(actorContext, transactionType, "");
     }
 
     public TransactionProxy(ActorContext actorContext, TransactionType transactionType,
-            TransactionChainProxy transactionChainProxy) {
+            String transactionChainId) {
         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;
 
         String memberName = actorContext.getCurrentMemberName();
         if(memberName == null){
@@ -221,7 +213,11 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             phantomReferenceCache.put(cleanup, cleanup);
         }
 
-        LOG.debug("Created txn {} of type {}", identifier, transactionType);
+        // Note : Currently mailbox-capacity comes from akka.conf and not from the config-subsystem
+        this.operationLimiter = new Semaphore(actorContext.getTransactionOutstandingOperationLimit());
+        this.operationCompleter = new OperationCompleter(operationLimiter);
+
+        LOG.debug("Created txn {} of type {} on chain {}", identifier, transactionType, transactionChainId);
     }
 
     @VisibleForTesting
@@ -237,47 +233,39 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         return recordedOperationFutures;
     }
 
+    @VisibleForTesting
+    boolean hasTransactionContext() {
+        for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
+            TransactionContext transactionContext = txFutureCallback.getTransactionContext();
+            if(transactionContext != null) {
+                return true;
+            }
+        }
+
+        return false;
+    }
+
     @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");
 
         LOG.debug("Tx {} read {}", identifier, path);
 
-        TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
-        TransactionContext transactionContext = txFutureCallback.getTransactionContext();
-
-        CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> future;
-        if(transactionContext != null) {
-            future = transactionContext.readData(path);
-        } else {
-            // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
-            // callback to be executed after the Tx is created.
-            final SettableFuture<Optional<NormalizedNode<?, ?>>> proxyFuture = SettableFuture.create();
-            txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
-                @Override
-                public void invoke(TransactionContext transactionContext) {
-                    Futures.addCallback(transactionContext.readData(path),
-                        new FutureCallback<Optional<NormalizedNode<?, ?>>>() {
-                            @Override
-                            public void onSuccess(Optional<NormalizedNode<?, ?>> data) {
-                                proxyFuture.set(data);
-                            }
-
-                            @Override
-                            public void onFailure(Throwable t) {
-                                proxyFuture.setException(t);
-                            }
-                        });
-                }
-            });
+        throttleOperation();
 
-            future = MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
-        }
+        final SettableFuture<Optional<NormalizedNode<?, ?>>> proxyFuture = SettableFuture.create();
 
-        return future;
+        TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
+        txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
+            @Override
+            public void invoke(TransactionContext transactionContext) {
+                transactionContext.readData(path, proxyFuture);
+            }
+        });
+
+        return MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
     }
 
     @Override
@@ -288,38 +276,19 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         LOG.debug("Tx {} exists {}", identifier, path);
 
-        TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
-        TransactionContext transactionContext = txFutureCallback.getTransactionContext();
-
-        CheckedFuture<Boolean, ReadFailedException> future;
-        if(transactionContext != null) {
-            future = transactionContext.dataExists(path);
-        } else {
-            // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
-            // callback to be executed after the Tx is created.
-            final SettableFuture<Boolean> proxyFuture = SettableFuture.create();
-            txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
-                @Override
-                public void invoke(TransactionContext transactionContext) {
-                    Futures.addCallback(transactionContext.dataExists(path),
-                        new FutureCallback<Boolean>() {
-                            @Override
-                            public void onSuccess(Boolean exists) {
-                                proxyFuture.set(exists);
-                            }
-
-                            @Override
-                            public void onFailure(Throwable t) {
-                                proxyFuture.setException(t);
-                            }
-                        });
-                }
-            });
+        throttleOperation();
 
-            future = MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
-        }
+        final SettableFuture<Boolean> proxyFuture = SettableFuture.create();
 
-        return future;
+        TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
+        txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
+            @Override
+            public void invoke(TransactionContext transactionContext) {
+                transactionContext.dataExists(path, proxyFuture);
+            }
+        });
+
+        return MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
     }
 
     private void checkModificationState() {
@@ -329,6 +298,25 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                 "Transaction is sealed - further modifications are not allowed");
     }
 
+    private void throttleOperation() {
+        throttleOperation(1);
+    }
+
+    private void throttleOperation(int acquirePermits) {
+        try {
+            if(!operationLimiter.tryAcquire(acquirePermits, actorContext.getDatastoreContext().getOperationTimeoutInSeconds(), TimeUnit.SECONDS)){
+                LOG.warn("Failed to acquire operation permit for transaction {}", getIdentifier());
+            }
+        } catch (InterruptedException e) {
+            if(LOG.isDebugEnabled()) {
+                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());
+            }
+        }
+    }
+
+
     @Override
     public void write(final YangInstanceIdentifier path, final NormalizedNode<?, ?> data) {
 
@@ -336,20 +324,15 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         LOG.debug("Tx {} write {}", identifier, path);
 
+        throttleOperation();
+
         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
-        TransactionContext transactionContext = txFutureCallback.getTransactionContext();
-        if(transactionContext != null) {
-            transactionContext.writeData(path, data);
-        } else {
-            // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
-            // callback to be executed after the Tx is created.
-            txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
-                @Override
-                public void invoke(TransactionContext transactionContext) {
-                    transactionContext.writeData(path, data);
-                }
-            });
-        }
+        txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
+            @Override
+            public void invoke(TransactionContext transactionContext) {
+                transactionContext.writeData(path, data);
+            }
+        });
     }
 
     @Override
@@ -359,20 +342,15 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         LOG.debug("Tx {} merge {}", identifier, path);
 
+        throttleOperation();
+
         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
-        TransactionContext transactionContext = txFutureCallback.getTransactionContext();
-        if(transactionContext != null) {
-            transactionContext.mergeData(path, data);
-        } else {
-            // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
-            // callback to be executed after the Tx is created.
-            txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
-                @Override
-                public void invoke(TransactionContext transactionContext) {
-                    transactionContext.mergeData(path, data);
-                }
-            });
-        }
+        txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
+            @Override
+            public void invoke(TransactionContext transactionContext) {
+                transactionContext.mergeData(path, data);
+            }
+        });
     }
 
     @Override
@@ -382,20 +360,15 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         LOG.debug("Tx {} delete {}", identifier, path);
 
+        throttleOperation();
+
         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
-        TransactionContext transactionContext = txFutureCallback.getTransactionContext();
-        if(transactionContext != null) {
-            transactionContext.deleteData(path);
-        } else {
-            // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
-            // callback to be executed after the Tx is created.
-            txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
-                @Override
-                public void invoke(TransactionContext transactionContext) {
-                    transactionContext.deleteData(path);
-                }
-            });
-        }
+        txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
+            @Override
+            public void invoke(TransactionContext transactionContext) {
+                transactionContext.deleteData(path);
+            }
+        });
     }
 
     @Override
@@ -403,6 +376,8 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         checkModificationState();
 
+        throttleOperation(txFutureCallbackMap.size());
+
         inReadyState = true;
 
         LOG.debug("Tx {} Readying {} transactions for commit", identifier,
@@ -412,35 +387,54 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
 
-            LOG.debug("Tx {} Readying transaction for shard {}", identifier,
-                        txFutureCallback.getShardName());
+            LOG.debug("Tx {} Readying transaction for shard {} chain {}", identifier,
+                        txFutureCallback.getShardName(), transactionChainId);
 
-            TransactionContext transactionContext = txFutureCallback.getTransactionContext();
-            if(transactionContext != null) {
-                cohortFutures.add(transactionContext.readyTransaction());
+            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 {
-                // The shard Tx hasn't been created yet so create a promise to ready the Tx later
-                // after it's created.
-                final Promise<ActorSelection> cohortPromise = akka.dispatch.Futures.promise();
-                txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
+                final Promise<ActorSelection> promise = akka.dispatch.Futures.promise();
+                txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
                     @Override
                     public void invoke(TransactionContext transactionContext) {
-                        cohortPromise.completeWith(transactionContext.readyTransaction());
+                        promise.completeWith(transactionContext.readyTransaction());
                     }
                 });
-
-                cohortFutures.add(cohortPromise.future());
+                future = promise.future();
             }
-        }
 
-        if(transactionChainProxy != null){
-            transactionChainProxy.onTransactionReady(cohortFutures);
+            cohortFutures.add(future);
         }
 
+        onTransactionReady(cohortFutures);
+
         return new ThreePhaseCommitCohortProxy(actorContext, cohortFutures,
                 identifier.toString());
     }
 
+    /**
+     * Method for derived classes to be notified when the transaction has been readied.
+     *
+     * @param cohortFutures the cohort Futures for each shard transaction.
+     */
+    protected void onTransactionReady(List<Future<ActorSelection>> cohortFutures) {
+    }
+
+    /**
+     * Method called to send a CreateTransaction message to a shard.
+     *
+     * @param shard the shard actor to send to
+     * @param serializedCreateMessage the serialized message to send
+     * @return the response Future
+     */
+    protected Future<Object> sendCreateTransaction(ActorSelection shard,
+            Object serializedCreateMessage) {
+        return actorContext.executeOperationAsync(shard, serializedCreateMessage);
+    }
+
     @Override
     public Object getIdentifier() {
         return this.identifier;
@@ -448,18 +442,13 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
     @Override
     public void close() {
-        for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
-            TransactionContext transactionContext = txFutureCallback.getTransactionContext();
-            if(transactionContext != null) {
-                transactionContext.closeTransaction();
-            } else {
-                txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
-                    @Override
-                    public void invoke(TransactionContext transactionContext) {
-                        transactionContext.closeTransaction();
-                    }
-                });
-            }
+        for (TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
+            txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
+                @Override
+                public void invoke(TransactionContext transactionContext) {
+                    transactionContext.closeTransaction();
+                }
+            });
         }
 
         txFutureCallbackMap.clear();
@@ -502,14 +491,15 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
     }
 
     public String getTransactionChainId() {
-        if(transactionChainProxy == null){
-            return "";
-        }
-        return transactionChainProxy.getTransactionChainId();
+        return transactionChainId;
+    }
+
+    protected ActorContext getActorContext() {
+        return actorContext;
     }
 
     /**
-     * Interface for a transaction operation to be invoked later.
+     * Interfaces for transaction operations to be invoked later.
      */
     private static interface TransactionOperation {
         void invoke(TransactionContext transactionContext);
@@ -576,22 +566,37 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
          * Adds a TransactionOperation to be executed after the CreateTransaction completes.
          */
         void addTxOperationOnComplete(TransactionOperation operation) {
+            boolean invokeOperation = true;
             synchronized(txOperationsOnComplete) {
                 if(transactionContext == null) {
                     LOG.debug("Tx {} Adding operation on complete {}", identifier);
 
+                    invokeOperation = false;
                     txOperationsOnComplete.add(operation);
-                } else {
-                    operation.invoke(transactionContext);
                 }
             }
+
+            if(invokeOperation) {
+                operation.invoke(transactionContext);
+            }
+        }
+
+        void enqueueTransactionOperation(final TransactionOperation op) {
+
+            if (transactionContext != null) {
+                op.invoke(transactionContext);
+            } else {
+                // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
+                // callback to be executed after the Tx is created.
+                addTxOperationOnComplete(op);
+            }
         }
 
         /**
          * Performs a CreateTransaction try async.
          */
         private void tryCreateTransaction() {
-            Future<Object> createTxFuture = actorContext.executeOperationAsync(primaryShard,
+            Future<Object> createTxFuture = sendCreateTransaction(primaryShard,
                     new CreateTransaction(identifier.toString(),
                             TransactionProxy.this.transactionType.ordinal(),
                             getTransactionChainId()).toSerializable());
@@ -621,34 +626,67 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                 }
             }
 
-            // Create the TransactionContext from the response or failure and execute delayed
-            // TransactionOperations. This entire section is done atomically (ie synchronized) with
-            // respect to #addTxOperationOnComplete to handle timing issues and ensure no
-            // TransactionOperation is missed and that they are processed in the order they occurred.
-            synchronized(txOperationsOnComplete) {
-                if(failure != null) {
-                    LOG.debug("Tx {} Creating NoOpTransaction because of error: {}", identifier,
-                            failure.getMessage());
-
-                    transactionContext = new NoOpTransactionContext(failure, identifier);
-                } else if (response.getClass().equals(CreateTransactionReply.SERIALIZABLE_CLASS)) {
-                    createValidTransactionContext(CreateTransactionReply.fromSerializable(response));
-                } else {
-                    IllegalArgumentException exception = new IllegalArgumentException(String.format(
+            // Create the TransactionContext from the response or failure. Store the new
+            // TransactionContext locally until we've completed invoking the
+            // TransactionOperations. This avoids thread timing issues which could cause
+            // out-of-order TransactionOperations. Eg, on a modification operation, if the
+            // TransactionContext is non-null, then we directly call the TransactionContext.
+            // However, at the same time, the code may be executing the cached
+            // TransactionOperations. So to avoid thus timing, we don't publish the
+            // TransactionContext until after we've executed all cached TransactionOperations.
+            TransactionContext localTransactionContext;
+            if(failure != null) {
+                LOG.debug("Tx {} Creating NoOpTransaction because of error: {}", identifier,
+                        failure.getMessage());
+
+                localTransactionContext = new NoOpTransactionContext(failure, identifier, operationLimiter);
+            } else if (response.getClass().equals(CreateTransactionReply.SERIALIZABLE_CLASS)) {
+                localTransactionContext = createValidTransactionContext(
+                        CreateTransactionReply.fromSerializable(response));
+            } else {
+                IllegalArgumentException exception = new IllegalArgumentException(String.format(
                         "Invalid reply type %s for CreateTransaction", response.getClass()));
 
-                    transactionContext = new NoOpTransactionContext(exception, identifier);
-                }
+                localTransactionContext = new NoOpTransactionContext(exception, identifier, operationLimiter);
+            }
+
+            executeTxOperatonsOnComplete(localTransactionContext);
+        }
+
+        private void executeTxOperatonsOnComplete(TransactionContext localTransactionContext) {
+            while(true) {
+                // Access to txOperationsOnComplete and transactionContext must be protected and atomic
+                // (ie synchronized) with respect to #addTxOperationOnComplete to handle timing
+                // issues and ensure no TransactionOperation is missed and that they are processed
+                // in the order they occurred.
+
+                // We'll make a local copy of the txOperationsOnComplete list to handle re-entrancy
+                // in case a TransactionOperation results in another transaction operation being
+                // queued (eg a put operation from a client read Future callback that is notified
+                // synchronously).
+                Collection<TransactionOperation> operationsBatch = null;
+                synchronized(txOperationsOnComplete) {
+                    if(txOperationsOnComplete.isEmpty()) {
+                        // We're done invoking the TransactionOperations so we can now publish the
+                        // TransactionContext.
+                        transactionContext = localTransactionContext;
+                        break;
+                    }
 
-                for(TransactionOperation oper: txOperationsOnComplete) {
-                    oper.invoke(transactionContext);
+                    operationsBatch = new ArrayList<>(txOperationsOnComplete);
+                    txOperationsOnComplete.clear();
                 }
 
-                txOperationsOnComplete.clear();
+                // Invoke TransactionOperations outside the sync block to avoid unnecessary blocking.
+                // A slight down-side is that we need to re-acquire the lock below but this should
+                // be negligible.
+                for(TransactionOperation oper: operationsBatch) {
+                    oper.invoke(localTransactionContext);
+                }
             }
         }
 
-        private void createValidTransactionContext(CreateTransactionReply reply) {
+        private TransactionContext createValidTransactionContext(CreateTransactionReply reply) {
             String transactionPath = reply.getTransactionPath();
 
             LOG.debug("Tx {} Received transaction actor path {}", identifier, transactionPath);
@@ -667,401 +705,10 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
             // 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(transactionPath, transactionActor, identifier,
-                actorContext, schemaContext, isTxActorLocal, reply.getVersion());
-        }
-    }
-
-    private interface TransactionContext {
-        void closeTransaction();
-
-        Future<ActorSelection> readyTransaction();
-
-        void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
-
-        void deleteData(YangInstanceIdentifier path);
-
-        void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
-
-        CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
-                final YangInstanceIdentifier path);
-
-        CheckedFuture<Boolean, ReadFailedException> dataExists(YangInstanceIdentifier path);
-
-        List<Future<Object>> getRecordedOperationFutures();
-    }
+            boolean isTxActorLocal = actorContext.isPathLocal(transactionPath);
 
-    private static abstract class AbstractTransactionContext implements TransactionContext {
-
-        protected final TransactionIdentifier identifier;
-        protected final List<Future<Object>> recordedOperationFutures = Lists.newArrayList();
-
-        AbstractTransactionContext(TransactionIdentifier identifier) {
-            this.identifier = identifier;
-        }
-
-        @Override
-        public List<Future<Object>> getRecordedOperationFutures() {
-            return recordedOperationFutures;
-        }
-    }
-
-    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 transactionPath;
-        private final ActorSelection actor;
-        private final boolean isTxActorLocal;
-        private final int remoteTransactionVersion;
-
-        private TransactionContextImpl(String transactionPath, ActorSelection actor, TransactionIdentifier identifier,
-                ActorContext actorContext, SchemaContext schemaContext,
-                boolean isTxActorLocal, int remoteTransactionVersion) {
-            super(identifier);
-            this.transactionPath = transactionPath;
-            this.actor = actor;
-            this.actorContext = actorContext;
-            this.schemaContext = schemaContext;
-            this.isTxActorLocal = isTxActorLocal;
-            this.remoteTransactionVersion = remoteTransactionVersion;
-        }
-
-        private ActorSelection getActor() {
-            return actor;
-        }
-
-        @Override
-        public void closeTransaction() {
-            LOG.debug("Tx {} closeTransaction called", identifier);
-
-            actorContext.sendOperationAsync(getActor(), new CloseTransaction().toSerializable());
-        }
-
-        @Override
-        public Future<ActorSelection> readyTransaction() {
-            LOG.debug("Tx {} readyTransaction called with {} previous recorded operations pending",
-                    identifier, recordedOperationFutures.size());
-
-            // Send the ReadyTransaction message to the Tx actor.
-
-            ReadyTransaction readyTransaction = new ReadyTransaction();
-            final Future<Object> replyFuture = actorContext.executeOperationAsync(getActor(),
-                isTxActorLocal ? readyTransaction : readyTransaction.toSerializable());
-
-            // 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.
-
-            List<Future<Object>> futureList =
-                    Lists.newArrayListWithCapacity(recordedOperationFutures.size() + 1);
-            futureList.addAll(recordedOperationFutures);
-            futureList.add(replyFuture);
-
-            Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(futureList,
-                    actorContext.getActorSystem().dispatcher());
-
-            // 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.
-
-            return combinedFutures.transform(new Mapper<Iterable<Object>, ActorSelection>() {
-                @Override
-                public ActorSelection checkedApply(Iterable<Object> notUsed) {
-                    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);
-                        String cohortPath = reply.getCohortPath();
-
-                        // In Helium we used to return the local path of the actor which represented
-                        // a remote ThreePhaseCommitCohort. The local path would then be converted to
-                        // a remote path using this resolvePath method. To maintain compatibility with
-                        // a Helium node we need to continue to do this conversion.
-                        // At some point in the future when upgrades from Helium are not supported
-                        // we could remove this code to resolvePath and just use the cohortPath as the
-                        // resolved cohortPath
-                        if(TransactionContextImpl.this.remoteTransactionVersion < CreateTransaction.HELIUM_1_VERSION) {
-                            cohortPath = actorContext.resolvePath(transactionPath, cohortPath);
-                        }
-
-                        return actorContext.actorSelection(cohortPath);
-
-                    } else {
-                        // Throwing an exception here will fail the Future.
-                        throw new IllegalArgumentException(String.format("Invalid reply type {}",
-                                serializedReadyReply.getClass()));
-                    }
-                }
-            }, SAME_FAILURE_TRANSFORMER, actorContext.getActorSystem().dispatcher());
-        }
-
-        @Override
-        public void deleteData(YangInstanceIdentifier path) {
-            LOG.debug("Tx {} deleteData called path = {}", identifier, path);
-
-            DeleteData deleteData = new DeleteData(path);
-            recordedOperationFutures.add(actorContext.executeOperationAsync(getActor(),
-                isTxActorLocal ? deleteData : deleteData.toSerializable()));
-        }
-
-        @Override
-        public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-            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) {
-            LOG.debug("Tx {} writeData called path = {}", identifier, path);
-
-            WriteData writeData = new WriteData(path, data, schemaContext);
-            recordedOperationFutures.add(actorContext.executeOperationAsync(getActor(),
-                isTxActorLocal ? writeData : writeData.toSerializable()));
-        }
-
-        @Override
-        public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
-                final YangInstanceIdentifier path) {
-
-            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.
-
-            if(recordedOperationFutures.isEmpty()) {
-                finishReadData(path, returnFuture);
-            } else {
-                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>>() {
-                    @Override
-                    public void onComplete(Throwable failure, Iterable<Object> notUsed)
-                            throws Throwable {
-                        if(failure != null) {
-                            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);
-                        }
-                    }
-                };
-
-                combinedFutures.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
-            }
-
-            return MappingCheckedFuture.create(returnFuture, ReadFailedException.MAPPER);
-        }
-
-        private void finishReadData(final YangInstanceIdentifier path,
-                final SettableFuture<Optional<NormalizedNode<?, ?>>> returnFuture) {
-
-            LOG.debug("Tx {} finishReadData called path = {}", identifier, path);
-
-            OnComplete<Object> onComplete = new OnComplete<Object>() {
-                @Override
-                public void onComplete(Throwable failure, Object readResponse) throws Throwable {
-                    if(failure != null) {
-                        LOG.debug("Tx {} read operation failed: {}", identifier, failure);
-                        returnFuture.setException(new ReadFailedException(
-                                "Error reading data for path " + path, failure));
-
-                    } else {
-                        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));
-                        }
-                    }
-                }
-            };
-
-            ReadData readData = new ReadData(path);
-            Future<Object> readFuture = actorContext.executeOperationAsync(getActor(),
-                isTxActorLocal ? readData : readData.toSerializable());
-
-            readFuture.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
-        }
-
-        @Override
-        public CheckedFuture<Boolean, ReadFailedException> dataExists(
-                final YangInstanceIdentifier path) {
-
-            LOG.debug("Tx {} dataExists called path = {}", identifier, path);
-
-            final SettableFuture<Boolean> 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 this
-            // request.
-
-            if(recordedOperationFutures.isEmpty()) {
-                finishDataExists(path, returnFuture);
-            } else {
-                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) {
-                            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);
-                        }
-                    }
-                };
-
-                combinedFutures.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
-            }
-
-            return MappingCheckedFuture.create(returnFuture, ReadFailedException.MAPPER);
-        }
-
-        private void finishDataExists(final YangInstanceIdentifier path,
-                final SettableFuture<Boolean> returnFuture) {
-
-            LOG.debug("Tx {} finishDataExists called path = {}", identifier, path);
-
-            OnComplete<Object> onComplete = new OnComplete<Object>() {
-                @Override
-                public void onComplete(Throwable failure, Object response) throws Throwable {
-                    if(failure != null) {
-                        LOG.debug("Tx {} dataExists operation failed: {}", identifier, failure);
-                        returnFuture.setException(new ReadFailedException(
-                                "Error checking data exists for path " + path, failure));
-                    } else {
-                        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));
-                        }
-                    }
-                }
-            };
-
-            DataExists dataExists = new DataExists(path);
-            Future<Object> future = actorContext.executeOperationAsync(getActor(),
-                isTxActorLocal ? dataExists : dataExists.toSerializable());
-
-            future.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
-        }
-    }
-
-    private static class NoOpTransactionContext extends AbstractTransactionContext {
-
-        private final Logger LOG = LoggerFactory.getLogger(NoOpTransactionContext.class);
-
-        private final Throwable failure;
-
-        public NoOpTransactionContext(Throwable failure, TransactionIdentifier identifier){
-            super(identifier);
-            this.failure = failure;
-        }
-
-        @Override
-        public void closeTransaction() {
-            LOG.debug("NoOpTransactionContext {} closeTransaction called", identifier);
-        }
-
-        @Override
-        public Future<ActorSelection> readyTransaction() {
-            LOG.debug("Tx {} readyTransaction called", identifier);
-            return akka.dispatch.Futures.failed(failure);
-        }
-
-        @Override
-        public void deleteData(YangInstanceIdentifier path) {
-            LOG.debug("Tx {} deleteData called path = {}", identifier, path);
-        }
-
-        @Override
-        public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-            LOG.debug("Tx {} mergeData called path = {}", identifier, path);
-        }
-
-        @Override
-        public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-            LOG.debug("Tx {} writeData called path = {}", identifier, path);
-        }
-
-        @Override
-        public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
-                YangInstanceIdentifier path) {
-            LOG.debug("Tx {} readData called path = {}", identifier, path);
-            return Futures.immediateFailedCheckedFuture(new ReadFailedException(
-                    "Error reading data for path " + path, failure));
-        }
-
-        @Override
-        public CheckedFuture<Boolean, ReadFailedException> dataExists(
-                YangInstanceIdentifier path) {
-            LOG.debug("Tx {} dataExists called path = {}", identifier, path);
-            return Futures.immediateFailedCheckedFuture(new ReadFailedException(
-                    "Error checking exists for path " + path, failure));
+            return new TransactionContextImpl(transactionPath, transactionActor, identifier,
+                actorContext, schemaContext, isTxActorLocal, reply.getVersion(), operationCompleter);
         }
     }
 }