Change DOM read/exists to FluentFuture
[mdsal.git] / dom / mdsal-dom-broker / src / main / java / org / opendaylight / mdsal / dom / broker / pingpong / PingPongTransactionChain.java
1 /*
2  * Copyright (c) 2014 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.mdsal.dom.broker.pingpong;
9
10 import com.google.common.base.Preconditions;
11 import com.google.common.base.Verify;
12 import com.google.common.util.concurrent.FluentFuture;
13 import com.google.common.util.concurrent.FutureCallback;
14 import com.google.common.util.concurrent.MoreExecutors;
15 import java.util.AbstractMap.SimpleImmutableEntry;
16 import java.util.Map.Entry;
17 import java.util.Optional;
18 import java.util.concurrent.CancellationException;
19 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
20 import javax.annotation.Nonnull;
21 import javax.annotation.concurrent.GuardedBy;
22 import org.opendaylight.mdsal.common.api.AsyncTransaction;
23 import org.opendaylight.mdsal.common.api.CommitInfo;
24 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
25 import org.opendaylight.mdsal.common.api.TransactionChain;
26 import org.opendaylight.mdsal.common.api.TransactionChainListener;
27 import org.opendaylight.mdsal.dom.api.DOMDataBroker;
28 import org.opendaylight.mdsal.dom.api.DOMDataTreeReadTransaction;
29 import org.opendaylight.mdsal.dom.api.DOMDataTreeReadWriteTransaction;
30 import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteTransaction;
31 import org.opendaylight.mdsal.dom.api.DOMTransactionChain;
32 import org.opendaylight.mdsal.dom.spi.ForwardingDOMDataReadWriteTransaction;
33 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
34 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
35 import org.slf4j.Logger;
36 import org.slf4j.LoggerFactory;
37
38 /**
39  * An implementation of {@link DOMTransactionChain}, which has a very specific
40  * behavior, which some users may find surprising. If keeps the general
41  * intent of the contract, but it makes sure there are never more than two
42  * transactions allocated at any given time: one of them is being committed,
43  * and while that is happening, the other one acts as the scratch pad. Once
44  * the committing transaction completes successfully, the scratch transaction
45  * is enqueued as soon as it is ready.
46  *
47  * <p>
48  * This mode of operation means that there is no inherent isolation between
49  * the front-end transactions and transactions cannot be reasonably cancelled.
50  *
51  * <p>
52  * It furthermore means that the transactions returned by {@link #newReadOnlyTransaction()}
53  * counts as an outstanding transaction and the user may not allocate multiple
54  * read-only transactions at the same time.
55  */
56 public final class PingPongTransactionChain implements DOMTransactionChain {
57     private static final Logger LOG = LoggerFactory.getLogger(PingPongTransactionChain.class);
58     private final TransactionChainListener listener;
59     private final DOMTransactionChain delegate;
60
61     @GuardedBy("this")
62     private boolean failed;
63     @GuardedBy("this")
64     private PingPongTransaction shutdownTx;
65     @GuardedBy("this")
66     private Entry<PingPongTransaction, Throwable> deadTx;
67
68     /**
69      * This updater is used to manipulate the "ready" transaction. We perform only atomic
70      * get-and-set on it.
71      */
72     private static final AtomicReferenceFieldUpdater<PingPongTransactionChain, PingPongTransaction> READY_UPDATER
73             = AtomicReferenceFieldUpdater
74             .newUpdater(PingPongTransactionChain.class, PingPongTransaction.class, "readyTx");
75     private volatile PingPongTransaction readyTx;
76
77     /**
78      * This updater is used to manipulate the "locked" transaction. A locked transaction
79      * means we know that the user still holds a transaction and should at some point call
80      * us. We perform on compare-and-swap to ensure we properly detect when a user is
81      * attempting to allocated multiple transactions concurrently.
82      */
83     private static final AtomicReferenceFieldUpdater<PingPongTransactionChain, PingPongTransaction> LOCKED_UPDATER
84             = AtomicReferenceFieldUpdater
85             .newUpdater(PingPongTransactionChain.class, PingPongTransaction.class, "lockedTx");
86     private volatile PingPongTransaction lockedTx;
87
88     /**
89      * This updater is used to manipulate the "inflight" transaction. There can be at most
90      * one of these at any given time. We perform only compare-and-swap on these.
91      */
92     private static final AtomicReferenceFieldUpdater<PingPongTransactionChain, PingPongTransaction> INFLIGHT_UPDATER
93             = AtomicReferenceFieldUpdater
94             .newUpdater(PingPongTransactionChain.class, PingPongTransaction.class, "inflightTx");
95     private volatile PingPongTransaction inflightTx;
96
97     PingPongTransactionChain(final DOMDataBroker broker, final TransactionChainListener listener) {
98         this.listener = Preconditions.checkNotNull(listener);
99         this.delegate = broker.createTransactionChain(new TransactionChainListener() {
100             @Override
101             public void onTransactionChainFailed(final TransactionChain<?, ?> chain,
102                                                  final AsyncTransaction<?, ?> transaction, final Throwable cause) {
103                 LOG.debug("Transaction chain {} reported failure in {}", chain, transaction, cause);
104                 delegateFailed(chain, cause);
105             }
106
107             @Override
108             public void onTransactionChainSuccessful(final TransactionChain<?, ?> chain) {
109                 delegateSuccessful(chain);
110             }
111         });
112     }
113
114     void delegateSuccessful(final TransactionChain<?, ?> chain) {
115         final Entry<PingPongTransaction, Throwable> canceled;
116         synchronized (this) {
117             // This looks weird, but we need not hold the lock while invoking callbacks
118             canceled = deadTx;
119         }
120
121         if (canceled == null) {
122             listener.onTransactionChainSuccessful(this);
123             return;
124         }
125
126         // Backend shutdown successful, but we have a batch of transactions we have to report as dead due to the
127         // user calling cancel().
128         final PingPongTransaction tx = canceled.getKey();
129         final Throwable cause = canceled.getValue();
130         LOG.debug("Transaction chain {} successful, failing cancelled transaction {}", chain, tx, cause);
131
132         listener.onTransactionChainFailed(this, tx.getFrontendTransaction(), cause);
133         tx.onFailure(cause);
134     }
135
136     void delegateFailed(final TransactionChain<?, ?> chain, final Throwable cause) {
137
138         final DOMDataTreeReadWriteTransaction frontend;
139         final PingPongTransaction tx = inflightTx;
140         if (tx == null) {
141             LOG.warn("Transaction chain {} failed with no pending transactions", chain);
142             frontend = null;
143         } else {
144             frontend = tx.getFrontendTransaction();
145         }
146
147         listener.onTransactionChainFailed(this, frontend, cause);
148
149         synchronized (this) {
150             failed = true;
151
152             /*
153              * If we do not have a locked transaction, we need to ensure that
154              * the backend transaction is cancelled. Otherwise we can defer
155              * until the user calls us.
156              */
157             if (lockedTx == null) {
158                 processIfReady();
159             }
160         }
161     }
162
163     private synchronized PingPongTransaction slowAllocateTransaction() {
164         Preconditions.checkState(shutdownTx == null, "Transaction chain %s has been shut down", this);
165
166         if (deadTx != null) {
167             throw new IllegalStateException(
168                     String.format("Transaction chain %s has failed due to transaction %s being canceled", this,
169                                   deadTx.getKey()), deadTx.getValue());
170         }
171
172         final DOMDataTreeReadWriteTransaction delegateTx = delegate.newReadWriteTransaction();
173         final PingPongTransaction newTx = new PingPongTransaction(delegateTx);
174
175         if (!LOCKED_UPDATER.compareAndSet(this, null, newTx)) {
176             delegateTx.cancel();
177             throw new IllegalStateException(
178                     String.format("New transaction %s raced with transaction %s", newTx, lockedTx));
179         }
180
181         return newTx;
182     }
183
184     private PingPongTransaction allocateTransaction() {
185         // Step 1: acquire current state
186         final PingPongTransaction oldTx = READY_UPDATER.getAndSet(this, null);
187
188         // Slow path: allocate a delegate transaction
189         if (oldTx == null) {
190             return slowAllocateTransaction();
191         }
192
193         // Fast path: reuse current transaction. We will check failures and similar on commit().
194         if (!LOCKED_UPDATER.compareAndSet(this, null, oldTx)) {
195             // Ouch. Delegate chain has not detected a duplicate transaction allocation. This is the best we can do.
196             oldTx.getTransaction().cancel();
197             throw new IllegalStateException(
198                     String.format("Reusable transaction %s raced with transaction %s", oldTx, lockedTx));
199         }
200
201         return oldTx;
202     }
203
204     /*
205      * This forces allocateTransaction() on a slow path, which has to happen after
206      * this method has completed executing. Also inflightTx may be updated outside
207      * the lock, hence we need to re-check.
208      */
209     @GuardedBy("this")
210     private void processIfReady() {
211         if (inflightTx == null) {
212             final PingPongTransaction tx = READY_UPDATER.getAndSet(this, null);
213             if (tx != null) {
214                 processTransaction(tx);
215             }
216         }
217     }
218
219     /**
220      * Process a ready transaction. The caller needs to ensure that
221      * each transaction is seen only once by this method.
222      *
223      * @param tx Transaction which needs processing.
224      */
225     @GuardedBy("this")
226     private void processTransaction(@Nonnull final PingPongTransaction tx) {
227         if (failed) {
228             LOG.debug("Cancelling transaction {}", tx);
229             tx.getTransaction().cancel();
230             return;
231         }
232
233         LOG.debug("Submitting transaction {}", tx);
234         if (!INFLIGHT_UPDATER.compareAndSet(this, null, tx)) {
235             LOG.warn("Submitting transaction {} while {} is still running", tx, inflightTx);
236         }
237
238         tx.getTransaction().commit().addCallback(new FutureCallback<CommitInfo>() {
239             @Override
240             public void onSuccess(final CommitInfo result) {
241                 transactionSuccessful(tx, result);
242             }
243
244             @Override
245             public void onFailure(final Throwable throwable) {
246                 transactionFailed(tx, throwable);
247             }
248         }, MoreExecutors.directExecutor());
249     }
250
251     /*
252      * We got invoked from the data store thread. We need to do two things:
253      * 1) release the in-flight transaction
254      * 2) process the potential next transaction
255      *
256      * We have to perform 2) under lock. We could perform 1) without locking, but that means the CAS result may
257      * not be accurate, as a user thread may submit the ready transaction before we acquire the lock -- and checking
258      * for next transaction is not enough, as that may have also be allocated (as a result of a quick
259      * submit/allocate/submit between 1) and 2)). Hence we'd end up doing the following:
260      * 1) CAS of inflightTx
261      * 2) take lock
262      * 3) volatile read of inflightTx
263      *
264      * Rather than doing that, we keep this method synchronized, hence performing only:
265      * 1) take lock
266      * 2) CAS of inflightTx
267      *
268      * Since the user thread is barred from submitting the transaction (in processIfReady), we can then proceed with
269      * the knowledge that inflightTx is null -- processTransaction() will still do a CAS, but that is only for
270      * correctness.
271      */
272     private synchronized void processNextTransaction(final PingPongTransaction tx) {
273         final boolean success = INFLIGHT_UPDATER.compareAndSet(this, tx, null);
274         Preconditions.checkState(success, "Completed transaction %s while %s was submitted", tx, inflightTx);
275
276         final PingPongTransaction nextTx = READY_UPDATER.getAndSet(this, null);
277         if (nextTx != null) {
278             processTransaction(nextTx);
279         } else if (shutdownTx != null) {
280             processTransaction(shutdownTx);
281             delegate.close();
282             shutdownTx = null;
283         }
284     }
285
286     void transactionSuccessful(final PingPongTransaction tx, final CommitInfo result) {
287         LOG.debug("Transaction {} completed successfully", tx);
288
289         tx.onSuccess(result);
290         processNextTransaction(tx);
291     }
292
293     void transactionFailed(final PingPongTransaction tx, final Throwable throwable) {
294         LOG.debug("Transaction {} failed", tx, throwable);
295
296         tx.onFailure(throwable);
297         processNextTransaction(tx);
298     }
299
300     void readyTransaction(@Nonnull final PingPongTransaction tx) {
301         // First mark the transaction as not locked.
302         final boolean lockedMatch = LOCKED_UPDATER.compareAndSet(this, tx, null);
303         Preconditions.checkState(lockedMatch, "Attempted to submit transaction %s while we have %s", tx, lockedTx);
304         LOG.debug("Transaction {} unlocked", tx);
305
306         /*
307          * The transaction is ready. It will then be picked up by either next allocation,
308          * or a background transaction completion callback.
309          */
310         final boolean success = READY_UPDATER.compareAndSet(this, null, tx);
311         Preconditions.checkState(success, "Transaction %s collided on ready state", tx, readyTx);
312         LOG.debug("Transaction {} readied", tx);
313
314         /*
315          * We do not see a transaction being in-flight, so we need to take care of dispatching
316          * the transaction to the backend. We are in the ready case, we cannot short-cut
317          * the checking of readyTx, as an in-flight transaction may have completed between us
318          * setting the field above and us checking.
319          */
320         if (inflightTx == null) {
321             synchronized (this) {
322                 processIfReady();
323             }
324         }
325     }
326
327     /**
328      * Transaction cancellation is a heavyweight operation. We only support cancelation of a locked transaction
329      * and return false for everything else. Cancelling such a transaction will result in all transactions in the
330      * batch to be cancelled.
331      *
332      * @param tx         Backend shared transaction
333      * @param frontendTx transaction
334      * @param isOpen     indicator whether the transaction was already closed
335      */
336     synchronized void cancelTransaction(final PingPongTransaction tx,
337             final DOMDataTreeReadWriteTransaction frontendTx) {
338         // Attempt to unlock the operation.
339         final boolean lockedMatch = LOCKED_UPDATER.compareAndSet(this, tx, null);
340         Verify.verify(lockedMatch, "Cancelling transaction %s collided with locked transaction %s", tx, lockedTx);
341
342         // Cancel the backend transaction, so we do not end up leaking it.
343         final boolean backendCancelled = tx.getTransaction().cancel();
344
345         if (failed) {
346             // The transaction has failed, this is probably the user just clearing up the transaction they had. We have
347             // already cancelled the transaction anyway,
348             return;
349         } else if (!backendCancelled) {
350             LOG.warn("Backend transaction cannot be cancelled during cancellation of {}, attempting to continue", tx);
351         }
352
353         // We have dealt with canceling the backend transaction and have unlocked the transaction. Since we are still
354         // inside the synchronized block, any allocations are blocking on the slow path. Now we have to decide the fate
355         // of this transaction chain.
356         //
357         // If there are no other frontend transactions in this batch we are aligned with backend state and we can
358         // continue processing.
359         if (frontendTx.equals(tx.getFrontendTransaction())) {
360             LOG.debug("Cancelled transaction {} was head of the batch, resuming processing", tx);
361             return;
362         }
363
364         // There are multiple frontend transactions in this batch. We have to report them as failed, which dooms this
365         // transaction chain, too. Since we just came off of a locked transaction, we do not have a ready transaction
366         // at the moment, but there may be some transaction in-flight. So we proceed to shutdown the backend chain
367         // and mark the fact that we should be turning its completion into a failure.
368         deadTx = new SimpleImmutableEntry<>(tx, new CancellationException("Transaction " + frontendTx + " canceled")
369                 .fillInStackTrace());
370         delegate.close();
371     }
372
373     @Override
374     public synchronized void close() {
375         final PingPongTransaction notLocked = lockedTx;
376         Preconditions
377                 .checkState(notLocked == null, "Attempted to close chain with outstanding transaction %s", notLocked);
378
379         // This is not reliable, but if we observe it to be null and the process has already completed,
380         // the backend transaction chain will throw the appropriate error.
381         Preconditions.checkState(shutdownTx == null, "Attempted to close an already-closed chain");
382
383         // This may be a reaction to our failure callback, in that case the backend is already shutdown
384         if (deadTx != null) {
385             LOG.debug("Delegate {} is already closed due to failure {}", delegate, deadTx);
386             return;
387         }
388
389         // Force allocations on slow path, picking up a potentially-outstanding transaction
390         final PingPongTransaction tx = READY_UPDATER.getAndSet(this, null);
391
392         if (tx != null) {
393             // We have one more transaction, which needs to be processed somewhere. If we do not
394             // a transaction in-flight, we need to push it down ourselves.
395             // If there is an in-flight transaction we will schedule this last one into a dedicated
396             // slot. Allocation slow path will check its presence and fail, the in-flight path will
397             // pick it up, submit and immediately close the chain.
398             if (inflightTx == null) {
399                 processTransaction(tx);
400                 delegate.close();
401             } else {
402                 shutdownTx = tx;
403             }
404         } else {
405             // Nothing outstanding, we can safely shutdown
406             delegate.close();
407         }
408     }
409
410     @Override
411     public DOMDataTreeReadTransaction newReadOnlyTransaction() {
412         final PingPongTransaction tx = allocateTransaction();
413
414         return new DOMDataTreeReadTransaction() {
415             @Override
416             public FluentFuture<Optional<NormalizedNode<?, ?>>> read(
417                     final LogicalDatastoreType store, final YangInstanceIdentifier path) {
418                 return tx.getTransaction().read(store, path);
419             }
420
421             @Override
422             public FluentFuture<Boolean> exists(final LogicalDatastoreType store, final YangInstanceIdentifier path) {
423                 return tx.getTransaction().exists(store, path);
424             }
425
426             @Override
427             public Object getIdentifier() {
428                 return tx.getTransaction().getIdentifier();
429             }
430
431             @Override
432             public void close() {
433                 readyTransaction(tx);
434             }
435         };
436     }
437
438     @Override
439     public DOMDataTreeReadWriteTransaction newReadWriteTransaction() {
440         final PingPongTransaction tx = allocateTransaction();
441         final DOMDataTreeReadWriteTransaction ret = new ForwardingDOMDataReadWriteTransaction() {
442             private boolean isOpen = true;
443
444             @Override
445             protected DOMDataTreeReadWriteTransaction delegate() {
446                 return tx.getTransaction();
447             }
448
449             @Override
450             public FluentFuture<? extends CommitInfo> commit() {
451                 readyTransaction(tx);
452                 isOpen = false;
453                 return FluentFuture.from(tx.getCommitFuture()).transformAsync(
454                     ignored -> CommitInfo.emptyFluentFuture(), MoreExecutors.directExecutor());
455             }
456
457             @Override
458             public boolean cancel() {
459                 if (isOpen) {
460                     cancelTransaction(tx, this);
461                     isOpen = false;
462                     return true;
463                 } else {
464                     return false;
465                 }
466             }
467
468             @Override
469             public void close() {
470                 // TODO Auto-generated method stub
471
472             }
473         };
474
475         tx.recordFrontendTransaction(ret);
476         return ret;
477     }
478
479     @Override
480     public DOMDataTreeWriteTransaction newWriteOnlyTransaction() {
481         return newReadWriteTransaction();
482     }
483 }