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 f28a1e5f73d4823fe31ba20d14fa4d986f9700da..f1ba4eabb999a6455688d93532877ae490df88df 100644 (file)
@@ -18,9 +18,11 @@ 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.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;
@@ -30,6 +32,7 @@ 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;
@@ -67,7 +70,19 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
     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 =
@@ -162,7 +177,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
      * PhantomReference.
      */
     private List<ActorSelection> remoteTransactionActors;
-    private AtomicBoolean remoteTransactionActorsMB;
+    private volatile AtomicBoolean remoteTransactionActorsMB;
 
     /**
      * Stores the create transaction results per shard.
@@ -175,8 +190,10 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
     private final String transactionChainId;
     private final SchemaContext schemaContext;
     private boolean inReadyState;
-    private final Semaphore operationLimiter;
-    private final OperationCompleter operationCompleter;
+
+    private volatile boolean initialized;
+    private Semaphore operationLimiter;
+    private OperationCompleter operationCompleter;
 
     public TransactionProxy(ActorContext actorContext, TransactionType transactionType) {
         this(actorContext, transactionType, "");
@@ -197,25 +214,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             memberName = "UNKNOWN-MEMBER";
         }
 
-        this.identifier = TransactionIdentifier.builder().memberName(memberName).counter(
-            counter.getAndIncrement()).build();
-
-        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.
-
-            remoteTransactionActors = Lists.newArrayList();
-            remoteTransactionActorsMB = new AtomicBoolean();
-
-            TransactionProxyCleanupPhantomReference cleanup =
-                new TransactionProxyCleanupPhantomReference(this);
-            phantomReferenceCache.put(cleanup, cleanup);
-        }
-
-        // Note : Currently mailbox-capacity comes from akka.conf and not from the config-subsystem
-        this.operationLimiter = new Semaphore(actorContext.getTransactionOutstandingOperationLimit());
-        this.operationCompleter = new OperationCompleter(operationLimiter);
+        this.identifier = new TransactionIdentifier(memberName, counter.getAndIncrement());
 
         LOG.debug("Created txn {} of type {} on chain {}", identifier, transactionType, transactionChainId);
     }
@@ -255,14 +254,17 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         throttleOperation();
 
+        final SettableFuture<Optional<NormalizedNode<?, ?>>> proxyFuture = SettableFuture.create();
+
         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
-        return txFutureCallback.enqueueReadOperation(new ReadOperation<Optional<NormalizedNode<?, ?>>>() {
+        txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
             @Override
-            public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> invoke(
-                    TransactionContext transactionContext) {
-                return transactionContext.readData(path);
+            public void invoke(TransactionContext transactionContext) {
+                transactionContext.readData(path, proxyFuture);
             }
         });
+
+        return MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
     }
 
     @Override
@@ -275,15 +277,18 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         throttleOperation();
 
+        final SettableFuture<Boolean> proxyFuture = SettableFuture.create();
+
         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
-        return txFutureCallback.enqueueReadOperation(new ReadOperation<Boolean>() {
+        txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
             @Override
-            public CheckedFuture<Boolean, ReadFailedException> invoke(TransactionContext transactionContext) {
-                return transactionContext.dataExists(path);
+            public void invoke(TransactionContext transactionContext) {
+                transactionContext.dataExists(path, proxyFuture);
             }
         });
-    }
 
+        return MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
+    }
 
     private void checkModificationState() {
         Preconditions.checkState(transactionType != TransactionType.READ_ONLY,
@@ -297,8 +302,19 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
     }
 
     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)){
+            if(!operationLimiter.tryAcquire(acquirePermits,
+                    actorContext.getDatastoreContext().getOperationTimeoutInSeconds(), TimeUnit.SECONDS)){
                 LOG.warn("Failed to acquire operation permit for transaction {}", getIdentifier());
             }
         } catch (InterruptedException e) {
@@ -321,7 +337,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         throttleOperation();
 
         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
-        txFutureCallback.enqueueModifyOperation(new TransactionOperation() {
+        txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
             @Override
             public void invoke(TransactionContext transactionContext) {
                 transactionContext.writeData(path, data);
@@ -339,7 +355,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         throttleOperation();
 
         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
-        txFutureCallback.enqueueModifyOperation(new TransactionOperation() {
+        txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
             @Override
             public void invoke(TransactionContext transactionContext) {
                 transactionContext.mergeData(path, data);
@@ -357,7 +373,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         throttleOperation();
 
         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
-        txFutureCallback.enqueueModifyOperation(new TransactionOperation() {
+        txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
             @Override
             public void invoke(TransactionContext transactionContext) {
                 transactionContext.deleteData(path);
@@ -370,13 +386,19 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         checkModificationState();
 
-        throttleOperation(txFutureCallbackMap.size());
-
         inReadyState = true;
 
         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;
+        }
+
+        throttleOperation(txFutureCallbackMap.size());
+
         List<Future<ActorSelection>> cohortFutures = Lists.newArrayList();
 
         for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
@@ -384,12 +406,21 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             LOG.debug("Tx {} Readying transaction for shard {} chain {}", identifier,
                         txFutureCallback.getShardName(), transactionChainId);
 
-            Future<ActorSelection> future = txFutureCallback.enqueueFutureOperation(new FutureOperation<ActorSelection>() {
-                @Override
-                public Future<ActorSelection> invoke(TransactionContext transactionContext) {
-                    return transactionContext.readyTransaction();
-                }
-            });
+            final TransactionContext transactionContext = txFutureCallback.getTransactionContext();
+            final Future<ActorSelection> future;
+            if (transactionContext != null) {
+                // avoid the creation of a promise and a TransactionOperation
+                future = transactionContext.readyTransaction();
+            } else {
+                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);
         }
@@ -408,18 +439,6 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
     protected void onTransactionReady(List<Future<ActorSelection>> cohortFutures) {
     }
 
-    /**
-     * Method called to send a CreateTransaction message to a shard.
-     *
-     * @param shard the shard actor to send to
-     * @param serializedCreateMessage the serialized message to send
-     * @return the response Future
-     */
-    protected Future<Object> sendCreateTransaction(ActorSelection shard,
-            Object serializedCreateMessage) {
-        return actorContext.executeOperationAsync(shard, serializedCreateMessage);
-    }
-
     @Override
     public Object getIdentifier() {
         return this.identifier;
@@ -428,7 +447,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
     @Override
     public void close() {
         for (TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
-            txFutureCallback.enqueueModifyOperation(new TransactionOperation() {
+            txFutureCallback.enqueueTransactionOperation(new TransactionOperation() {
                 @Override
                 public void invoke(TransactionContext transactionContext) {
                     transactionContext.closeTransaction();
@@ -438,7 +457,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         txFutureCallbackMap.clear();
 
-        if(transactionType == TransactionType.READ_ONLY) {
+        if(remoteTransactionActorsMB != null) {
             remoteTransactionActors.clear();
             remoteTransactionActorsMB.set(true);
         }
@@ -448,14 +467,17 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         return ShardStrategyFactory.getStrategy(path).findShard(path);
     }
 
+    protected Future<ActorSelection> sendFindPrimaryShardAsync(String shardName) {
+        return actorContext.findPrimaryShardAsync(shardName);
+    }
+
     private TransactionFutureCallback getOrCreateTxFutureCallback(YangInstanceIdentifier path) {
         String shardName = shardNameFromIdentifier(path);
         TransactionFutureCallback txFutureCallback = txFutureCallbackMap.get(shardName);
         if(txFutureCallback == null) {
-            Future<ActorSelection> findPrimaryFuture = actorContext.findPrimaryShardAsync(shardName);
+            Future<ActorSelection> findPrimaryFuture = sendFindPrimaryShardAsync(shardName);
 
-            final TransactionFutureCallback newTxFutureCallback =
-                    new TransactionFutureCallback(shardName);
+            final TransactionFutureCallback newTxFutureCallback = new TransactionFutureCallback(shardName);
 
             txFutureCallback = newTxFutureCallback;
             txFutureCallbackMap.put(shardName, txFutureCallback);
@@ -464,12 +486,12 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                 @Override
                 public void onComplete(Throwable failure, ActorSelection primaryShard) {
                     if(failure != null) {
-                        newTxFutureCallback.onComplete(failure, null);
+                        newTxFutureCallback.createTransactionContext(failure, null);
                     } else {
                         newTxFutureCallback.setPrimaryShard(primaryShard);
                     }
                 }
-            }, actorContext.getActorSystem().dispatcher());
+            }, actorContext.getClientDispatcher());
         }
 
         return txFutureCallback;
@@ -490,20 +512,6 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         void invoke(TransactionContext transactionContext);
     }
 
-    /**
-     * This interface returns a Guava Future
-     */
-    private static interface ReadOperation<T> {
-        CheckedFuture<T, ReadFailedException> invoke(TransactionContext transactionContext);
-    }
-
-    /**
-     * This interface returns a Scala Future
-     */
-    private static interface FutureOperation<T> {
-        Future<T> 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
@@ -555,85 +563,43 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
          * 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();
+
+            if(transactionType == TransactionType.WRITE_ONLY &&
+                    actorContext.getDatastoreContext().isWriteOnlyTransactionOptimizationsEnabled()) {
+                LOG.debug("Tx {} Primary shard {} found - creating WRITE_ONLY transaction context",
+                        identifier, primaryShard);
+
+                // 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();
+            }
         }
 
         /**
          * 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);
+                    LOG.debug("Tx {} Adding operation on complete", identifier);
 
+                    invokeOperation = false;
                     txOperationsOnComplete.add(operation);
-                } else {
-                    operation.invoke(transactionContext);
                 }
             }
-        }
 
-
-        <T> Future<T> enqueueFutureOperation(final FutureOperation<T> op) {
-
-            Future<T> future;
-
-            if (transactionContext != null) {
-                future = 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.
-                final Promise<T> promise = akka.dispatch.Futures.promise();
-                addTxOperationOnComplete(new TransactionOperation() {
-                    @Override
-                    public void invoke(TransactionContext transactionContext) {
-                        promise.completeWith(op.invoke(transactionContext));
-                    }
-                });
-
-                future = promise.future();
+            if(invokeOperation) {
+                operation.invoke(transactionContext);
             }
-
-            return future;
         }
 
-        <T> CheckedFuture<T, ReadFailedException> enqueueReadOperation(final ReadOperation<T> op) {
-
-            CheckedFuture<T, ReadFailedException> future;
-
-            if (transactionContext != null) {
-                future = 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.
-                final SettableFuture<T> proxyFuture = SettableFuture.create();
-                addTxOperationOnComplete(new TransactionOperation() {
-                    @Override
-                    public void invoke(TransactionContext transactionContext) {
-                        Futures.addCallback(op.invoke(transactionContext), new FutureCallback<T>() {
-                            @Override
-                            public void onSuccess(T data) {
-                                proxyFuture.set(data);
-                            }
-
-                            @Override
-                            public void onFailure(Throwable t) {
-                                proxyFuture.setException(t);
-                            }
-                        });
-                    }
-                });
-
-                future = MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
-            }
-
-            return future;
-        }
-
-        void enqueueModifyOperation(final TransactionOperation op) {
+        void enqueueTransactionOperation(final TransactionOperation op) {
 
             if (transactionContext != null) {
                 op.invoke(transactionContext);
@@ -648,12 +614,17 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
          * Performs a CreateTransaction try async.
          */
         private void tryCreateTransaction() {
-            Future<Object> createTxFuture = sendCreateTransaction(primaryShard,
-                    new CreateTransaction(identifier.toString(),
-                            TransactionProxy.this.transactionType.ordinal(),
-                            getTransactionChainId()).toSerializable());
+            if(LOG.isDebugEnabled()) {
+                LOG.debug("Tx {} Primary shard {} found - trying create transaction", identifier, primaryShard);
+            }
 
-            createTxFuture.onComplete(this, actorContext.getActorSystem().dispatcher());
+            Object serializedCreateMessage = new CreateTransaction(identifier.toString(),
+                    TransactionProxy.this.transactionType.ordinal(),
+                    getTransactionChainId()).toSerializable();
+
+            Future<Object> createTxFuture = actorContext.executeOperationAsync(primaryShard, serializedCreateMessage);
+
+            createTxFuture.onComplete(this, actorContext.getClientDispatcher());
         }
 
         @Override
@@ -673,59 +644,103 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                                 public void run() {
                                     tryCreateTransaction();
                                 }
-                            }, actorContext.getActorSystem().dispatcher());
+                            }, actorContext.getClientDispatcher());
                     return;
                 }
             }
 
