BUG-8618: eliminate SimpleShardDataTreeCohort subclasses
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / ShardDataTree.java
index 4eeed882fa1a3ee63a7f9deea5a04d2957212b4b..27577b27d8d4f9342a90ed6db72ec2a5ca70c06c 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;
@@ -44,6 +43,7 @@ import org.opendaylight.controller.cluster.access.concepts.LocalHistoryIdentifie
 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;
@@ -106,8 +106,15 @@ 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);
 
-    private final Map<LocalHistoryIdentifier, ShardDataTreeTransactionChain> transactionChains = new HashMap<>();
+    /**
+     * 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<>();
@@ -136,6 +143,8 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
 
     private SchemaContext schemaContext;
 
+    private int currentTransactionBatch;
+
     ShardDataTree(final Shard shard, final SchemaContext schemaContext, final TipProducingDataTree dataTree,
             final ShardDataTreeChangeListenerPublisher treeChangeListenerPublisher,
             final ShardDataChangeListenerPublisher dataChangeListenerPublisher, final String logContext,
@@ -171,8 +180,8 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         return logContext;
     }
 
-    final Ticker ticker() {
-        return shard.ticker();
+    final long readTime() {
+        return shard.ticker().read();
     }
 
     public TipProducingDataTree getDataTree() {
@@ -188,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.
      *
@@ -253,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);
     }
 
     /**
@@ -379,45 +392,43 @@ 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);
-            } else {
-                allMetadataAbortedTransaction(((AbortTransactionPayload) payload).getIdentifier());
             }
+            allMetadataAbortedTransaction(((AbortTransactionPayload) payload).getIdentifier());
         } else if (payload instanceof PurgeTransactionPayload) {
             if (identifier != null) {
                 payloadReplicationComplete((PurgeTransactionPayload) payload);
-            } else {
-                allMetadataPurgedTransaction(((PurgeTransactionPayload) payload).getIdentifier());
             }
+            allMetadataPurgedTransaction(((PurgeTransactionPayload) payload).getIdentifier());
         } else if (payload instanceof CloseLocalHistoryPayload) {
             if (identifier != null) {
                 payloadReplicationComplete((CloseLocalHistoryPayload) payload);
-            } else {
-                allMetadataClosedLocalHistory(((CloseLocalHistoryPayload) payload).getIdentifier());
             }
+            allMetadataClosedLocalHistory(((CloseLocalHistoryPayload) payload).getIdentifier());
         } else if (payload instanceof CreateLocalHistoryPayload) {
             if (identifier != null) {
                 payloadReplicationComplete((CreateLocalHistoryPayload)payload);
-            } else {
-                allMetadataCreatedLocalHistory(((CreateLocalHistoryPayload) payload).getIdentifier());
             }
+            allMetadataCreatedLocalHistory(((CreateLocalHistoryPayload) payload).getIdentifier());
         } else if (payload instanceof PurgeLocalHistoryPayload) {
             if (identifier != null) {
                 payloadReplicationComplete((PurgeLocalHistoryPayload)payload);
-            } else {
-                allMetadataPurgedLocalHistory(((PurgeLocalHistoryPayload) payload).getIdentifier());
             }
+            allMetadataPurgedLocalHistory(((PurgeLocalHistoryPayload) payload).getIdentifier());
         } else {
             LOG.warn("{}: ignoring unhandled identifier {} payload {}", logContext, identifier, payload);
         }
@@ -509,12 +520,15 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         return ret;
     }
 
-    ShardDataTreeTransactionChain ensureTransactionChain(final LocalHistoryIdentifier historyId) {
+    ShardDataTreeTransactionChain ensureTransactionChain(final LocalHistoryIdentifier historyId,
+            @Nullable final Runnable callback) {
         ShardDataTreeTransactionChain chain = transactionChains.get(historyId);
         if (chain == null) {
             chain = new ShardDataTreeTransactionChain(historyId, this);
             transactionChains.put(historyId, chain);
-            shard.persistPayload(historyId, CreateLocalHistoryPayload.create(historyId), true);
+            replicatePayload(historyId, CreateLocalHistoryPayload.create(historyId), callback);
+        } else if (callback != null) {
+            callback.run();
         }
 
         return chain;
@@ -525,7 +539,7 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
             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) {
@@ -534,7 +548,7 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
                     .newModification());
         }
 
-        return ensureTransactionChain(txId.getHistoryId()).newReadWriteTransaction(txId);
+        return ensureTransactionChain(txId.getHistoryId(), null).newReadWriteTransaction(txId);
     }
 
     @VisibleForTesting
@@ -659,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());
 
@@ -699,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();
@@ -708,12 +719,10 @@ 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 = ticker().read();
+                entry.lastAccess = readTime();
                 return;
             } catch (ConflictingModificationAppliedException e) {
                 LOG.warn("{}: Store Tx {}: Conflicting modification for path {}.", logContext, cohort.getIdentifier(),
@@ -776,8 +785,12 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
     }
 
     void startCanCommit(final SimpleShardDataTreeCohort cohort) {
-        final SimpleShardDataTreeCohort current = pendingTransactions.peek().cohort;
-        if (!cohort.equals(current)) {
+        final CommitEntry head = pendingTransactions.peek();
+        if (head == null) {
+            LOG.warn("{}: No transactions enqueued while attempting to start canCommit on {}", logContext, cohort);
+            return;
+        }
+        if (!cohort.equals(head.cohort)) {
             LOG.debug("{}: Transaction {} scheduled for canCommit step", logContext, cohort.getIdentifier());
             return;
         }
@@ -813,7 +826,7 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
         // Set the tip of the data tree.
         tip = Verify.verifyNotNull(candidate);
 
-        entry.lastAccess = ticker().read();
+        entry.lastAccess = readTime();
 
         pendingTransactions.remove();
         pendingCommits.add(entry);
@@ -927,17 +940,16 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
     @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, ticker().read()));
+        final SimpleShardDataTreeCohort cohort = new SimpleShardDataTreeCohort(this, mod, txId, failure);
+        pendingTransactions.add(new CommitEntry(cohort, readTime()));
         return cohort;
     }
 
     @Override
-    ShardDataTreeCohort createReadyCohort(final TransactionIdentifier txId,
-            final DataTreeModification mod) {
-        SimpleShardDataTreeCohort cohort = new SimpleShardDataTreeCohort.Normal(this, mod, txId,
+    ShardDataTreeCohort createReadyCohort(final TransactionIdentifier txId, final DataTreeModification mod) {
+        SimpleShardDataTreeCohort cohort = new SimpleShardDataTreeCohort(this, mod, txId,
                 cohortRegistry.createCohort(schemaContext, txId, COMMIT_STEP_TIMEOUT));
-        pendingTransactions.add(new CommitEntry(cohort, ticker().read()));
+        pendingTransactions.add(new CommitEntry(cohort, readTime()));
         return cohort;
     }
 
@@ -948,33 +960,37 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
             return createReadyCohort(txId, mod);
         }
 
-        return ensureTransactionChain(txId.getHistoryId()).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 = ticker().read();
+        final long now = readTime();
 
         final Queue<CommitEntry> currentQueue = !pendingFinishCommits.isEmpty() ? pendingFinishCommits :
             !pendingCommits.isEmpty() ? pendingCommits : pendingTransactions;
         final CommitEntry currentTx = currentQueue.peek();
         if (currentTx != null && currentTx.lastAccess + timeout < now) {
+            final State state = currentTx.cohort.getState();
             LOG.warn("{}: Current transaction {} has timed out after {} ms in state {}", logContext,
-                    currentTx.cohort.getIdentifier(), transactionCommitTimeoutMillis, currentTx.cohort.getState());
+                    currentTx.cohort.getIdentifier(), transactionCommitTimeoutMillis, state);
             boolean processNext = true;
-            switch (currentTx.cohort.getState()) {
+            final TimeoutException cohortFailure = new TimeoutException("Backend timeout in state " + state + " after "
+                    + transactionCommitTimeoutMillis + "ms");
+
+            switch (state) {
                 case CAN_COMMIT_PENDING:
-                    currentQueue.remove().cohort.failedCanCommit(new TimeoutException());
+                    currentQueue.remove().cohort.failedCanCommit(cohortFailure);
                     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.
-                    currentQueue.remove().cohort.reportFailure(new TimeoutException());
+                    currentQueue.remove().cohort.reportFailure(cohortFailure);
                     break;
                 case PRE_COMMIT_PENDING:
-                    currentQueue.remove().cohort.failedPreCommit(new TimeoutException());
+                    currentQueue.remove().cohort.failedPreCommit(cohortFailure);
                     break;
                 case PRE_COMMIT_COMPLETE:
                     // FIXME: this is a legacy behavior problem. Three-phase commit protocol specifies that after we
@@ -994,7 +1010,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.
-                    currentQueue.remove().cohort.reportFailure(new TimeoutException());
+                    currentQueue.remove().cohort.reportFailure(cohortFailure);
                     break;
                 case COMMIT_PENDING:
                     LOG.warn("{}: Transaction {} is still committing, cannot abort", logContext,
@@ -1002,10 +1018,12 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
                     currentTx.lastAccess = now;
                     processNext = false;
                     return;
+                case READY:
+                    currentQueue.remove().cohort.reportFailure(cohortFailure);
+                    break;
                 case ABORTED:
                 case COMMITTED:
                 case FAILED:
-                case READY:
                 default:
                     currentQueue.remove();
             }
@@ -1111,4 +1129,8 @@ public class ShardDataTree extends ShardDataTreeTransactionParent {
             runOnPendingTransactionsComplete = null;
         }
     }
+
+    ShardStats getStats() {
+        return shard.getShardMBean();
+    }
 }