Fix String.format() strings
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / TransactionProxy.java
index d93bae22e08d9fddb3f1ac7d9c12aa4a278d0ff6..af1e4e92b60ef80effd9b3d443fd9743cef54900 100644 (file)
@@ -25,6 +25,7 @@ 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;
@@ -42,6 +43,8 @@ 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.SerializableMessage;
+import org.opendaylight.controller.cluster.datastore.messages.VersionedSerializableMessage;
 import org.opendaylight.controller.cluster.datastore.messages.WriteData;
 import org.opendaylight.controller.cluster.datastore.shardstrategy.ShardStrategyFactory;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
@@ -156,8 +159,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             if(remoteTransactionActorsMB.get()) {
                 for(ActorSelection actor : remoteTransactionActors) {
                     LOG.trace("Sending CloseTransaction to {}", actor);
-                    actorContext.sendOperationAsync(actor,
-                            new CloseTransaction().toSerializable());
+                    actorContext.sendOperationAsync(actor, CloseTransaction.INSTANCE.toSerializable());
                 }
             }
         }
@@ -184,6 +186,8 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
     private final String transactionChainId;
     private final SchemaContext schemaContext;
     private boolean inReadyState;
+    private final Semaphore operationLimiter;
+    private final OperationCompleter operationCompleter;
 
     public TransactionProxy(ActorContext actorContext, TransactionType transactionType) {
         this(actorContext, transactionType, "");
@@ -220,7 +224,11 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             phantomReferenceCache.put(cleanup, cleanup);
         }
 
-        LOG.debug("Created txn {} of type {}", identifier, transactionType);
+        // Note : Currently mailbox-capacity comes from akka.conf and not from the config-subsystem
+        this.operationLimiter = new Semaphore(actorContext.getTransactionOutstandingOperationLimit());
+        this.operationCompleter = new OperationCompleter(operationLimiter);
+
+        LOG.debug("Created txn {} of type {} on chain {}", identifier, transactionType, transactionChainId);
     }
 
     @VisibleForTesting
@@ -249,46 +257,23 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
     }
 
     @Override
-    public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> read(
-            final YangInstanceIdentifier path) {
+    public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> read(final YangInstanceIdentifier path) {
 
         Preconditions.checkState(transactionType != TransactionType.WRITE_ONLY,
                 "Read operation on write-only transaction is not allowed");
 
         LOG.debug("Tx {} read {}", identifier, path);
 
-        TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
-        TransactionContext transactionContext = txFutureCallback.getTransactionContext();
-
-        CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> future;
-        if(transactionContext != null) {
-            future = transactionContext.readData(path);
-        } else {
-            // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
-            // callback to be executed after the Tx is created.
-            final SettableFuture<Optional<NormalizedNode<?, ?>>> proxyFuture = SettableFuture.create();
-            txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
-                @Override
-                public void invoke(TransactionContext transactionContext) {
-                    Futures.addCallback(transactionContext.readData(path),
-                        new FutureCallback<Optional<NormalizedNode<?, ?>>>() {
-                            @Override
-                            public void onSuccess(Optional<NormalizedNode<?, ?>> data) {
-                                proxyFuture.set(data);
-                            }
-
-                            @Override
-                            public void onFailure(Throwable t) {
-                                proxyFuture.setException(t);
-                            }
-                        });
-                }
-            });
-
-            future = MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
-        }
+        throttleOperation();
 
-        return future;
+        TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
+        return txFutureCallback.enqueueReadOperation(new ReadOperation<Optional<NormalizedNode<?, ?>>>() {
+            @Override
+            public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> invoke(
+                    TransactionContext transactionContext) {
+                return transactionContext.readData(path);
+            }
+        });
     }
 
     @Override
@@ -299,40 +284,18 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         LOG.debug("Tx {} exists {}", identifier, path);
 
