atomic-storage: remove type dependency at segment level I/O
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / ShardCommitCoordinator.java
index 3451934e25109aae47ef9b6612b47551338ab447..946203b6b76aa5e2c4b4f94a849a9430f2d3fa06 100644 (file)
@@ -7,36 +7,39 @@
  */
 package org.opendaylight.controller.cluster.datastore;
 
+import static java.util.Objects.requireNonNull;
+
 import akka.actor.ActorRef;
 import akka.actor.Status.Failure;
 import akka.serialization.Serialization;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
+import com.google.common.primitives.UnsignedLong;
+import com.google.common.util.concurrent.FutureCallback;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.LinkedList;
-import java.util.List;
 import java.util.Map;
-import java.util.Queue;
-import java.util.concurrent.ExecutionException;
-import org.opendaylight.controller.cluster.datastore.DataTreeCohortActorRegistry.CohortRegistryCommand;
+import org.eclipse.jdt.annotation.NonNull;
+import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 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.CanCommitTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransactionReply;
 import org.opendaylight.controller.cluster.datastore.messages.CommitTransaction;
+import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
 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.messages.VersionedExternalizableMessage;
 import org.opendaylight.controller.cluster.datastore.utils.AbstractBatchedModificationsCursor;
-import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
 import org.opendaylight.yangtools.concepts.Identifier;
-import org.opendaylight.yangtools.util.StringIdentifier;
-import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.opendaylight.yangtools.yang.common.Empty;
+import org.opendaylight.yangtools.yang.data.tree.api.DataTreeCandidate;
 import org.slf4j.Logger;
 
 /**
@@ -44,91 +47,51 @@ import org.slf4j.Logger;
  *
  * @author Thomas Pantelis
  */
