Improve segmented journal actor metrics
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / SimpleShardDataTreeCohort.java
index bb016a28bdb29b994a8c23697e6c6fe7843a2d2b..2c7d13189f61a832fe386ed383cb1b4b193120ad 100644 (file)
@@ -7,35 +7,34 @@
  */
 package org.opendaylight.controller.cluster.datastore;
 
-import akka.dispatch.ExecutionContexts;
-import akka.dispatch.OnComplete;
+import static com.google.common.base.Verify.verifyNotNull;
+import static java.util.Objects.requireNonNull;
+
+import com.google.common.base.MoreObjects.ToStringHelper;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Verify;
 import com.google.common.primitives.UnsignedLong;
 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.SettableFuture;
 import java.util.Optional;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeoutException;
+import java.util.SortedSet;
+import java.util.concurrent.CompletionStage;
+import org.eclipse.jdt.annotation.NonNull;
+import org.eclipse.jdt.annotation.Nullable;
 import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
-import org.opendaylight.controller.cluster.datastore.utils.PruningDataTreeModification;
-import org.opendaylight.yangtools.concepts.Identifiable;
-import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidate;
-import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidateTip;
-import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeModification;
+import org.opendaylight.yangtools.yang.common.Empty;
+import org.opendaylight.yangtools.yang.data.tree.api.DataTreeCandidate;
+import org.opendaylight.yangtools.yang.data.tree.api.DataTreeCandidateTip;
+import org.opendaylight.yangtools.yang.data.tree.api.DataTreeModification;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.concurrent.Future;
 
