Bug 3206: CDS - issues with direct commit
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / ShardCommitCoordinator.java
index 97816a55ccb5920e289eed12d665de8ca6a2ce35..f3e1e33e347f7760820bce49897bf2aa3c20a5b6 100644 (file)
@@ -12,12 +12,11 @@ import akka.actor.Status;
 import akka.serialization.Serialization;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.RemovalCause;
-import com.google.common.cache.RemovalListener;
-import com.google.common.cache.RemovalNotification;
+import com.google.common.base.Stopwatch;
+import java.util.HashMap;
+import java.util.Iterator;
 import java.util.LinkedList;
+import java.util.Map;
 import java.util.Queue;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
@@ -45,13 +44,15 @@ public class ShardCommitCoordinator {
         ShardDataTreeCohort decorate(String transactionID, ShardDataTreeCohort actual);
     }
 
-    private final Cache<String, CohortEntry> cohortCache;
+    private final Map<String, CohortEntry> cohortCache = new HashMap<>();
 
     private CohortEntry currentCohortEntry;
 
     private final ShardDataTree dataTree;
 
-    private final Queue<CohortEntry> queuedCohortEntries;
+    // We use a LinkedList here to avoid synchronization overhead with concurrent queue impls
+    // since this should only be accessed on the shard's dispatcher.
+    private final Queue<CohortEntry> queuedCohortEntries = new LinkedList<>();
 
     private int queueCapacity;
 