+@Deprecated(since = "9.0.0", forRemoval = true)
 final class ShardCommitCoordinator {
 
-    // Interface hook for unit tests to replace or decorate the DOMStoreThreePhaseCommitCohorts.
+    // Interface hook for unit tests to replace or decorate the ShardDataTreeCohorts.
+    @VisibleForTesting
     public interface CohortDecorator {
         ShardDataTreeCohort decorate(Identifier transactionID, ShardDataTreeCohort actual);
     }
 
     private final Map<Identifier, CohortEntry> cohortCache = new HashMap<>();
 
-    private CohortEntry currentCohortEntry;
-
     private final ShardDataTree dataTree;
 
-    private final DataTreeCohortActorRegistry cohortRegistry = new DataTreeCohortActorRegistry();
-
-    // 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;
-
     private final Logger log;
 
     private final String name;
 
-    private final long cacheExpiryTimeoutInMillis;
-
-    // This is a hook for unit tests to replace or decorate the DOMStoreThreePhaseCommitCohorts.
+    // This is a hook for unit tests to replace or decorate the ShardDataTreeCohorts.
+    @VisibleForTesting
     private CohortDecorator cohortDecorator;
 
     private ReadyTransactionReply readyTransactionReply;
 
-    private Runnable runOnPendingTransactionsComplete;
-
-    ShardCommitCoordinator(ShardDataTree dataTree, long cacheExpiryTimeoutInMillis, int queueCapacity, Logger log,
-            String name) {
-
-        this.queueCapacity = queueCapacity;
+    ShardCommitCoordinator(final ShardDataTree dataTree, final Logger log, final String name) {
         this.log = log;
         this.name = name;
-        this.dataTree = Preconditions.checkNotNull(dataTree);
-        this.cacheExpiryTimeoutInMillis = cacheExpiryTimeoutInMillis;
-    }
-
-    int getQueueSize() {
-        return queuedCohortEntries.size();
+        this.dataTree = requireNonNull(dataTree);
     }
 
     int getCohortCacheSize() {
         return cohortCache.size();
     }
 
-    void setQueueCapacity(int queueCapacity) {
-        this.queueCapacity = queueCapacity;
+    private String persistenceId() {
+        return dataTree.logContext();
     }
 
-    private ReadyTransactionReply readyTransactionReply(Shard shard) {
-        if(readyTransactionReply == null) {
-            readyTransactionReply = new ReadyTransactionReply(Serialization.serializedActorPath(shard.self()));
+    private ReadyTransactionReply readyTransactionReply(final ActorRef cohort) {
+        if (readyTransactionReply == null) {
+            readyTransactionReply = new ReadyTransactionReply(Serialization.serializedActorPath(cohort));
         }
 
         return readyTransactionReply;
     }
 
-    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());
-
-            final 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 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.
@@ -136,22 +99,17 @@ final class ShardCommitCoordinator {
      * @param ready the ForwardedReadyTransaction message to process
      * @param sender the sender of the message
      * @param shard the transaction's shard actor
-     * @param schema
      */
-    void handleForwardedReadyTransaction(ForwardedReadyTransaction ready, ActorRef sender, Shard shard,
-            SchemaContext schema) {
+    void handleForwardedReadyTransaction(final ForwardedReadyTransaction ready, final ActorRef sender,
+            final Shard shard) {
         log.debug("{}: Readying transaction {}, client version {}", name,
-                ready.getTransactionID(), ready.getTxnClientVersion());
+                ready.getTransactionId(), ready.getTxnClientVersion());
 
-        final ShardDataTreeCohort cohort = ready.getTransaction().ready();
-        final CohortEntry cohortEntry = new CohortEntry(ready.getTransactionID(), cohort, cohortRegistry, schema, ready.getTxnClientVersion());
-        cohortCache.put(cohortEntry.getTransactionID(), cohortEntry);
+        final ShardDataTreeCohort cohort = ready.getTransaction().ready(ready.getParticipatingShardNames());
+        final CohortEntry cohortEntry = CohortEntry.createReady(cohort, ready.getTxnClientVersion());
+        cohortCache.put(cohortEntry.getTransactionId(), cohortEntry);
 
-        if(!queueCohortEntry(cohortEntry, sender, shard)) {
-            return;
-        }
-
-        if(ready.isDoImmediateCommit()) {
+        if (ready.isDoImmediateCommit()) {
             cohortEntry.setDoImmediateCommit(true);
             cohortEntry.setReplySender(sender);
             cohortEntry.setShard(shard);
@@ -159,7 +117,7 @@ final class ShardCommitCoordinator {
         } 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());
+            sender.tell(readyTransactionReply(shard.self()), shard.self());
         }
     }
 
@@ -171,54 +129,50 @@ final class ShardCommitCoordinator {
      *
      * @param batched the BatchedModifications message to process
      * @param sender the sender of the message
-     * @param shard the transaction's shard actor
      */
-    void handleBatchedModifications(BatchedModifications batched, ActorRef sender, Shard shard, SchemaContext schema) {
-        CohortEntry cohortEntry = cohortCache.get(new StringIdentifier(batched.getTransactionID()));
-        if(cohortEntry == null) {
-            cohortEntry = new CohortEntry(batched.getTransactionID(),
-                    dataTree.newReadWriteTransaction(batched.getTransactionID(), batched.getTransactionChainID()),
-                    cohortRegistry, schema,  batched.getVersion());
-            cohortCache.put(cohortEntry.getTransactionID(), cohortEntry);
+    @SuppressFBWarnings(value = "THROWS_METHOD_THROWS_RUNTIMEEXCEPTION", justification = "Replay of captured failure")
+    void handleBatchedModifications(final BatchedModifications batched, final ActorRef sender, final Shard shard) {
+        CohortEntry cohortEntry = cohortCache.get(batched.getTransactionId());
+        if (cohortEntry == null || cohortEntry.isSealed()) {
+            cohortEntry = CohortEntry.createOpen(dataTree.newReadWriteTransaction(batched.getTransactionId()),
+                batched.getVersion());
+            cohortCache.put(cohortEntry.getTransactionId(), cohortEntry);
         }
 
-        if(log.isDebugEnabled()) {
+        if (log.isDebugEnabled()) {
             log.debug("{}: Applying {} batched modifications for Tx {}", name,
-                    batched.getModifications().size(), batched.getTransactionID());
+                    batched.getModifications().size(), batched.getTransactionId());
         }
 
         cohortEntry.applyModifications(batched.getModifications());
 
-        if(batched.isReady()) {
-            if(cohortEntry.getLastBatchedModificationsException() != null) {
-                cohortCache.remove(cohortEntry.getTransactionID());
+        if (batched.isReady()) {
+            if (cohortEntry.getLastBatchedModificationsException() != null) {
+                cohortCache.remove(cohortEntry.getTransactionId());
                 throw cohortEntry.getLastBatchedModificationsException();
             }
 
-            if(cohortEntry.getTotalBatchedModificationsReceived() != batched.getTotalMessagesSent()) {
-                cohortCache.remove(cohortEntry.getTransactionID());
+            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;
+            if (log.isDebugEnabled()) {
+                log.debug("{}: Readying Tx {} of {} operations, client version {}", name,
+                        batched.getTransactionId(), cohortEntry.getTotalOperationsProcessed(), batched.getVersion());
             }
 
-            if(log.isDebugEnabled()) {
-                log.debug("{}: Readying Tx {}, client version {}", name,
-                        batched.getTransactionID(), batched.getVersion());
-            }
-
-            cohortEntry.ready(cohortDecorator, batched.isDoCommitOnReady());
+            cohortEntry.setDoImmediateCommit(batched.isDoCommitOnReady());
+            cohortEntry.ready(batched.getParticipatingShardNames(), cohortDecorator);
 
-            if(batched.isDoCommitOnReady()) {
+            if (batched.isDoCommitOnReady()) {
                 cohortEntry.setReplySender(sender);
                 cohortEntry.setShard(shard);
                 handleCanCommit(cohortEntry);
             } else {
-                sender.tell(readyTransactionReply(shard), shard.self());
+                sender.tell(readyTransactionReply(shard.self()), shard.self());
             }
         } else {
             sender.tell(new BatchedModificationsReply(batched.getModifications().size()), shard.self());
@@ -234,34 +188,30 @@ final class ShardCommitCoordinator {
      * @param sender the sender of the message
      * @param shard the transaction's shard actor
      */
-    void handleReadyLocalTransaction(ReadyLocalTransaction message, ActorRef sender, Shard shard,
-            SchemaContext schema) {
-        final ShardDataTreeCohort cohort = new SimpleShardDataTreeCohort(dataTree, message.getModification(),
-                message.getTransactionID());
-        final CohortEntry cohortEntry = new CohortEntry(message.getTransactionID(), cohort, cohortRegistry, schema,
-                DataStoreVersions.CURRENT_VERSION);
-        cohortCache.put(cohortEntry.getTransactionID(), cohortEntry);
+    void handleReadyLocalTransaction(final ReadyLocalTransaction message, final ActorRef sender, final Shard shard) {
+        final TransactionIdentifier txId = message.getTransactionId();
+        final ShardDataTreeCohort cohort = dataTree.newReadyCohort(txId, message.getModification(),
+                message.getParticipatingShardNames());
+        final CohortEntry cohortEntry = CohortEntry.createReady(cohort, DataStoreVersions.CURRENT_VERSION);
+        cohortCache.put(cohortEntry.getTransactionId(), cohortEntry);
         cohortEntry.setDoImmediateCommit(message.isDoCommitOnReady());
 
-        if(!queueCohortEntry(cohortEntry, sender, shard)) {
-            return;
-        }
-
-        log.debug("{}: Applying local modifications for Tx {}", name, message.getTransactionID());
+        log.debug("{}: Applying local modifications for Tx {}", name, txId);
 
         if (message.isDoCommitOnReady()) {
             cohortEntry.setReplySender(sender);
             cohortEntry.setShard(shard);
             handleCanCommit(cohortEntry);
         } else {
-            sender.tell(readyTransactionReply(shard), shard.self());
+            sender.tell(readyTransactionReply(shard.self()), shard.self());
         }
     }
 
+    @Deprecated(since = "9.0.0", forRemoval = true)
     Collection<BatchedModifications> createForwardedBatchedModifications(final BatchedModifications from,
             final int maxModificationsPerBatch) {
-        CohortEntry cohortEntry = getAndRemoveCohortEntry(new StringIdentifier(from.getTransactionID()));
-        if(cohortEntry == null || cohortEntry.getTransaction() == null) {
+        CohortEntry cohortEntry = cohortCache.remove(from.getTransactionId());
+        if (cohortEntry == null || cohortEntry.getTransaction() == null) {
             return Collections.singletonList(from);
         }
 
@@ -271,10 +221,9 @@ final class ShardCommitCoordinator {
         cohortEntry.getTransaction().getSnapshot().applyToCursor(new AbstractBatchedModificationsCursor() {
             @Override
             protected BatchedModifications getModifications() {
-                if(newModifications.isEmpty() ||
-                        newModifications.getLast().getModifications().size() >= maxModificationsPerBatch) {
-                    newModifications.add(new BatchedModifications(from.getTransactionID(),
-                            from.getVersion(), from.getTransactionChainID()));
+                if (newModifications.isEmpty()
+                        || newModifications.getLast().getModifications().size() >= maxModificationsPerBatch) {
+                    newModifications.add(new BatchedModifications(from.getTransactionId(), from.getVersion()));
                 }
 
                 return newModifications.getLast();
@@ -283,39 +232,37 @@ final class ShardCommitCoordinator {
 
         BatchedModifications last = newModifications.getLast();
         last.setDoCommitOnReady(from.isDoCommitOnReady());
-        last.setReady(from.isReady());
+        if (from.isReady()) {
+            last.setReady(from.getParticipatingShardNames());
+        }
         last.setTotalMessagesSent(newModifications.size());
         return newModifications;
     }
 
-    private void handleCanCommit(CohortEntry cohortEntry) {
-        cohortEntry.updateLastAccessTime();
-
-        if(currentCohortEntry != null) {
-            // 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.
+    private void handleCanCommit(final CohortEntry cohortEntry) {
+        cohortEntry.canCommit(new FutureCallback<>() {
+            @Override
+            public void onSuccess(final Empty result) {
+                log.debug("{}: canCommit for {}: success", name, cohortEntry.getTransactionId());
 
-            if(log.isDebugEnabled()) {
-                log.debug("{}: Commit for Tx {} already in progress - skipping canCommit for {} for now",
-                        name, currentCohortEntry.getTransactionID(), cohortEntry.getTransactionID());
+                if (cohortEntry.isDoImmediateCommit()) {
+                    doCommit(cohortEntry);
+                } else {
+                    cohortEntry.getReplySender().tell(
+                        CanCommitTransactionReply.yes(cohortEntry.getClientVersion()).toSerializable(),
+                        cohortEntry.getShard().self());
+                }
             }
 
-            return;
-        }
+            @Override
+            public void onFailure(final Throwable failure) {
+                log.debug("{}: An exception occurred during canCommit for {}", name, cohortEntry.getTransactionId(),
+                    failure);
 
-        // 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() != null ? queuedCohortEntries.peek().getTransactionID() : "???",
-                                cohortEntry.getTransactionID());
+                cohortCache.remove(cohortEntry.getTransactionId());
+                cohortEntry.getReplySender().tell(new Failure(failure), cohortEntry.getShard().self());
             }
-        }
+        });
     }
 
     /**
@@ -325,16 +272,16 @@ final class ShardCommitCoordinator {
      * @param sender the actor to which to send the response
      * @param shard the transaction's shard actor
      */
-    void handleCanCommit(Identifier transactionID, final ActorRef sender, final Shard shard) {
+    void handleCanCommit(final Identifier 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.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.
+        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 or it was aborted.
             IllegalStateException ex = new IllegalStateException(
-                    String.format("%s: No cohort entry found for transaction %s", name, transactionID));
-            log.error(ex.getMessage());
+                    String.format("%s: Cannot canCommit transaction %s - no cohort entry found", name, transactionID));
+            log.error("{}: Inconsistency during transaction {} canCommit", name, transactionID, ex);
             sender.tell(new Failure(ex), shard.self());
             return;
         }
@@ -345,70 +292,54 @@ final class ShardCommitCoordinator {
         handleCanCommit(cohortEntry);
     }
 
-    private void doCanCommit(final CohortEntry cohortEntry) {
-        boolean canCommit = false;
-        try {
-            canCommit = cohortEntry.canCommit();
-
-            log.debug("{}: canCommit for {}: {}", name, cohortEntry.getTransactionID(), canCommit);
-
-            if(cohortEntry.isDoImmediateCommit()) {
-                if(canCommit) {
-                    doCommit(cohortEntry);
-                } else {
-                    cohortEntry.getReplySender().tell(new Failure(new TransactionCommitFailedException(
-                                "Can Commit failed, no detailed cause available.")), cohortEntry.getShard().self());
-                }
-            } else {
-                cohortEntry.getReplySender().tell(
-                        canCommit ? CanCommitTransactionReply.yes(cohortEntry.getClientVersion()).toSerializable() :
-                            CanCommitTransactionReply.no(cohortEntry.getClientVersion()).toSerializable(),
-                        cohortEntry.getShard().self());
-            }
-        } catch (Exception e) {
-            log.debug("{}: An exception occurred during canCommit", name, e);
-
-            Throwable failure = e;
-            if(e instanceof ExecutionException) {
-                failure = e.getCause();
-            }
-
-            cohortEntry.getReplySender().tell(new Failure(failure), cohortEntry.getShard().self());
-        } finally {
-            if(!canCommit) {
-                // Remove the entry from the cache now.
-                currentTransactionComplete(cohortEntry.getTransactionID(), true);
-            }
-        }
-    }
-
-    private boolean doCommit(CohortEntry cohortEntry) {
-        log.debug("{}: Committing transaction {}", name, cohortEntry.getTransactionID());
-
-        boolean success = false;
+    void doCommit(final CohortEntry cohortEntry) {
+        log.debug("{}: Committing transaction {}", name, cohortEntry.getTransactionId());
 
         // 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.
+        cohortEntry.preCommit(new FutureCallback<DataTreeCandidate>() {
+            @Override
+            public void onSuccess(final DataTreeCandidate candidate) {
+                finishCommit(cohortEntry.getReplySender(), cohortEntry);
+            }
 
-        try {
-            cohortEntry.preCommit();
+            @Override
+            public void onFailure(final Throwable failure) {
+                log.error("{} An exception occurred while preCommitting transaction {}", name,
+                        cohortEntry.getTransactionId(), failure);
 
-            cohortEntry.getShard().continueCommit(cohortEntry);
+                cohortCache.remove(cohortEntry.getTransactionId());
+                cohortEntry.getReplySender().tell(new Failure(failure), cohortEntry.getShard().self());
+            }
+        });
+    }
 
-            cohortEntry.updateLastAccessTime();
+    void finishCommit(final @NonNull ActorRef sender, final @NonNull CohortEntry cohortEntry) {
+        log.debug("{}: Finishing commit for transaction {}", persistenceId(), cohortEntry.getTransactionId());
 
-            success = true;
-        } catch (Exception e) {
-            log.error("{} An exception occurred while preCommitting transaction {}",
-                    name, cohortEntry.getTransactionID(), e);
-            cohortEntry.getReplySender().tell(new Failure(e), cohortEntry.getShard().self());
+        cohortEntry.commit(new FutureCallback<UnsignedLong>() {
+            @Override
+            public void onSuccess(final UnsignedLong result) {
+                final TransactionIdentifier txId = cohortEntry.getTransactionId();
+                log.debug("{}: Transaction {} committed as {}, sending response to {}", persistenceId(), txId, result,
+                    sender);
+
+                cohortCache.remove(cohortEntry.getTransactionId());
+                sender.tell(CommitTransactionReply.instance(cohortEntry.getClientVersion()).toSerializable(),
+                    cohortEntry.getShard().self());
+            }
 
-            currentTransactionComplete(cohortEntry.getTransactionID(), true);
-        }
+            @Override
+            public void onFailure(final Throwable failure) {
+                final TransactionIdentifier txId = cohortEntry.getTransactionId();
+                log.error("{}, An exception occurred while committing transaction {}", persistenceId(), txId, failure);
 
-        return success;
+                cohortCache.remove(cohortEntry.getTransactionId());
+                sender.tell(new Failure(failure), cohortEntry.getShard().self());
+            }
+        });
     }
 
     /**
@@ -417,267 +348,135 @@ final class ShardCommitCoordinator {
      * @param transactionID the ID of the transaction to commit
      * @param sender the actor to which to send the response
      * @param shard the transaction's shard actor
-     * @return true if the transaction was successfully prepared, false otherwise.
      */
-    boolean handleCommit(final Identifier 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.
+    void handleCommit(final Identifier transactionID, final ActorRef sender, final Shard shard) {
+        final CohortEntry cohortEntry = cohortCache.get(transactionID);
+        if (cohortEntry == null) {
+            // Either a long time passed between canCommit and commit and the entry was expired from the cache
+            // or it was aborted.
             IllegalStateException ex = new IllegalStateException(
-                    String.format("%s: Cannot commit transaction %s - it is not the current transaction",
-                            name, transactionID));
-            log.error(ex.getMessage());
+                    String.format("%s: Cannot commit transaction %s - no cohort entry found", name, transactionID));
+            log.error("{}: Inconsistency during transaction {} commit", name, transactionID, ex);
             sender.tell(new Failure(ex), shard.self());
-            return false;
+            return;
         }
 
         cohortEntry.setReplySender(sender);
-        return doCommit(cohortEntry);
+        doCommit(cohortEntry);
     }
 
+    @SuppressWarnings("checkstyle:IllegalCatch")
     void handleAbort(final Identifier 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) {
+        CohortEntry cohortEntry = cohortCache.remove(transactionID);
+        if (cohortEntry == null) {
             return;
         }
 
         log.debug("{}: Aborting transaction {}", name, transactionID);
 
         final ActorRef self = shard.getSelf();
-        try {
-            cohortEntry.abort();
+        cohortEntry.abort(new FutureCallback<>() {
+            @Override
+            public void onSuccess(final Empty result) {
+                if (sender != null) {
+                    sender.tell(AbortTransactionReply.instance(cohortEntry.getClientVersion()).toSerializable(), self);
+                }
+            }
 
-            shard.getShardMBean().incrementAbortTransactionsCount();
+            @Override
+            public void onFailure(final Throwable failure) {
+                log.error("{}: An exception happened during abort", name, failure);
 
-            if(sender != null) {
-                sender.tell(AbortTransactionReply.instance(cohortEntry.getClientVersion()).toSerializable(), self);
+                if (sender != null) {
+                    sender.tell(new Failure(failure), self);
+                }
             }
-        } catch (Exception e) {
-            log.error("{}: An exception happened during abort", name, e);
+        });
 
-            if(sender != null) {
-                sender.tell(new Failure(e), self);
-            }
-        }
+        shard.getShardMBean().incrementAbortTransactionsCount();
     }
 
     void checkForExpiredTransactions(final long timeout, final Shard shard) {
-        CohortEntry cohortEntry = getCurrentCohortEntry();
-        if(cohortEntry != null) {
-            if(cohortEntry.isExpired(timeout)) {
-                log.warn("{}: Current transaction {} has timed out after {} ms - aborting",
-                        name, cohortEntry.getTransactionID(), timeout);
-
-                handleAbort(cohortEntry.getTransactionID(), null, shard);
-            }
-        }
-
-        cleanupExpiredCohortEntries();
+        cohortCache.values().removeIf(CohortEntry::isFailed);
     }
 
     void abortPendingTransactions(final String reason, final Shard shard) {
-        if(currentCohortEntry == null && queuedCohortEntries.isEmpty()) {
-            return;
-        }
+        final var failure = new Failure(new RuntimeException(reason));
+        final var pending = dataTree.getAndClearPendingTransactions();
 
-        List<CohortEntry> cohortEntries = getAndClearPendingCohortEntries();
+        log.debug("{}: Aborting {} pending queued transactions", name, pending.size());
 
-        log.debug("{}: Aborting {} pending queued transactions", name, cohortEntries.size());
-
-        for(CohortEntry cohortEntry: cohortEntries) {
-            if(cohortEntry.getReplySender() != null) {
-                cohortEntry.getReplySender().tell(new Failure(new RuntimeException(reason)), shard.self());
+        for (var cohort : pending) {
+            final var cohortEntry = cohortCache.remove(cohort.transactionId());
+            if (cohortEntry != null) {
+                final var replySender = cohortEntry.getReplySender();
+                if (replySender != null) {
+                    replySender.tell(failure, shard.self());
+                }
             }
         }
-    }
-
-    private List<CohortEntry> getAndClearPendingCohortEntries() {
-        List<CohortEntry> cohortEntries = new ArrayList<>();
 
-        if(currentCohortEntry != null) {
-            cohortEntries.add(currentCohortEntry);
-            cohortCache.remove(currentCohortEntry.getTransactionID());
-            currentCohortEntry = null;
-        }
-
-        for(CohortEntry cohortEntry: queuedCohortEntries) {
-            cohortEntries.add(cohortEntry);
-            cohortCache.remove(cohortEntry.getTransactionID());
-        }
-
-        queuedCohortEntries.clear();
-        return cohortEntries;
+        cohortCache.clear();
     }
 
-    Collection<Object> convertPendingTransactionsToMessages(final int maxModificationsPerBatch) {
-        if(currentCohortEntry == null && queuedCohortEntries.isEmpty()) {
-            return Collections.emptyList();
-        }
-
-        Collection<Object> messages = new ArrayList<>();
-        List<CohortEntry> cohortEntries = getAndClearPendingCohortEntries();
-        for(CohortEntry cohortEntry: cohortEntries) {
-            if(cohortEntry.isExpired(cacheExpiryTimeoutInMillis) || cohortEntry.isAborted()) {
+    Collection<?> convertPendingTransactionsToMessages(final int maxModificationsPerBatch) {
+        final var messages = new ArrayList<VersionedExternalizableMessage>();
+        for (var cohort : dataTree.getAndClearPendingTransactions()) {
+            final var cohortEntry = cohortCache.remove(cohort.transactionId());
+            if (cohortEntry == null) {
                 continue;
             }
 
-            final LinkedList<BatchedModifications> newModifications = new LinkedList<>();
+            final var newMessages = new ArrayDeque<BatchedModifications>();
             cohortEntry.getDataTreeModification().applyToCursor(new AbstractBatchedModificationsCursor() {
                 @Override
                 protected BatchedModifications getModifications() {
-                    if(newModifications.isEmpty() ||
-                            newModifications.getLast().getModifications().size() >= maxModificationsPerBatch) {
-                        newModifications.add(new BatchedModifications(cohortEntry.getTransactionID().getString(),
-                                cohortEntry.getClientVersion(), ""));
-        }
+                    final var lastBatch = newMessages.peekLast();
+                    if (lastBatch != null && lastBatch.getModifications().size() >= maxModificationsPerBatch) {
+                        return lastBatch;
+                    }
 
-                    return newModifications.getLast();
+                    // Allocate a new message
+                    final var ret = new BatchedModifications(cohortEntry.getTransactionId(),
+                        cohortEntry.getClientVersion());
+                    newMessages.add(ret);
+                    return ret;
                 }
             });
 
-            if(!newModifications.isEmpty()) {
-                BatchedModifications last = newModifications.getLast();
-                last.setDoCommitOnReady(cohortEntry.isDoImmediateCommit());
-                last.setReady(true);
-                last.setTotalMessagesSent(newModifications.size());
-                messages.addAll(newModifications);
-
-                if(!cohortEntry.isDoImmediateCommit() && cohortEntry.getState() == CohortEntry.State.CAN_COMMITTED) {
-                    messages.add(new CanCommitTransaction(cohortEntry.getTransactionID().getString(),
-                            cohortEntry.getClientVersion()));
-                }
-
-                if(!cohortEntry.isDoImmediateCommit() && cohortEntry.getState() == CohortEntry.State.PRE_COMMITTED) {
-                    messages.add(new CommitTransaction(cohortEntry.getTransactionID().getString(),
-                            cohortEntry.getClientVersion()));
-                }
-            }
-        }
-
-        return messages;
-    }
-
-    /**
-     * Returns the cohort entry for the Tx commit currently in progress if the given transaction ID
-     * matches the current entry.
-     *
-     * @param transactionID the ID of the transaction
-     * @return the current CohortEntry or null if the given transaction ID does not match the
-     *         current entry.
-     */
-    CohortEntry getCohortEntryIfCurrent(Identifier transactionID) {
-        if(isCurrentTransaction(transactionID)) {
-            return currentCohortEntry;
-        }
-
-        return null;
-    }
-
-    CohortEntry getCurrentCohortEntry() {
-        return currentCohortEntry;
-    }
-
-    CohortEntry getAndRemoveCohortEntry(Identifier transactionID) {
-        return cohortCache.remove(transactionID);
-    }
-
-    boolean isCurrentTransaction(Identifier transactionID) {
-        return currentCohortEntry != null &&
-                currentCohortEntry.getTransactionID().equals(transactionID);
-    }
-
-    /**
-     * This method is called when a transaction is complete, successful or not. If the given
-     * given transaction ID matches the current in-progress transaction, the next cohort entry,
-     * if any, is dequeued and processed.
-     *
-     * @param transactionID the ID of the completed transaction
-     * @param removeCohortEntry if true the CohortEntry for the transaction is also removed from
-     *        the cache.
-     */
-    void currentTransactionComplete(Identifier transactionID, boolean removeCohortEntry) {
-        if(removeCohortEntry) {
-            cohortCache.remove(transactionID);
-        }
-
-        if(isCurrentTransaction(transactionID)) {
-            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.
-        final Iterator<CohortEntry> iter = queuedCohortEntries.iterator();
-        while(iter.hasNext()) {
-            final CohortEntry next = iter.next();
-            if(next.isReadyToCommit()) {
-                if(currentCohortEntry == null) {
-                    if(log.isDebugEnabled()) {
-                        log.debug("{}: Next entry to canCommit {}", name, next);
+            final var last = newMessages.peekLast();
+            if (last != null) {
+                final boolean immediate = cohortEntry.isDoImmediateCommit();
+                last.setDoCommitOnReady(immediate);
+                last.setReady(cohortEntry.getParticipatingShardNames());
+                last.setTotalMessagesSent(newMessages.size());
+
+                messages.addAll(newMessages);
+
+                if (!immediate) {
+                    switch (cohort.getState()) {
+                        case CAN_COMMIT_COMPLETE:
+                        case CAN_COMMIT_PENDING:
+                            messages.add(new CanCommitTransaction(cohortEntry.getTransactionId(),
+                                cohortEntry.getClientVersion()));
+                            break;
+                        case PRE_COMMIT_COMPLETE:
+                        case PRE_COMMIT_PENDING:
+                            messages.add(new CommitTransaction(cohortEntry.getTransactionId(),
+                                cohortEntry.getClientVersion()));
+                            break;
+                        default:
+                            break;
                     }
-
-                    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);
-            } else if(!next.isAborted()) {
-                break;
             }
-
-            iter.remove();
-            cohortCache.remove(next.getTransactionID());
         }
 
-        maybeRunOperationOnPendingTransactionsComplete();
-    }
-
-    void cleanupExpiredCohortEntries() {
-        maybeProcessNextCohortEntry();
-    }
-
-    void setRunOnPendingTransactionsComplete(Runnable operation) {
-        runOnPendingTransactionsComplete = operation;
-        maybeRunOperationOnPendingTransactionsComplete();
-    }
-
-    private void maybeRunOperationOnPendingTransactionsComplete() {
-        if(runOnPendingTransactionsComplete != null && currentCohortEntry == null && queuedCohortEntries.isEmpty()) {
-            log.debug("{}: Pending transactions complete - running operation {}", name, runOnPendingTransactionsComplete);
-
-            runOnPendingTransactionsComplete.run();
-            runOnPendingTransactionsComplete = null;
-        }
+        return messages;
     }
 
     @VisibleForTesting
-    void setCohortDecorator(CohortDecorator cohortDecorator) {
+    void setCohortDecorator(final CohortDecorator cohortDecorator) {
         this.cohortDecorator = cohortDecorator;
     }
-
-   void processCohortRegistryCommand(ActorRef sender, CohortRegistryCommand message) {
-        cohortRegistry.process(sender, message);
-    }
 }