Remove deprecated ShardDataTree#commit method
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / ShardDataTree.java
index 78b49a60ae5cef444b28633fd9b8fe397d117914..694be4d1d16ac1e723359531fac964908bfdc3f3 100644 (file)
@@ -14,7 +14,6 @@ import com.google.common.base.MoreObjects;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Stopwatch;
-import com.google.common.base.Ticker;
 import com.google.common.base.Verify;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
@@ -24,7 +23,6 @@ import com.google.common.primitives.UnsignedLong;
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import java.io.File;
 import java.io.IOException;
-import java.util.AbstractMap.SimpleEntry;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -39,13 +37,21 @@ import java.util.concurrent.TimeoutException;
 import java.util.function.Consumer;
 import java.util.function.UnaryOperator;
 import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 import javax.annotation.concurrent.NotThreadSafe;
 import org.opendaylight.controller.cluster.access.concepts.LocalHistoryIdentifier;
 import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
 import org.opendaylight.controller.cluster.datastore.DataTreeCohortActorRegistry.CohortRegistryCommand;
 import org.opendaylight.controller.cluster.datastore.ShardDataTreeCohort.State;
+import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
+import org.opendaylight.controller.cluster.datastore.persisted.AbortTransactionPayload;
+import org.opendaylight.controller.cluster.datastore.persisted.AbstractIdentifiablePayload;
+import org.opendaylight.controller.cluster.datastore.persisted.CloseLocalHistoryPayload;
 import org.opendaylight.controller.cluster.datastore.persisted.CommitTransactionPayload;
+import org.opendaylight.controller.cluster.datastore.persisted.CreateLocalHistoryPayload;
 import org.opendaylight.controller.cluster.datastore.persisted.MetadataShardDataTreeSnapshot;
+import org.opendaylight.controller.cluster.datastore.persisted.PurgeLocalHistoryPayload;
+import org.opendaylight.controller.cluster.datastore.persisted.PurgeTransactionPayload;
 import org.opendaylight.controller.cluster.datastore.persisted.ShardDataTreeSnapshot;
 import org.opendaylight.controller.cluster.datastore.persisted.ShardDataTreeSnapshotMetadata;
 import org.opendaylight.controller.cluster.datastore.utils.DataTreeModificationOutput;
@@ -56,7 +62,6 @@ import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListene
 import org.opendaylight.controller.md.sal.common.api.data.OptimisticLockFailedException;
 import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
 import org.opendaylight.controller.md.sal.dom.api.DOMDataTreeChangeListener;
-import org.opendaylight.controller.md.sal.dom.store.impl.DataChangeListenerRegistration;
 import org.opendaylight.yangtools.concepts.Identifier;
 import org.opendaylight.yangtools.concepts.ListenerRegistration;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
@@ -69,7 +74,6 @@ import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeSnapshot;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeTip;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException;
-import org.opendaylight.yangtools.yang.data.api.schema.tree.ModificationType;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.TipProducingDataTree;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.TipProducingDataTreeTip;
 import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType;
@@ -102,11 +106,25 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
     private static final Timeout COMMIT_STEP_TIMEOUT = new Timeout(Duration.create(5, TimeUnit.SECONDS));
     private static final Logger LOG = LoggerFactory.getLogger(ShardDataTree.class);
 
+    /**
+     * Process this many transactions in a single batched run. If we exceed this limit, we need to schedule later
+     * execution to finish up the batch. This is necessary in case of a long list of transactions which progress
+     * immediately through their preCommit phase -- if that happens, their completion eats up stack frames and could
+     * result in StackOverflowError.
+     */
+    private static final int MAX_TRANSACTION_BATCH = 100;
+
     private final Map<LocalHistoryIdentifier, ShardDataTreeTransactionChain> transactionChains = new HashMap<>();
     private final DataTreeCohortActorRegistry cohortRegistry = new DataTreeCohortActorRegistry();
     private final Queue<CommitEntry> pendingTransactions = new ArrayDeque<>();
     private final Queue<CommitEntry> pendingCommits = new ArrayDeque<>();
     private final Queue<CommitEntry> pendingFinishCommits = new ArrayDeque<>();
