BUG-5280: expose queue messages during reconnect
[controller.git] / opendaylight / md-sal / cds-access-client / src / main / java / org / opendaylight / controller / cluster / access / client / AbstractClientConnection.java
index bade34cb2f81f29bcf6aebfe9897dbd722c5916c..0366e7ace2496c0f5edd5433f237ca09db569a01 100644 (file)
@@ -10,20 +10,16 @@ package org.opendaylight.controller.cluster.access.client;
 import akka.actor.ActorRef;
 import com.google.common.annotations.VisibleForTesting;
 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.Iterator;
-import java.util.Map.Entry;
 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;
 import javax.annotation.concurrent.NotThreadSafe;
 import org.opendaylight.controller.cluster.access.concepts.Request;
-import org.opendaylight.controller.cluster.access.concepts.RequestEnvelope;
 import org.opendaylight.controller.cluster.access.concepts.RequestException;
 import org.opendaylight.controller.cluster.access.concepts.Response;
 import org.opendaylight.controller.cluster.access.concepts.ResponseEnvelope;
@@ -48,33 +44,30 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
     @VisibleForTesting
     static final long REQUEST_TIMEOUT_NANOS = TimeUnit.SECONDS.toNanos(30);
 
-    private final Queue<TransmittedConnectionEntry> inflight;
-    private final Queue<ConnectionEntry> pending;
-
+    private final Lock lock = new ReentrantLock();
     private final ClientActorContext context;
+    @GuardedBy("lock")
+    private final TransmitQueue queue;
     private final Long cookie;
 
-    private volatile ReconnectForwarder successor;
     private volatile RequestException poisoned;
     private long lastProgress;
 
-    private AbstractClientConnection(final ClientActorContext context, final Long cookie,
-            final Queue<TransmittedConnectionEntry> inflight, 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.inflight = Preconditions.checkNotNull(inflight);
-        this.pending = Preconditions.checkNotNull(pending);
+        this.queue = Preconditions.checkNotNull(queue);
         this.lastProgress = readTime();
     }
 
