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