Slice front-end request messages
[controller.git] / opendaylight / md-sal / cds-access-client / src / main / java / org / opendaylight / controller / cluster / access / client / AbstractClientConnection.java
index 0366e7ace2496c0f5edd5433f237ca09db569a01..afee418fd6926e55abbafac2a4b3d6a120a58f67 100644 (file)
@@ -9,10 +9,14 @@ package org.opendaylight.controller.cluster.access.client;
 
 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.Optional;
 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;
@@ -23,6 +27,7 @@ import org.opendaylight.controller.cluster.access.concepts.Request;
 import org.opendaylight.controller.cluster.access.concepts.RequestException;
 import org.opendaylight.controller.cluster.access.concepts.Response;
 import org.opendaylight.controller.cluster.access.concepts.ResponseEnvelope;
+import org.opendaylight.controller.cluster.access.concepts.RuntimeRequestException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.concurrent.duration.FiniteDuration;
@@ -38,11 +43,35 @@ import scala.concurrent.duration.FiniteDuration;
 public abstract class AbstractClientConnection<T extends BackendInfo> {
     private static final Logger LOG = LoggerFactory.getLogger(AbstractClientConnection.class);
 
-    // Keep these constants in nanoseconds, as that prevents unnecessary conversions in the fast path
-    @VisibleForTesting
-    static final long NO_PROGRESS_TIMEOUT_NANOS = TimeUnit.MINUTES.toNanos(15);
-    @VisibleForTesting
-    static final long REQUEST_TIMEOUT_NANOS = TimeUnit.SECONDS.toNanos(30);
+    /*
+     * Timers involved in communication with the backend. There are three tiers which are spaced out to allow for
+     * recovery at each tier. Keep these constants in nanoseconds, as that prevents unnecessary conversions in the fast
+     * path.
+     */
+    /**
+     * 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 attempt to reconnect
+     * it.
+     */
+    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.
+     */
+    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.
+     */
+    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);
+
+    // Upper bound on the time a thread is forced to sleep to keep queue size under control
+    private static final long MAX_DELAY_SECONDS = 5;
+    private static final long MAX_DELAY_NANOS = TimeUnit.SECONDS.toNanos(MAX_DELAY_SECONDS);
 
     private final Lock lock = new ReentrantLock();
     private final ClientActorContext context;
@@ -50,24 +79,45 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
     private final TransmitQueue queue;
     private final Long cookie;
 
+    @GuardedBy("lock")
+    private boolean haveTimer;
+
+    /**
+     * Time reference when we saw any activity from the backend.
+     */
+    private long lastReceivedTicks;
+
     private volatile RequestException poisoned;
-    private long lastProgress;
 
