Merge "Bug 1534: Changed blocking calls to async in dist data store"
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / ThreePhaseCommitCohortProxy.java
index 915b13dd8bc234a6cbf898658b8e6479333b36c2..fc455b193e27118f6dcfcc2de93032f2676c5619 100644 (file)
@@ -10,11 +10,13 @@ package org.opendaylight.controller.cluster.datastore;
 
 import akka.actor.ActorPath;
 import akka.actor.ActorSelection;
+import akka.dispatch.Futures;
+import akka.dispatch.OnComplete;
 
+import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.SettableFuture;
 
-import org.opendaylight.controller.cluster.datastore.exceptions.TimeoutException;
 import org.opendaylight.controller.cluster.datastore.messages.AbortTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.AbortTransactionReply;
 import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransaction;
@@ -28,113 +30,156 @@ import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCoh
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import scala.concurrent.Future;
+
 import java.util.Collections;
 import java.util.List;
-import java.util.concurrent.Callable;
 
 /**
  * ThreePhaseCommitCohortProxy represents a set of remote cohort proxies
  */
-public class ThreePhaseCommitCohortProxy implements
-    DOMStoreThreePhaseCommitCohort{
+public class ThreePhaseCommitCohortProxy implements DOMStoreThreePhaseCommitCohort{
 
-    private static final Logger
-        LOG = LoggerFactory.getLogger(DistributedDataStore.class);
+    private static final Logger LOG = LoggerFactory.getLogger(DistributedDataStore.class);
 
     private final ActorContext actorContext;
     private final List<ActorPath> cohortPaths;
-    private final ListeningExecutorService executor;
     private final String transactionId;
 
-
-    public ThreePhaseCommitCohortProxy(ActorContext actorContext,
-        List<ActorPath> cohortPaths,
-        String transactionId,
-        ListeningExecutorService executor) {
-
+    public ThreePhaseCommitCohortProxy(ActorContext actorContext, List<ActorPath> cohortPaths,
+            String transactionId) {
         this.actorContext = actorContext;
         this.cohortPaths = cohortPaths;
         this.transactionId = transactionId;
-        this.executor = executor;
     }
 
-    @Override public ListenableFuture<Boolean> canCommit() {
-        Callable<Boolean> call = new Callable<Boolean>() {
+    @Override
+    public ListenableFuture<Boolean> canCommit() {
+        LOG.debug("txn {} canCommit", transactionId);
 
+        Future<Iterable<Object>> combinedFuture =
+                invokeCohorts(new CanCommitTransaction().toSerializable());
+
+        final SettableFuture<Boolean> returnFuture = SettableFuture.create();
+
+        combinedFuture.onComplete(new OnComplete<Iterable<Object>>() {
             @Override
-            public Boolean call() throws Exception {
-                for(ActorPath actorPath : cohortPaths){
-                    ActorSelection cohort = actorContext.actorSelection(actorPath);
-
-                    try {
-                        Object response =
-                                actorContext.executeRemoteOperation(cohort,
-                                        new CanCommitTransaction().toSerializable(),
-                                        ActorContext.ASK_DURATION);
-
-                        if (response.getClass().equals(CanCommitTransactionReply.SERIALIZABLE_CLASS)) {
-                            CanCommitTransactionReply reply =
-                                    CanCommitTransactionReply.fromSerializable(response);
-                            if (!reply.getCanCommit()) {
-                                return false;
-                            }
+            public void onComplete(Throwable failure, Iterable<Object> responses) throws Throwable {
+                if(failure != null) {
+                    returnFuture.setException(failure);
+                    return;
+                }
+
+                boolean result = true;
+                for(Object response: responses) {
+                    if (response.getClass().equals(CanCommitTransactionReply.SERIALIZABLE_CLASS)) {
+                        CanCommitTransactionReply reply =
+                                CanCommitTransactionReply.fromSerializable(response);
+                        if (!reply.getCanCommit()) {
+                            result = false;
+                            break;
                         }
-                    } catch(RuntimeException e){
-                        LOG.error("Unexpected Exception", e);
-                        return false;
+                    } else {
+                        LOG.error("Unexpected response type {}", response.getClass());
+                        returnFuture.setException(new IllegalArgumentException(
+                                String.format("Unexpected response type {}", response.getClass())));
+                        return;
                     }
                 }
 
-                return true;
+                returnFuture.set(Boolean.valueOf(result));
             }
-        };
+        }, actorContext.getActorSystem().dispatcher());
+
+        return returnFuture;
+    }
+
+    private Future<Iterable<Object>> invokeCohorts(Object message) {
+        List<Future<Object>> futureList = Lists.newArrayListWithCapacity(cohortPaths.size());
+        for(ActorPath actorPath : cohortPaths) {
+
+            LOG.debug("txn {} Sending {} to {}", transactionId, message, actorPath);
 
-        return executor.submit(call);
+            ActorSelection cohort = actorContext.actorSelection(actorPath);
+
+            futureList.add(actorContext.executeRemoteOperationAsync(cohort, message,
+                    ActorContext.ASK_DURATION));
+        }
+
+        return Futures.sequence(futureList, actorContext.getActorSystem().dispatcher());
     }
 
-    @Override public ListenableFuture<Void> preCommit() {
-        return voidOperation(new PreCommitTransaction().toSerializable(), PreCommitTransactionReply.SERIALIZABLE_CLASS);
+    @Override
+    public ListenableFuture<Void> preCommit() {
+        LOG.debug("txn {} preCommit", transactionId);
+        return voidOperation(new PreCommitTransaction().toSerializable(),
+                PreCommitTransactionReply.SERIALIZABLE_CLASS, true);
     }
 
-    @Override public ListenableFuture<Void> abort() {
-        return voidOperation(new AbortTransaction().toSerializable(), AbortTransactionReply.SERIALIZABLE_CLASS);
+    @Override
+    public ListenableFuture<Void> abort() {
+        LOG.debug("txn {} abort", transactionId);
+
+        // Note - we pass false for propagateException. In the front-end data broker, this method
+        // is called when one of the 3 phases fails with an exception. We'd rather have that
+        // original exception propagated to the client. If our abort fails and we propagate the
+        // exception then that exception will supersede and suppress the original exception. But
+        // it's the original exception that is the root cause and of more interest to the client.
+
+        return voidOperation(new AbortTransaction().toSerializable(),
+                AbortTransactionReply.SERIALIZABLE_CLASS, false);
     }
 
-    @Override public ListenableFuture<Void> commit() {
-        return voidOperation(new CommitTransaction().toSerializable(), CommitTransactionReply.SERIALIZABLE_CLASS);
+    @Override
+    public ListenableFuture<Void> commit() {
+        LOG.debug("txn {} commit", transactionId);
+        return voidOperation(new CommitTransaction().toSerializable(),
+                CommitTransactionReply.SERIALIZABLE_CLASS, true);
     }
 
-    private ListenableFuture<Void> voidOperation(final Object message, final Class expectedResponseClass){
-        Callable<Void> call = new Callable<Void>() {
-
-            @Override public Void call() throws Exception {
-                for(ActorPath actorPath : cohortPaths){
-                    ActorSelection cohort = actorContext.actorSelection(actorPath);
-
-                    try {
-                        Object response =
-                            actorContext.executeRemoteOperation(cohort,
-                                message,
-                                ActorContext.ASK_DURATION);
-
-                        if (response != null && !response.getClass()
-                            .equals(expectedResponseClass)) {
-                            throw new RuntimeException(
-                                String.format(
-                                    "did not get the expected response \n\t\t expected : %s \n\t\t actual   : %s",
-                                    expectedResponseClass.toString(),
-                                    response.getClass().toString())
-                            );
+    private ListenableFuture<Void> voidOperation(final Object message,
+            final Class<?> expectedResponseClass, final boolean propagateException) {
+
+        Future<Iterable<Object>> combinedFuture = invokeCohorts(message);
+
+        final SettableFuture<Void> returnFuture = SettableFuture.create();
+
+        combinedFuture.onComplete(new OnComplete<Iterable<Object>>() {
+            @Override
+            public void onComplete(Throwable failure, Iterable<Object> responses) throws Throwable {
+
+                Throwable exceptionToPropagate = failure;
+                if(exceptionToPropagate == null) {
+                    for(Object response: responses) {
+                        if(!response.getClass().equals(expectedResponseClass)) {
+                            exceptionToPropagate = new IllegalArgumentException(
+                                    String.format("Unexpected response type {}",
+                                            response.getClass()));
+                            break;
                         }
-                    } catch(TimeoutException e){
-                        LOG.error(String.format("A timeout occurred when processing operation : %s", message));
                     }
                 }
-                return null;
+
+                if(exceptionToPropagate != null) {
+                    if(propagateException) {
+                        // We don't log the exception here to avoid redundant logging since we're
+                        // propagating to the caller in MD-SAL core who will log it.
+                        returnFuture.setException(exceptionToPropagate);
+                    } else {
+                        // Since the caller doesn't want us to propagate the exception we'll also
+                        // not log it normally. But it's usually not good to totally silence
+                        // exceptions so we'll log it to debug level.
+                        LOG.debug(String.format("%s failed",  message.getClass().getSimpleName()),
+                                exceptionToPropagate);
+                        returnFuture.set(null);
+                    }
+                } else {
+                    returnFuture.set(null);
+                }
             }
-        };
+        }, actorContext.getActorSystem().dispatcher());
 
-        return executor.submit(call);
+        return returnFuture;
     }
 
     public List<ActorPath> getCohortPaths() {