Migrate OSGI compendium reference
[controller.git] / opendaylight / md-sal / sal-dom-broker / src / main / java / org / opendaylight / controller / md / sal / dom / broker / impl / PingPongTransactionChain.java
index c7e8ced21a4139b6d870dc4de1bb606834f4ad5f..e72315520f4cdf51f94105ed7f71a4f59aa4de5f 100644 (file)
@@ -7,33 +7,34 @@
  */
 package org.opendaylight.controller.md.sal.dom.broker.impl;
 
+import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.base.Verify.verify;
+import static java.util.Objects.requireNonNull;
+
 import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Verify;
 import com.google.common.util.concurrent.CheckedFuture;
+import com.google.common.util.concurrent.FluentFuture;
 import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.MoreExecutors;
 import java.util.AbstractMap.SimpleImmutableEntry;
 import java.util.Map.Entry;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
-import javax.annotation.Nonnull;
-import javax.annotation.concurrent.GuardedBy;
-import org.opendaylight.controller.md.sal.common.api.TransactionStatus;
+import org.checkerframework.checker.lock.qual.GuardedBy;
+import org.checkerframework.checker.lock.qual.Holding;
+import org.eclipse.jdt.annotation.NonNull;
 import org.opendaylight.controller.md.sal.common.api.data.AsyncTransaction;
 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
 import org.opendaylight.controller.md.sal.common.api.data.TransactionChain;
 import org.opendaylight.controller.md.sal.common.api.data.TransactionChainListener;
-import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
 import org.opendaylight.controller.md.sal.dom.api.DOMDataBroker;
 import org.opendaylight.controller.md.sal.dom.api.DOMDataReadOnlyTransaction;
 import org.opendaylight.controller.md.sal.dom.api.DOMDataReadWriteTransaction;
 import org.opendaylight.controller.md.sal.dom.api.DOMDataWriteTransaction;
 import org.opendaylight.controller.md.sal.dom.api.DOMTransactionChain;
 import org.opendaylight.controller.md.sal.dom.spi.ForwardingDOMDataReadWriteTransaction;
-import org.opendaylight.yangtools.yang.common.RpcResult;
+import org.opendaylight.mdsal.common.api.CommitInfo;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 import org.slf4j.Logger;
@@ -48,13 +49,16 @@ import org.slf4j.LoggerFactory;
  * the committing transaction completes successfully, the scratch transaction
  * is enqueued as soon as it is ready.
  *
+ * <p>
  * This mode of operation means that there is no inherent isolation between
  * the front-end transactions and transactions cannot be reasonably cancelled.
  *