+    // Private constructor to avoid code duplication.
+    private AbstractClientConnection(final AbstractClientConnection<T> oldConn, final TransmitQueue newQueue) {
+        this.context = Preconditions.checkNotNull(oldConn.context);
+        this.cookie = Preconditions.checkNotNull(oldConn.cookie);
+        this.queue = Preconditions.checkNotNull(newQueue);
+        // Will be updated in finishReplay if needed.
+        this.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) {
+    AbstractClientConnection(final ClientActorContext context, final Long cookie, final int queueDepth) {
         this.context = Preconditions.checkNotNull(context);
         this.cookie = Preconditions.checkNotNull(cookie);
-        this.queue = Preconditions.checkNotNull(queue);
-        this.lastProgress = readTime();
+        this.queue = new TransmitQueue.Halted(queueDepth);
+        this.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> oldConn) {
+        this(oldConn, new TransmitQueue.Halted(oldConn.queue, oldConn.currentTime()));
     }
 
+    // This constructor is only to be called (indirectly) by ConnectedClientConnection constructor.
     // Do not allow subclassing outside of this package
-    AbstractClientConnection(final AbstractClientConnection<T> oldConnection) {
-        this.context = oldConnection.context;
-        this.cookie = oldConnection.cookie;
-        this.lastProgress = oldConnection.lastProgress;
-        this.queue = new TransmitQueue.Halted();
+    AbstractClientConnection(final AbstractClientConnection<T> oldConn, final T newBackend, final int queueDepth) {
+        this(oldConn, new TransmitQueue.Transmitting(oldConn.queue, queueDepth, newBackend, oldConn.currentTime(),
+                Preconditions.checkNotNull(oldConn.context).messageSlicer()));
     }
 
     public final ClientActorContext context() {
@@ -82,58 +132,144 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
         return context.self();
     }
 
+    public final long currentTime() {
+        return context.ticker().read();
+    }
+
     /**
      * Send a request to the backend and invoke a specified callback when it finishes. This method is safe to invoke
      * from any thread.
      *
+     * <p>This method may put the caller thread to sleep in order to throttle the request rate.
+     * The callback may be called before the sleep finishes.
+     *
      * @param request Request to send
      * @param callback Callback to invoke
      */
     public final void sendRequest(final Request<?, ?> request, final Consumer<Response<?, ?>> callback) {
-        final RequestException maybePoison = poisoned;
-        if (maybePoison != null) {
-            throw new IllegalStateException("Connection " + this + " has been poisoned", maybePoison);
-        }
+        final long now = currentTime();
+        sendEntry(new ConnectionEntry(request, callback, now), now);
+    }
 
-        final ConnectionEntry entry = new ConnectionEntry(request, callback, readTime());
+    /**
+     * Send a request to the backend and invoke a specified callback when it finishes. This method is safe to invoke
+     * from any thread.
+     *
+     * <p>
+     * Note that unlike {@link #sendRequest(Request, Consumer)}, this method does not exert backpressure, hence it
+     * should never be called from an application thread.
+     *
+     * @param request Request to send
+     * @param callback Callback to invoke
+     * @param enqueuedTicks Time (according to {@link #currentTime()} of request enqueue
+     */
+    public final void enqueueRequest(final Request<?, ?> request, final Consumer<Response<?, ?>> callback,
+            final long enqueuedTicks) {
+        enqueueEntry(new ConnectionEntry(request, callback, enqueuedTicks), currentTime());
+    }
 
+    private long enqueueOrForward(final ConnectionEntry entry, final long now) {
         lock.lock();
         try {
-            queue.enqueue(entry, entry.getEnqueuedTicks());
+            commonEnqueue(entry, now);
+            return queue.enqueueOrForward(entry, now);
         } finally {
             lock.unlock();
         }
     }
 
+    /**
+     * 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();
+        }
+    }
+
+    @GuardedBy("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 Iterable<ConnectionEntry> startReplay() {
+    final Collection<ConnectionEntry> startReplay() {
         lock.lock();
-        return queue.asIterable();
+        return queue.drain();
     }
 
     @GuardedBy("lock")
     final void finishReplay(final ReconnectForwarder forwarder) {
-        queue.setForwarder(forwarder, readTime());
+        setForwarder(forwarder);
+
+        /*
+         * The process of replaying all messages may have taken a significant chunk of time, depending on type
+         * of messages, queue depth and available processing power. In extreme situations this may have already
+         * exceeded BACKEND_ALIVE_TIMEOUT_NANOS, in which case we are running the risk of not making reasonable forward
+         * progress before we start a reconnect cycle.
+         *
+         * Note that the timer is armed after we have sent the first message, hence we should be seeing a response
+         * from the backend before we see a timeout, simply due to how the mailbox operates.
+         *
+         * At any rate, reset the timestamp once we complete reconnection (which an atomic transition from the
+         * perspective of outside world), as that makes it a bit easier to reason about timing of events.
+         */
+        lastReceivedTicks = currentTime();
         lock.unlock();
     }
 
     @GuardedBy("lock")
     final void setForwarder(final ReconnectForwarder forwarder) {
-        queue.setForwarder(forwarder, readTime());
+        queue.setForwarder(forwarder, currentTime());
     }
 
     @GuardedBy("lock")
-    abstract ClientActorBehavior<T> reconnectConnection(ClientActorBehavior<T> current);
+    abstract ClientActorBehavior<T> lockedReconnect(ClientActorBehavior<T> current,
+            RequestException runtimeRequestException);
 
-    private long readTime() {
-        return context.ticker().read();
+    final void sendEntry(final ConnectionEntry entry, final long now) {
+        long delay = enqueueOrForward(entry, now);
+        try {
+            if (delay >= DEBUG_DELAY_NANOS) {
+                if (delay > MAX_DELAY_NANOS) {
+                    LOG.info("Capping {} throttle delay from {} to {} seconds", this,
+                        TimeUnit.NANOSECONDS.toSeconds(delay), MAX_DELAY_SECONDS, new Throwable());
+                    delay = MAX_DELAY_NANOS;
+                }
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("{}: Sleeping for {}ms on connection {}", context.persistenceId(),
+                        TimeUnit.NANOSECONDS.toMillis(delay), this);
+                }
+            }
+            TimeUnit.NANOSECONDS.sleep(delay);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            LOG.debug("Interrupted after sleeping {}ns", e, currentTime() - now);
+        }
     }
 
-    final void enqueueEntry(final ConnectionEntry entry, final long now) {
+    final ClientActorBehavior<T> reconnect(final ClientActorBehavior<T> current, final RequestException cause) {
         lock.lock();
         try {
-            queue.enqueue(entry, now);
+            return lockedReconnect(current, cause);
         } finally {
             lock.unlock();
         }
@@ -144,35 +280,55 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
      *
      * @param delay Delay, in nanoseconds
      */
-    private void scheduleTimer(final FiniteDuration delay) {
-        LOG.debug("{}: scheduling timeout in {}", context.persistenceId(), delay);
-        context.executeInActor(this::runTimer, delay);
+    @GuardedBy("lock")
+    private void scheduleTimer(final long delay) {
+        if (haveTimer) {
+            LOG.debug("{}: timer already scheduled on {}", context.persistenceId(), this);
+            return;
+        }
+        if (queue.hasSuccessor()) {
+            LOG.debug("{}: connection {} has a successor, not scheduling timer", context.persistenceId(), this);
+            return;
+        }
+
+        // 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, context.config().getBackendAlivenessTimerInterval());
+
+        final FiniteDuration dur = FiniteDuration.fromNanos(normalized);
+        LOG.debug("{}: connection {} scheduling timeout in {}", context.persistenceId(), this, dur);
+        context.executeInActor(this::runTimer, dur);
+        haveTimer = true;
     }
 
     /**
      * 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<FiniteDuration> delay;
+        final Optional<Long> delay;
 
         lock.lock();
         try {
-            final long now = readTime();
-            if (!queue.isEmpty()) {
-                final long ticksSinceProgress = now - lastProgress;
-                if (ticksSinceProgress >= NO_PROGRESS_TIMEOUT_NANOS) {
-                    LOG.error("Queue {} has not seen progress in {} seconds, failing all requests", this,
-                        TimeUnit.NANOSECONDS.toSeconds(ticksSinceProgress));
-
-                    lockedPoison(new NoProgressException(ticksSinceProgress));
-                    current.removeConnection(this);
-                    return current;
-                }
+            haveTimer = false;
+            final long now = currentTime();
+
+            LOG.debug("{}: running timer on {}", context.persistenceId(), this);
+
+            // 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 >= context.config().getNoProgressTimeout()) {
+                LOG.error("Queue {} has not seen progress in {} seconds, failing all requests", this,
+                    TimeUnit.NANOSECONDS.toSeconds(ticksSinceProgress));
+
+                lockedPoison(new NoProgressException(ticksSinceProgress));
+                current.removeConnection(this);
+                return current;
             }
 
             // Requests are always scheduled in sequence, hence checking for timeout is relatively straightforward.
@@ -181,22 +337,26 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
             delay = lockedCheckTimeout(now);
             if (delay == null) {
                 // We have timed out. There is no point in scheduling a timer
-                return reconnectConnection(current);
+                LOG.debug("{}: connection {} timed out", context.persistenceId(), this);
+                return lockedReconnect(current, new RuntimeRequestException("Backend connection timed out",
+                    new TimeoutException()));
+            }
+
+            if (delay.isPresent()) {
+                // If there is new delay, schedule a timer
+                scheduleTimer(delay.get());
+            } else {
+                LOG.debug("{}: not scheduling timeout on {}", context.persistenceId(), this);
             }
         } finally {
             lock.unlock();
         }
 
-        if (delay.isPresent()) {
-            // If there is new delay, schedule a timer
-            scheduleTimer(delay.get());
-        }
-
         return current;
     }
 
     @VisibleForTesting
-    final Optional<FiniteDuration> checkTimeout(final long now) {
+    final Optional<Long> checkTimeout(final long now) {
         lock.lock();
         try {
             return lockedCheckTimeout(now);
@@ -205,6 +365,10 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
         }
     }
 
+    long backendSilentTicks(final long now) {
+        return now - lastReceivedTicks;
+    }
+
     /*
      * We are using tri-state return here to indicate one of three conditions:
      * - if there is no timeout to schedule, return Optional.empty()
@@ -214,19 +378,42 @@ 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<FiniteDuration> lockedCheckTimeout(final long now) {
-        final ConnectionEntry head = queue.peek();
-        if (head == null) {
+    private Optional<Long> lockedCheckTimeout(final long now) {
+        if (queue.isEmpty()) {
+            LOG.debug("{}: connection {} is empty", context.persistenceId(), this);
             return Optional.empty();
         }
 
-        final long delay = head.getEnqueuedTicks() - now + REQUEST_TIMEOUT_NANOS;
-        if (delay <= 0) {
-            LOG.debug("Connection {} timed out", this);
+        final long backendSilentTicks = backendSilentTicks(now);
+        if (backendSilentTicks >= context.config().getBackendAlivenessTimerInterval()) {
+            LOG.debug("{}: Connection {} has not seen activity from backend for {} nanoseconds, timing out",
+                context.persistenceId(), this, backendSilentTicks);
             return null;
         }
 
-        return Optional.of(FiniteDuration.apply(delay, TimeUnit.NANOSECONDS));
+        int tasksTimedOut = 0;
+        for (ConnectionEntry head = queue.peek(); head != null; head = queue.peek()) {
+            final long beenOpen = now - head.getEnqueuedTicks();
+            final long requestTimeout = context.config().getRequestTimeout();
+            if (beenOpen < requestTimeout) {
+                return Optional.of(requestTimeout - beenOpen);
+            }
+
+            tasksTimedOut++;
+            queue.remove(now);
+            LOG.debug("{}: Connection {} timed out entry {}", context.persistenceId(), this, head);
+
+            final double time = beenOpen * 1.0 / 1_000_000_000;
+            head.complete(head.getRequest().toRequestFailure(
+                new RequestTimeoutException("Timed out after " + time + "seconds")));
+        }
+
+        LOG.debug("Connection {} timed out {} tasks", this, tasksTimedOut);
+        if (tasksTimedOut != 0) {
+            queue.tryTransmit(now);
+        }
+
+        return Optional.empty();
     }
 
     final void poison(final RequestException cause) {
@@ -240,25 +427,44 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
 
     @GuardedBy("lock")
     private void lockedPoison(final RequestException cause) {
-        poisoned = cause;
+        poisoned = enrichPoison(cause);
         queue.poison(cause);
     }
 
+    RequestException enrichPoison(final RequestException ex) {
+        return ex;
+    }
+
     @VisibleForTesting
     final RequestException poisoned() {
         return poisoned;
     }
 
-    final void receiveResponse(final ResponseEnvelope<?> envelope) {
-        final long now = readTime();
+    void receiveResponse(final ResponseEnvelope<?> envelope) {
+        final long now = currentTime();
+        lastReceivedTicks = now;
 
+        final Optional<TransmittedConnectionEntry> maybeEntry;
         lock.lock();
         try {
-            queue.complete(envelope, now);
+            maybeEntry = queue.complete(envelope, now);
         } finally {
             lock.unlock();
         }
 
-        lastProgress = readTime();
+        if (maybeEntry.isPresent()) {
+            final TransmittedConnectionEntry entry = maybeEntry.get();
+            LOG.debug("Completing {} with {}", entry, envelope);
+            entry.complete(envelope.getMessage());
+        }
+    }
+
+    @Override
+    public final String toString() {
+        return addToStringAttributes(MoreObjects.toStringHelper(this).omitNullValues()).toString();
+    }
+
+    ToStringHelper addToStringAttributes(final ToStringHelper toStringHelper) {
+        return toStringHelper.add("client", context.getIdentifier()).add("cookie", cookie).add("poisoned", poisoned);
     }
 }