BUG 1735 Registering a data change listener should be asynchronous
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / TransactionProxy.java
index 2e7b2feb85bd97c7b80d09b8156f4b44c6fb343e..97a9ff0bf379ef3b8f6568ed37603ed285ba35a5 100644 (file)
@@ -13,6 +13,8 @@ import akka.actor.ActorSelection;
 import akka.dispatch.OnComplete;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.FinalizablePhantomReference;
+import com.google.common.base.FinalizableReferenceQueue;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
@@ -52,6 +54,8 @@ import scala.runtime.AbstractFunction1;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 
 /**
@@ -67,6 +71,11 @@ import java.util.concurrent.atomic.AtomicLong;
  * </p>
  */
 public class TransactionProxy implements DOMStoreReadWriteTransaction {
+
+    private final TransactionChainProxy transactionChainProxy;
+
+
+
     public enum TransactionType {
         READ_ONLY,
         WRITE_ONLY,
@@ -87,18 +96,113 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         LOG = LoggerFactory.getLogger(TransactionProxy.class);
 
 
+    /**
+     * Used to enqueue the PhantomReferences for read-only TransactionProxy instances. The
+     * FinalizableReferenceQueue is safe to use statically in an OSGi environment as it uses some
+     * trickery to clean up its internal thread when the bundle is unloaded.
+     */
+    private static final FinalizableReferenceQueue phantomReferenceQueue =
+                                                                  new FinalizableReferenceQueue();
+
+    /**
+     * This stores the TransactionProxyCleanupPhantomReference instances statically, This is
+     * necessary because PhantomReferences need a hard reference so they're not garbage collected.
+     * Once finalized, the TransactionProxyCleanupPhantomReference removes itself from this map
+     * and thus becomes eligible for garbage collection.
+     */
+    private static final Map<TransactionProxyCleanupPhantomReference,
+                             TransactionProxyCleanupPhantomReference> phantomReferenceCache =
+                                                                        new ConcurrentHashMap<>();
+
+    /**
+     * A PhantomReference that closes remote transactions for a TransactionProxy when it's
+     * garbage collected. This is used for read-only transactions as they're not explicitly closed
+     * by clients. So the only way to detect that a transaction is no longer in use and it's safe
+     * to clean up is when it's garbage collected. It's inexact as to when an instance will be GC'ed
+     * but TransactionProxy instances should generally be short-lived enough to avoid being moved
+     * to the old generation space and thus should be cleaned up in a timely manner as the GC
+     * runs on the young generation (eden, swap1...) space much more frequently.
+     */
+    private static class TransactionProxyCleanupPhantomReference
+                                           extends FinalizablePhantomReference<TransactionProxy> {
+
+        private final List<ActorSelection> remoteTransactionActors;
+        private final AtomicBoolean remoteTransactionActorsMB;
+        private final ActorContext actorContext;
+        private final TransactionIdentifier identifier;
+
+        protected TransactionProxyCleanupPhantomReference(TransactionProxy referent) {
+            super(referent, phantomReferenceQueue);
+
+            // Note we need to cache the relevant fields from the TransactionProxy as we can't
+            // have a hard reference to the TransactionProxy instance itself.
+
+            remoteTransactionActors = referent.remoteTransactionActors;
+            remoteTransactionActorsMB = referent.remoteTransactionActorsMB;
+            actorContext = referent.actorContext;
+            identifier = referent.identifier;
+        }
+
+        @Override
+        public void finalizeReferent() {
+            LOG.trace("Cleaning up {} Tx actors for TransactionProxy {}",
+                    remoteTransactionActors.size(), identifier);
+
+            phantomReferenceCache.remove(this);
+
+            // Access the memory barrier volatile to ensure all previous updates to the
+            // remoteTransactionActors list are visible to this thread.
+
+            if(remoteTransactionActorsMB.get()) {
+                for(ActorSelection actor : remoteTransactionActors) {
+                    LOG.trace("Sending CloseTransaction to {}", actor);
+                    actorContext.sendRemoteOperationAsync(actor,
+                            new CloseTransaction().toSerializable());
+                }
+            }
+        }
+    }
+
+    /**
+     * Stores the remote Tx actors for each requested data store path to be used by the
+     * PhantomReference to close the remote Tx's. This is only used for read-only Tx's. The
+     * remoteTransactionActorsMB volatile serves as a memory barrier to publish updates to the
+     * remoteTransactionActors list so they will be visible to the thread accessing the
+     * PhantomReference.
+     */
+    private List<ActorSelection> remoteTransactionActors;
+    private AtomicBoolean remoteTransactionActorsMB;
+
+    private final Map<String, TransactionContext> remoteTransactionPaths = new HashMap<>();
+
     private final TransactionType transactionType;
     private final ActorContext actorContext;
-    private final Map<String, TransactionContext> remoteTransactionPaths = new HashMap<>();
     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) {
+        this(actorContext, transactionType, null);
+    }
+
+    @VisibleForTesting
+    List<Future<Object>> getRecordedOperationFutures() {
+        List<Future<Object>> recordedOperationFutures = Lists.newArrayList();
+        for(TransactionContext transactionContext : remoteTransactionPaths.values()) {
+            recordedOperationFutures.addAll(transactionContext.getRecordedOperationFutures());
+        }
+
+        return recordedOperationFutures;
+    }
+
+    public TransactionProxy(ActorContext actorContext, TransactionType transactionType, TransactionChainProxy transactionChainProxy) {
+        this.actorContext = Preconditions.checkNotNull(actorContext,
+            "actorContext should not be null");
+        this.transactionType = Preconditions.checkNotNull(transactionType,
+            "transactionType should not be null");
+        this.schemaContext = Preconditions.checkNotNull(actorContext.getSchemaContext(),
+            "schemaContext should not be null");
+        this.transactionChainProxy = transactionChainProxy;
 
         String memberName = actorContext.getCurrentMemberName();
         if(memberName == null){
@@ -106,20 +210,22 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         }
 
         this.identifier = TransactionIdentifier.builder().memberName(memberName).counter(
-                counter.getAndIncrement()).build();
+            counter.getAndIncrement()).build();
 
-        LOG.debug("Created txn {} of type {}", identifier, transactionType);
+        if(transactionType == TransactionType.READ_ONLY) {
+            // Read-only Tx's aren't explicitly closed by the client so we create a PhantomReference
+            // to close the remote Tx's when this instance is no longer in use and is garbage
+            // collected.
 
-    }
+            remoteTransactionActors = Lists.newArrayList();
+            remoteTransactionActorsMB = new AtomicBoolean();
 
-    @VisibleForTesting
-    List<Future<Object>> getRecordedOperationFutures() {
-        List<Future<Object>> recordedOperationFutures = Lists.newArrayList();
-        for(TransactionContext transactionContext : remoteTransactionPaths.values()) {
-            recordedOperationFutures.addAll(transactionContext.getRecordedOperationFutures());
+            TransactionProxyCleanupPhantomReference cleanup =
+                new TransactionProxyCleanupPhantomReference(this);
+            phantomReferenceCache.put(cleanup, cleanup);
         }
 
-        return recordedOperationFutures;
+        LOG.debug("Created txn {} of type {}", identifier, transactionType);
     }
 
     @Override
@@ -153,7 +259,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         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");
+                "Transaction is sealed - further modifications are not allowed");
     }
 
     @Override
