X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fcds-access-client%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Faccess%2Fclient%2FAbstractClientConnection.java;h=f190e14b61e01d4f2a48c78e1a6761fd84028e78;hp=5f23557dc3b9ab3bc478fd1fc18c17cded8a8740;hb=HEAD;hpb=94362614be25e34e8427c02799daffb8cae29d94 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 5f23557dc3..f34760ec03 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 @@ -7,22 +7,25 @@ */ package org.opendaylight.controller.cluster.access.client; +import static java.util.Objects.requireNonNull; + 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.List; import java.util.Optional; +import java.util.OptionalLong; 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; -import javax.annotation.Nonnull; -import javax.annotation.concurrent.GuardedBy; -import javax.annotation.concurrent.NotThreadSafe; +import org.checkerframework.checker.lock.qual.GuardedBy; +import org.checkerframework.checker.lock.qual.Holding; +import org.eclipse.jdt.annotation.NonNull; import org.opendaylight.controller.cluster.access.concepts.Request; import org.opendaylight.controller.cluster.access.concepts.RequestException; import org.opendaylight.controller.cluster.access.concepts.Response; @@ -35,11 +38,10 @@ import scala.concurrent.duration.FiniteDuration; /** * Base class for a connection to the backend. Responsible to queueing and dispatch of requests toward the backend. * Can be in three conceptual states: Connecting, Connected and Reconnecting, which are represented by public final - * classes exposed from this package. + * classes exposed from this package. This class NOT thread-safe, not are its subclasses expected to be thread-safe. * * @author Robert Varga */ -@NotThreadSafe public abstract class AbstractClientConnection { private static final Logger LOG = LoggerFactory.getLogger(AbstractClientConnection.class); @@ -74,10 +76,11 @@ public abstract class AbstractClientConnection { private static final long MAX_DELAY_NANOS = TimeUnit.SECONDS.toNanos(MAX_DELAY_SECONDS); private final Lock lock = new ReentrantLock(); - private final ClientActorContext context; + private final @NonNull ClientActorContext context; + private final @NonNull Long cookie; + private final String backendName; @GuardedBy("lock") private final TransmitQueue queue; - private final Long cookie; @GuardedBy("lock") private boolean haveTimer; @@ -89,32 +92,51 @@ public abstract class AbstractClientConnection { private volatile RequestException poisoned; + // Private constructor to avoid code duplication. + private AbstractClientConnection(final AbstractClientConnection oldConn, final TransmitQueue newQueue, + final String backendName) { + context = oldConn.context; + cookie = oldConn.cookie; + this.backendName = requireNonNull(backendName); + queue = requireNonNull(newQueue); + // Will be updated in finishReplay if needed. + 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 String backendName, + final int queueDepth) { + this.context = requireNonNull(context); + this.cookie = requireNonNull(cookie); + this.backendName = requireNonNull(backendName); + queue = new TransmitQueue.Halted(queueDepth); + lastReceivedTicks = currentTime(); + } + + // This constructor is only to be called (indirectly) by ReconnectingClientConnection constructor. // 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.queue = Preconditions.checkNotNull(queue); - this.lastReceivedTicks = currentTime(); + AbstractClientConnection(final AbstractClientConnection oldConn) { + this(oldConn, new TransmitQueue.Halted(oldConn.queue, oldConn.currentTime()), oldConn.backendName); } + // This constructor is only to be called (indirectly) by ConnectedClientConnection constructor. // Do not allow subclassing outside of this package - 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; + AbstractClientConnection(final AbstractClientConnection oldConn, final T newBackend, + final int queueDepth) { + this(oldConn, new TransmitQueue.Transmitting(oldConn.queue, queueDepth, newBackend, oldConn.currentTime(), + requireNonNull(oldConn.context).messageSlicer()), newBackend.getName()); } - public final ClientActorContext context() { + public final @NonNull ClientActorContext context() { return context; } - public final @Nonnull Long cookie() { + public final @NonNull Long cookie() { return cookie; } - public final ActorRef localActor() { + public final @NonNull ActorRef localActor() { return context.self(); } @@ -143,7 +165,7 @@ public abstract class AbstractClientConnection { * *

* Note that unlike {@link #sendRequest(Request, Consumer)}, this method does not exert backpressure, hence it - * should never be called from an application thread. + * should never be called from an application thread and serves mostly for moving requests between queues. * * @param request Request to send * @param callback Callback to invoke @@ -154,24 +176,47 @@ public abstract class AbstractClientConnection { enqueueEntry(new ConnectionEntry(request, callback, enqueuedTicks), currentTime()); } - public final long enqueueEntry(final ConnectionEntry entry, final long now) { + private long enqueueOrForward(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); - } + commonEnqueue(entry, now); + return queue.enqueueOrForward(entry, now); + } finally { + lock.unlock(); + } + } - if (queue.isEmpty()) { - // The queue is becoming non-empty, schedule a timer. - scheduleTimer(entry.getEnqueuedTicks() + context.config().getRequestTimeout() - now); - } - return queue.enqueue(entry, now); + /** + * 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(); } } + @Holding("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 getBackendInfo(); final Collection startReplay() { @@ -179,7 +224,7 @@ public abstract class AbstractClientConnection { return queue.drain(); } - @GuardedBy("lock") + @Holding("lock") final void finishReplay(final ReconnectForwarder forwarder) { setForwarder(forwarder); @@ -199,17 +244,17 @@ public abstract class AbstractClientConnection { lock.unlock(); } - @GuardedBy("lock") + @Holding("lock") final void setForwarder(final ReconnectForwarder forwarder) { queue.setForwarder(forwarder, currentTime()); } - @GuardedBy("lock") + @Holding("lock") abstract ClientActorBehavior lockedReconnect(ClientActorBehavior current, RequestException runtimeRequestException); final void sendEntry(final ConnectionEntry entry, final long now) { - long delay = enqueueEntry(entry, now); + long delay = enqueueOrForward(entry, now); try { if (delay >= DEBUG_DELAY_NANOS) { if (delay > MAX_DELAY_NANOS) { @@ -225,7 +270,7 @@ public abstract class AbstractClientConnection { TimeUnit.NANOSECONDS.sleep(delay); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - LOG.debug("Interrupted after sleeping {}ns", e, currentTime() - now); + LOG.debug("Interrupted after sleeping {}ns", currentTime() - now, e); } } @@ -243,7 +288,7 @@ public abstract class AbstractClientConnection { * * @param delay Delay, in nanoseconds */ - @GuardedBy("lock") + @Holding("lock") private void scheduleTimer(final long delay) { if (haveTimer) { LOG.debug("{}: timer already scheduled on {}", context.persistenceId(), this); @@ -273,9 +318,10 @@ public abstract class AbstractClientConnection { */ @VisibleForTesting final ClientActorBehavior runTimer(final ClientActorBehavior current) { - final Optional delay; - lock.lock(); + + final List poisonEntries; + final NoProgressException poisonCause; try { haveTimer = false; final long now = currentTime(); @@ -285,41 +331,43 @@ public abstract class AbstractClientConnection { // 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)); + if (ticksSinceProgress < context.config().getNoProgressTimeout()) { + // 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 OptionalLong 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())); + } - lockedPoison(new NoProgressException(ticksSinceProgress)); - current.removeConnection(this); - return current; - } + if (delay.isPresent()) { + // If there is new delay, schedule a timer + scheduleTimer(delay.orElseThrow()); + } else { + LOG.debug("{}: not scheduling timeout on {}", context.persistenceId(), this); + } - // 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 - LOG.debug("{}: connection {} timed out", context.persistenceId(), this); - return lockedReconnect(current, new RuntimeRequestException("Backend connection timed out", - new TimeoutException())); + return current; } - if (delay.isPresent()) { - // If there is new delay, schedule a timer - scheduleTimer(delay.get()); - } else { - LOG.debug("{}: not scheduling timeout on {}", context.persistenceId(), this); - } + LOG.error("Queue {} has not seen progress in {} seconds, failing all requests", this, + TimeUnit.NANOSECONDS.toSeconds(ticksSinceProgress)); + poisonCause = new NoProgressException(ticksSinceProgress); + poisonEntries = lockedPoison(poisonCause); + current.removeConnection(this); } finally { lock.unlock(); } + poison(poisonEntries, poisonCause); return current; } @VisibleForTesting - final Optional checkTimeout(final long now) { + final OptionalLong checkTimeout(final long now) { lock.lock(); try { return lockedCheckTimeout(now); @@ -341,10 +389,10 @@ public abstract class AbstractClientConnection { @SuppressFBWarnings(value = "NP_OPTIONAL_RETURN_NULL", justification = "Returning null Optional is documented in the API contract.") @GuardedBy("lock") - private Optional lockedCheckTimeout(final long now) { + private OptionalLong lockedCheckTimeout(final long now) { if (queue.isEmpty()) { LOG.debug("{}: connection {} is empty", context.persistenceId(), this); - return Optional.empty(); + return OptionalLong.empty(); } final long backendSilentTicks = backendSilentTicks(now); @@ -359,16 +407,14 @@ public abstract class AbstractClientConnection { final long beenOpen = now - head.getEnqueuedTicks(); final long requestTimeout = context.config().getRequestTimeout(); if (beenOpen < requestTimeout) { - return Optional.of(requestTimeout - beenOpen); + return OptionalLong.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"))); + timeoutEntry(head, beenOpen); } LOG.debug("Connection {} timed out {} tasks", this, tasksTimedOut); @@ -376,22 +422,49 @@ public abstract class AbstractClientConnection { queue.tryTransmit(now); } - return Optional.empty(); + return OptionalLong.empty(); + } + + private void timeoutEntry(final ConnectionEntry entry, final long beenOpen) { + // Timeouts needs to be re-scheduled on actor thread because we are holding the lock on the current queue, + // which may be the tail of a successor chain. This is a problem if the callback attempts to send a request + // because that will attempt to lock the chain from the start, potentially causing a deadlock if there is + // a concurrent attempt to transmit. + context.executeInActor(current -> { + final double time = beenOpen * 1.0 / 1_000_000_000; + entry.complete(entry.getRequest().toRequestFailure( + new RequestTimeoutException(entry.getRequest() + " timed out after " + time + + " seconds. The backend for " + backendName + " is not available."))); + return current; + }); } final void poison(final RequestException cause) { + final List entries; + lock.lock(); try { - lockedPoison(cause); + entries = lockedPoison(cause); } finally { lock.unlock(); } + + poison(entries, cause); } - @GuardedBy("lock") - private void lockedPoison(final RequestException cause) { + // Do not hold any locks while calling this + private static void poison(final Collection entries, final RequestException cause) { + for (ConnectionEntry e : entries) { + final Request request = e.getRequest(); + LOG.trace("Poisoning request {}", request, cause); + e.complete(request.toRequestFailure(cause)); + } + } + + @Holding("lock") + private List lockedPoison(final RequestException cause) { poisoned = enrichPoison(cause); - queue.poison(cause); + return queue.poison(); } RequestException enrichPoison(final RequestException ex) { @@ -416,7 +489,7 @@ public abstract class AbstractClientConnection { } if (maybeEntry.isPresent()) { - final TransmittedConnectionEntry entry = maybeEntry.get(); + final TransmittedConnectionEntry entry = maybeEntry.orElseThrow(); LOG.debug("Completing {} with {}", entry, envelope); entry.complete(envelope.getMessage()); }