BUG-5280: add executionTimeNanos
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / ShardDataTree.java
1 /*
2  * Copyright (c) 2015 Cisco Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.controller.cluster.datastore;
9
10 import akka.actor.ActorRef;
11 import akka.util.Timeout;
12 import com.google.common.annotations.VisibleForTesting;
13 import com.google.common.base.Optional;
14 import com.google.common.base.Preconditions;
15 import com.google.common.base.Stopwatch;
16 import com.google.common.base.Ticker;
17 import com.google.common.base.Verify;
18 import com.google.common.collect.ImmutableList;
19 import com.google.common.collect.ImmutableMap;
20 import com.google.common.collect.ImmutableMap.Builder;
21 import com.google.common.primitives.UnsignedLong;
22 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
23 import java.io.File;
24 import java.io.IOException;
25 import java.util.AbstractMap.SimpleEntry;
26 import java.util.ArrayDeque;
27 import java.util.ArrayList;
28 import java.util.Collection;
29 import java.util.HashMap;
30 import java.util.Iterator;
31 import java.util.Map;
32 import java.util.Map.Entry;
33 import java.util.Queue;
34 import java.util.concurrent.ExecutionException;
35 import java.util.concurrent.TimeUnit;
36 import java.util.concurrent.TimeoutException;
37 import java.util.function.UnaryOperator;
38 import javax.annotation.Nonnull;
39 import javax.annotation.concurrent.NotThreadSafe;
40 import org.opendaylight.controller.cluster.access.concepts.LocalHistoryIdentifier;
41 import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
42 import org.opendaylight.controller.cluster.datastore.DataTreeCohortActorRegistry.CohortRegistryCommand;
43 import org.opendaylight.controller.cluster.datastore.ShardDataTreeCohort.State;
44 import org.opendaylight.controller.cluster.datastore.persisted.CommitTransactionPayload;
45 import org.opendaylight.controller.cluster.datastore.persisted.MetadataShardDataTreeSnapshot;
46 import org.opendaylight.controller.cluster.datastore.persisted.ShardDataTreeSnapshot;
47 import org.opendaylight.controller.cluster.datastore.persisted.ShardDataTreeSnapshotMetadata;
48 import org.opendaylight.controller.cluster.datastore.utils.DataTreeModificationOutput;
49 import org.opendaylight.controller.cluster.datastore.utils.PruningDataTreeModification;
50 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
51 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataBroker.DataChangeScope;
52 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener;
53 import org.opendaylight.controller.md.sal.common.api.data.OptimisticLockFailedException;
54 import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
55 import org.opendaylight.controller.md.sal.dom.api.DOMDataTreeChangeListener;
56 import org.opendaylight.controller.md.sal.dom.store.impl.DataChangeListenerRegistration;
57 import org.opendaylight.yangtools.concepts.Identifier;
58 import org.opendaylight.yangtools.concepts.ListenerRegistration;
59 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
60 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
61 import org.opendaylight.yangtools.yang.data.api.schema.tree.ConflictingModificationAppliedException;
62 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidate;
63 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidateTip;
64 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidates;
65 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification;
66 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeSnapshot;
67 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException;
68 import org.opendaylight.yangtools.yang.data.api.schema.tree.ModificationType;
69 import org.opendaylight.yangtools.yang.data.api.schema.tree.TipProducingDataTree;
70 import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType;
71 import org.opendaylight.yangtools.yang.data.impl.schema.tree.InMemoryDataTreeFactory;
72 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
73 import org.slf4j.Logger;
74 import org.slf4j.LoggerFactory;
75 import scala.concurrent.duration.Duration;
76
77 /**
78  * Internal shard state, similar to a DOMStore, but optimized for use in the actor system,
79  * e.g. it does not expose public interfaces and assumes it is only ever called from a
80  * single thread.
81  *
82  * <p>
83  * This class is not part of the API contract and is subject to change at any time.
84  */
85 @NotThreadSafe
86 public class ShardDataTree extends ShardDataTreeTransactionParent {
87     private static final class CommitEntry {
88         final SimpleShardDataTreeCohort cohort;
89         long lastAccess;
90
91         CommitEntry(final SimpleShardDataTreeCohort cohort, final long now) {
92             this.cohort = Preconditions.checkNotNull(cohort);
93             lastAccess = now;
94         }
95     }
96
97     private static final Timeout COMMIT_STEP_TIMEOUT = new Timeout(Duration.create(5, TimeUnit.SECONDS));
98     private static final Logger LOG = LoggerFactory.getLogger(ShardDataTree.class);
99
100     private final Map<LocalHistoryIdentifier, ShardDataTreeTransactionChain> transactionChains = new HashMap<>();
101     private final DataTreeCohortActorRegistry cohortRegistry = new DataTreeCohortActorRegistry();
102     private final Queue<CommitEntry> pendingTransactions = new ArrayDeque<>();
103     private final ShardDataTreeChangeListenerPublisher treeChangeListenerPublisher;
104     private final ShardDataChangeListenerPublisher dataChangeListenerPublisher;
105     private final Collection<ShardDataTreeMetadata<?>> metadata;
106     private final TipProducingDataTree dataTree;
107     private final String logContext;
108     private final Shard shard;
109     private Runnable runOnPendingTransactionsComplete;
110
111     private SchemaContext schemaContext;
112
113     public ShardDataTree(final Shard shard, final SchemaContext schemaContext, final TipProducingDataTree dataTree,
114             final ShardDataTreeChangeListenerPublisher treeChangeListenerPublisher,
115             final ShardDataChangeListenerPublisher dataChangeListenerPublisher, final String logContext,
116             final ShardDataTreeMetadata<?>... metadata) {
117         this.dataTree = Preconditions.checkNotNull(dataTree);
118         updateSchemaContext(schemaContext);
119
120         this.shard = Preconditions.checkNotNull(shard);
121         this.treeChangeListenerPublisher = Preconditions.checkNotNull(treeChangeListenerPublisher);
122         this.dataChangeListenerPublisher = Preconditions.checkNotNull(dataChangeListenerPublisher);
123         this.logContext = Preconditions.checkNotNull(logContext);
124         this.metadata = ImmutableList.copyOf(metadata);
125     }
126
127     public ShardDataTree(final Shard shard, final SchemaContext schemaContext, final TreeType treeType,
128             final ShardDataTreeChangeListenerPublisher treeChangeListenerPublisher,
129             final ShardDataChangeListenerPublisher dataChangeListenerPublisher, final String logContext) {
130         this(shard, schemaContext, InMemoryDataTreeFactory.getInstance().create(treeType),
131                 treeChangeListenerPublisher, dataChangeListenerPublisher, logContext);
132     }
133
134     @VisibleForTesting
135     public ShardDataTree(final Shard shard, final SchemaContext schemaContext, final TreeType treeType) {
136         this(shard, schemaContext, treeType, new DefaultShardDataTreeChangeListenerPublisher(),
137                 new DefaultShardDataChangeListenerPublisher(), "");
138     }
139
140     final String logContext() {
141         return logContext;
142     }
143
144     final Ticker ticker() {
145         return shard.ticker();
146     }
147
148     public TipProducingDataTree getDataTree() {
149         return dataTree;
150     }
151
152     SchemaContext getSchemaContext() {
153         return schemaContext;
154     }
155
156     void updateSchemaContext(final SchemaContext newSchemaContext) {
157         dataTree.setSchemaContext(newSchemaContext);
158         this.schemaContext = Preconditions.checkNotNull(newSchemaContext);
159     }
160
161     /**
162      * Take a snapshot of current state for later recovery.
163      *
164      * @return A state snapshot
165      */
166     @Nonnull ShardDataTreeSnapshot takeStateSnapshot() {
167         final NormalizedNode<?, ?> rootNode = dataTree.takeSnapshot().readNode(YangInstanceIdentifier.EMPTY).get();
168         final Builder<Class<? extends ShardDataTreeSnapshotMetadata<?>>, ShardDataTreeSnapshotMetadata<?>> metaBuilder =
169                 ImmutableMap.builder();
170
171         for (ShardDataTreeMetadata<?> m : metadata) {
172             final ShardDataTreeSnapshotMetadata<?> meta = m.toSnapshot();
173             if (meta != null) {
174                 metaBuilder.put(meta.getType(), meta);
175             }
176         }
177
178         return new MetadataShardDataTreeSnapshot(rootNode, metaBuilder.build());
179     }
180
181     private void applySnapshot(@Nonnull final ShardDataTreeSnapshot snapshot,
182             final UnaryOperator<DataTreeModification> wrapper) throws DataValidationFailedException {
183         final Stopwatch elapsed = Stopwatch.createStarted();
184
185         if (!pendingTransactions.isEmpty()) {
186             LOG.warn("{}: applying state snapshot with pending transactions", logContext);
187         }
188
189         final Map<Class<? extends ShardDataTreeSnapshotMetadata<?>>, ShardDataTreeSnapshotMetadata<?>> snapshotMeta;
190         if (snapshot instanceof MetadataShardDataTreeSnapshot) {
191             snapshotMeta = ((MetadataShardDataTreeSnapshot) snapshot).getMetadata();
192         } else {
193             snapshotMeta = ImmutableMap.of();
194         }
195
196         for (ShardDataTreeMetadata<?> m : metadata) {
197             final ShardDataTreeSnapshotMetadata<?> s = snapshotMeta.get(m.getSupportedType());
198             if (s != null) {
199                 m.applySnapshot(s);
200             } else {
201                 m.reset();
202             }
203         }
204
205         final DataTreeModification mod = wrapper.apply(dataTree.takeSnapshot().newModification());
206         // delete everything first
207         mod.delete(YangInstanceIdentifier.EMPTY);
208
209         final java.util.Optional<NormalizedNode<?, ?>> maybeNode = snapshot.getRootNode();
210         if (maybeNode.isPresent()) {
211             // Add everything from the remote node back
212             mod.write(YangInstanceIdentifier.EMPTY, maybeNode.get());
213         }
214         mod.ready();
215
216         final DataTreeModification unwrapped = unwrap(mod);
217         dataTree.validate(unwrapped);
218         DataTreeCandidateTip candidate = dataTree.prepare(unwrapped);
219         dataTree.commit(candidate);
220         notifyListeners(candidate);
221
222         LOG.debug("{}: state snapshot applied in %s", logContext, elapsed);
223     }
224
225     /**
226      * Apply a snapshot coming from the leader. This method assumes the leader and follower SchemaContexts match and
227      * does not perform any pruning.
228      *
229      * @param snapshot Snapshot that needs to be applied
230      * @throws DataValidationFailedException when the snapshot fails to apply
231      */
232     void applySnapshot(@Nonnull final ShardDataTreeSnapshot snapshot) throws DataValidationFailedException {
233         applySnapshot(snapshot, UnaryOperator.identity());
234     }
235
236     private PruningDataTreeModification wrapWithPruning(final DataTreeModification delegate) {
237         return new PruningDataTreeModification(delegate, dataTree, schemaContext);
238     }
239
240     private static DataTreeModification unwrap(final DataTreeModification modification) {
241         if (modification instanceof PruningDataTreeModification) {
242             return ((PruningDataTreeModification)modification).delegate();
243         }
244         return modification;
245     }
246
247     /**
248      * Apply a snapshot coming from recovery. This method does not assume the SchemaContexts match and performs data
249      * pruning in an attempt to adjust the state to our current SchemaContext.
250      *
251      * @param snapshot Snapshot that needs to be applied
252      * @throws DataValidationFailedException when the snapshot fails to apply
253      */
254     void applyRecoverySnapshot(final @Nonnull ShardDataTreeSnapshot snapshot) throws DataValidationFailedException {
255         applySnapshot(snapshot, this::wrapWithPruning);
256     }
257
258     @SuppressWarnings("checkstyle:IllegalCatch")
259     private void applyRecoveryCandidate(final DataTreeCandidate candidate) throws DataValidationFailedException {
260         final PruningDataTreeModification mod = wrapWithPruning(dataTree.takeSnapshot().newModification());
261         DataTreeCandidates.applyToModification(mod, candidate);
262         mod.ready();
263
264         final DataTreeModification unwrapped = mod.delegate();
265         LOG.trace("{}: Applying recovery modification {}", logContext, unwrapped);
266
267         try {
268             dataTree.validate(unwrapped);
269             dataTree.commit(dataTree.prepare(unwrapped));
270         } catch (Exception e) {
271             File file = new File(System.getProperty("karaf.data", "."),
272                     "failed-recovery-payload-" + logContext + ".out");
273             DataTreeModificationOutput.toFile(file, unwrapped);
274             throw new IllegalStateException(String.format(
275                     "%s: Failed to apply recovery payload. Modification data was written to file %s",
276                     logContext, file), e);
277         }
278     }
279
280     /**
281      * Apply a payload coming from recovery. This method does not assume the SchemaContexts match and performs data
282      * pruning in an attempt to adjust the state to our current SchemaContext.
283      *
284      * @param payload Payload
285      * @throws IOException when the snapshot fails to deserialize
286      * @throws DataValidationFailedException when the snapshot fails to apply
287      */
288     void applyRecoveryPayload(final @Nonnull Payload payload) throws IOException, DataValidationFailedException {
289         if (payload instanceof CommitTransactionPayload) {
290             final Entry<TransactionIdentifier, DataTreeCandidate> e =
291                     ((CommitTransactionPayload) payload).getCandidate();
292             applyRecoveryCandidate(e.getValue());
293             allMetadataCommittedTransaction(e.getKey());
294         } else if (payload instanceof DataTreeCandidatePayload) {
295             applyRecoveryCandidate(((DataTreeCandidatePayload) payload).getCandidate());
296         } else {
297             LOG.debug("{}: ignoring unhandled payload {}", logContext, payload);
298         }
299     }
300
301     private void applyReplicatedCandidate(final Identifier identifier, final DataTreeCandidate foreign)
302             throws DataValidationFailedException {
303         LOG.debug("{}: Applying foreign transaction {}", logContext, identifier);
304
305         final DataTreeModification mod = dataTree.takeSnapshot().newModification();
306         DataTreeCandidates.applyToModification(mod, foreign);
307         mod.ready();
308
309         LOG.trace("{}: Applying foreign modification {}", logContext, mod);
310         dataTree.validate(mod);
311         final DataTreeCandidate candidate = dataTree.prepare(mod);
312         dataTree.commit(candidate);
313
314         notifyListeners(candidate);
315     }
316
317     /**
318      * Apply a payload coming from the leader, which could actually be us. This method assumes the leader and follower
319      * SchemaContexts match and does not perform any pruning.
320      *
321      * @param identifier Payload identifier as returned from RaftActor
322      * @param payload Payload
323      * @throws IOException when the snapshot fails to deserialize
324      * @throws DataValidationFailedException when the snapshot fails to apply
325      */
326     void applyReplicatedPayload(final Identifier identifier, final Payload payload) throws IOException,
327             DataValidationFailedException {
328         /*
329          * This is a bit more involved than it needs to be due to to the fact we do not want to be touching the payload
330          * if we are the leader and it has originated with us.
331          *
332          * The identifier will only ever be non-null when we were the leader which achieved consensus. Unfortunately,
333          * though, this may not be the case anymore, as we are being called some time afterwards and we may not be
334          * acting in that capacity anymore.
335          *
336          * In any case, we know that this is an entry coming from replication, hence we can be sure we will not observe
337          * pre-Boron state -- which limits the number of options here.
338          */
339         if (payload instanceof CommitTransactionPayload) {
340             if (identifier == null) {
341                 final Entry<TransactionIdentifier, DataTreeCandidate> e =
342                         ((CommitTransactionPayload) payload).getCandidate();
343                 applyReplicatedCandidate(e.getKey(), e.getValue());
344                 allMetadataCommittedTransaction(e.getKey());
345             } else {
346                 Verify.verify(identifier instanceof TransactionIdentifier);
347                 payloadReplicationComplete((TransactionIdentifier) identifier);
348             }
349         } else {
350             LOG.warn("{}: ignoring unhandled identifier {} payload {}", logContext, identifier, payload);
351         }
352     }
353
354     private void payloadReplicationComplete(final TransactionIdentifier txId) {
355         final CommitEntry current = pendingTransactions.peek();
356         if (current == null) {
357             LOG.warn("{}: No outstanding transactions, ignoring consensus on transaction {}", logContext, txId);
358             return;
359         }
360
361         if (!current.cohort.getIdentifier().equals(txId)) {
362             LOG.warn("{}: Head of queue is {}, ignoring consensus on transaction {}", logContext,
363                 current.cohort.getIdentifier(), txId);
364             return;
365         }
366
367         finishCommit(current.cohort);
368     }
369
370     private void allMetadataCommittedTransaction(final TransactionIdentifier txId) {
371         for (ShardDataTreeMetadata<?> m : metadata) {
372             m.onTransactionCommitted(txId);
373         }
374     }
375
376     ShardDataTreeTransactionChain ensureTransactionChain(final LocalHistoryIdentifier localHistoryIdentifier) {
377         ShardDataTreeTransactionChain chain = transactionChains.get(localHistoryIdentifier);
378         if (chain == null) {
379             chain = new ShardDataTreeTransactionChain(localHistoryIdentifier, this);
380             transactionChains.put(localHistoryIdentifier, chain);
381         }
382
383         return chain;
384     }
385
386     ReadOnlyShardDataTreeTransaction newReadOnlyTransaction(final TransactionIdentifier txId) {
387         if (txId.getHistoryId().getHistoryId() == 0) {
388             return new ReadOnlyShardDataTreeTransaction(txId, dataTree.takeSnapshot());
389         }
390
391         return ensureTransactionChain(txId.getHistoryId()).newReadOnlyTransaction(txId);
392     }
393
394     ReadWriteShardDataTreeTransaction newReadWriteTransaction(final TransactionIdentifier txId) {
395         if (txId.getHistoryId().getHistoryId() == 0) {
396             return new ReadWriteShardDataTreeTransaction(ShardDataTree.this, txId, dataTree.takeSnapshot()
397                     .newModification());
398         }
399
400         return ensureTransactionChain(txId.getHistoryId()).newReadWriteTransaction(txId);
401     }
402
403     @VisibleForTesting
404     public void notifyListeners(final DataTreeCandidate candidate) {
405         treeChangeListenerPublisher.publishChanges(candidate, logContext);
406         dataChangeListenerPublisher.publishChanges(candidate, logContext);
407     }
408
409     void notifyOfInitialData(final DataChangeListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier,
410             NormalizedNode<?, ?>>> listenerReg, final Optional<DataTreeCandidate> currentState) {
411         if (currentState.isPresent()) {
412             ShardDataChangeListenerPublisher localPublisher = dataChangeListenerPublisher.newInstance();
413             localPublisher.registerDataChangeListener(listenerReg.getPath(), listenerReg.getInstance(),
414                     listenerReg.getScope());
415             localPublisher.publishChanges(currentState.get(), logContext);
416         }
417     }
418
419     void notifyOfInitialData(final YangInstanceIdentifier path, final DOMDataTreeChangeListener listener,
420             final Optional<DataTreeCandidate> currentState) {
421         if (currentState.isPresent()) {
422             ShardDataTreeChangeListenerPublisher localPublisher = treeChangeListenerPublisher.newInstance();
423             localPublisher.registerTreeChangeListener(path, listener);
424             localPublisher.publishChanges(currentState.get(), logContext);
425         }
426     }
427
428     void closeAllTransactionChains() {
429         for (ShardDataTreeTransactionChain chain : transactionChains.values()) {
430             chain.close();
431         }
432
433         transactionChains.clear();
434     }
435
436     void closeTransactionChain(final LocalHistoryIdentifier transactionChainId) {
437         final ShardDataTreeTransactionChain chain = transactionChains.remove(transactionChainId);
438         if (chain != null) {
439             chain.close();
440         } else {
441             LOG.debug("{}: Closing non-existent transaction chain {}", logContext, transactionChainId);
442         }
443     }
444
445     Entry<DataChangeListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>>,
446             Optional<DataTreeCandidate>> registerChangeListener(final YangInstanceIdentifier path,
447                     final AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>> listener,
448                     final DataChangeScope scope) {
449         DataChangeListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>> reg =
450                 dataChangeListenerPublisher.registerDataChangeListener(path, listener, scope);
451
452         return new SimpleEntry<>(reg, readCurrentData());
453     }
454
455     private Optional<DataTreeCandidate> readCurrentData() {
456         final Optional<NormalizedNode<?, ?>> currentState =
457                 dataTree.takeSnapshot().readNode(YangInstanceIdentifier.EMPTY);
458         return currentState.isPresent() ? Optional.of(DataTreeCandidates.fromNormalizedNode(
459             YangInstanceIdentifier.EMPTY, currentState.get())) : Optional.<DataTreeCandidate>absent();
460     }
461
462     public Entry<ListenerRegistration<DOMDataTreeChangeListener>, Optional<DataTreeCandidate>>
463             registerTreeChangeListener(final YangInstanceIdentifier path, final DOMDataTreeChangeListener listener) {
464         final ListenerRegistration<DOMDataTreeChangeListener> reg =
465                 treeChangeListenerPublisher.registerTreeChangeListener(path, listener);
466
467         return new SimpleEntry<>(reg, readCurrentData());
468     }
469
470     int getQueueSize() {
471         return pendingTransactions.size();
472     }
473
474     @Override
475     void abortTransaction(final AbstractShardDataTreeTransaction<?> transaction) {
476         // Intentional no-op
477     }
478
479     @Override
480     ShardDataTreeCohort finishTransaction(final ReadWriteShardDataTreeTransaction transaction) {
481         final DataTreeModification snapshot = transaction.getSnapshot();
482         snapshot.ready();
483
484         return createReadyCohort(transaction.getIdentifier(), snapshot);
485     }
486
487     public Optional<NormalizedNode<?, ?>> readNode(final YangInstanceIdentifier path) {
488         return dataTree.takeSnapshot().readNode(path);
489     }
490
491     DataTreeSnapshot takeSnapshot() {
492         return dataTree.takeSnapshot();
493     }
494
495     @VisibleForTesting
496     public DataTreeModification newModification() {
497         return dataTree.takeSnapshot().newModification();
498     }
499
500     /**
501      * Commits a modification.
502      *
503      * @deprecated This method violates DataTree containment and will be removed.
504      */
505     @VisibleForTesting
506     @Deprecated
507     public DataTreeCandidate commit(final DataTreeModification modification) throws DataValidationFailedException {
508         modification.ready();
509         dataTree.validate(modification);
510         DataTreeCandidate candidate = dataTree.prepare(modification);
511         dataTree.commit(candidate);
512         return candidate;
513     }
514
515     public Collection<ShardDataTreeCohort> getAndClearPendingTransactions() {
516         Collection<ShardDataTreeCohort> ret = new ArrayList<>(pendingTransactions.size());
517         for (CommitEntry entry: pendingTransactions) {
518             ret.add(entry.cohort);
519         }
520
521         pendingTransactions.clear();
522         return ret;
523     }
524
525     @SuppressWarnings("checkstyle:IllegalCatch")
526     private void processNextTransaction() {
527         while (!pendingTransactions.isEmpty()) {
528             final CommitEntry entry = pendingTransactions.peek();
529             final SimpleShardDataTreeCohort cohort = entry.cohort;
530             final DataTreeModification modification = cohort.getDataTreeModification();
531
532             if (cohort.getState() != State.CAN_COMMIT_PENDING) {
533                 break;
534             }
535
536             LOG.debug("{}: Validating transaction {}", logContext, cohort.getIdentifier());
537             Exception cause;
538             try {
539                 dataTree.validate(modification);
540                 LOG.debug("{}: Transaction {} validated", logContext, cohort.getIdentifier());
541                 cohort.successfulCanCommit();
542                 entry.lastAccess = shard.ticker().read();
543                 return;
544             } catch (ConflictingModificationAppliedException e) {
545                 LOG.warn("{}: Store Tx {}: Conflicting modification for path {}.", logContext, cohort.getIdentifier(),
546                     e.getPath());
547                 cause = new OptimisticLockFailedException("Optimistic lock failed.", e);
548             } catch (DataValidationFailedException e) {
549                 LOG.warn("{}: Store Tx {}: Data validation failed for path {}.", logContext, cohort.getIdentifier(),
550                     e.getPath(), e);
551
552                 // For debugging purposes, allow dumping of the modification. Coupled with the above
553                 // precondition log, it should allow us to understand what went on.
554                 LOG.debug("{}: Store Tx {}: modifications: {} tree: {}", cohort.getIdentifier(), modification,
555                         dataTree);
556                 cause = new TransactionCommitFailedException("Data did not pass validation.", e);
557             } catch (Exception e) {
558                 LOG.warn("{}: Unexpected failure in validation phase", logContext, e);
559                 cause = e;
560             }
561
562             // Failure path: propagate the failure, remove the transaction from the queue and loop to the next one
563             pendingTransactions.poll().cohort.failedCanCommit(cause);
564         }
565
566         maybeRunOperationOnPendingTransactionsComplete();
567     }
568
569     void startCanCommit(final SimpleShardDataTreeCohort cohort) {
570         final SimpleShardDataTreeCohort current = pendingTransactions.peek().cohort;
571         if (!cohort.equals(current)) {
572             LOG.debug("{}: Transaction {} scheduled for canCommit step", logContext, cohort.getIdentifier());
573             return;
574         }
575
576         processNextTransaction();
577     }
578
579     private void failPreCommit(final Exception cause) {
580         shard.getShardMBean().incrementFailedTransactionsCount();
581         pendingTransactions.poll().cohort.failedPreCommit(cause);
582         processNextTransaction();
583     }
584
585     @SuppressWarnings("checkstyle:IllegalCatch")
586     void startPreCommit(final SimpleShardDataTreeCohort cohort) {
587         final CommitEntry entry = pendingTransactions.peek();
588         Preconditions.checkState(entry != null, "Attempted to pre-commit of %s when no transactions pending", cohort);
589
590         final SimpleShardDataTreeCohort current = entry.cohort;
591         Verify.verify(cohort.equals(current), "Attempted to pre-commit %s while %s is pending", cohort, current);
592         final DataTreeCandidateTip candidate;
593         try {
594             candidate = dataTree.prepare(cohort.getDataTreeModification());
595         } catch (Exception e) {
596             failPreCommit(e);
597             return;
598         }
599
600         try {
601             cohort.userPreCommit(candidate);
602         } catch (ExecutionException | TimeoutException e) {
603             failPreCommit(e);
604             return;
605         }
606
607         entry.lastAccess = shard.ticker().read();
608         cohort.successfulPreCommit(candidate);
609     }
610
611     private void failCommit(final Exception cause) {
612         shard.getShardMBean().incrementFailedTransactionsCount();
613         pendingTransactions.poll().cohort.failedCommit(cause);
614         processNextTransaction();
615     }
616
617     @SuppressWarnings("checkstyle:IllegalCatch")
618     private void finishCommit(final SimpleShardDataTreeCohort cohort) {
619         final TransactionIdentifier txId = cohort.getIdentifier();
620         final DataTreeCandidate candidate = cohort.getCandidate();
621
622         LOG.debug("{}: Resuming commit of transaction {}", logContext, txId);
623
624         try {
625             dataTree.commit(candidate);
626         } catch (Exception e) {
627             LOG.error("{}: Failed to commit transaction {}", logContext, txId, e);
628             failCommit(e);
629             return;
630         }
631
632         shard.getShardMBean().incrementCommittedTransactionCount();
633         shard.getShardMBean().setLastCommittedTransactionTime(System.currentTimeMillis());
634
635         // FIXME: propagate journal index
636         pendingTransactions.poll().cohort.successfulCommit(UnsignedLong.ZERO);
637
638         LOG.trace("{}: Transaction {} committed, proceeding to notify", logContext, txId);
639         notifyListeners(candidate);
640
641         processNextTransaction();
642     }
643
644     void startCommit(final SimpleShardDataTreeCohort cohort, final DataTreeCandidate candidate) {
645         final CommitEntry entry = pendingTransactions.peek();
646         Preconditions.checkState(entry != null, "Attempted to start commit of %s when no transactions pending", cohort);
647
648         final SimpleShardDataTreeCohort current = entry.cohort;
649         Verify.verify(cohort.equals(current), "Attempted to commit %s while %s is pending", cohort, current);
650
651         if (shard.canSkipPayload() || candidate.getRootNode().getModificationType() == ModificationType.UNMODIFIED) {
652             LOG.debug("{}: No replication required, proceeding to finish commit", logContext);
653             finishCommit(cohort);
654             return;
655         }
656
657         final TransactionIdentifier txId = cohort.getIdentifier();
658         final Payload payload;
659         try {
660             payload = CommitTransactionPayload.create(txId, candidate);
661         } catch (IOException e) {
662             LOG.error("{}: Failed to encode transaction {} candidate {}", logContext, txId, candidate, e);
663             pendingTransactions.poll().cohort.failedCommit(e);
664             return;
665         }
666
667         // Once completed, we will continue via payloadReplicationComplete
668         entry.lastAccess = shard.ticker().read();
669         shard.persistPayload(txId, payload);
670         LOG.debug("{}: Transaction {} submitted to persistence", logContext, txId);
671     }
672
673     void processCohortRegistryCommand(final ActorRef sender, final CohortRegistryCommand message) {
674         cohortRegistry.process(sender, message);
675     }
676
677     @Override
678     ShardDataTreeCohort createReadyCohort(final TransactionIdentifier txId,
679             final DataTreeModification modification) {
680         SimpleShardDataTreeCohort cohort = new SimpleShardDataTreeCohort(this, modification, txId,
681                 cohortRegistry.createCohort(schemaContext, txId, COMMIT_STEP_TIMEOUT));
682         pendingTransactions.add(new CommitEntry(cohort, shard.ticker().read()));
683         return cohort;
684     }
685
686     @SuppressFBWarnings(value = {"RV_RETURN_VALUE_IGNORED", "DB_DUPLICATE_SWITCH_CLAUSES"},
687             justification = "See inline comments below.")
688     void checkForExpiredTransactions(final long transactionCommitTimeoutMillis) {
689         final long timeout = TimeUnit.MILLISECONDS.toNanos(transactionCommitTimeoutMillis);
690         final long now = shard.ticker().read();
691         final CommitEntry currentTx = pendingTransactions.peek();
692         if (currentTx != null && currentTx.lastAccess + timeout < now) {
693             LOG.warn("{}: Current transaction {} has timed out after {} ms in state {}", logContext,
694                     currentTx.cohort.getIdentifier(), transactionCommitTimeoutMillis, currentTx.cohort.getState());
695             boolean processNext = true;
696             switch (currentTx.cohort.getState()) {
697                 case CAN_COMMIT_PENDING:
698                     pendingTransactions.poll().cohort.failedCanCommit(new TimeoutException());
699                     break;
700                 case CAN_COMMIT_COMPLETE:
701                     // The suppression of the FindBugs "DB_DUPLICATE_SWITCH_CLAUSES" warning pertains to this clause
702                     // whose code is duplicated with PRE_COMMIT_COMPLETE. The clauses aren't combined in case the code
703                     // in PRE_COMMIT_COMPLETE is changed.
704                     pendingTransactions.poll().cohort.reportFailure(new TimeoutException());
705                     break;
706                 case PRE_COMMIT_PENDING:
707                     pendingTransactions.poll().cohort.failedPreCommit(new TimeoutException());
708                     break;
709                 case PRE_COMMIT_COMPLETE:
710                     // FIXME: this is a legacy behavior problem. Three-phase commit protocol specifies that after we
711                     //        are ready we should commit the transaction, not abort it. Our current software stack does
712                     //        not allow us to do that consistently, because we persist at the time of commit, hence
713                     //        we can end up in a state where we have pre-committed a transaction, then a leader failover
714                     //        occurred ... the new leader does not see the pre-committed transaction and does not have
715                     //        a running timer. To fix this we really need two persistence events.
716                     //
717                     //        The first one, done at pre-commit time will hold the transaction payload. When consensus
718                     //        is reached, we exit the pre-commit phase and start the pre-commit timer. Followers do not
719                     //        apply the state in this event.
720                     //
721                     //        The second one, done at commit (or abort) time holds only the transaction identifier and
722                     //        signals to followers that the state should (or should not) be applied.
723                     //
724                     //        In order to make the pre-commit timer working across failovers, though, we need
725                     //        a per-shard cluster-wide monotonic time, so a follower becoming the leader can accurately
726                     //        restart the timer.
727                     pendingTransactions.poll().cohort.reportFailure(new TimeoutException());
728                     break;
729                 case COMMIT_PENDING:
730                     LOG.warn("{}: Transaction {} is still committing, cannot abort", logContext,
731                         currentTx.cohort.getIdentifier());
732                     currentTx.lastAccess = now;
733                     processNext = false;
734                     return;
735                 case ABORTED:
736                 case COMMITTED:
737                 case FAILED:
738                 case READY:
739                 default:
740                     // The suppression of the FindBugs "RV_RETURN_VALUE_IGNORED" warning pertains to this line. In
741                     // this case, we just want to drop the current entry that expired and thus ignore the return value.
742                     // In fact we really shouldn't hit this case but we handle all enums for completeness.
743                     pendingTransactions.poll();
744             }
745
746             if (processNext) {
747                 processNextTransaction();
748             }
749         }
750     }
751
752     @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED", justification = "See inline comment below.")
753     void startAbort(final SimpleShardDataTreeCohort cohort) {
754         final Iterator<CommitEntry> it = pendingTransactions.iterator();
755         if (!it.hasNext()) {
756             LOG.debug("{}: no open transaction while attempting to abort {}", logContext, cohort.getIdentifier());
757             return;
758         }
759
760         // First entry is special, as it may already be committing
761         final CommitEntry first = it.next();
762         if (cohort.equals(first.cohort)) {
763             if (cohort.getState() != State.COMMIT_PENDING) {
764                 LOG.debug("{}: aborted head of queue {} in state {}", logContext, cohort.getIdentifier(),
765                     cohort.getIdentifier());
766
767                 // The suppression of the FindBugs "RV_RETURN_VALUE_IGNORED" warning pertains to this line. In
768                 // this case, we've already obtained the head of the queue above via the Iterator and we just want to
769                 // remove it here.
770                 pendingTransactions.poll();
771                 processNextTransaction();
772             } else {
773                 LOG.warn("{}: transaction {} is committing, skipping abort", logContext, cohort.getIdentifier());
774             }
775
776             return;
777         }
778
779         while (it.hasNext()) {
780             final CommitEntry e = it.next();
781             if (cohort.equals(e.cohort)) {
782                 LOG.debug("{}: aborting queued transaction {}", logContext, cohort.getIdentifier());
783                 it.remove();
784                 return;
785             }
786         }
787
788         LOG.debug("{}: aborted transaction {} not found in the queue", logContext, cohort.getIdentifier());
789     }
790
791     void setRunOnPendingTransactionsComplete(final Runnable operation) {
792         runOnPendingTransactionsComplete = operation;
793         maybeRunOperationOnPendingTransactionsComplete();
794     }
795
796     private void maybeRunOperationOnPendingTransactionsComplete() {
797         if (runOnPendingTransactionsComplete != null && pendingTransactions.isEmpty()) {
798             LOG.debug("{}: Pending transactions complete - running operation {}", logContext,
799                     runOnPendingTransactionsComplete);
800
801             runOnPendingTransactionsComplete.run();
802             runOnPendingTransactionsComplete = null;
803         }
804     }
805 }