@@ -212,6 +318,10 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             cohortPathFutures.add(transactionContext.readyTransaction());
         }
 
+        if(transactionChainProxy != null){
+            transactionChainProxy.onTransactionReady(cohortPathFutures);
+        }
+
         return new ThreePhaseCommitCohortProxy(actorContext, cohortPathFutures,
                 identifier.toString());
     }
@@ -226,6 +336,13 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         for(TransactionContext transactionContext : remoteTransactionPaths.values()) {
             transactionContext.closeTransaction();
         }
+
+        remoteTransactionPaths.clear();
+
+        if(transactionType == TransactionType.READ_ONLY) {
+            remoteTransactionActors.clear();
+            remoteTransactionActorsMB.set(true);
+        }
     }
 
     private TransactionContext transactionContext(YangInstanceIdentifier path){
@@ -237,21 +354,27 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         return ShardStrategyFactory.getStrategy(path).findShard(path);
     }
 
-    private void createTransactionIfMissing(ActorContext actorContext, YangInstanceIdentifier path) {
+    private void createTransactionIfMissing(ActorContext actorContext,
+        YangInstanceIdentifier path) {
+
+        if(transactionChainProxy != null){
+            transactionChainProxy.waitTillCurrentTransactionReady();
+        }
+
         String shardName = ShardStrategyFactory.getStrategy(path).findShard(path);
 
         TransactionContext transactionContext =
             remoteTransactionPaths.get(shardName);
 
-        if(transactionContext != null){
+        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);
+                new CreateTransaction(identifier.toString(), this.transactionType.ordinal(),
+                    getTransactionChainId()).toSerializable());
             if (response.getClass().equals(CreateTransactionReply.SERIALIZABLE_CLASS)) {
                 CreateTransactionReply reply =
                     CreateTransactionReply.fromSerializable(response);
@@ -260,23 +383,41 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
                 LOG.debug("Tx {} Received transaction path = {}", identifier, transactionPath);
 
-                ActorSelection transactionActor =
-                    actorContext.actorSelection(transactionPath);
-                transactionContext =
-                    new TransactionContextImpl(shardName, transactionPath,
-                        transactionActor);
+                ActorSelection transactionActor = actorContext.actorSelection(transactionPath);
+
+                if (transactionType == TransactionType.READ_ONLY) {
+                    // Add the actor to the remoteTransactionActors list for access by the
+                    // cleanup PhantonReference.
+                    remoteTransactionActors.add(transactionActor);
+
+                    // Write to the memory barrier volatile to publish the above update to the
+                    // remoteTransactionActors list for thread visibility.
+                    remoteTransactionActorsMB.set(true);
+                }
+
+                transactionContext = new TransactionContextImpl(shardName, transactionPath,
+                    transactionActor, identifier, actorContext, schemaContext);
 
                 remoteTransactionPaths.put(shardName, transactionContext);
             } else {
                 throw new IllegalArgumentException(String.format(
-                        "Invalid reply type {} for CreateTransaction", response.getClass()));
+                    "Invalid reply type {} for CreateTransaction", response.getClass()));
             }
