Optimizations, Monitoring and Logging
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / TransactionProxy.java
index 32bb7d0951964975b850c8a1a685ce7d95c03f47..5f9f1f83c4dbf15b38ea5a3d13d534354b4527e2 100644 (file)
@@ -9,10 +9,17 @@
 package org.opendaylight.controller.cluster.datastore;
 
 import akka.actor.ActorPath;
+import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
+import akka.actor.Props;
 import com.google.common.base.Optional;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListenableFutureTask;
+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.ListeningExecutorService;
+import org.opendaylight.controller.cluster.datastore.exceptions.PrimaryNotFoundException;
+import org.opendaylight.controller.cluster.datastore.exceptions.TimeoutException;
+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;
@@ -23,18 +30,23 @@ 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.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.yang.data.api.InstanceIdentifier;
+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 java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
-import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicLong;
 
 /**
@@ -50,7 +62,6 @@ import java.util.concurrent.atomic.AtomicLong;
  * </p>
  */
 public class TransactionProxy implements DOMStoreReadWriteTransaction {
-
     public enum TransactionType {
         READ_ONLY,
         WRITE_ONLY,
@@ -59,90 +70,100 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
     private static final AtomicLong counter = new AtomicLong();
 
-    private final TransactionType readOnly;
+    private static final Logger
+        LOG = LoggerFactory.getLogger(TransactionProxy.class);
+
+
+    private final TransactionType transactionType;
     private final ActorContext actorContext;
-    private final Map<String, ActorSelection> remoteTransactionPaths = new HashMap<>();
-    private final String identifier;
+    private final Map<String, TransactionContext> remoteTransactionPaths = new HashMap<>();
+    private final TransactionIdentifier identifier;
+    private final ListeningExecutorService executor;
+    private final SchemaContext schemaContext;
 
     public TransactionProxy(
         ActorContext actorContext,
-        TransactionType readOnly) {
-
-        this.identifier = "transaction-" + counter.getAndIncrement();
-        this.readOnly = readOnly;
-        this.actorContext = actorContext;
+        TransactionType transactionType,
+        ListeningExecutorService executor,
+        SchemaContext schemaContext
+    ) {
+        this.actorContext = Preconditions.checkNotNull(actorContext, "actorContext should not be null");
+        this.transactionType = Preconditions.checkNotNull(transactionType, "transactionType should not be null");
+        this.executor = Preconditions.checkNotNull(executor, "executor should not be null");
+        this.schemaContext = Preconditions.checkNotNull(schemaContext, "schemaContext should not be null");
 
-        Object response = actorContext.executeShardOperation(Shard.DEFAULT_NAME, new CreateTransaction(), ActorContext.ASK_DURATION);
-        if(response instanceof CreateTransactionReply){
-            CreateTransactionReply reply = (CreateTransactionReply) response;
-            remoteTransactionPaths.put(Shard.DEFAULT_NAME, actorContext.actorSelection(reply.getTransactionPath()));
+        String memberName = actorContext.getCurrentMemberName();
+        if(memberName == null){
+            memberName = "UNKNOWN-MEMBER";
         }
+        this.identifier = TransactionIdentifier.builder().memberName(memberName).counter(counter.getAndIncrement()).build();
+
+        LOG.debug("Created txn {}", identifier);
+
     }
 
     @Override
-    public ListenableFuture<Optional<NormalizedNode<?, ?>>> read(final InstanceIdentifier path) {
-        final ActorSelection remoteTransaction = remoteTransactionFromIdentifier(path);
-
-        Callable<Optional<NormalizedNode<?,?>>> call = new Callable() {
-
-            @Override public Optional<NormalizedNode<?,?>> call() throws Exception {
-                Object response = actorContext
-                    .executeRemoteOperation(remoteTransaction, new ReadData(path),
-                        ActorContext.ASK_DURATION);
-                if(response instanceof ReadDataReply){
-                    ReadDataReply reply = (ReadDataReply) response;
-                    //FIXME : A cast should not be required here ???
-                    return (Optional<NormalizedNode<?, ?>>) Optional.of(reply.getNormalizedNode());
-                }
+    public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> read(
+            final YangInstanceIdentifier path) {
 
-                return Optional.absent();
-            }
-        };
-
-        ListenableFutureTask<Optional<NormalizedNode<?, ?>>>
-            future = ListenableFutureTask.create(call);
+        LOG.debug("txn {} read {}", identifier, path);
 
-        //FIXME : Use a thread pool here
-        Executors.newSingleThreadExecutor().submit(future);
+        createTransactionIfMissing(actorContext, path);
 
-        return future;
+        return transactionContext(path).readData(path);
     }
 
     @Override
-    public void write(InstanceIdentifier path, NormalizedNode<?, ?> data) {
-        final ActorSelection remoteTransaction = remoteTransactionFromIdentifier(path);
-        remoteTransaction.tell(new WriteData(path, data), null);
+    public void write(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
+
+        LOG.debug("txn {} write {}", identifier, path);
+
+        createTransactionIfMissing(actorContext, path);
+
+        transactionContext(path).writeData(path, data);
     }
 
     @Override
-    public void merge(InstanceIdentifier path, NormalizedNode<?, ?> data) {
-        final ActorSelection remoteTransaction = remoteTransactionFromIdentifier(path);
-        remoteTransaction.tell(new MergeData(path, data), null);
+    public void merge(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
+
+        LOG.debug("txn {} merge {}", identifier, path);
+
+        createTransactionIfMissing(actorContext, path);
+
+        transactionContext(path).mergeData(path, data);
     }
 
     @Override
-    public void delete(InstanceIdentifier path) {
-        final ActorSelection remoteTransaction = remoteTransactionFromIdentifier(path);
-        remoteTransaction.tell(new DeleteData(path), null);
+    public void delete(YangInstanceIdentifier path) {
+
+        LOG.debug("txn {} delete {}", identifier, path);
+
+        createTransactionIfMissing(actorContext, path);
+
+        transactionContext(path).deleteData(path);
     }
 
     @Override
     public DOMStoreThreePhaseCommitCohort ready() {
         List<ActorPath> cohortPaths = new ArrayList<>();
 
-        for(ActorSelection remoteTransaction : remoteTransactionPaths.values()) {
-            Object result = actorContext.executeRemoteOperation(remoteTransaction,
-                new ReadyTransaction(),
-                ActorContext.ASK_DURATION
-            );
+        LOG.debug("txn {} Trying to get {} transactions ready for commit", identifier, remoteTransactionPaths.size());
 
-            if(result instanceof ReadyTransactionReply){
-                ReadyTransactionReply reply = (ReadyTransactionReply) result;
-                cohortPaths.add(reply.getCohortPath());
+        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());
+                cohortPaths.add(actorContext.actorFor(resolvedCohortPath));
             }
         }
 
-        return new ThreePhaseCommitCohortProxy(cohortPaths);
+        return new ThreePhaseCommitCohortProxy(actorContext, cohortPaths, identifier.toString(), executor);
     }
 
     @Override
@@ -152,17 +173,209 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
     @Override
     public void close() {
-        for(ActorSelection remoteTransaction : remoteTransactionPaths.values()) {
-            remoteTransaction.tell(new CloseTransaction(), null);
+        for(TransactionContext transactionContext : remoteTransactionPaths.values()) {
+            transactionContext.closeTransaction();
         }
     }
 
-    private ActorSelection remoteTransactionFromIdentifier(InstanceIdentifier path){
+    private TransactionContext transactionContext(YangInstanceIdentifier path){
         String shardName = shardNameFromIdentifier(path);
         return remoteTransactionPaths.get(shardName);
     }
 
-    private String shardNameFromIdentifier(InstanceIdentifier path){
-        return Shard.DEFAULT_NAME;
+    private String shardNameFromIdentifier(YangInstanceIdentifier path){
+        return ShardStrategyFactory.getStrategy(path).findShard(path);
+    }
+
+    private void createTransactionIfMissing(ActorContext actorContext, YangInstanceIdentifier path) {
+        String shardName = ShardStrategyFactory.getStrategy(path).findShard(path);
+
+        TransactionContext transactionContext =
+            remoteTransactionPaths.get(shardName);
+
+        if(transactionContext != null){
+            // A transaction already exists with that shard
+            return;
+        }
+
+        try {
+            Object response = actorContext.executeShardOperation(shardName,
+                new CreateTransaction(identifier.toString(),this.transactionType.ordinal() ).toSerializable(),
+                ActorContext.ASK_DURATION);
+            if (response.getClass()
+                .equals(CreateTransactionReply.SERIALIZABLE_CLASS)) {
+                CreateTransactionReply reply =
+                    CreateTransactionReply.fromSerializable(response);
+
+                String transactionPath = reply.getTransactionPath();
+
+                LOG.debug("txn {} Received transaction path = {}", identifier, transactionPath);
+
+                ActorSelection transactionActor =
+                    actorContext.actorSelection(transactionPath);
+                transactionContext =
+                    new TransactionContextImpl(shardName, transactionPath,
+                        transactionActor);
+
+                remoteTransactionPaths.put(shardName, transactionContext);
+            }
+        } catch(TimeoutException | PrimaryNotFoundException e){
+            LOG.error("txn {} Creating NoOpTransaction because of : {}", identifier, e.getMessage());
+            remoteTransactionPaths.put(shardName,
+                new NoOpTransactionContext(shardName));
+        }
+    }
+
+    private interface TransactionContext {
+        String getShardName();
+
+        String getResolvedCohortPath(String cohortPath);
+
+        public void closeTransaction();
+
+        public Object readyTransaction();
+
+        void deleteData(YangInstanceIdentifier path);
+
+        void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
+
+        CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
+                final YangInstanceIdentifier path);
+
+        void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
     }
+
+
+    private class TransactionContextImpl implements TransactionContext{
+        private final String shardName;
+        private final String actorPath;
+        private final ActorSelection  actor;
+
+
+        private TransactionContextImpl(String shardName, String actorPath,
+            ActorSelection actor) {
+            this.shardName = shardName;
+            this.actorPath = actorPath;
+            this.actor = actor;
+        }
+
+        @Override public String getShardName() {
+            return shardName;
+        }
+
+        private ActorSelection getActor() {
+            return actor;
+        }
+
+        @Override public String getResolvedCohortPath(String cohortPath){
+            return actorContext.resolvePath(actorPath, cohortPath);
+        }
+
+        @Override public void closeTransaction() {
+            getActor().tell(
+                new CloseTransaction().toSerializable(), null);
+        }
+
+        @Override public Object readyTransaction() {
+            return actorContext.executeRemoteOperation(getActor(),
+                new ReadyTransaction().toSerializable(),
+                ActorContext.ASK_DURATION
+            );
+
+        }
+
+        @Override public void deleteData(YangInstanceIdentifier path) {
+            getActor().tell(new DeleteData(path).toSerializable(), null);
+        }
+
+        @Override public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data){
+            getActor().tell(new MergeData(path, data, schemaContext).toSerializable(), null);
+        }
+
+        @Override public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
+                final YangInstanceIdentifier path) {
+
+            Callable<Optional<NormalizedNode<?,?>>> call = new Callable<Optional<NormalizedNode<?,?>>>() {
+
+                @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();
+                        }
+                        return Optional.<NormalizedNode<?,?>>of(reply.getNormalizedNode());
+                    }
+
+                    return Optional.absent();
+                }
+            };
+
+            return MappingCheckedFuture.create(executor.submit(call), ReadFailedException.MAPPER);
+        }
+
+        @Override public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
+            getActor().tell(new WriteData(path, data, schemaContext).toSerializable(), null);
+        }
+
+    }
+
+    private class NoOpTransactionContext implements TransactionContext {
+
+        private final Logger
+            LOG = LoggerFactory.getLogger(NoOpTransactionContext.class);
+
+        private final String shardName;
+
+        private ActorRef cohort;
+
+        public NoOpTransactionContext(String shardName){
+            this.shardName = shardName;
+        }
+        @Override public String getShardName() {
+            return  shardName;
+
+        }
+
+        @Override public String getResolvedCohortPath(String cohortPath) {
+            return cohort.path().toString();
+        }
+
+        @Override public void closeTransaction() {
+            LOG.warn("txn {} closeTransaction called", identifier);
+        }
+
+        @Override public Object readyTransaction() {
+            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.warn("txt {} deleteData called path = {}", identifier, path);
+        }
+
+        @Override public void mergeData(YangInstanceIdentifier path,
+            NormalizedNode<?, ?> data) {
+            LOG.warn("txn {} mergeData called path = {}", identifier, path);
+        }
+
+        @Override
+        public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
+            YangInstanceIdentifier path) {
+            LOG.warn("txn {} readData called path = {}", identifier, path);
+            return Futures.immediateCheckedFuture(
+                Optional.<NormalizedNode<?, ?>>absent());
+        }
+
+        @Override public void writeData(YangInstanceIdentifier path,
+            NormalizedNode<?, ?> data) {
+            LOG.warn("txn {} writeData called path = {}", identifier, path);
+        }
+    }
+
+
+
 }