Bug 2055: Handle Tx create in TransactionProxy resiliently
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / TransactionProxy.java
index 715f48c3492156d1b14005462da2c26aacb1768c..239207a60ab58cbbbefc03ee150b9cc250230d86 100644 (file)
@@ -9,8 +9,8 @@
 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;
@@ -18,10 +18,10 @@ 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 org.opendaylight.controller.cluster.datastore.exceptions.PrimaryNotFoundException;
+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;
@@ -46,17 +46,17 @@ import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 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 scala.concurrent.Promise;
+import scala.concurrent.duration.FiniteDuration;
 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;
+import javax.annotation.concurrent.GuardedBy;
 
 /**
  * TransactionProxy acts as a proxy for one or more transactions that were created on a remote shard
@@ -72,18 +72,14 @@ import java.util.concurrent.atomic.AtomicLong;
  */
 public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
-    private final TransactionChainProxy transactionChainProxy;
-
-
-
-    public enum TransactionType {
+    public static enum TransactionType {
         READ_ONLY,
         WRITE_ONLY,
         READ_WRITE
     }
 
-    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,9 +88,13 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
     private static final AtomicLong counter = new AtomicLong();
 
-    private static final Logger
-        LOG = LoggerFactory.getLogger(TransactionProxy.class);
+    private static final Logger LOG = LoggerFactory.getLogger(TransactionProxy.class);
 
+    /**
+     * Time interval in between transaction create retries.
+     */
+    private static final FiniteDuration CREATE_TX_TRY_INTERVAL =
+            FiniteDuration.create(1, TimeUnit.SECONDS);
 
     /**
      * Used to enqueue the PhantomReferences for read-only TransactionProxy instances. The
@@ -157,7 +157,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                 for(ActorSelection actor : remoteTransactionActors) {
                     LOG.trace("Sending CloseTransaction to {}", actor);
                     actorContext.sendOperationAsync(actor,
-                        new CloseTransaction().toSerializable());
+                            new CloseTransaction().toSerializable());
                 }
             }
         }
@@ -173,11 +173,15 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
     private List<ActorSelection> remoteTransactionActors;
     private 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;
+    private final TransactionChainProxy transactionChainProxy;
     private final SchemaContext schemaContext;
     private boolean inReadyState;
 
@@ -185,17 +189,8 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         this(actorContext, transactionType, null);
     }
 
-    @VisibleForTesting
-    List<Future<Object>> getRecordedOperationFutures() {
-        List<Future<Object>> recordedOperationFutures = Lists.newArrayList();
-        for(TransactionContext transactionContext : remoteTransactionPaths.values()) {
-            recordedOperationFutures.addAll(transactionContext.getRecordedOperationFutures());
-        }
-
-        return recordedOperationFutures;
-    }
-
-    public TransactionProxy(ActorContext actorContext, TransactionType transactionType, TransactionChainProxy transactionChainProxy) {
+    public TransactionProxy(ActorContext actorContext, TransactionType transactionType,
+            TransactionChainProxy transactionChainProxy) {
         this.actorContext = Preconditions.checkNotNull(actorContext,
             "actorContext should not be null");
         this.transactionType = Preconditions.checkNotNull(transactionType,
@@ -224,9 +219,21 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                 new TransactionProxyCleanupPhantomReference(this);
             phantomReferenceCache.put(cleanup, cleanup);
         }
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("Created txn {} of type {}", identifier, transactionType);
+
+        LOG.debug("Created txn {} of type {}", identifier, transactionType);
+    }
+
+    @VisibleForTesting
+    List<Future<Object>> getRecordedOperationFutures() {
+        List<Future<Object>> recordedOperationFutures = Lists.newArrayList();
+        for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
+            TransactionContext transactionContext = txFutureCallback.getTransactionContext();
+            if(transactionContext != null) {
+                recordedOperationFutures.addAll(transactionContext.getRecordedOperationFutures());
+            }
         }
+
+        return recordedOperationFutures;
     }
 
     @Override
@@ -236,26 +243,82 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         Preconditions.checkState(transactionType != TransactionType.WRITE_ONLY,
                 "Read operation on write-only transaction is not allowed");
 
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("Tx {} read {}", identifier, path);
+        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);
+                            }
+                        });
+                }
+            });
+
+            future = MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
         }
-        createTransactionIfMissing(actorContext, path);
 
-        return transactionContext(path).readData(path);
+        return future;
     }
 
     @Override
-    public CheckedFuture<Boolean, ReadFailedException> exists(YangInstanceIdentifier path) {
+    public CheckedFuture<Boolean, ReadFailedException> exists(final YangInstanceIdentifier path) {
 
         Preconditions.checkState(transactionType != TransactionType.WRITE_ONLY,
                 "Exists operation on write-only transaction is not allowed");
 
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("Tx {} exists {}", identifier, path);
+        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);
+                            }
+                        });
+                }
+            });
+
+            future = MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
         }
-        createTransactionIfMissing(actorContext, path);
 
-        return transactionContext(path).dataExists(path);
+        return future;
     }
 
     private void checkModificationState() {
@@ -266,41 +329,72 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
     }
 
     @Override
-    public void write(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
+    public void write(final YangInstanceIdentifier path, final NormalizedNode<?, ?> data) {
 
         checkModificationState();
 
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("Tx {} write {}", identifier, path);
-        }
-        createTransactionIfMissing(actorContext, path);
+        LOG.debug("Tx {} write {}", identifier, path);
 
-        transactionContext(path).writeData(path, data);
+        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);
+                }
+            });
+        }
     }
 
     @Override
-    public void merge(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
+    public void merge(final YangInstanceIdentifier path, final NormalizedNode<?, ?> data) {
 
         checkModificationState();
 
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("Tx {} merge {}", identifier, path);
-        }
-        createTransactionIfMissing(actorContext, path);
+        LOG.debug("Tx {} merge {}", identifier, path);
 
-        transactionContext(path).mergeData(path, data);
+        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);
+                }
+            });
+        }
     }
 
     @Override
-    public void delete(YangInstanceIdentifier path) {
+    public void delete(final YangInstanceIdentifier path) {
 
         checkModificationState();
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("Tx {} delete {}", identifier, path);
-        }
-        createTransactionIfMissing(actorContext, path);
 
-        transactionContext(path).deleteData(path);
+        LOG.debug("Tx {} delete {}", identifier, path);
+
+        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);
+                }
+            });
+        }
     }
 
     @Override
@@ -310,19 +404,32 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         inReadyState = true;
 
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("Tx {} Trying to get {} transactions ready for commit", identifier,
-                remoteTransactionPaths.size());
-        }
+        LOG.debug("Tx {} Readying {} transactions for commit", identifier,
+                    txFutureCallbackMap.size());
+
         List<Future<ActorSelection>> cohortFutures = Lists.newArrayList();
 
-        for(TransactionContext transactionContext : remoteTransactionPaths.values()) {
+        for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
+
+            LOG.debug("Tx {} Readying transaction for shard {}", identifier,
+                        txFutureCallback.getShardName());
+
+            TransactionContext transactionContext = txFutureCallback.getTransactionContext();
+            if(transactionContext != null) {
+                cohortFutures.add(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() {
+                    @Override
+                    public void invoke(TransactionContext transactionContext) {
+                        cohortPromise.completeWith(transactionContext.readyTransaction());
+                    }
+                });
 
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} Readying transaction for shard {}", identifier,
-                    transactionContext.getShardName());
+                cohortFutures.add(cohortPromise.future());
             }
-            cohortFutures.add(transactionContext.readyTransaction());
         }
 
         if(transactionChainProxy != null){
@@ -340,11 +447,21 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
     @Override
     public void close() {
-        for(TransactionContext transactionContext : remoteTransactionPaths.values()) {
-            transactionContext.closeTransaction();
+        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();
+                    }
+                });
+            }
         }
 
-        remoteTransactionPaths.clear();
+        txFutureCallbackMap.clear();
 
         if(transactionType == TransactionType.READ_ONLY) {
             remoteTransactionActors.clear();
@@ -352,94 +469,211 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         }
     }
 
-    private TransactionContext transactionContext(YangInstanceIdentifier path){
+    private String shardNameFromIdentifier(YangInstanceIdentifier path){
+        return ShardStrategyFactory.getStrategy(path).findShard(path);
+    }
+
+    private TransactionFutureCallback getOrCreateTxFutureCallback(YangInstanceIdentifier path) {
         String shardName = shardNameFromIdentifier(path);
-        return remoteTransactionPaths.get(shardName);
+        TransactionFutureCallback txFutureCallback = txFutureCallbackMap.get(shardName);
+        if(txFutureCallback == null) {
+            Future<ActorSelection> findPrimaryFuture = actorContext.findPrimaryShardAsync(shardName);
+
+            final TransactionFutureCallback newTxFutureCallback =
+                    new TransactionFutureCallback(shardName);
+
+            txFutureCallback = newTxFutureCallback;
+            txFutureCallbackMap.put(shardName, txFutureCallback);
+
+            findPrimaryFuture.onComplete(new OnComplete<ActorSelection>() {
+                @Override
+                public void onComplete(Throwable failure, ActorSelection primaryShard) {
+                    if(failure != null) {
+                        newTxFutureCallback.onComplete(failure, null);
+                    } else {
+                        newTxFutureCallback.setPrimaryShard(primaryShard);
+                    }
+                }
+            }, actorContext.getActorSystem().dispatcher());
+        }
+
+        return txFutureCallback;
     }
 
-    private String shardNameFromIdentifier(YangInstanceIdentifier path){
-        return ShardStrategyFactory.getStrategy(path).findShard(path);
+    public String getTransactionChainId() {
+        if(transactionChainProxy == null){
+            return "";
+        }
+        return transactionChainProxy.getTransactionChainId();
     }
 
-    private void createTransactionIfMissing(ActorContext actorContext,
-        YangInstanceIdentifier path) {
+    /**
+     * Interface for a transaction operation to be invoked later.
+     */
+    private static interface TransactionOperation {
+        void invoke(TransactionContext transactionContext);
+    }
 
-        if(transactionChainProxy != null){
-            transactionChainProxy.waitTillCurrentTransactionReady();
+    /**
+     * Implements a Future OnComplete callback for a CreateTransaction message. This class handles
+     * retries, up to a limit, if the shard doesn't have a leader yet. This is done by scheduling a
+     * retry task after a short delay.
+     * <p>
+     * The end result from a completed CreateTransaction message is a TransactionContext that is
+     * used to perform transaction operations. Transaction operations that occur before the
+     * CreateTransaction completes are cache and executed once the CreateTransaction completes,
+     * successfully or not.
+     */
+    private class TransactionFutureCallback extends OnComplete<Object> {
+
+        /**
+         * The list of transaction operations to execute once the CreateTransaction completes.
+         */
+        @GuardedBy("txOperationsOnComplete")
+        private final List<TransactionOperation> txOperationsOnComplete = Lists.newArrayList();
+
+        /**
+         * The TransactionContext resulting from the CreateTransaction reply.
+         */
+        private volatile TransactionContext transactionContext;
+
+        /**
+         * The target primary shard.
+         */
+        private volatile ActorSelection primaryShard;
+
+        private volatile int createTxTries = (int) (actorContext.getDatastoreContext().
+                getShardLeaderElectionTimeout().duration().toMillis() /
+                CREATE_TX_TRY_INTERVAL.toMillis());
+
+        private final String shardName;
+
+        TransactionFutureCallback(String shardName) {
+            this.shardName = shardName;
         }
 
-        String shardName = ShardStrategyFactory.getStrategy(path).findShard(path);
+        String getShardName() {
+            return shardName;
+        }
 
-        TransactionContext transactionContext =
-            remoteTransactionPaths.get(shardName);
+        TransactionContext getTransactionContext() {
+            return transactionContext;
+        }
 
-        if (transactionContext != null) {
-            // A transaction already exists with that shard
-            return;
+
+        /**
+         * Sets the target primary shard and initiates a CreateTransaction try.
+         */
+        void setPrimaryShard(ActorSelection primaryShard) {
+            LOG.debug("Tx {} Primary shard found - trying create transaction", identifier);
+
+            this.primaryShard = primaryShard;
+            tryCreateTransaction();
         }
 
-        try {
-            Optional<ActorSelection> primaryShard = actorContext.findPrimaryShard(shardName);
-            if (!primaryShard.isPresent()) {
-                throw new PrimaryNotFoundException("Primary could not be found for shard " + shardName);
+        /**
+         * Adds a TransactionOperation to be executed after the CreateTransaction completes.
+         */
+        void addTxOperationOnComplete(TransactionOperation operation) {
+            synchronized(txOperationsOnComplete) {
+                if(transactionContext == null) {
+                    LOG.debug("Tx {} Adding operation on complete {}", identifier);
+
+                    txOperationsOnComplete.add(operation);
+                } else {
+                    operation.invoke(transactionContext);
+                }
             }
+        }
 
-            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);
+        /**
+         * Performs a CreateTransaction try async.
+         */
+        private void tryCreateTransaction() {
+            Future<Object> createTxFuture = actorContext.executeOperationAsync(primaryShard,
+                    new CreateTransaction(identifier.toString(),
+                            TransactionProxy.this.transactionType.ordinal(),
+                            getTransactionChainId()).toSerializable());
 
-                String transactionPath = reply.getTransactionPath();
+            createTxFuture.onComplete(this, actorContext.getActorSystem().dispatcher());
+        }
 
-                if(LOG.isDebugEnabled()) {
-                    LOG.debug("Tx {} Received transaction path = {}", identifier, transactionPath);
+        @Override
+        public void onComplete(Throwable failure, Object response) {
+            if(failure instanceof NoShardLeaderException) {
+                // There's no leader for the shard yet - schedule and try again, unless we're out
+                // of retries. Note: createTxTries is volatile as it may be written by different
+                // threads however not concurrently, therefore decrementing it non-atomically here
+                // is ok.
+                if(--createTxTries > 0) {
+                    LOG.debug("Tx {} Shard {} has no leader yet - scheduling create Tx retry",
+                            identifier, shardName);
+
+                    actorContext.getActorSystem().scheduler().scheduleOnce(CREATE_TX_TRY_INTERVAL,
+                            new Runnable() {
+                                @Override
+                                public void run() {
+                                    tryCreateTransaction();
+                                }
+                            }, actorContext.getActorSystem().dispatcher());
+                    return;
                 }
-                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);
+            // 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(
+                        "Invalid reply type %s for CreateTransaction", response.getClass()));
+
+                    transactionContext = new NoOpTransactionContext(exception, identifier);
+                }
 
-                    // Write to the memory barrier volatile to publish the above update to the
-                    // remoteTransactionActors list for thread visibility.
-                    remoteTransactionActorsMB.set(true);
+                for(TransactionOperation oper: txOperationsOnComplete) {
+                    oper.invoke(transactionContext);
                 }
 
-                // 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);
+                txOperationsOnComplete.clear();
+            }
+        }
 
-                transactionContext = new TransactionContextImpl(shardName, transactionPath,
-                    transactionActor, identifier, actorContext, schemaContext, isTxActorLocal);
+        private void createValidTransactionContext(CreateTransactionReply reply) {
+            String transactionPath = reply.getTransactionPath();
 
-                remoteTransactionPaths.put(shardName, transactionContext);
-            } else {
-                throw new IllegalArgumentException(String.format(
-                    "Invalid reply type {} for CreateTransaction", response.getClass()));
-            }
-        } catch (Exception e) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} Creating NoOpTransaction because of : {}", identifier, e.getMessage());
+            LOG.debug("Tx {} Received transaction actor 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);
             }