-        } catch(Exception e){
+        } catch (Exception e) {
             LOG.debug("Tx {} Creating NoOpTransaction because of : {}", identifier, e.getMessage());
-            remoteTransactionPaths.put(shardName, new NoOpTransactionContext(shardName, e));
+            remoteTransactionPaths
+                .put(shardName, new NoOpTransactionContext(shardName, e, identifier));
         }
     }
 
+    public String getTransactionChainId() {
+        if(transactionChainProxy == null){
+            return "";
+        }
+        return transactionChainProxy.getTransactionChainId();
+    }
+
+
     private interface TransactionContext {
         String getShardName();
 
@@ -298,13 +439,15 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         List<Future<Object>> getRecordedOperationFutures();
     }
 
-    private abstract class AbstractTransactionContext implements TransactionContext {
+    private static abstract class AbstractTransactionContext implements TransactionContext {
 
+        protected final TransactionIdentifier identifier;
         protected final String shardName;
         protected final List<Future<Object>> recordedOperationFutures = Lists.newArrayList();
 
-        AbstractTransactionContext(String shardName) {
+        AbstractTransactionContext(String shardName, TransactionIdentifier identifier) {
             this.shardName = shardName;
+            this.identifier = identifier;
         }
 
         @Override
@@ -318,17 +461,22 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         }
     }
 
-    private class TransactionContextImpl extends AbstractTransactionContext {
+    private static class TransactionContextImpl extends AbstractTransactionContext {
         private final Logger LOG = LoggerFactory.getLogger(TransactionContextImpl.class);
 
+        private final ActorContext actorContext;
+        private final SchemaContext schemaContext;
         private final String actorPath;
         private final ActorSelection actor;
 
         private TransactionContextImpl(String shardName, String actorPath,
-            ActorSelection actor) {
-            super(shardName);
+                ActorSelection actor, TransactionIdentifier identifier, ActorContext actorContext,
+                SchemaContext schemaContext) {
+            super(shardName, identifier);
             this.actorPath = actorPath;
             this.actor = actor;
+            this.actorContext = actorContext;
+            this.schemaContext = schemaContext;
         }
 
         private ActorSelection getActor() {
@@ -353,7 +501,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             // Send the ReadyTransaction message to the Tx actor.
 
             final Future<Object> replyFuture = actorContext.executeRemoteOperationAsync(getActor(),
-                    new ReadyTransaction().toSerializable(), ActorContext.ASK_DURATION);
+                    new ReadyTransaction().toSerializable());
 
             // 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
@@ -413,23 +561,21 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
         public void deleteData(YangInstanceIdentifier path) {
             LOG.debug("Tx {} deleteData called path = {}", identifier, path);
             recordedOperationFutures.add(actorContext.executeRemoteOperationAsync(getActor(),
-                    new DeleteData(path).toSerializable(), ActorContext.ASK_DURATION ));
+                    new DeleteData(path).toSerializable() ));
         }
 
         @Override
         public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
             LOG.debug("Tx {} mergeData called path = {}", identifier, path);
             recordedOperationFutures.add(actorContext.executeRemoteOperationAsync(getActor(),
-                    new MergeData(path, data, schemaContext).toSerializable(),
-                    ActorContext.ASK_DURATION));
+                    new MergeData(path, data, schemaContext).toSerializable()));
         }
 
         @Override
         public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
             LOG.debug("Tx {} writeData called path = {}", identifier, path);
             recordedOperationFutures.add(actorContext.executeRemoteOperationAsync(getActor(),
-                    new WriteData(path, data, schemaContext).toSerializable(),
-                    ActorContext.ASK_DURATION));
+                    new WriteData(path, data, schemaContext).toSerializable()));
         }
 
         @Override
