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