613f9adbc9355c4cabb8a776f31069bdfba6660c
[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.MoreObjects;
14 import com.google.common.base.Optional;
15 import com.google.common.base.Preconditions;
16 import com.google.common.base.Stopwatch;
17 import com.google.common.base.Ticker;
18 import com.google.common.base.Verify;
19 import com.google.common.collect.ImmutableList;
20 import com.google.common.collect.ImmutableMap;
21 import com.google.common.collect.ImmutableMap.Builder;
22 import com.google.common.collect.Iterables;
23 import com.google.common.primitives.UnsignedLong;
24 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
25 import java.io.File;
26 import java.io.IOException;
27 import java.util.AbstractMap.SimpleEntry;
28 import java.util.ArrayDeque;
29 import java.util.ArrayList;
30 import java.util.Collection;
31 import java.util.HashMap;
32 import java.util.Iterator;
33 import java.util.Map;
34 import java.util.Map.Entry;
35 import java.util.Queue;
36 import java.util.concurrent.ExecutionException;
37 import java.util.concurrent.TimeUnit;
38 import java.util.concurrent.TimeoutException;
39 import java.util.function.Consumer;
40 import java.util.function.UnaryOperator;
41 import javax.annotation.Nonnull;
42 import javax.annotation.Nullable;
43 import javax.annotation.concurrent.NotThreadSafe;
44 import org.opendaylight.controller.cluster.access.concepts.LocalHistoryIdentifier;
45 import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
46 import org.opendaylight.controller.cluster.datastore.DataTreeCohortActorRegistry.CohortRegistryCommand;
47 import org.opendaylight.controller.cluster.datastore.ShardDataTreeCohort.State;
48 import org.opendaylight.controller.cluster.datastore.persisted.AbstractIdentifiablePayload;
49 import org.opendaylight.controller.cluster.datastore.persisted.CloseLocalHistoryPayload;
50 import org.opendaylight.controller.cluster.datastore.persisted.CommitTransactionPayload;
51 import org.opendaylight.controller.cluster.datastore.persisted.CreateLocalHistoryPayload;
52 import org.opendaylight.controller.cluster.datastore.persisted.MetadataShardDataTreeSnapshot;
53 import org.opendaylight.controller.cluster.datastore.persisted.PurgeLocalHistoryPayload;
54 import org.opendaylight.controller.cluster.datastore.persisted.ShardDataTreeSnapshot;
55 import org.opendaylight.controller.cluster.datastore.persisted.ShardDataTreeSnapshotMetadata;
56 import org.opendaylight.controller.cluster.datastore.utils.DataTreeModificationOutput;
57 import org.opendaylight.controller.cluster.datastore.utils.PruningDataTreeModification;
58 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
59 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataBroker.DataChangeScope;
60 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener;
61 import org.opendaylight.controller.md.sal.common.api.data.OptimisticLockFailedException;
62 import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
63 import org.opendaylight.controller.md.sal.dom.api.DOMDataTreeChangeListener;
64 import org.opendaylight.controller.md.sal.dom.store.impl.DataChangeListenerRegistration;
65 import org.opendaylight.yangtools.concepts.Identifier;
66 import org.opendaylight.yangtools.concepts.ListenerRegistration;
67 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
68 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
69 import org.opendaylight.yangtools.yang.data.api.schema.tree.ConflictingModificationAppliedException;
70 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidate;
71 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidateTip;
72 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidates;
73 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification;
74 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeSnapshot;
75 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeTip;
76 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException;
77 import org.opendaylight.yangtools.yang.data.api.schema.tree.TipProducingDataTree;
78 import org.opendaylight.yangtools.yang.data.api.schema.tree.TipProducingDataTreeTip;
79 import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType;
80 import org.opendaylight.yangtools.yang.data.impl.schema.tree.InMemoryDataTreeFactory;
81 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
82 import org.slf4j.Logger;
83 import org.slf4j.LoggerFactory;
84 import scala.concurrent.duration.Duration;
85
86 /**
87  * Internal shard state, similar to a DOMStore, but optimized for use in the actor system,
88  * e.g. it does not expose public interfaces and assumes it is only ever called from a
89  * single thread.
90  *
91  * <p>
92  * This class is not part of the API contract and is subject to change at any time.
93  */
94 @NotThreadSafe
95 public class ShardDataTree extends ShardDataTreeTransactionParent {
96     private static final class CommitEntry {
97         final SimpleShardDataTreeCohort cohort;
98         long lastAccess;
99
100         CommitEntry(final SimpleShardDataTreeCohort cohort, final long now) {
101             this.cohort = Preconditions.checkNotNull(cohort);
102             lastAccess = now;
103         }
104     }
105
106     private static final Timeout COMMIT_STEP_TIMEOUT = new Timeout(Duration.create(5, TimeUnit.SECONDS));
107     private static final Logger LOG = LoggerFactory.getLogger(ShardDataTree.class);
108
109     private final Map<LocalHistoryIdentifier, ShardDataTreeTransactionChain> transactionChains = new HashMap<>();
110
111     private final DataTreeCohortActorRegistry cohortRegistry = new DataTreeCohortActorRegistry();
112     private final Queue<CommitEntry> pendingTransactions = new ArrayDeque<>();
113     private final Queue<CommitEntry> pendingCommits = new ArrayDeque<>();
114     private final Queue<CommitEntry> pendingFinishCommits = new ArrayDeque<>();
115
116     /**
117      * Callbacks that need to be invoked once a payload is replicated.
118      */
119     private final Map<Payload, Runnable> replicationCallbacks = new HashMap<>();
120
121     private final ShardDataTreeChangeListenerPublisher treeChangeListenerPublisher;
122     private final ShardDataChangeListenerPublisher dataChangeListenerPublisher;
123     private final Collection<ShardDataTreeMetadata<?>> metadata;
124     private final TipProducingDataTree dataTree;
125     private final String logContext;
126     private final Shard shard;
127     private Runnable runOnPendingTransactionsComplete;
128
129     /**
130      * Optimistic {@link DataTreeCandidate} preparation. Since our DataTree implementation is a
131      * {@link TipProducingDataTree}, each {@link DataTreeCandidate} is also a {@link DataTreeTip}, e.g. another
132      * candidate can be prepared on top of it. They still need to be committed in sequence. Here we track the current
133      * tip of the data tree, which is the last DataTreeCandidate we have in flight, or the DataTree itself.
134      */
135     private TipProducingDataTreeTip tip;
136
137     private SchemaContext schemaContext;
138
139     public ShardDataTree(final Shard shard, final SchemaContext schemaContext, final TipProducingDataTree dataTree,
140             final ShardDataTreeChangeListenerPublisher treeChangeListenerPublisher,
141             final ShardDataChangeListenerPublisher dataChangeListenerPublisher, final String logContext,
142             final ShardDataTreeMetadata<?>... metadata) {
143         this.dataTree = Preconditions.checkNotNull(dataTree);
144         updateSchemaContext(schemaContext);
145
146         this.shard = Preconditions.checkNotNull(shard);
147         this.treeChangeListenerPublisher = Preconditions.checkNotNull(treeChangeListenerPublisher);
148         this.dataChangeListenerPublisher = Preconditions.checkNotNull(dataChangeListenerPublisher);
149         this.logContext = Preconditions.checkNotNull(logContext);
150         this.metadata = ImmutableList.copyOf(metadata);
151         tip = dataTree;
152     }
153
154     public ShardDataTree(final Shard shard, final SchemaContext schemaContext, final TreeType treeType,
155             final YangInstanceIdentifier root,
156             final ShardDataTreeChangeListenerPublisher treeChangeListenerPublisher,
157             final ShardDataChangeListenerPublisher dataChangeListenerPublisher, final String logContext) {
158         this(shard, schemaContext, InMemoryDataTreeFactory.getInstance().create(treeType, root),
159                 treeChangeListenerPublisher, dataChangeListenerPublisher, logContext);
160     }
161
162     @VisibleForTesting
163     public ShardDataTree(final Shard shard, final SchemaContext schemaContext, final TreeType treeType) {
164         this(shard, schemaContext, treeType, YangInstanceIdentifier.EMPTY,
165                 new DefaultShardDataTreeChangeListenerPublisher(), new DefaultShardDataChangeListenerPublisher(), "");
166     }
167
168     final String logContext() {
169         return logContext;
170     }
171
172     final Ticker ticker() {
173         return shard.ticker();
174     }
175
176     public TipProducingDataTree getDataTree() {
177         return dataTree;
178     }
179
180     SchemaContext getSchemaContext() {
181         return schemaContext;
182     }
183
184     void updateSchemaContext(final SchemaContext newSchemaContext) {
185         dataTree.setSchemaContext(newSchemaContext);
186         this.schemaContext = Preconditions.checkNotNull(newSchemaContext);
187     }
188
189     /**
190      * Take a snapshot of current state for later recovery.
191      *
192      * @return A state snapshot
193      */
194     @Nonnull ShardDataTreeSnapshot takeStateSnapshot() {
195         final NormalizedNode<?, ?> rootNode = dataTree.takeSnapshot().readNode(YangInstanceIdentifier.EMPTY).get();
196         final Builder<Class<? extends ShardDataTreeSnapshotMetadata<?>>, ShardDataTreeSnapshotMetadata<?>> metaBuilder =
197                 ImmutableMap.builder();
198
199         for (ShardDataTreeMetadata<?> m : metadata) {
200             final ShardDataTreeSnapshotMetadata<?> meta = m.toSnapshot();
201             if (meta != null) {
202                 metaBuilder.put(meta.getType(), meta);
203             }
204         }
205
206         return new MetadataShardDataTreeSnapshot(rootNode, metaBuilder.build());
207     }
208
209     private boolean anyPendingTransactions() {
210         return !pendingTransactions.isEmpty() || !pendingCommits.isEmpty() || !pendingFinishCommits.isEmpty();
211     }
212
213     private void applySnapshot(@Nonnull final ShardDataTreeSnapshot snapshot,
214             final UnaryOperator<DataTreeModification> wrapper) throws DataValidationFailedException {
215         final Stopwatch elapsed = Stopwatch.createStarted();
216
217         if (anyPendingTransactions()) {
218             LOG.warn("{}: applying state snapshot with pending transactions", logContext);
219         }
220
221         final Map<Class<? extends ShardDataTreeSnapshotMetadata<?>>, ShardDataTreeSnapshotMetadata<?>> snapshotMeta;
222         if (snapshot instanceof MetadataShardDataTreeSnapshot) {
223             snapshotMeta = ((MetadataShardDataTreeSnapshot) snapshot).getMetadata();
224         } else {
225             snapshotMeta = ImmutableMap.of();
226         }
227
228         for (ShardDataTreeMetadata<?> m : metadata) {
229             final ShardDataTreeSnapshotMetadata<?> s = snapshotMeta.get(m.getSupportedType());
230             if (s != null) {
231                 m.applySnapshot(s);
232             } else {
233                 m.reset();
234             }
235         }
236
237         final DataTreeModification mod = wrapper.apply(dataTree.takeSnapshot().newModification());
238         // delete everything first
239         mod.delete(YangInstanceIdentifier.EMPTY);
240
241         final java.util.Optional<NormalizedNode<?, ?>> maybeNode = snapshot.getRootNode();
242         if (maybeNode.isPresent()) {
243             // Add everything from the remote node back
244             mod.write(YangInstanceIdentifier.EMPTY, maybeNode.get());
245         }
246         mod.ready();
247
248         final DataTreeModification unwrapped = unwrap(mod);
249         dataTree.validate(unwrapped);
250         DataTreeCandidateTip candidate = dataTree.prepare(unwrapped);
251         dataTree.commit(candidate);
252         notifyListeners(candidate);
253
254         LOG.debug("{}: state snapshot applied in %s", logContext, elapsed);
255     }
256
257     /**
258      * Apply a snapshot coming from the leader. This method assumes the leader and follower SchemaContexts match and
259      * does not perform any pruning.
260      *
261      * @param snapshot Snapshot that needs to be applied
262      * @throws DataValidationFailedException when the snapshot fails to apply
263      */
264     void applySnapshot(@Nonnull final ShardDataTreeSnapshot snapshot) throws DataValidationFailedException {
265         applySnapshot(snapshot, UnaryOperator.identity());
266     }
267
268     private PruningDataTreeModification wrapWithPruning(final DataTreeModification delegate) {
269         return new PruningDataTreeModification(delegate, dataTree, schemaContext);
270     }
271
272     private static DataTreeModification unwrap(final DataTreeModification modification) {
273         if (modification instanceof PruningDataTreeModification) {
274             return ((PruningDataTreeModification)modification).delegate();
275         }
276         return modification;
277     }
278
279     /**
280      * Apply a snapshot coming from recovery. This method does not assume the SchemaContexts match and performs data
281      * pruning in an attempt to adjust the state to our current SchemaContext.
282      *
283      * @param snapshot Snapshot that needs to be applied
284      * @throws DataValidationFailedException when the snapshot fails to apply
285      */
286     void applyRecoverySnapshot(final @Nonnull ShardDataTreeSnapshot snapshot) throws DataValidationFailedException {
287         applySnapshot(snapshot, this::wrapWithPruning);
288     }
289
290     @SuppressWarnings("checkstyle:IllegalCatch")
291     private void applyRecoveryCandidate(final DataTreeCandidate candidate) throws DataValidationFailedException {
292         final PruningDataTreeModification mod = wrapWithPruning(dataTree.takeSnapshot().newModification());
293         DataTreeCandidates.applyToModification(mod, candidate);
294         mod.ready();
295
296         final DataTreeModification unwrapped = mod.delegate();
297         LOG.trace("{}: Applying recovery modification {}", logContext, unwrapped);
298
299         try {
300             dataTree.validate(unwrapped);
301             dataTree.commit(dataTree.prepare(unwrapped));
302         } catch (Exception e) {
303             File file = new File(System.getProperty("karaf.data", "."),
304                     "failed-recovery-payload-" + logContext + ".out");
305             DataTreeModificationOutput.toFile(file, unwrapped);
306             throw new IllegalStateException(String.format(
307                     "%s: Failed to apply recovery payload. Modification data was written to file %s",
308                     logContext, file), e);
309         }
310     }
311
312     /**
313      * Apply a payload coming from recovery. This method does not assume the SchemaContexts match and performs data
314      * pruning in an attempt to adjust the state to our current SchemaContext.
315      *
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 applyRecoveryPayload(final @Nonnull Payload payload) throws IOException, DataValidationFailedException {
321         if (payload instanceof CommitTransactionPayload) {
322             final Entry<TransactionIdentifier, DataTreeCandidate> e =
323                     ((CommitTransactionPayload) payload).getCandidate();
324             applyRecoveryCandidate(e.getValue());
325             allMetadataCommittedTransaction(e.getKey());
326         } else if (payload instanceof CreateLocalHistoryPayload) {
327             allMetadataCreatedLocalHistory(((CreateLocalHistoryPayload) payload).getIdentifier());
328         } else if (payload instanceof CloseLocalHistoryPayload) {
329             allMetadataClosedLocalHistory(((CloseLocalHistoryPayload) payload).getIdentifier());
330         } else if (payload instanceof PurgeLocalHistoryPayload) {
331             allMetadataPurgedLocalHistory(((PurgeLocalHistoryPayload) payload).getIdentifier());
332         } else if (payload instanceof DataTreeCandidatePayload) {
333             applyRecoveryCandidate(((DataTreeCandidatePayload) payload).getCandidate());
334         } else {
335             LOG.debug("{}: ignoring unhandled payload {}", logContext, payload);
336         }
337     }
338
339     private void applyReplicatedCandidate(final Identifier identifier, final DataTreeCandidate foreign)
340             throws DataValidationFailedException {
341         LOG.debug("{}: Applying foreign transaction {}", logContext, identifier);
342
343         final DataTreeModification mod = dataTree.takeSnapshot().newModification();
344         DataTreeCandidates.applyToModification(mod, foreign);
345         mod.ready();
346
347         LOG.trace("{}: Applying foreign modification {}", logContext, mod);
348         dataTree.validate(mod);
349         final DataTreeCandidate candidate = dataTree.prepare(mod);
350         dataTree.commit(candidate);
351
352         notifyListeners(candidate);
353     }
354
355     /**
356      * Apply a payload coming from the leader, which could actually be us. This method assumes the leader and follower
357      * SchemaContexts match and does not perform any pruning.
358      *
359      * @param identifier Payload identifier as returned from RaftActor
360      * @param payload Payload
361      * @throws IOException when the snapshot fails to deserialize
362      * @throws DataValidationFailedException when the snapshot fails to apply
363      */
364     void applyReplicatedPayload(final Identifier identifier, final Payload payload) throws IOException,
365             DataValidationFailedException {
366         /*
367          * 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
368          * if we are the leader and it has originated with us.
369          *
370          * The identifier will only ever be non-null when we were the leader which achieved consensus. Unfortunately,
371          * though, this may not be the case anymore, as we are being called some time afterwards and we may not be
372          * acting in that capacity anymore.
373          *
374          * In any case, we know that this is an entry coming from replication, hence we can be sure we will not observe
375          * pre-Boron state -- which limits the number of options here.
376          */
377         if (payload instanceof CommitTransactionPayload) {
378             if (identifier == null) {
379                 final Entry<TransactionIdentifier, DataTreeCandidate> e =
380                         ((CommitTransactionPayload) payload).getCandidate();
381                 applyReplicatedCandidate(e.getKey(), e.getValue());
382                 allMetadataCommittedTransaction(e.getKey());
383             } else {
384                 Verify.verify(identifier instanceof TransactionIdentifier);
385                 payloadReplicationComplete((TransactionIdentifier) identifier);
386             }
387         } else if (payload instanceof CloseLocalHistoryPayload) {
388             if (identifier != null) {
389                 payloadReplicationComplete((CloseLocalHistoryPayload) payload);
390             } else {
391                 allMetadataClosedLocalHistory(((CloseLocalHistoryPayload) payload).getIdentifier());
392             }
393         } else if (payload instanceof CreateLocalHistoryPayload) {
394             if (identifier != null) {
395                 payloadReplicationComplete((CreateLocalHistoryPayload)payload);
396             } else {
397                 allMetadataCreatedLocalHistory(((CreateLocalHistoryPayload) payload).getIdentifier());
398             }
399         } else if (payload instanceof PurgeLocalHistoryPayload) {
400             if (identifier != null) {
401                 payloadReplicationComplete((PurgeLocalHistoryPayload)payload);
402             } else {
403                 allMetadataPurgedLocalHistory(((PurgeLocalHistoryPayload) payload).getIdentifier());
404             }
405         } else {
406             LOG.warn("{}: ignoring unhandled identifier {} payload {}", logContext, identifier, payload);
407         }
408     }
409
410     private void replicatePayload(final Identifier id, final Payload payload, @Nullable final Runnable callback) {
411         if (callback != null) {
412             replicationCallbacks.put(payload, callback);
413         }
414         shard.persistPayload(id, payload, true);
415     }
416
417     private void payloadReplicationComplete(final AbstractIdentifiablePayload<?> payload) {
418         final Runnable callback = replicationCallbacks.remove(payload);
419         if (callback != null) {
420             LOG.debug("{}: replication of {} completed, invoking {}", logContext, payload.getIdentifier(), callback);
421             callback.run();
422         } else {
423             LOG.debug("{}: replication of {} has no callback", logContext, payload.getIdentifier());
424         }
425     }
426
427     private void payloadReplicationComplete(final TransactionIdentifier txId) {
428         final CommitEntry current = pendingFinishCommits.peek();
429         if (current == null) {
430             LOG.warn("{}: No outstanding transactions, ignoring consensus on transaction {}", logContext, txId);
431             return;
432         }
433
434         if (!current.cohort.getIdentifier().equals(txId)) {
435             LOG.debug("{}: Head of pendingFinishCommits queue is {}, ignoring consensus on transaction {}", logContext,
436                 current.cohort.getIdentifier(), txId);
437             return;
438         }
439
440         finishCommit(current.cohort);
441     }
442
443     private void allMetadataCommittedTransaction(final TransactionIdentifier txId) {
444         for (ShardDataTreeMetadata<?> m : metadata) {
445             m.onTransactionCommitted(txId);
446         }
447     }
448
449     private void allMetadataCreatedLocalHistory(final LocalHistoryIdentifier historyId) {
450         for (ShardDataTreeMetadata<?> m : metadata) {
451             m.onHistoryCreated(historyId);
452         }
453     }
454
455     private void allMetadataClosedLocalHistory(final LocalHistoryIdentifier historyId) {
456         for (ShardDataTreeMetadata<?> m : metadata) {
457             m.onHistoryClosed(historyId);
458         }
459     }
460
461     private void allMetadataPurgedLocalHistory(final LocalHistoryIdentifier historyId) {
462         for (ShardDataTreeMetadata<?> m : metadata) {
463             m.onHistoryPurged(historyId);
464         }
465     }
466
467     ShardDataTreeTransactionChain ensureTransactionChain(final LocalHistoryIdentifier historyId) {
468         ShardDataTreeTransactionChain chain = transactionChains.get(historyId);
469         if (chain == null) {
470             chain = new ShardDataTreeTransactionChain(historyId, this);
471             transactionChains.put(historyId, chain);
472             shard.persistPayload(historyId, CreateLocalHistoryPayload.create(historyId), true);
473         }
474
475         return chain;
476     }
477
478     ReadOnlyShardDataTreeTransaction newReadOnlyTransaction(final TransactionIdentifier txId) {
479         if (txId.getHistoryId().getHistoryId() == 0) {
480             return new ReadOnlyShardDataTreeTransaction(txId, dataTree.takeSnapshot());
481         }
482
483         return ensureTransactionChain(txId.getHistoryId()).newReadOnlyTransaction(txId);
484     }
485
486     ReadWriteShardDataTreeTransaction newReadWriteTransaction(final TransactionIdentifier txId) {
487         if (txId.getHistoryId().getHistoryId() == 0) {
488             return new ReadWriteShardDataTreeTransaction(ShardDataTree.this, txId, dataTree.takeSnapshot()
489                     .newModification());
490         }
491
492         return ensureTransactionChain(txId.getHistoryId()).newReadWriteTransaction(txId);
493     }
494
495     @VisibleForTesting
496     public void notifyListeners(final DataTreeCandidate candidate) {
497         treeChangeListenerPublisher.publishChanges(candidate, logContext);
498         dataChangeListenerPublisher.publishChanges(candidate, logContext);
499     }
500
501     void notifyOfInitialData(final DataChangeListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier,
502             NormalizedNode<?, ?>>> listenerReg, final Optional<DataTreeCandidate> currentState) {
503         if (currentState.isPresent()) {
504             ShardDataChangeListenerPublisher localPublisher = dataChangeListenerPublisher.newInstance();
505             localPublisher.registerDataChangeListener(listenerReg.getPath(), listenerReg.getInstance(),
506                     listenerReg.getScope());
507             localPublisher.publishChanges(currentState.get(), logContext);
508         }
509     }
510
511     void notifyOfInitialData(final YangInstanceIdentifier path, final DOMDataTreeChangeListener listener,
512             final Optional<DataTreeCandidate> currentState) {
513         if (currentState.isPresent()) {
514             ShardDataTreeChangeListenerPublisher localPublisher = treeChangeListenerPublisher.newInstance();
515             localPublisher.registerTreeChangeListener(path, listener);
516             localPublisher.publishChanges(currentState.get(), logContext);
517         }
518     }
519
520     /**
521      * Immediately close all transaction chains.
522      */
523     void closeAllTransactionChains() {
524         for (ShardDataTreeTransactionChain chain : transactionChains.values()) {
525             chain.close();
526         }
527
528         transactionChains.clear();
529     }
530
531     /**
532      * Close a single transaction chain.
533      *
534      * @param id History identifier
535      * @param callback Callback to invoke upon completion, may be null
536      */
537     void closeTransactionChain(final LocalHistoryIdentifier id, @Nullable final Runnable callback) {
538         final ShardDataTreeTransactionChain chain = transactionChains.get(id);
539         if (chain == null) {
540             LOG.debug("{}: Closing non-existent transaction chain {}", logContext, id);
541             if (callback != null) {
542                 callback.run();
543             }
544             return;
545         }
546
547         chain.close();
548         replicatePayload(id, CloseLocalHistoryPayload.create(id), callback);
549     }
550
551     /**
552      * Purge a single transaction chain.
553      *
554      * @param id History identifier
555      * @param callback Callback to invoke upon completion, may be null
556      */
557     void purgeTransactionChain(final LocalHistoryIdentifier id, @Nullable final Runnable callback) {
558         final ShardDataTreeTransactionChain chain = transactionChains.remove(id);
559         if (chain == null) {
560             LOG.debug("{}: Purging non-existent transaction chain {}", logContext, id);
561             if (callback != null) {
562                 callback.run();
563             }
564             return;
565         }
566
567         replicatePayload(id, PurgeLocalHistoryPayload.create(id), callback);
568     }
569
570     Entry<DataChangeListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>>,
571             Optional<DataTreeCandidate>> registerChangeListener(final YangInstanceIdentifier path,
572                     final AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>> listener,
573                     final DataChangeScope scope) {
574         DataChangeListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>> reg =
575                 dataChangeListenerPublisher.registerDataChangeListener(path, listener, scope);
576
577         return new SimpleEntry<>(reg, readCurrentData());
578     }
579
580     private Optional<DataTreeCandidate> readCurrentData() {
581         final Optional<NormalizedNode<?, ?>> currentState =
582                 dataTree.takeSnapshot().readNode(YangInstanceIdentifier.EMPTY);
583         return currentState.isPresent() ? Optional.of(DataTreeCandidates.fromNormalizedNode(
584             YangInstanceIdentifier.EMPTY, currentState.get())) : Optional.<DataTreeCandidate>absent();
585     }
586
587     public Entry<ListenerRegistration<DOMDataTreeChangeListener>, Optional<DataTreeCandidate>>
588             registerTreeChangeListener(final YangInstanceIdentifier path, final DOMDataTreeChangeListener listener) {
589         final ListenerRegistration<DOMDataTreeChangeListener> reg =
590                 treeChangeListenerPublisher.registerTreeChangeListener(path, listener);
591
592         return new SimpleEntry<>(reg, readCurrentData());
593     }
594
595     int getQueueSize() {
596         return pendingTransactions.size() + pendingCommits.size() + pendingFinishCommits.size();
597     }
598
599     @Override
600     void abortTransaction(final AbstractShardDataTreeTransaction<?> transaction) {
601         // Intentional no-op
602     }
603
604     @Override
605     ShardDataTreeCohort finishTransaction(final ReadWriteShardDataTreeTransaction transaction) {
606         final DataTreeModification snapshot = transaction.getSnapshot();
607         snapshot.ready();
608
609         return createReadyCohort(transaction.getIdentifier(), snapshot);
610     }
611
612     public Optional<NormalizedNode<?, ?>> readNode(final YangInstanceIdentifier path) {
613         return dataTree.takeSnapshot().readNode(path);
614     }
615
616     DataTreeSnapshot takeSnapshot() {
617         return dataTree.takeSnapshot();
618     }
619
620     @VisibleForTesting
621     public DataTreeModification newModification() {
622         return dataTree.takeSnapshot().newModification();
623     }
624
625     /**
626      * Commits a modification.
627      *
628      * @deprecated This method violates DataTree containment and will be removed.
629      */
630     @VisibleForTesting
631     @Deprecated
632     public DataTreeCandidate commit(final DataTreeModification modification) throws DataValidationFailedException {
633         // Direct modification commit is a utility, which cannot be used while we have transactions in-flight
634         Preconditions.checkState(tip == dataTree, "Cannot modify data tree while transacgitons are pending");
635
636         modification.ready();
637         dataTree.validate(modification);
638         DataTreeCandidate candidate = dataTree.prepare(modification);
639         dataTree.commit(candidate);
640         return candidate;
641     }
642
643     public Collection<ShardDataTreeCohort> getAndClearPendingTransactions() {
644         Collection<ShardDataTreeCohort> ret = new ArrayList<>(getQueueSize());
645
646         for (CommitEntry entry: pendingFinishCommits) {
647             ret.add(entry.cohort);
648         }
649
650         for (CommitEntry entry: pendingCommits) {
651             ret.add(entry.cohort);
652         }
653
654         for (CommitEntry entry: pendingTransactions) {
655             ret.add(entry.cohort);
656         }
657
658         pendingFinishCommits.clear();
659         pendingCommits.clear();
660         pendingTransactions.clear();
661         tip = dataTree;
662         return ret;
663     }
664
665     @SuppressWarnings("checkstyle:IllegalCatch")
666     private void processNextPendingTransaction() {
667         processNextPending(pendingTransactions, State.CAN_COMMIT_PENDING, entry -> {
668             final SimpleShardDataTreeCohort cohort = entry.cohort;
669             final DataTreeModification modification = cohort.getDataTreeModification();
670
671             LOG.debug("{}: Validating transaction {}", logContext, cohort.getIdentifier());
672             Exception cause;
673             try {
674                 tip.validate(modification);
675                 LOG.debug("{}: Transaction {} validated", logContext, cohort.getIdentifier());
676                 cohort.successfulCanCommit();
677                 entry.lastAccess = ticker().read();
678                 return;
679             } catch (ConflictingModificationAppliedException e) {
680                 LOG.warn("{}: Store Tx {}: Conflicting modification for path {}.", logContext, cohort.getIdentifier(),
681                     e.getPath());
682                 cause = new OptimisticLockFailedException("Optimistic lock failed.", e);
683             } catch (DataValidationFailedException e) {
684                 LOG.warn("{}: Store Tx {}: Data validation failed for path {}.", logContext, cohort.getIdentifier(),
685                     e.getPath(), e);
686
687                 // For debugging purposes, allow dumping of the modification. Coupled with the above
688                 // precondition log, it should allow us to understand what went on.
689                 LOG.debug("{}: Store Tx {}: modifications: {} tree: {}", cohort.getIdentifier(), modification,
690                         dataTree);
691                 cause = new TransactionCommitFailedException("Data did not pass validation.", e);
692             } catch (Exception e) {
693                 LOG.warn("{}: Unexpected failure in validation phase", logContext, e);
694                 cause = e;
695             }
696
697             // Failure path: propagate the failure, remove the transaction from the queue and loop to the next one
698             pendingTransactions.poll().cohort.failedCanCommit(cause);
699         });
700     }
701
702     private void processNextPending() {
703         processNextPendingCommit();
704         processNextPendingTransaction();
705     }
706
707     private void processNextPending(final Queue<CommitEntry> queue, final State allowedState,
708             final Consumer<CommitEntry> processor) {
709         while (!queue.isEmpty()) {
710             final CommitEntry entry = queue.peek();
711             final SimpleShardDataTreeCohort cohort = entry.cohort;
712
713             if (cohort.isFailed()) {
714                 LOG.debug("{}: Removing failed transaction {}", logContext, cohort.getIdentifier());
715                 queue.remove();
716                 continue;
717             }
718
719             if (cohort.getState() == allowedState) {
720                 processor.accept(entry);
721             }
722
723             break;
724         }
725
726         maybeRunOperationOnPendingTransactionsComplete();
727     }
728
729     private void processNextPendingCommit() {
730         processNextPending(pendingCommits, State.COMMIT_PENDING,
731             entry -> startCommit(entry.cohort, entry.cohort.getCandidate()));
732     }
733
734     private boolean peekNextPendingCommit() {
735         final CommitEntry first = pendingCommits.peek();
736         return first != null && first.cohort.getState() == State.COMMIT_PENDING;
737     }
738
739     void startCanCommit(final SimpleShardDataTreeCohort cohort) {
740         final SimpleShardDataTreeCohort current = pendingTransactions.peek().cohort;
741         if (!cohort.equals(current)) {
742             LOG.debug("{}: Transaction {} scheduled for canCommit step", logContext, cohort.getIdentifier());
743             return;
744         }
745
746         processNextPendingTransaction();
747     }
748
749     private void failPreCommit(final Exception cause) {
750         shard.getShardMBean().incrementFailedTransactionsCount();
751         pendingTransactions.poll().cohort.failedPreCommit(cause);
752         processNextPendingTransaction();
753     }
754
755     @SuppressWarnings("checkstyle:IllegalCatch")
756     void startPreCommit(final SimpleShardDataTreeCohort cohort) {
757         final CommitEntry entry = pendingTransactions.peek();
758         Preconditions.checkState(entry != null, "Attempted to pre-commit of %s when no transactions pending", cohort);
759
760         final SimpleShardDataTreeCohort current = entry.cohort;
761         Verify.verify(cohort.equals(current), "Attempted to pre-commit %s while %s is pending", cohort, current);
762
763         LOG.debug("{}: Preparing transaction {}", logContext, current.getIdentifier());
764
765         final DataTreeCandidateTip candidate;
766         try {
767             candidate = tip.prepare(cohort.getDataTreeModification());
768             cohort.userPreCommit(candidate);
769         } catch (ExecutionException | TimeoutException | RuntimeException e) {
770             failPreCommit(e);
771             return;
772         }
773
774         // Set the tip of the data tree.
775         tip = Verify.verifyNotNull(candidate);
776
777         entry.lastAccess = ticker().read();
778
779         pendingTransactions.remove();
780         pendingCommits.add(entry);
781
782         LOG.debug("{}: Transaction {} prepared", logContext, current.getIdentifier());
783
784         cohort.successfulPreCommit(candidate);
785
786         processNextPendingTransaction();
787     }
788
789     private void failCommit(final Exception cause) {
790         shard.getShardMBean().incrementFailedTransactionsCount();
791         pendingFinishCommits.poll().cohort.failedCommit(cause);
792         processNextPending();
793     }
794
795     @SuppressWarnings("checkstyle:IllegalCatch")
796     private void finishCommit(final SimpleShardDataTreeCohort cohort) {
797         final TransactionIdentifier txId = cohort.getIdentifier();
798         final DataTreeCandidate candidate = cohort.getCandidate();
799
800         LOG.debug("{}: Resuming commit of transaction {}", logContext, txId);
801
802         if (tip == candidate) {
803             // All pending candidates have been committed, reset the tip to the data tree.
804             tip = dataTree;
805         }
806
807         try {
808             dataTree.commit(candidate);
809         } catch (Exception e) {
810             LOG.error("{}: Failed to commit transaction {}", logContext, txId, e);
811             failCommit(e);
812             return;
813         }
814
815         shard.getShardMBean().incrementCommittedTransactionCount();
816         shard.getShardMBean().setLastCommittedTransactionTime(System.currentTimeMillis());
817
818         // FIXME: propagate journal index
819         pendingFinishCommits.poll().cohort.successfulCommit(UnsignedLong.ZERO);
820
821         LOG.trace("{}: Transaction {} committed, proceeding to notify", logContext, txId);
822         notifyListeners(candidate);
823
824         processNextPending();
825     }
826
827     void startCommit(final SimpleShardDataTreeCohort cohort, final DataTreeCandidate candidate) {
828         final CommitEntry entry = pendingCommits.peek();
829         Preconditions.checkState(entry != null, "Attempted to start commit of %s when no transactions pending", cohort);
830
831         final SimpleShardDataTreeCohort current = entry.cohort;
832         if (!cohort.equals(current)) {
833             LOG.debug("{}: Transaction {} scheduled for commit step", logContext, cohort.getIdentifier());
834             return;
835         }
836
837         LOG.debug("{}: Starting commit for transaction {}", logContext, current.getIdentifier());
838
839         final TransactionIdentifier txId = cohort.getIdentifier();
840         final Payload payload;
841         try {
842             payload = CommitTransactionPayload.create(txId, candidate);
843         } catch (IOException e) {
844             LOG.error("{}: Failed to encode transaction {} candidate {}", logContext, txId, candidate, e);
845             pendingCommits.poll().cohort.failedCommit(e);
846             processNextPending();
847             return;
848         }
849
850         // We process next transactions pending canCommit before we call persistPayload to possibly progress subsequent
851         // transactions to the COMMIT_PENDING state so the payloads can be batched for replication. This is done for
852         // single-shard transactions that immediately transition from canCommit to preCommit to commit. Note that
853         // if the next pending transaction is progressed to COMMIT_PENDING and this method (startCommit) is called,
854         // the next transaction will not attempt to replicate b/c the current transaction is still at the head of the
855         // pendingCommits queue.
856         processNextPendingTransaction();
857
858         // After processing next pending transactions, we can now remove the current transaction from pendingCommits.
859         // Note this must be done before the call to peekNextPendingCommit below so we check the next transaction
860         // in order to properly determine the batchHint flag for the call to persistPayload.
861         pendingCommits.remove();
862         pendingFinishCommits.add(entry);
863
864         // See if the next transaction is pending commit (ie in the COMMIT_PENDING state) so it can be batched with
865         // this transaction for replication.
866         boolean replicationBatchHint = peekNextPendingCommit();
867
868         // Once completed, we will continue via payloadReplicationComplete
869         shard.persistPayload(txId, payload, replicationBatchHint);
870
871         entry.lastAccess = shard.ticker().read();
872
873         LOG.debug("{}: Transaction {} submitted to persistence", logContext, txId);
874
875         // Process the next transaction pending commit, if any. If there is one it will be batched with this
876         // transaction for replication.
877         processNextPendingCommit();
878     }
879
880     Collection<ActorRef> getCohortActors() {
881         return cohortRegistry.getCohortActors();
882     }
883
884     void processCohortRegistryCommand(final ActorRef sender, final CohortRegistryCommand message) {
885         cohortRegistry.process(sender, message);
886     }
887
888     @Override
889     ShardDataTreeCohort createReadyCohort(final TransactionIdentifier txId,
890             final DataTreeModification modification) {
891         SimpleShardDataTreeCohort cohort = new SimpleShardDataTreeCohort(this, modification, txId,
892                 cohortRegistry.createCohort(schemaContext, txId, COMMIT_STEP_TIMEOUT));
893         pendingTransactions.add(new CommitEntry(cohort, ticker().read()));
894         return cohort;
895     }
896
897     @SuppressFBWarnings(value = "DB_DUPLICATE_SWITCH_CLAUSES", justification = "See inline comments below.")
898     void checkForExpiredTransactions(final long transactionCommitTimeoutMillis) {
899         final long timeout = TimeUnit.MILLISECONDS.toNanos(transactionCommitTimeoutMillis);
900         final long now = ticker().read();
901
902         final Queue<CommitEntry> currentQueue = !pendingFinishCommits.isEmpty() ? pendingFinishCommits :
903             !pendingCommits.isEmpty() ? pendingCommits : pendingTransactions;
904         final CommitEntry currentTx = currentQueue.peek();
905         if (currentTx != null && currentTx.lastAccess + timeout < now) {
906             LOG.warn("{}: Current transaction {} has timed out after {} ms in state {}", logContext,
907                     currentTx.cohort.getIdentifier(), transactionCommitTimeoutMillis, currentTx.cohort.getState());
908             boolean processNext = true;
909             switch (currentTx.cohort.getState()) {
910                 case CAN_COMMIT_PENDING:
911                     currentQueue.remove().cohort.failedCanCommit(new TimeoutException());
912                     break;
913                 case CAN_COMMIT_COMPLETE:
914                     // The suppression of the FindBugs "DB_DUPLICATE_SWITCH_CLAUSES" warning pertains to this clause
915                     // whose code is duplicated with PRE_COMMIT_COMPLETE. The clauses aren't combined in case the code
916                     // in PRE_COMMIT_COMPLETE is changed.
917                     currentQueue.remove().cohort.reportFailure(new TimeoutException());
918                     break;
919                 case PRE_COMMIT_PENDING:
920                     currentQueue.remove().cohort.failedPreCommit(new TimeoutException());
921                     break;
922                 case PRE_COMMIT_COMPLETE:
923                     // FIXME: this is a legacy behavior problem. Three-phase commit protocol specifies that after we
924                     //        are ready we should commit the transaction, not abort it. Our current software stack does
925                     //        not allow us to do that consistently, because we persist at the time of commit, hence
926                     //        we can end up in a state where we have pre-committed a transaction, then a leader failover
927                     //        occurred ... the new leader does not see the pre-committed transaction and does not have
928                     //        a running timer. To fix this we really need two persistence events.
929                     //
930                     //        The first one, done at pre-commit time will hold the transaction payload. When consensus
931                     //        is reached, we exit the pre-commit phase and start the pre-commit timer. Followers do not
932                     //        apply the state in this event.
933                     //
934                     //        The second one, done at commit (or abort) time holds only the transaction identifier and
935                     //        signals to followers that the state should (or should not) be applied.
936                     //
937                     //        In order to make the pre-commit timer working across failovers, though, we need
938                     //        a per-shard cluster-wide monotonic time, so a follower becoming the leader can accurately
939                     //        restart the timer.
940                     currentQueue.remove().cohort.reportFailure(new TimeoutException());
941                     break;
942                 case COMMIT_PENDING:
943                     LOG.warn("{}: Transaction {} is still committing, cannot abort", logContext,
944                         currentTx.cohort.getIdentifier());
945                     currentTx.lastAccess = now;
946                     processNext = false;
947                     return;
948                 case ABORTED:
949                 case COMMITTED:
950                 case FAILED:
951                 case READY:
952                 default:
953                     currentQueue.remove();
954             }
955
956             if (processNext) {
957                 processNextPending();
958             }
959         }
960     }
961
962     boolean startAbort(final SimpleShardDataTreeCohort cohort) {
963         final Iterator<CommitEntry> it = Iterables.concat(pendingFinishCommits, pendingCommits,
964                 pendingTransactions).iterator();
965         if (!it.hasNext()) {
966             LOG.debug("{}: no open transaction while attempting to abort {}", logContext, cohort.getIdentifier());
967             return true;
968         }
969
970         // First entry is special, as it may already be committing
971         final CommitEntry first = it.next();
972         if (cohort.equals(first.cohort)) {
973             if (cohort.getState() != State.COMMIT_PENDING) {
974                 LOG.debug("{}: aborting head of queue {} in state {}", logContext, cohort.getIdentifier(),
975                     cohort.getIdentifier());
976
977                 it.remove();
978                 if (cohort.getCandidate() != null) {
979                     rebaseTransactions(it, dataTree);
980                 }
981
982                 processNextPending();
983                 return true;
984             }
985
986             LOG.warn("{}: transaction {} is committing, skipping abort", logContext, cohort.getIdentifier());
987             return false;
988         }
989
990         TipProducingDataTreeTip newTip = MoreObjects.firstNonNull(first.cohort.getCandidate(), dataTree);
991         while (it.hasNext()) {
992             final CommitEntry e = it.next();
993             if (cohort.equals(e.cohort)) {
994                 LOG.debug("{}: aborting queued transaction {}", logContext, cohort.getIdentifier());
995
996                 it.remove();
997                 if (cohort.getCandidate() != null) {
998                     rebaseTransactions(it, newTip);
999                 }
1000
1001                 return true;
1002             } else {
1003                 newTip = MoreObjects.firstNonNull(e.cohort.getCandidate(), newTip);
1004             }
1005         }
1006
1007         LOG.debug("{}: aborted transaction {} not found in the queue", logContext, cohort.getIdentifier());
1008         return true;
1009     }
1010
1011     @SuppressWarnings("checkstyle:IllegalCatch")
1012     private void rebaseTransactions(final Iterator<CommitEntry> iter, @Nonnull final TipProducingDataTreeTip newTip) {
1013         tip = Preconditions.checkNotNull(newTip);
1014         while (iter.hasNext()) {
1015             final SimpleShardDataTreeCohort cohort = iter.next().cohort;
1016             if (cohort.getState() == State.CAN_COMMIT_COMPLETE) {
1017                 LOG.debug("{}: Revalidating queued transaction {}", logContext, cohort.getIdentifier());
1018
1019                 try {
1020                     tip.validate(cohort.getDataTreeModification());
1021                 } catch (DataValidationFailedException | RuntimeException e) {
1022                     LOG.debug("{}: Failed to revalidate queued transaction {}", logContext, cohort.getIdentifier(), e);
1023                     cohort.reportFailure(e);
1024                 }
1025             } else if (cohort.getState() == State.PRE_COMMIT_COMPLETE) {
1026                 LOG.debug("{}: Repreparing queued transaction {}", logContext, cohort.getIdentifier());
1027
1028                 try {
1029                     tip.validate(cohort.getDataTreeModification());
1030                     DataTreeCandidateTip candidate = tip.prepare(cohort.getDataTreeModification());
1031                     cohort.userPreCommit(candidate);
1032
1033                     cohort.setNewCandidate(candidate);
1034                     tip = candidate;
1035                 } catch (ExecutionException | TimeoutException | RuntimeException | DataValidationFailedException e) {
1036                     LOG.debug("{}: Failed to reprepare queued transaction {}", logContext, cohort.getIdentifier(), e);
1037                     cohort.reportFailure(e);
1038                 }
1039             }
1040         }
1041     }
1042
1043     void setRunOnPendingTransactionsComplete(final Runnable operation) {
1044         runOnPendingTransactionsComplete = operation;
1045         maybeRunOperationOnPendingTransactionsComplete();
1046     }
1047
1048     private void maybeRunOperationOnPendingTransactionsComplete() {
1049         if (runOnPendingTransactionsComplete != null && !anyPendingTransactions()) {
1050             LOG.debug("{}: Pending transactions complete - running operation {}", logContext,
1051                     runOnPendingTransactionsComplete);
1052
1053             runOnPendingTransactionsComplete.run();
1054             runOnPendingTransactionsComplete = null;
1055         }
1056     }
1057 }