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