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