BUG-7033: Implement pipe-lining in ShardDataTree
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / ShardDataTree.java
index a04bf62a96d9e964208e0ac76cd29f104df649a6..064f6f5d8a476c6451ff9b4bd9bce0c4344549e0 100644 (file)
@@ -10,6 +10,7 @@ package org.opendaylight.controller.cluster.datastore;
 import akka.actor.ActorRef;
 import akka.util.Timeout;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.MoreObjects;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Stopwatch;
@@ -18,6 +19,7 @@ import com.google.common.base.Verify;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableMap.Builder;
+import com.google.common.collect.Iterables;
 import com.google.common.primitives.UnsignedLong;
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import java.io.File;
@@ -102,6 +104,8 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
     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<>();
     private final ShardDataTreeChangeListenerPublisher treeChangeListenerPublisher;
     private final ShardDataChangeListenerPublisher dataChangeListenerPublisher;
     private final Collection<ShardDataTreeMetadata<?>> metadata;
@@ -136,15 +140,17 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
     }
 
     public ShardDataTree(final Shard shard, final SchemaContext schemaContext, final TreeType treeType,
+            final YangInstanceIdentifier root,
             final ShardDataTreeChangeListenerPublisher treeChangeListenerPublisher,
             final ShardDataChangeListenerPublisher dataChangeListenerPublisher, final String logContext) {
-        this(shard, schemaContext, InMemoryDataTreeFactory.getInstance().create(treeType),
+        this(shard, schemaContext, InMemoryDataTreeFactory.getInstance().create(treeType, root),
                 treeChangeListenerPublisher, dataChangeListenerPublisher, logContext);
     }
 
     @VisibleForTesting
     public ShardDataTree(final Shard shard, final SchemaContext schemaContext, final TreeType treeType) {
-        this(shard, schemaContext, treeType, new DefaultShardDataTreeChangeListenerPublisher(),
+        this(shard, schemaContext, treeType, YangInstanceIdentifier.EMPTY,
+                new DefaultShardDataTreeChangeListenerPublisher(),
                 new DefaultShardDataChangeListenerPublisher(), "");
     }
 
@@ -189,11 +195,15 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         return new MetadataShardDataTreeSnapshot(rootNode, metaBuilder.build());
     }
 
+    private boolean anyPendingTransactions() {
+        return !pendingTransactions.isEmpty() || !pendingCommits.isEmpty() || !pendingFinishCommits.isEmpty();
+    }
+
     private void applySnapshot(@Nonnull final ShardDataTreeSnapshot snapshot,
             final UnaryOperator<DataTreeModification> wrapper) throws DataValidationFailedException {
         final Stopwatch elapsed = Stopwatch.createStarted();
 
-        if (!pendingTransactions.isEmpty()) {
+        if (anyPendingTransactions()) {
             LOG.warn("{}: applying state snapshot with pending transactions", logContext);
         }
 
@@ -363,7 +373,7 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
     }
 
     private void payloadReplicationComplete(final TransactionIdentifier txId) {
-        final CommitEntry current = pendingTransactions.peek();
+        final CommitEntry current = pendingFinishCommits.peek();
         if (current == null) {
             LOG.warn("{}: No outstanding transactions, ignoring consensus on transaction {}", logContext, txId);
             return;
@@ -479,7 +489,7 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
     }
 
     int getQueueSize() {
-        return pendingTransactions.size();
+        return pendingTransactions.size() + pendingCommits.size() + pendingFinishCommits.size();
     }
 
     @Override
@@ -527,23 +537,41 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
     }
 
     public Collection<ShardDataTreeCohort> getAndClearPendingTransactions() {
-        Collection<ShardDataTreeCohort> ret = new ArrayList<>(pendingTransactions.size());
+        Collection<ShardDataTreeCohort> ret = new ArrayList<>(pendingTransactions.size() + pendingCommits.size()
+                + pendingFinishCommits.size());
+
+        for (CommitEntry entry: pendingFinishCommits) {
+            ret.add(entry.cohort);
+        }
+
+        for (CommitEntry entry: pendingCommits) {
+            ret.add(entry.cohort);
+        }
+
         for (CommitEntry entry: pendingTransactions) {
             ret.add(entry.cohort);
         }
 
+        pendingFinishCommits.clear();
+        pendingCommits.clear();
         pendingTransactions.clear();
         tip = dataTree;
         return ret;
     }
 
     @SuppressWarnings("checkstyle:IllegalCatch")
-    private void processNextTransaction() {
+    private void processNextPendingTransaction() {
         while (!pendingTransactions.isEmpty()) {
             final CommitEntry entry = pendingTransactions.peek();
             final SimpleShardDataTreeCohort cohort = entry.cohort;
             final DataTreeModification modification = cohort.getDataTreeModification();
 
+            if (cohort.isFailed()) {
+                LOG.debug("{}: Removing failed transaction {}", logContext, cohort.getIdentifier());
+                pendingTransactions.remove();
+                continue;
+            }
+
             if (cohort.getState() != State.CAN_COMMIT_PENDING) {
                 break;
             }
@@ -581,6 +609,32 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         maybeRunOperationOnPendingTransactionsComplete();
     }
 
+    private void processNextPendingCommit() {
+        while (!pendingCommits.isEmpty()) {
+            final CommitEntry entry = pendingCommits.peek();
+            final SimpleShardDataTreeCohort cohort = entry.cohort;
+
+            if (cohort.isFailed()) {
+                LOG.debug("{}: Removing failed transaction {}", logContext, cohort.getIdentifier());
+                pendingCommits.remove();
+                continue;
+            }
+
+            if (cohort.getState() == State.COMMIT_PENDING) {
+                startCommit(cohort, cohort.getCandidate());
+            }
+
+            break;
+        }
+
+        maybeRunOperationOnPendingTransactionsComplete();
+    }
+
+    private void processNextPending() {
+        processNextPendingCommit();
+        processNextPendingTransaction();
+    }
+
     void startCanCommit(final SimpleShardDataTreeCohort cohort) {
         final SimpleShardDataTreeCohort current = pendingTransactions.peek().cohort;
         if (!cohort.equals(current)) {
@@ -588,13 +642,13 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
             return;
         }
 
-        processNextTransaction();
+        processNextPendingTransaction();
     }
 
     private void failPreCommit(final Exception cause) {
         shard.getShardMBean().incrementFailedTransactionsCount();
         pendingTransactions.poll().cohort.failedPreCommit(cause);
-        processNextTransaction();
+        processNextPendingTransaction();
     }
 
     @SuppressWarnings("checkstyle:IllegalCatch")
@@ -623,13 +677,18 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         tip = Verify.verifyNotNull(candidate);
 
         entry.lastAccess = shard.ticker().read();
+
+        pendingTransactions.remove();
+        pendingCommits.add(entry);
         cohort.successfulPreCommit(candidate);
+
+        processNextPendingTransaction();
     }
 
     private void failCommit(final Exception cause) {
         shard.getShardMBean().incrementFailedTransactionsCount();
-        pendingTransactions.poll().cohort.failedCommit(cause);
-        processNextTransaction();
+        pendingFinishCommits.poll().cohort.failedCommit(cause);
+        processNextPending();
     }
 
     @SuppressWarnings("checkstyle:IllegalCatch")
@@ -639,6 +698,11 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
 
         LOG.debug("{}: Resuming commit of transaction {}", logContext, txId);
 
+        if (tip == candidate) {
+            // All pending candidates have been committed, reset the tip to the data tree.
+            tip = dataTree;
+        }
+
         try {
             dataTree.commit(candidate);
         } catch (Exception e) {
@@ -647,32 +711,32 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
             return;
         }
 
-        // All pending candidates have been committed, reset the tip to the data tree
-        if (tip == candidate) {
-            tip = dataTree;
-        }
-
         shard.getShardMBean().incrementCommittedTransactionCount();
         shard.getShardMBean().setLastCommittedTransactionTime(System.currentTimeMillis());
 
         // FIXME: propagate journal index
-        pendingTransactions.poll().cohort.successfulCommit(UnsignedLong.ZERO);
+        pendingFinishCommits.poll().cohort.successfulCommit(UnsignedLong.ZERO);
 
         LOG.trace("{}: Transaction {} committed, proceeding to notify", logContext, txId);
         notifyListeners(candidate);
 
-        processNextTransaction();
+        processNextPending();
     }
 
     void startCommit(final SimpleShardDataTreeCohort cohort, final DataTreeCandidate candidate) {
-        final CommitEntry entry = pendingTransactions.peek();
+        final CommitEntry entry = pendingCommits.peek();
         Preconditions.checkState(entry != null, "Attempted to start commit of %s when no transactions pending", cohort);
 
         final SimpleShardDataTreeCohort current = entry.cohort;
-        Verify.verify(cohort.equals(current), "Attempted to commit %s while %s is pending", cohort, current);
+        if (!cohort.equals(current)) {
+            LOG.debug("{}: Transaction {} scheduled for commit step", logContext, cohort.getIdentifier());
+            return;
+        }
 
         if (shard.canSkipPayload() || candidate.getRootNode().getModificationType() == ModificationType.UNMODIFIED) {
             LOG.debug("{}: No replication required, proceeding to finish commit", logContext);
+            pendingCommits.remove();
+            pendingFinishCommits.add(entry);
             finishCommit(cohort);
             return;
         }
@@ -681,16 +745,22 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         final Payload payload;
         try {
             payload = CommitTransactionPayload.create(txId, candidate);
+
+            // Once completed, we will continue via payloadReplicationComplete
+            entry.lastAccess = shard.ticker().read();
+            shard.persistPayload(txId, payload);
+
+            LOG.debug("{}: Transaction {} submitted to persistence", logContext, txId);
+
+            pendingCommits.remove();
+            pendingFinishCommits.add(entry);
         } catch (IOException e) {
             LOG.error("{}: Failed to encode transaction {} candidate {}", logContext, txId, candidate, e);
-            pendingTransactions.poll().cohort.failedCommit(e);
+            pendingCommits.poll().cohort.failedCommit(e);
             return;
         }
 
-        // Once completed, we will continue via payloadReplicationComplete
-        entry.lastAccess = shard.ticker().read();
-        shard.persistPayload(txId, payload);
-        LOG.debug("{}: Transaction {} submitted to persistence", logContext, txId);
+        processNextPending();
     }
 
     void processCohortRegistryCommand(final ActorRef sender, final CohortRegistryCommand message) {
@@ -710,23 +780,26 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
     void checkForExpiredTransactions(final long transactionCommitTimeoutMillis) {
         final long timeout = TimeUnit.MILLISECONDS.toNanos(transactionCommitTimeoutMillis);
         final long now = shard.ticker().read();
-        final CommitEntry currentTx = pendingTransactions.peek();
+
+        final Queue<CommitEntry> currentQueue = !pendingFinishCommits.isEmpty() ? pendingFinishCommits :
+            !pendingCommits.isEmpty() ? pendingCommits : pendingTransactions;
+        final CommitEntry currentTx = currentQueue.peek();
         if (currentTx != null && currentTx.lastAccess + timeout < now) {
             LOG.warn("{}: Current transaction {} has timed out after {} ms in state {}", logContext,
                     currentTx.cohort.getIdentifier(), transactionCommitTimeoutMillis, currentTx.cohort.getState());
             boolean processNext = true;
             switch (currentTx.cohort.getState()) {
                 case CAN_COMMIT_PENDING:
-                    pendingTransactions.remove().cohort.failedCanCommit(new TimeoutException());
+                    currentQueue.remove().cohort.failedCanCommit(new TimeoutException());
                     break;
                 case CAN_COMMIT_COMPLETE:
                     // The suppression of the FindBugs "DB_DUPLICATE_SWITCH_CLAUSES" warning pertains to this clause
                     // whose code is duplicated with PRE_COMMIT_COMPLETE. The clauses aren't combined in case the code
                     // in PRE_COMMIT_COMPLETE is changed.
-                    pendingTransactions.remove().cohort.reportFailure(new TimeoutException());
+                    currentQueue.remove().cohort.reportFailure(new TimeoutException());
                     break;
                 case PRE_COMMIT_PENDING:
-                    pendingTransactions.remove().cohort.failedPreCommit(new TimeoutException());
+                    currentQueue.remove().cohort.failedPreCommit(new TimeoutException());
                     break;
                 case PRE_COMMIT_COMPLETE:
                     // FIXME: this is a legacy behavior problem. Three-phase commit protocol specifies that after we
@@ -746,7 +819,7 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
                     //        In order to make the pre-commit timer working across failovers, though, we need
                     //        a per-shard cluster-wide monotonic time, so a follower becoming the leader can accurately
                     //        restart the timer.
-                    pendingTransactions.remove().cohort.reportFailure(new TimeoutException());
+                    currentQueue.remove().cohort.reportFailure(new TimeoutException());
                     break;
                 case COMMIT_PENDING:
                     LOG.warn("{}: Transaction {} is still committing, cannot abort", logContext,
@@ -759,17 +832,18 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
                 case FAILED:
                 case READY:
                 default:
-                    pendingTransactions.remove();
+                    currentQueue.remove();
             }
 
             if (processNext) {
-                processNextTransaction();
+                processNextPending();
             }
         }
     }
 
     boolean startAbort(final SimpleShardDataTreeCohort cohort) {
-        final Iterator<CommitEntry> it = pendingTransactions.iterator();
+        final Iterator<CommitEntry> it = Iterables.concat(pendingFinishCommits, pendingCommits,
+                pendingTransactions).iterator();
         if (!it.hasNext()) {
             LOG.debug("{}: no open transaction while attempting to abort {}", logContext, cohort.getIdentifier());
             return true;
@@ -783,8 +857,11 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
                     cohort.getIdentifier());
 
                 it.remove();
-                rebasePreCommittedTransactions(it, dataTree);
-                processNextTransaction();
+                if (cohort.getCandidate() != null) {
+                    rebaseTransactions(it, dataTree);
+                }
+
+                processNextPending();
                 return true;
             }
 
@@ -792,16 +869,20 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
             return false;
         }
 
-        TipProducingDataTreeTip newTip = dataTree;
+        TipProducingDataTreeTip newTip = MoreObjects.firstNonNull(first.cohort.getCandidate(), dataTree);
         while (it.hasNext()) {
             final CommitEntry e = it.next();
             if (cohort.equals(e.cohort)) {
                 LOG.debug("{}: aborting queued transaction {}", logContext, cohort.getIdentifier());
+
                 it.remove();
-                rebasePreCommittedTransactions(it, newTip);
+                if (cohort.getCandidate() != null) {
+                    rebaseTransactions(it, newTip);
+                }
+
                 return true;
             } else {
-                newTip = cohort.getCandidate();
+                newTip = MoreObjects.firstNonNull(e.cohort.getCandidate(), dataTree);
             }
         }
 
@@ -810,8 +891,8 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
     }
 
     @SuppressWarnings("checkstyle:IllegalCatch")
-    private void rebasePreCommittedTransactions(Iterator<CommitEntry> iter, TipProducingDataTreeTip newTip) {
-        tip = newTip;
+    private void rebaseTransactions(Iterator<CommitEntry> iter, @Nonnull TipProducingDataTreeTip newTip) {
+        tip = Preconditions.checkNotNull(newTip);
         while (iter.hasNext()) {
             final SimpleShardDataTreeCohort cohort = iter.next().cohort;
             if (cohort.getState() == State.CAN_COMMIT_COMPLETE) {
@@ -847,7 +928,7 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
     }
 
     private void maybeRunOperationOnPendingTransactionsComplete() {
-        if (runOnPendingTransactionsComplete != null && pendingTransactions.isEmpty()) {
+        if (runOnPendingTransactionsComplete != null && !anyPendingTransactions()) {
             LOG.debug("{}: Pending transactions complete - running operation {}", logContext,
                     runOnPendingTransactionsComplete);