X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fcds-access-client%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Faccess%2Fclient%2FAbstractClientConnection.java;h=47c0676979b94ea291532e5b713aee0bbcbece56;hb=585e116247f9b616579ffad1785a972621d928e7;hp=bade34cb2f81f29bcf6aebfe9897dbd722c5916c;hpb=b604e392408dfbf46ee0a7e1ac4cd27170a83bff;p=controller.git diff --git a/opendaylight/md-sal/cds-access-client/src/main/java/org/opendaylight/controller/cluster/access/client/AbstractClientConnection.java b/opendaylight/md-sal/cds-access-client/src/main/java/org/opendaylight/controller/cluster/access/client/AbstractClientConnection.java index bade34cb2f..47c0676979 100644 --- a/opendaylight/md-sal/cds-access-client/src/main/java/org/opendaylight/controller/cluster/access/client/AbstractClientConnection.java +++ b/opendaylight/md-sal/cds-access-client/src/main/java/org/opendaylight/controller/cluster/access/client/AbstractClientConnection.java @@ -9,21 +9,20 @@ 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.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; @@ -42,39 +41,63 @@ import scala.concurrent.duration.FiniteDuration; public abstract class AbstractClientConnection { private static final Logger LOG = LoggerFactory.getLogger(AbstractClientConnection.class); - // Keep these constants in nanoseconds, as that prevents unnecessary conversions in the fast path + /* + * 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 attept to reconnect + * it. + */ @VisibleForTesting - static final long NO_PROGRESS_TIMEOUT_NANOS = TimeUnit.MINUTES.toNanos(15); + static final long 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.SECONDS.toNanos(30); + static final long REQUEST_TIMEOUT_NANOS = TimeUnit.MINUTES.toNanos(2); - private final Queue inflight; - private final Queue pending; + /** + * 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); + 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; + + /** + * Time reference when we saw any activity from the backend. + */ + private long lastReceivedTicks; + private volatile RequestException poisoned; - private long lastProgress; - private AbstractClientConnection(final ClientActorContext context, final Long cookie, - final Queue inflight, final Queue 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.lastProgress = readTime(); + this.queue = Preconditions.checkNotNull(queue); + this.lastReceivedTicks = currentTime(); } // 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 oldConnection) { - this(oldConnection.context, oldConnection.cookie, oldConnection.inflight, oldConnection.pending); + AbstractClientConnection(final AbstractClientConnection oldConnection, final int targetQueueSize) { + this.context = oldConnection.context; + this.cookie = oldConnection.cookie; + this.queue = new TransmitQueue.Halted(targetQueueSize); + this.lastReceivedTicks = oldConnection.lastReceivedTicks; } public final ClientActorContext context() { @@ -89,91 +112,101 @@ public abstract class AbstractClientConnection { 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. * + *

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> 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; - } - - enqueueEntry(entry); - - final ReconnectForwarder afterQueue = successor; - if (afterQueue != null) { - synchronized (this) { - spliceToSuccessor(afterQueue); - } + 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); } } - 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); + /** + * Send a request to the backend and invoke a specified callback when it finishes. This method is safe to invoke + * from any thread. + * + *

+ * 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> callback, + final long enqueuedTicks) { + enqueueEntry(new ConnectionEntry(request, callback, enqueuedTicks), currentTime()); } public abstract Optional getBackendInfo(); - abstract ClientActorBehavior reconnectConnection(ClientActorBehavior current); - - abstract int remoteMaxMessages(); - - abstract Entry prepareForTransmit(Request req); + final Iterable startReplay() { + lock.lock(); + return queue.asIterable(); + } - @GuardedBy("this") - final void spliceToSuccessor(final ReconnectForwarder successor) { - ConnectionEntry entry = inflight.poll(); - while (entry != null) { - successor.forwardEntry(entry); - entry = inflight.poll(); - } + @GuardedBy("lock") + final void finishReplay(final ReconnectForwarder forwarder) { + queue.setForwarder(forwarder); + lock.unlock(); + } - entry = pending.poll(); - while (entry != null) { - successor.forwardEntry(entry); - entry = pending.poll(); + @GuardedBy("lock") + final void setForwarder(final ReconnectForwarder forwarder) { + final long now = currentTime(); + final Iterator it = queue.asIterable().iterator(); + while (it.hasNext()) { + final ConnectionEntry e = it.next(); + forwarder.forwardEntry(e, now); + it.remove(); } - } - private long readTime() { - return context.ticker().read(); + queue.setForwarder(forwarder); } - private void transmit(final ConnectionEntry entry) { - final Entry tuple = prepareForTransmit(entry.getRequest()); - final RequestEnvelope req = tuple.getValue(); + @GuardedBy("lock") + abstract ClientActorBehavior lockedReconnect(ClientActorBehavior current); - // 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); + 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 ActorRef actor = tuple.getKey(); - LOG.trace("Transmitting request {} as {} to {}", entry.getRequest(), req, actor); - actor.tell(req, ActorRef.noSender()); + if (queue.isEmpty()) { + // The queue is becoming non-empty, schedule a timer. + scheduleTimer(entry.getEnqueuedTicks() + REQUEST_TIMEOUT_NANOS - now); + } + return queue.enqueue(entry, now); + } finally { + lock.unlock(); + } } - 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 ClientActorBehavior reconnect(final ClientActorBehavior current) { + lock.lock(); + try { + return lockedReconnect(current); + } finally { + lock.unlock(); } } @@ -182,9 +215,25 @@ public abstract class AbstractClientConnection { * * @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", context.persistenceId()); + return; + } + if (queue.hasSuccessor()) { + LOG.debug("{}: connection has successor, not scheduling timer", context.persistenceId()); + 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 FiniteDuration dur = FiniteDuration.fromNanos(normalized); + LOG.debug("{}: scheduling timeout in {}", context.persistenceId(), dur); + context.executeInActor(this::runTimer, dur); + haveTimer = true; } /** @@ -196,37 +245,54 @@ public abstract class AbstractClientConnection { */ @VisibleForTesting final ClientActorBehavior runTimer(final ClientActorBehavior current) { - final long now = readTime(); - - if (!inflight.isEmpty() || !pending.isEmpty()) { - final long ticksSinceProgress = now - lastProgress; + final Optional 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. + // 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)); - 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 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; } + @VisibleForTesting + final Optional 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,53 +301,54 @@ public abstract class AbstractClientConnection { */ @SuppressFBWarnings(value = "NP_OPTIONAL_RETURN_NULL", justification = "Returning null Optional is documented in the API contract.") - private Optional checkTimeout(final ConnectionEntry head, final long now) { - if (head == null) { + @GuardedBy("lock") + private Optional lockedCheckTimeout(final long now) { + if (queue.isEmpty()) { return Optional.empty(); } - final long delay = head.getEnqueuedTicks() - now + REQUEST_TIMEOUT_NANOS; - if (delay <= 0) { - LOG.debug("Connection {} timed out", this); + final long backendSilentTicks = now - lastReceivedTicks; + if (backendSilentTicks >= BACKEND_ALIVE_TIMEOUT_NANOS) { + LOG.debug("Connection {} has not seen activity from backend for {} nanoseconds, timing out", 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(); + if (beenOpen < REQUEST_TIMEOUT_NANOS) { + return Optional.of(REQUEST_TIMEOUT_NANOS - beenOpen); + } - /* - * 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 checkTimeout(final long now) { - final Optional xmit = checkTimeout(inflight.peek(), now); - if (xmit == null) { - return null; - } - final Optional pend = checkTimeout(pending.peek(), now); - if (pend == null) { - return null; + tasksTimedOut++; + queue.remove(now); + LOG.debug("Connection {} timed out entryt {}", this, head); + head.complete(head.getRequest().toRequestFailure( + new RequestTimeoutException("Timed out after " + beenOpen + "ns"))); } - if (!xmit.isPresent()) { - return pend; - } - if (!pend.isPresent()) { - return xmit; + + LOG.debug("Connection {} timed out {} tasks", this, tasksTimedOut); + if (tasksTimedOut != 0) { + queue.tryTransmit(now); } - return Optional.of(xmit.get().min(pend.get())); + return Optional.empty(); } final void poison(final RequestException cause) { - poisoned = cause; + lock.lock(); + try { + lockedPoison(cause); + } finally { + lock.unlock(); + } + } - poisonQueue(inflight, cause); - poisonQueue(pending, cause); + @GuardedBy("lock") + private void lockedPoison(final RequestException cause) { + poisoned = cause; + queue.poison(cause); } @VisibleForTesting @@ -290,97 +357,30 @@ public abstract class AbstractClientConnection { } final void receiveResponse(final ResponseEnvelope envelope) { - Optional maybeEntry = findMatchingEntry(inflight, envelope); - if (maybeEntry == null) { - LOG.debug("Request for {} not found in inflight queue, checking pending queue", envelope); - maybeEntry = findMatchingEntry(pending, envelope); + final long now = currentTime(); + lastReceivedTicks = now; + + final Optional maybeEntry; + lock.lock(); + try { + maybeEntry = queue.complete(envelope, now); + } finally { + lock.unlock(); } - if (maybeEntry == null || !maybeEntry.isPresent()) { - LOG.warn("No request matching {} found, ignoring response", envelope); - return; + if (maybeEntry.isPresent()) { + final TransmittedConnectionEntry entry = maybeEntry.get(); + LOG.debug("Completing {} with {}", entry, envelope); + entry.complete(envelope.getMessage()); } - - 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; - } - - LOG.debug("Transmitting entry {}", e); - transmit(e); - toSend--; - } - - lastProgress = readTime(); } - private static void poisonQueue(final Queue 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(); + @Override + public final String toString() { + return addToStringAttributes(MoreObjects.toStringHelper(this).omitNullValues()).toString(); } - /* - * 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 findMatchingEntry(final Queue 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 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; + ToStringHelper addToStringAttributes(final ToStringHelper toStringHelper) { + return toStringHelper.add("client", context.getIdentifier()).add("cookie", cookie).add("poisoned", poisoned); } }