Speed up enumeration lookups
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / TransactionContextImpl.java
index 530a36cff657304005ebd4b43a5bb1e46449951b..3a209630c3344ca149032c2cc1d4f06b134ccf42 100644 (file)
@@ -15,18 +15,20 @@ import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.SettableFuture;
 import java.util.List;
 import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
+import org.opendaylight.controller.cluster.datastore.messages.BatchedModificationsReply;
 import org.opendaylight.controller.cluster.datastore.messages.CloseTransaction;
 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.SerializableMessage;
-import org.opendaylight.controller.cluster.datastore.messages.VersionedSerializableMessage;
-import org.opendaylight.controller.cluster.datastore.messages.WriteData;
+import org.opendaylight.controller.cluster.datastore.modification.DeleteModification;
+import org.opendaylight.controller.cluster.datastore.modification.MergeModification;
+import org.opendaylight.controller.cluster.datastore.modification.Modification;
+import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
@@ -36,23 +38,24 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.concurrent.Future;
 
-final class TransactionContextImpl extends AbstractTransactionContext {
+public class TransactionContextImpl extends AbstractTransactionContext {
     private static final Logger LOG = LoggerFactory.getLogger(TransactionContextImpl.class);
 
+    private final String transactionChainId;
     private final ActorContext actorContext;
-    private final String transactionPath;
     private final ActorSelection actor;
     private final boolean isTxActorLocal;
     private final short remoteTransactionVersion;
-    private final OperationCompleter operationCompleter;
 
+    private final OperationCompleter operationCompleter;
+    private BatchedModifications batchedModifications;
 
-    TransactionContextImpl(String transactionPath, ActorSelection actor, TransactionIdentifier identifier,
-            ActorContext actorContext, SchemaContext schemaContext,
-            boolean isTxActorLocal, short remoteTransactionVersion, OperationCompleter operationCompleter) {
+    protected TransactionContextImpl(ActorSelection actor, TransactionIdentifier identifier,
+            String transactionChainId, ActorContext actorContext, SchemaContext schemaContext, boolean isTxActorLocal,
+            short remoteTransactionVersion, OperationCompleter operationCompleter) {
         super(identifier);
-        this.transactionPath = transactionPath;
         this.actor = actor;
+        this.transactionChainId = transactionChainId;
         this.actorContext = actorContext;
         this.isTxActorLocal = isTxActorLocal;
         this.remoteTransactionVersion = remoteTransactionVersion;
@@ -60,7 +63,7 @@ final class TransactionContextImpl extends AbstractTransactionContext {
     }
 
     private Future<Object> completeOperation(Future<Object> operationFuture){
-        operationFuture.onComplete(this.operationCompleter, actorContext.getActorSystem().dispatcher());
+        operationFuture.onComplete(this.operationCompleter, actorContext.getClientDispatcher());
         return operationFuture;
     }
 
@@ -69,13 +72,16 @@ final class TransactionContextImpl extends AbstractTransactionContext {
         return actor;
     }
 
-    private Future<Object> executeOperationAsync(SerializableMessage msg) {
-        return completeOperation(actorContext.executeOperationAsync(getActor(), isTxActorLocal ? msg : msg.toSerializable()));
+    protected ActorContext getActorContext() {
+        return actorContext;
+    }
+
+    protected short getRemoteTransactionVersion() {
+        return remoteTransactionVersion;
     }
 
-    private Future<Object> executeOperationAsync(VersionedSerializableMessage msg) {
-        return completeOperation(actorContext.executeOperationAsync(getActor(), isTxActorLocal ? msg :
-                msg.toSerializable(remoteTransactionVersion)));
+    protected Future<Object> executeOperationAsync(SerializableMessage msg) {
+        return completeOperation(actorContext.executeOperationAsync(getActor(), isTxActorLocal ? msg : msg.toSerializable()));
     }
 
     @Override
@@ -90,22 +96,29 @@ final class TransactionContextImpl extends AbstractTransactionContext {
         LOG.debug("Tx {} readyTransaction called with {} previous recorded operations pending",
                 identifier, recordedOperationFutures.size());
 
+        // Send the remaining batched modifications if any.
+
+        sendAndRecordBatchedModifications();
+
         // Send the ReadyTransaction message to the Tx actor.
 
-        final Future<Object> replyFuture = executeOperationAsync(ReadyTransaction.INSTANCE);
+        Future<Object> readyReplyFuture = executeOperationAsync(ReadyTransaction.INSTANCE);
+
+        return combineRecordedOperationsFutures(readyReplyFuture);
+    }
 
+    protected Future<ActorSelection> combineRecordedOperationsFutures(final Future<Object> withLastReplyFuture) {
         // Combine all the previously recorded put/merge/delete operation reply Futures and the
         // ReadyTransactionReply Future into one Future. If any one fails then the combined
         // Future will fail. We need all prior operations and the ready operation to succeed
         // in order to attempt commit.
 
-        List<Future<Object>> futureList =
-                Lists.newArrayListWithCapacity(recordedOperationFutures.size() + 1);
+        List<Future<Object>> futureList = Lists.newArrayListWithCapacity(recordedOperationFutures.size() + 1);
         futureList.addAll(recordedOperationFutures);
-        futureList.add(replyFuture);
+        futureList.add(withLastReplyFuture);
 
         Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(futureList,
-                actorContext.getActorSystem().dispatcher());
+                actorContext.getClientDispatcher());
 
         // Transform the combined Future into a Future that returns the cohort actor path from
         // the ReadyTransactionReply. That's the end result of the ready operation.
@@ -124,56 +137,90 @@ final class TransactionContextImpl extends AbstractTransactionContext {
                 // de-serializing each reply.
 
                 // Note the Future get call here won't block as it's complete.
-                Object serializedReadyReply = replyFuture.value().get().get();
+                Object serializedReadyReply = withLastReplyFuture.value().get().get();
                 if (serializedReadyReply instanceof ReadyTransactionReply) {
                     return actorContext.actorSelection(((ReadyTransactionReply)serializedReadyReply).getCohortPath());
-
+                } else if(serializedReadyReply instanceof BatchedModificationsReply) {
+                    return actorContext.actorSelection(((BatchedModificationsReply)serializedReadyReply).getCohortPath());
                 } else if(serializedReadyReply.getClass().equals(ReadyTransactionReply.SERIALIZABLE_CLASS)) {
                     ReadyTransactionReply reply = ReadyTransactionReply.fromSerializable(serializedReadyReply);
-                    String cohortPath = reply.getCohortPath();
-
-                    // In Helium we used to return the local path of the actor which represented
-                    // a remote ThreePhaseCommitCohort. The local path would then be converted to
-                    // a remote path using this resolvePath method. To maintain compatibility with
-                    // a Helium node we need to continue to do this conversion.
-                    // At some point in the future when upgrades from Helium are not supported
-                    // we could remove this code to resolvePath and just use the cohortPath as the
-                    // resolved cohortPath
-                    if(TransactionContextImpl.this.remoteTransactionVersion <
-                            DataStoreVersions.HELIUM_1_VERSION) {
-                        cohortPath = actorContext.resolvePath(transactionPath, cohortPath);
-                    }
-
+                    String cohortPath = deserializeCohortPath(reply.getCohortPath());
                     return actorContext.actorSelection(cohortPath);
-
                 } else {
                     // Throwing an exception here will fail the Future.
-                    throw new IllegalArgumentException(String.format("Invalid reply type %s",
-                            serializedReadyReply.getClass()));
+                    throw new IllegalArgumentException(String.format("%s: Invalid reply type %s",
+                            identifier, serializedReadyReply.getClass()));
                 }
             }
-        }, TransactionProxy.SAME_FAILURE_TRANSFORMER, actorContext.getActorSystem().dispatcher());
+        }, TransactionProxy.SAME_FAILURE_TRANSFORMER, actorContext.getClientDispatcher());
+    }
+
+    protected String deserializeCohortPath(String cohortPath) {
+        return cohortPath;
+    }
+
+    private void batchModification(Modification modification) {
+        if(batchedModifications == null) {
+            batchedModifications = new BatchedModifications(identifier.toString(), remoteTransactionVersion,
+                    transactionChainId);
+        }
+
+        batchedModifications.addModification(modification);
+
+        if(batchedModifications.getModifications().size() >=
+                actorContext.getDatastoreContext().getShardBatchedModificationCount()) {
+            sendAndRecordBatchedModifications();
+        }
+    }
+
+    private void sendAndRecordBatchedModifications() {
+        Future<Object> sentFuture = sendBatchedModifications();
+        if(sentFuture != null) {
+            recordedOperationFutures.add(sentFuture);
+        }
+    }
+
+    protected Future<Object> sendBatchedModifications() {
+        return sendBatchedModifications(false);
+    }
+
+    protected Future<Object> sendBatchedModifications(boolean ready) {
+        Future<Object> sent = null;
+        if(batchedModifications != null) {
+            if(LOG.isDebugEnabled()) {
+                LOG.debug("Tx {} sending {} batched modifications, ready: {}", identifier,
+                        batchedModifications.getModifications().size(), ready);
+            }
+
+            batchedModifications.setReady(ready);
+            sent = executeOperationAsync(batchedModifications);
+
+            batchedModifications = new BatchedModifications(identifier.toString(), remoteTransactionVersion,
+                    transactionChainId);
+        }
+
+        return sent;
     }
 
     @Override
     public void deleteData(YangInstanceIdentifier path) {
         LOG.debug("Tx {} deleteData called path = {}", identifier, path);
 
-        recordedOperationFutures.add(executeOperationAsync(new DeleteData(path)));
+        batchModification(new DeleteModification(path));
     }
 
     @Override
     public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
         LOG.debug("Tx {} mergeData called path = {}", identifier, path);
 
-        recordedOperationFutures.add(executeOperationAsync(new MergeData(path, data)));
+        batchModification(new MergeModification(path, data));
     }
 
     @Override
     public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
         LOG.debug("Tx {} writeData called path = {}", identifier, path);
 