-        TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
-        TransactionContext transactionContext = txFutureCallback.getTransactionContext();
-
-        CheckedFuture<Boolean, ReadFailedException> future;
-        if(transactionContext != null) {
-            future = transactionContext.dataExists(path);
-        } else {
-            // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
-            // callback to be executed after the Tx is created.
-            final SettableFuture<Boolean> proxyFuture = SettableFuture.create();
-            txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
-                @Override
-                public void invoke(TransactionContext transactionContext) {
-                    Futures.addCallback(transactionContext.dataExists(path),
-                        new FutureCallback<Boolean>() {
-                            @Override
-                            public void onSuccess(Boolean exists) {
-                                proxyFuture.set(exists);
-                            }
+        throttleOperation();
 
-                            @Override
-                            public void onFailure(Throwable t) {
-                                proxyFuture.setException(t);
-                            }
-                        });
-                }
-            });
-
-            future = MappingCheckedFuture.create(proxyFuture, ReadFailedException.MAPPER);
-        }
-
-        return future;
+        TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
+        return txFutureCallback.enqueueReadOperation(new ReadOperation<Boolean>() {
+            @Override
+            public CheckedFuture<Boolean, ReadFailedException> invoke(TransactionContext transactionContext) {
+                return transactionContext.dataExists(path);
+            }
+        });
     }
 
+
     private void checkModificationState() {
         Preconditions.checkState(transactionType != TransactionType.READ_ONLY,
                 "Modification operation on read-only transaction is not allowed");
@@ -340,6 +303,25 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                 "Transaction is sealed - further modifications are not allowed");
     }
 
+    private void throttleOperation() {
+        throttleOperation(1);
+    }
+
+    private void throttleOperation(int acquirePermits) {
+        try {
+            if(!operationLimiter.tryAcquire(acquirePermits, actorContext.getDatastoreContext().getOperationTimeoutInSeconds(), TimeUnit.SECONDS)){
+                LOG.warn("Failed to acquire operation permit for transaction {}", getIdentifier());
+            }
+        } catch (InterruptedException e) {
+            if(LOG.isDebugEnabled()) {
+                LOG.debug("Interrupted when trying to acquire operation permit for transaction " + getIdentifier().toString(), e);
+            } else {
+                LOG.warn("Interrupted when trying to acquire operation permit for transaction {}", getIdentifier());
+            }
+        }
+    }
+
+
     @Override
     public void write(final YangInstanceIdentifier path, final NormalizedNode<?, ?> data) {
 
@@ -347,20 +329,15 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         LOG.debug("Tx {} write {}", identifier, path);
 
+        throttleOperation();
+
         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
-        TransactionContext transactionContext = txFutureCallback.getTransactionContext();
-        if(transactionContext != null) {
-            transactionContext.writeData(path, data);
-        } else {
-            // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
-            // callback to be executed after the Tx is created.
-            txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
-                @Override
-                public void invoke(TransactionContext transactionContext) {
-                    transactionContext.writeData(path, data);
-                }
-            });
-        }
+        txFutureCallback.enqueueModifyOperation(new TransactionOperation() {
+            @Override
+            public void invoke(TransactionContext transactionContext) {
+                transactionContext.writeData(path, data);
+            }
+        });
     }
 
     @Override
@@ -370,20 +347,15 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         LOG.debug("Tx {} merge {}", identifier, path);
 
+        throttleOperation();
+
         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
-        TransactionContext transactionContext = txFutureCallback.getTransactionContext();
-        if(transactionContext != null) {
-            transactionContext.mergeData(path, data);
-        } else {
-            // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
-            // callback to be executed after the Tx is created.
-            txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
-                @Override
-                public void invoke(TransactionContext transactionContext) {
-                    transactionContext.mergeData(path, data);
-                }
-            });
-        }
+        txFutureCallback.enqueueModifyOperation(new TransactionOperation() {
+            @Override
+            public void invoke(TransactionContext transactionContext) {
+                transactionContext.mergeData(path, data);
+            }
+        });
     }
 
     @Override
@@ -393,20 +365,15 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         LOG.debug("Tx {} delete {}", identifier, path);
 
+        throttleOperation();
+
         TransactionFutureCallback txFutureCallback = getOrCreateTxFutureCallback(path);
-        TransactionContext transactionContext = txFutureCallback.getTransactionContext();
-        if(transactionContext != null) {
-            transactionContext.deleteData(path);
-        } else {
-            // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
-            // callback to be executed after the Tx is created.
-            txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
-                @Override
-                public void invoke(TransactionContext transactionContext) {
-                    transactionContext.deleteData(path);
-                }
-            });
-        }
+        txFutureCallback.enqueueModifyOperation(new TransactionOperation() {
+            @Override
+            public void invoke(TransactionContext transactionContext) {
+                transactionContext.deleteData(path);
+            }
+        });
     }
 
     @Override
@@ -414,6 +381,8 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         checkModificationState();
 