+
+    /**
+     * Callbacks that need to be invoked once a payload is replicated.
+     */
+    private final Map<Payload, Runnable> replicationCallbacks = new HashMap<>();
+
     private final ShardDataTreeChangeListenerPublisher treeChangeListenerPublisher;
     private final ShardDataChangeListenerPublisher dataChangeListenerPublisher;
     private final Collection<ShardDataTreeMetadata<?>> metadata;
@@ -125,7 +143,9 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
 
     private SchemaContext schemaContext;
 
-    public ShardDataTree(final Shard shard, final SchemaContext schemaContext, final TipProducingDataTree dataTree,
+    private int currentTransactionBatch;
+
+    ShardDataTree(final Shard shard, final SchemaContext schemaContext, final TipProducingDataTree dataTree,
             final ShardDataTreeChangeListenerPublisher treeChangeListenerPublisher,
             final ShardDataChangeListenerPublisher dataChangeListenerPublisher, final String logContext,
             final ShardDataTreeMetadata<?>... metadata) {
@@ -140,27 +160,28 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         tip = dataTree;
     }
 
-    public ShardDataTree(final Shard shard, final SchemaContext schemaContext, final TreeType treeType,
+    ShardDataTree(final Shard shard, final SchemaContext schemaContext, final TreeType treeType,
             final YangInstanceIdentifier root,
             final ShardDataTreeChangeListenerPublisher treeChangeListenerPublisher,
-            final ShardDataChangeListenerPublisher dataChangeListenerPublisher, final String logContext) {
+            final ShardDataChangeListenerPublisher dataChangeListenerPublisher, final String logContext,
+            final ShardDataTreeMetadata<?>... metadata) {
         this(shard, schemaContext, InMemoryDataTreeFactory.getInstance().create(treeType, root),
-                treeChangeListenerPublisher, dataChangeListenerPublisher, logContext);
+                treeChangeListenerPublisher, dataChangeListenerPublisher, logContext, metadata);
     }
 
     @VisibleForTesting
     public ShardDataTree(final Shard shard, final SchemaContext schemaContext, final TreeType treeType) {
         this(shard, schemaContext, treeType, YangInstanceIdentifier.EMPTY,
-                new DefaultShardDataTreeChangeListenerPublisher(),
-                new DefaultShardDataChangeListenerPublisher(), "");
+                new DefaultShardDataTreeChangeListenerPublisher(""),
+                new DefaultShardDataChangeListenerPublisher(""), "");
     }
 
     final String logContext() {
         return logContext;
     }
 
