BUG-8056: make doCommit/finishCommit package-private
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / ShardCommitCoordinator.java
index 54f15fcb4bd03115d97eccc72166b0b12efbca2f..691a4da66c9e82523d3bd7a97bb6fdbc0b9da7d9 100644 (file)
@@ -8,27 +8,37 @@
 package org.opendaylight.controller.cluster.datastore;
 
 import akka.actor.ActorRef;
-import akka.actor.Status;
+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.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.primitives.UnsignedLong;
+import com.google.common.util.concurrent.FutureCallback;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.Iterator;
 import java.util.LinkedList;
-import java.util.Queue;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
+import java.util.Map;
+import javax.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.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.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.yangtools.concepts.Identifier;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidate;
 import org.slf4j.Logger;
 
 /**
@@ -36,76 +46,77 @@ import org.slf4j.Logger;
  *
  * @author Thomas Pantelis
  */
-public class ShardCommitCoordinator {
+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 {
-        DOMStoreThreePhaseCommitCohort decorate(String transactionID, DOMStoreThreePhaseCommitCohort actual);
+        ShardDataTreeCohort decorate(Identifier transactionID, ShardDataTreeCohort actual);
     }
 
-    private final Cache<String, CohortEntry> cohortCache;
-
-    private CohortEntry currentCohortEntry;
-
-    private final DOMTransactionFactory transactionFactory;
-
-    private final Queue<CohortEntry> queuedCohortEntries;
+    private final Map<Identifier, CohortEntry> cohortCache = new HashMap<>();
 
-    private int queueCapacity;
+    private final ShardDataTree dataTree;
 
     private final Logger log;
 
     private final String name;
 
-    private final String shardActorPath;
-
-    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());
-                    }
-                }
-            };
-
-    // 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;
 
-    public ShardCommitCoordinator(DOMTransactionFactory transactionFactory,
-            long cacheExpiryTimeoutInSec, int queueCapacity, ActorRef shardActor, Logger log, String name) {
+    private ReadyTransactionReply readyTransactionReply;
 
-        this.queueCapacity = queueCapacity;
+    ShardCommitCoordinator(final ShardDataTree dataTree, final Logger log, final String name) {
         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();
+    int getCohortCacheSize() {
+        return cohortCache.size();
+    }
 
-        // 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<>();
+    private String persistenceId() {
+        return dataTree.logContext();
     }
 
-    public void setQueueCapacity(int queueCapacity) {
-        this.queueCapacity = queueCapacity;
+    private ReadyTransactionReply readyTransactionReply(final ActorRef cohort) {
+        if (readyTransactionReply == null) {
+            readyTransactionReply = new ReadyTransactionReply(Serialization.serializedActorPath(cohort));
+        }
+
+        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
+     * @param ready the ForwardedReadyTransaction message to process
+     * @param sender the sender of the message
+     * @param shard the transaction's shard actor
      */
-    public void transactionReady(String transactionID, DOMStoreThreePhaseCommitCohort cohort,
-            MutableCompositeModification modification) {
-
-        cohortCache.put(transactionID, new CohortEntry(transactionID, cohort, modification));
+    void handleForwardedReadyTransaction(final ForwardedReadyTransaction ready, final ActorRef sender,
+            final Shard shard) {
+        log.debug("{}: Readying transaction {}, client version {}", name,
+                ready.getTransactionId(), ready.getTxnClientVersion());
+
+        final ShardDataTreeCohort cohort = ready.getTransaction().ready();
+        final CohortEntry cohortEntry = CohortEntry.createReady(cohort, ready.getTxnClientVersion());
+        cohortCache.put(cohortEntry.getTransactionId(), cohortEntry);
+
+        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.self()), shard.self());
+        }
     }
 
     /**
@@ -114,271 +125,359 @@ public class ShardCommitCoordinator {
      * DOMStoreWriteTransaction, one is created. The batched modifications are applied to the write Tx. If
      * the BatchedModifications is ready to commit then a DOMStoreThreePhaseCommitCohort is created.
      *
-     * @param batched the BatchedModifications
-     * @param shardActor the transaction's shard actor
-     *
-     * @throws ExecutionException if an error occurs loading the cache
+     * @param batched the BatchedModifications message to process
+     * @param sender the sender of the message
      */
-    public BatchedModificationsReply handleTransactionModifications(BatchedModifications batched)
-            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(),
-                        batched.getTransactionChainID()));
-            cohortCache.put(batched.getTransactionID(), cohortEntry);
+    void handleBatchedModifications(final BatchedModifications batched, final ActorRef sender, final Shard shard) {
+        CohortEntry cohortEntry = cohortCache.get(batched.getTransactionId());
+        if (cohortEntry == null) {
+            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());
 
-        String cohortPath = null;
-        if(batched.isReady()) {
-            if(log.isDebugEnabled()) {
+        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 (log.isDebugEnabled()) {
                 log.debug("{}: Readying Tx {}, client version {}", name,
-                        batched.getTransactionID(), batched.getVersion());
+                        batched.getTransactionId(), batched.getVersion());
             }
 
+            cohortEntry.setDoImmediateCommit(batched.isDoCommitOnReady());
             cohortEntry.ready(cohortDecorator);
-            cohortPath = shardActorPath;
-        }
 
-        return new BatchedModificationsReply(batched.getModifications().size(), cohortPath);
+            if (batched.isDoCommitOnReady()) {
+                cohortEntry.setReplySender(sender);
+                cohortEntry.setShard(shard);
+                handleCanCommit(cohortEntry);
+            } else {
+                sender.tell(readyTransactionReply(shard.self()), shard.self());
+            }
+        } else {
+            sender.tell(new BatchedModificationsReply(batched.getModifications().size()), shard.self());
+        }
     }
 
     /**
-     * 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 message the ReadyLocalTransaction message to process
+     * @param sender the sender of the message
      * @param shard the transaction's shard actor
      */