@@ -59,15 +60,7 @@ public class ShardCommitCoordinator {
 
     private final String name;
 
-    private final RemovalListener<String, CohortEntry> cacheRemovalListener =
-            new RemovalListener<String, CohortEntry>() {
-                @Override
-                public void onRemoval(RemovalNotification<String, CohortEntry> notification) {
-                    if(notification.getCause() == RemovalCause.EXPIRED) {
-                        log.warn("{}: Transaction {} was timed out of the cache", name, notification.getKey());
-                    }
-                }
-            };
+    private final long cacheExpiryTimeoutInMillis;
 
     // This is a hook for unit tests to replace or decorate the DOMStoreThreePhaseCommitCohorts.
     private CohortDecorator cohortDecorator;
@@ -75,19 +68,13 @@ public class ShardCommitCoordinator {
     private ReadyTransactionReply readyTransactionReply;
 
     public ShardCommitCoordinator(ShardDataTree dataTree,
-            long cacheExpiryTimeoutInSec, int queueCapacity, ActorRef shardActor, Logger log, String name) {
+            long cacheExpiryTimeoutInMillis, int queueCapacity, ActorRef shardActor, Logger log, String name) {
 
         this.queueCapacity = queueCapacity;
         this.log = log;
         this.name = name;
         this.dataTree = Preconditions.checkNotNull(dataTree);
-
-        cohortCache = CacheBuilder.newBuilder().expireAfterAccess(cacheExpiryTimeoutInSec, TimeUnit.SECONDS).
-                removalListener(cacheRemovalListener).build();
-
-        // We use a LinkedList here to avoid synchronization overhead with concurrent queue impls
-        // since this should only be accessed on the shard's dispatcher.
-        queuedCohortEntries = new LinkedList<>();
+        this.cacheExpiryTimeoutInMillis = cacheExpiryTimeoutInMillis;
     }
 
     public void setQueueCapacity(int queueCapacity) {
@@ -102,6 +89,23 @@ public class ShardCommitCoordinator {
         return readyTransactionReply;
     }
 
+    private boolean queueCohortEntry(CohortEntry cohortEntry, ActorRef sender, Shard shard) {
+        if(queuedCohortEntries.size() < queueCapacity) {
+            queuedCohortEntries.offer(cohortEntry);
+            return true;
+        } else {
+            cohortCache.remove(cohortEntry.getTransactionID());
+
+            RuntimeException ex = new RuntimeException(
+                    String.format("%s: Could not enqueue transaction %s - the maximum commit queue"+
+                                  " capacity %d has been reached.",
+                                  name, cohortEntry.getTransactionID(), queueCapacity));
+            log.error(ex.getMessage());
+            sender.tell(new Status.Failure(ex), shard.self());
+            return false;
+        }
+    }
+
     /**
      * 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.
@@ -114,6 +118,10 @@ public class ShardCommitCoordinator {
                 (MutableCompositeModification) ready.getModification());
         cohortCache.put(ready.getTransactionID(), cohortEntry);
 
+        if(!queueCohortEntry(cohortEntry, sender, shard)) {
+            return;
+        }
+
         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.
@@ -156,9 +164,9 @@ public class ShardCommitCoordinator {
      *
      * @throws ExecutionException if an error occurs loading the cache
      */
-    boolean handleBatchedModifications(BatchedModifications batched, ActorRef sender, Shard shard)
+    void handleBatchedModifications(BatchedModifications batched, ActorRef sender, Shard shard)
             throws ExecutionException {
-        CohortEntry cohortEntry = cohortCache.getIfPresent(batched.getTransactionID());
+        CohortEntry cohortEntry = cohortCache.get(batched.getTransactionID());
         if(cohortEntry == null) {
             cohortEntry = new CohortEntry(batched.getTransactionID(),
                     dataTree.newReadWriteTransaction(batched.getTransactionID(),
@@ -174,6 +182,10 @@ public class ShardCommitCoordinator {
         cohortEntry.applyModifications(batched.getModifications());
 
         if(batched.isReady()) {
+            if(!queueCohortEntry(cohortEntry, sender, shard)) {
+                return;
+            }
+
             if(log.isDebugEnabled()) {
                 log.debug("{}: Readying Tx {}, client version {}", name,
                         batched.getTransactionID(), batched.getVersion());
@@ -191,8 +203,6 @@ public class ShardCommitCoordinator {
         } else {
             sender.tell(new BatchedModificationsReply(batched.getModifications().size()), shard.self());
         }
-
-        return batched.isReady();
     }
 
     /**
@@ -208,6 +218,11 @@ public class ShardCommitCoordinator {
         final CohortEntry cohortEntry = new CohortEntry(message.getTransactionID(), cohort);
         cohortCache.put(message.getTransactionID(), cohortEntry);
         cohortEntry.setDoImmediateCommit(message.isDoCommitOnReady());
+
+        if(!queueCohortEntry(cohortEntry, sender, shard)) {
+            return;
+        }
+
         log.debug("{}: Applying local modifications for Tx {}", name, message.getTransactionID());
 
         if (message.isDoCommitOnReady()) {
@@ -222,36 +237,31 @@ public class ShardCommitCoordinator {
     private void handleCanCommit(CohortEntry cohortEntry) {
         String transactionID = cohortEntry.getTransactionID();
 
-        if(log.isDebugEnabled()) {
-            log.debug("{}: Processing canCommit for transaction {} for shard {}",
-                    name, transactionID, cohortEntry.getShard().self().path());
-        }
+        cohortEntry.updateLastAccessTime();
 
         if(currentCohortEntry != null) {
-            // There's already a Tx commit in progress - attempt to queue this entry to be
-            // committed after the current Tx completes.
-            log.debug("{}: Transaction {} is already in progress - queueing transaction {}",
-                    name, currentCohortEntry.getTransactionID(), transactionID);
+            // There's already a Tx commit in progress so we can't process this entry yet - but it's in the
+            // queue and will get processed after all prior entries complete.
 
-            if(queuedCohortEntries.size() < queueCapacity) {
-                queuedCohortEntries.offer(cohortEntry);
-            } else {
-                removeCohortEntry(transactionID);
-
-                RuntimeException ex = new RuntimeException(
-                        String.format("%s: Could not enqueue transaction %s - the maximum commit queue"+
-                                      " capacity %d has been reached.",
-                                      name, transactionID, queueCapacity));
-                log.error(ex.getMessage());
-                cohortEntry.getReplySender().tell(new Status.Failure(ex), cohortEntry.getShard().self());
+            if(log.isDebugEnabled()) {
+                log.debug("{}: Commit for Tx {} already in progress - skipping canCommit for {} for now",
+                        name, currentCohortEntry.getTransactionID(), transactionID);
             }
-        } else {
-            // No Tx commit currently in progress - make this the current entry and proceed with
-            // canCommit.
-            cohortEntry.updateLastAccessTime();
-            currentCohortEntry = cohortEntry;
 
-            doCanCommit(cohortEntry);
+            return;
+        }
+
+        // No Tx commit currently in progress - check if this entry is the next one in the queue, If so make
+        // it the current entry and proceed with canCommit.
+        // Purposely checking reference equality here.
+        if(queuedCohortEntries.peek() == cohortEntry) {
+            currentCohortEntry = queuedCohortEntries.poll();
+            doCanCommit(currentCohortEntry);
+        } else {
+            if(log.isDebugEnabled()) {
+                log.debug("{}: Tx {} is the next pending canCommit - skipping {} for now",
+                        name, queuedCohortEntries.peek().getTransactionID(), transactionID);
+            }
         }
     }
 
@@ -265,7 +275,7 @@ public class ShardCommitCoordinator {
     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);
+        final CohortEntry cohortEntry = cohortCache.get(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.
@@ -283,7 +293,6 @@ public 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
@@ -291,6 +300,8 @@ public class ShardCommitCoordinator {
             // currently uses a same thread executor anyway.
             canCommit = cohortEntry.getCohort().canCommit().get();
 
+            log.debug("{}: canCommit for {}: {}", name, cohortEntry.getTransactionID(), canCommit);
+
             if(cohortEntry.isDoImmediateCommit()) {
                 if(canCommit) {
                     doCommit(cohortEntry);
@@ -304,7 +315,7 @@ public class ShardCommitCoordinator {
                             CanCommitTransactionReply.NO.toSerializable(), cohortEntry.getShard().self());
             }
         } catch (Exception e) {
-            log.debug("{}: An exception occurred during canCommit: {}", name, e);
+            log.debug("{}: An exception occurred during canCommit", name, e);
 
             Throwable failure = e;
             if(e instanceof ExecutionException) {
@@ -367,6 +378,7 @@ public class ShardCommitCoordinator {
             return false;
         }
 
+        cohortEntry.setReplySender(sender);
         return doCommit(cohortEntry);
     }
 
@@ -391,13 +403,7 @@ public class ShardCommitCoordinator {
     }
 
     public CohortEntry getAndRemoveCohortEntry(String transactionID) {
-        CohortEntry cohortEntry = cohortCache.getIfPresent(transactionID);
-        cohortCache.invalidate(transactionID);
-        return cohortEntry;
-    }
-
-    public void removeCohortEntry(String transactionID) {
-        cohortCache.invalidate(transactionID);
+        return cohortCache.remove(transactionID);
     }
 
     public boolean isCurrentTransaction(String transactionID) {
@@ -416,33 +422,66 @@ public class ShardCommitCoordinator {
      */
     public void currentTransactionComplete(String transactionID, boolean removeCohortEntry) {
         if(removeCohortEntry) {
-            removeCohortEntry(transactionID);
+            cohortCache.remove(transactionID);
         }
 
         if(isCurrentTransaction(transactionID)) {
-            // Dequeue the next cohort entry waiting in the queue.
-            currentCohortEntry = queuedCohortEntries.poll();
-            if(currentCohortEntry != null) {
-                currentCohortEntry.updateLastAccessTime();
-                doCanCommit(currentCohortEntry);
+            currentCohortEntry = null;
+
+            log.debug("{}: currentTransactionComplete: {}", name, transactionID);
+
+            maybeProcessNextCohortEntry();
+        }
+    }
+
+    private void maybeProcessNextCohortEntry() {
+        // Check if there's a next cohort entry waiting in the queue and if it is ready to commit. Also
+        // clean out expired entries.
+        Iterator<CohortEntry> iter = queuedCohortEntries.iterator();
+        while(iter.hasNext()) {
+            CohortEntry next = iter.next();
+            if(next.isReadyToCommit()) {
+                if(currentCohortEntry == null) {
+                    if(log.isDebugEnabled()) {
+                        log.debug("{}: Next entry to canCommit {}", name, next);
+                    }
+
+                    iter.remove();
+                    currentCohortEntry = next;
+                    currentCohortEntry.updateLastAccessTime();
+                    doCanCommit(currentCohortEntry);
+                }
+
+                break;
+            } 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 {
+                break;
             }
         }
     }
 
+    void cleanupExpiredCohortEntries() {
+        maybeProcessNextCohortEntry();
+    }
+
     @VisibleForTesting
     void setCohortDecorator(CohortDecorator cohortDecorator) {
         this.cohortDecorator = cohortDecorator;
     }
 
-
     static class CohortEntry {
         private final String transactionID;
         private ShardDataTreeCohort cohort;
         private final ReadWriteShardDataTreeTransaction transaction;
         private ActorRef replySender;
         private Shard shard;
-        private long lastAccessTime;
         private boolean doImmediateCommit;
+        private final Stopwatch lastAccessTimer = Stopwatch.createStarted();
 
         CohortEntry(String transactionID, ReadWriteShardDataTreeTransaction transaction) {
             this.transaction = Preconditions.checkNotNull(transaction);
@@ -463,11 +502,8 @@ public class ShardCommitCoordinator {
         }
 
         void updateLastAccessTime() {
-            lastAccessTime = System.currentTimeMillis();
-        }
-
-        long getLastAccessTime() {
-            return lastAccessTime;
+            lastAccessTimer.reset();
+            lastAccessTimer.start();
         }
 
         String getTransactionID() {
@@ -497,6 +533,14 @@ public class ShardCommitCoordinator {
             }
         }
 
+        boolean isReadyToCommit() {
+            return replySender != null;
+        }
+
+        boolean isExpired(long expireTimeInMillis) {
+            return lastAccessTimer.elapsed(TimeUnit.MILLISECONDS) >= expireTimeInMillis;
+        }
+
         boolean isDoImmediateCommit() {
             return doImmediateCommit;
         }
@@ -520,5 +564,13 @@ public class ShardCommitCoordinator {
         void setShard(Shard shard) {
             this.shard = shard;
         }
+
+        @Override
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append("CohortEntry [transactionID=").append(transactionID).append(", doImmediateCommit=")
+                    .append(doImmediateCommit).append("]");
+            return builder.toString();
+        }
     }
 }