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