-    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(final ReadyLocalTransaction message, final ActorRef sender, final Shard shard) {
+        final ShardDataTreeCohort cohort = dataTree.createReadyCohort(message.getTransactionId(),
+            message.getModification());
+        final CohortEntry cohortEntry = CohortEntry.createReady(cohort, DataStoreVersions.CURRENT_VERSION);
+        cohortCache.put(cohortEntry.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.self()), 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;
+    Collection<BatchedModifications> createForwardedBatchedModifications(final BatchedModifications from,
+            final int maxModificationsPerBatch) {
+        CohortEntry cohortEntry = cohortCache.remove(from.getTransactionId());
+        if (cohortEntry == null || cohortEntry.getTransaction() == null) {
+            return Collections.singletonList(from);
         }
 
-        cohortEntry.setCanCommitSender(sender);
-        cohortEntry.setShard(shard);
+        cohortEntry.applyModifications(from.getModifications());
 
-        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);
+        final LinkedList<BatchedModifications> newModifications = new LinkedList<>();
+        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()));
+                }
 
-            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());
-                sender.tell(new Status.Failure(ex), shard);
+                return newModifications.getLast();
             }
-        } else {
-            // No Tx commit currently in progress - make this the current entry and proceed with
-            // canCommit.
-            cohortEntry.updateLastAccessTime();
-            currentCohortEntry = cohortEntry;
+        });
 
-            doCanCommit(cohortEntry);
-        }
+        BatchedModifications last = newModifications.getLast();
+        last.setDoCommitOnReady(from.isDoCommitOnReady());
+        last.setReady(from.isReady());
+        last.setTotalMessagesSent(newModifications.size());
+        return newModifications;
     }
 
-    private void doCanCommit(final CohortEntry cohortEntry) {
-
-        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();
+    private void handleCanCommit(final CohortEntry cohortEntry) {
+        cohortEntry.canCommit(new FutureCallback<Void>() {
+            @Override
+            public void onSuccess(final Void result) {
+                log.debug("{}: canCommit for {}: success", name, cohortEntry.getTransactionId());
+
+                if (cohortEntry.isDoImmediateCommit()) {
+                    doCommit(cohortEntry);
+                } else {
+                    cohortEntry.getReplySender().tell(
+                        CanCommitTransactionReply.yes(cohortEntry.getClientVersion()).toSerializable(),
+                        cohortEntry.getShard().self());
+                }
+            }
 
-            cohortEntry.getCanCommitSender().tell(
-                    canCommit ? CanCommitTransactionReply.YES.toSerializable() :
-                        CanCommitTransactionReply.NO.toSerializable(), cohortEntry.getShard());
+            @Override
+            public void onFailure(final Throwable failure) {
+                log.debug("{}: An exception occurred during canCommit for {}: {}", name,
+                        cohortEntry.getTransactionId(), failure);
 
-            if(!canCommit) {
-                // Remove the entry from the cache now since the Tx will be aborted.
-                removeCohortEntry(cohortEntry.getTransactionID());
+                cohortCache.remove(cohortEntry.getTransactionId());
+                cohortEntry.getReplySender().tell(new Failure(failure), cohortEntry.getShard().self());
             }
-        } catch (InterruptedException | ExecutionException e) {
-            log.debug("{}: An exception occurred during canCommit: {}", name, e);
-
-            // 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());
-        }
+        });
     }
 
     /**
-     * Returns the cohort entry for the Tx commit currently in progress if the given transaction ID
-     * matches the current entry.
+     * This method handles the canCommit phase for a transaction.
      *
-     * @param transactionID the ID of the transaction
-     * @return the current CohortEntry or null if the given transaction ID does not match the
-     *         current entry.
+     * @param transactionID the ID of the transaction to canCommit
+     * @param sender the actor to which to send the response
+     * @param shard the transaction's shard actor
      */
-    public CohortEntry getCohortEntryIfCurrent(String transactionID) {
-        if(isCurrentTransaction(transactionID)) {
-            return currentCohortEntry;
+    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 or it was aborted.
+            IllegalStateException ex = new IllegalStateException(
+                    String.format("%s: Cannot canCommit transaction %s - no cohort entry found", name, transactionID));
+            log.error(ex.getMessage());
+            sender.tell(new Failure(ex), shard.self());
+            return;
         }
 
-        return null;
-    }
+        cohortEntry.setReplySender(sender);
+        cohortEntry.setShard(shard);
 
-    public CohortEntry getCurrentCohortEntry() {
-        return currentCohortEntry;
+        handleCanCommit(cohortEntry);
     }
 
-    public CohortEntry getAndRemoveCohortEntry(String transactionID) {
-        CohortEntry cohortEntry = cohortCache.getIfPresent(transactionID);
-        cohortCache.invalidate(transactionID);
-        return cohortEntry;
-    }
+    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);
+            }
 