-            remoteTransactionPaths
-                .put(shardName, new NoOpTransactionContext(shardName, e, identifier));
-        }
-    }
 
-    public String getTransactionChainId() {
-        if(transactionChainProxy == null){
-            return "";
+            // 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(transactionActor, identifier,
+                actorContext, schemaContext, isTxActorLocal);
         }
-        return transactionChainProxy.getTransactionChainId();
     }
 
-
     private interface TransactionContext {
-        String getShardName();
-
         void closeTransaction();
 
         Future<ActorSelection> readyTransaction();
@@ -461,19 +695,12 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
     private static abstract class AbstractTransactionContext implements TransactionContext {
 
         protected final TransactionIdentifier identifier;
-        protected final String shardName;
         protected final List<Future<Object>> recordedOperationFutures = Lists.newArrayList();
 
-        AbstractTransactionContext(String shardName, TransactionIdentifier identifier) {
-            this.shardName = shardName;
+        AbstractTransactionContext(TransactionIdentifier identifier) {
             this.identifier = identifier;
         }
 
-        @Override
-        public String getShardName() {
-            return shardName;
-        }
-
         @Override
         public List<Future<Object>> getRecordedOperationFutures() {
             return recordedOperationFutures;
@@ -485,15 +712,13 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         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;
+        private TransactionContextImpl(ActorSelection actor, TransactionIdentifier identifier,
+                ActorContext actorContext, SchemaContext schemaContext,
+                boolean isTxActorLocal) {
+            super(identifier);
             this.actor = actor;
             this.actorContext = actorContext;
             this.schemaContext = schemaContext;
@@ -506,18 +731,16 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         @Override
         public void closeTransaction() {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} closeTransaction called", identifier);
-            }
+            LOG.debug("Tx {} closeTransaction called", identifier);
+
             actorContext.sendOperationAsync(getActor(), new CloseTransaction().toSerializable());
         }
 
         @Override
         public Future<ActorSelection> readyTransaction() {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} readyTransaction called with {} previous recorded operations pending",
+            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();
@@ -540,13 +763,12 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             // 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 AbstractFunction1<Iterable<Object>, ActorSelection>() {
+            return combinedFutures.transform(new Mapper<Iterable<Object>, ActorSelection>() {
                 @Override
-                public ActorSelection apply(Iterable<Object> notUsed) {
-                    if(LOG.isDebugEnabled()) {
-                        LOG.debug("Tx {} readyTransaction: pending recorded operations succeeded",
+                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
@@ -574,9 +796,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         @Override
         public void deleteData(YangInstanceIdentifier path) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} deleteData called path = {}", identifier, path);
-            }
+            LOG.debug("Tx {} deleteData called path = {}", identifier, path);
 
             DeleteData deleteData = new DeleteData(path);
             recordedOperationFutures.add(actorContext.executeOperationAsync(getActor(),
@@ -585,9 +805,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         @Override
         public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} mergeData called path = {}", identifier, path);
-            }
+            LOG.debug("Tx {} mergeData called path = {}", identifier, path);
 
             MergeData mergeData = new MergeData(path, data, schemaContext);
             recordedOperationFutures.add(actorContext.executeOperationAsync(getActor(),
@@ -596,9 +814,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         @Override
         public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} writeData called path = {}", identifier, path);
-            }
+            LOG.debug("Tx {} writeData called path = {}", identifier, path);
 
             WriteData writeData = new WriteData(path, data, schemaContext);
             recordedOperationFutures.add(actorContext.executeOperationAsync(getActor(),
@@ -609,9 +825,8 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
                 final YangInstanceIdentifier path) {
 
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} readData called path = {}", identifier, 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
@@ -621,10 +836,9 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             if(recordedOperationFutures.isEmpty()) {
                 finishReadData(path, returnFuture);
             } else {
-                if(LOG.isDebugEnabled()) {
-                    LOG.debug("Tx {} readData: verifying {} previous recorded operations",
+                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.
@@ -638,10 +852,8 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                     public void onComplete(Throwable failure, Iterable<Object> notUsed)
                             throws Throwable {
                         if(failure != null) {
-                            if(LOG.isDebugEnabled()) {
-                                LOG.debug("Tx {} readData: a recorded operation failed: {}",
+                            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));
@@ -660,23 +872,18 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         private void finishReadData(final YangInstanceIdentifier path,
                 final SettableFuture<Optional<NormalizedNode<?, ?>>> returnFuture) {
 
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} finishReadData called path = {}", identifier, path);
-            }
+            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) {
-                        if(LOG.isDebugEnabled()) {
-                            LOG.debug("Tx {} read operation failed: {}", identifier, failure);
-                        }
+                        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);
-                        }
+                        LOG.debug("Tx {} read operation succeeded", identifier, failure);
 
                         if (readResponse instanceof ReadDataReply) {
                             ReadDataReply reply = (ReadDataReply) readResponse;
@@ -705,9 +912,8 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         public CheckedFuture<Boolean, ReadFailedException> dataExists(
                 final YangInstanceIdentifier path) {
 
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} dataExists called path = {}", identifier, 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
@@ -718,10 +924,9 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             if(recordedOperationFutures.isEmpty()) {
                 finishDataExists(path, returnFuture);
             } else {
-                if(LOG.isDebugEnabled()) {
-                    LOG.debug("Tx {} dataExists: verifying {} previous recorded operations",
+                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.
@@ -734,10 +939,8 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                     public void onComplete(Throwable failure, Iterable<Object> notUsed)
                             throws Throwable {
                         if(failure != null) {
-                            if(LOG.isDebugEnabled()) {
-                                LOG.debug("Tx {} dataExists: a recorded operation failed: {}",
+                            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));
@@ -756,22 +959,17 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         private void finishDataExists(final YangInstanceIdentifier path,
                 final SettableFuture<Boolean> returnFuture) {
 
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} finishDataExists called path = {}", identifier, path);
-            }
+            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) {
-                        if(LOG.isDebugEnabled()) {
-                            LOG.debug("Tx {} dataExists operation failed: {}", identifier, failure);
-                        }
+                        LOG.debug("Tx {} dataExists operation failed: {}", identifier, failure);
                         returnFuture.setException(new ReadFailedException(
                                 "Error checking data exists for path " + path, failure));
                     } else {
-                        if(LOG.isDebugEnabled()) {
-                            LOG.debug("Tx {} dataExists operation succeeded", identifier, failure);
-                        }
+                        LOG.debug("Tx {} dataExists operation succeeded", identifier, failure);
 
                         if (response instanceof DataExistsReply) {
                             returnFuture.set(Boolean.valueOf(((DataExistsReply) response).exists()));
@@ -799,66 +997,51 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         private final Logger LOG = LoggerFactory.getLogger(NoOpTransactionContext.class);
 
-        private final Exception failure;
+        private final Throwable failure;
 
-        public NoOpTransactionContext(String shardName, Exception failure,
-                TransactionIdentifier identifier){
-            super(shardName, identifier);
+        public NoOpTransactionContext(Throwable failure, TransactionIdentifier identifier){
+            super(identifier);
             this.failure = failure;
         }
 
         @Override
         public void closeTransaction() {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("NoOpTransactionContext {} closeTransaction called", identifier);
-            }
+            LOG.debug("NoOpTransactionContext {} closeTransaction called", identifier);
         }
 
         @Override
         public Future<ActorSelection> readyTransaction() {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} readyTransaction called", identifier);
-            }
+            LOG.debug("Tx {} readyTransaction called", identifier);
             return akka.dispatch.Futures.failed(failure);
         }
 
         @Override
         public void deleteData(YangInstanceIdentifier path) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} deleteData called path = {}", identifier, path);
-            }
+            LOG.debug("Tx {} deleteData called path = {}", identifier, path);
         }
 
         @Override
         public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} mergeData called path = {}", identifier, path);
-            }
+            LOG.debug("Tx {} mergeData called path = {}", identifier, path);
         }
 
         @Override
         public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} writeData called path = {}", identifier, path);
-            }
+            LOG.debug("Tx {} writeData called path = {}", identifier, path);
         }
 
         @Override
         public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
-            YangInstanceIdentifier path) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} readData called path = {}", identifier, path);
-            }
+                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) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} dataExists called path = {}", identifier, path);
-            }
+                YangInstanceIdentifier path) {
+            LOG.debug("Tx {} dataExists called path = {}", identifier, path);
             return Futures.immediateFailedCheckedFuture(new ReadFailedException(
                     "Error checking exists for path " + path, failure));
         }