-        recordedOperationFutures.add(executeOperationAsync(new WriteData(path, data)));
+        batchModification(new WriteModification(path, data));
     }
 
     @Override
@@ -182,6 +229,10 @@ final class TransactionContextImpl extends AbstractTransactionContext {
 
         LOG.debug("Tx {} readData called path = {}", identifier, path);
 
+        // Send the remaining batched modifications if any.
+
+        sendAndRecordBatchedModifications();
+
         // If there were any previous recorded put/merge/delete operation reply Futures then we
         // must wait for them to successfully complete. This is necessary to honor the read
         // uncommitted semantics of the public API contract. If any one fails then fail the read.
@@ -198,7 +249,7 @@ final class TransactionContextImpl extends AbstractTransactionContext {
 
             Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(
                     Lists.newArrayList(recordedOperationFutures),
-                    actorContext.getActorSystem().dispatcher());
+                    actorContext.getClientDispatcher());
 
             OnComplete<Iterable<Object>> onComplete = new OnComplete<Iterable<Object>>() {
                 @Override
@@ -216,7 +267,7 @@ final class TransactionContextImpl extends AbstractTransactionContext {
                 }
             };
 
-            combinedFutures.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
+            combinedFutures.onComplete(onComplete, actorContext.getClientDispatcher());
         }
 
     }