@@ -493,6 +639,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
 
                         returnFuture.setException(new ReadFailedException(
                                 "Error reading data for path " + path, failure));
+
                     } else {
                         LOG.debug("Tx {} read operation succeeded", identifier, failure);
 
@@ -514,7 +661,7 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             };
 
             Future<Object> readFuture = actorContext.executeRemoteOperationAsync(getActor(),
-                    new ReadData(path).toSerializable(), ActorContext.ASK_DURATION);
+                    new ReadData(path).toSerializable());
             readFuture.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
         }
 
@@ -595,19 +742,20 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
             };
 
             Future<Object> future = actorContext.executeRemoteOperationAsync(getActor(),
-                    new DataExists(path).toSerializable(), ActorContext.ASK_DURATION);
+                    new DataExists(path).toSerializable());
             future.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
         }
     }
 
-    private class NoOpTransactionContext extends AbstractTransactionContext {
+    private static class NoOpTransactionContext extends AbstractTransactionContext {
 
         private final Logger LOG = LoggerFactory.getLogger(NoOpTransactionContext.class);
 
         private final Exception failure;
 
-        public NoOpTransactionContext(String shardName, Exception failure){
-            super(shardName);
+        public NoOpTransactionContext(String shardName, Exception failure,
+                TransactionIdentifier identifier){
+            super(shardName, identifier);
             this.failure = failure;
         }