Add type of transaction being created to log message
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / TransactionProxy.java
index c85d32012fed9ee036a96b2e3663c061eabd0385..6183c489c4cdbc56a1ba9ac0c3c3939963277edb 100644 (file)
@@ -12,14 +12,20 @@ import akka.actor.ActorPath;
 import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
 import akka.actor.Props;
+import akka.dispatch.OnComplete;
+
 import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.CheckedFuture;
 import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListenableFutureTask;
-import org.opendaylight.controller.cluster.datastore.exceptions.TimeoutException;
+import com.google.common.util.concurrent.SettableFuture;
+
+import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.CloseTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
+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;
@@ -29,20 +35,22 @@ import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionRe
 import org.opendaylight.controller.cluster.datastore.messages.WriteData;
 import org.opendaylight.controller.cluster.datastore.shardstrategy.ShardStrategyFactory;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
+import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadWriteTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
+import org.opendaylight.yangtools.util.concurrent.MappingCheckedFuture;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import scala.concurrent.Future;
+
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
 import java.util.concurrent.atomic.AtomicLong;
 
 /**
@@ -73,37 +81,69 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
     private final TransactionType transactionType;
     private final ActorContext actorContext;
     private final Map<String, TransactionContext> remoteTransactionPaths = new HashMap<>();
-    private final String identifier;
-    private final ExecutorService executor;
+    private final TransactionIdentifier identifier;
     private final SchemaContext schemaContext;
+    private boolean inReadyState;
+
+    public TransactionProxy(ActorContext actorContext, TransactionType transactionType,
+            SchemaContext schemaContext) {
+        this.actorContext = Preconditions.checkNotNull(actorContext, "actorContext should not be null");
+        this.transactionType = Preconditions.checkNotNull(transactionType, "transactionType should not be null");
+        this.schemaContext = Preconditions.checkNotNull(schemaContext, "schemaContext should not be null");
 
-    public TransactionProxy(
-        ActorContext actorContext,
-        TransactionType transactionType,
-        ExecutorService executor,
-        SchemaContext schemaContext
-    ) {
+        String memberName = actorContext.getCurrentMemberName();
+        if(memberName == null){
+            memberName = "UNKNOWN-MEMBER";
+        }
 
-        this.identifier = actorContext.getCurrentMemberName() + "-txn-" + counter.getAndIncrement();
-        this.transactionType = transactionType;
-        this.actorContext = actorContext;
-        this.executor = executor;
-        this.schemaContext = schemaContext;
+        this.identifier = TransactionIdentifier.builder().memberName(memberName).counter(
+                counter.getAndIncrement()).build();
 
+        LOG.debug("Created txn {} of type {}", identifier, transactionType);
 
     }
 
     @Override
-    public ListenableFuture<Optional<NormalizedNode<?, ?>>> 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("txn {} read {}", identifier, path);
 
         createTransactionIfMissing(actorContext, path);
 
         return transactionContext(path).readData(path);
     }
 
+    @Override
+    public CheckedFuture<Boolean, ReadFailedException> exists(YangInstanceIdentifier path) {
+
+        Preconditions.checkState(transactionType != TransactionType.WRITE_ONLY,
+                "Exists operation on write-only transaction is not allowed");
+
+        LOG.debug("txn {} exists {}", identifier, path);
+
+        createTransactionIfMissing(actorContext, path);
+
+        return transactionContext(path).dataExists(path);
+    }
+
+    private void checkModificationState() {
+        Preconditions.checkState(transactionType != TransactionType.READ_ONLY,
+                "Modification operation on read-only transaction is not allowed");
+        Preconditions.checkState(!inReadyState,
+                "Transaction is sealed - further modifications are allowed");
+    }
+
     @Override
     public void write(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
 
+        checkModificationState();
+
+        LOG.debug("txn {} write {}", identifier, path);
+
         createTransactionIfMissing(actorContext, path);
 
         transactionContext(path).writeData(path, data);
@@ -112,6 +152,10 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
     @Override
     public void merge(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
 
+        checkModificationState();
+
+        LOG.debug("txn {} merge {}", identifier, path);
+
         createTransactionIfMissing(actorContext, path);
 
         transactionContext(path).mergeData(path, data);
@@ -120,6 +164,10 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
     @Override
     public void delete(YangInstanceIdentifier path) {
 
+        checkModificationState();
+
+        LOG.debug("txn {} delete {}", identifier, path);
+
         createTransactionIfMissing(actorContext, path);
 
         transactionContext(path).deleteData(path);
@@ -127,20 +175,36 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
     @Override
     public DOMStoreThreePhaseCommitCohort ready() {
+
+        checkModificationState();
+
+        inReadyState = true;
+
         List<ActorPath> cohortPaths = new ArrayList<>();
 
+        LOG.debug("txn {} Trying to get {} transactions ready for commit", identifier,
+                remoteTransactionPaths.size());
+
         for(TransactionContext transactionContext : remoteTransactionPaths.values()) {
+
+            LOG.debug("txn {} Readying transaction for shard {}", identifier,
+                    transactionContext.getShardName());
+
             Object result = transactionContext.readyTransaction();
 
             if(result.getClass().equals(ReadyTransactionReply.SERIALIZABLE_CLASS)){
-                ReadyTransactionReply reply = ReadyTransactionReply.fromSerializable(actorContext.getActorSystem(),result);
-                String resolvedCohortPath = transactionContext
-                    .getResolvedCohortPath(reply.getCohortPath().toString());
+                ReadyTransactionReply reply = ReadyTransactionReply.fromSerializable(
+                        actorContext.getActorSystem(),result);
+                String resolvedCohortPath = transactionContext.getResolvedCohortPath(
+                        reply.getCohortPath().toString());
                 cohortPaths.add(actorContext.actorFor(resolvedCohortPath));
+            } else {
+                LOG.error("Was expecting {} but got {}", ReadyTransactionReply.SERIALIZABLE_CLASS,
+                        result.getClass());
             }
         }
 
-        return new ThreePhaseCommitCohortProxy(actorContext, cohortPaths, identifier, executor);
+        return new ThreePhaseCommitCohortProxy(actorContext, cohortPaths, identifier.toString());
     }
 
     @Override
@@ -177,16 +241,15 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         try {
             Object response = actorContext.executeShardOperation(shardName,
-                new CreateTransaction(identifier).toSerializable(),
+                new CreateTransaction(identifier.toString(),this.transactionType.ordinal() ).toSerializable(),
                 ActorContext.ASK_DURATION);
-            if (response.getClass()
-                .equals(CreateTransactionReply.SERIALIZABLE_CLASS)) {
+            if (response.getClass().equals(CreateTransactionReply.SERIALIZABLE_CLASS)) {
                 CreateTransactionReply reply =
                     CreateTransactionReply.fromSerializable(response);
 
                 String transactionPath = reply.getTransactionPath();
 
-                LOG.info("Received transaction path = {}"  , transactionPath );
+                LOG.debug("txn {} Received transaction path = {}", identifier, transactionPath);
 
                 ActorSelection transactionActor =
                     actorContext.actorSelection(transactionPath);
@@ -195,9 +258,13 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                         transactionActor);
 
                 remoteTransactionPaths.put(shardName, transactionContext);
+            } else {
+                LOG.error("Was expecting {} but got {}", CreateTransactionReply.SERIALIZABLE_CLASS,
+                        response.getClass());
             }
-        } catch(TimeoutException e){
-            remoteTransactionPaths.put(shardName, new NoOpTransactionContext(shardName));
+        } catch(Exception e){
+            LOG.error("txn {} Creating NoOpTransaction because of : {}", identifier, e.getMessage());
+            remoteTransactionPaths.put(shardName, new NoOpTransactionContext(shardName, e));
         }
     }
 
@@ -214,16 +281,19 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
         void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
 
-        ListenableFuture<Optional<NormalizedNode<?, ?>>> readData(final YangInstanceIdentifier path);
+        CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
+                final YangInstanceIdentifier path);
 
         void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
+
+        CheckedFuture<Boolean, ReadFailedException> dataExists(YangInstanceIdentifier path);
     }
 
 
-    private class TransactionContextImpl implements TransactionContext{
+    private class TransactionContextImpl implements TransactionContext {
         private final String shardName;
         private final String actorPath;
-        private final ActorSelection  actor;
+        private final ActorSelection actor;
 
 
         private TransactionContextImpl(String shardName, String actorPath,
@@ -233,7 +303,8 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             this.actor = actor;
         }
 
-        @Override public String getShardName() {
+        @Override
+        public String getShardName() {
             return shardName;
         }
 
@@ -241,64 +312,106 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             return actor;
         }
 
-        @Override public String getResolvedCohortPath(String cohortPath){
+        @Override
+        public String getResolvedCohortPath(String cohortPath) {
             return actorContext.resolvePath(actorPath, cohortPath);
         }
 
-        @Override public void closeTransaction() {
-            getActor().tell(
-                new CloseTransaction().toSerializable(), null);
+        @Override
+        public void closeTransaction() {
+            actorContext.sendRemoteOperationAsync(getActor(), new CloseTransaction().toSerializable());
         }
 
-        @Override public Object readyTransaction() {
+        @Override
+        public Object readyTransaction() {
             return actorContext.executeRemoteOperation(getActor(),
-                new ReadyTransaction().toSerializable(),
-                ActorContext.ASK_DURATION
-            );
-
+                    new ReadyTransaction().toSerializable(), ActorContext.ASK_DURATION);
         }
 
-        @Override public void deleteData(YangInstanceIdentifier path) {
-            getActor().tell(new DeleteData(path).toSerializable(), null);
+        @Override
+        public void deleteData(YangInstanceIdentifier path) {
+            actorContext.sendRemoteOperationAsync(getActor(), new DeleteData(path).toSerializable() );
         }
 
-        @Override public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data){
-            getActor().tell(new MergeData(path, data, schemaContext).toSerializable(), null);
+        @Override
+        public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
+            actorContext.sendRemoteOperationAsync(getActor(),
+                    new MergeData(path, data, schemaContext).toSerializable());
         }
 
-        @Override public ListenableFuture<Optional<NormalizedNode<?, ?>>> readData(final YangInstanceIdentifier path) {
-
-            Callable<Optional<NormalizedNode<?,?>>> call = new Callable() {
-
-                @Override public Optional<NormalizedNode<?,?>> call() throws Exception {
-                    Object response = actorContext
-                        .executeRemoteOperation(getActor(), new ReadData(path).toSerializable(),
-                            ActorContext.ASK_DURATION);
-                    if(response.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)){
-                        ReadDataReply reply = ReadDataReply.fromSerializable(schemaContext,path, response);
-                        if(reply.getNormalizedNode() == null){
-                            return Optional.absent();
+        @Override
+        public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
+            final YangInstanceIdentifier path) {
+
+            final SettableFuture<Optional<NormalizedNode<?, ?>>> returnFuture = SettableFuture.create();
+
+            OnComplete<Object> onComplete = new OnComplete<Object>() {
+                @Override
+                public void onComplete(Throwable failure, Object response) throws Throwable {
+                    if(failure != null) {
+                        returnFuture.setException(new ReadFailedException(
+                                "Error reading data for path " + path, failure));
+                    } else {
+                        if (response.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)) {
+                            ReadDataReply reply = ReadDataReply.fromSerializable(schemaContext,
+                                    path, response);
+                            if (reply.getNormalizedNode() == null) {
+                                returnFuture.set(Optional.<NormalizedNode<?, ?>>absent());
+                            } else {
+                                returnFuture.set(Optional.<NormalizedNode<?, ?>>of(
+                                        reply.getNormalizedNode()));
+                            }
+                        } else {
+                            returnFuture.setException(new ReadFailedException(
+                                    "Invalid response reading data for path " + path));
                         }
-                        //FIXME : A cast should not be required here ???
-                        return (Optional<NormalizedNode<?, ?>>) Optional.of(reply.getNormalizedNode());
                     }
-
-                    return Optional.absent();
                 }
             };
 
-            ListenableFutureTask<Optional<NormalizedNode<?, ?>>>
-                future = ListenableFutureTask.create(call);
-
-            executor.submit(future);
+            Future<Object> future = actorContext.executeRemoteOperationAsync(getActor(),
+                    new ReadData(path).toSerializable(), ActorContext.ASK_DURATION);
+            future.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
 
-            return future;
+            return MappingCheckedFuture.create(returnFuture, ReadFailedException.MAPPER);
         }
 
-        @Override public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-            getActor().tell(new WriteData(path, data, schemaContext).toSerializable(), null);
+        @Override
+        public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
+            actorContext.sendRemoteOperationAsync(getActor(),
+                    new WriteData(path, data, schemaContext).toSerializable());
         }
 
+        @Override
+        public CheckedFuture<Boolean, ReadFailedException> dataExists(
+                final YangInstanceIdentifier path) {
+
+            final SettableFuture<Boolean> returnFuture = SettableFuture.create();
+
+            OnComplete<Object> onComplete = new OnComplete<Object>() {
+                @Override
+                public void onComplete(Throwable failure, Object response) throws Throwable {
+                    if(failure != null) {
+                        returnFuture.setException(new ReadFailedException(
+                                "Error checking exists for path " + path, failure));
+                    } else {
+                        if (response.getClass().equals(DataExistsReply.SERIALIZABLE_CLASS)) {
+                            returnFuture.set(Boolean.valueOf(DataExistsReply.
+                                        fromSerializable(response).exists()));
+                        } else {
+                            returnFuture.setException(new ReadFailedException(
+                                    "Invalid response checking exists for path " + path));
+                        }
+                    }
+                }
+            };
+
+            Future<Object> future = actorContext.executeRemoteOperationAsync(getActor(),
+                    new DataExists(path).toSerializable(), ActorContext.ASK_DURATION);
+            future.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
+
+            return MappingCheckedFuture.create(returnFuture, ReadFailedException.MAPPER);
+        }
     }
 
     private class NoOpTransactionContext implements TransactionContext {
@@ -307,51 +420,66 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             LOG = LoggerFactory.getLogger(NoOpTransactionContext.class);
 
         private final String shardName;
+        private final Exception failure;
 
         private ActorRef cohort;
 
-        public NoOpTransactionContext(String shardName){
+        public NoOpTransactionContext(String shardName, Exception failure){
             this.shardName = shardName;
+            this.failure = failure;
         }
-        @Override public String getShardName() {
+
+        @Override
+        public String getShardName() {
             return  shardName;
 
         }
 
-        @Override public String getResolvedCohortPath(String cohortPath) {
+        @Override
+        public String getResolvedCohortPath(String cohortPath) {
             return cohort.path().toString();
         }
 
-        @Override public void closeTransaction() {
-            LOG.error("closeTransaction called");
+        @Override
+        public void closeTransaction() {
+            LOG.warn("txn {} closeTransaction called", identifier);
         }
 
         @Override public Object readyTransaction() {
-            LOG.error("readyTransaction called");
+            LOG.warn("txn {} readyTransaction called", identifier);
             cohort = actorContext.getActorSystem().actorOf(Props.create(NoOpCohort.class));
             return new ReadyTransactionReply(cohort.path()).toSerializable();
         }
 
-        @Override public void deleteData(YangInstanceIdentifier path) {
-            LOG.error("deleteData called path = {}", path);
+        @Override
+        public void deleteData(YangInstanceIdentifier path) {
+            LOG.warn("txt {} deleteData called path = {}", identifier, path);
         }
 
-        @Override public void mergeData(YangInstanceIdentifier path,
+        @Override
+        public void mergeData(YangInstanceIdentifier path,
             NormalizedNode<?, ?> data) {
-            LOG.error("mergeData called path = {}", path);
+            LOG.warn("txn {} mergeData called path = {}", identifier, path);
         }
 
         @Override
-        public ListenableFuture<Optional<NormalizedNode<?, ?>>> readData(
+        public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
             YangInstanceIdentifier path) {
-            LOG.error("readData called path = {}", path);
-            return Futures.immediateFuture(
-                Optional.<NormalizedNode<?, ?>>absent());
+            LOG.warn("txn {} readData called path = {}", identifier, path);
+            return Futures.immediateFailedCheckedFuture(new ReadFailedException(
+                    "Error reading data for path " + path, failure));
         }
 
         @Override public void writeData(YangInstanceIdentifier path,
             NormalizedNode<?, ?> data) {
-            LOG.error("writeData called path = {}", path);
+            LOG.warn("txn {} writeData called path = {}", identifier, path);
+        }
+
+        @Override public CheckedFuture<Boolean, ReadFailedException> dataExists(
+            YangInstanceIdentifier path) {
+            LOG.warn("txn {} dataExists called path = {}", identifier, path);
+            return Futures.immediateFailedCheckedFuture(new ReadFailedException(
+                    "Error checking exists for path " + path, failure));
         }
     }