BUG-8422: Propagate enqueue time
[controller.git] / opendaylight / md-sal / cds-access-client / src / main / java / org / opendaylight / controller / cluster / access / client / AbstractClientConnection.java
index 170b1507a91254a6c7c13e2297158a6914e62b59..d37893bd7c3a9862ad89fb9d88b94337d0355f13 100644 (file)
@@ -9,13 +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 com.google.common.base.Verify;
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-import java.util.ArrayDeque;
 import java.util.Optional;
-import java.util.Queue;
 import java.util.concurrent.TimeUnit;
+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;
@@ -45,30 +46,33 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
     @VisibleForTesting
     static final long REQUEST_TIMEOUT_NANOS = TimeUnit.SECONDS.toNanos(30);
 
-    private final Queue<ConnectionEntry> pending;
+    private static final FiniteDuration REQUEST_TIMEOUT_DURATION = FiniteDuration.apply(REQUEST_TIMEOUT_NANOS,
+        TimeUnit.NANOSECONDS);
+
+    private final Lock lock = new ReentrantLock();
     private final ClientActorContext context;
+    @GuardedBy("lock")
+    private final TransmitQueue queue;
     private final Long cookie;
 
-    private volatile ReconnectForwarder successor;
+    @GuardedBy("lock")
+    private boolean haveTimer;
+
     private volatile RequestException poisoned;
-    private long lastProgress;
 
-    private AbstractClientConnection(final ClientActorContext context, final Long cookie,
-            final Queue<ConnectionEntry> pending) {
+    // 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.pending = Preconditions.checkNotNull(pending);
-        this.lastProgress = readTime();
-    }
-
-    // Do not allow subclassing outside of this package
-    AbstractClientConnection(final ClientActorContext context, final Long cookie) {
-        this(context, cookie, new ArrayDeque<>(1));
+        this.queue = Preconditions.checkNotNull(queue);
     }
 
     // Do not allow subclassing outside of this package