+        throttleOperation(txFutureCallbackMap.size());
+
         inReadyState = true;
 
         LOG.debug("Tx {} Readying {} transactions for commit", identifier,
@@ -423,25 +392,17 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
 
-            LOG.debug("Tx {} Readying transaction for shard {}", identifier,
-                        txFutureCallback.getShardName());
+            LOG.debug("Tx {} Readying transaction for shard {} chain {}", identifier,
+                        txFutureCallback.getShardName(), transactionChainId);
 
-            TransactionContext transactionContext = txFutureCallback.getTransactionContext();
-            if(transactionContext != null) {
-                cohortFutures.add(transactionContext.readyTransaction());
-            } 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());
-                    }
-                });
+            Future<ActorSelection> future = txFutureCallback.enqueueFutureOperation(new FutureOperation<ActorSelection>() {
+                @Override
+                public Future<ActorSelection> invoke(TransactionContext transactionContext) {
+                    return transactionContext.readyTransaction();
+                }
+            });
 
-                cohortFutures.add(cohortPromise.future());
-            }
+            cohortFutures.add(future);
         }
 
         onTransactionReady(cohortFutures);
@@ -477,18 +438,13 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
     @Override
     public void close() {
-        for(TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
-            TransactionContext transactionContext = txFutureCallback.getTransactionContext();
-            if(transactionContext != null) {
-                transactionContext.closeTransaction();
-            } else {
-                txFutureCallback.addTxOperationOnComplete(new TransactionOperation() {
-                    @Override
-                    public void invoke(TransactionContext transactionContext) {
-                        transactionContext.closeTransaction();
-                    }
-                });
-            }
+        for (TransactionFutureCallback txFutureCallback : txFutureCallbackMap.values()) {
+            txFutureCallback.enqueueModifyOperation(new TransactionOperation() {
+                @Override
+                public void invoke(TransactionContext transactionContext) {
+                    transactionContext.closeTransaction();
+                }
+            });
         }
 
         txFutureCallbackMap.clear();
@@ -534,13 +490,31 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         return transactionChainId;
     }
 
+    protected ActorContext getActorContext() {
+        return actorContext;
+    }
+
     /**
-     * Interface for a transaction operation to be invoked later.
+     * Interfaces for transaction operations to be invoked later.
      */
     private static interface TransactionOperation {
         void invoke(TransactionContext transactionContext);
     }
 
+    /**
+     * 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
@@ -613,6 +587,74 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             }
         }
 
+
+        <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();
+            }
+
+            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) {
+
+            if (transactionContext != null) {
+                op.invoke(transactionContext);
+            } else {
+                // The shard Tx hasn't been created yet so add the Tx operation to the Tx Future
+                // callback to be executed after the Tx is created.
+                addTxOperationOnComplete(op);
+            }
+        }
+
         /**
          * Performs a CreateTransaction try async.
          */
@@ -664,7 +706,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                     LOG.debug("Tx {} Creating NoOpTransaction because of error: {}", identifier,
                             failure.getMessage());
 
-                    localTransactionContext = new NoOpTransactionContext(failure, identifier);
+                    localTransactionContext = new NoOpTransactionContext(failure, identifier, operationLimiter);
                 } else if (response.getClass().equals(CreateTransactionReply.SERIALIZABLE_CLASS)) {
                     localTransactionContext = createValidTransactionContext(
                             CreateTransactionReply.fromSerializable(response));
@@ -672,7 +714,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                     IllegalArgumentException exception = new IllegalArgumentException(String.format(
                         "Invalid reply type %s for CreateTransaction", response.getClass()));
 
-                    localTransactionContext = new NoOpTransactionContext(exception, identifier);
+                    localTransactionContext = new NoOpTransactionContext(exception, identifier, operationLimiter);
                 }
 
                 for(TransactionOperation oper: txOperationsOnComplete) {
@@ -706,10 +748,10 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
             // TxActor is always created where the leader of the shard is.
             // Check if TxActor is created in the same node
-            boolean isTxActorLocal = actorContext.isLocalPath(transactionPath);
+            boolean isTxActorLocal = actorContext.isPathLocal(transactionPath);
 
             return new TransactionContextImpl(transactionPath, transactionActor, identifier,
-                actorContext, schemaContext, isTxActorLocal, reply.getVersion());
+                actorContext, schemaContext, isTxActorLocal, reply.getVersion(), operationCompleter);
         }
     }
 