@@ -255,7 +306,7 @@ final class TransactionContextImpl extends AbstractTransactionContext {
 
         Future<Object> readFuture = executeOperationAsync(new ReadData(path));
 
-        readFuture.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
+        readFuture.onComplete(onComplete, actorContext.getClientDispatcher());
     }
 
     @Override
@@ -263,6 +314,10 @@ final class TransactionContextImpl extends AbstractTransactionContext {
 
         LOG.debug("Tx {} dataExists called path = {}", identifier, path);
 
+        // Send the remaining batched modifications if any.
+
+        sendAndRecordBatchedModifications();
+
         // If there were any previous recorded put/merge/delete operation reply Futures then we
         // must wait for them to successfully complete. This is necessary to honor the read
         // uncommitted semantics of the public API contract. If any one fails then fail this
@@ -280,7 +335,7 @@ final class TransactionContextImpl extends AbstractTransactionContext {
 
             Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(
                     Lists.newArrayList(recordedOperationFutures),
-                    actorContext.getActorSystem().dispatcher());
+                    actorContext.getClientDispatcher());
             OnComplete<Iterable<Object>> onComplete = new OnComplete<Iterable<Object>>() {
                 @Override
                 public void onComplete(Throwable failure, Iterable<Object> notUsed)
@@ -297,7 +352,7 @@ final class TransactionContextImpl extends AbstractTransactionContext {
                 }
             };
 
-            combinedFutures.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
+            combinedFutures.onComplete(onComplete, actorContext.getClientDispatcher());
         }
     }
 
@@ -332,6 +387,6 @@ final class TransactionContextImpl extends AbstractTransactionContext {
 
         Future<Object> future = executeOperationAsync(new DataExists(path));
 
-        future.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
+        future.onComplete(onComplete, actorContext.getClientDispatcher());
     }
 }