Split out TransactionContext classes 73/14573/1
authorRobert Varga <rovarga@cisco.com>
Tue, 27 Jan 2015 15:13:36 +0000 (16:13 +0100)
committerRobert Varga <rovarga@cisco.com>
Wed, 28 Jan 2015 16:19:00 +0000 (17:19 +0100)
The classes are static and quite large, splitting them out into separate
files allows for better readability.

Change-Id: I12f6da96b7bf128549bf4d990bcc797fdb348f0d
Signed-off-by: Robert Varga <rovarga@cisco.com>
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/AbstractTransactionContext.java [new file with mode: 0644]
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/NoOpTransactionContext.java [new file with mode: 0644]
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/OperationCompleter.java [new file with mode: 0644]
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionContext.java [new file with mode: 0644]
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionContextImpl.java [new file with mode: 0644]
opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionProxy.java

diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/AbstractTransactionContext.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/AbstractTransactionContext.java
new file mode 100644 (file)
index 0000000..933e87a
--- /dev/null
@@ -0,0 +1,28 @@
+/*
+ * Copyright (c) 2015 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+package org.opendaylight.controller.cluster.datastore;
+
+import com.google.common.collect.Lists;
+import java.util.List;
+import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import scala.concurrent.Future;
+
+abstract class AbstractTransactionContext implements TransactionContext {
+
+    protected final TransactionIdentifier identifier;
+    protected final List<Future<Object>> recordedOperationFutures = Lists.newArrayList();
+
+    AbstractTransactionContext(TransactionIdentifier identifier) {
+        this.identifier = identifier;
+    }
+
+    @Override
+    public List<Future<Object>> getRecordedOperationFutures() {
+        return recordedOperationFutures;
+    }
+}
\ No newline at end of file
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/NoOpTransactionContext.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/NoOpTransactionContext.java
new file mode 100644 (file)
index 0000000..fc10b9c
--- /dev/null
@@ -0,0 +1,82 @@
+/*
+ * Copyright (c) 2015 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+package org.opendaylight.controller.cluster.datastore;
+
+import akka.actor.ActorSelection;
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.CheckedFuture;
+import com.google.common.util.concurrent.Futures;
+import java.util.concurrent.Semaphore;
+import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.concurrent.Future;
+
+final class NoOpTransactionContext extends AbstractTransactionContext {
+    private static final Logger LOG = LoggerFactory.getLogger(NoOpTransactionContext.class);
+
+    private final Throwable failure;
+    private final Semaphore operationLimiter;
+
+    public NoOpTransactionContext(Throwable failure, TransactionIdentifier identifier, Semaphore operationLimiter) {
+        super(identifier);
+        this.failure = failure;
+        this.operationLimiter = operationLimiter;
+    }
+
+    @Override
+    public void closeTransaction() {
+        LOG.debug("NoOpTransactionContext {} closeTransaction called", identifier);
+    }
+
+    @Override
+    public Future<ActorSelection> readyTransaction() {
+        LOG.debug("Tx {} readyTransaction called", identifier);
+        operationLimiter.release();
+        return akka.dispatch.Futures.failed(failure);
+    }
+
+    @Override
+    public void deleteData(YangInstanceIdentifier path) {
+        LOG.debug("Tx {} deleteData called path = {}", identifier, path);
+        operationLimiter.release();
+    }
+
+    @Override
+    public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
+        LOG.debug("Tx {} mergeData called path = {}", identifier, path);
+        operationLimiter.release();
+    }
+
+    @Override
+    public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
+        LOG.debug("Tx {} writeData called path = {}", identifier, path);
+        operationLimiter.release();
+    }
+
+    @Override
+    public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
+            YangInstanceIdentifier path) {
+        LOG.debug("Tx {} readData called path = {}", identifier, path);
+        operationLimiter.release();
+        return Futures.immediateFailedCheckedFuture(new ReadFailedException(
+                "Error reading data for path " + path, failure));
+    }
+
+    @Override
+    public CheckedFuture<Boolean, ReadFailedException> dataExists(
+            YangInstanceIdentifier path) {
+        LOG.debug("Tx {} dataExists called path = {}", identifier, path);
+        operationLimiter.release();
+        return Futures.immediateFailedCheckedFuture(new ReadFailedException(
+                "Error checking exists for path " + path, failure));
+    }
+}
\ No newline at end of file
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/OperationCompleter.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/OperationCompleter.java
new file mode 100644 (file)
index 0000000..09fa61b
--- /dev/null
@@ -0,0 +1,25 @@
+/*
+ * Copyright (c) 2015 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+package org.opendaylight.controller.cluster.datastore;
+
+import akka.dispatch.OnComplete;
+import com.google.common.base.Preconditions;
+import java.util.concurrent.Semaphore;
+
+final class OperationCompleter extends OnComplete<Object> {
+    private final Semaphore operationLimiter;
+
+    OperationCompleter(Semaphore operationLimiter){
+        this.operationLimiter = Preconditions.checkNotNull(operationLimiter);
+    }
+
+    @Override
+    public void onComplete(Throwable throwable, Object o){
+        this.operationLimiter.release();
+    }
+}
\ No newline at end of file
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionContext.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionContext.java
new file mode 100644 (file)
index 0000000..b6af31e
--- /dev/null
@@ -0,0 +1,40 @@
+/*
+ * Copyright (c) 2015 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+package org.opendaylight.controller.cluster.datastore;
+
+import akka.actor.ActorSelection;
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.CheckedFuture;
+import java.util.List;
+import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
+import scala.concurrent.Future;
+
+/*
+ * FIXME: why do we need this interface? It should be possible to integrate it with
+ *        AbstractTransactionContext, which is the only implementation anyway.
+ */
+interface TransactionContext {
+    void closeTransaction();
+
+    Future<ActorSelection> readyTransaction();
+
+    void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
+
+    void deleteData(YangInstanceIdentifier path);
+
+    void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
+
+    CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
+            final YangInstanceIdentifier path);
+
+    CheckedFuture<Boolean, ReadFailedException> dataExists(YangInstanceIdentifier path);
+
+    List<Future<Object>> getRecordedOperationFutures();
+}
\ No newline at end of file
diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionContextImpl.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/TransactionContextImpl.java
new file mode 100644 (file)
index 0000000..ce2c99e
--- /dev/null
@@ -0,0 +1,347 @@
+/*
+ * Copyright (c) 2015 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+package org.opendaylight.controller.cluster.datastore;
+
+import akka.actor.ActorSelection;
+import akka.dispatch.Mapper;
+import akka.dispatch.OnComplete;
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.CheckedFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import java.util.List;
+import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIdentifier;
+import org.opendaylight.controller.cluster.datastore.messages.CloseTransaction;
+import org.opendaylight.controller.cluster.datastore.messages.DataExists;
+import org.opendaylight.controller.cluster.datastore.messages.DataExistsReply;
+import org.opendaylight.controller.cluster.datastore.messages.DeleteData;
+import org.opendaylight.controller.cluster.datastore.messages.MergeData;
+import org.opendaylight.controller.cluster.datastore.messages.ReadData;
+import org.opendaylight.controller.cluster.datastore.messages.ReadDataReply;
+import org.opendaylight.controller.cluster.datastore.messages.ReadyTransaction;
+import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
+import org.opendaylight.controller.cluster.datastore.messages.SerializableMessage;
+import org.opendaylight.controller.cluster.datastore.messages.VersionedSerializableMessage;
+import org.opendaylight.controller.cluster.datastore.messages.WriteData;
+import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
+import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
+import org.opendaylight.yangtools.util.concurrent.MappingCheckedFuture;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
+import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.concurrent.Future;
+
+final class TransactionContextImpl extends AbstractTransactionContext {
+    private static final Logger LOG = LoggerFactory.getLogger(TransactionContextImpl.class);
+
+    private final ActorContext actorContext;
+    private final String transactionPath;
+    private final ActorSelection actor;
+    private final boolean isTxActorLocal;
+    private final short remoteTransactionVersion;
+    private final OperationCompleter operationCompleter;
+
+
+    TransactionContextImpl(String transactionPath, ActorSelection actor, TransactionIdentifier identifier,
+            ActorContext actorContext, SchemaContext schemaContext,
+            boolean isTxActorLocal, short remoteTransactionVersion, OperationCompleter operationCompleter) {
+        super(identifier);
+        this.transactionPath = transactionPath;
+        this.actor = actor;
+        this.actorContext = actorContext;
+        this.isTxActorLocal = isTxActorLocal;
+        this.remoteTransactionVersion = remoteTransactionVersion;
+        this.operationCompleter = operationCompleter;
+    }
+
+    private Future<Object> completeOperation(Future<Object> operationFuture){
+        operationFuture.onComplete(this.operationCompleter, actorContext.getActorSystem().dispatcher());
+        return operationFuture;
+    }
+
+
+    private ActorSelection getActor() {
+        return actor;
+    }
+
+    private Future<Object> executeOperationAsync(SerializableMessage msg) {
+        return completeOperation(actorContext.executeOperationAsync(getActor(), isTxActorLocal ? msg : msg.toSerializable()));
+    }
+
+    private Future<Object> executeOperationAsync(VersionedSerializableMessage msg) {
+        return completeOperation(actorContext.executeOperationAsync(getActor(), isTxActorLocal ? msg :
+                msg.toSerializable(remoteTransactionVersion)));
+    }
+
+    @Override
+    public void closeTransaction() {
+        LOG.debug("Tx {} closeTransaction called", identifier);
+
+        actorContext.sendOperationAsync(getActor(), CloseTransaction.INSTANCE.toSerializable());
+    }
+
+    @Override
+    public Future<ActorSelection> readyTransaction() {
+        LOG.debug("Tx {} readyTransaction called with {} previous recorded operations pending",
+                identifier, recordedOperationFutures.size());
+
+        // Send the ReadyTransaction message to the Tx actor.
+
+        final Future<Object> replyFuture = executeOperationAsync(ReadyTransaction.INSTANCE);
+
+        // Combine all the previously recorded put/merge/delete operation reply Futures and the
+        // ReadyTransactionReply Future into one Future. If any one fails then the combined
+        // Future will fail. We need all prior operations and the ready operation to succeed
+        // in order to attempt commit.
+
+        List<Future<Object>> futureList =
+                Lists.newArrayListWithCapacity(recordedOperationFutures.size() + 1);
+        futureList.addAll(recordedOperationFutures);
+        futureList.add(replyFuture);
+
+        Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(futureList,
+                actorContext.getActorSystem().dispatcher());
+
+        // Transform the combined Future into a Future that returns the cohort actor path from
+        // the ReadyTransactionReply. That's the end result of the ready operation.
+
+        return combinedFutures.transform(new Mapper<Iterable<Object>, ActorSelection>() {
+            @Override
+            public ActorSelection checkedApply(Iterable<Object> notUsed) {
+                LOG.debug("Tx {} readyTransaction: pending recorded operations succeeded",
+                        identifier);
+
+                // At this point all the Futures succeeded and we need to extract the cohort
+                // actor path from the ReadyTransactionReply. For the recorded operations, they
+                // don't return any data so we're only interested that they completed
+                // successfully. We could be paranoid and verify the correct reply types but
+                // that really should never happen so it's not worth the overhead of
+                // de-serializing each reply.
+
+                // Note the Future get call here won't block as it's complete.
+                Object serializedReadyReply = replyFuture.value().get().get();
+                if (serializedReadyReply instanceof ReadyTransactionReply) {
+                    return actorContext.actorSelection(((ReadyTransactionReply)serializedReadyReply).getCohortPath());
+
+                } else if(serializedReadyReply.getClass().equals(ReadyTransactionReply.SERIALIZABLE_CLASS)) {
+                    ReadyTransactionReply reply = ReadyTransactionReply.fromSerializable(serializedReadyReply);
+                    String cohortPath = reply.getCohortPath();
+
+                    // In Helium we used to return the local path of the actor which represented
+                    // a remote ThreePhaseCommitCohort. The local path would then be converted to
+                    // a remote path using this resolvePath method. To maintain compatibility with
+                    // a Helium node we need to continue to do this conversion.
+                    // At some point in the future when upgrades from Helium are not supported
+                    // we could remove this code to resolvePath and just use the cohortPath as the
+                    // resolved cohortPath
+                    if(TransactionContextImpl.this.remoteTransactionVersion <
+                            DataStoreVersions.HELIUM_1_VERSION) {
+                        cohortPath = actorContext.resolvePath(transactionPath, cohortPath);
+                    }
+
+                    return actorContext.actorSelection(cohortPath);
+
+                } else {
+                    // Throwing an exception here will fail the Future.
+                    throw new IllegalArgumentException(String.format("Invalid reply type %s",
+                            serializedReadyReply.getClass()));
+                }
+            }
+        }, TransactionProxy.SAME_FAILURE_TRANSFORMER, actorContext.getActorSystem().dispatcher());
+    }
+
+    @Override
+    public void deleteData(YangInstanceIdentifier path) {
+        LOG.debug("Tx {} deleteData called path = {}", identifier, path);
+
+        recordedOperationFutures.add(executeOperationAsync(new DeleteData(path)));
+    }
+
+    @Override
+    public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
+        LOG.debug("Tx {} mergeData called path = {}", identifier, path);
+
+        recordedOperationFutures.add(executeOperationAsync(new MergeData(path, data)));
+    }
+
+    @Override
+    public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
+        LOG.debug("Tx {} writeData called path = {}", identifier, path);
+
+        recordedOperationFutures.add(executeOperationAsync(new WriteData(path, data)));
+    }
+
+    @Override
+    public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
+            final YangInstanceIdentifier path) {
+
+        LOG.debug("Tx {} readData called path = {}", identifier, path);
+
+        final SettableFuture<Optional<NormalizedNode<?, ?>>> returnFuture = SettableFuture.create();
+
+        // If there were any previous recorded put/merge/delete operation reply Futures then we
+        // must wait for them to successfully complete. This is necessary to honor the read
+        // uncommitted semantics of the public API contract. If any one fails then fail the read.
+
+        if(recordedOperationFutures.isEmpty()) {
+            finishReadData(path, returnFuture);
+        } else {
+            LOG.debug("Tx {} readData: verifying {} previous recorded operations",
+                    identifier, recordedOperationFutures.size());
+
+            // Note: we make a copy of recordedOperationFutures to be on the safe side in case
+            // Futures#sequence accesses the passed List on a different thread, as
+            // recordedOperationFutures is not synchronized.
+
+            Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(
+                    Lists.newArrayList(recordedOperationFutures),
+                    actorContext.getActorSystem().dispatcher());
+
+            OnComplete<Iterable<Object>> onComplete = new OnComplete<Iterable<Object>>() {
+                @Override
+                public void onComplete(Throwable failure, Iterable<Object> notUsed)
+                        throws Throwable {
+                    if(failure != null) {
+                        LOG.debug("Tx {} readData: a recorded operation failed: {}",
+                                identifier, failure);
+                        returnFuture.setException(new ReadFailedException(
+                                "The read could not be performed because a previous put, merge,"
+                                + "or delete operation failed", failure));
+                    } else {
+                        finishReadData(path, returnFuture);
+                    }
+                }
+            };
+
+            combinedFutures.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
+        }
+
+        return MappingCheckedFuture.create(returnFuture, ReadFailedException.MAPPER);
+    }
+
+    private void finishReadData(final YangInstanceIdentifier path,
+            final SettableFuture<Optional<NormalizedNode<?, ?>>> returnFuture) {
+
+        LOG.debug("Tx {} finishReadData called path = {}", identifier, path);
+
+        OnComplete<Object> onComplete = new OnComplete<Object>() {
+            @Override
+            public void onComplete(Throwable failure, Object readResponse) throws Throwable {
+                if(failure != null) {
+                    LOG.debug("Tx {} read operation failed: {}", identifier, failure);
+                    returnFuture.setException(new ReadFailedException(
+                            "Error reading data for path " + path, failure));
+
+                } else {
+                    LOG.debug("Tx {} read operation succeeded", identifier, failure);
+
+                    if (readResponse instanceof ReadDataReply) {
+                        ReadDataReply reply = (ReadDataReply) readResponse;
+                        returnFuture.set(Optional.<NormalizedNode<?, ?>>fromNullable(reply.getNormalizedNode()));
+
+                    } else if (ReadDataReply.isSerializedType(readResponse)) {
+                        ReadDataReply reply = ReadDataReply.fromSerializable(readResponse);
+                        returnFuture.set(Optional.<NormalizedNode<?, ?>>fromNullable(reply.getNormalizedNode()));
+
+                    } else {
+                        returnFuture.setException(new ReadFailedException(
+                            "Invalid response reading data for path " + path));
+                    }
+                }
+            }
+        };
+
+        Future<Object> readFuture = executeOperationAsync(new ReadData(path));
+
+        readFuture.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
+    }
+
+    @Override
+    public CheckedFuture<Boolean, ReadFailedException> dataExists(
+            final YangInstanceIdentifier path) {
+
+        LOG.debug("Tx {} dataExists called path = {}", identifier, path);
+
+        final SettableFuture<Boolean> returnFuture = SettableFuture.create();
+
+        // If there were any previous recorded put/merge/delete operation reply Futures then we
+        // must wait for them to successfully complete. This is necessary to honor the read
+        // uncommitted semantics of the public API contract. If any one fails then fail this
+        // request.
+
+        if(recordedOperationFutures.isEmpty()) {
+            finishDataExists(path, returnFuture);
+        } else {
+            LOG.debug("Tx {} dataExists: verifying {} previous recorded operations",
+                    identifier, recordedOperationFutures.size());
+
+            // Note: we make a copy of recordedOperationFutures to be on the safe side in case
+            // Futures#sequence accesses the passed List on a different thread, as
+            // recordedOperationFutures is not synchronized.
+
+            Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(
+                    Lists.newArrayList(recordedOperationFutures),
+                    actorContext.getActorSystem().dispatcher());
+            OnComplete<Iterable<Object>> onComplete = new OnComplete<Iterable<Object>>() {
+                @Override
+                public void onComplete(Throwable failure, Iterable<Object> notUsed)
+                        throws Throwable {
+                    if(failure != null) {
+                        LOG.debug("Tx {} dataExists: a recorded operation failed: {}",
+                                identifier, failure);
+                        returnFuture.setException(new ReadFailedException(
+                                "The data exists could not be performed because a previous "
+                                + "put, merge, or delete operation failed", failure));
+                    } else {
+                        finishDataExists(path, returnFuture);
+                    }
+                }
+            };
+
+            combinedFutures.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
+        }
+
+        return MappingCheckedFuture.create(returnFuture, ReadFailedException.MAPPER);
+    }
+
+    private void finishDataExists(final YangInstanceIdentifier path,
+            final SettableFuture<Boolean> returnFuture) {
+
+        LOG.debug("Tx {} finishDataExists called path = {}", identifier, path);
+
+        OnComplete<Object> onComplete = new OnComplete<Object>() {
+            @Override
+            public void onComplete(Throwable failure, Object response) throws Throwable {
+                if(failure != null) {
+                    LOG.debug("Tx {} dataExists operation failed: {}", identifier, failure);
+                    returnFuture.setException(new ReadFailedException(
+                            "Error checking data exists for path " + path, failure));
+                } else {
+                    LOG.debug("Tx {} dataExists operation succeeded", identifier, failure);
+
+                    if (response instanceof DataExistsReply) {
+                        returnFuture.set(Boolean.valueOf(((DataExistsReply) response).exists()));
+
+                    } else if (response.getClass().equals(DataExistsReply.SERIALIZABLE_CLASS)) {
+                        returnFuture.set(Boolean.valueOf(DataExistsReply.fromSerializable(response).exists()));
+
+                    } else {
+                        returnFuture.setException(new ReadFailedException(
+                                "Invalid response checking exists for path " + path));
+                    }
+                }
+            }
+        };
+
+        Future<Object> future = executeOperationAsync(new DataExists(path));
+
+        future.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
+    }
+}
\ No newline at end of file
index af1e4e92b60ef80effd9b3d443fd9743cef54900..f28a1e5f73d4823fe31ba20d14fa4d986f9700da 100644 (file)
@@ -35,17 +35,6 @@ import org.opendaylight.controller.cluster.datastore.identifiers.TransactionIden
 import org.opendaylight.controller.cluster.datastore.messages.CloseTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
 import org.opendaylight.controller.cluster.datastore.messages.CloseTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
-import org.opendaylight.controller.cluster.datastore.messages.DataExists;
-import org.opendaylight.controller.cluster.datastore.messages.DataExistsReply;
-import org.opendaylight.controller.cluster.datastore.messages.DeleteData;
-import org.opendaylight.controller.cluster.datastore.messages.MergeData;
-import org.opendaylight.controller.cluster.datastore.messages.ReadData;
-import org.opendaylight.controller.cluster.datastore.messages.ReadDataReply;
-import org.opendaylight.controller.cluster.datastore.messages.ReadyTransaction;
-import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
-import org.opendaylight.controller.cluster.datastore.messages.SerializableMessage;
-import org.opendaylight.controller.cluster.datastore.messages.VersionedSerializableMessage;
-import org.opendaylight.controller.cluster.datastore.messages.WriteData;
 import org.opendaylight.controller.cluster.datastore.shardstrategy.ShardStrategyFactory;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
 import org.opendaylight.controller.cluster.datastore.shardstrategy.ShardStrategyFactory;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
@@ -754,420 +743,4 @@ public class TransactionProxy implements DOMStoreReadWriteTransaction {
                 actorContext, schemaContext, isTxActorLocal, reply.getVersion(), operationCompleter);
         }
     }
                 actorContext, schemaContext, isTxActorLocal, reply.getVersion(), operationCompleter);
         }
     }
-
-    private interface TransactionContext {
-        void closeTransaction();
-
-        Future<ActorSelection> readyTransaction();
-
-        void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
-
-        void deleteData(YangInstanceIdentifier path);
-
-        void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data);
-
-        CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
-                final YangInstanceIdentifier path);
-
-        CheckedFuture<Boolean, ReadFailedException> dataExists(YangInstanceIdentifier path);
-
-        List<Future<Object>> getRecordedOperationFutures();
-    }
-
-    private static abstract class AbstractTransactionContext implements TransactionContext {
-
-        protected final TransactionIdentifier identifier;
-        protected final List<Future<Object>> recordedOperationFutures = Lists.newArrayList();
-
-        AbstractTransactionContext(TransactionIdentifier identifier) {
-            this.identifier = identifier;
-        }
-
-        @Override
-        public List<Future<Object>> getRecordedOperationFutures() {
-            return recordedOperationFutures;
-        }
-    }
-
-    private static class TransactionContextImpl extends AbstractTransactionContext {
-        private final Logger LOG = LoggerFactory.getLogger(TransactionContextImpl.class);
-
-        private final ActorContext actorContext;
-        private final String transactionPath;
-        private final ActorSelection actor;
-        private final boolean isTxActorLocal;
-        private final short remoteTransactionVersion;
-        private final OperationCompleter operationCompleter;
-
-
-        private TransactionContextImpl(String transactionPath, ActorSelection actor, TransactionIdentifier identifier,
-                ActorContext actorContext, SchemaContext schemaContext,
-                boolean isTxActorLocal, short remoteTransactionVersion, OperationCompleter operationCompleter) {
-            super(identifier);
-            this.transactionPath = transactionPath;
-            this.actor = actor;
-            this.actorContext = actorContext;
-            this.isTxActorLocal = isTxActorLocal;
-            this.remoteTransactionVersion = remoteTransactionVersion;
-            this.operationCompleter = operationCompleter;
-        }
-
-        private Future<Object> completeOperation(Future<Object> operationFuture){
-            operationFuture.onComplete(this.operationCompleter, actorContext.getActorSystem().dispatcher());
-            return operationFuture;
-        }
-
-
-        private ActorSelection getActor() {
-            return actor;
-        }
-
-        private Future<Object> executeOperationAsync(SerializableMessage msg) {
-            return completeOperation(actorContext.executeOperationAsync(getActor(), isTxActorLocal ? msg : msg.toSerializable()));
-        }
-
-        private Future<Object> executeOperationAsync(VersionedSerializableMessage msg) {
-            return completeOperation(actorContext.executeOperationAsync(getActor(), isTxActorLocal ? msg :
-                    msg.toSerializable(remoteTransactionVersion)));
-        }
-
-        @Override
-        public void closeTransaction() {
-            LOG.debug("Tx {} closeTransaction called", identifier);
-
-            actorContext.sendOperationAsync(getActor(), CloseTransaction.INSTANCE.toSerializable());
-        }
-
-        @Override
-        public Future<ActorSelection> readyTransaction() {
-            LOG.debug("Tx {} readyTransaction called with {} previous recorded operations pending",
-                    identifier, recordedOperationFutures.size());
-
-            // Send the ReadyTransaction message to the Tx actor.
-
-            final Future<Object> replyFuture = executeOperationAsync(ReadyTransaction.INSTANCE);
-
-            // Combine all the previously recorded put/merge/delete operation reply Futures and the
-            // ReadyTransactionReply Future into one Future. If any one fails then the combined
-            // Future will fail. We need all prior operations and the ready operation to succeed
-            // in order to attempt commit.
-
-            List<Future<Object>> futureList =
-                    Lists.newArrayListWithCapacity(recordedOperationFutures.size() + 1);
-            futureList.addAll(recordedOperationFutures);
-            futureList.add(replyFuture);
-
-            Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(futureList,
-                    actorContext.getActorSystem().dispatcher());
-
-            // Transform the combined Future into a Future that returns the cohort actor path from
-            // the ReadyTransactionReply. That's the end result of the ready operation.
-
-            return combinedFutures.transform(new Mapper<Iterable<Object>, ActorSelection>() {
-                @Override
-                public ActorSelection checkedApply(Iterable<Object> notUsed) {
-                    LOG.debug("Tx {} readyTransaction: pending recorded operations succeeded",
-                            identifier);
-
-                    // At this point all the Futures succeeded and we need to extract the cohort
-                    // actor path from the ReadyTransactionReply. For the recorded operations, they
-                    // don't return any data so we're only interested that they completed
-                    // successfully. We could be paranoid and verify the correct reply types but
-                    // that really should never happen so it's not worth the overhead of
-                    // de-serializing each reply.
-
-                    // Note the Future get call here won't block as it's complete.
-                    Object serializedReadyReply = replyFuture.value().get().get();
-                    if (serializedReadyReply instanceof ReadyTransactionReply) {
-                        return actorContext.actorSelection(((ReadyTransactionReply)serializedReadyReply).getCohortPath());
-
-                    } else if(serializedReadyReply.getClass().equals(ReadyTransactionReply.SERIALIZABLE_CLASS)) {
-                        ReadyTransactionReply reply = ReadyTransactionReply.fromSerializable(serializedReadyReply);
-                        String cohortPath = reply.getCohortPath();
-
-                        // In Helium we used to return the local path of the actor which represented
-                        // a remote ThreePhaseCommitCohort. The local path would then be converted to
-                        // a remote path using this resolvePath method. To maintain compatibility with
-                        // a Helium node we need to continue to do this conversion.
-                        // At some point in the future when upgrades from Helium are not supported
-                        // we could remove this code to resolvePath and just use the cohortPath as the
-                        // resolved cohortPath
-                        if(TransactionContextImpl.this.remoteTransactionVersion <
-                                DataStoreVersions.HELIUM_1_VERSION) {
-                            cohortPath = actorContext.resolvePath(transactionPath, cohortPath);
-                        }
-
-                        return actorContext.actorSelection(cohortPath);
-
-                    } else {
-                        // Throwing an exception here will fail the Future.
-                        throw new IllegalArgumentException(String.format("Invalid reply type %s",
-                                serializedReadyReply.getClass()));
-                    }
-                }
-            }, SAME_FAILURE_TRANSFORMER, actorContext.getActorSystem().dispatcher());
-        }
-
-        @Override
-        public void deleteData(YangInstanceIdentifier path) {
-            LOG.debug("Tx {} deleteData called path = {}", identifier, path);
-
-            recordedOperationFutures.add(executeOperationAsync(new DeleteData(path)));
-        }
-
-        @Override
-        public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-            LOG.debug("Tx {} mergeData called path = {}", identifier, path);
-
-            recordedOperationFutures.add(executeOperationAsync(new MergeData(path, data)));
-        }
-
-        @Override
-        public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-            LOG.debug("Tx {} writeData called path = {}", identifier, path);
-
-            recordedOperationFutures.add(executeOperationAsync(new WriteData(path, data)));
-        }
-
-        @Override
-        public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
-                final YangInstanceIdentifier path) {
-
-            LOG.debug("Tx {} readData called path = {}", identifier, path);
-
-            final SettableFuture<Optional<NormalizedNode<?, ?>>> returnFuture = SettableFuture.create();
-
-            // If there were any previous recorded put/merge/delete operation reply Futures then we
-            // must wait for them to successfully complete. This is necessary to honor the read
-            // uncommitted semantics of the public API contract. If any one fails then fail the read.
-
-            if(recordedOperationFutures.isEmpty()) {
-                finishReadData(path, returnFuture);
-            } else {
-                LOG.debug("Tx {} readData: verifying {} previous recorded operations",
-                        identifier, recordedOperationFutures.size());
-
-                // Note: we make a copy of recordedOperationFutures to be on the safe side in case
-                // Futures#sequence accesses the passed List on a different thread, as
-                // recordedOperationFutures is not synchronized.
-
-                Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(
-                        Lists.newArrayList(recordedOperationFutures),
-                        actorContext.getActorSystem().dispatcher());
-
-                OnComplete<Iterable<Object>> onComplete = new OnComplete<Iterable<Object>>() {
-                    @Override
-                    public void onComplete(Throwable failure, Iterable<Object> notUsed)
-                            throws Throwable {
-                        if(failure != null) {
-                            LOG.debug("Tx {} readData: a recorded operation failed: {}",
-                                    identifier, failure);
-                            returnFuture.setException(new ReadFailedException(
-                                    "The read could not be performed because a previous put, merge,"
-                                    + "or delete operation failed", failure));
-                        } else {
-                            finishReadData(path, returnFuture);
-                        }
-                    }
-                };
-
-                combinedFutures.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
-            }
-
-            return MappingCheckedFuture.create(returnFuture, ReadFailedException.MAPPER);
-        }
-
-        private void finishReadData(final YangInstanceIdentifier path,
-                final SettableFuture<Optional<NormalizedNode<?, ?>>> returnFuture) {
-
-            LOG.debug("Tx {} finishReadData called path = {}", identifier, path);
-
-            OnComplete<Object> onComplete = new OnComplete<Object>() {
-                @Override
-                public void onComplete(Throwable failure, Object readResponse) throws Throwable {
-                    if(failure != null) {
-                        LOG.debug("Tx {} read operation failed: {}", identifier, failure);
-                        returnFuture.setException(new ReadFailedException(
-                                "Error reading data for path " + path, failure));
-
-                    } else {
-                        LOG.debug("Tx {} read operation succeeded", identifier, failure);
-
-                        if (readResponse instanceof ReadDataReply) {
-                            ReadDataReply reply = (ReadDataReply) readResponse;
-                            returnFuture.set(Optional.<NormalizedNode<?, ?>>fromNullable(reply.getNormalizedNode()));
-
-                        } else if (ReadDataReply.isSerializedType(readResponse)) {
-                            ReadDataReply reply = ReadDataReply.fromSerializable(readResponse);
-                            returnFuture.set(Optional.<NormalizedNode<?, ?>>fromNullable(reply.getNormalizedNode()));
-
-                        } else {
-                            returnFuture.setException(new ReadFailedException(
-                                "Invalid response reading data for path " + path));
-                        }
-                    }
-                }
-            };
-
-            Future<Object> readFuture = executeOperationAsync(new ReadData(path));
-
-            readFuture.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
-        }
-
-        @Override
-        public CheckedFuture<Boolean, ReadFailedException> dataExists(
-                final YangInstanceIdentifier path) {
-
-            LOG.debug("Tx {} dataExists called path = {}", identifier, path);
-
-            final SettableFuture<Boolean> returnFuture = SettableFuture.create();
-
-            // If there were any previous recorded put/merge/delete operation reply Futures then we
-            // must wait for them to successfully complete. This is necessary to honor the read
-            // uncommitted semantics of the public API contract. If any one fails then fail this
-            // request.
-
-            if(recordedOperationFutures.isEmpty()) {
-                finishDataExists(path, returnFuture);
-            } else {
-                LOG.debug("Tx {} dataExists: verifying {} previous recorded operations",
-                        identifier, recordedOperationFutures.size());
-
-                // Note: we make a copy of recordedOperationFutures to be on the safe side in case
-                // Futures#sequence accesses the passed List on a different thread, as
-                // recordedOperationFutures is not synchronized.
-
-                Future<Iterable<Object>> combinedFutures = akka.dispatch.Futures.sequence(
-                        Lists.newArrayList(recordedOperationFutures),
-                        actorContext.getActorSystem().dispatcher());
-                OnComplete<Iterable<Object>> onComplete = new OnComplete<Iterable<Object>>() {
-                    @Override
-                    public void onComplete(Throwable failure, Iterable<Object> notUsed)
-                            throws Throwable {
-                        if(failure != null) {
-                            LOG.debug("Tx {} dataExists: a recorded operation failed: {}",
-                                    identifier, failure);
-                            returnFuture.setException(new ReadFailedException(
-                                    "The data exists could not be performed because a previous "
-                                    + "put, merge, or delete operation failed", failure));
-                        } else {
-                            finishDataExists(path, returnFuture);
-                        }
-                    }
-                };
-
-                combinedFutures.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
-            }
-
-            return MappingCheckedFuture.create(returnFuture, ReadFailedException.MAPPER);
-        }
-
-        private void finishDataExists(final YangInstanceIdentifier path,
-                final SettableFuture<Boolean> returnFuture) {
-
-            LOG.debug("Tx {} finishDataExists called path = {}", identifier, path);
-
-            OnComplete<Object> onComplete = new OnComplete<Object>() {
-                @Override
-                public void onComplete(Throwable failure, Object response) throws Throwable {
-                    if(failure != null) {
-                        LOG.debug("Tx {} dataExists operation failed: {}", identifier, failure);
-                        returnFuture.setException(new ReadFailedException(
-                                "Error checking data exists for path " + path, failure));
-                    } else {
-                        LOG.debug("Tx {} dataExists operation succeeded", identifier, failure);
-
-                        if (response instanceof DataExistsReply) {
-                            returnFuture.set(Boolean.valueOf(((DataExistsReply) response).exists()));
-
-                        } else if (response.getClass().equals(DataExistsReply.SERIALIZABLE_CLASS)) {
-                            returnFuture.set(Boolean.valueOf(DataExistsReply.fromSerializable(response).exists()));
-
-                        } else {
-                            returnFuture.setException(new ReadFailedException(
-                                    "Invalid response checking exists for path " + path));
-                        }
-                    }
-                }
-            };
-
-            Future<Object> future = executeOperationAsync(new DataExists(path));
-
-            future.onComplete(onComplete, actorContext.getActorSystem().dispatcher());
-        }
-    }
-
-    private static class NoOpTransactionContext extends AbstractTransactionContext {
-
-        private final Logger LOG = LoggerFactory.getLogger(NoOpTransactionContext.class);
-
-        private final Throwable failure;
-        private final Semaphore operationLimiter;
-
-        public NoOpTransactionContext(Throwable failure, TransactionIdentifier identifier, Semaphore operationLimiter){
-            super(identifier);
-            this.failure = failure;
-            this.operationLimiter = operationLimiter;
-        }
-
-        @Override
-        public void closeTransaction() {
-            LOG.debug("NoOpTransactionContext {} closeTransaction called", identifier);
-        }
-
-        @Override
-        public Future<ActorSelection> readyTransaction() {
-            LOG.debug("Tx {} readyTransaction called", identifier);
-            operationLimiter.release();
-            return akka.dispatch.Futures.failed(failure);
-        }
-
-        @Override
-        public void deleteData(YangInstanceIdentifier path) {
-            LOG.debug("Tx {} deleteData called path = {}", identifier, path);
-            operationLimiter.release();
-        }
-
-        @Override
-        public void mergeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-            LOG.debug("Tx {} mergeData called path = {}", identifier, path);
-            operationLimiter.release();
-        }
-
-        @Override
-        public void writeData(YangInstanceIdentifier path, NormalizedNode<?, ?> data) {
-            LOG.debug("Tx {} writeData called path = {}", identifier, path);
-            operationLimiter.release();
-        }
-
-        @Override
-        public CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> readData(
-                YangInstanceIdentifier path) {
-            LOG.debug("Tx {} readData called path = {}", identifier, path);
-            operationLimiter.release();
-            return Futures.immediateFailedCheckedFuture(new ReadFailedException(
-                    "Error reading data for path " + path, failure));
-        }
-
-        @Override
-        public CheckedFuture<Boolean, ReadFailedException> dataExists(
-                YangInstanceIdentifier path) {
-            LOG.debug("Tx {} dataExists called path = {}", identifier, path);
-            operationLimiter.release();
-            return Futures.immediateFailedCheckedFuture(new ReadFailedException(
-                    "Error checking exists for path " + path, failure));
-        }
-    }
-
-    private static class OperationCompleter extends OnComplete<Object> {
-        private final Semaphore operationLimiter;
-        OperationCompleter(Semaphore operationLimiter){
-            this.operationLimiter = operationLimiter;
-        }
-
-        @Override
-        public void onComplete(Throwable throwable, Object o){
-            this.operationLimiter.release();
-        }
-    }
 }
 }