CDS: Implement front-end support for local transactions
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / ShardCommitCoordinator.java
index 54f15fcb4bd03115d97eccc72166b0b12efbca2f..97816a55ccb5920e289eed12d665de8ca6a2ce35 100644 (file)
@@ -21,14 +21,16 @@ import java.util.LinkedList;
 import java.util.Queue;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
+import org.opendaylight.controller.cluster.datastore.compat.BackwardsCompatibleThreePhaseCommitCohort;
 import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
 import org.opendaylight.controller.cluster.datastore.messages.BatchedModificationsReply;
-import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransactionReply;
+import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction;
+import org.opendaylight.controller.cluster.datastore.messages.ReadyLocalTransaction;
+import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
 import org.opendaylight.controller.cluster.datastore.modification.Modification;
 import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
-import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
-import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
+import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
 import org.slf4j.Logger;
 
 /**
@@ -40,14 +42,14 @@ public class ShardCommitCoordinator {
 
     // Interface hook for unit tests to replace or decorate the DOMStoreThreePhaseCommitCohorts.
     public interface CohortDecorator {
-        DOMStoreThreePhaseCommitCohort decorate(String transactionID, DOMStoreThreePhaseCommitCohort actual);
+        ShardDataTreeCohort decorate(String transactionID, ShardDataTreeCohort actual);
     }
 
     private final Cache<String, CohortEntry> cohortCache;
 
     private CohortEntry currentCohortEntry;
 
-    private final DOMTransactionFactory transactionFactory;
+    private final ShardDataTree dataTree;
 
     private final Queue<CohortEntry> queuedCohortEntries;
 
@@ -57,8 +59,6 @@ public class ShardCommitCoordinator {
 
     private final String name;
 
-    private final String shardActorPath;
-
     private final RemovalListener<String, CohortEntry> cacheRemovalListener =
             new RemovalListener<String, CohortEntry>() {
                 @Override
@@ -72,15 +72,15 @@ public class ShardCommitCoordinator {
     // This is a hook for unit tests to replace or decorate the DOMStoreThreePhaseCommitCohorts.
     private CohortDecorator cohortDecorator;
 
-    public ShardCommitCoordinator(DOMTransactionFactory transactionFactory,
+    private ReadyTransactionReply readyTransactionReply;
+
+    public ShardCommitCoordinator(ShardDataTree dataTree,
             long cacheExpiryTimeoutInSec, int queueCapacity, ActorRef shardActor, Logger log, String name) {
 
         this.queueCapacity = queueCapacity;
         this.log = log;
         this.name = name;
-        this.transactionFactory = transactionFactory;
-
-        shardActorPath = Serialization.serializedActorPath(shardActor);
+        this.dataTree = Preconditions.checkNotNull(dataTree);
 
         cohortCache = CacheBuilder.newBuilder().expireAfterAccess(cacheExpiryTimeoutInSec, TimeUnit.SECONDS).
                 removalListener(cacheRemovalListener).build();
@@ -94,18 +94,55 @@ public class ShardCommitCoordinator {
         this.queueCapacity = queueCapacity;
     }
 
+    private ReadyTransactionReply readyTransactionReply(Shard shard) {
+        if(readyTransactionReply == null) {
+            readyTransactionReply = new ReadyTransactionReply(Serialization.serializedActorPath(shard.self()));
+        }
+
+        return readyTransactionReply;
+    }
+
     /**
      * This method is called to ready a transaction that was prepared by ShardTransaction actor. It caches
      * the prepared cohort entry for the given transactions ID in preparation for the subsequent 3-phase commit.
-     *
-     * @param transactionID the ID of the transaction
-     * @param cohort the cohort to participate in the transaction commit
-     * @param modification the modifications made by the transaction
      */
