Merge "Removing { } from NormalizedNodeJsonBodyWriter"
[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.util.concurrent.CheckedFuture;
13 import com.google.common.util.concurrent.FutureCallback;
14 import com.google.common.util.concurrent.Futures;
15 import com.google.common.util.concurrent.ListenableFuture;
16 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
17 import javax.annotation.Nonnull;
18 import javax.annotation.concurrent.GuardedBy;
19 import org.opendaylight.controller.md.sal.common.api.TransactionStatus;
20 import org.opendaylight.controller.md.sal.common.api.data.AsyncTransaction;
21 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
22 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
23 import org.opendaylight.controller.md.sal.common.api.data.TransactionChain;
24 import org.opendaylight.controller.md.sal.common.api.data.TransactionChainListener;
25 import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
26 import org.opendaylight.controller.md.sal.dom.api.DOMDataBroker;
27 import org.opendaylight.controller.md.sal.dom.api.DOMDataReadOnlyTransaction;
28 import org.opendaylight.controller.md.sal.dom.api.DOMDataReadWriteTransaction;
29 import org.opendaylight.controller.md.sal.dom.api.DOMDataWriteTransaction;
30 import org.opendaylight.controller.md.sal.dom.api.DOMTransactionChain;
31 import org.opendaylight.controller.md.sal.dom.spi.ForwardingDOMDataReadWriteTransaction;
32 import org.opendaylight.yangtools.yang.common.RpcResult;
33 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
34 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
35 import org.slf4j.Logger;
36 import org.slf4j.LoggerFactory;
37
38 /**
39  * An implementation of {@link DOMTransactionChain}, which has a very specific
40  * behavior, which some users may find surprising. If keeps the general
41  * intent of the contract, but it makes sure there are never more than two
42  * transactions allocated at any given time: one of them is being committed,
43  * and while that is happening, the other one acts as the scratch pad. Once
44  * the committing transaction completes successfully, the scratch transaction
45  * is enqueued as soon as it is ready.
46  *
47  * This mode of operation means that there is no inherent isolation between
48  * the front-end transactions and transactions cannot be reasonably cancelled.
49  *
50  * It furthermore means that the transactions returned by {@link #newReadOnlyTransaction()}
51  * counts as an outstanding transaction and the user may not allocate multiple
52  * 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     private final DOMTransactionChain delegate;
57
58     @GuardedBy("this")
59     private boolean failed;
60
61     /**
62      * This updater is used to manipulate the "ready" transaction. We perform only atomic
63      * get-and-set on it.
64      */
65     private static final AtomicReferenceFieldUpdater<PingPongTransactionChain, PingPongTransaction> READY_UPDATER =
66             AtomicReferenceFieldUpdater.newUpdater(PingPongTransactionChain.class, PingPongTransaction.class, "readyTx");
67     private volatile PingPongTransaction readyTx;
68
69     /**
70      * This updater is used to manipulate the "locked" transaction. A locked transaction
71      * means we know that the user still holds a transaction and should at some point call
72      * us. We perform on compare-and-swap to ensure we properly detect when a user is
73      * attempting to allocated multiple transactions concurrently.
74      */
75     private static final AtomicReferenceFieldUpdater<PingPongTransactionChain, PingPongTransaction> LOCKED_UPDATER =
76             AtomicReferenceFieldUpdater.newUpdater(PingPongTransactionChain.class, PingPongTransaction.class, "lockedTx");
77     private volatile PingPongTransaction lockedTx;
78
79     /**
80      * This updater is used to manipulate the "inflight" transaction. There can be at most
81      * one of these at any given time. We perform only compare-and-swap on these.
82      */
83     private static final AtomicReferenceFieldUpdater<PingPongTransactionChain, PingPongTransaction> INFLIGHT_UPDATER =
84             AtomicReferenceFieldUpdater.newUpdater(PingPongTransactionChain.class, PingPongTransaction.class, "inflightTx");
85     private volatile PingPongTransaction inflightTx;
86
87     PingPongTransactionChain(final DOMDataBroker broker, final TransactionChainListener listener) {
88         this.delegate = broker.createTransactionChain(new TransactionChainListener() {
89             @Override
90             public void onTransactionChainFailed(final TransactionChain<?, ?> chain, final AsyncTransaction<?, ?> transaction, final Throwable cause) {
91                 LOG.debug("Delegate chain {} reported failure in {}", chain, transaction, cause);
92
93                 final DOMDataReadWriteTransaction frontend;
94                 final PingPongTransaction tx = inflightTx;
95                 if (tx == null) {
96                     LOG.warn("Transaction chain {} failed with no pending transactions", chain);
97                     frontend = null;
98                 } else {
99                     frontend = tx.getFrontendTransaction();
100                 }
101
102                 listener.onTransactionChainFailed(PingPongTransactionChain.this, frontend, cause);
103                 delegateFailed();
104             }
105
106             @Override
107             public void onTransactionChainSuccessful(final TransactionChain<?, ?> chain) {
108                 listener.onTransactionChainSuccessful(PingPongTransactionChain.this);
109             }
110         });
111     }
112
113     private synchronized void delegateFailed() {
114         failed = true;
115
116         /*
117          * If we do not have a locked transaction, we need to ensure that
118          * the backend transaction is cancelled. Otherwise we can defer
119          * until the user calls us.
120          */
121         if (lockedTx == null) {
122             processIfReady();
123         }
124     }
125
126     private synchronized PingPongTransaction slowAllocateTransaction() {
127         final DOMDataReadWriteTransaction delegateTx = delegate.newReadWriteTransaction();
128         final PingPongTransaction newTx = new PingPongTransaction(delegateTx);
129
130         if (!LOCKED_UPDATER.compareAndSet(this, null, newTx)) {
131             delegateTx.cancel();
132             throw new IllegalStateException(String.format("New transaction %s raced with transacion %s", newTx, lockedTx));
133         }
134
135         return newTx;
136     }
137
138     private PingPongTransaction allocateTransaction() {
139         // Step 1: acquire current state
140         final PingPongTransaction oldTx = READY_UPDATER.getAndSet(this, null);
141
142         // Slow path: allocate a delegate transaction
143         if (oldTx == null) {
144             return slowAllocateTransaction();
145         }
146
147         // Fast path: reuse current transaction. We will check
148         //            failures and similar on submit().
149         if (!LOCKED_UPDATER.compareAndSet(this, null, oldTx)) {
150             // Ouch. Delegate chain has not detected a duplicate
151             // transaction allocation. This is the best we can do.
152             oldTx.getTransaction().cancel();
153             throw new IllegalStateException(String.format("Reusable transaction %s raced with transaction %s", oldTx, lockedTx));
154         }
155
156         return oldTx;
157     }
158
159     /*
160      * This forces allocateTransaction() on a slow path, which has to happen after
161      * this method has completed executing. Also inflightTx may be updated outside
162      * the lock, hence we need to re-check.
163      */
164     @GuardedBy("this")
165     private void processIfReady() {
166         if (inflightTx == null) {
167             final PingPongTransaction tx = READY_UPDATER.getAndSet(this, null);
168             if (tx != null) {
169                 processTransaction(tx);
170             }
171         }
172     }
173
174     /**
175      * Process a ready transaction. The caller needs to ensure that
176      * each transaction is seen only once by this method.
177      *
178      * @param tx Transaction which needs processing.
179      */
180     @GuardedBy("this")
181     private void processTransaction(final @Nonnull PingPongTransaction tx) {
182         if (failed) {
183             LOG.debug("Cancelling transaction {}", tx);
184             tx.getTransaction().cancel();
185             return;
186         }
187
188         LOG.debug("Submitting transaction {}", tx);
189         if (!INFLIGHT_UPDATER.compareAndSet(this, null, tx)) {
190             LOG.warn("Submitting transaction {} while {} is still running", tx, inflightTx);
191         }
192
193         Futures.addCallback(tx.getTransaction().submit(), new FutureCallback<Void>() {
194             @Override
195             public void onSuccess(final Void result) {
196                 transactionSuccessful(tx, result);
197             }
198
199             @Override
200             public void onFailure(final Throwable t) {
201                 transactionFailed(tx, t);
202             }
203         });
204     }
205
206     private void transactionSuccessful(final PingPongTransaction tx, final Void result) {
207         LOG.debug("Transaction {} completed successfully", tx);
208
209         final boolean success = INFLIGHT_UPDATER.compareAndSet(this, tx, null);
210         Preconditions.checkState(success, "Successful transaction %s while %s was submitted", tx, inflightTx);
211
212         synchronized (this) {
213             processIfReady();
214         }
215
216         // Can run unsynchronized
217         tx.onSuccess(result);
218     }
219
220     private void transactionFailed(final PingPongTransaction tx, final Throwable t) {
221         LOG.debug("Transaction {} failed", tx, t);
222
223         final boolean success = INFLIGHT_UPDATER.compareAndSet(this, tx, null);
224         Preconditions.checkState(success, "Failed transaction %s while %s was submitted", tx, inflightTx);
225
226         tx.onFailure(t);
227     }
228
229     private void readyTransaction(final @Nonnull PingPongTransaction tx) {
230         // First mark the transaction as not locked.
231         final boolean lockedMatch = LOCKED_UPDATER.compareAndSet(this, tx, null);
232         Preconditions.checkState(lockedMatch, "Attempted to submit transaction %s while we have %s", tx, lockedTx);
233         LOG.debug("Transaction {} unlocked", tx);
234
235         /*
236          * The transaction is ready. It will then be picked up by either next allocation,
237          * or a background transaction completion callback.
238          */
239         final boolean success = READY_UPDATER.compareAndSet(this, null, tx);
240         Preconditions.checkState(success, "Transaction %s collided on ready state", tx, readyTx);
241         LOG.debug("Transaction {} readied", tx);
242
243         /*
244          * We do not see a transaction being in-flight, so we need to take care of dispatching
245          * the transaction to the backend. We are in the ready case, we cannot short-cut
246          * the checking of readyTx, as an in-flight transaction may have completed between us
247          * setting the field above and us checking.
248          */
249         if (inflightTx == null) {
250             synchronized (this) {
251                 processIfReady();
252             }
253         }
254     }
255
256     @Override
257     public synchronized void close() {
258         final PingPongTransaction notLocked = lockedTx;
259         Preconditions.checkState(notLocked == null, "Attempted to close chain with outstanding transaction %s", notLocked);
260
261         // Force allocations on slow path. We will complete the rest
262         final PingPongTransaction tx = READY_UPDATER.getAndSet(this, null);
263
264         // Make sure no transaction is outstanding. Otherwise sleep a bit and retry
265         while (inflightTx != null) {
266             LOG.debug("Busy-waiting for in-flight transaction {} to complete", inflightTx);
267             Thread.yield();
268             continue;
269         }
270
271         // If we have an outstanding transaction, send it down
272         if (tx != null) {
273             processTransaction(tx);
274         }
275
276         // All done, close the delegate. All new allocations should fail.
277         delegate.close();
278     }
279
280     @Override
281     public DOMDataReadOnlyTransaction newReadOnlyTransaction() {
282         final PingPongTransaction tx = allocateTransaction();
283
284         return new DOMDataReadOnlyTransaction() {
285             @Override
286             public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> read(final LogicalDatastoreType store,
287                     final YangInstanceIdentifier path) {
288                 return tx.getTransaction().read(store, path);
289             }
290
291             @Override
292             public CheckedFuture<Boolean, ReadFailedException> exists(final LogicalDatastoreType store,
293                     final YangInstanceIdentifier path) {
294                 return tx.getTransaction().exists(store, path);
295             }
296
297             @Override
298             public Object getIdentifier() {
299                 return tx.getTransaction().getIdentifier();
300             }
301
302             @Override
303             public void close() {
304                 readyTransaction(tx);
305             }
306         };
307     }
308
309     @Override
310     public DOMDataReadWriteTransaction newReadWriteTransaction() {
311         final PingPongTransaction tx = allocateTransaction();
312         final DOMDataReadWriteTransaction ret = new ForwardingDOMDataReadWriteTransaction() {
313             @Override
314             protected DOMDataReadWriteTransaction delegate() {
315                 return tx.getTransaction();
316             }
317
318             @Override
319             public CheckedFuture<Void, TransactionCommitFailedException> submit() {
320                 readyTransaction(tx);
321                 return tx.getSubmitFuture();
322             }
323
324             @Override
325             public ListenableFuture<RpcResult<TransactionStatus>> commit() {
326                 readyTransaction(tx);
327                 return tx.getCommitFuture();
328             }
329
330             @Override
331             public boolean cancel() {
332                 throw new UnsupportedOperationException("Transaction cancellation is not supported");
333             }
334         };
335
336         tx.recordFrontendTransaction(ret);
337         return ret;
338     }
339
340     @Override
341     public DOMDataWriteTransaction newWriteOnlyTransaction() {
342         return newReadWriteTransaction();
343     }
344 }