Deprecate old MD-SAL APIs for removal
[controller.git] / opendaylight / md-sal / sal-dom-broker / src / main / java / org / opendaylight / controller / md / sal / dom / broker / impl / DOMForwardedWriteTransaction.java
index f791522a2a77214e9a233f966801fac096948618..58c91001c4e8887fec5a297507567fd69dc3a649 100644 (file)
 package org.opendaylight.controller.md.sal.dom.broker.impl;
 
 import static com.google.common.base.Preconditions.checkState;
+import static java.util.Objects.requireNonNull;
 
-import javax.annotation.concurrent.GuardedBy;
-
-import org.opendaylight.controller.md.sal.common.api.TransactionStatus;
+import com.google.common.base.Supplier;
+import com.google.common.util.concurrent.FluentFuture;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
 import org.opendaylight.controller.md.sal.dom.api.DOMDataWriteTransaction;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
-import org.opendaylight.yangtools.yang.common.RpcResult;
-import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier;
+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 com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.util.concurrent.ListenableFuture;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- *
- *
  * Read-Write Transaction, which is composed of several
- * {@link DOMStoreWriteTransaction} transactions. Subtransaction is selected by
+ * {@link DOMStoreWriteTransaction} transactions. A sub-transaction is selected by
  * {@link LogicalDatastoreType} type parameter in:
  *
+ * <p>
  * <ul>