-            // 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) {
-                // Store the new TransactionContext locally until we've completed invoking the
-                // TransactionOperations. This avoids thread timing issues which could cause
-                // out-of-order TransactionOperations. Eg, on a modification operation, if the
-                // TransactionContext is non-null, then we directly call the TransactionContext.
-                // However, at the same time, the code may be executing the cached
-                // TransactionOperations. So to avoid thus timing, we don't publish the
-                // TransactionContext until after we've executed all cached TransactionOperations.
-                TransactionContext localTransactionContext;
-                if(failure != null) {
-                    LOG.debug("Tx {} Creating NoOpTransaction because of error: {}", identifier,
-                            failure.getMessage());
-
-                    localTransactionContext = new NoOpTransactionContext(failure, identifier, operationLimiter);
-                } else if (response.getClass().equals(CreateTransactionReply.SERIALIZABLE_CLASS)) {
-                    localTransactionContext = createValidTransactionContext(
-                            CreateTransactionReply.fromSerializable(response));
-                } else {
-                    IllegalArgumentException exception = new IllegalArgumentException(String.format(
+            createTransactionContext(failure, response);
+        }
+
+        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()));
 
-                    localTransactionContext = new NoOpTransactionContext(exception, identifier, operationLimiter);
+                localTransactionContext = new NoOpTransactionContext(exception, identifier, operationLimiter);
+            }
+
+            executeTxOperatonsOnComplete(localTransactionContext);
+        }
+
+        private void executeTxOperatonsOnComplete(TransactionContext localTransactionContext) {
+            while(true) {
+                // Access to txOperationsOnComplete and transactionContext must be protected and atomic
+                // (ie synchronized) with respect to #addTxOperationOnComplete to handle timing
+                // issues and ensure no TransactionOperation is missed and that they are processed
+                // in the order they occurred.
+
+                // We'll make a local copy of the txOperationsOnComplete list to handle re-entrancy
+                // in case a TransactionOperation results in another transaction operation being
+                // queued (eg a put operation from a client read Future callback that is notified
+                // synchronously).
+                Collection<TransactionOperation> operationsBatch = null;
+                synchronized(txOperationsOnComplete) {
+                    if(txOperationsOnComplete.isEmpty()) {
+                        // We're done invoking the TransactionOperations so we can now publish the
+                        // TransactionContext.
+                        transactionContext = localTransactionContext;
+                        break;
+                    }
+
+                    operationsBatch = new ArrayList<>(txOperationsOnComplete);
+                    txOperationsOnComplete.clear();
                 }
 
-                for(TransactionOperation oper: txOperationsOnComplete) {
+                // 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);
                 }
-
-                txOperationsOnComplete.clear();
-
-                // We're done invoking the TransactionOperations so we can now publish the
-                // TransactionContext.
-                transactionContext = localTransactionContext;
             }
         }
 
         private TransactionContext createValidTransactionContext(CreateTransactionReply reply) {
-            String transactionPath = reply.getTransactionPath();
+            LOG.debug("Tx {} Received {}", identifier, reply);
 
-            LOG.debug("Tx {} Received transaction actor path {}", identifier, transactionPath);
+            return createValidTransactionContext(actorContext.actorSelection(reply.getTransactionPath()),
+                    reply.getTransactionPath(), reply.getVersion());
+        }
 