-    AbstractClientConnection(final AbstractClientConnection<T> oldConnection) {
-        this(oldConnection.context, oldConnection.cookie, oldConnection.pending);
+    AbstractClientConnection(final AbstractClientConnection<T> oldConnection, final int targetQueueSize) {
+        this.context = oldConnection.context;
+        this.cookie = oldConnection.cookie;
+        this.queue = new TransmitQueue.Halted(targetQueueSize);
     }
 
     public final ClientActorContext context() {
@@ -83,67 +87,94 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
         return context.self();
     }
 
-    final long readTime() {
+    public final long currentTime() {
         return context.ticker().read();
     }
 
-    final Queue<ConnectionEntry> pending() {
-        return pending;
+    /**
+     * 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 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);
+        }
     }
 
     /**
      * 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 sendRequest(final Request<?, ?> request, final Consumer<Response<?, ?>> callback) {
-        Preconditions.checkState(poisoned == null, "Connection %s has been poisoned", this);
+    public final void enqueueRequest(final Request<?, ?> request, final Consumer<Response<?, ?>> callback,
+            final long enqueuedTicks) {
+        enqueueEntry(new ConnectionEntry(request, callback, enqueuedTicks), currentTime());
+    }
 
-        final ReconnectForwarder beforeQueue = successor;
-        final ConnectionEntry entry = new ConnectionEntry(request, callback, readTime());
-        if (beforeQueue != null) {
-            LOG.trace("Forwarding entry {} from {} to {}", entry, this, beforeQueue);
-            beforeQueue.forwardEntry(entry);
-            return;
-        }
+    public abstract Optional<T> getBackendInfo();
 
-        enqueueEntry(entry);
+    final Iterable<ConnectionEntry> startReplay() {
+        lock.lock();
+        return queue.asIterable();
+    }
 
-        final ReconnectForwarder afterQueue = successor;
-        if (afterQueue != null) {
-            synchronized (this) {
-                spliceToSuccessor(afterQueue);
-            }
-        }
+    @GuardedBy("lock")
+    final void finishReplay(final ReconnectForwarder forwarder) {
+        setForwarder(forwarder);
+        lock.unlock();
     }
 
-    public final synchronized void setForwarder(final ReconnectForwarder forwarder) {
-        Verify.verify(successor == null, "Successor {} already set on connection {}", successor, this);
-        successor = Preconditions.checkNotNull(forwarder);
-        LOG.debug("Connection {} superseded by {}, splicing queue", this, successor);
-        spliceToSuccessor(forwarder);
+    @GuardedBy("lock")
+    final void setForwarder(final ReconnectForwarder forwarder) {
+        queue.setForwarder(forwarder, currentTime());
     }
 
-    public abstract Optional<T> getBackendInfo();
+    @GuardedBy("lock")
+    abstract ClientActorBehavior<T> lockedReconnect(ClientActorBehavior<T> current);
 
-    @GuardedBy("this")
-    void spliceToSuccessor(final ReconnectForwarder successor) {
-        ConnectionEntry entry = pending.poll();
-        while (entry != null) {
-            successor.forwardEntry(entry);
-            entry = pending.poll();
-        }
-    }
+    final long enqueueEntry(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);
+            }
 
-    final ConnectionEntry dequeEntry() {
-        lastProgress = readTime();
-        return pending.poll();
+            if (queue.isEmpty()) {
+                // The queue is becoming non-empty, schedule a timer
+                scheduleTimer(REQUEST_TIMEOUT_DURATION);
+            }
+            return queue.enqueue(entry, now);
+        } finally {
+            lock.unlock();
+        }
     }
 
-    void enqueueEntry(final ConnectionEntry entry) {
-        pending.add(entry);
+    final ClientActorBehavior<T> reconnect(final ClientActorBehavior<T> current) {
+        lock.lock();
+        try {
+            return lockedReconnect(current);
+        } finally {
+            lock.unlock();
+        }
     }
 
     /**
@@ -151,53 +182,75 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
      *
      * @param delay Delay, in nanoseconds
      */
+    @GuardedBy("lock")
     private void scheduleTimer(final FiniteDuration delay) {
+        if (haveTimer) {
+            LOG.debug("{}: timer already scheduled", context.persistenceId());
+            return;
+        }
+        if (queue.hasSuccessor()) {
+            LOG.debug("{}: connection has successor, not scheduling timer", context.persistenceId());
+            return;
+        }
         LOG.debug("{}: scheduling timeout in {}", context.persistenceId(), delay);
         context.executeInActor(this::runTimer, delay);
+        haveTimer = true;
     }
 
     /**
-     * Check queue timeouts and return true if a timeout has occurred.
+     * 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.
      *
-     * @return True if a timeout occurred
-     * @throws NoProgressException if the queue failed to make progress for an extended
-     *                             time.
+     * @param current Current behavior
+     * @return Next behavior to use
      */
     @VisibleForTesting
     final ClientActorBehavior<T> runTimer(final ClientActorBehavior<T> current) {
-        final long now = readTime();
-
-        if (!isEmpty()) {
-            final long ticksSinceProgress = now - lastProgress;
+        final Optional<FiniteDuration> delay;
+
+        lock.lock();
+        try {
+            haveTimer = false;
+            final long now = currentTime();
+            // The following line is only reliable when queue is not forwarding, but such state should not last long.
+            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));
 
-                poison(new NoProgressException(ticksSinceProgress));
+                lockedPoison(new NoProgressException(ticksSinceProgress));
                 current.removeConnection(this);
                 return current;
             }
-        }
 
-        // 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 Optional<FiniteDuration> delay = checkTimeout(now);
-        if (delay == null) {
-            // We have timed out. There is no point in scheduling a timer
-            return reconnectConnection(current);
-        }
+            // 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
+                return lockedReconnect(current);
+            }
 
-        if (delay.isPresent()) {
-            // If there is new delay, schedule a timer
-            scheduleTimer(delay.get());
+            if (delay.isPresent()) {
+                // If there is new delay, schedule a timer
+                scheduleTimer(delay.get());
+            }
+        } finally {
+            lock.unlock();
         }
 
         return current;
     }
 
