Remove unused ShardCommitCoordinator#CohortEntry constructor
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / ShardCommitCoordinator.java
index 1b838ae0e6c6e3c32ae604846d32ae280e32bc78..0d63115754698688f5c64fba9d46d63b93336f87 100644 (file)
@@ -21,6 +21,7 @@ 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.AbortTransactionReply;
 import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
 import org.opendaylight.controller.cluster.datastore.messages.BatchedModificationsReply;
 import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransactionReply;
@@ -28,8 +29,8 @@ import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTran
 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.md.sal.common.api.data.TransactionCommitFailedException;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidate;
 import org.slf4j.Logger;
 
 /**
@@ -68,7 +69,7 @@ class ShardCommitCoordinator {
     private ReadyTransactionReply readyTransactionReply;
 
     ShardCommitCoordinator(ShardDataTree dataTree,
-            long cacheExpiryTimeoutInMillis, int queueCapacity, ActorRef shardActor, Logger log, String name) {
+            long cacheExpiryTimeoutInMillis, int queueCapacity, Logger log, String name) {
 
         this.queueCapacity = queueCapacity;
         this.log = log;
@@ -77,6 +78,10 @@ class ShardCommitCoordinator {
         this.cacheExpiryTimeoutInMillis = cacheExpiryTimeoutInMillis;
     }
 
+    int getQueueSize() {
+        return queuedCohortEntries.size();
+    }
+
     void setQueueCapacity(int queueCapacity) {
         this.queueCapacity = queueCapacity;
     }
@@ -92,6 +97,10 @@ class ShardCommitCoordinator {
     private boolean queueCohortEntry(CohortEntry cohortEntry, ActorRef sender, Shard shard) {
         if(queuedCohortEntries.size() < queueCapacity) {
             queuedCohortEntries.offer(cohortEntry);
+
+            log.debug("{}: Enqueued transaction {}, queue size {}", name, cohortEntry.getTransactionID(),
+                    queuedCohortEntries.size());
+
             return true;
         } else {
             cohortCache.remove(cohortEntry.getTransactionID());
@@ -118,8 +127,7 @@ class ShardCommitCoordinator {
         log.debug("{}: Readying transaction {}, client version {}", name,
                 ready.getTransactionID(), ready.getTxnClientVersion());
 
-        CohortEntry cohortEntry = new CohortEntry(ready.getTransactionID(), ready.getCohort(),
-                (MutableCompositeModification) ready.getModification());
+        CohortEntry cohortEntry = new CohortEntry(ready.getTransactionID(), ready.getCohort());
         cohortCache.put(ready.getTransactionID(), cohortEntry);
 
         if(!queueCohortEntry(cohortEntry, sender, shard)) {
@@ -184,6 +192,18 @@ class ShardCommitCoordinator {
         cohortEntry.applyModifications(batched.getModifications());
 
         if(batched.isReady()) {
+            if(cohortEntry.getLastBatchedModificationsException() != null) {
+                cohortCache.remove(cohortEntry.getTransactionID());
+                throw cohortEntry.getLastBatchedModificationsException();
+            }
+
+            if(cohortEntry.getTotalBatchedModificationsReceived() != batched.getTotalMessagesSent()) {
+                cohortCache.remove(cohortEntry.getTransactionID());
+                throw new IllegalStateException(String.format(
+                        "The total number of batched messages received %d does not match the number sent %d",
+                        cohortEntry.getTotalBatchedModificationsReceived(), batched.getTotalMessagesSent()));
+            }
+
             if(!queueCohortEntry(cohortEntry, sender, shard)) {
                 return;
             }
@@ -216,7 +236,8 @@ class ShardCommitCoordinator {
      * @param shard the transaction's shard actor
      */
     void handleReadyLocalTransaction(ReadyLocalTransaction message, ActorRef sender, Shard shard) {
-        final ShardDataTreeCohort cohort = new SimpleShardDataTreeCohort(dataTree, message.getModification());
+        final ShardDataTreeCohort cohort = new SimpleShardDataTreeCohort(dataTree, message.getModification(),
+                message.getTransactionID());
         final CohortEntry cohortEntry = new CohortEntry(message.getTransactionID(), cohort);
         cohortCache.put(message.getTransactionID(), cohortEntry);
         cohortEntry.setDoImmediateCommit(message.isDoCommitOnReady());
@@ -297,10 +318,7 @@ class ShardCommitCoordinator {
     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.
-            canCommit = cohortEntry.getCohort().canCommit().get();
+            canCommit = cohortEntry.canCommit();
 
             log.debug("{}: canCommit for {}: {}", name, cohortEntry.getTransactionID(), canCommit);
 
@@ -344,10 +362,7 @@ class ShardCommitCoordinator {
         // 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();
+            cohortEntry.preCommit();
 
             cohortEntry.getShard().continueCommit(cohortEntry);
 
@@ -392,6 +407,41 @@ class ShardCommitCoordinator {
         return doCommit(cohortEntry);
     }
 
+    void handleAbort(final String transactionID, final ActorRef sender, final Shard shard) {
+        CohortEntry cohortEntry = getCohortEntryIfCurrent(transactionID);
+        if(cohortEntry != null) {
+            // We don't remove the cached cohort entry here (ie pass false) in case the Tx was
+            // aborted during replication in which case we may still commit locally if replication
+            // succeeds.
+            currentTransactionComplete(transactionID, false);
+        } else {
+            cohortEntry = getAndRemoveCohortEntry(transactionID);
+        }
+
+        if(cohortEntry == null) {
+            return;
+        }
+
+        log.debug("{}: Aborting transaction {}", name, transactionID);
+
+        final ActorRef self = shard.getSelf();
+        try {
+            cohortEntry.abort();
+
+            shard.getShardMBean().incrementAbortTransactionsCount();
+
+            if(sender != null) {
+                sender.tell(new AbortTransactionReply().toSerializable(), self);
+            }
+        } catch (Exception e) {
+            log.error("{}: An exception happened during abort", name, e);
+
+            if(sender != null) {
+                sender.tell(new akka.actor.Status.Failure(e), self);
+            }
+        }
+    }
+
     /**
      * Returns the cohort entry for the Tx commit currently in progress if the given transaction ID
      * matches the current entry.
@@ -466,12 +516,12 @@ class ShardCommitCoordinator {
             } else if(next.isExpired(cacheExpiryTimeoutInMillis)) {
                 log.warn("{}: canCommit for transaction {} was not received within {} ms - entry removed from cache",
                         name, next.getTransactionID(), cacheExpiryTimeoutInMillis);
-
-                iter.remove();
-                cohortCache.remove(next.getTransactionID());
-            } else {
+            } else if(!next.isAborted()) {
                 break;
             }
+
+            iter.remove();
+            cohortCache.remove(next.getTransactionID());
         }
     }
 
@@ -488,23 +538,19 @@ class ShardCommitCoordinator {
         private final String transactionID;
         private ShardDataTreeCohort cohort;
         private final ReadWriteShardDataTreeTransaction transaction;
+        private RuntimeException lastBatchedModificationsException;
         private ActorRef replySender;
         private Shard shard;
         private boolean doImmediateCommit;
         private final Stopwatch lastAccessTimer = Stopwatch.createStarted();
+        private int totalBatchedModificationsReceived;
+        private boolean aborted;
 
         CohortEntry(String transactionID, ReadWriteShardDataTreeTransaction transaction) {
             this.transaction = Preconditions.checkNotNull(transaction);
             this.transactionID = transactionID;
         }
 
-        CohortEntry(String transactionID, ShardDataTreeCohort cohort,
-                MutableCompositeModification compositeModification) {
-            this.transactionID = transactionID;
-            this.cohort = cohort;
-            this.transaction = null;
-        }
-
         CohortEntry(String transactionID, ShardDataTreeCohort cohort) {
             this.transactionID = transactionID;
             this.cohort = cohort;
@@ -520,16 +566,54 @@ class ShardCommitCoordinator {
             return transactionID;
         }
 
-        ShardDataTreeCohort getCohort() {
-            return cohort;
+        DataTreeCandidate getCandidate() {
+            return cohort.getCandidate();
+        }
+
+        int getTotalBatchedModificationsReceived() {
+            return totalBatchedModificationsReceived;
+        }
+
+        RuntimeException getLastBatchedModificationsException() {
+            return lastBatchedModificationsException;
         }
 
         void applyModifications(Iterable<Modification> modifications) {
-            for (Modification modification : modifications) {
-                modification.apply(transaction.getSnapshot());
+            totalBatchedModificationsReceived++;
+            if(lastBatchedModificationsException == null) {
+                for (Modification modification : modifications) {
+                        try {
+                            modification.apply(transaction.getSnapshot());
+                        } catch (RuntimeException e) {
+                            lastBatchedModificationsException = e;
+                            throw e;
+                        }
+                }
             }
         }
 
+        boolean canCommit() throws InterruptedException, ExecutionException {
+            // We block on the future here (and also preCommit(), commit(), abort()) so we don't have to worry
+            // about possibly accessing our state on a different thread outside of our dispatcher.
+            // TODO: the ShardDataTreeCohort returns immediate Futures anyway which begs the question - why
+            // bother even returning Futures from ShardDataTreeCohort if we have to treat them synchronously
+            // anyway?. The Futures are really a remnant from when we were using the InMemoryDataBroker.
+            return cohort.canCommit().get();
+        }
+
+        void preCommit() throws InterruptedException, ExecutionException {
+            cohort.preCommit().get();
+        }
+
+        void commit() throws InterruptedException, ExecutionException {
+            cohort.commit().get();
+        }
+
+        void abort() throws InterruptedException, ExecutionException {
+            aborted = true;
+            cohort.abort().get();
+        }
+
         void ready(CohortDecorator cohortDecorator, boolean doImmediateCommit) {
             Preconditions.checkState(cohort == null, "cohort was already set");
 
@@ -575,6 +659,11 @@ class ShardCommitCoordinator {
             this.shard = shard;
         }
 
+
+        boolean isAborted() {
+            return aborted;
+        }
+
         @Override
         public String toString() {
             StringBuilder builder = new StringBuilder();