-    // Do not allow subclassing outside of this package
-    AbstractClientConnection(final ClientActorContext context, final Long cookie) {
-        this(context, cookie, new ArrayDeque<>(), new ArrayDeque<>(1));
-    }
-
     // Do not allow subclassing outside of this package
     AbstractClientConnection(final AbstractClientConnection<T> oldConnection) {
-        this(oldConnection.context, oldConnection.cookie, oldConnection.inflight, oldConnection.pending);
+        this.context = oldConnection.context;
+        this.cookie = oldConnection.cookie;
+        this.lastProgress = oldConnection.lastProgress;
+        this.queue = new TransmitQueue.Halted();
     }
 
     public final ClientActorContext context() {
@@ -97,83 +90,52 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
      * @param callback Callback to invoke
      */
     public final void sendRequest(final Request<?, ?> request, final Consumer<Response<?, ?>> callback) {
-        Preconditions.checkState(poisoned == null, "Connection %s has been poisoned", this);
-
-        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;
+        final RequestException maybePoison = poisoned;
+        if (maybePoison != null) {
+            throw new IllegalStateException("Connection " + this + " has been poisoned", maybePoison);
         }
 
-        enqueueEntry(entry);
+        final ConnectionEntry entry = new ConnectionEntry(request, callback, readTime());
 
-        final ReconnectForwarder afterQueue = successor;
-        if (afterQueue != null) {
-            synchronized (this) {
-                spliceToSuccessor(afterQueue);
-            }
+        lock.lock();
+        try {
+            queue.enqueue(entry, entry.getEnqueuedTicks());
+        } finally {
+            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);
-    }
-
     public abstract Optional<T> getBackendInfo();
 
-    abstract ClientActorBehavior<T> reconnectConnection(ClientActorBehavior<T> current);
-
-    abstract int remoteMaxMessages();
-
-    abstract Entry<ActorRef, RequestEnvelope> prepareForTransmit(Request<?, ?> req);
-
-    @GuardedBy("this")
-    final void spliceToSuccessor(final ReconnectForwarder successor) {
-        ConnectionEntry entry = inflight.poll();
-        while (entry != null) {
-            successor.forwardEntry(entry);
-            entry = inflight.poll();
-        }
-
-        entry = pending.poll();
-        while (entry != null) {
-            successor.forwardEntry(entry);
-            entry = pending.poll();
-        }
+    final Iterable<ConnectionEntry> startReplay() {
+        lock.lock();
+        return queue.asIterable();
     }
 
-    private long readTime() {
-        return context.ticker().read();
+    @GuardedBy("lock")
+    final void finishReplay(final ReconnectForwarder forwarder) {
+        queue.setForwarder(forwarder, readTime());
+        lock.unlock();
     }
 
-    private void transmit(final ConnectionEntry entry) {
-        final Entry<ActorRef, RequestEnvelope> tuple = prepareForTransmit(entry.getRequest());
-        final RequestEnvelope req = tuple.getValue();
+    @GuardedBy("lock")
+    final void setForwarder(final ReconnectForwarder forwarder) {
+        queue.setForwarder(forwarder, readTime());
+    }
 
-        // We need to enqueue the request before we send it to the actor, as we may be executing on a different thread
-        // than the client actor thread, in which case the round-trip could be made faster than we can enqueue --
-        // in which case the receive routine would not find the entry.
-        final TransmittedConnectionEntry txEntry = new TransmittedConnectionEntry(entry, req.getSessionId(),
-            req.getTxSequence(), readTime());
-        inflight.add(txEntry);
+    @GuardedBy("lock")
+    abstract ClientActorBehavior<T> reconnectConnection(ClientActorBehavior<T> current);
 
-        final ActorRef actor = tuple.getKey();
-        LOG.trace("Transmitting request {} as {} to {}", entry.getRequest(), req, actor);
-        actor.tell(req, ActorRef.noSender());
+    private long readTime() {
+        return context.ticker().read();
     }
 
-    final void enqueueEntry(final ConnectionEntry entry) {
-        if (inflight.size() < remoteMaxMessages()) {
-            transmit(entry);
-            LOG.debug("Enqueued request {} to queue {}", entry.getRequest(), this);
-        } else {
-            LOG.debug("Queue is at capacity, delayed sending of request {}", entry.getRequest());
-            pending.add(entry);
+    final void enqueueEntry(final ConnectionEntry entry, final long now) {
+        lock.lock();
+        try {
+            queue.enqueue(entry, now);
+        } finally {
+            lock.unlock();
         }
     }
 
@@ -196,27 +158,33 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
      */
     @VisibleForTesting
     final ClientActorBehavior<T> runTimer(final ClientActorBehavior<T> current) {
-        final long now = readTime();
-
-        if (!inflight.isEmpty() || !pending.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));
-
-                poison(new NoProgressException(ticksSinceProgress));
-                current.removeConnection(this);
-                return current;
+        final Optional<FiniteDuration> 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;
+                }
             }
-        }
 
-        // 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 reconnectConnection(current);
+            }
+        } finally {
+            lock.unlock();
         }
 
         if (delay.isPresent()) {
@@ -227,6 +195,16 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
         return current;
     }
 
+    @VisibleForTesting
+    final Optional<FiniteDuration> checkTimeout(final long now) {
+        lock.lock();
+        try {
+            return lockedCheckTimeout(now);
+        } finally {
+            lock.unlock();
+        }
+    }
+
     /*
      * We are using tri-state return here to indicate one of three conditions:
      * - if there is no timeout to schedule, return Optional.empty()
@@ -235,7 +213,9 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
      */
     @SuppressFBWarnings(value = "NP_OPTIONAL_RETURN_NULL",
             justification = "Returning null Optional is documented in the API contract.")
-    private 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();
         }
@@ -249,39 +229,19 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
         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.")
-    @VisibleForTesting
-    final Optional<FiniteDuration> checkTimeout(final long now) {
-        final Optional<FiniteDuration> xmit = checkTimeout(inflight.peek(), now);
-        if (xmit == null) {
-            return null;
-        }
-        final Optional<FiniteDuration> pend = checkTimeout(pending.peek(), now);
-        if (pend == null) {
-            return null;
-        }
-        if (!xmit.isPresent()) {
-            return pend;
-        }
-        if (!pend.isPresent()) {
-            return xmit;
+    final void poison(final RequestException cause) {
+        lock.lock();
+        try {
+            lockedPoison(cause);
+        } finally {
+            lock.unlock();
         }
-
-        return Optional.of(xmit.get().min(pend.get()));
     }
 
-    final void poison(final RequestException cause) {
+    @GuardedBy("lock")
+    private void lockedPoison(final RequestException cause) {
         poisoned = cause;
-
-        poisonQueue(inflight, cause);
-        poisonQueue(pending, cause);
+        queue.poison(cause);
     }
 
     @VisibleForTesting
@@ -290,97 +250,15 @@ public abstract class AbstractClientConnection<T extends BackendInfo> {
     }
 
     final void receiveResponse(final ResponseEnvelope<?> envelope) {
-        Optional<TransmittedConnectionEntry> maybeEntry = findMatchingEntry(inflight, envelope);
-        if (maybeEntry == null) {
-            LOG.debug("Request for {} not found in inflight queue, checking pending queue", envelope);
-            maybeEntry = findMatchingEntry(pending, envelope);
-        }
-
-        if (maybeEntry == null || !maybeEntry.isPresent()) {
-            LOG.warn("No request matching {} found, ignoring response", envelope);
-            return;
-        }
-
-        final TransmittedConnectionEntry entry = maybeEntry.get();
-        LOG.debug("Completing {} with {}", entry, envelope);
-        entry.complete(envelope.getMessage());
-
-        // We have freed up a slot, try to transmit something
-        int toSend = remoteMaxMessages() - inflight.size();
-        while (toSend > 0) {
-            final ConnectionEntry e = pending.poll();
-            if (e == null) {
-                break;
-            }
+        final long now = readTime();
 
-            LOG.debug("Transmitting entry {}", e);
-            transmit(e);
-            toSend--;
+        lock.lock();
+        try {
+            queue.complete(envelope, now);
+        } finally {
+            lock.unlock();
         }
 
         lastProgress = readTime();
     }
-
-    private 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));
-        }
-        queue.clear();
-    }
-
-    /*
-     * We are using tri-state return here to indicate one of three conditions:
-     * - if a matching entry is found, return an Optional containing it
-     * - if a matching entry is not found, but it makes sense to keep looking at other queues, return null
-     * - if a conflicting entry is encountered, indicating we should ignore this request, return an empty Optional
-     */
-    @SuppressFBWarnings(value = "NP_OPTIONAL_RETURN_NULL",
-            justification = "Returning null Optional is documented in the API contract.")
-    private static Optional<TransmittedConnectionEntry> findMatchingEntry(final Queue<? extends ConnectionEntry> queue,
-            final ResponseEnvelope<?> envelope) {
-        // Try to find the request in a queue. Responses may legally come back in a different order, hence we need
-        // to use an iterator
-        final Iterator<? extends ConnectionEntry> it = queue.iterator();
-        while (it.hasNext()) {
-            final ConnectionEntry e = it.next();
-            final Request<?, ?> request = e.getRequest();
-            final Response<?, ?> response = envelope.getMessage();
-
-            // First check for matching target, or move to next entry
-            if (!request.getTarget().equals(response.getTarget())) {
-                continue;
-            }
-
-            // Sanity-check logical sequence, ignore any out-of-order messages
-            if (request.getSequence() != response.getSequence()) {
-                LOG.debug("Expecting sequence {}, ignoring response {}", request.getSequence(), envelope);
-                return Optional.empty();
-            }
-
-            // Check if the entry has (ever) been transmitted
-            if (!(e instanceof TransmittedConnectionEntry)) {
-                return Optional.empty();
-            }
-
-            final TransmittedConnectionEntry te = (TransmittedConnectionEntry) e;
-
-            // Now check session match
-            if (envelope.getSessionId() != te.getSessionId()) {
-                LOG.debug("Expecting session {}, ignoring response {}", te.getSessionId(), envelope);
-                return Optional.empty();
-            }
-            if (envelope.getTxSequence() != te.getTxSequence()) {
-                LOG.warn("Expecting txSequence {}, ignoring response {}", te.getTxSequence(), envelope);
-                return Optional.empty();
-            }
-
-            LOG.debug("Completing request {} with {}", request, envelope);
-            it.remove();
-            return Optional.of(te);
-        }
-
-        return null;
-    }
 }