+ * <p>
  * It furthermore means that the transactions returned by {@link #newReadOnlyTransaction()}
  * counts as an outstanding transaction and the user may not allocate multiple
  * read-only transactions at the same time.
  */
+@Deprecated(forRemoval = true)
 public final class PingPongTransactionChain implements DOMTransactionChain {
     private static final Logger LOG = LoggerFactory.getLogger(PingPongTransactionChain.class);
     private final TransactionChainListener listener;
@@ -71,8 +75,9 @@ public final class PingPongTransactionChain implements DOMTransactionChain {
      * This updater is used to manipulate the "ready" transaction. We perform only atomic
      * get-and-set on it.
      */
-    private static final AtomicReferenceFieldUpdater<PingPongTransactionChain, PingPongTransaction> READY_UPDATER =
-            AtomicReferenceFieldUpdater.newUpdater(PingPongTransactionChain.class, PingPongTransaction.class, "readyTx");
+    private static final AtomicReferenceFieldUpdater<PingPongTransactionChain, PingPongTransaction> READY_UPDATER
+            = AtomicReferenceFieldUpdater
+            .newUpdater(PingPongTransactionChain.class, PingPongTransaction.class, "readyTx");
     private volatile PingPongTransaction readyTx;
 
     /**
@@ -81,24 +86,26 @@ public final class PingPongTransactionChain implements DOMTransactionChain {
      * us. We perform on compare-and-swap to ensure we properly detect when a user is
      * attempting to allocated multiple transactions concurrently.
      */
-    private static final AtomicReferenceFieldUpdater<PingPongTransactionChain, PingPongTransaction> LOCKED_UPDATER =
-            AtomicReferenceFieldUpdater.newUpdater(PingPongTransactionChain.class, PingPongTransaction.class, "lockedTx");
+    private static final AtomicReferenceFieldUpdater<PingPongTransactionChain, PingPongTransaction> LOCKED_UPDATER
+            = AtomicReferenceFieldUpdater
+            .newUpdater(PingPongTransactionChain.class, PingPongTransaction.class, "lockedTx");
     private volatile PingPongTransaction lockedTx;
 
     /**
      * This updater is used to manipulate the "inflight" transaction. There can be at most
      * one of these at any given time. We perform only compare-and-swap on these.
      */
-    private static final AtomicReferenceFieldUpdater<PingPongTransactionChain, PingPongTransaction> INFLIGHT_UPDATER =
-            AtomicReferenceFieldUpdater.newUpdater(PingPongTransactionChain.class, PingPongTransaction.class, "inflightTx");
+    private static final AtomicReferenceFieldUpdater<PingPongTransactionChain, PingPongTransaction> INFLIGHT_UPDATER
+            = AtomicReferenceFieldUpdater
+            .newUpdater(PingPongTransactionChain.class, PingPongTransaction.class, "inflightTx");
     private volatile PingPongTransaction inflightTx;
 
     PingPongTransactionChain(final DOMDataBroker broker, final TransactionChainListener listener) {
-        this.listener = Preconditions.checkNotNull(listener);
+        this.listener = requireNonNull(listener);
         this.delegate = broker.createTransactionChain(new TransactionChainListener() {
             @Override
             public void onTransactionChainFailed(final TransactionChain<?, ?> chain,
-                    final AsyncTransaction<?, ?> transaction, final Throwable cause) {
+                                                 final AsyncTransaction<?, ?> transaction, final Throwable cause) {
                 LOG.debug("Transaction chain {} reported failure in {}", chain, transaction, cause);
                 delegateFailed(chain, cause);
             }
@@ -160,12 +167,12 @@ public final class PingPongTransactionChain implements DOMTransactionChain {
     }
 
     private synchronized PingPongTransaction slowAllocateTransaction() {
-        Preconditions.checkState(shutdownTx == null, "Transaction chain %s has been shut down", this);
+        checkState(shutdownTx == null, "Transaction chain %s has been shut down", this);
 
         if (deadTx != null) {
-            throw new IllegalStateException(String.format(
-                "Transaction chain %s has failed due to transaction %s being canceled", this, deadTx.getKey()),
-                deadTx.getValue());
+            throw new IllegalStateException(
+                    String.format("Transaction chain %s has failed due to transaction %s being canceled", this,
+                                  deadTx.getKey()), deadTx.getValue());
         }
 
         final DOMDataReadWriteTransaction delegateTx = delegate.newReadWriteTransaction();
@@ -173,7 +180,8 @@ public final class PingPongTransactionChain implements DOMTransactionChain {
 
         if (!LOCKED_UPDATER.compareAndSet(this, null, newTx)) {
             delegateTx.cancel();
-            throw new IllegalStateException(String.format("New transaction %s raced with transaction %s", newTx, lockedTx));
+            throw new IllegalStateException(
+                    String.format("New transaction %s raced with transaction %s", newTx, lockedTx));
         }
 
         return newTx;
@@ -188,11 +196,12 @@ public final class PingPongTransactionChain implements DOMTransactionChain {
             return slowAllocateTransaction();
         }
 
-        // Fast path: reuse current transaction. We will check failures and similar on submit().
+        // Fast path: reuse current transaction. We will check failures and similar on commit().
         if (!LOCKED_UPDATER.compareAndSet(this, null, oldTx)) {
             // Ouch. Delegate chain has not detected a duplicate transaction allocation. This is the best we can do.
             oldTx.getTransaction().cancel();
-            throw new IllegalStateException(String.format("Reusable transaction %s raced with transaction %s", oldTx, lockedTx));
+            throw new IllegalStateException(
+                    String.format("Reusable transaction %s raced with transaction %s", oldTx, lockedTx));
         }
 
         return oldTx;
@@ -203,7 +212,7 @@ public final class PingPongTransactionChain implements DOMTransactionChain {
      * this method has completed executing. Also inflightTx may be updated outside
      * the lock, hence we need to re-check.
      */
-    @GuardedBy("this")
+    @Holding("this")
     private void processIfReady() {
         if (inflightTx == null) {
             final PingPongTransaction tx = READY_UPDATER.getAndSet(this, null);
@@ -219,8 +228,8 @@ public final class PingPongTransactionChain implements DOMTransactionChain {
      *
      * @param tx Transaction which needs processing.
      */
-    @GuardedBy("this")
-    private void processTransaction(@Nonnull final PingPongTransaction tx) {
+    @Holding("this")
+    private void processTransaction(final @NonNull PingPongTransaction tx) {
         if (failed) {
             LOG.debug("Cancelling transaction {}", tx);
             tx.getTransaction().cancel();
@@ -232,17 +241,17 @@ public final class PingPongTransactionChain implements DOMTransactionChain {
             LOG.warn("Submitting transaction {} while {} is still running", tx, inflightTx);
         }
 
-        Futures.addCallback(tx.getTransaction().submit(), new FutureCallback<Void>() {
+        tx.getTransaction().commit().addCallback(new FutureCallback<CommitInfo>() {
             @Override
-            public void onSuccess(final Void result) {
+            public void onSuccess(final CommitInfo result) {
                 transactionSuccessful(tx, result);
             }
 
             @Override
-            public void onFailure(final Throwable t) {
-                transactionFailed(tx, t);
+            public void onFailure(final Throwable throwable) {
+                transactionFailed(tx, throwable);
             }
-        });
+        }, MoreExecutors.directExecutor());
     }
 
     /*
@@ -268,7 +277,7 @@ public final class PingPongTransactionChain implements DOMTransactionChain {
      */
     private synchronized void processNextTransaction(final PingPongTransaction tx) {
         final boolean success = INFLIGHT_UPDATER.compareAndSet(this, tx, null);
-        Preconditions.checkState(success, "Completed transaction %s while %s was submitted", tx, inflightTx);
+        checkState(success, "Completed transaction %s while %s was submitted", tx, inflightTx);
 
         final PingPongTransaction nextTx = READY_UPDATER.getAndSet(this, null);
         if (nextTx != null) {
@@ -280,24 +289,24 @@ public final class PingPongTransactionChain implements DOMTransactionChain {
         }
     }
 
-    void transactionSuccessful(final PingPongTransaction tx, final Void result) {
+    void transactionSuccessful(final PingPongTransaction tx, final CommitInfo result) {
         LOG.debug("Transaction {} completed successfully", tx);
 
         tx.onSuccess(result);
         processNextTransaction(tx);
     }
 
-    void transactionFailed(final PingPongTransaction tx, final Throwable t) {
-        LOG.debug("Transaction {} failed", tx, t);
+    void transactionFailed(final PingPongTransaction tx, final Throwable throwable) {
+        LOG.debug("Transaction {} failed", tx, throwable);
 
-        tx.onFailure(t);
+        tx.onFailure(throwable);
         processNextTransaction(tx);
     }
 
-    void readyTransaction(@Nonnull final PingPongTransaction tx) {
+    void readyTransaction(final @NonNull PingPongTransaction tx) {
         // First mark the transaction as not locked.
         final boolean lockedMatch = LOCKED_UPDATER.compareAndSet(this, tx, null);
-        Preconditions.checkState(lockedMatch, "Attempted to submit transaction %s while we have %s", tx, lockedTx);
+        checkState(lockedMatch, "Attempted to submit transaction %s while we have %s", tx, lockedTx);
         LOG.debug("Transaction {} unlocked", tx);
 
         /*
@@ -305,7 +314,7 @@ public final class PingPongTransactionChain implements DOMTransactionChain {
          * or a background transaction completion callback.
          */
         final boolean success = READY_UPDATER.compareAndSet(this, null, tx);
-        Preconditions.checkState(success, "Transaction %s collided on ready state", tx, readyTx);
+        checkState(success, "Transaction %s collided on ready state", tx, readyTx);
         LOG.debug("Transaction {} readied", tx);
 
         /*
@@ -326,15 +335,14 @@ public final class PingPongTransactionChain implements DOMTransactionChain {
      * and return false for everything else. Cancelling such a transaction will result in all transactions in the
      * batch to be cancelled.
      *
-     * @param tx Backend shared transaction
-     * @param frontendTx
-     * @param isOpen indicator whether the transaction was already closed
-     * @return True if cancellation succeeded, false otherwise
+     * @param tx         Backend shared transaction
+     * @param frontendTx transaction
+     * @param isOpen     indicator whether the transaction was already closed
      */
     synchronized void cancelTransaction(final PingPongTransaction tx, final DOMDataReadWriteTransaction frontendTx) {
         // Attempt to unlock the operation.
         final boolean lockedMatch = LOCKED_UPDATER.compareAndSet(this, tx, null);
-        Verify.verify(lockedMatch, "Cancelling transaction {} collided with locked transaction {}", tx, lockedTx);
+        verify(lockedMatch, "Cancelling transaction %s collided with locked transaction %s", tx, lockedTx);
 
         // Cancel the backend transaction, so we do not end up leaking it.
         final boolean backendCancelled = tx.getTransaction().cancel();
@@ -362,19 +370,19 @@ public final class PingPongTransactionChain implements DOMTransactionChain {
         // transaction chain, too. Since we just came off of a locked transaction, we do not have a ready transaction
         // at the moment, but there may be some transaction in-flight. So we proceed to shutdown the backend chain
         // and mark the fact that we should be turning its completion into a failure.
-        deadTx = new SimpleImmutableEntry<>(tx,
-                new CancellationException("Transaction " + frontendTx + " canceled").fillInStackTrace());
+        deadTx = new SimpleImmutableEntry<>(tx, new CancellationException("Transaction " + frontendTx + " canceled")
+                .fillInStackTrace());
         delegate.close();
     }
 
     @Override
     public synchronized void close() {
         final PingPongTransaction notLocked = lockedTx;
-        Preconditions.checkState(notLocked == null, "Attempted to close chain with outstanding transaction %s", notLocked);
+        checkState(notLocked == null, "Attempted to close chain with outstanding transaction %s", notLocked);
 
         // This is not reliable, but if we observe it to be null and the process has already completed,
         // the backend transaction chain will throw the appropriate error.
-        Preconditions.checkState(shutdownTx == null, "Attempted to close an already-closed chain");
+        checkState(shutdownTx == null, "Attempted to close an already-closed chain");
 
         // This may be a reaction to our failure callback, in that case the backend is already shutdown
         if (deadTx != null) {
@@ -409,14 +417,14 @@ public final class PingPongTransactionChain implements DOMTransactionChain {
 
         return new DOMDataReadOnlyTransaction() {
             @Override
-            public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> read(final LogicalDatastoreType store,
-                    final YangInstanceIdentifier path) {
+            public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> read(
+                    final LogicalDatastoreType store, final YangInstanceIdentifier path) {
                 return tx.getTransaction().read(store, path);
             }
 
             @Override
             public CheckedFuture<Boolean, ReadFailedException> exists(final LogicalDatastoreType store,
-                    final YangInstanceIdentifier path) {
+                                                                      final YangInstanceIdentifier path) {
                 return tx.getTransaction().exists(store, path);
             }
 
@@ -444,18 +452,11 @@ public final class PingPongTransactionChain implements DOMTransactionChain {
             }
 
             @Override
-            public CheckedFuture<Void, TransactionCommitFailedException> submit() {
-                readyTransaction(tx);
-                isOpen = false;
-                return tx.getSubmitFuture();
-            }
-
-            @Deprecated
-            @Override
-            public ListenableFuture<RpcResult<TransactionStatus>> commit() {
+            public FluentFuture<? extends CommitInfo> commit() {
                 readyTransaction(tx);
                 isOpen = false;
-                return tx.getCommitFuture();
+                return FluentFuture.from(tx.getCommitFuture()).transformAsync(
+                    ignored -> CommitInfo.emptyFluentFuture(), MoreExecutors.directExecutor());
             }
 
             @Override