Poison entries outside of main lock
[controller.git] / opendaylight / md-sal / cds-access-client / src / main / java / org / opendaylight / controller / cluster / access / client / AbstractClientConnection.java
1 /*
2  * Copyright (c) 2016 Cisco Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.controller.cluster.access.client;
9
10 import static java.util.Objects.requireNonNull;
11
12 import akka.actor.ActorRef;
13 import com.google.common.annotations.VisibleForTesting;
14 import com.google.common.base.MoreObjects;
15 import com.google.common.base.MoreObjects.ToStringHelper;
16 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
17 import java.util.Collection;
18 import java.util.List;
19 import java.util.Optional;
20 import java.util.concurrent.TimeUnit;
21 import java.util.concurrent.TimeoutException;
22 import java.util.concurrent.locks.Lock;
23 import java.util.concurrent.locks.ReentrantLock;
24 import java.util.function.Consumer;
25 import org.checkerframework.checker.lock.qual.GuardedBy;
26 import org.checkerframework.checker.lock.qual.Holding;
27 import org.eclipse.jdt.annotation.NonNull;
28 import org.opendaylight.controller.cluster.access.concepts.Request;
29 import org.opendaylight.controller.cluster.access.concepts.RequestException;
30 import org.opendaylight.controller.cluster.access.concepts.Response;
31 import org.opendaylight.controller.cluster.access.concepts.ResponseEnvelope;
32 import org.opendaylight.controller.cluster.access.concepts.RuntimeRequestException;
33 import org.slf4j.Logger;
34 import org.slf4j.LoggerFactory;
35 import scala.concurrent.duration.FiniteDuration;
36
37 /**
38  * Base class for a connection to the backend. Responsible to queueing and dispatch of requests toward the backend.
39  * Can be in three conceptual states: Connecting, Connected and Reconnecting, which are represented by public final
40  * classes exposed from this package. This class NOT thread-safe, not are its subclasses expected to be thread-safe.
41  *
42  * @author Robert Varga
43  */
44 public abstract class AbstractClientConnection<T extends BackendInfo> {
45     private static final Logger LOG = LoggerFactory.getLogger(AbstractClientConnection.class);
46
47     /*
48      * Timers involved in communication with the backend. There are three tiers which are spaced out to allow for
49      * recovery at each tier. Keep these constants in nanoseconds, as that prevents unnecessary conversions in the fast
50      * path.
51      */
52     /**
53      * Backend aliveness timer. This is reset whenever we receive a response from the backend and kept armed whenever
54      * we have an outstanding request. If when this time expires, we tear down this connection and attempt to reconnect
55      * it.
56      */
57     public static final long DEFAULT_BACKEND_ALIVE_TIMEOUT_NANOS = TimeUnit.SECONDS.toNanos(30);
58
59     /**
60      * Request timeout. If the request fails to complete within this time since it was originally enqueued, we time
61      * the request out.
62      */
63     public static final long DEFAULT_REQUEST_TIMEOUT_NANOS = TimeUnit.MINUTES.toNanos(2);
64
65     /**
66      * No progress timeout. A client fails to make any forward progress in this time, it will terminate itself.
67      */
68     public static final long DEFAULT_NO_PROGRESS_TIMEOUT_NANOS = TimeUnit.MINUTES.toNanos(15);
69
70     // Emit a debug entry if we sleep for more that this amount
71     private static final long DEBUG_DELAY_NANOS = TimeUnit.MILLISECONDS.toNanos(100);
72
73     // Upper bound on the time a thread is forced to sleep to keep queue size under control
74     private static final long MAX_DELAY_SECONDS = 5;
75     private static final long MAX_DELAY_NANOS = TimeUnit.SECONDS.toNanos(MAX_DELAY_SECONDS);
76
77     private final Lock lock = new ReentrantLock();
78     private final ClientActorContext context;
79     @GuardedBy("lock")
80     private final TransmitQueue queue;
81     private final @NonNull Long cookie;
82     private final String backendName;
83
84     @GuardedBy("lock")
85     private boolean haveTimer;
86
87     /**
88      * Time reference when we saw any activity from the backend.
89      */
90     private long lastReceivedTicks;
91
92     private volatile RequestException poisoned;
93
94     // Private constructor to avoid code duplication.
95     private AbstractClientConnection(final AbstractClientConnection<T> oldConn, final TransmitQueue newQueue,
96             final String backendName) {
97         this.context = requireNonNull(oldConn.context);
98         this.cookie = requireNonNull(oldConn.cookie);
99         this.backendName = requireNonNull(backendName);
100         this.queue = requireNonNull(newQueue);
101         // Will be updated in finishReplay if needed.
102         this.lastReceivedTicks = oldConn.lastReceivedTicks;
103     }
104
105     // This constructor is only to be called by ConnectingClientConnection constructor.
106     // Do not allow subclassing outside of this package
107     AbstractClientConnection(final ClientActorContext context, final Long cookie, final String backendName,
108             final int queueDepth) {
109         this.context = requireNonNull(context);
110         this.cookie = requireNonNull(cookie);
111         this.backendName = requireNonNull(backendName);
112         this.queue = new TransmitQueue.Halted(queueDepth);
113         this.lastReceivedTicks = currentTime();
114     }
115
116     // This constructor is only to be called (indirectly) by ReconnectingClientConnection constructor.
117     // Do not allow subclassing outside of this package
118     AbstractClientConnection(final AbstractClientConnection<T> oldConn) {
119         this(oldConn, new TransmitQueue.Halted(oldConn.queue, oldConn.currentTime()), oldConn.backendName);
120     }
121
122     // This constructor is only to be called (indirectly) by ConnectedClientConnection constructor.
123     // Do not allow subclassing outside of this package
124     AbstractClientConnection(final AbstractClientConnection<T> oldConn, final T newBackend,
125             final int queueDepth) {
126         this(oldConn, new TransmitQueue.Transmitting(oldConn.queue, queueDepth, newBackend, oldConn.currentTime(),
127             requireNonNull(oldConn.context).messageSlicer()), newBackend.getName());
128     }
129
130     public final ClientActorContext context() {
131         return context;
132     }
133
134     public final @NonNull Long cookie() {
135         return cookie;
136     }
137
138     public final ActorRef localActor() {
139         return context.self();
140     }
141
142     public final long currentTime() {
143         return context.ticker().read();
144     }
145
146     /**
147      * Send a request to the backend and invoke a specified callback when it finishes. This method is safe to invoke
148      * from any thread.
149      *
150      * <p>This method may put the caller thread to sleep in order to throttle the request rate.
151      * The callback may be called before the sleep finishes.
152      *
153      * @param request Request to send
154      * @param callback Callback to invoke
155      */
156     public final void sendRequest(final Request<?, ?> request, final Consumer<Response<?, ?>> callback) {
157         final long now = currentTime();
158         sendEntry(new ConnectionEntry(request, callback, now), now);
159     }
160
161     /**
162      * Send a request to the backend and invoke a specified callback when it finishes. This method is safe to invoke
163      * from any thread.
164      *
165      * <p>
166      * Note that unlike {@link #sendRequest(Request, Consumer)}, this method does not exert backpressure, hence it
167      * should never be called from an application thread.
168      *
169      * @param request Request to send
170      * @param callback Callback to invoke
171      * @param enqueuedTicks Time (according to {@link #currentTime()} of request enqueue
172      */
173     public final void enqueueRequest(final Request<?, ?> request, final Consumer<Response<?, ?>> callback,
174             final long enqueuedTicks) {
175         enqueueEntry(new ConnectionEntry(request, callback, enqueuedTicks), currentTime());
176     }
177
178     private long enqueueOrForward(final ConnectionEntry entry, final long now) {
179         lock.lock();
180         try {
181             commonEnqueue(entry, now);
182             return queue.enqueueOrForward(entry, now);
183         } finally {
184             lock.unlock();
185         }
186     }
187
188     /**
189      * Enqueue an entry, possibly also transmitting it.
190      */
191     public final void enqueueEntry(final ConnectionEntry entry, final long now) {
192         lock.lock();
193         try {
194             commonEnqueue(entry, now);
195             queue.enqueueOrReplay(entry, now);
196         } finally {
197             lock.unlock();
198         }
199     }
200
201     @Holding("lock")
202     private void commonEnqueue(final ConnectionEntry entry, final long now) {
203         final RequestException maybePoison = poisoned;
204         if (maybePoison != null) {
205             throw new IllegalStateException("Connection " + this + " has been poisoned", maybePoison);
206         }
207
208         if (queue.isEmpty()) {
209             // The queue is becoming non-empty, schedule a timer.
210             scheduleTimer(entry.getEnqueuedTicks() + context.config().getRequestTimeout() - now);
211         }
212     }
213
214     // To be called from ClientActorBehavior on ConnectedClientConnection after entries are replayed.
215     final void cancelDebt() {
216         queue.cancelDebt(currentTime());
217     }
218
219     public abstract Optional<T> getBackendInfo();
220
221     final Collection<ConnectionEntry> startReplay() {
222         lock.lock();
223         return queue.drain();
224     }
225
226     @Holding("lock")
227     final void finishReplay(final ReconnectForwarder forwarder) {
228         setForwarder(forwarder);
229
230         /*
231          * The process of replaying all messages may have taken a significant chunk of time, depending on type
232          * of messages, queue depth and available processing power. In extreme situations this may have already
233          * exceeded BACKEND_ALIVE_TIMEOUT_NANOS, in which case we are running the risk of not making reasonable forward
234          * progress before we start a reconnect cycle.
235          *
236          * Note that the timer is armed after we have sent the first message, hence we should be seeing a response
237          * from the backend before we see a timeout, simply due to how the mailbox operates.
238          *
239          * At any rate, reset the timestamp once we complete reconnection (which an atomic transition from the
240          * perspective of outside world), as that makes it a bit easier to reason about timing of events.
241          */
242         lastReceivedTicks = currentTime();
243         lock.unlock();
244     }
245
246     @Holding("lock")
247     final void setForwarder(final ReconnectForwarder forwarder) {
248         queue.setForwarder(forwarder, currentTime());
249     }
250
251     @Holding("lock")
252     abstract ClientActorBehavior<T> lockedReconnect(ClientActorBehavior<T> current,
253             RequestException runtimeRequestException);
254
255     final void sendEntry(final ConnectionEntry entry, final long now) {
256         long delay = enqueueOrForward(entry, now);
257         try {
258             if (delay >= DEBUG_DELAY_NANOS) {
259                 if (delay > MAX_DELAY_NANOS) {
260                     LOG.info("Capping {} throttle delay from {} to {} seconds", this,
261                         TimeUnit.NANOSECONDS.toSeconds(delay), MAX_DELAY_SECONDS, new Throwable());
262                     delay = MAX_DELAY_NANOS;
263                 }
264                 if (LOG.isDebugEnabled()) {
265                     LOG.debug("{}: Sleeping for {}ms on connection {}", context.persistenceId(),
266                         TimeUnit.NANOSECONDS.toMillis(delay), this);
267                 }
268             }
269             TimeUnit.NANOSECONDS.sleep(delay);
270         } catch (InterruptedException e) {
271             Thread.currentThread().interrupt();
272             LOG.debug("Interrupted after sleeping {}ns", currentTime() - now, e);
273         }
274     }
275
276     final ClientActorBehavior<T> reconnect(final ClientActorBehavior<T> current, final RequestException cause) {
277         lock.lock();
278         try {
279             return lockedReconnect(current, cause);
280         } finally {
281             lock.unlock();
282         }
283     }
284
285     /**
286      * Schedule a timer to fire on the actor thread after a delay.
287      *
288      * @param delay Delay, in nanoseconds
289      */
290     @Holding("lock")
291     private void scheduleTimer(final long delay) {
292         if (haveTimer) {
293             LOG.debug("{}: timer already scheduled on {}", context.persistenceId(), this);
294             return;
295         }
296         if (queue.hasSuccessor()) {
297             LOG.debug("{}: connection {} has a successor, not scheduling timer", context.persistenceId(), this);
298             return;
299         }
300
301         // If the delay is negative, we need to schedule an action immediately. While the caller could have checked
302         // for that condition and take appropriate action, but this is more convenient and less error-prone.
303         final long normalized =  delay <= 0 ? 0 : Math.min(delay, context.config().getBackendAlivenessTimerInterval());
304
305         final FiniteDuration dur = FiniteDuration.fromNanos(normalized);
306         LOG.debug("{}: connection {} scheduling timeout in {}", context.persistenceId(), this, dur);
307         context.executeInActor(this::runTimer, dur);
308         haveTimer = true;
309     }
310
311     /**
312      * Check this queue for timeout and initiate reconnection if that happened. If the queue has not made progress
313      * in {@link #DEFAULT_NO_PROGRESS_TIMEOUT_NANOS} nanoseconds, it will be aborted.
314      *
315      * @param current Current behavior
316      * @return Next behavior to use
317      */
318     @VisibleForTesting
319     final ClientActorBehavior<T> runTimer(final ClientActorBehavior<T> current) {
320         lock.lock();
321
322         final List<ConnectionEntry> poisonEntries;
323         final NoProgressException poisonCause;
324         try {
325             haveTimer = false;
326             final long now = currentTime();
327
328             LOG.debug("{}: running timer on {}", context.persistenceId(), this);
329
330             // The following line is only reliable when queue is not forwarding, but such state should not last long.
331             // FIXME: BUG-8422: this may not be accurate w.r.t. replayed entries
332             final long ticksSinceProgress = queue.ticksStalling(now);
333             if (ticksSinceProgress < context.config().getNoProgressTimeout()) {
334                 // Requests are always scheduled in sequence, hence checking for timeout is relatively straightforward.
335                 // Note we use also inquire about the delay, so we can re-schedule if needed, hence the unusual
336                 // tri-state return convention.
337                 final Optional<Long> delay = lockedCheckTimeout(now);
338                 if (delay == null) {
339                     // We have timed out. There is no point in scheduling a timer
340                     LOG.debug("{}: connection {} timed out", context.persistenceId(), this);
341                     return lockedReconnect(current, new RuntimeRequestException("Backend connection timed out",
342                         new TimeoutException()));
343                 }
344
345                 if (delay.isPresent()) {
346                     // If there is new delay, schedule a timer
347                     scheduleTimer(delay.get());
348                 } else {
349                     LOG.debug("{}: not scheduling timeout on {}", context.persistenceId(), this);
350                 }
351
352                 return current;
353             }
354
355             LOG.error("Queue {} has not seen progress in {} seconds, failing all requests", this,
356                 TimeUnit.NANOSECONDS.toSeconds(ticksSinceProgress));
357             poisonCause = new NoProgressException(ticksSinceProgress);
358             poisonEntries = lockedPoison(poisonCause);
359             current.removeConnection(this);
360         } finally {
361             lock.unlock();
362         }
363
364         poison(poisonEntries, poisonCause);
365         return current;
366     }
367
368     @VisibleForTesting
369     final Optional<Long> checkTimeout(final long now) {
370         lock.lock();
371         try {
372             return lockedCheckTimeout(now);
373         } finally {
374             lock.unlock();
375         }
376     }
377
378     long backendSilentTicks(final long now) {
379         return now - lastReceivedTicks;
380     }
381
382     /*
383      * We are using tri-state return here to indicate one of three conditions:
384      * - if there is no timeout to schedule, return Optional.empty()
385      * - if there is a timeout to schedule, return a non-empty optional
386      * - if this connections has timed out, return null
387      */
388     @SuppressFBWarnings(value = "NP_OPTIONAL_RETURN_NULL",
389             justification = "Returning null Optional is documented in the API contract.")
390     @GuardedBy("lock")
391     private Optional<Long> lockedCheckTimeout(final long now) {
392         if (queue.isEmpty()) {
393             LOG.debug("{}: connection {} is empty", context.persistenceId(), this);
394             return Optional.empty();
395         }
396
397         final long backendSilentTicks = backendSilentTicks(now);
398         if (backendSilentTicks >= context.config().getBackendAlivenessTimerInterval()) {
399             LOG.debug("{}: Connection {} has not seen activity from backend for {} nanoseconds, timing out",
400                 context.persistenceId(), this, backendSilentTicks);
401             return null;
402         }
403
404         int tasksTimedOut = 0;
405         for (ConnectionEntry head = queue.peek(); head != null; head = queue.peek()) {
406             final long beenOpen = now - head.getEnqueuedTicks();
407             final long requestTimeout = context.config().getRequestTimeout();
408             if (beenOpen < requestTimeout) {
409                 return Optional.of(requestTimeout - beenOpen);
410             }
411
412             tasksTimedOut++;
413             queue.remove(now);
414             LOG.debug("{}: Connection {} timed out entry {}", context.persistenceId(), this, head);
415
416             timeoutEntry(head, beenOpen);
417         }
418
419         LOG.debug("Connection {} timed out {} tasks", this, tasksTimedOut);
420         if (tasksTimedOut != 0) {
421             queue.tryTransmit(now);
422         }
423
424         return Optional.empty();
425     }
426
427     private void timeoutEntry(final ConnectionEntry entry, final long beenOpen) {
428         // Timeouts needs to be re-scheduled on actor thread because we are holding the lock on the current queue,
429         // which may be the tail of a successor chain. This is a problem if the callback attempts to send a request
430         // because that will attempt to lock the chain from the start, potentially causing a deadlock if there is
431         // a concurrent attempt to transmit.
432         context.executeInActor(current -> {
433             final double time = beenOpen * 1.0 / 1_000_000_000;
434             entry.complete(entry.getRequest().toRequestFailure(
435                 new RequestTimeoutException(entry.getRequest() + " timed out after " + time
436                         + " seconds. The backend for " + backendName + " is not available.")));
437             return current;
438         });
439     }
440
441     final void poison(final RequestException cause) {
442         final List<ConnectionEntry> entries;
443
444         lock.lock();
445         try {
446             entries = lockedPoison(cause);
447         } finally {
448             lock.unlock();
449         }
450
451         poison(entries, cause);
452     }
453
454     // Do not hold any locks while calling this
455     private static void poison(final Collection<? extends ConnectionEntry> entries, final RequestException cause) {
456         for (ConnectionEntry e : entries) {
457             final Request<?, ?> request = e.getRequest();
458             LOG.trace("Poisoning request {}", request, cause);
459             e.complete(request.toRequestFailure(cause));
460         }
461     }
462
463     @Holding("lock")
464     private List<ConnectionEntry> lockedPoison(final RequestException cause) {
465         poisoned = enrichPoison(cause);
466         return queue.poison();
467     }
468
469     RequestException enrichPoison(final RequestException ex) {
470         return ex;
471     }
472
473     @VisibleForTesting
474     final RequestException poisoned() {
475         return poisoned;
476     }
477
478     void receiveResponse(final ResponseEnvelope<?> envelope) {
479         final long now = currentTime();
480         lastReceivedTicks = now;
481
482         final Optional<TransmittedConnectionEntry> maybeEntry;
483         lock.lock();
484         try {
485             maybeEntry = queue.complete(envelope, now);
486         } finally {
487             lock.unlock();
488         }
489
490         if (maybeEntry.isPresent()) {
491             final TransmittedConnectionEntry entry = maybeEntry.get();
492             LOG.debug("Completing {} with {}", entry, envelope);
493             entry.complete(envelope.getMessage());
494         }
495     }
496
497     @Override
498     public final String toString() {
499         return addToStringAttributes(MoreObjects.toStringHelper(this).omitNullValues()).toString();
500     }
501
502     ToStringHelper addToStringAttributes(final ToStringHelper toStringHelper) {
503         return toStringHelper.add("client", context.getIdentifier()).add("cookie", cookie).add("poisoned", poisoned);
504     }
505 }