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