-    public void transactionReady(String transactionID, DOMStoreThreePhaseCommitCohort cohort,
-            MutableCompositeModification modification) {
+    public void handleForwardedReadyTransaction(ForwardedReadyTransaction ready, ActorRef sender, Shard shard) {
+        log.debug("{}: Readying transaction {}, client version {}", name,
+                ready.getTransactionID(), ready.getTxnClientVersion());
+
+        CohortEntry cohortEntry = new CohortEntry(ready.getTransactionID(), ready.getCohort(),
+                (MutableCompositeModification) ready.getModification());
+        cohortCache.put(ready.getTransactionID(), cohortEntry);
+
+        if(ready.getTxnClientVersion() < DataStoreVersions.LITHIUM_VERSION) {
+            // Return our actor path as we'll handle the three phase commit except if the Tx client
+            // version < Helium-1 version which means the Tx was initiated by a base Helium version node.
+            // In that case, the subsequent 3-phase commit messages won't contain the transactionId so to
+            // maintain backwards compatibility, we create a separate cohort actor to provide the compatible behavior.
+            ActorRef replyActorPath = shard.self();
+            if(ready.getTxnClientVersion() < DataStoreVersions.HELIUM_1_VERSION) {
+                log.debug("{}: Creating BackwardsCompatibleThreePhaseCommitCohort", name);
+                replyActorPath = shard.getContext().actorOf(BackwardsCompatibleThreePhaseCommitCohort.props(
+                        ready.getTransactionID()));
+            }
 
-        cohortCache.put(transactionID, new CohortEntry(transactionID, cohort, modification));
+            ReadyTransactionReply readyTransactionReply =
+                    new ReadyTransactionReply(Serialization.serializedActorPath(replyActorPath),
+                            ready.getTxnClientVersion());
+            sender.tell(ready.isReturnSerialized() ? readyTransactionReply.toSerializable() :
+                readyTransactionReply, shard.self());
+        } else {
+            if(ready.isDoImmediateCommit()) {
+                cohortEntry.setDoImmediateCommit(true);
+                cohortEntry.setReplySender(sender);
+                cohortEntry.setShard(shard);
+                handleCanCommit(cohortEntry);
+            } else {
+                // The caller does not want immediate commit - the 3-phase commit will be coordinated by the
+                // front-end so send back a ReadyTransactionReply with our actor path.
+                sender.tell(readyTransactionReply(shard), shard.self());
+            }
+        }
     }
 
     /**
@@ -119,13 +156,12 @@ public class ShardCommitCoordinator {
      *
      * @throws ExecutionException if an error occurs loading the cache
      */
-    public BatchedModificationsReply handleTransactionModifications(BatchedModifications batched)
+    boolean handleBatchedModifications(BatchedModifications batched, ActorRef sender, Shard shard)
             throws ExecutionException {
         CohortEntry cohortEntry = cohortCache.getIfPresent(batched.getTransactionID());
         if(cohortEntry == null) {
             cohortEntry = new CohortEntry(batched.getTransactionID(),
-                    transactionFactory.<DOMStoreWriteTransaction>newTransaction(
-                        TransactionProxy.TransactionType.WRITE_ONLY, batched.getTransactionID(),
+                    dataTree.newReadWriteTransaction(batched.getTransactionID(),
                         batched.getTransactionChainID()));
             cohortCache.put(batched.getTransactionID(), cohortEntry);
         }
@@ -137,50 +173,59 @@ public class ShardCommitCoordinator {
 
         cohortEntry.applyModifications(batched.getModifications());
 
-        String cohortPath = null;
         if(batched.isReady()) {
             if(log.isDebugEnabled()) {
                 log.debug("{}: Readying Tx {}, client version {}", name,
                         batched.getTransactionID(), batched.getVersion());
             }
 
-            cohortEntry.ready(cohortDecorator);
-            cohortPath = shardActorPath;
+            cohortEntry.ready(cohortDecorator, batched.isDoCommitOnReady());
+
+            if(batched.isDoCommitOnReady()) {
+                cohortEntry.setReplySender(sender);
+                cohortEntry.setShard(shard);
+                handleCanCommit(cohortEntry);
+            } else {
+                sender.tell(readyTransactionReply(shard), shard.self());
+            }
+        } else {
+            sender.tell(new BatchedModificationsReply(batched.getModifications().size()), shard.self());
         }
 
-        return new BatchedModificationsReply(batched.getModifications().size(), cohortPath);
+        return batched.isReady();
     }
 
     /**
-     * This method handles the canCommit phase for a transaction.
+     * This method handles {@link ReadyLocalTransaction} message. All transaction modifications have
+     * been prepared beforehand by the sender and we just need to drive them through into the dataTree.
      *
-     * @param canCommit the CanCommitTransaction message
-     * @param sender the actor that sent the message
-     * @param shard the transaction's shard actor
+     * @param message
+     * @param sender
+     * @param shard
      */
-    public void handleCanCommit(CanCommitTransaction canCommit, final ActorRef sender,
-            final ActorRef shard) {
-        String transactionID = canCommit.getTransactionID();
-        if(log.isDebugEnabled()) {
-            log.debug("{}: Processing canCommit for transaction {} for shard {}",
-                    name, transactionID, shard.path());
+    void handleReadyLocalTransaction(ReadyLocalTransaction message, ActorRef sender, Shard shard) {
+        final ShardDataTreeCohort cohort = new SimpleShardDataTreeCohort(dataTree, message.getModification());
+        final CohortEntry cohortEntry = new CohortEntry(message.getTransactionID(), cohort);
+        cohortCache.put(message.getTransactionID(), cohortEntry);
+        cohortEntry.setDoImmediateCommit(message.isDoCommitOnReady());
+        log.debug("{}: Applying local modifications for Tx {}", name, message.getTransactionID());
+
+        if (message.isDoCommitOnReady()) {
+            cohortEntry.setReplySender(sender);
+            cohortEntry.setShard(shard);
+            handleCanCommit(cohortEntry);
+        } else {
+            sender.tell(readyTransactionReply(shard), shard.self());
         }
+    }
 
-        // Lookup the cohort entry that was cached previously (or should have been) by
-        // transactionReady (via the ForwardedReadyTransaction message).
-        final CohortEntry cohortEntry = cohortCache.getIfPresent(transactionID);
-        if(cohortEntry == null) {
-            // Either canCommit was invoked before ready(shouldn't happen)  or a long time passed
-            // between canCommit and ready and the entry was expired from the cache.
-            IllegalStateException ex = new IllegalStateException(
-                    String.format("%s: No cohort entry found for transaction %s", name, transactionID));
-            log.error(ex.getMessage());
-            sender.tell(new Status.Failure(ex), shard);
-            return;
-        }
+    private void handleCanCommit(CohortEntry cohortEntry) {
+        String transactionID = cohortEntry.getTransactionID();
 
-        cohortEntry.setCanCommitSender(sender);
-        cohortEntry.setShard(shard);
+        if(log.isDebugEnabled()) {
+            log.debug("{}: Processing canCommit for transaction {} for shard {}",
+                    name, transactionID, cohortEntry.getShard().self().path());
+        }
 
         if(currentCohortEntry != null) {
             // There's already a Tx commit in progress - attempt to queue this entry to be
@@ -198,7 +243,7 @@ public class ShardCommitCoordinator {
                                       " capacity %d has been reached.",
                                       name, transactionID, queueCapacity));
                 log.error(ex.getMessage());
-                sender.tell(new Status.Failure(ex), shard);
+                cohortEntry.getReplySender().tell(new Status.Failure(ex), cohortEntry.getShard().self());
             }
         } else {
             // No Tx commit currently in progress - make this the current entry and proceed with
@@ -210,29 +255,119 @@ public class ShardCommitCoordinator {
         }
     }
 
+    /**
+     * This method handles the canCommit phase for a transaction.
+     *
+     * @param canCommit the CanCommitTransaction message
+     * @param sender the actor that sent the message
+     * @param shard the transaction's shard actor
+     */
+    public void handleCanCommit(String transactionID, final ActorRef sender, final Shard shard) {
+        // Lookup the cohort entry that was cached previously (or should have been) by
+        // transactionReady (via the ForwardedReadyTransaction message).
+        final CohortEntry cohortEntry = cohortCache.getIfPresent(transactionID);
+        if(cohortEntry == null) {
+            // Either canCommit was invoked before ready(shouldn't happen)  or a long time passed
+            // between canCommit and ready and the entry was expired from the cache.
+            IllegalStateException ex = new IllegalStateException(
+                    String.format("%s: No cohort entry found for transaction %s", name, transactionID));
+            log.error(ex.getMessage());
+            sender.tell(new Status.Failure(ex), shard.self());
+            return;
+        }
+
+        cohortEntry.setReplySender(sender);
+        cohortEntry.setShard(shard);
+
+        handleCanCommit(cohortEntry);
+    }
+
     private void doCanCommit(final CohortEntry cohortEntry) {
 
+        boolean canCommit = false;
         try {
             // We block on the future here so we don't have to worry about possibly accessing our
             // state on a different thread outside of our dispatcher. Also, the data store
             // currently uses a same thread executor anyway.
-            Boolean canCommit = cohortEntry.getCohort().canCommit().get();
+            canCommit = cohortEntry.getCohort().canCommit().get();
+
+            if(cohortEntry.isDoImmediateCommit()) {
+                if(canCommit) {
+                    doCommit(cohortEntry);
+                } else {
+                    cohortEntry.getReplySender().tell(new Status.Failure(new TransactionCommitFailedException(
+                                "Can Commit failed, no detailed cause available.")), cohortEntry.getShard().self());
+                }
+            } else {
+                cohortEntry.getReplySender().tell(
+                        canCommit ? CanCommitTransactionReply.YES.toSerializable() :
+                            CanCommitTransactionReply.NO.toSerializable(), cohortEntry.getShard().self());
+            }
+        } catch (Exception e) {
+            log.debug("{}: An exception occurred during canCommit: {}", name, e);
 
-            cohortEntry.getCanCommitSender().tell(
-                    canCommit ? CanCommitTransactionReply.YES.toSerializable() :
-                        CanCommitTransactionReply.NO.toSerializable(), cohortEntry.getShard());
+            Throwable failure = e;
+            if(e instanceof ExecutionException) {
+                failure = e.getCause();
+            }
 
+            cohortEntry.getReplySender().tell(new Status.Failure(failure), cohortEntry.getShard().self());
+        } finally {
             if(!canCommit) {
-                // Remove the entry from the cache now since the Tx will be aborted.
-                removeCohortEntry(cohortEntry.getTransactionID());
+                // Remove the entry from the cache now.
+                currentTransactionComplete(cohortEntry.getTransactionID(), true);
             }
-        } catch (InterruptedException | ExecutionException e) {
-            log.debug("{}: An exception occurred during canCommit: {}", name, e);
+        }
+    }
+
+    private boolean doCommit(CohortEntry cohortEntry) {
+        log.debug("{}: Committing transaction {}", name, cohortEntry.getTransactionID());
+
+        boolean success = false;
+
+        // We perform the preCommit phase here atomically with the commit phase. This is an
+        // optimization to eliminate the overhead of an extra preCommit message. We lose front-end
+        // coordination of preCommit across shards in case of failure but preCommit should not
+        // normally fail since we ensure only one concurrent 3-phase commit.
+
+        try {
+            // We block on the future here so we don't have to worry about possibly accessing our
+            // state on a different thread outside of our dispatcher. Also, the data store
+            // currently uses a same thread executor anyway.
+            cohortEntry.getCohort().preCommit().get();
 
-            // Remove the entry from the cache now since the Tx will be aborted.
-            removeCohortEntry(cohortEntry.getTransactionID());
-            cohortEntry.getCanCommitSender().tell(new Status.Failure(e), cohortEntry.getShard());
+            cohortEntry.getShard().continueCommit(cohortEntry);
+
+            cohortEntry.updateLastAccessTime();
+
+            success = true;
+        } catch (Exception e) {
+            log.error("{} An exception occurred while preCommitting transaction {}",
+                    name, cohortEntry.getTransactionID(), e);
+            cohortEntry.getReplySender().tell(new akka.actor.Status.Failure(e), cohortEntry.getShard().self());
+
+            currentTransactionComplete(cohortEntry.getTransactionID(), true);
         }
+
+        return success;
+    }
+
+    boolean handleCommit(final String transactionID, final ActorRef sender, final Shard shard) {
+        // Get the current in-progress cohort entry in the commitCoordinator if it corresponds to
+        // this transaction.
+        final CohortEntry cohortEntry = getCohortEntryIfCurrent(transactionID);
+        if(cohortEntry == null) {
+            // We're not the current Tx - the Tx was likely expired b/c it took too long in
+            // between the canCommit and commit messages.
+            IllegalStateException ex = new IllegalStateException(
+                    String.format("%s: Cannot commit transaction %s - it is not the current transaction",
+                            name, transactionID));
+            log.error(ex.getMessage());
+            sender.tell(new akka.actor.Status.Failure(ex), shard.self());
+            return false;
+        }
+
+        return doCommit(cohortEntry);
     }
 
     /**
@@ -302,24 +437,28 @@ public class ShardCommitCoordinator {
 
     static class CohortEntry {
         private final String transactionID;
-        private DOMStoreThreePhaseCommitCohort cohort;
-        private final MutableCompositeModification compositeModification;
-        private final DOMStoreWriteTransaction transaction;
-        private ActorRef canCommitSender;
-        private ActorRef shard;
+        private ShardDataTreeCohort cohort;
+        private final ReadWriteShardDataTreeTransaction transaction;
+        private ActorRef replySender;
+        private Shard shard;
         private long lastAccessTime;
+        private boolean doImmediateCommit;
 
-        CohortEntry(String transactionID, DOMStoreWriteTransaction transaction) {
-            this.compositeModification = new MutableCompositeModification();
-            this.transaction = transaction;
+        CohortEntry(String transactionID, ReadWriteShardDataTreeTransaction transaction) {
+            this.transaction = Preconditions.checkNotNull(transaction);
             this.transactionID = transactionID;
         }
 
-        CohortEntry(String transactionID, DOMStoreThreePhaseCommitCohort cohort,
+        CohortEntry(String transactionID, ShardDataTreeCohort cohort,
                 MutableCompositeModification compositeModification) {
             this.transactionID = transactionID;
             this.cohort = cohort;
-            this.compositeModification = compositeModification;
+            this.transaction = null;
+        }
+
+        CohortEntry(String transactionID, ShardDataTreeCohort cohort) {
+            this.transactionID = transactionID;
+            this.cohort = cohort;
             this.transaction = null;
         }
 
@@ -335,24 +474,21 @@ public class ShardCommitCoordinator {
             return transactionID;
         }
 
-        DOMStoreThreePhaseCommitCohort getCohort() {
+        ShardDataTreeCohort getCohort() {
             return cohort;
         }
 
-        MutableCompositeModification getModification() {
-            return compositeModification;
-        }
-
         void applyModifications(Iterable<Modification> modifications) {
-            for(Modification modification: modifications) {
-                compositeModification.addModification(modification);
-                modification.apply(transaction);
+            for (Modification modification : modifications) {
+                modification.apply(transaction.getSnapshot());
             }
         }
 
-        void ready(CohortDecorator cohortDecorator) {
+        void ready(CohortDecorator cohortDecorator, boolean doImmediateCommit) {
             Preconditions.checkState(cohort == null, "cohort was already set");
 
+            setDoImmediateCommit(doImmediateCommit);
+
             cohort = transaction.ready();
 
             if(cohortDecorator != null) {
@@ -361,24 +497,28 @@ public class ShardCommitCoordinator {
             }
         }
 
-        ActorRef getCanCommitSender() {
-            return canCommitSender;
+        boolean isDoImmediateCommit() {
+            return doImmediateCommit;
         }
 
-        void setCanCommitSender(ActorRef canCommitSender) {
-            this.canCommitSender = canCommitSender;
+        void setDoImmediateCommit(boolean doImmediateCommit) {
+            this.doImmediateCommit = doImmediateCommit;
         }
 
-        ActorRef getShard() {
-            return shard;
+        ActorRef getReplySender() {
+            return replySender;
         }
 
-        void setShard(ActorRef shard) {
-            this.shard = shard;
+        void setReplySender(ActorRef replySender) {
+            this.replySender = replySender;
+        }
+
+        Shard getShard() {
+            return shard;
         }
 
-        boolean hasModifications(){
-            return compositeModification.getModifications().size() > 0;
+        void setShard(Shard shard) {
+            this.shard = shard;
         }
     }
 }