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