Adjust Tx rate limiter for unused transactions
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / TransactionProxy.java
index 6183c489c4cdbc56a1ba9ac0c3c3939963277edb..f1ba4eabb999a6455688d93532877ae490df88df 100644 (file)
@@ -8,31 +8,36 @@
 
 package org.opendaylight.controller.cluster.datastore;
 
-import akka.actor.ActorPath;
-import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
-import akka.actor.Props;
+import akka.dispatch.Mapper;
 import akka.dispatch.OnComplete;
-
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.FinalizablePhantomReference;
+import com.google.common.base.FinalizableReferenceQueue;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.CheckedFuture;
-import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
-
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+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.compat.PreLithiumTransactionContextImpl;
+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;
@@ -44,14 +49,9 @@ 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.concurrent.Future;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicLong;
+import scala.concurrent.Promise;
+import scala.concurrent.duration.FiniteDuration;
 
 /**
  * TransactionProxy acts as a proxy for one or more transactions that were created on a remote shard
@@ -66,111 +66,319 @@ import java.util.concurrent.atomic.AtomicLong;
  * </p>
  */
 public class TransactionProxy implements DOMStoreReadWriteTransaction {
-    public enum TransactionType {
+
+    public static enum TransactionType {
         READ_ONLY,
         WRITE_ONLY,
-        READ_WRITE
+        READ_WRITE;
+
+        public static TransactionType fromInt(int type) {
+            if(type == WRITE_ONLY.ordinal()) {
+                return WRITE_ONLY;
+            } else if(type == READ_WRITE.ordinal()) {
+                return READ_WRITE;
+            } else if(type == READ_ONLY.ordinal()) {
+                return READ_ONLY;
+            } else {
+                throw new IllegalArgumentException("In TransactionType enum value" + type);
+            }
+        }
     }
 
+    static final Mapper<Throwable, Throwable> SAME_FAILURE_TRANSFORMER =
+                                                              new Mapper<Throwable, Throwable>() {
+        @Override
+        public Throwable apply(Throwable failure) {
+            return failure;
+        }
+    };
+
     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
+     * FinalizableReferenceQueue is safe to use statically in an OSGi environment as it uses some
+     * trickery to clean up its internal thread when the bundle is unloaded.
+     */
+    private static final FinalizableReferenceQueue phantomReferenceQueue =
+                                                                  new FinalizableReferenceQueue();
+
+    /**
+     * This stores the TransactionProxyCleanupPhantomReference instances statically, This is
+     * necessary because PhantomReferences need a hard reference so they're not garbage collected.
+     * Once finalized, the TransactionProxyCleanupPhantomReference removes itself from this map
+     * and thus becomes eligible for garbage collection.
+     */
+    private static final Map<TransactionProxyCleanupPhantomReference,
+                             TransactionProxyCleanupPhantomReference> phantomReferenceCache =
+                                                                        new ConcurrentHashMap<>();
+
+    /**
+     * A PhantomReference that closes remote transactions for a TransactionProxy when it's
+     * garbage collected. This is used for read-only transactions as they're not explicitly closed
+     * by clients. So the only way to detect that a transaction is no longer in use and it's safe
+     * to clean up is when it's garbage collected. It's inexact as to when an instance will be GC'ed
+     * but TransactionProxy instances should generally be short-lived enough to avoid being moved
+     * to the old generation space and thus should be cleaned up in a timely manner as the GC
+     * runs on the young generation (eden, swap1...) space much more frequently.
+     */
+    private static class TransactionProxyCleanupPhantomReference
+                                           extends FinalizablePhantomReference<TransactionProxy> {
+
+        private final List<ActorSelection> remoteTransactionActors;
+        private final AtomicBoolean remoteTransactionActorsMB;
+        private final ActorContext actorContext;
+        private final TransactionIdentifier identifier;
+
+        protected TransactionProxyCleanupPhantomReference(TransactionProxy referent) {
+            super(referent, phantomReferenceQueue);
+
+            // Note we need to cache the relevant fields from the TransactionProxy as we can't
+            // have a hard reference to the TransactionProxy instance itself.
+
+            remoteTransactionActors = referent.remoteTransactionActors;
+            remoteTransactionActorsMB = referent.remoteTransactionActorsMB;
+            actorContext = referent.actorContext;
+            identifier = referent.identifier;
+        }
+
+        @Override
+        public void finalizeReferent() {
+            LOG.trace("Cleaning up {} Tx actors for TransactionProxy {}",
+                    remoteTransactionActors.size(), identifier);
+
+            phantomReferenceCache.remove(this);
+
+            // Access the memory barrier volatile to ensure all previous updates to the
+            // remoteTransactionActors list are visible to this thread.
+
+            if(remoteTransactionActorsMB.get()) {
+                for(ActorSelection actor : remoteTransactionActors) {
+                    LOG.trace("Sending CloseTransaction to {}", actor);
+                    actorContext.sendOperationAsync(actor, CloseTransaction.INSTANCE.toSerializable());
+                }
+            }
+        }
+    }
 
+    /**
+     * Stores the remote Tx actors for each requested data store path to be used by the
+     * PhantomReference to close the remote Tx's. This is only used for read-only Tx's. The
+     * remoteTransactionActorsMB volatile serves as a memory barrier to publish updates to the
+     * remoteTransactionActors list so they will be visible to the thread accessing the
+     * PhantomReference.
+     */
+    private List<ActorSelection> remoteTransactionActors;
+    private volatile AtomicBoolean remoteTransactionActorsMB;
+
+    /**
+     * 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 Map<String, TransactionContext> remoteTransactionPaths = new HashMap<>();
     private final TransactionIdentifier identifier;
+    private final String transactionChainId;
     private final SchemaContext schemaContext;
     private boolean inReadyState;
 
+    private volatile boolean initialized;
+    private Semaphore operationLimiter;
+    private OperationCompleter operationCompleter;
+
+    public TransactionProxy(ActorContext actorContext, TransactionType transactionType) {
+        this(actorContext, transactionType, "");
+    }
+
     public TransactionProxy(ActorContext actorContext, TransactionType transactionType,
-            SchemaContext schemaContext) {
-        this.actorContext = Preconditions.checkNotNull(actorContext, "actorContext should not be null");
-        this.transactionType = Preconditions.checkNotNull(transactionType, "transactionType should not be null");
-        this.schemaContext = Preconditions.checkNotNull(schemaContext, "schemaContext should not be null");
+            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.transactionChainId = transactionChainId;
 
         String memberName = actorContext.getCurrentMemberName();
         if(memberName == null){
             memberName = "UNKNOWN-MEMBER";
         }
 
-        this.identifier = TransactionIdentifier.builder().memberName(memberName).counter(
-                counter.getAndIncrement()).build();
+        this.identifier = new TransactionIdentifier(memberName, counter.getAndIncrement());
+
+        LOG.debug("Created txn {} of type {} on chain {}", identifier, transactionType, transactionChainId);
+    }
+
+    @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;
+    }
 
-        LOG.debug("Created txn {} of type {}", identifier, transactionType);
+    @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("txn {} read {}", identifier, path);
+        LOG.debug("Tx {} read {}", identifier, path);
+
+        throttleOperation();
 
-        createTransactionIfMissing(actorContext, path);
+        final SettableFuture<Optional<NormalizedNode<?, ?>>> proxyFuture = SettableFuture.create();
 
-        return transactionContext(path).readData(path);
+        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
-    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");
 
-        LOG.debug("txn {} exists {}", identifier, path);
+        LOG.debug("Tx {} exists {}", identifier, path);
+
+        throttleOperation();
 
-        createTransactionIfMissing(actorContext, path);
+        final SettableFuture<Boolean> proxyFuture = SettableFuture.create();
 
-        return transactionContext(path).dataExists(path);
+        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() {
         Preconditions.checkState(transactionType != TransactionType.READ_ONLY,
                 "Modification operation on read-only transaction is not allowed");
         Preconditions.checkState(!inReadyState,
-                "Transaction is sealed - further modifications are allowed");
+                "Transaction is sealed - further modifications are not allowed");
     }
 
+    private void throttleOperation() {
+        throttleOperation(1);
+    }
+
+    private void throttleOperation(int acquirePermits) {
+        if(!initialized) {
+            // Note : Currently mailbox-capacity comes from akka.conf and not from the config-subsystem
+            operationLimiter = new Semaphore(actorContext.getTransactionOutstandingOperationLimit());
+            operationCompleter = new OperationCompleter(operationLimiter);
+
+            // Make sure we write this last because it's volatile and will also publish the non-volatile writes
+            // above as well so they'll be visible to other threads.
+            initialized = true;
+        }
+
+        try {
+            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(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
+    public void write(final YangInstanceIdentifier path, final NormalizedNode<?, ?> data) {
 
         checkModificationState();
 
-        LOG.debug("txn {} write {}", identifier, path);
+        LOG.debug("Tx {} write {}", identifier, path);
 
-        createTransactionIfMissing(actorContext, path);
+        throttleOperation();
 
-        transactionContext(path).writeData(path, data);
+        TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
+        txFutureCallback.enqueueTransactionOperation(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();
 
-        LOG.debug("txn {} merge {}", identifier, path);
+        LOG.debug("Tx {} merge {}", identifier, path);
 
-        createTransactionIfMissing(actorContext, path);
+        throttleOperation();
 
-        transactionContext(path).mergeData(path, data);
+        TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
+        txFutureCallback.enqueueTransactionOperation(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();
 
-        LOG.debug("txn {} delete {}", identifier, path);
+        LOG.debug("Tx {} delete {}", identifier, path);
 
-        createTransactionIfMissing(actorContext, path);
+        throttleOperation();
 
-        transactionContext(path).deleteData(path);
+        TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
+        txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
+            @Override
+            public void invoke(TransactionContext transactionContext) {
+                transactionContext.deleteData(path);
+            }
+        });
     }
 
     @Override
@@ -180,31 +388,55 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         inReadyState = true;
 
-        List<ActorPath> cohortPaths = new ArrayList<>();
+        LOG.debug("Tx {} Readying {} transactions for commit", identifier,
+                    txFutureCallbackMap.size());
+
+        if(txFutureCallbackMap.size() == 0) {
+            onTransactionReady(Collections.<Future<ActorSelection>>emptyList());
+            TransactionRateLimitingCallback.adjustRateLimitForUnusedTransaction(actorContext);
+            return NoOpDOMStoreThreePhaseCommitCohort.INSTANCE;
+        }
 
-        LOG.debug("txn {} Trying to get {} transactions ready for commit", identifier,
-                remoteTransactionPaths.size());
+        throttleOperation(txFutureCallbackMap.size());
 
-        for(TransactionContext transactionContext : remoteTransactionPaths.values()) {
+        List<Future<ActorSelection>> cohortFutures = Lists.newArrayList();
 
-            LOG.debug("txn {} Readying transaction for shard {}", identifier,
-                    transactionContext.getShardName());
+        for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
 
-            Object result = transactionContext.readyTransaction();
+            LOG.debug("Tx {} Readying transaction for shard {} chain {}", identifier,
+                        txFutureCallback.getShardName(), transactionChainId);
 
-            if(result.getClass().equals(ReadyTransactionReply.SERIALIZABLE_CLASS)){
-                ReadyTransactionReply reply = ReadyTransactionReply.fromSerializable(
-                        actorContext.getActorSystem(),result);
-                String resolvedCohortPath = transactionContext.getResolvedCohortPath(
-                        reply.getCohortPath().toString());
-                cohortPaths.add(actorContext.actorFor(resolvedCohortPath));
+            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 {
-                LOG.error("Was expecting {} but got {}", ReadyTransactionReply.SERIALIZABLE_CLASS,
-                        result.getClass());
+                final Promise<ActorSelection> promise = akka.dispatch.Futures.promise();
+                txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
+                    @Override
+                    public void invoke(TransactionContext transactionContext) {
+                        promise.completeWith(transactionContext.readyTransaction());
+                    }
+                });
+                future = promise.future();
             }
+
+            cohortFutures.add(future);
         }
 
-        return new ThreePhaseCommitCohortProxy(actorContext, cohortPaths, identifier.toString());
+        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) {
     }
 
     @Override
@@ -214,275 +446,358 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
     @Override
     public void close() {
-        for(TransactionContext transactionContext : remoteTransactionPaths.values()) {
-            transactionContext.closeTransaction();
+        for (TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
+            txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
+                @Override
+                public void invoke(TransactionContext transactionContext) {
+                    transactionContext.closeTransaction();
+                }
+            });
         }
-    }
 
-    private TransactionContext transactionContext(YangInstanceIdentifier path){
-        String shardName = shardNameFromIdentifier(path);
-        return remoteTransactionPaths.get(shardName);
+        txFutureCallbackMap.clear();
+
+        if(remoteTransactionActorsMB != null) {
+            remoteTransactionActors.clear();
+            remoteTransactionActorsMB.set(true);
+        }
     }
 
     private String shardNameFromIdentifier(YangInstanceIdentifier path){
         return ShardStrategyFactory.getStrategy(path).findShard(path);
     }
 
-    private void createTransactionIfMissing(ActorContext actorContext, YangInstanceIdentifier path) {
-        String shardName = ShardStrategyFactory.getStrategy(path).findShard(path);
-
-        TransactionContext transactionContext =
-            remoteTransactionPaths.get(shardName);
-
-        if(transactionContext != null){
-            // A transaction already exists with that shard
-            return;
-        }
-
-        try {
-            Object response = actorContext.executeShardOperation(shardName,
-                new CreateTransaction(identifier.toString(),this.transactionType.ordinal() ).toSerializable(),
-                ActorContext.ASK_DURATION);
-            if (response.getClass().equals(CreateTransactionReply.SERIALIZABLE_CLASS)) {
-                CreateTransactionReply reply =
-                    CreateTransactionReply.fromSerializable(response);
-
-                String transactionPath = reply.getTransactionPath();
-
-                LOG.debug("txn {} Received transaction path = {}", identifier, transactionPath);
-
-                ActorSelection transactionActor =
-                    actorContext.actorSelection(transactionPath);
-                transactionContext =
-                    new TransactionContextImpl(shardName, transactionPath,
-                        transactionActor);
-
-                remoteTransactionPaths.put(shardName, transactionContext);
-            } else {
-                LOG.error("Was expecting {} but got {}", CreateTransactionReply.SERIALIZABLE_CLASS,
-                        response.getClass());
-            }
-        } catch(Exception e){
-            LOG.error("txn {} Creating NoOpTransaction because of : {}", identifier, e.getMessage());
-            remoteTransactionPaths.put(shardName, new NoOpTransactionContext(shardName, e));
-        }
+    protected Future<ActorSelection> sendFindPrimaryShardAsync(String shardName) {
+        return actorContext.findPrimaryShardAsync(shardName);
     }
 
-    private interface TransactionContext {
-        String getShardName();
-
-        String getResolvedCohortPath(String cohortPath);
+    private TransactionFutureCallback getOrCreateTxFutureCallback(YangInstanceIdentifier path) {
+        String shardName = shardNameFromIdentifier(path);
+        TransactionFutureCallback txFutureCallback = txFutureCallbackMap.get(shardName);
+        if(txFutureCallback == null) {
+            Future<ActorSelection> findPrimaryFuture = sendFindPrimaryShardAsync(shardName);
 
-        public void closeTransaction();
+            final TransactionFutureCallback newTxFutureCallback = new TransactionFutureCallback(shardName);
 
-        public Object readyTransaction();
+            txFutureCallback = newTxFutureCallback;
+            txFutureCallbackMap.put(shardName, txFutureCallback);
 
-        void deleteData(YangInstanceIdentifier path);
+            findPrimaryFuture.onComplete(new OnComplete<ActorSelection>() {
+                @Override
+                public void onComplete(Throwable failure, ActorSelection primaryShard) {
+                    if(failure != null) {
+                        newTxFutureCallback.createTransactionContext(failure, null);
+                    } else {
+                        newTxFutureCallback.setPrimaryShard(primaryShard);
+                    }
+                }
+            }, actorContext.getClientDispatcher());
+        }
 
-        void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
+        return txFutureCallback;
+    }
 
-        CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
-                final YangInstanceIdentifier path);
+    public String getTransactionChainId() {
+        return transactionChainId;
+    }
 
-        void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
+    protected ActorContext getActorContext() {
+        return actorContext;
+    }
 
-        CheckedFuture<Boolean, ReadFailedException> dataExists(YangInstanceIdentifier path);
+    /**
+     * Interfaces for transaction operations to be invoked later.
+     */
+    private static interface TransactionOperation {
+        void invoke(TransactionContext transactionContext);
     }
 
+    /**
+     * 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 class TransactionContextImpl implements TransactionContext {
         private final String shardName;
-        private final String actorPath;
-        private final ActorSelection actor;
 
-
-        private TransactionContextImpl(String shardName, String actorPath,
-            ActorSelection actor) {
+        TransactionFutureCallback(String shardName) {
             this.shardName = shardName;
-            this.actorPath = actorPath;
-            this.actor = actor;
         }
 
-        @Override
-        public String getShardName() {
+        String getShardName() {
             return shardName;
         }
 
-        private ActorSelection getActor() {
-            return actor;
+        TransactionContext getTransactionContext() {
+            return transactionContext;
         }
 
-        @Override
-        public String getResolvedCohortPath(String cohortPath) {
-            return actorContext.resolvePath(actorPath, cohortPath);
-        }
 
-        @Override
-        public void closeTransaction() {
-            actorContext.sendRemoteOperationAsync(getActor(), new CloseTransaction().toSerializable());
-        }
+        /**
+         * Sets the target primary shard and initiates a CreateTransaction try.
+         */
+        void setPrimaryShard(ActorSelection primaryShard) {
+            this.primaryShard = primaryShard;
 
-        @Override
-        public Object readyTransaction() {
-            return actorContext.executeRemoteOperation(getActor(),
-                    new ReadyTransaction().toSerializable(), ActorContext.ASK_DURATION);
-        }
+            if(transactionType == TransactionType.WRITE_ONLY &&
+                    actorContext.getDatastoreContext().isWriteOnlyTransactionOptimizationsEnabled()) {
+                LOG.debug("Tx {} Primary shard {} found - creating WRITE_ONLY transaction context",
+                        identifier, primaryShard);
 
-        @Override
-        public void deleteData(YangInstanceIdentifier path) {
-            actorContext.sendRemoteOperationAsync(getActor(), new DeleteData(path).toSerializable() );
+                // For write-only Tx's we prepare the transaction modifications directly on the shard actor
+                // to avoid the overhead of creating a separate transaction actor.
+                // FIXME: can't assume the shard version is LITHIUM_VERSION - need to obtain it somehow.
+                executeTxOperatonsOnComplete(createValidTransactionContext(this.primaryShard,
+                        this.primaryShard.path().toString(), DataStoreVersions.LITHIUM_VERSION));
+            } else {
+                tryCreateTransaction();
+            }
         }
 
-        @Override
-        public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-            actorContext.sendRemoteOperationAsync(getActor(),
-                    new MergeData(path, data, schemaContext).toSerializable());
+        /**
+         * 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);
+                }
+            }
+
+            if(invokeOperation) {
+                operation.invoke(transactionContext);
+            }
         }
 
-        @Override
-        public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
-            final YangInstanceIdentifier path) {
+        void enqueueTransactionOperation(final TransactionOperation op) {
 
-            final SettableFuture<Optional<NormalizedNode<?, ?>>> returnFuture = SettableFuture.create();
+            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);
+            }
+        }
 
-            OnComplete<Object> onComplete = new OnComplete<Object>() {
-                @Override
-                public void onComplete(Throwable failure, Object response) throws Throwable {
-                    if(failure != null) {
-                        returnFuture.setException(new ReadFailedException(
-                                "Error reading data for path " + path, failure));
-                    } else {
-                        if (response.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)) {
-                            ReadDataReply reply = ReadDataReply.fromSerializable(schemaContext,
-                                    path, response);
-                            if (reply.getNormalizedNode() == null) {
-                                returnFuture.set(Optional.<NormalizedNode<?, ?>>absent());
-                            } else {
-                                returnFuture.set(Optional.<NormalizedNode<?, ?>>of(
-                                        reply.getNormalizedNode()));
-                            }
-                        } else {
-                            returnFuture.setException(new ReadFailedException(
-                                    "Invalid response reading data for path " + path));
-                        }
-                    }
-                }
-            };
+        /**
+         * Performs a CreateTransaction try async.
+         */
+        private void tryCreateTransaction() {
+            if(LOG.isDebugEnabled()) {
+                LOG.debug("Tx {} Primary shard {} found - trying create transaction", identifier, primaryShard);
+            }
+
+            Object serializedCreateMessage = new CreateTransaction(identifier.toString(),
+                    TransactionProxy.this.transactionType.ordinal(),
+                    getTransactionChainId()).toSerializable();
 
-            Future<Object> future = actorContext.executeRemoteOperationAsync(getActor(),
-                    new ReadData(path).toSerializable(), ActorContext.ASK_DURATION);
-            future.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
+            Future<Object> createTxFuture = actorContext.executeOperationAsync(primaryShard, serializedCreateMessage);
 
-            return MappingCheckedFuture.create(returnFuture, ReadFailedException.MAPPER);
+            createTxFuture.onComplete(this, actorContext.getClientDispatcher());
         }
 
         @Override
-        public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-            actorContext.sendRemoteOperationAsync(getActor(),
-                    new WriteData(path, data, schemaContext).toSerializable());
+        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.getClientDispatcher());
+                    return;
+                }
+            }
+
+            createTransactionContext(failure, response);
         }
 
