Bump upstreams
[mdsal.git] / dom / mdsal-dom-spi / src / main / java / org / opendaylight / mdsal / dom / spi / AbstractPingPongTransactionChain.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 com.google.common.base.Verify.verify;
12 import static java.util.Objects.requireNonNull;
13
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 edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
18 import java.lang.invoke.MethodHandles;
19 import java.lang.invoke.VarHandle;
20 import java.util.Map;
21 import java.util.Map.Entry;
22 import java.util.Optional;
23 import java.util.concurrent.CancellationException;
24 import java.util.function.Function;
25 import org.checkerframework.checker.lock.qual.GuardedBy;
26 import org.checkerframework.checker.lock.qual.Holding;
27 import org.eclipse.jdt.annotation.NonNull;
28 import org.eclipse.jdt.annotation.Nullable;
29 import org.opendaylight.mdsal.common.api.CommitInfo;
30 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
31 import org.opendaylight.mdsal.dom.api.DOMDataTreeReadTransaction;
32 import org.opendaylight.mdsal.dom.api.DOMDataTreeReadWriteTransaction;
33 import org.opendaylight.mdsal.dom.api.DOMDataTreeTransaction;
34 import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteTransaction;
35 import org.opendaylight.mdsal.dom.api.DOMTransactionChain;
36 import org.opendaylight.mdsal.dom.api.DOMTransactionChainListener;
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  * The actual implementation of {@link PingPongTransactionChain}. Split out to allow deeper testing while keeping the
44  * externally-visible implementation final.
45  */
46 abstract class AbstractPingPongTransactionChain implements DOMTransactionChain {
47     private static final Logger LOG = LoggerFactory.getLogger(AbstractPingPongTransactionChain.class);
48
49     private final DOMTransactionChainListener listener;
50     private final DOMTransactionChain delegate;
51
52     @GuardedBy("this")
53     private boolean closed;
54     @GuardedBy("this")
55     private boolean failed;
56     @GuardedBy("this")
57     private PingPongTransaction shutdownTx;
58     @GuardedBy("this")
59     private Entry<PingPongTransaction, Throwable> deadTx;
60
61     //  This VarHandle is used to manipulate the "ready" transaction. We perform only atomic get-and-set on it.
62     private static final VarHandle READY_TX;
63     @SuppressWarnings("unused")
64     @SuppressFBWarnings(value = "UUF_UNUSED_FIELD", justification = "https://github.com/spotbugs/spotbugs/issues/2749")
65     private volatile PingPongTransaction readyTx;
66
67     /*
68      * This VarHandle is used to manipulate the "locked" transaction. A locked transaction means we know that the user
69      * still holds a transaction and should at some point call us. We perform on compare-and-swap to ensure we properly
70      * detect when a user is attempting to allocated multiple transactions concurrently.
71      */
72     private static final VarHandle LOCKED_TX;
73     @SuppressFBWarnings(value = "UWF_UNWRITTEN_FIELD",
74         justification = "https://github.com/spotbugs/spotbugs/issues/2749")
75     private volatile PingPongTransaction lockedTx;
76
77     /*
78      * This updater is used to manipulate the "inflight" transaction. There can be at most one of these at any given
79      * time. We perform only compare-and-swap on these.
80      */
81     private static final VarHandle INFLIGHT_TX;
82     @SuppressFBWarnings(value = "UWF_UNWRITTEN_FIELD",
83         justification = "https://github.com/spotbugs/spotbugs/issues/2749")
84     private volatile PingPongTransaction inflightTx;
85
86     static {
87         final var lookup = MethodHandles.lookup();
88         try {
89             INFLIGHT_TX = lookup.findVarHandle(AbstractPingPongTransactionChain.class, "inflightTx",
90                 PingPongTransaction.class);
91             LOCKED_TX = lookup.findVarHandle(AbstractPingPongTransactionChain.class, "lockedTx",
92                 PingPongTransaction.class);
93             READY_TX = lookup.findVarHandle(AbstractPingPongTransactionChain.class, "readyTx",
94                 PingPongTransaction.class);
95         } catch (NoSuchFieldException | IllegalAccessException e) {
96             throw new ExceptionInInitializerError(e);
97         }
98     }
99
100     AbstractPingPongTransactionChain(final Function<DOMTransactionChainListener, DOMTransactionChain> delegateFactory,
101             final DOMTransactionChainListener listener) {
102         this.listener = requireNonNull(listener);
103         delegate = delegateFactory.apply(new DOMTransactionChainListener() {
104             @Override
105             public void onTransactionChainFailed(final DOMTransactionChain chain,
106                     final DOMDataTreeTransaction 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 DOMTransactionChain chain) {
113                 delegateSuccessful(chain);
114             }
115         });
116     }
117
118     private void delegateSuccessful(final DOMTransactionChain 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     private void delegateFailed(final DOMTransactionChain chain, final Throwable cause) {
141         final DOMDataTreeReadWriteTransaction frontend;
142         final PingPongTransaction tx = inflightTx;
143         if (tx == null) {
144             LOG.warn("Transaction chain {} failed with no pending transactions", chain);
145             frontend = null;
146         } else {
147             frontend = tx.getFrontendTransaction();
148         }
149
150         listener.onTransactionChainFailed(this, frontend, cause);
151
152         synchronized (this) {
153             failed = true;
154
155             /*
156              * If we do not have a locked transaction, we need to ensure that the backend transaction is cancelled.
157              * Otherwise we can defer until the user calls us.
158              */
159             if (lockedTx == null) {
160                 processIfReady();
161             }
162         }
163     }
164
165     private synchronized @NonNull PingPongTransaction slowAllocateTransaction() {
166         checkState(shutdownTx == null, "Transaction chain %s has been shut down", this);
167
168         if (deadTx != null) {
169             throw new IllegalStateException(String.format(
170                 "Transaction chain %s has failed due to transaction %s being canceled", this, deadTx.getKey()),
171                 deadTx.getValue());
172         }
173
174         final DOMDataTreeReadWriteTransaction delegateTx = delegate.newReadWriteTransaction();
175         final PingPongTransaction newTx = new PingPongTransaction(delegateTx);
176
177         final Object witness = LOCKED_TX.compareAndExchange(this, null, newTx);
178         if (witness != null) {
179             delegateTx.cancel();
180             throw new IllegalStateException(
181                     String.format("New transaction %s raced with transaction %s", newTx, witness));
182         }
183
184         return newTx;
185     }
186
187     private @Nullable PingPongTransaction acquireReadyTx() {
188         return (PingPongTransaction) READY_TX.getAndSet(this, null);
189     }
190
191     private @NonNull PingPongTransaction allocateTransaction() {
192         // Step 1: acquire current state
193         final PingPongTransaction oldTx = acquireReadyTx();
194
195         // Slow path: allocate a delegate transaction
196         if (oldTx == null) {
197             return slowAllocateTransaction();
198         }
199
200         // Fast path: reuse current transaction. We will check failures and similar on commit().
201         final Object witness = LOCKED_TX.compareAndExchange(this, null, oldTx);
202         if (witness != null) {
203             // Ouch. Delegate chain has not detected a duplicate transaction allocation. This is the best we can do.
204             oldTx.getTransaction().cancel();
205             throw new IllegalStateException(String.format("Reusable transaction %s raced with transaction %s", oldTx,
206                 witness));
207         }
208
209         return oldTx;
210     }
211
212     /**
213      * This forces allocateTransaction() on a slow path, which has to happen after this method has completed executing.
214      * Also inflightTx may be updated outside the lock, hence we need to re-check.
215      */
216     @Holding("this")
217     private void processIfReady() {
218         if (inflightTx == null) {
219             final PingPongTransaction tx = acquireReadyTx();
220             if (tx != null) {
221                 processTransaction(tx);
222             }
223         }
224     }
225
226     /**
227      * Process a ready transaction. The caller needs to ensure that each transaction is seen only once by this method.
228      *
229      * @param tx Transaction which needs processing.
230      */
231     @Holding("this")
232     private void processTransaction(final @NonNull PingPongTransaction tx) {
233         if (failed) {
234             LOG.debug("Cancelling transaction {}", tx);
235             tx.getTransaction().cancel();
236             return;
237         }
238
239         LOG.debug("Submitting transaction {}", tx);
240         final Object witness = INFLIGHT_TX.compareAndExchange(this, null, tx);
241         if (witness != null) {
242             LOG.warn("Submitting transaction {} while {} is still running", tx, witness);
243         }
244
245         tx.getTransaction().commit().addCallback(new FutureCallback<CommitInfo>() {
246             @Override
247             public void onSuccess(final CommitInfo result) {
248                 transactionSuccessful(tx, result);
249             }
250
251             @Override
252             public void onFailure(final Throwable throwable) {
253                 transactionFailed(tx, throwable);
254             }
255         }, MoreExecutors.directExecutor());
256     }
257
258     /*
259      * We got invoked from the data store thread. We need to do two things:
260      * 1) release the in-flight transaction
261      * 2) process the potential next transaction
262      *
263      * We have to perform 2) under lock. We could perform 1) without locking, but that means the CAS result may
264      * not be accurate, as a user thread may submit the ready transaction before we acquire the lock -- and checking
265      * for next transaction is not enough, as that may have also be allocated (as a result of a quick
266      * submit/allocate/submit between 1) and 2)). Hence we'd end up doing the following:
267      * 1) CAS of inflightTx
268      * 2) take lock
269      * 3) volatile read of inflightTx
270      *
271      * Rather than doing that, we keep this method synchronized, hence performing only:
272      * 1) take lock
273      * 2) CAS of inflightTx
274      *
275      * Since the user thread is barred from submitting the transaction (in processIfReady), we can then proceed with
276      * the knowledge that inflightTx is null -- processTransaction() will still do a CAS, but that is only for
277      * correctness.
278      */
279     private synchronized void processNextTransaction(final PingPongTransaction tx) {
280         final Object witness = INFLIGHT_TX.compareAndExchange(this, tx, null);
281         checkState(witness == tx, "Completed transaction %s while %s was submitted", tx, witness);
282
283         final PingPongTransaction nextTx = acquireReadyTx();
284         if (nextTx == null) {
285             final PingPongTransaction local = shutdownTx;
286             if (local != null) {
287                 processTransaction(local);
288                 delegate.close();
289                 shutdownTx = null;
290             }
291         } else {
292             processTransaction(nextTx);
293         }
294     }
295
296     private void transactionSuccessful(final PingPongTransaction tx, final CommitInfo result) {
297         LOG.debug("Transaction {} completed successfully", tx);
298
299         tx.onSuccess(result);
300         processNextTransaction(tx);
301     }
302
303     private void transactionFailed(final PingPongTransaction tx, final Throwable throwable) {
304         LOG.debug("Transaction {} failed", tx, throwable);
305
306         tx.onFailure(throwable);
307         processNextTransaction(tx);
308     }
309
310     private void readyTransaction(final @NonNull PingPongTransaction tx) {
311         // First mark the transaction as not locked.
312         final Object lockedWitness = LOCKED_TX.compareAndExchange(this, tx, null);
313         checkState(lockedWitness == tx, "Attempted to submit transaction %s while we have %s", tx, lockedWitness);
314         LOG.debug("Transaction {} unlocked", tx);
315
316         /*
317          * The transaction is ready. It will then be picked up by either next allocation,
318          * or a background transaction completion callback.
319          */
320         final Object readyWitness = READY_TX.compareAndExchange(this, null, tx);
321         checkState(readyWitness == null, "Transaction %s collided on ready state with %s", tx, readyWitness);
322         LOG.debug("Transaction {} readied", tx);
323
324         /*
325          * We do not see a transaction being in-flight, so we need to take care of dispatching
326          * the transaction to the backend. We are in the ready case, we cannot short-cut
327          * the checking of readyTx, as an in-flight transaction may have completed between us
328          * setting the field above and us checking.
329          */
330         if (inflightTx == null) {
331             synchronized (this) {
332                 processIfReady();
333             }
334         }
335     }
336
337     /**
338      * Transaction cancellation is a heavyweight operation. We only support cancelation of a locked transaction
339      * and return false for everything else. Cancelling such a transaction will result in all transactions in the
340      * batch to be cancelled.
341      *
342      * @param tx Backend shared transaction
343      * @param frontendTx transaction
344      * @return {@code true} if the transaction was cancelled successfully
345      */
346     private synchronized boolean cancelTransaction(final PingPongTransaction tx,
347             final DOMDataTreeReadWriteTransaction frontendTx) {
348         // Attempt to unlock the operation.
349         final Object witness = LOCKED_TX.compareAndExchange(this, tx, null);
350         verify(witness == tx, "Cancelling transaction %s collided with locked transaction %s", tx, witness);
351
352         // Cancel the backend transaction, so we do not end up leaking it.
353         final boolean backendCancelled = tx.getTransaction().cancel();
354
355         if (failed) {
356             // The transaction has failed, this is probably the user just clearing up the transaction they had. We have
357             // already cancelled the transaction anyway,
358             return true;
359         }
360
361         // We have dealt with cancelling the backend transaction and have unlocked the transaction. Since we are still
362         // inside the synchronized block, any allocations are blocking on the slow path. Now we have to decide the fate
363         // of this transaction chain.
364         //
365         // If there are no other frontend transactions in this batch we are aligned with backend state and we can
366         // continue processing.
367         if (frontendTx.equals(tx.getFrontendTransaction())) {
368             if (backendCancelled) {
369                 LOG.debug("Cancelled transaction {} was head of the batch, resuming processing", tx);
370                 return true;
371             }
372
373             // Backend refused to cancel the transaction. Reinstate it to locked state.
374             final Object reinstateWitness = LOCKED_TX.compareAndExchange(this, null, tx);
375             verify(reinstateWitness == null, "Reinstating transaction %s collided with locked transaction %s", tx,
376                 reinstateWitness);
377             return false;
378         }
379
380         if (!backendCancelled) {
381             LOG.warn("Backend transaction cannot be cancelled during cancellation of {}, attempting to continue", tx);
382         }
383
384         // There are multiple frontend transactions in this batch. We have to report them as failed, which dooms this
385         // transaction chain, too. Since we just came off of a locked transaction, we do not have a ready transaction
386         // at the moment, but there may be some transaction in-flight. So we proceed to shutdown the backend chain
387         // and mark the fact that we should be turning its completion into a failure.
388         deadTx = Map.entry(tx, new CancellationException("Transaction " + frontendTx + " canceled").fillInStackTrace());
389         delegate.close();
390         return true;
391     }
392
393     @Override
394     public final synchronized void close() {
395         if (closed) {
396             LOG.debug("Attempted to close an already-closed chain");
397             return;
398         }
399
400         // Note: we do not derive from AbstractRegistration due to ordering of this check
401         final var notLocked = lockedTx;
402         if (notLocked != null) {
403             throw new IllegalStateException("Attempted to close chain with outstanding transaction " + notLocked);
404         }
405         closed = true;
406
407         // This may be a reaction to our failure callback, in that case the backend is already shutdown
408         if (deadTx != null) {
409             LOG.debug("Delegate {} is already closed due to failure {}", delegate, deadTx);
410             return;
411         }
412
413         // Force allocations on slow path, picking up a potentially-outstanding transaction
414         final var tx = acquireReadyTx();
415         if (tx != null) {
416             // We have one more transaction, which needs to be processed somewhere. If we do not
417             // a transaction in-flight, we need to push it down ourselves.
418             // If there is an in-flight transaction we will schedule this last one into a dedicated
419             // slot. Allocation slow path will check its presence and fail, the in-flight path will
420             // pick it up, submit and immediately close the chain.
421             if (inflightTx == null) {
422                 processTransaction(tx);
423                 delegate.close();
424             } else {
425                 shutdownTx = tx;
426             }
427         } else {
428             // Nothing outstanding, we can safely shutdown
429             delegate.close();
430         }
431     }
432
433     @Override
434     public final DOMDataTreeReadTransaction newReadOnlyTransaction() {
435         return new PingPongReadTransaction(allocateTransaction());
436     }
437
438     @Override
439     public final DOMDataTreeReadWriteTransaction newReadWriteTransaction() {
440         final PingPongTransaction tx = allocateTransaction();
441         final DOMDataTreeReadWriteTransaction ret = new PingPongReadWriteTransaction(tx);
442         tx.recordFrontendTransaction(ret);
443         return ret;
444     }
445
446     @Override
447     public final DOMDataTreeWriteTransaction newWriteOnlyTransaction() {
448         return newReadWriteTransaction();
449     }
450
451     private final class PingPongReadTransaction implements DOMDataTreeReadTransaction {
452         private final @NonNull PingPongTransaction tx;
453
454         PingPongReadTransaction(final PingPongTransaction tx) {
455             this.tx = requireNonNull(tx);
456         }
457
458         @Override
459         public FluentFuture<Optional<NormalizedNode>> read(final LogicalDatastoreType store,
460                 final YangInstanceIdentifier path) {
461             return tx.getTransaction().read(store, path);
462         }
463
464         @Override
465         public FluentFuture<Boolean> exists(final LogicalDatastoreType store, final YangInstanceIdentifier path) {
466             return tx.getTransaction().exists(store, path);
467         }
468
469         @Override
470         public Object getIdentifier() {
471             return tx.getTransaction().getIdentifier();
472         }
473
474         @Override
475         public void close() {
476             readyTransaction(tx);
477         }
478     }
479
480     private final class PingPongReadWriteTransaction extends ForwardingDOMDataReadWriteTransaction {
481         private final @NonNull PingPongTransaction tx;
482
483         private boolean isOpen = true;
484
485         PingPongReadWriteTransaction(final PingPongTransaction tx) {
486             this.tx = requireNonNull(tx);
487         }
488
489         @Override
490         public FluentFuture<? extends CommitInfo> commit() {
491             readyTransaction(tx);
492             isOpen = false;
493             return tx.getCommitFuture().transform(ignored -> CommitInfo.empty(), MoreExecutors.directExecutor());
494         }
495
496         @Override
497         public boolean cancel() {
498             if (isOpen && cancelTransaction(tx, this)) {
499                 isOpen = false;
500                 return true;
501             }
502             return false;
503         }
504
505         @Override
506         protected DOMDataTreeReadWriteTransaction delegate() {
507             return tx.getTransaction();
508         }
509     }
510 }