-    boolean isEmpty() {
-        return pending.isEmpty();
+    @VisibleForTesting
+    final Optional<FiniteDuration> checkTimeout(final long now) {
+        lock.lock();
+        try {
+            return lockedCheckTimeout(now);
+        } finally {
+            lock.unlock();
+        }
     }
 
     /*
@@ -208,44 +261,35 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
      */
     @SuppressFBWarnings(value = "NP_OPTIONAL_RETURN_NULL",
             justification = "Returning null Optional is documented in the API contract.")
-    final Optional<FiniteDuration> checkTimeout(final ConnectionEntry head, final long now) {
+    @GuardedBy("lock")
+    private Optional<FiniteDuration> lockedCheckTimeout(final long now) {
+        final ConnectionEntry head = queue.peek();
         if (head == null) {
             return Optional.empty();
         }
 
-        final long delay = head.getEnqueuedTicks() - now + REQUEST_TIMEOUT_NANOS;
-        if (delay <= 0) {
-            LOG.debug("Connection {} timed out", this);
+        final long beenOpen = now - head.getEnqueuedTicks();
+        if (beenOpen >= REQUEST_TIMEOUT_NANOS) {
+            LOG.debug("Connection {} has a request not completed for {} nanoseconds, timing out", this, beenOpen);
             return null;
         }
 
-        return Optional.of(FiniteDuration.apply(delay, TimeUnit.NANOSECONDS));
-    }
-
-    /*
-     * We are using tri-state return here to indicate one of three conditions:
-     * - if there is no timeout to schedule, return Optional.empty()
-     * - if there is a timeout to schedule, return a non-empty optional
-     * - if this connections has timed out, return null
-     */
-    @SuppressFBWarnings(value = "NP_OPTIONAL_RETURN_NULL",
-            justification = "Returning null Optional is documented in the API contract.")
-    Optional<FiniteDuration> checkTimeout(final long now) {
-        return checkTimeout(pending.peek(), now);
+        return Optional.of(FiniteDuration.apply(REQUEST_TIMEOUT_NANOS - beenOpen, TimeUnit.NANOSECONDS));
     }
 
-    static void poisonQueue(final Queue<? extends ConnectionEntry> queue, final RequestException cause) {
-        for (ConnectionEntry e : queue) {
-            final Request<?, ?> request = e.getRequest();
-            LOG.trace("Poisoning request {}", request, cause);
-            e.complete(request.toRequestFailure(cause));
+    final void poison(final RequestException cause) {
+        lock.lock();
+        try {
+            lockedPoison(cause);
+        } finally {
+            lock.unlock();
         }
-        queue.clear();
     }
 
-    void poison(final RequestException cause) {
+    @GuardedBy("lock")
+    private void lockedPoison(final RequestException cause) {
         poisoned = cause;
-        poisonQueue(pending, cause);
+        queue.poison(cause);
     }
 
     @VisibleForTesting
@@ -253,7 +297,30 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
         return poisoned;
     }
 
-    abstract ClientActorBehavior<T> reconnectConnection(ClientActorBehavior<T> current);
+    final void receiveResponse(final ResponseEnvelope<?> envelope) {
+        final long now = currentTime();
 
-    abstract void receiveResponse(final ResponseEnvelope<?> envelope);
+        final Optional<TransmittedConnectionEntry> maybeEntry;
+        lock.lock();
+        try {
+            maybeEntry = queue.complete(envelope, now);
+        } finally {
+            lock.unlock();
+        }
+
+        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);
+    }
 }