Improve segmented journal actor metrics
[controller.git] / opendaylight / md-sal / cds-access-client / src / main / java / org / opendaylight / controller / cluster / access / client / AbstractClientConnection.java
index abd668010a16ad09c3c0d573c0a0320cd59a9e85..f34760ec03c0057f2393d4a9504b6399de68ecb9 100644 (file)
@@ -7,22 +7,25 @@
  */
 package org.opendaylight.controller.cluster.access.client;
 
+import static java.util.Objects.requireNonNull;
+
 import akka.actor.ActorRef;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.MoreObjects;
 import com.google.common.base.MoreObjects.ToStringHelper;
-import com.google.common.base.Preconditions;
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import java.util.Collection;
+import java.util.List;
 import java.util.Optional;
+import java.util.OptionalLong;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.function.Consumer;
-import javax.annotation.Nonnull;
-import javax.annotation.concurrent.GuardedBy;
-import javax.annotation.concurrent.NotThreadSafe;
+import org.checkerframework.checker.lock.qual.GuardedBy;
+import org.checkerframework.checker.lock.qual.Holding;
+import org.eclipse.jdt.annotation.NonNull;
 import org.opendaylight.controller.cluster.access.concepts.Request;
 import org.opendaylight.controller.cluster.access.concepts.RequestException;
 import org.opendaylight.controller.cluster.access.concepts.Response;
@@ -35,11 +38,10 @@ import scala.concurrent.duration.FiniteDuration;
 /**
  * Base class for a connection to the backend. Responsible to queueing and dispatch of requests toward the backend.
  * Can be in three conceptual states: Connecting, Connected and Reconnecting, which are represented by public final
- * classes exposed from this package.
+ * classes exposed from this package. This class NOT thread-safe, not are its subclasses expected to be thread-safe.
  *
  * @author Robert Varga
  */