-    public void removeCohortEntry(String transactionID) {
-        cohortCache.invalidate(transactionID);
+            @Override
+            public void onFailure(final Throwable failure) {
+                log.error("{} An exception occurred while preCommitting transaction {}", name,
+                        cohortEntry.getTransactionId(), failure);
+
+                cohortCache.remove(cohortEntry.getTransactionId());
+                cohortEntry.getReplySender().tell(new Failure(failure), cohortEntry.getShard().self());
+            }
+        });
     }
 
-    public boolean isCurrentTransaction(String transactionID) {
-        return currentCohortEntry != null &&
-                currentCohortEntry.getTransactionID().equals(transactionID);
+    void finishCommit(@Nonnull final ActorRef sender, @Nonnull final CohortEntry cohortEntry) {
+        log.debug("{}: Finishing commit for transaction {}", persistenceId(), cohortEntry.getTransactionId());
+
+        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());
+            }
+
+            @Override
+            public void onFailure(final Throwable failure) {
+                log.error("{}, An exception occurred while committing transaction {}", persistenceId(),
+                        cohortEntry.getTransactionId(), failure);
+
+                cohortCache.remove(cohortEntry.getTransactionId());
+                sender.tell(new Failure(failure), cohortEntry.getShard().self());
+            }
+        });
     }
 
     /**
-     * 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.
+     * This method handles the preCommit and commit phases for a transaction.
      *
-     * @param transactionID the ID of the completed transaction
-     * @param removeCohortEntry if true the CohortEntry for the transaction is also removed from
-     *        the cache.
+     * @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
      */
-    public void currentTransactionComplete(String transactionID, boolean removeCohortEntry) {
-        if(removeCohortEntry) {
-            removeCohortEntry(transactionID);
+    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 - no cohort entry found", name, transactionID));
+            log.error(ex.getMessage());
+            sender.tell(new Failure(ex), shard.self());
+            return;
         }
 
-        if(isCurrentTransaction(transactionID)) {
-            // Dequeue the next cohort entry waiting in the queue.
-            currentCohortEntry = queuedCohortEntries.poll();
-            if(currentCohortEntry != null) {
-                currentCohortEntry.updateLastAccessTime();
-                doCanCommit(currentCohortEntry);
-            }
-        }
+        cohortEntry.setReplySender(sender);
+        doCommit(cohortEntry);
     }
 
-    @VisibleForTesting
-    void setCohortDecorator(CohortDecorator cohortDecorator) {
-        this.cohortDecorator = cohortDecorator;
-    }
+    @SuppressWarnings("checkstyle:IllegalCatch")
+    void handleAbort(final Identifier transactionID, final ActorRef sender, final Shard shard) {
+        CohortEntry cohortEntry = cohortCache.remove(transactionID);
+        if (cohortEntry == null) {
+            return;
+        }
 
+        log.debug("{}: Aborting transaction {}", name, transactionID);
 
-    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 long lastAccessTime;
+        final ActorRef self = shard.getSelf();
+        cohortEntry.abort(new FutureCallback<Void>() {
+            @Override
+            public void onSuccess(final Void result) {
+                if (sender != null) {
+                    sender.tell(AbortTransactionReply.instance(cohortEntry.getClientVersion()).toSerializable(), self);
+                }
+            }
 
-        CohortEntry(String transactionID, DOMStoreWriteTransaction transaction) {
-            this.compositeModification = new MutableCompositeModification();
-            this.transaction = transaction;
-            this.transactionID = transactionID;
-        }
+            @Override
+            public void onFailure(final Throwable failure) {
+                log.error("{}: An exception happened during abort", name, failure);
 
-        CohortEntry(String transactionID, DOMStoreThreePhaseCommitCohort cohort,
-                MutableCompositeModification compositeModification) {
-            this.transactionID = transactionID;
-            this.cohort = cohort;
-            this.compositeModification = compositeModification;
-            this.transaction = null;
-        }
+                if (sender != null) {
+                    sender.tell(new Failure(failure), self);
+                }
+            }
+        });
 
-        void updateLastAccessTime() {
-            lastAccessTime = System.currentTimeMillis();
-        }
+        shard.getShardMBean().incrementAbortTransactionsCount();
+    }
 