-        @Override
-        public CheckedFuture<Boolean, ReadFailedException> dataExists(
-                final YangInstanceIdentifier path) {
+        private void createTransactionContext(Throwable failure, Object response) {
+            // Mainly checking for state violation here to perform a volatile read of "initialized" to
+            // ensure updates to operationLimter et al are visible to this thread (ie we're doing
+            // "piggy-back" synchronization here).
+            Preconditions.checkState(initialized, "Tx was not propertly initialized.");
+
+            // 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);
+
+                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()));
 
-            final SettableFuture<Boolean> returnFuture = SettableFuture.create();
+                localTransactionContext = new NoOpTransactionContext(exception, identifier, operationLimiter);
+            }
 
-            OnComplete<Object> onComplete = new OnComplete<Object>() {
-                @Override
-                public void onComplete(Throwable failure, Object response) throws Throwable {
-                    if(failure != null) {
-                        returnFuture.setException(new ReadFailedException(
-                                "Error checking exists for path " + path, failure));
-                    } 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));
-                        }
+            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;
                     }
+
+                    operationsBatch = new ArrayList<>(txOperationsOnComplete);
+                    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);
                 }
-            };
+            }
+        }
 
-            Future<Object> future = actorContext.executeRemoteOperationAsync(getActor(),
-                    new DataExists(path).toSerializable(), ActorContext.ASK_DURATION);
-            future.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
+        private TransactionContext createValidTransactionContext(CreateTransactionReply reply) {
+            LOG.debug("Tx {} Received {}", identifier, reply);
 
-            return MappingCheckedFuture.create(returnFuture, ReadFailedException.MAPPER);
+            return createValidTransactionContext(actorContext.actorSelection(reply.getTransactionPath()),
+                    reply.getTransactionPath(), reply.getVersion());
         }