-    final Ticker ticker() {
-        return shard.ticker();
+    final long readTime() {
+        return shard.ticker().read();
     }
 
     public TipProducingDataTree getDataTree() {
@@ -176,6 +197,10 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         this.schemaContext = Preconditions.checkNotNull(newSchemaContext);
     }
 
+    void resetTransactionBatch() {
+        currentTransactionBatch = 0;
+    }
+
     /**
      * Take a snapshot of current state for later recovery.
      *
@@ -241,7 +266,7 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         dataTree.commit(candidate);
         notifyListeners(candidate);
 
-        LOG.debug("{}: state snapshot applied in %s", logContext, elapsed);
+        LOG.debug("{}: state snapshot applied in {}", logContext, elapsed);
     }
 
     /**
@@ -313,8 +338,16 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
                     ((CommitTransactionPayload) payload).getCandidate();
             applyRecoveryCandidate(e.getValue());
             allMetadataCommittedTransaction(e.getKey());
-        } else if (payload instanceof DataTreeCandidatePayload) {
-            applyRecoveryCandidate(((DataTreeCandidatePayload) payload).getCandidate());
+        } else if (payload instanceof AbortTransactionPayload) {
+            allMetadataAbortedTransaction(((AbortTransactionPayload) payload).getIdentifier());
+        } else if (payload instanceof PurgeTransactionPayload) {
+            allMetadataPurgedTransaction(((PurgeTransactionPayload) payload).getIdentifier());
+        } else if (payload instanceof CreateLocalHistoryPayload) {
+            allMetadataCreatedLocalHistory(((CreateLocalHistoryPayload) payload).getIdentifier());
+        } else if (payload instanceof CloseLocalHistoryPayload) {
+            allMetadataClosedLocalHistory(((CloseLocalHistoryPayload) payload).getIdentifier());
+        } else if (payload instanceof PurgeLocalHistoryPayload) {
+            allMetadataPurgedLocalHistory(((PurgeLocalHistoryPayload) payload).getIdentifier());
         } else {
             LOG.debug("{}: ignoring unhandled payload {}", logContext, payload);
         }
@@ -359,20 +392,65 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
          * pre-Boron state -- which limits the number of options here.
          */
         if (payload instanceof CommitTransactionPayload) {
+            final TransactionIdentifier txId;
             if (identifier == null) {
                 final Entry<TransactionIdentifier, DataTreeCandidate> e =
                         ((CommitTransactionPayload) payload).getCandidate();
-                applyReplicatedCandidate(e.getKey(), e.getValue());
-                allMetadataCommittedTransaction(e.getKey());
+                txId = e.getKey();
+                applyReplicatedCandidate(txId, e.getValue());
             } else {
                 Verify.verify(identifier instanceof TransactionIdentifier);
-                payloadReplicationComplete((TransactionIdentifier) identifier);
+                txId = (TransactionIdentifier) identifier;
+                payloadReplicationComplete(txId);
+            }
+            allMetadataCommittedTransaction(txId);
+        } else if (payload instanceof AbortTransactionPayload) {
+            if (identifier != null) {
+                payloadReplicationComplete((AbortTransactionPayload) payload);
+            }
+            allMetadataAbortedTransaction(((AbortTransactionPayload) payload).getIdentifier());
+        } else if (payload instanceof PurgeTransactionPayload) {
+            if (identifier != null) {
+                payloadReplicationComplete((PurgeTransactionPayload) payload);
+            }
+            allMetadataPurgedTransaction(((PurgeTransactionPayload) payload).getIdentifier());
+        } else if (payload instanceof CloseLocalHistoryPayload) {
+            if (identifier != null) {
+                payloadReplicationComplete((CloseLocalHistoryPayload) payload);
             }
+            allMetadataClosedLocalHistory(((CloseLocalHistoryPayload) payload).getIdentifier());
+        } else if (payload instanceof CreateLocalHistoryPayload) {
+            if (identifier != null) {
+                payloadReplicationComplete((CreateLocalHistoryPayload)payload);
+            }
+            allMetadataCreatedLocalHistory(((CreateLocalHistoryPayload) payload).getIdentifier());
+        } else if (payload instanceof PurgeLocalHistoryPayload) {
+            if (identifier != null) {
+                payloadReplicationComplete((PurgeLocalHistoryPayload)payload);
+            }
+            allMetadataPurgedLocalHistory(((PurgeLocalHistoryPayload) payload).getIdentifier());
         } else {
             LOG.warn("{}: ignoring unhandled identifier {} payload {}", logContext, identifier, payload);
         }
     }
 