-        long getLastAccessTime() {
-            return lastAccessTime;
+    void checkForExpiredTransactions(final long timeout, final Shard shard) {
+        Iterator<CohortEntry> iter = cohortCache.values().iterator();
+        while (iter.hasNext()) {
+            CohortEntry cohortEntry = iter.next();
+            if (cohortEntry.isFailed()) {
+                iter.remove();
+            }
         }
+    }
 
-        String getTransactionID() {
-            return transactionID;
-        }
+    void abortPendingTransactions(final String reason, final Shard shard) {
+        final Failure failure = new Failure(new RuntimeException(reason));
+        Collection<ShardDataTreeCohort> pending = dataTree.getAndClearPendingTransactions();
 
-        DOMStoreThreePhaseCommitCohort getCohort() {
-            return cohort;
-        }
+        log.debug("{}: Aborting {} pending queued transactions", name, pending.size());
 
-        MutableCompositeModification getModification() {
-            return compositeModification;
-        }
+        for (ShardDataTreeCohort cohort : pending) {
+            CohortEntry cohortEntry = cohortCache.remove(cohort.getIdentifier());
+            if (cohortEntry == null) {
+                continue;
+            }
 
-        void applyModifications(Iterable<Modification> modifications) {
-            for(Modification modification: modifications) {
-                compositeModification.addModification(modification);
-                modification.apply(transaction);
+            if (cohortEntry.getReplySender() != null) {
+                cohortEntry.getReplySender().tell(failure, shard.self());
             }
         }
 
-        void ready(CohortDecorator cohortDecorator) {
-            Preconditions.checkState(cohort == null, "cohort was already set");
-
-            cohort = transaction.ready();
+        cohortCache.clear();
+    }
 
-            if(cohortDecorator != null) {
-                // Call the hook for unit tests.
-                cohort = cohortDecorator.decorate(transactionID, cohort);
+    Collection<?> convertPendingTransactionsToMessages(final int maxModificationsPerBatch) {
+        final Collection<VersionedExternalizableMessage> messages = new ArrayList<>();
+        for (ShardDataTreeCohort cohort : dataTree.getAndClearPendingTransactions()) {
+            CohortEntry cohortEntry = cohortCache.remove(cohort.getIdentifier());
+            if (cohortEntry == null) {
+                continue;
             }
-        }
 
-        ActorRef getCanCommitSender() {
-            return canCommitSender;
-        }
+            final Deque<BatchedModifications> newMessages = new ArrayDeque<>();
+            cohortEntry.getDataTreeModification().applyToCursor(new AbstractBatchedModificationsCursor() {
+                @Override
+                protected BatchedModifications getModifications() {
+                    final BatchedModifications lastBatch = newMessages.peekLast();
 
-        void setCanCommitSender(ActorRef canCommitSender) {
-            this.canCommitSender = canCommitSender;
-        }
+                    if (lastBatch != null && lastBatch.getModifications().size() >= maxModificationsPerBatch) {
+                        return lastBatch;
+                    }
 
-        ActorRef getShard() {
-            return shard;
+                    // Allocate a new message
+                    final BatchedModifications ret = new BatchedModifications(cohortEntry.getTransactionId(),
+                        cohortEntry.getClientVersion());
+                    newMessages.add(ret);
+                    return ret;
+                }
+            });
+
+            final BatchedModifications last = newMessages.peekLast();
+            if (last != null) {
+                final boolean immediate = cohortEntry.isDoImmediateCommit();
+                last.setDoCommitOnReady(immediate);
+                last.setReady(true);
+                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;
+                    }
+                }
+            }
         }
 
-        void setShard(ActorRef shard) {
-            this.shard = shard;
-        }
+        return messages;
+    }
 
-        boolean hasModifications(){
-            return compositeModification.getModifications().size() > 0;
-        }
+    @VisibleForTesting
+    void setCohortDecorator(final CohortDecorator cohortDecorator) {
+        this.cohortDecorator = cohortDecorator;
     }
 }