-    }
 
-    private class NoOpTransactionContext implements TransactionContext {
+        private TransactionContext createValidTransactionContext(ActorSelection transactionActor,
+                String transactionPath, short remoteTransactionVersion) {
 
-        private final Logger
-            LOG = LoggerFactory.getLogger(NoOpTransactionContext.class);
+            if (transactionType == TransactionType.READ_ONLY) {
+                // Read-only Tx's aren't explicitly closed by the client so we create a PhantomReference
+                // to close the remote Tx's when this instance is no longer in use and is garbage
+                // collected.
 
-        private final String shardName;
-        private final Exception failure;
+                if(remoteTransactionActorsMB == null) {
+                    remoteTransactionActors = Lists.newArrayList();
+                    remoteTransactionActorsMB = new AtomicBoolean();
 
-        private ActorRef cohort;
+                    TransactionProxyCleanupPhantomReference cleanup =
+                            new TransactionProxyCleanupPhantomReference(TransactionProxy.this);
+                    phantomReferenceCache.put(cleanup, cleanup);
+                }
 
-        public NoOpTransactionContext(String shardName, Exception failure){
-            this.shardName = shardName;
-            this.failure = failure;
-        }
+                // Add the actor to the remoteTransactionActors list for access by the
+                // cleanup PhantonReference.
+                remoteTransactionActors.add(transactionActor);
 
-        @Override
-        public String getShardName() {
-            return  shardName;
+                // Write to the memory barrier volatile to publish the above update to the
+                // remoteTransactionActors list for thread visibility.
+                remoteTransactionActorsMB.set(true);
+            }
 
+            // TxActor is always created where the leader of the shard is.
+            // Check if TxActor is created in the same node
+            boolean isTxActorLocal = actorContext.isPathLocal(transactionPath);
+
+            if(remoteTransactionVersion < DataStoreVersions.LITHIUM_VERSION) {
+                return new PreLithiumTransactionContextImpl(transactionPath, transactionActor, identifier,
+                        transactionChainId, actorContext, schemaContext, isTxActorLocal, remoteTransactionVersion,
+                        operationCompleter);
+            } else if (transactionType == TransactionType.WRITE_ONLY &&
+                    actorContext.getDatastoreContext().isWriteOnlyTransactionOptimizationsEnabled()) {
+                return new WriteOnlyTransactionContextImpl(transactionActor, identifier, transactionChainId,
+                    actorContext, schemaContext, isTxActorLocal, remoteTransactionVersion, operationCompleter);
+            } else {
+                return new TransactionContextImpl(transactionActor, identifier, transactionChainId,
+                        actorContext, schemaContext, isTxActorLocal, remoteTransactionVersion, operationCompleter);
+            }
         }
+    }
 
-        @Override
-        public String getResolvedCohortPath(String cohortPath) {
-            return cohort.path().toString();
-        }
+    private static class NoOpDOMStoreThreePhaseCommitCohort implements DOMStoreThreePhaseCommitCohort {
+        static NoOpDOMStoreThreePhaseCommitCohort INSTANCE = new NoOpDOMStoreThreePhaseCommitCohort();
 
-        @Override
-        public void closeTransaction() {
-            LOG.warn("txn {} closeTransaction called", identifier);
-        }
+        private static final ListenableFuture<Void> IMMEDIATE_VOID_SUCCESS =
+                com.google.common.util.concurrent.Futures.immediateFuture(null);
+        private static final ListenableFuture<Boolean> IMMEDIATE_BOOLEAN_SUCCESS =
+                com.google.common.util.concurrent.Futures.immediateFuture(Boolean.TRUE);
 
-        @Override public Object readyTransaction() {
-            LOG.warn("txn {} readyTransaction called", identifier);
-            cohort = actorContext.getActorSystem().actorOf(Props.create(NoOpCohort.class));
-            return new ReadyTransactionReply(cohort.path()).toSerializable();
+        private NoOpDOMStoreThreePhaseCommitCohort() {
         }
 
         @Override
-        public void deleteData(YangInstanceIdentifier path) {
-            LOG.warn("txt {} deleteData called path = {}", identifier, path);
+        public ListenableFuture<Boolean> canCommit() {
+            return IMMEDIATE_BOOLEAN_SUCCESS;
         }
 
         @Override
-        public void mergeData(YangInstanceIdentifier path,
-            NormalizedNode<?, ?> data) {
-            LOG.warn("txn {} mergeData called path = {}", identifier, path);
+        public ListenableFuture<Void> preCommit() {
+            return IMMEDIATE_VOID_SUCCESS;
         }
 
         @Override
-        public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
-            YangInstanceIdentifier path) {
-            LOG.warn("txn {} readData called path = {}", identifier, path);
-            return Futures.immediateFailedCheckedFuture(new ReadFailedException(
-                    "Error reading data for path " + path, failure));
-        }
-
-        @Override public void writeData(YangInstanceIdentifier path,
-            NormalizedNode<?, ?> data) {
-            LOG.warn("txn {} writeData called path = {}", identifier, path);
+        public ListenableFuture<Void> abort() {
+            return IMMEDIATE_VOID_SUCCESS;
         }
 
-        @Override public CheckedFuture<Boolean, ReadFailedException> dataExists(
-            YangInstanceIdentifier path) {
-            LOG.warn("txn {} dataExists called path = {}", identifier, path);
-            return Futures.immediateFailedCheckedFuture(new ReadFailedException(
-                    "Error checking exists for path " + path, failure));
+        @Override
+        public ListenableFuture<Void> commit() {
+            return IMMEDIATE_VOID_SUCCESS;
         }
     }
-
-
-
 }