- * <li>{@link #put(LogicalDatastoreType, InstanceIdentifier, NormalizedNode)}
- * <li>{@link #delete(LogicalDatastoreType, InstanceIdentifier)}
- * <li>{@link #merge(LogicalDatastoreType, InstanceIdentifier, NormalizedNode)}
+ * <li>{@link #put(LogicalDatastoreType, YangInstanceIdentifier, NormalizedNode)}
+ * <li>{@link #delete(LogicalDatastoreType, YangInstanceIdentifier)}
+ * <li>{@link #merge(LogicalDatastoreType, YangInstanceIdentifier, NormalizedNode)}
  * </ul>
+ *
  * <p>
  * {@link #commit()} will result in invocation of
- * {@link DOMDataCommitImplementation#commit(org.opendaylight.controller.md.sal.dom.api.DOMDataWriteTransaction, Iterable)}
- * invocation with all {@link org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort} for underlying
- * transactions.
+ * {@link DOMDataCommitImplementation#submit(org.opendaylight.controller.md.sal.dom.api.DOMDataWriteTransaction,
+ * Iterable)} invocation with all {@link org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort}
+ * for underlying transactions.
  *
- * @param <T>
- *            Subtype of {@link DOMStoreWriteTransaction} which is used as
+ * @param <T> Subtype of {@link DOMStoreWriteTransaction} which is used as
  *            subtransaction.
  */
+@Deprecated(forRemoval = true)
 class DOMForwardedWriteTransaction<T extends DOMStoreWriteTransaction> extends
         AbstractDOMForwardedCompositeTransaction<LogicalDatastoreType, T> implements DOMDataWriteTransaction {
+    @SuppressWarnings("rawtypes")
+    private static final AtomicReferenceFieldUpdater<DOMForwardedWriteTransaction,
+            AbstractDOMForwardedTransactionFactory>
+            IMPL_UPDATER = AtomicReferenceFieldUpdater
+            .newUpdater(DOMForwardedWriteTransaction.class, AbstractDOMForwardedTransactionFactory.class, "commitImpl");
+    @SuppressWarnings("rawtypes")
+    private static final AtomicReferenceFieldUpdater<DOMForwardedWriteTransaction, Future> FUTURE_UPDATER
+            = AtomicReferenceFieldUpdater.newUpdater(DOMForwardedWriteTransaction.class, Future.class, "commitFuture");
+    private static final Logger LOG = LoggerFactory.getLogger(DOMForwardedWriteTransaction.class);
+    private static final Future<?> CANCELLED_FUTURE = Futures.immediateCancelledFuture();
 
     /**
-     *  Implementation of real commit.
-     *
-     *  Transaction can not be commited if commitImpl is null,
-     *  so this seting this property to null is also used to
-     *  prevent write to
-     *  already commited / canceled transaction {@link #checkNotCanceled()
-     *
-     *
+     * Implementation of real commit. It also acts as an indication that
+     * the transaction is running -- which we flip atomically using
+     * {@link #IMPL_UPDATER}.
      */
-    @GuardedBy("this")
-    private volatile DOMDataCommitImplementation commitImpl;
+    private volatile AbstractDOMForwardedTransactionFactory<?> commitImpl;
 
     /**
+     * Future task of transaction commit. It starts off as null, but is
+     * set appropriately on {@link #commit()} and {@link #cancel()} via
+     * {@link AtomicReferenceFieldUpdater#lazySet(Object, Object)}.
      *
-     * Future task of transaction commit.
-     *
-     * This value is initially null, and is once updated if transaction
-     * is commited {@link #commit()}.
-     * If this future exists, transaction MUST not be commited again
-     * and all modifications should fail. See {@link #checkNotCommited()}.
-     *
+     * <p>
+     * Lazy set is safe for use because it is only referenced to in the
+     * {@link #cancel()} slow path, where we will busy-wait for it. The
+     * fast path gets the benefit of a store-store barrier instead of the
+     * usual store-load barrier.
      */
-    @GuardedBy("this")
-    private volatile ListenableFuture<RpcResult<TransactionStatus>> commitFuture;
+    private volatile Future<?> commitFuture;
 
-    protected DOMForwardedWriteTransaction(final Object identifier,
-            final ImmutableMap<LogicalDatastoreType, T> backingTxs, final DOMDataCommitImplementation commitImpl) {
+    protected DOMForwardedWriteTransaction(final Object identifier, final Map<LogicalDatastoreType, T> backingTxs,
+                                           final AbstractDOMForwardedTransactionFactory<?> commitImpl) {
         super(identifier, backingTxs);
-        this.commitImpl = Preconditions.checkNotNull(commitImpl, "commitImpl must not be null.");
+        this.commitImpl = requireNonNull(commitImpl, "commitImpl must not be null.");
     }
 
     @Override
-    public void put(final LogicalDatastoreType store, final InstanceIdentifier path, final NormalizedNode<?, ?> data) {
-        checkNotReady();
+    public void put(final LogicalDatastoreType store, final YangInstanceIdentifier path,
+                    final NormalizedNode<?, ?> data) {
+        checkRunning(commitImpl);
         getSubtransaction(store).write(path, data);
     }
 
     @Override
-    public void delete(final LogicalDatastoreType store, final InstanceIdentifier path) {
-        checkNotReady();
+    public void delete(final LogicalDatastoreType store, final YangInstanceIdentifier path) {
+        checkRunning(commitImpl);
         getSubtransaction(store).delete(path);
     }
 
     @Override
-    public void merge(final LogicalDatastoreType store, final InstanceIdentifier path, final NormalizedNode<?, ?> data) {
-        checkNotReady();
+    public void merge(final LogicalDatastoreType store, final YangInstanceIdentifier path,
+                      final NormalizedNode<?, ?> data) {
+        checkRunning(commitImpl);
         getSubtransaction(store).merge(path, data);
     }
 
     @Override
-    public synchronized boolean cancel() {
-        // Transaction is already canceled, we are safe to return true
-        final boolean cancelationResult;
-        if (commitImpl == null && commitFuture != null) {
-            // Transaction is submitted, we try to cancel future.
-            cancelationResult = commitFuture.cancel(false);
-        } else if(commitImpl == null) {
+    public boolean cancel() {
+        final AbstractDOMForwardedTransactionFactory<?> impl = IMPL_UPDATER.getAndSet(this, null);
+        if (impl != null) {
+            LOG.trace("Transaction {} cancelled before submit", getIdentifier());
+            FUTURE_UPDATER.lazySet(this, CANCELLED_FUTURE);
+            closeSubtransactions();
             return true;
-        } else {
-            cancelationResult = true;
-            commitImpl = null;
         }
-        return cancelationResult;
 
+        // The transaction is in process of being submitted or cancelled. Busy-wait
+        // for the corresponding future.
+        Future<?> future;
+        do {
+            future = commitFuture;
+        } while (future == null);
+
+        return future.cancel(false);
     }
 
     @Override
-    public synchronized ListenableFuture<RpcResult<TransactionStatus>> commit() {
-        checkNotReady();
-
-        ImmutableList.Builder<DOMStoreThreePhaseCommitCohort> cohortsBuilder = ImmutableList.builder();
-        for (DOMStoreWriteTransaction subTx : getSubtransactions()) {
-            cohortsBuilder.add(subTx.ready());
-        }
-        ImmutableList<DOMStoreThreePhaseCommitCohort> cohorts = cohortsBuilder.build();
-        commitFuture = commitImpl.commit(this, cohorts);
-
-        /*
-         *We remove reference to Commit Implementation in order
-         *to prevent memory leak
-         */
-        commitImpl = null;
-        return commitFuture;
+    public FluentFuture<? extends CommitInfo> commit() {
+        return FluentFuture.from(doCommit(CommitInfo::empty));
     }
 
-    private void checkNotReady() {
-        checkNotCommited();
-        checkNotCanceled();
-    }
+    @SuppressWarnings("checkstyle:IllegalCatch")
+    private <V> ListenableFuture<V> doCommit(final Supplier<V> futureValueSupplier) {
+        final AbstractDOMForwardedTransactionFactory<?> impl = IMPL_UPDATER.getAndSet(this, null);
+        checkRunning(impl);
 
-    private void checkNotCanceled() {
-        Preconditions.checkState(commitImpl != null, "Transaction was canceled.");
-    }
+        final Collection<T> txns = getSubtransactions();
+        final Collection<DOMStoreThreePhaseCommitCohort> cohorts = new ArrayList<>(txns.size());
+
+        ListenableFuture<V> ret;
+        try {
+            for (DOMStoreWriteTransaction txn : txns) {
+                cohorts.add(txn.ready());
+            }
 
-    private void checkNotCommited() {
-        checkState(commitFuture == null, "Transaction was already submited.");
+            ret = impl.commit(this, cohorts, futureValueSupplier);
+        } catch (RuntimeException e) {
+            ret = FluentFuture.from(Futures.immediateFailedFuture(
+                    TransactionCommitFailedExceptionMapper.COMMIT_ERROR_MAPPER.apply(e)));
+        }
+        FUTURE_UPDATER.lazySet(this, ret);
+        return ret;
     }
 
-}
\ No newline at end of file
+    private void checkRunning(final AbstractDOMForwardedTransactionFactory<?> impl) {
+        checkState(impl != null, "Transaction %s is no longer running", getIdentifier());
+    }
+}