Verify BatchedModifications messages sent vs received
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / TransactionContextImpl.java
index 3a209630c3344ca149032c2cc1d4f06b134ccf42..c722918c5cfed8ad7062e63911ae60fa45aad7fa 100644 (file)
@@ -1,5 +1,6 @@
 /*
  * Copyright (c) 2015 Cisco Systems, Inc. and others.  All rights reserved.
+ * Copyright (c) 2015 Brocade Communications Systems, Inc. and others.  All rights reserved.
  *
  * This program and the accompanying materials are made available under the
  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
@@ -11,18 +12,14 @@ import akka.actor.ActorSelection;
 import akka.dispatch.Mapper;
 import akka.dispatch.OnComplete;
 import com.google.common.base.Optional;
-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.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.modification.DeleteModification;
@@ -49,6 +46,7 @@ public class TransactionContextImpl extends AbstractTransactionContext {
 
     private final OperationCompleter operationCompleter;
     private BatchedModifications batchedModifications;
+    private int totalBatchedModificationsSent;
 
     protected TransactionContextImpl(ActorSelection actor, TransactionIdentifier identifier,
             String transactionChainId, ActorContext actorContext, SchemaContext schemaContext, boolean isTxActorLocal,
@@ -86,97 +84,63 @@ public class TransactionContextImpl extends AbstractTransactionContext {
 
     @Override
     public void closeTransaction() {
-        LOG.debug("Tx {} closeTransaction called", identifier);
+        LOG.debug("Tx {} closeTransaction called", getIdentifier());
 
         actorContext.sendOperationAsync(getActor(), CloseTransaction.INSTANCE.toSerializable());
     }
 
     @Override
     public Future<ActorSelection> readyTransaction() {
-        LOG.debug("Tx {} readyTransaction called with {} previous recorded operations pending",
-                identifier, recordedOperationFutures.size());
+        LOG.debug("Tx {} readyTransaction called", getIdentifier());
 
-        // Send the remaining batched modifications if any.
+        // Send the remaining batched modifications, if any, with the ready flag set.
 
-        sendAndRecordBatchedModifications();
+        Future<Object> lastModificationsFuture = sendBatchedModifications(true);
 
-        // Send the ReadyTransaction message to the Tx actor.
-
-        Future<Object> readyReplyFuture = executeOperationAsync(ReadyTransaction.INSTANCE);
-
-        return combineRecordedOperationsFutures(readyReplyFuture);
+        return transformReadyReply(lastModificationsFuture);
     }
 
-    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);
-        futureList.addAll(recordedOperationFutures);
-        futureList.add(withLastReplyFuture);
-
-        Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(futureList,
-                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.
+    protected Future<ActorSelection> transformReadyReply(final Future<Object> readyReplyFuture) {
+        // Transform the last reply Future into a Future that returns the cohort actor path from
+        // the last reply message. That's the end result of the ready operation.
 
-        return combinedFutures.transform(new Mapper<Iterable<Object>, ActorSelection>() {
+        return readyReplyFuture.transform(new Mapper<Object, ActorSelection>() {
             @Override
-            public ActorSelection checkedApply(Iterable<Object> notUsed) {
-                LOG.debug("Tx {} readyTransaction: pending recorded operations succeeded",
-                        identifier);
-
-                // At this point all the Futures succeeded and we need to extract the cohort
-                // actor path from the ReadyTransactionReply. For the recorded operations, they
-                // don't return any data so we're only interested that they completed
-                // successfully. We could be paranoid and verify the correct reply types but
-                // that really should never happen so it's not worth the overhead of
-                // de-serializing each reply.
-
-                // Note the Future get call here won't block as it's complete.
-                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 = deserializeCohortPath(reply.getCohortPath());
-                    return actorContext.actorSelection(cohortPath);
-                } else {
-                    // Throwing an exception here will fail the Future.
-                    throw new IllegalArgumentException(String.format("%s: Invalid reply type %s",
-                            identifier, serializedReadyReply.getClass()));
+            public ActorSelection checkedApply(Object serializedReadyReply) {
+                LOG.debug("Tx {} readyTransaction", getIdentifier());
+
+                // At this point the ready operation succeeded and we need to extract the cohort
+                // actor path from the reply.
+                if(ReadyTransactionReply.isSerializedType(serializedReadyReply)) {
+                    ReadyTransactionReply readyTxReply = ReadyTransactionReply.fromSerializable(serializedReadyReply);
+                    return actorContext.actorSelection(extractCohortPathFrom(readyTxReply));
                 }
+
+                // Throwing an exception here will fail the Future.
+                throw new IllegalArgumentException(String.format("%s: Invalid reply type %s",
+                        getIdentifier(), serializedReadyReply.getClass()));
             }
         }, TransactionProxy.SAME_FAILURE_TRANSFORMER, actorContext.getClientDispatcher());
     }
 
-    protected String deserializeCohortPath(String cohortPath) {
-        return cohortPath;
+    protected String extractCohortPathFrom(ReadyTransactionReply readyTxReply) {
+        return readyTxReply.getCohortPath();
+    }
+
+    private BatchedModifications newBatchedModifications() {
+        return new BatchedModifications(getIdentifier().toString(), remoteTransactionVersion, transactionChainId);
     }
 
     private void batchModification(Modification modification) {
         if(batchedModifications == null) {
-            batchedModifications = new BatchedModifications(identifier.toString(), remoteTransactionVersion,
-                    transactionChainId);
+            batchedModifications = newBatchedModifications();
         }
 
         batchedModifications.addModification(modification);
 
         if(batchedModifications.getModifications().size() >=
                 actorContext.getDatastoreContext().getShardBatchedModificationCount()) {
-            sendAndRecordBatchedModifications();
-        }
-    }
-
-    private void sendAndRecordBatchedModifications() {
-        Future<Object> sentFuture = sendBatchedModifications();
-        if(sentFuture != null) {
-            recordedOperationFutures.add(sentFuture);
+            sendBatchedModifications();
         }
     }
 
@@ -186,17 +150,25 @@ public class TransactionContextImpl extends AbstractTransactionContext {
 
     protected Future<Object> sendBatchedModifications(boolean ready) {
         Future<Object> sent = null;
-        if(batchedModifications != null) {
+        if(ready || (batchedModifications != null && !batchedModifications.getModifications().isEmpty())) {
+            if(batchedModifications == null) {
+                batchedModifications = newBatchedModifications();
+            }
+
             if(LOG.isDebugEnabled()) {
-                LOG.debug("Tx {} sending {} batched modifications, ready: {}", identifier,
+                LOG.debug("Tx {} sending {} batched modifications, ready: {}", getIdentifier(),
                         batchedModifications.getModifications().size(), ready);
             }
 
             batchedModifications.setReady(ready);
+            batchedModifications.setTotalMessagesSent(++totalBatchedModificationsSent);
             sent = executeOperationAsync(batchedModifications);
 
-            batchedModifications = new BatchedModifications(identifier.toString(), remoteTransactionVersion,
-                    transactionChainId);
+            if(ready) {
+                batchedModifications = null;
+            } else {
+                batchedModifications = newBatchedModifications();
+            }
         }
 
         return sent;
@@ -204,89 +176,46 @@ public class TransactionContextImpl extends AbstractTransactionContext {
 
     @Override
     public void deleteData(YangInstanceIdentifier path) {
-        LOG.debug("Tx {} deleteData called path = {}", identifier, path);
+        LOG.debug("Tx {} deleteData called path = {}", getIdentifier(), path);
 
         batchModification(new DeleteModification(path));
     }
 
     @Override
     public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-        LOG.debug("Tx {} mergeData called path = {}", identifier, path);
+        LOG.debug("Tx {} mergeData called path = {}", getIdentifier(), path);
 
         batchModification(new MergeModification(path, data));
     }
 
     @Override
     public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-        LOG.debug("Tx {} writeData called path = {}", identifier, path);
+        LOG.debug("Tx {} writeData called path = {}", getIdentifier(), path);
 
         batchModification(new WriteModification(path, data));
     }
 
     @Override
-    public void readData(
-            final YangInstanceIdentifier path,final SettableFuture<Optional<NormalizedNode<?, ?>>> returnFuture ) {
-
-        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.
+    public void readData(final YangInstanceIdentifier path,
+            final SettableFuture<Optional<NormalizedNode<?, ?>>> returnFuture ) {
 
-        if(recordedOperationFutures.isEmpty()) {
-            finishReadData(path, returnFuture);
-        } else {
-            LOG.debug("Tx {} readData: verifying {} previous recorded operations",
-                    identifier, recordedOperationFutures.size());
+        LOG.debug("Tx {} readData called path = {}", getIdentifier(), path);
 
-            // Note: we make a copy of recordedOperationFutures to be on the safe side in case
-            // Futures#sequence accesses the passed List on a different thread, as
-            // recordedOperationFutures is not synchronized.
+        // Send any batched modifications. This is necessary to honor the read uncommitted semantics of the
+        // public API contract.
 
-            Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(
-                    Lists.newArrayList(recordedOperationFutures),
-                    actorContext.getClientDispatcher());
-
-            OnComplete<Iterable<Object>> onComplete = new OnComplete<Iterable<Object>>() {
-                @Override
-                public void onComplete(Throwable failure, Iterable<Object> notUsed)
-                        throws Throwable {
-                    if(failure != null) {
-                        LOG.debug("Tx {} readData: a recorded operation failed: {}",
-                                identifier, failure);
-                        returnFuture.setException(new ReadFailedException(
-                                "The read could not be performed because a previous put, merge,"
-                                + "or delete operation failed", failure));
-                    } else {
-                        finishReadData(path, returnFuture);
-                    }
-                }
-            };
-
-            combinedFutures.onComplete(onComplete, actorContext.getClientDispatcher());
-        }
-
-    }
-
-    private void finishReadData(final YangInstanceIdentifier path,
-            final SettableFuture<Optional<NormalizedNode<?, ?>>> returnFuture) {
-
-        LOG.debug("Tx {} finishReadData called path = {}", identifier, path);
+        sendBatchedModifications();
 
         OnComplete<Object> onComplete = new OnComplete<Object>() {
             @Override
             public void onComplete(Throwable failure, Object readResponse) throws Throwable {
                 if(failure != null) {
-                    LOG.debug("Tx {} read operation failed: {}", identifier, failure);
+                    LOG.debug("Tx {} read operation failed: {}", getIdentifier(), failure);
                     returnFuture.setException(new ReadFailedException(
                             "Error reading data for path " + path, failure));
 
                 } else {
-                    LOG.debug("Tx {} read operation succeeded", identifier, failure);
+                    LOG.debug("Tx {} read operation succeeded", getIdentifier(), failure);
 
                     if (readResponse instanceof ReadDataReply) {
                         ReadDataReply reply = (ReadDataReply) readResponse;
@@ -312,64 +241,22 @@ public class TransactionContextImpl extends AbstractTransactionContext {
     @Override
     public void dataExists(final YangInstanceIdentifier path, final SettableFuture<Boolean> returnFuture) {
 
-        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
-        // request.
-
-        if(recordedOperationFutures.isEmpty()) {
-            finishDataExists(path, returnFuture);
-        } else {
-            LOG.debug("Tx {} dataExists: verifying {} previous recorded operations",
-                    identifier, recordedOperationFutures.size());
-
-            // Note: we make a copy of recordedOperationFutures to be on the safe side in case
-            // Futures#sequence accesses the passed List on a different thread, as
-            // recordedOperationFutures is not synchronized.
-
-            Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(
-                    Lists.newArrayList(recordedOperationFutures),
-                    actorContext.getClientDispatcher());
-            OnComplete<Iterable<Object>> onComplete = new OnComplete<Iterable<Object>>() {
-                @Override
-                public void onComplete(Throwable failure, Iterable<Object> notUsed)
-                        throws Throwable {
-                    if(failure != null) {
-                        LOG.debug("Tx {} dataExists: a recorded operation failed: {}",
-                                identifier, failure);
-                        returnFuture.setException(new ReadFailedException(
-                                "The data exists could not be performed because a previous "
-                                + "put, merge, or delete operation failed", failure));
-                    } else {
-                        finishDataExists(path, returnFuture);
-                    }
-                }
-            };
-
-            combinedFutures.onComplete(onComplete, actorContext.getClientDispatcher());
-        }
-    }
+        LOG.debug("Tx {} dataExists called path = {}", getIdentifier(), path);
 
-    private void finishDataExists(final YangInstanceIdentifier path,
-            final SettableFuture<Boolean> returnFuture) {
+        // Send any batched modifications. This is necessary to honor the read uncommitted semantics of the
+        // public API contract.
 
-        LOG.debug("Tx {} finishDataExists called path = {}", identifier, path);
+        sendBatchedModifications();
 
         OnComplete<Object> onComplete = new OnComplete<Object>() {
             @Override
             public void onComplete(Throwable failure, Object response) throws Throwable {
                 if(failure != null) {
-                    LOG.debug("Tx {} dataExists operation failed: {}", identifier, failure);
+                    LOG.debug("Tx {} dataExists operation failed: {}", getIdentifier(), failure);
                     returnFuture.setException(new ReadFailedException(
                             "Error checking data exists for path " + path, failure));
                 } else {
-                    LOG.debug("Tx {} dataExists operation succeeded", identifier, failure);
+                    LOG.debug("Tx {} dataExists operation succeeded", getIdentifier(), failure);
 
                     if (response instanceof DataExistsReply) {
                         returnFuture.set(Boolean.valueOf(((DataExistsReply) response).exists()));