@@ -751,33 +793,49 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         private final Logger LOG = LoggerFactory.getLogger(TransactionContextImpl.class);
 
         private final ActorContext actorContext;
-        private final SchemaContext schemaContext;
         private final String transactionPath;
         private final ActorSelection actor;
         private final boolean isTxActorLocal;
-        private final int remoteTransactionVersion;
+        private final short remoteTransactionVersion;
+        private final OperationCompleter operationCompleter;
+
 
         private TransactionContextImpl(String transactionPath, ActorSelection actor, TransactionIdentifier identifier,
                 ActorContext actorContext, SchemaContext schemaContext,
-                boolean isTxActorLocal, int remoteTransactionVersion) {
+                boolean isTxActorLocal, short remoteTransactionVersion, OperationCompleter operationCompleter) {
             super(identifier);
             this.transactionPath = transactionPath;
             this.actor = actor;
             this.actorContext = actorContext;
-            this.schemaContext = schemaContext;
             this.isTxActorLocal = isTxActorLocal;
             this.remoteTransactionVersion = remoteTransactionVersion;
+            this.operationCompleter = operationCompleter;
+        }
+
+        private Future<Object> completeOperation(Future<Object> operationFuture){
+            operationFuture.onComplete(this.operationCompleter, actorContext.getActorSystem().dispatcher());
+            return operationFuture;
         }
 
+
         private ActorSelection getActor() {
             return actor;
         }
 
+        private Future<Object> executeOperationAsync(SerializableMessage msg) {
+            return completeOperation(actorContext.executeOperationAsync(getActor(), isTxActorLocal ? msg : msg.toSerializable()));
+        }
+
+        private Future<Object> executeOperationAsync(VersionedSerializableMessage msg) {
+            return completeOperation(actorContext.executeOperationAsync(getActor(), isTxActorLocal ? msg :
+                    msg.toSerializable(remoteTransactionVersion)));
+        }
+
         @Override
         public void closeTransaction() {
             LOG.debug("Tx {} closeTransaction called", identifier);
 
-            actorContext.sendOperationAsync(getActor(), new CloseTransaction().toSerializable());
+            actorContext.sendOperationAsync(getActor(), CloseTransaction.INSTANCE.toSerializable());
         }
 
         @Override
@@ -787,9 +845,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
             // Send the ReadyTransaction message to the Tx actor.
 
-            ReadyTransaction readyTransaction = new ReadyTransaction();
-            final Future<Object> replyFuture = actorContext.executeOperationAsync(getActor(),
-                isTxActorLocal ? readyTransaction : readyTransaction.toSerializable());
+            final Future<Object> replyFuture = executeOperationAsync(ReadyTransaction.INSTANCE);
 
             // Combine all the previously recorded put/merge/delete operation reply Futures and the
             // ReadyTransactionReply Future into one Future. If any one fails then the combined
@@ -836,7 +892,8 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                         // At some point in the future when upgrades from Helium are not supported
                         // we could remove this code to resolvePath and just use the cohortPath as the
                         // resolved cohortPath
-                        if(TransactionContextImpl.this.remoteTransactionVersion < CreateTransaction.HELIUM_1_VERSION) {
+                        if(TransactionContextImpl.this.remoteTransactionVersion <
+                                DataStoreVersions.HELIUM_1_VERSION) {
                             cohortPath = actorContext.resolvePath(transactionPath, cohortPath);
                         }
 
@@ -844,7 +901,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
                     } else {
                         // Throwing an exception here will fail the Future.
-                        throw new IllegalArgumentException(String.format("Invalid reply type {}",
+                        throw new IllegalArgumentException(String.format("Invalid reply type %s",
                                 serializedReadyReply.getClass()));
                     }
                 }
@@ -855,27 +912,21 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         public void deleteData(YangInstanceIdentifier path) {
             LOG.debug("Tx {} deleteData called path = {}", identifier, path);
 
-            DeleteData deleteData = new DeleteData(path);
-            recordedOperationFutures.add(actorContext.executeOperationAsync(getActor(),
-                isTxActorLocal ? deleteData : deleteData.toSerializable()));
+            recordedOperationFutures.add(executeOperationAsync(new DeleteData(path)));
         }
 
         @Override
         public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
             LOG.debug("Tx {} mergeData called path = {}", identifier, path);
 
-            MergeData mergeData = new MergeData(path, data, schemaContext);
-            recordedOperationFutures.add(actorContext.executeOperationAsync(getActor(),
-                isTxActorLocal ? mergeData : mergeData.toSerializable()));
+            recordedOperationFutures.add(executeOperationAsync(new MergeData(path, data)));
         }
 
         @Override
         public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
             LOG.debug("Tx {} writeData called path = {}", identifier, path);
 
