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