-@NotThreadSafe
 public abstract class AbstractClientConnection<T extends BackendInfo> {
     private static final Logger LOG = LoggerFactory.getLogger(AbstractClientConnection.class);
 
@@ -50,24 +52,21 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
      */
     /**
      * Backend aliveness timer. This is reset whenever we receive a response from the backend and kept armed whenever
-     * we have an outstanding request. If when this time expires, we tear down this connection and attept to reconnect
+     * we have an outstanding request. If when this time expires, we tear down this connection and attempt to reconnect
      * it.
      */
-    @VisibleForTesting
-    static final long BACKEND_ALIVE_TIMEOUT_NANOS = TimeUnit.SECONDS.toNanos(30);
+    public static final long DEFAULT_BACKEND_ALIVE_TIMEOUT_NANOS = TimeUnit.SECONDS.toNanos(30);
 
     /**
      * Request timeout. If the request fails to complete within this time since it was originally enqueued, we time
      * the request out.
      */
-    @VisibleForTesting
-    static final long REQUEST_TIMEOUT_NANOS = TimeUnit.MINUTES.toNanos(2);
+    public static final long DEFAULT_REQUEST_TIMEOUT_NANOS = TimeUnit.MINUTES.toNanos(2);
 
     /**
      * No progress timeout. A client fails to make any forward progress in this time, it will terminate itself.
      */
-    @VisibleForTesting
-    static final long NO_PROGRESS_TIMEOUT_NANOS = TimeUnit.MINUTES.toNanos(15);
+    public static final long DEFAULT_NO_PROGRESS_TIMEOUT_NANOS = TimeUnit.MINUTES.toNanos(15);
 
     // Emit a debug entry if we sleep for more that this amount
     private static final long DEBUG_DELAY_NANOS = TimeUnit.MILLISECONDS.toNanos(100);
@@ -77,10 +76,11 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
     private static final long MAX_DELAY_NANOS = TimeUnit.SECONDS.toNanos(MAX_DELAY_SECONDS);
 
     private final Lock lock = new ReentrantLock();
-    private final ClientActorContext context;
+    private final @NonNull ClientActorContext context;
+    private final @NonNull Long cookie;
+    private final String backendName;
     @GuardedBy("lock")
     private final TransmitQueue queue;
-    private final Long cookie;
 
     @GuardedBy("lock")
     private boolean haveTimer;
@@ -92,32 +92,51 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
 
     private volatile RequestException poisoned;
 
+    // Private constructor to avoid code duplication.
+    private AbstractClientConnection(final AbstractClientConnection<T> oldConn, final TransmitQueue newQueue,
+            final String backendName) {
+        context = oldConn.context;
+        cookie = oldConn.cookie;
+        this.backendName = requireNonNull(backendName);
+        queue = requireNonNull(newQueue);
+        // Will be updated in finishReplay if needed.
+        lastReceivedTicks = oldConn.lastReceivedTicks;
+    }
+
+    // This constructor is only to be called by ConnectingClientConnection constructor.
     // Do not allow subclassing outside of this package
-    AbstractClientConnection(final ClientActorContext context, final Long cookie,
-            final TransmitQueue queue) {
-        this.context = Preconditions.checkNotNull(context);
-        this.cookie = Preconditions.checkNotNull(cookie);
-        this.queue = Preconditions.checkNotNull(queue);
-        this.lastReceivedTicks = currentTime();
+    AbstractClientConnection(final ClientActorContext context, final Long cookie, final String backendName,
+            final int queueDepth) {
+        this.context = requireNonNull(context);
+        this.cookie = requireNonNull(cookie);
+        this.backendName = requireNonNull(backendName);
+        queue = new TransmitQueue.Halted(queueDepth);
+        lastReceivedTicks = currentTime();
     }
 
+    // This constructor is only to be called (indirectly) by ReconnectingClientConnection constructor.
     // Do not allow subclassing outside of this package
-    AbstractClientConnection(final AbstractClientConnection<T> oldConnection, final int targetQueueSize) {
-        this.context = oldConnection.context;
-        this.cookie = oldConnection.cookie;
-        this.queue = new TransmitQueue.Halted(targetQueueSize);
-        this.lastReceivedTicks = oldConnection.lastReceivedTicks;
+    AbstractClientConnection(final AbstractClientConnection<T> oldConn) {
+        this(oldConn, new TransmitQueue.Halted(oldConn.queue, oldConn.currentTime()), oldConn.backendName);
     }
 
-    public final ClientActorContext context() {
+    // This constructor is only to be called (indirectly) by ConnectedClientConnection constructor.
+    // Do not allow subclassing outside of this package
+    AbstractClientConnection(final AbstractClientConnection<T> oldConn, final T newBackend,
+            final int queueDepth) {
+        this(oldConn, new TransmitQueue.Transmitting(oldConn.queue, queueDepth, newBackend, oldConn.currentTime(),
+            requireNonNull(oldConn.context).messageSlicer()), newBackend.getName());
+    }
+
+    public final @NonNull ClientActorContext context() {
         return context;
     }
 
-    public final @Nonnull Long cookie() {
+    public final @NonNull Long cookie() {
         return cookie;
     }
 
-    public final ActorRef localActor() {
+    public final @NonNull ActorRef localActor() {
         return context.self();
     }
 
@@ -146,7 +165,7 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
      *
      * <p>
      * Note that unlike {@link #sendRequest(Request, Consumer)}, this method does not exert backpressure, hence it
-     * should never be called from an application thread.
+     * should never be called from an application thread and serves mostly for moving requests between queues.
      *
      * @param request Request to send
      * @param callback Callback to invoke
@@ -157,24 +176,47 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
         enqueueEntry(new ConnectionEntry(request, callback, enqueuedTicks), currentTime());
     }
 
-    public final long enqueueEntry(final ConnectionEntry entry, final long now) {
+    private long enqueueOrForward(final ConnectionEntry entry, final long now) {
         lock.lock();
         try {
-            final RequestException maybePoison = poisoned;
-            if (maybePoison != null) {
-                throw new IllegalStateException("Connection " + this + " has been poisoned", maybePoison);
-            }
+            commonEnqueue(entry, now);
+            return queue.enqueueOrForward(entry, now);
+        } finally {
+            lock.unlock();
+        }
+    }
 
-            if (queue.isEmpty()) {
-                // The queue is becoming non-empty, schedule a timer.
-                scheduleTimer(entry.getEnqueuedTicks() + REQUEST_TIMEOUT_NANOS - now);
-            }
-            return queue.enqueue(entry, now);
+    /**
+     * Enqueue an entry, possibly also transmitting it.
+     */
+    public final void enqueueEntry(final ConnectionEntry entry, final long now) {
+        lock.lock();
+        try {
+            commonEnqueue(entry, now);
+            queue.enqueueOrReplay(entry, now);
         } finally {
             lock.unlock();
         }
     }
 
+    @Holding("lock")
+    private void commonEnqueue(final ConnectionEntry entry, final long now) {
+        final RequestException maybePoison = poisoned;
+        if (maybePoison != null) {
+            throw new IllegalStateException("Connection " + this + " has been poisoned", maybePoison);
+        }
+
+        if (queue.isEmpty()) {
+            // The queue is becoming non-empty, schedule a timer.
+            scheduleTimer(entry.getEnqueuedTicks() + context.config().getRequestTimeout() - now);
+        }
+    }
+
+    // To be called from ClientActorBehavior on ConnectedClientConnection after entries are replayed.
+    final void cancelDebt() {
+        queue.cancelDebt(currentTime());
+    }
+
     public abstract Optional<T> getBackendInfo();
 
     final Collection<ConnectionEntry> startReplay() {
@@ -182,7 +224,7 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
         return queue.drain();
     }
 
-    @GuardedBy("lock")
+    @Holding("lock")
     final void finishReplay(final ReconnectForwarder forwarder) {
         setForwarder(forwarder);
 
@@ -202,17 +244,17 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
         lock.unlock();
     }
 
-    @GuardedBy("lock")
+    @Holding("lock")
     final void setForwarder(final ReconnectForwarder forwarder) {
         queue.setForwarder(forwarder, currentTime());
     }
 
-    @GuardedBy("lock")
+    @Holding("lock")
     abstract ClientActorBehavior<T> lockedReconnect(ClientActorBehavior<T> current,
             RequestException runtimeRequestException);
 
     final void sendEntry(final ConnectionEntry entry, final long now) {
-        long delay = enqueueEntry(entry, now);
+        long delay = enqueueOrForward(entry, now);
         try {
             if (delay >= DEBUG_DELAY_NANOS) {
                 if (delay > MAX_DELAY_NANOS) {
@@ -228,7 +270,7 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
             TimeUnit.NANOSECONDS.sleep(delay);
         } catch (InterruptedException e) {
             Thread.currentThread().interrupt();
-            LOG.debug("Interrupted after sleeping {}ns", e, currentTime() - now);
+            LOG.debug("Interrupted after sleeping {}ns", currentTime() - now, e);
         }
     }
 
@@ -246,7 +288,7 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
      *
      * @param delay Delay, in nanoseconds
      */
-    @GuardedBy("lock")
+    @Holding("lock")
     private void scheduleTimer(final long delay) {
         if (haveTimer) {
             LOG.debug("{}: timer already scheduled on {}", context.persistenceId(), this);
@@ -259,7 +301,7 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
 
         // If the delay is negative, we need to schedule an action immediately. While the caller could have checked
         // for that condition and take appropriate action, but this is more convenient and less error-prone.
-        final long normalized =  delay <= 0 ? 0 : Math.min(delay, BACKEND_ALIVE_TIMEOUT_NANOS);
+        final long normalized =  delay <= 0 ? 0 : Math.min(delay, context.config().getBackendAlivenessTimerInterval());
 
         final FiniteDuration dur = FiniteDuration.fromNanos(normalized);
         LOG.debug("{}: connection {} scheduling timeout in {}", context.persistenceId(), this, dur);
@@ -269,16 +311,17 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
 
     /**
      * Check this queue for timeout and initiate reconnection if that happened. If the queue has not made progress
-     * in {@link #NO_PROGRESS_TIMEOUT_NANOS} nanoseconds, it will be aborted.
+     * in {@link #DEFAULT_NO_PROGRESS_TIMEOUT_NANOS} nanoseconds, it will be aborted.
      *
      * @param current Current behavior
      * @return Next behavior to use
      */
     @VisibleForTesting
     final ClientActorBehavior<T> runTimer(final ClientActorBehavior<T> current) {
-        final Optional<Long> delay;
-
         lock.lock();
+
+        final List<ConnectionEntry> poisonEntries;
+        final NoProgressException poisonCause;
         try {
             haveTimer = false;
             final long now = currentTime();
@@ -288,41 +331,43 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
             // The following line is only reliable when queue is not forwarding, but such state should not last long.
             // FIXME: BUG-8422: this may not be accurate w.r.t. replayed entries
             final long ticksSinceProgress = queue.ticksStalling(now);
-            if (ticksSinceProgress >= NO_PROGRESS_TIMEOUT_NANOS) {
-                LOG.error("Queue {} has not seen progress in {} seconds, failing all requests", this,
-                    TimeUnit.NANOSECONDS.toSeconds(ticksSinceProgress));
+            if (ticksSinceProgress < context.config().getNoProgressTimeout()) {
+                // Requests are always scheduled in sequence, hence checking for timeout is relatively straightforward.
+                // Note we use also inquire about the delay, so we can re-schedule if needed, hence the unusual
+                // tri-state return convention.
+                final OptionalLong delay = lockedCheckTimeout(now);
+                if (delay == null) {
+                    // We have timed out. There is no point in scheduling a timer
+                    LOG.debug("{}: connection {} timed out", context.persistenceId(), this);
+                    return lockedReconnect(current, new RuntimeRequestException("Backend connection timed out",
+                        new TimeoutException()));
+                }
 
-                lockedPoison(new NoProgressException(ticksSinceProgress));
-                current.removeConnection(this);
-                return current;
-            }
+                if (delay.isPresent()) {
+                    // If there is new delay, schedule a timer
+                    scheduleTimer(delay.orElseThrow());
+                } else {
+                    LOG.debug("{}: not scheduling timeout on {}", context.persistenceId(), this);
+                }
 
-            // Requests are always scheduled in sequence, hence checking for timeout is relatively straightforward.
-            // Note we use also inquire about the delay, so we can re-schedule if needed, hence the unusual tri-state
-            // return convention.
-            delay = lockedCheckTimeout(now);
-            if (delay == null) {
-                // We have timed out. There is no point in scheduling a timer
-                LOG.debug("{}: connection {} timed out", context.persistenceId(), this);
-                return lockedReconnect(current, new RuntimeRequestException("Backend connection timed out",
-                    new TimeoutException()));
+                return current;
             }
 
-            if (delay.isPresent()) {
-                // If there is new delay, schedule a timer
-                scheduleTimer(delay.get());
-            } else {
-                LOG.debug("{}: not scheduling timeout on {}", context.persistenceId(), this);
-            }
+            LOG.error("Queue {} has not seen progress in {} seconds, failing all requests", this,
+                TimeUnit.NANOSECONDS.toSeconds(ticksSinceProgress));
+            poisonCause = new NoProgressException(ticksSinceProgress);
+            poisonEntries = lockedPoison(poisonCause);
+            current.removeConnection(this);
         } finally {
             lock.unlock();
         }
 
+        poison(poisonEntries, poisonCause);
         return current;
     }
 
     @VisibleForTesting
-    final Optional<Long> checkTimeout(final long now) {
+    final OptionalLong checkTimeout(final long now) {
         lock.lock();
         try {
             return lockedCheckTimeout(now);
@@ -344,14 +389,14 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
     @SuppressFBWarnings(value = "NP_OPTIONAL_RETURN_NULL",
             justification = "Returning null Optional is documented in the API contract.")
     @GuardedBy("lock")
-    private Optional<Long> lockedCheckTimeout(final long now) {
+    private OptionalLong lockedCheckTimeout(final long now) {
         if (queue.isEmpty()) {
             LOG.debug("{}: connection {} is empty", context.persistenceId(), this);
-            return Optional.empty();
+            return OptionalLong.empty();
         }
 
         final long backendSilentTicks = backendSilentTicks(now);
-        if (backendSilentTicks >= BACKEND_ALIVE_TIMEOUT_NANOS) {
+        if (backendSilentTicks >= context.config().getBackendAlivenessTimerInterval()) {
             LOG.debug("{}: Connection {} has not seen activity from backend for {} nanoseconds, timing out",
                 context.persistenceId(), this, backendSilentTicks);
             return null;
@@ -360,15 +405,16 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
         int tasksTimedOut = 0;
         for (ConnectionEntry head = queue.peek(); head != null; head = queue.peek()) {
             final long beenOpen = now - head.getEnqueuedTicks();
-            if (beenOpen < REQUEST_TIMEOUT_NANOS) {
-                return Optional.of(REQUEST_TIMEOUT_NANOS - beenOpen);
+            final long requestTimeout = context.config().getRequestTimeout();
+            if (beenOpen < requestTimeout) {
+                return OptionalLong.of(requestTimeout - beenOpen);
             }
 
             tasksTimedOut++;
             queue.remove(now);
             LOG.debug("{}: Connection {} timed out entry {}", context.persistenceId(), this, head);
-            head.complete(head.getRequest().toRequestFailure(
-                new RequestTimeoutException("Timed out after " + beenOpen + "ns")));
+
+            timeoutEntry(head, beenOpen);
         }
 
         LOG.debug("Connection {} timed out {} tasks", this, tasksTimedOut);
@@ -376,22 +422,49 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
             queue.tryTransmit(now);
         }
 
-        return Optional.empty();
+        return OptionalLong.empty();
+    }
+
+    private void timeoutEntry(final ConnectionEntry entry, final long beenOpen) {
+        // Timeouts needs to be re-scheduled on actor thread because we are holding the lock on the current queue,
+        // which may be the tail of a successor chain. This is a problem if the callback attempts to send a request
+        // because that will attempt to lock the chain from the start, potentially causing a deadlock if there is
+        // a concurrent attempt to transmit.
+        context.executeInActor(current -> {
+            final double time = beenOpen * 1.0 / 1_000_000_000;
+            entry.complete(entry.getRequest().toRequestFailure(
+                new RequestTimeoutException(entry.getRequest() + " timed out after " + time
+                        + " seconds. The backend for " + backendName + " is not available.")));
+            return current;
+        });
     }
 
     final void poison(final RequestException cause) {
+        final List<ConnectionEntry> entries;
+
         lock.lock();
         try {
-            lockedPoison(cause);
+            entries = lockedPoison(cause);
         } finally {
             lock.unlock();
         }
+
+        poison(entries, cause);
     }
 
-    @GuardedBy("lock")
-    private void lockedPoison(final RequestException cause) {
+    // Do not hold any locks while calling this
+    private static void poison(final Collection<? extends ConnectionEntry> entries, final RequestException cause) {
+        for (ConnectionEntry e : entries) {
+            final Request<?, ?> request = e.getRequest();
+            LOG.trace("Poisoning request {}", request, cause);
+            e.complete(request.toRequestFailure(cause));
+        }
+    }
+
+    @Holding("lock")
+    private List<ConnectionEntry> lockedPoison(final RequestException cause) {
         poisoned = enrichPoison(cause);
-        queue.poison(cause);
+        return queue.poison();
     }
 
     RequestException enrichPoison(final RequestException ex) {
@@ -403,7 +476,7 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
         return poisoned;
     }
 
-    final void receiveResponse(final ResponseEnvelope<?> envelope) {
+    void receiveResponse(final ResponseEnvelope<?> envelope) {
         final long now = currentTime();
         lastReceivedTicks = now;
 
@@ -416,7 +489,7 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
         }
 
         if (maybeEntry.isPresent()) {
-            final TransmittedConnectionEntry entry = maybeEntry.get();
+            final TransmittedConnectionEntry entry = maybeEntry.orElseThrow();
             LOG.debug("Completing {} with {}", entry, envelope);
             entry.complete(envelope.getMessage());
         }