-            WriteData writeData = new WriteData(path, data, schemaContext);
-            recordedOperationFutures.add(actorContext.executeOperationAsync(getActor(),
-                isTxActorLocal ? writeData : writeData.toSerializable()));
+            recordedOperationFutures.add(executeOperationAsync(new WriteData(path, data)));
         }
 
         @Override
@@ -946,8 +997,8 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                             ReadDataReply reply = (ReadDataReply) readResponse;
                             returnFuture.set(Optional.<NormalizedNode<?, ?>>fromNullable(reply.getNormalizedNode()));
 
-                        } else if (readResponse.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)) {
-                            ReadDataReply reply = ReadDataReply.fromSerializable(schemaContext, path, readResponse);
+                        } else if (ReadDataReply.isSerializedType(readResponse)) {
+                            ReadDataReply reply = ReadDataReply.fromSerializable(readResponse);
                             returnFuture.set(Optional.<NormalizedNode<?, ?>>fromNullable(reply.getNormalizedNode()));
 
                         } else {
@@ -958,9 +1009,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                 }
             };
 
-            ReadData readData = new ReadData(path);
-            Future<Object> readFuture = actorContext.executeOperationAsync(getActor(),
-                isTxActorLocal ? readData : readData.toSerializable());
+            Future<Object> readFuture = executeOperationAsync(new ReadData(path));
 
             readFuture.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
         }
@@ -1042,9 +1091,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                 }
             };
 
-            DataExists dataExists = new DataExists(path);
-            Future<Object> future = actorContext.executeOperationAsync(getActor(),
-                isTxActorLocal ? dataExists : dataExists.toSerializable());
+            Future<Object> future = executeOperationAsync(new DataExists(path));
 
             future.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
         }
@@ -1055,10 +1102,12 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         private final Logger LOG = LoggerFactory.getLogger(NoOpTransactionContext.class);
 
         private final Throwable failure;
+        private final Semaphore operationLimiter;
 
-        public NoOpTransactionContext(Throwable failure, TransactionIdentifier identifier){
+        public NoOpTransactionContext(Throwable failure, TransactionIdentifier identifier, Semaphore operationLimiter){
             super(identifier);
             this.failure = failure;
+            this.operationLimiter = operationLimiter;
         }
 
         @Override
@@ -1069,28 +1118,33 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         @Override
         public Future<ActorSelection> readyTransaction() {
             LOG.debug("Tx {} readyTransaction called", identifier);
+            operationLimiter.release();
             return akka.dispatch.Futures.failed(failure);
         }
 
         @Override
         public void deleteData(YangInstanceIdentifier path) {
             LOG.debug("Tx {} deleteData called path = {}", identifier, path);
+            operationLimiter.release();
         }
 
         @Override
         public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
             LOG.debug("Tx {} mergeData called path = {}", identifier, path);
+            operationLimiter.release();
         }
 
         @Override
         public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
             LOG.debug("Tx {} writeData called path = {}", identifier, path);
+            operationLimiter.release();
         }
 
         @Override
         public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
                 YangInstanceIdentifier path) {
             LOG.debug("Tx {} readData called path = {}", identifier, path);
+            operationLimiter.release();
             return Futures.immediateFailedCheckedFuture(new ReadFailedException(
                     "Error reading data for path " + path, failure));
         }
@@ -1099,8 +1153,21 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         public CheckedFuture<Boolean, ReadFailedException> dataExists(
                 YangInstanceIdentifier path) {
             LOG.debug("Tx {} dataExists called path = {}", identifier, path);
+            operationLimiter.release();
             return Futures.immediateFailedCheckedFuture(new ReadFailedException(
                     "Error checking exists for path " + path, failure));
         }
     }
+
+    private static class OperationCompleter extends OnComplete<Object> {
+        private final Semaphore operationLimiter;
+        OperationCompleter(Semaphore operationLimiter){
+            this.operationLimiter = operationLimiter;
+        }
+
+        @Override
+        public void onComplete(Throwable throwable, Object o){
+            this.operationLimiter.release();
+        }
+    }
 }