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 6cb89eec1601e5fa901f89e7ac37d7e254698d66..afee418fd6926e55abbafac2a4b3d6a120a58f67 100644 (file)
@@ -50,24 +50,28 @@ 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);
+
+    // 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;
@@ -85,21 +89,35 @@ 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) {
+        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.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> 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()));
+    }
+
+    // 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(),
+                Preconditions.checkNotNull(oldConn.context).messageSlicer()));
     }
 
     public final ClientActorContext context() {
@@ -130,13 +148,7 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
      */
     public final void sendRequest(final Request<?, ?> request, final Consumer<Response<?, ?>> callback) {
         final long now = currentTime();
-        final long delay = enqueueEntry(new ConnectionEntry(request, callback, now), now);
-        try {
-            TimeUnit.NANOSECONDS.sleep(delay);
-        } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            LOG.debug("Interrupted after sleeping {}ns", e, currentTime() - now);
-        }
+        sendEntry(new ConnectionEntry(request, callback, now), now);
     }
 
     /**
@@ -156,6 +168,47 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
         enqueueEntry(new ConnectionEntry(request, callback, enqueuedTicks), currentTime());
     }
 
+    private long enqueueOrForward(final ConnectionEntry entry, final long now) {
+        lock.lock();
+        try {
+            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 Collection<ConnectionEntry> startReplay() {
@@ -166,6 +219,20 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
     @GuardedBy("lock")
     final void finishReplay(final ReconnectForwarder forwarder) {
         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();
     }
 
@@ -178,21 +245,24 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
     abstract ClientActorBehavior<T> lockedReconnect(ClientActorBehavior<T> current,
             RequestException runtimeRequestException);
 
-    final long enqueueEntry(final ConnectionEntry entry, final long now) {
-        lock.lock();
+    final void sendEntry(final ConnectionEntry entry, final long now) {
+        long delay = enqueueOrForward(entry, now);
         try {
-            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() + REQUEST_TIMEOUT_NANOS - now);
+            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);
+                }
             }
-            return queue.enqueue(entry, now);
-        } finally {
-            lock.unlock();
+            TimeUnit.NANOSECONDS.sleep(delay);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            LOG.debug("Interrupted after sleeping {}ns", e, currentTime() - now);
         }
     }
 
@@ -213,27 +283,27 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
     @GuardedBy("lock")
     private void scheduleTimer(final long delay) {
         if (haveTimer) {
-            LOG.debug("{}: timer already scheduled", context.persistenceId());
+            LOG.debug("{}: timer already scheduled on {}", context.persistenceId(), this);
             return;
         }
         if (queue.hasSuccessor()) {
-            LOG.debug("{}: connection has successor, not scheduling timer", context.persistenceId());
+            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, BACKEND_ALIVE_TIMEOUT_NANOS);
+        final long normalized =  delay <= 0 ? 0 : Math.min(delay, context.config().getBackendAlivenessTimerInterval());
 
         final FiniteDuration dur = FiniteDuration.fromNanos(normalized);
-        LOG.debug("{}: scheduling timeout in {}", context.persistenceId(), dur);
+        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
@@ -246,10 +316,13 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
         try {
             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 >= NO_PROGRESS_TIMEOUT_NANOS) {
+            if (ticksSinceProgress >= context.config().getNoProgressTimeout()) {
                 LOG.error("Queue {} has not seen progress in {} seconds, failing all requests", this,
                     TimeUnit.NANOSECONDS.toSeconds(ticksSinceProgress));
 
@@ -264,6 +337,7 @@ 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
+                LOG.debug("{}: connection {} timed out", context.persistenceId(), this);
                 return lockedReconnect(current, new RuntimeRequestException("Backend connection timed out",
                     new TimeoutException()));
             }
@@ -271,6 +345,8 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
             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();
@@ -304,28 +380,32 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
     @GuardedBy("lock")
     private Optional<Long> lockedCheckTimeout(final long now) {
         if (queue.isEmpty()) {
+            LOG.debug("{}: connection {} is empty", context.persistenceId(), this);
             return Optional.empty();
         }
 
         final long backendSilentTicks = backendSilentTicks(now);
-        if (backendSilentTicks >= BACKEND_ALIVE_TIMEOUT_NANOS) {
-            LOG.debug("Connection {} has not seen activity from backend for {} nanoseconds, timing out", this,
-                backendSilentTicks);
+        if (backendSilentTicks >= context.config().getBackendAlivenessTimerInterval()) {
+            LOG.debug("{}: Connection {} has not seen activity from backend for {} nanoseconds, timing out",
+                context.persistenceId(), this, backendSilentTicks);
             return null;
         }
 
         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 Optional.of(requestTimeout - beenOpen);
             }
 
             tasksTimedOut++;
             queue.remove(now);
-            LOG.debug("Connection {} timed out entryt {}", this, head);
+            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 " + beenOpen + "ns")));
+                new RequestTimeoutException("Timed out after " + time + "seconds")));
         }
 
         LOG.debug("Connection {} timed out {} tasks", this, tasksTimedOut);
@@ -360,7 +440,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;