-            ActorSelection transactionActor = actorContext.actorSelection(transactionPath);
+        private TransactionContext createValidTransactionContext(ActorSelection transactionActor,
+                String transactionPath, short remoteTransactionVersion) {
 
             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.
+
+                if(remoteTransactionActorsMB == null) {
+                    remoteTransactionActors = Lists.newArrayList();
+                    remoteTransactionActorsMB = new AtomicBoolean();
+
+                    TransactionProxyCleanupPhantomReference cleanup =
+                            new TransactionProxyCleanupPhantomReference(TransactionProxy.this);
+                    phantomReferenceCache.put(cleanup, cleanup);
+                }
+
                 // Add the actor to the remoteTransactionActors list for access by the
                 // cleanup PhantonReference.
                 remoteTransactionActors.add(transactionActor);
@@ -739,8 +754,50 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             // Check if TxActor is created in the same node
             boolean isTxActorLocal = actorContext.isPathLocal(transactionPath);
 
-            return new TransactionContextImpl(transactionPath, transactionActor, identifier,
-                actorContext, schemaContext, isTxActorLocal, reply.getVersion(), operationCompleter);
+            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);
+            }
+        }
+    }
+
+    private static class NoOpDOMStoreThreePhaseCommitCohort implements DOMStoreThreePhaseCommitCohort {
+        static NoOpDOMStoreThreePhaseCommitCohort INSTANCE = new NoOpDOMStoreThreePhaseCommitCohort();
+
+        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);
+
+        private NoOpDOMStoreThreePhaseCommitCohort() {
+        }
+
+        @Override
+        public ListenableFuture<Boolean> canCommit() {
+            return IMMEDIATE_BOOLEAN_SUCCESS;
+        }
+
+        @Override
+        public ListenableFuture<Void> preCommit() {
+            return IMMEDIATE_VOID_SUCCESS;
+        }
+
+        @Override
+        public ListenableFuture<Void> abort() {
+            return IMMEDIATE_VOID_SUCCESS;
+        }
+
+        @Override
+        public ListenableFuture<Void> commit() {
+            return IMMEDIATE_VOID_SUCCESS;
         }
     }
 }