+    private void replicatePayload(final Identifier id, final Payload payload, @Nullable final Runnable callback) {
+        if (callback != null) {
+            replicationCallbacks.put(payload, callback);
+        }
+        shard.persistPayload(id, payload, true);
+    }
+
+    private void payloadReplicationComplete(final AbstractIdentifiablePayload<?> payload) {
+        final Runnable callback = replicationCallbacks.remove(payload);
+        if (callback != null) {
+            LOG.debug("{}: replication of {} completed, invoking {}", logContext, payload.getIdentifier(), callback);
+            callback.run();
+        } else {
+            LOG.debug("{}: replication of {} has no callback", logContext, payload.getIdentifier());
+        }
+    }
+
     private void payloadReplicationComplete(final TransactionIdentifier txId) {
         final CommitEntry current = pendingFinishCommits.peek();
         if (current == null) {
@@ -389,17 +467,68 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         finishCommit(current.cohort);
     }
 
+    private void allMetadataAbortedTransaction(final TransactionIdentifier txId) {
+        for (ShardDataTreeMetadata<?> m : metadata) {
+            m.onTransactionAborted(txId);
+        }
+    }
+
     private void allMetadataCommittedTransaction(final TransactionIdentifier txId) {
         for (ShardDataTreeMetadata<?> m : metadata) {
             m.onTransactionCommitted(txId);
         }
     }
 
-    ShardDataTreeTransactionChain ensureTransactionChain(final LocalHistoryIdentifier localHistoryIdentifier) {
-        ShardDataTreeTransactionChain chain = transactionChains.get(localHistoryIdentifier);
+    private void allMetadataPurgedTransaction(final TransactionIdentifier txId) {
+        for (ShardDataTreeMetadata<?> m : metadata) {
+            m.onTransactionPurged(txId);
+        }
+    }
+
+    private void allMetadataCreatedLocalHistory(final LocalHistoryIdentifier historyId) {
+        for (ShardDataTreeMetadata<?> m : metadata) {
+            m.onHistoryCreated(historyId);
+        }
+    }
+
+    private void allMetadataClosedLocalHistory(final LocalHistoryIdentifier historyId) {
+        for (ShardDataTreeMetadata<?> m : metadata) {
+            m.onHistoryClosed(historyId);
+        }
+    }
+
+    private void allMetadataPurgedLocalHistory(final LocalHistoryIdentifier historyId) {
+        for (ShardDataTreeMetadata<?> m : metadata) {
+            m.onHistoryPurged(historyId);
+        }
+    }
+
+    /**
+     * Create a transaction chain for specified history. Unlike {@link #ensureTransactionChain(LocalHistoryIdentifier)},
+     * this method is used for re-establishing state when we are taking over
+     *
+     * @param historyId Local history identifier
+     * @param closed True if the chain should be created in closed state (i.e. pending purge)
+     * @return Transaction chain handle
+     */
+    ShardDataTreeTransactionChain recreateTransactionChain(final LocalHistoryIdentifier historyId,
+            final boolean closed) {
+        final ShardDataTreeTransactionChain ret = new ShardDataTreeTransactionChain(historyId, this);
+        final ShardDataTreeTransactionChain existing = transactionChains.putIfAbsent(historyId, ret);
+        Preconditions.checkState(existing == null, "Attempted to recreate chain %s, but %s already exists", historyId,
+                existing);
+        return ret;
+    }
+
+    ShardDataTreeTransactionChain ensureTransactionChain(final LocalHistoryIdentifier historyId,
+            @Nullable final Runnable callback) {
+        ShardDataTreeTransactionChain chain = transactionChains.get(historyId);
         if (chain == null) {
-            chain = new ShardDataTreeTransactionChain(localHistoryIdentifier, this);
-            transactionChains.put(localHistoryIdentifier, chain);
+            chain = new ShardDataTreeTransactionChain(historyId, this);
+            transactionChains.put(historyId, chain);
+            replicatePayload(historyId, CreateLocalHistoryPayload.create(historyId), callback);
+        } else if (callback != null) {
+            callback.run();
         }
 
         return chain;
@@ -407,10 +536,10 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
 
     ReadOnlyShardDataTreeTransaction newReadOnlyTransaction(final TransactionIdentifier txId) {
         if (txId.getHistoryId().getHistoryId() == 0) {
-            return new ReadOnlyShardDataTreeTransaction(txId, dataTree.takeSnapshot());
+            return new ReadOnlyShardDataTreeTransaction(this, txId, dataTree.takeSnapshot());
         }
 
-        return ensureTransactionChain(txId.getHistoryId()).newReadOnlyTransaction(txId);
+        return ensureTransactionChain(txId.getHistoryId(), null).newReadOnlyTransaction(txId);
     }
 
     ReadWriteShardDataTreeTransaction newReadWriteTransaction(final TransactionIdentifier txId) {
@@ -419,74 +548,86 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
                     .newModification());
         }
 
-        return ensureTransactionChain(txId.getHistoryId()).newReadWriteTransaction(txId);
+        return ensureTransactionChain(txId.getHistoryId(), null).newReadWriteTransaction(txId);
     }
 
     @VisibleForTesting
     public void notifyListeners(final DataTreeCandidate candidate) {
-        treeChangeListenerPublisher.publishChanges(candidate, logContext);
-        dataChangeListenerPublisher.publishChanges(candidate, logContext);
-    }
-
-    void notifyOfInitialData(final DataChangeListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier,
-            NormalizedNode<?, ?>>> listenerReg, final Optional<DataTreeCandidate> currentState) {
-        if (currentState.isPresent()) {
-            ShardDataChangeListenerPublisher localPublisher = dataChangeListenerPublisher.newInstance();
-            localPublisher.registerDataChangeListener(listenerReg.getPath(), listenerReg.getInstance(),
-                    listenerReg.getScope());
-            localPublisher.publishChanges(currentState.get(), logContext);
-        }
-    }
-
-    void notifyOfInitialData(final YangInstanceIdentifier path, final DOMDataTreeChangeListener listener,
-            final Optional<DataTreeCandidate> currentState) {
-        if (currentState.isPresent()) {
-            ShardDataTreeChangeListenerPublisher localPublisher = treeChangeListenerPublisher.newInstance();
-            localPublisher.registerTreeChangeListener(path, listener);
-            localPublisher.publishChanges(currentState.get(), logContext);
-        }
+        treeChangeListenerPublisher.publishChanges(candidate);
+        dataChangeListenerPublisher.publishChanges(candidate);
     }
 
-    void closeAllTransactionChains() {
+    /**
+     * Immediately purge all state relevant to leader. This includes all transaction chains and any scheduled
+     * replication callbacks.
+     */
+    void purgeLeaderState() {
         for (ShardDataTreeTransactionChain chain : transactionChains.values()) {
             chain.close();
         }
 
         transactionChains.clear();
+        replicationCallbacks.clear();
     }
 
-    void closeTransactionChain(final LocalHistoryIdentifier transactionChainId) {
-        final ShardDataTreeTransactionChain chain = transactionChains.remove(transactionChainId);
-        if (chain != null) {
-            chain.close();
-        } else {
-            LOG.debug("{}: Closing non-existent transaction chain {}", logContext, transactionChainId);
+    /**
+     * Close a single transaction chain.
+     *
+     * @param id History identifier
+     * @param callback Callback to invoke upon completion, may be null
+     */
+    void closeTransactionChain(final LocalHistoryIdentifier id, @Nullable final Runnable callback) {
+        final ShardDataTreeTransactionChain chain = transactionChains.get(id);
+        if (chain == null) {
+            LOG.debug("{}: Closing non-existent transaction chain {}", logContext, id);
+            if (callback != null) {
+                callback.run();
+            }
+            return;
         }
+
+        chain.close();
+        replicatePayload(id, CloseLocalHistoryPayload.create(id), callback);
     }
 
-    Entry<DataChangeListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>>,
-            Optional<DataTreeCandidate>> registerChangeListener(final YangInstanceIdentifier path,
-                    final AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>> listener,
-                    final DataChangeScope scope) {
-        DataChangeListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>> reg =
-                dataChangeListenerPublisher.registerDataChangeListener(path, listener, scope);
+    /**
+     * Purge a single transaction chain.
+     *
+     * @param id History identifier
+     * @param callback Callback to invoke upon completion, may be null
+     */
+    void purgeTransactionChain(final LocalHistoryIdentifier id, @Nullable final Runnable callback) {
+        final ShardDataTreeTransactionChain chain = transactionChains.remove(id);
+        if (chain == null) {
+            LOG.debug("{}: Purging non-existent transaction chain {}", logContext, id);
+            if (callback != null) {
+                callback.run();
+            }
+            return;
+        }
+
+        replicatePayload(id, PurgeLocalHistoryPayload.create(id), callback);
+    }
 
-        return new SimpleEntry<>(reg, readCurrentData());
+    void registerDataChangeListener(final YangInstanceIdentifier path,
+            final AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>> listener,
+            final DataChangeScope scope, final Optional<DataTreeCandidate> initialState,
+            final Consumer<ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>>>
+                    onRegistration) {
+        dataChangeListenerPublisher.registerDataChangeListener(path, listener, scope, initialState, onRegistration);
     }
 
-    private Optional<DataTreeCandidate> readCurrentData() {
+    Optional<DataTreeCandidate> readCurrentData() {
         final Optional<NormalizedNode<?, ?>> currentState =
                 dataTree.takeSnapshot().readNode(YangInstanceIdentifier.EMPTY);
         return currentState.isPresent() ? Optional.of(DataTreeCandidates.fromNormalizedNode(
             YangInstanceIdentifier.EMPTY, currentState.get())) : Optional.<DataTreeCandidate>absent();
     }
 
-    public Entry<ListenerRegistration<DOMDataTreeChangeListener>, Optional<DataTreeCandidate>>
-            registerTreeChangeListener(final YangInstanceIdentifier path, final DOMDataTreeChangeListener listener) {
-        final ListenerRegistration<DOMDataTreeChangeListener> reg =
-                treeChangeListenerPublisher.registerTreeChangeListener(path, listener);
-
-        return new SimpleEntry<>(reg, readCurrentData());
+    public void registerTreeChangeListener(final YangInstanceIdentifier path, final DOMDataTreeChangeListener listener,
+            final Optional<DataTreeCandidate> initialState,
+            final Consumer<ListenerRegistration<DOMDataTreeChangeListener>> onRegistration) {
+        treeChangeListenerPublisher.registerTreeChangeListener(path, listener, initialState, onRegistration);
     }
 
     int getQueueSize() {
@@ -494,8 +635,16 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
     }
 
     @Override
-    void abortTransaction(final AbstractShardDataTreeTransaction<?> transaction) {
-        // Intentional no-op
+    void abortTransaction(final AbstractShardDataTreeTransaction<?> transaction, final Runnable callback) {
+        final TransactionIdentifier id = transaction.getIdentifier();
+        LOG.debug("{}: aborting transaction {}", logContext, id);
+        replicatePayload(id, AbortTransactionPayload.create(id), callback);
+    }
+
+    @Override
+    void abortFromTransactionActor(final AbstractShardDataTreeTransaction<?> transaction) {
+        // No-op for free-standing transactions
+
     }
 
     @Override
@@ -506,6 +655,11 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         return createReadyCohort(transaction.getIdentifier(), snapshot);
     }
 
+    void purgeTransaction(final TransactionIdentifier id, final Runnable callback) {
+        LOG.debug("{}: purging transaction {}", logContext, id);
+        replicatePayload(id, PurgeTransactionPayload.create(id), callback);
+    }
+
     public Optional<NormalizedNode<?, ?>> readNode(final YangInstanceIdentifier path) {
         return dataTree.takeSnapshot().readNode(path);
     }
@@ -519,24 +673,6 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         return dataTree.takeSnapshot().newModification();
     }
 
-    /**
-     * Commits a modification.
-     *
-     * @deprecated This method violates DataTree containment and will be removed.
-     */
-    @VisibleForTesting
-    @Deprecated
-    public DataTreeCandidate commit(final DataTreeModification modification) throws DataValidationFailedException {
-        // Direct modification commit is a utility, which cannot be used while we have transactions in-flight
-        Preconditions.checkState(tip == dataTree, "Cannot modify data tree while transacgitons are pending");
-
-        modification.ready();
-        dataTree.validate(modification);
-        DataTreeCandidate candidate = dataTree.prepare(modification);
-        dataTree.commit(candidate);
-        return candidate;
-    }
-
     public Collection<ShardDataTreeCohort> getAndClearPendingTransactions() {
         Collection<ShardDataTreeCohort> ret = new ArrayList<>(getQueueSize());
 
@@ -559,8 +695,23 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         return ret;
     }
 
+    /**
+     * Called some time after {@link #processNextPendingTransaction()} decides to stop processing.
+     */
+    void resumeNextPendingTransaction() {
+        LOG.debug("{}: attempting to resume transaction processing", logContext);
+        processNextPending();
+    }
+
     @SuppressWarnings("checkstyle:IllegalCatch")
     private void processNextPendingTransaction() {
+        ++currentTransactionBatch;
+        if (currentTransactionBatch > MAX_TRANSACTION_BATCH) {
+            LOG.debug("{}: Already processed {}, scheduling continuation", logContext, currentTransactionBatch);
+            shard.scheduleNextPendingTransaction();
+            return;
+        }
+
         processNextPending(pendingTransactions, State.CAN_COMMIT_PENDING, entry -> {
             final SimpleShardDataTreeCohort cohort = entry.cohort;
             final DataTreeModification modification = cohort.getDataTreeModification();
@@ -568,10 +719,12 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
             LOG.debug("{}: Validating transaction {}", logContext, cohort.getIdentifier());
             Exception cause;
             try {
+                cohort.throwCanCommitFailure();
+
                 tip.validate(modification);
                 LOG.debug("{}: Transaction {} validated", logContext, cohort.getIdentifier());
                 cohort.successfulCanCommit();
-                entry.lastAccess = shard.ticker().read();
+                entry.lastAccess = readTime();
                 return;
             } catch (ConflictingModificationAppliedException e) {
                 LOG.warn("{}: Store Tx {}: Conflicting modification for path {}.", logContext, cohort.getIdentifier(),
@@ -597,12 +750,12 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
     }
 
     private void processNextPending() {
-        processNextPendingFinishCommit();
         processNextPendingCommit();
         processNextPendingTransaction();
     }
 
-    private void processNextPending(Queue<CommitEntry> queue, State allowedState, Consumer<CommitEntry> processor) {
+    private void processNextPending(final Queue<CommitEntry> queue, final State allowedState,
+            final Consumer<CommitEntry> processor) {
         while (!queue.isEmpty()) {
             final CommitEntry entry = queue.peek();
             final SimpleShardDataTreeCohort cohort = entry.cohort;
@@ -628,11 +781,6 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
             entry -> startCommit(entry.cohort, entry.cohort.getCandidate()));
     }
 
-    private void processNextPendingFinishCommit() {
-        processNextPending(pendingFinishCommits, State.FINISH_COMMIT_PENDING,
-            entry -> payloadReplicationComplete(entry.cohort.getIdentifier()));
-    }
-
     private boolean peekNextPendingCommit() {
         final CommitEntry first = pendingCommits.peek();
         return first != null && first.cohort.getState() == State.COMMIT_PENDING;
@@ -676,7 +824,7 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         // Set the tip of the data tree.
         tip = Verify.verifyNotNull(candidate);
 
-        entry.lastAccess = shard.ticker().read();
+        entry.lastAccess = readTime();
 
         pendingTransactions.remove();
         pendingCommits.add(entry);
@@ -739,15 +887,6 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         LOG.debug("{}: Starting commit for transaction {}", logContext, current.getIdentifier());
 
         final TransactionIdentifier txId = cohort.getIdentifier();
-        if (shard.canSkipPayload() || candidate.getRootNode().getModificationType() == ModificationType.UNMODIFIED) {
-            LOG.debug("{}: No replication required, proceeding to finish commit", logContext);
-            pendingCommits.remove();
-            pendingFinishCommits.add(entry);
-            cohort.finishCommitPending();
-            payloadReplicationComplete(txId);
-            return;
-        }
-
         final Payload payload;
         try {
             payload = CommitTransactionPayload.create(txId, candidate);
@@ -788,23 +927,45 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         processNextPendingCommit();
     }
 
+    Collection<ActorRef> getCohortActors() {
+        return cohortRegistry.getCohortActors();
+    }
+
     void processCohortRegistryCommand(final ActorRef sender, final CohortRegistryCommand message) {
         cohortRegistry.process(sender, message);
     }
 
+    @Override
+    ShardDataTreeCohort createFailedCohort(final TransactionIdentifier txId, final DataTreeModification mod,
+            final Exception failure) {
+        SimpleShardDataTreeCohort cohort = new SimpleShardDataTreeCohort.DeadOnArrival(this, mod, txId, failure);
+        pendingTransactions.add(new CommitEntry(cohort, readTime()));
+        return cohort;
+    }
+
     @Override
     ShardDataTreeCohort createReadyCohort(final TransactionIdentifier txId,
-            final DataTreeModification modification) {
-        SimpleShardDataTreeCohort cohort = new SimpleShardDataTreeCohort(this, modification, txId,
+            final DataTreeModification mod) {
+        SimpleShardDataTreeCohort cohort = new SimpleShardDataTreeCohort.Normal(this, mod, txId,
                 cohortRegistry.createCohort(schemaContext, txId, COMMIT_STEP_TIMEOUT));
-        pendingTransactions.add(new CommitEntry(cohort, shard.ticker().read()));
+        pendingTransactions.add(new CommitEntry(cohort, readTime()));
         return cohort;
     }
 
+    // Exposed for ShardCommitCoordinator so it does not have deal with local histories (it does not care), this mimics
+    // the newReadWriteTransaction()
+    ShardDataTreeCohort newReadyCohort(final TransactionIdentifier txId, final DataTreeModification mod) {
+        if (txId.getHistoryId().getHistoryId() == 0) {
+            return createReadyCohort(txId, mod);
+        }
+
+        return ensureTransactionChain(txId.getHistoryId(), null).createReadyCohort(txId, mod);
+    }
+
     @SuppressFBWarnings(value = "DB_DUPLICATE_SWITCH_CLAUSES", justification = "See inline comments below.")
     void checkForExpiredTransactions(final long transactionCommitTimeoutMillis) {
         final long timeout = TimeUnit.MILLISECONDS.toNanos(transactionCommitTimeoutMillis);
-        final long now = shard.ticker().read();
+        final long now = readTime();
 
         final Queue<CommitEntry> currentQueue = !pendingFinishCommits.isEmpty() ? pendingFinishCommits :
             !pendingCommits.isEmpty() ? pendingCommits : pendingTransactions;
@@ -916,7 +1077,7 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
     }
 
     @SuppressWarnings("checkstyle:IllegalCatch")
-    private void rebaseTransactions(Iterator<CommitEntry> iter, @Nonnull TipProducingDataTreeTip newTip) {
+    private void rebaseTransactions(final Iterator<CommitEntry> iter, @Nonnull final TipProducingDataTreeTip newTip) {
         tip = Preconditions.checkNotNull(newTip);
         while (iter.hasNext()) {
             final SimpleShardDataTreeCohort cohort = iter.next().cohort;
@@ -961,4 +1122,8 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
             runOnPendingTransactionsComplete = null;
         }
     }
+
+    ShardStats getStats() {
+        return shard.getShardMBean();
+    }
 }