-final class SimpleShardDataTreeCohort extends ShardDataTreeCohort implements Identifiable<TransactionIdentifier> {
+final class SimpleShardDataTreeCohort extends ShardDataTreeCohort {
     private static final Logger LOG = LoggerFactory.getLogger(SimpleShardDataTreeCohort.class);
-    private static final ListenableFuture<Void> VOID_FUTURE = Futures.immediateFuture(null);
+
     private final DataTreeModification transaction;
     private final ShardDataTree dataTree;
-    private final TransactionIdentifier transactionId;
+    private final @NonNull TransactionIdentifier transactionId;
     private final CompositeDataTreeCohort userCohorts;
+    private final @Nullable SortedSet<String> participatingShardNames;
 
     private State state = State.READY;
     private DataTreeCandidateTip candidate;
@@ -43,15 +42,27 @@ final class SimpleShardDataTreeCohort extends ShardDataTreeCohort implements Ide
     private Exception nextFailure;
 
     SimpleShardDataTreeCohort(final ShardDataTree dataTree, final DataTreeModification transaction,
-            final TransactionIdentifier transactionId, final CompositeDataTreeCohort userCohorts) {
-        this.dataTree = Preconditions.checkNotNull(dataTree);
-        this.transaction = Preconditions.checkNotNull(transaction);
-        this.transactionId = Preconditions.checkNotNull(transactionId);
-        this.userCohorts = Preconditions.checkNotNull(userCohorts);
+            final TransactionIdentifier transactionId, final CompositeDataTreeCohort userCohorts,
+            final Optional<SortedSet<String>> participatingShardNames) {
+        this.dataTree = requireNonNull(dataTree);
+        this.transaction = requireNonNull(transaction);
+        this.transactionId = requireNonNull(transactionId);
+        this.userCohorts = requireNonNull(userCohorts);
+        this.participatingShardNames = requireNonNull(participatingShardNames).orElse(null);
+    }
+
+    SimpleShardDataTreeCohort(final ShardDataTree dataTree, final DataTreeModification transaction,
+        final TransactionIdentifier transactionId, final Exception nextFailure) {
+        this.dataTree = requireNonNull(dataTree);
+        this.transaction = requireNonNull(transaction);
+        this.transactionId = requireNonNull(transactionId);
+        userCohorts = null;
+        participatingShardNames = null;
+        this.nextFailure = requireNonNull(nextFailure);
     }
 
     @Override
-    public TransactionIdentifier getIdentifier() {
+    TransactionIdentifier transactionId() {
         return transactionId;
     }
 
@@ -61,34 +72,41 @@ final class SimpleShardDataTreeCohort extends ShardDataTreeCohort implements Ide
     }
 
     @Override
-    public DataTreeModification getDataTreeModification() {
-        DataTreeModification dataTreeModification = transaction;
-        if (transaction instanceof PruningDataTreeModification){
-            dataTreeModification = ((PruningDataTreeModification) transaction).getResultingModification();
-        }
-        return dataTreeModification;
+    DataTreeModification getDataTreeModification() {
+        return transaction;
     }
 
-    private void checkState(State expected) {
-        Preconditions.checkState(state == expected, "State %s does not match expected state %s", state, expected);
+    @Override
+    Optional<SortedSet<String>> getParticipatingShardNames() {
+        return Optional.ofNullable(participatingShardNames);
+    }
+
+    private void checkState(final State expected) {
+        Preconditions.checkState(state == expected, "State %s does not match expected state %s for %s",
+                state, expected, transactionId());
     }
 
     @Override
-    public void canCommit(final FutureCallback<Void> callback) {
-        if(state == State.CAN_COMMIT_PENDING) {
+    public void canCommit(final FutureCallback<Empty> newCallback) {
+        if (state == State.CAN_COMMIT_PENDING) {
             return;
         }
 
         checkState(State.READY);
-        this.callback = Preconditions.checkNotNull(callback);
+        callback = requireNonNull(newCallback);
         state = State.CAN_COMMIT_PENDING;
-        dataTree.startCanCommit(this);
+
+        if (nextFailure == null) {
+            dataTree.startCanCommit(this);
+        } else {
+            failedCanCommit(nextFailure);
+        }
     }
 
     @Override
-    public void preCommit(final FutureCallback<DataTreeCandidate> callback) {
+    public void preCommit(final FutureCallback<DataTreeCandidate> newCallback) {
         checkState(State.CAN_COMMIT_COMPLETE);
-        this.callback = Preconditions.checkNotNull(callback);
+        callback = requireNonNull(newCallback);
         state = State.PRE_COMMIT_PENDING;
 
         if (nextFailure == null) {
@@ -99,54 +117,59 @@ final class SimpleShardDataTreeCohort extends ShardDataTreeCohort implements Ide
     }
 
     @Override
-    public ListenableFuture<Void> abort() {
-        dataTree.startAbort(this);
+    public void abort(final FutureCallback<Empty> abortCallback) {
+        if (!dataTree.startAbort(this)) {
+            abortCallback.onSuccess(Empty.value());
+            return;
+        }
+
+        candidate = null;
         state = State.ABORTED;
 
-        final Optional<Future<Iterable<Object>>> maybeAborts = userCohorts.abort();
+        final Optional<CompletionStage<?>> maybeAborts = userCohorts.abort();
         if (!maybeAborts.isPresent()) {
-            return VOID_FUTURE;
-        }
-
-        final Future<Iterable<Object>> aborts = maybeAborts.get();
-        if (aborts.isCompleted()) {
-            return VOID_FUTURE;
+            abortCallback.onSuccess(Empty.value());
+            return;
         }
 
-        final SettableFuture<Void> ret = SettableFuture.create();
-        aborts.onComplete(new OnComplete<Iterable<Object>>() {
-            @Override
-            public void onComplete(final Throwable failure, final Iterable<Object> objs) {
-                if (failure != null) {
-                    ret.setException(failure);
-                } else {
-                    ret.set(null);
-                }
+        maybeAborts.orElseThrow().whenComplete((noop, failure) -> {
+            if (failure != null) {
+                abortCallback.onFailure(failure);
+            } else {
+                abortCallback.onSuccess(Empty.value());
             }
-        }, ExecutionContexts.global());
-
-        return ret;
+        });
     }
 
     @Override
-    public void commit(final FutureCallback<UnsignedLong> callback) {
+    public void commit(final FutureCallback<UnsignedLong> newCallback) {
         checkState(State.PRE_COMMIT_COMPLETE);
-        this.callback = Preconditions.checkNotNull(callback);
+        callback = requireNonNull(newCallback);
         state = State.COMMIT_PENDING;
-        dataTree.startCommit(this, candidate);
+
+        if (nextFailure == null) {
+            dataTree.startCommit(this, candidate);
+        } else {
+            failedCommit(nextFailure);
+        }
     }
 
     private <T> FutureCallback<T> switchState(final State newState) {
         @SuppressWarnings("unchecked")
-        final FutureCallback<T> ret = (FutureCallback<T>) this.callback;
-        this.callback = null;
+        final FutureCallback<T> ret = (FutureCallback<T>) callback;
+        callback = null;
         LOG.debug("Transaction {} changing state from {} to {}", transactionId, state, newState);
-        this.state = newState;
+        state = newState;
         return ret;
     }
 
+    void setNewCandidate(final DataTreeCandidateTip dataTreeCandidate) {
+        checkState(State.PRE_COMMIT_COMPLETE);
+        candidate = verifyNotNull(dataTreeCandidate);
+    }
+
     void successfulCanCommit() {
-        switchState(State.CAN_COMMIT_COMPLETE).onSuccess(null);
+        switchState(State.CAN_COMMIT_COMPLETE).onSuccess(Empty.value());
     }
 
     void failedCanCommit(final Exception cause) {
@@ -157,23 +180,50 @@ final class SimpleShardDataTreeCohort extends ShardDataTreeCohort implements Ide
      * Run user-defined canCommit and preCommit hooks. We want to run these before we initiate persistence so that
      * any failure to validate is propagated before we record the transaction.
      *
-     * @param candidate {@link DataTreeCandidate} under consideration
-     * @throws ExecutionException
-     * @throws TimeoutException
+     * @param dataTreeCandidate {@link DataTreeCandidate} under consideration
+     * @param futureCallback the callback to invoke on completion, which may be immediate or async.
      */
-    // FIXME: this should be asynchronous
-    void userPreCommit(final DataTreeCandidate candidate) throws ExecutionException, TimeoutException {
-        userCohorts.canCommit(candidate);
-        userCohorts.preCommit();
+    void userPreCommit(final DataTreeCandidate dataTreeCandidate, final FutureCallback<Empty> futureCallback) {
+        userCohorts.reset();
+
+        final Optional<CompletionStage<Empty>> maybeCanCommitFuture = userCohorts.canCommit(dataTreeCandidate);
+        if (!maybeCanCommitFuture.isPresent()) {
+            doUserPreCommit(futureCallback);
+            return;
+        }
+
+        maybeCanCommitFuture.orElseThrow().whenComplete((noop, failure) -> {
+            if (failure != null) {
+                futureCallback.onFailure(failure);
+            } else {
+                doUserPreCommit(futureCallback);
+            }
+        });
     }
 
-    void successfulPreCommit(final DataTreeCandidateTip candidate) {
-        LOG.trace("Transaction {} prepared candidate {}", transaction, candidate);
-        this.candidate = Verify.verifyNotNull(candidate);
-        switchState(State.PRE_COMMIT_COMPLETE).onSuccess(candidate);
+    private void doUserPreCommit(final FutureCallback<Empty> futureCallback) {
+        final Optional<CompletionStage<Empty>> maybePreCommitFuture = userCohorts.preCommit();
+        if (!maybePreCommitFuture.isPresent()) {
+            futureCallback.onSuccess(Empty.value());
+            return;
+        }
+
+        maybePreCommitFuture.orElseThrow().whenComplete((noop, failure) -> {
+            if (failure != null) {
+                futureCallback.onFailure(failure);
+            } else {
+                futureCallback.onSuccess(Empty.value());
+            }
+        });
     }
 
-    void failedPreCommit(final Exception cause) {
+    void successfulPreCommit(final DataTreeCandidateTip dataTreeCandidate) {
+        LOG.trace("Transaction {} prepared candidate {}", transaction, dataTreeCandidate);
+        candidate = verifyNotNull(dataTreeCandidate);
+        switchState(State.PRE_COMMIT_COMPLETE).onSuccess(dataTreeCandidate);
+    }
+
+    void failedPreCommit(final Throwable cause) {
         if (LOG.isTraceEnabled()) {
             LOG.trace("Transaction {} failed to prepare", transaction, cause);
         } else {
@@ -184,15 +234,25 @@ final class SimpleShardDataTreeCohort extends ShardDataTreeCohort implements Ide
         switchState(State.FAILED).onFailure(cause);
     }
 
-    void successfulCommit(final UnsignedLong journalIndex) {
-        try {
-            userCohorts.commit();
-        } catch (TimeoutException | ExecutionException e) {
-            // We are probably dead, depending on what the cohorts end up doing
-            LOG.error("User cohorts failed to commit", e);
+    void successfulCommit(final UnsignedLong journalIndex, final Runnable onComplete) {
+        final Optional<CompletionStage<Empty>> maybeCommitFuture = userCohorts.commit();
+        if (!maybeCommitFuture.isPresent()) {
+            finishSuccessfulCommit(journalIndex, onComplete);
+            return;
         }
 
+        maybeCommitFuture.orElseThrow().whenComplete((noop, failure) -> {
+            if (failure != null) {
+                LOG.error("User cohorts failed to commit", failure);
+            }
+
+            finishSuccessfulCommit(journalIndex, onComplete);
+        });
+    }
+
+    private void finishSuccessfulCommit(final UnsignedLong journalIndex, final Runnable onComplete) {
         switchState(State.COMMITTED).onSuccess(journalIndex);
+        onComplete.run();
     }
 
     void failedCommit(final Exception cause) {
@@ -212,11 +272,20 @@ final class SimpleShardDataTreeCohort extends ShardDataTreeCohort implements Ide
     }
 
     void reportFailure(final Exception cause) {
-        this.nextFailure = Preconditions.checkNotNull(cause);
+        if (nextFailure == null) {
+            nextFailure = requireNonNull(cause);
+        } else {
+            LOG.debug("Transaction {} already has a set failure, not updating it", transactionId, cause);
+        }
     }
 
     @Override
     public boolean isFailed() {
         return state == State.FAILED || nextFailure != null;
     }
+
+    @Override
+    ToStringHelper addToStringAttributes(final ToStringHelper toStringHelper) {
+        return super.addToStringAttributes(toStringHelper).add("nextFailure", nextFailure);
+    }
 }