BUG-8452: make NoShardLeaderException retriable
[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 akka.actor.ActorRef;
11 import com.google.common.annotations.VisibleForTesting;
12 import com.google.common.base.MoreObjects;
13 import com.google.common.base.MoreObjects.ToStringHelper;
14 import com.google.common.base.Preconditions;
15 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
16 import java.util.Iterator;
17 import java.util.Optional;
18 import java.util.concurrent.TimeUnit;
19 import java.util.concurrent.TimeoutException;
20 import java.util.concurrent.locks.Lock;
21 import java.util.concurrent.locks.ReentrantLock;
22 import java.util.function.Consumer;
23 import javax.annotation.Nonnull;
24 import javax.annotation.concurrent.GuardedBy;
25 import javax.annotation.concurrent.NotThreadSafe;
26 import org.opendaylight.controller.cluster.access.concepts.Request;
27 import org.opendaylight.controller.cluster.access.concepts.RequestException;
28 import org.opendaylight.controller.cluster.access.concepts.Response;
29 import org.opendaylight.controller.cluster.access.concepts.ResponseEnvelope;
30 import org.opendaylight.controller.cluster.access.concepts.RuntimeRequestException;
31 import org.slf4j.Logger;
32 import org.slf4j.LoggerFactory;
33 import scala.concurrent.duration.FiniteDuration;
34
35 /**
36  * Base class for a connection to the backend. Responsible to queueing and dispatch of requests toward the backend.
37  * Can be in three conceptual states: Connecting, Connected and Reconnecting, which are represented by public final
38  * classes exposed from this package.
39  *
40  * @author Robert Varga
41  */
42 @NotThreadSafe
43 public abstract class AbstractClientConnection<T extends BackendInfo> {
44     private static final Logger LOG = LoggerFactory.getLogger(AbstractClientConnection.class);
45
46     /*
47      * Timers involved in communication with the backend. There are three tiers which are spaced out to allow for
48      * recovery at each tier. Keep these constants in nanoseconds, as that prevents unnecessary conversions in the fast
49      * path.
50      */
51     /**
52      * Backend aliveness timer. This is reset whenever we receive a response from the backend and kept armed whenever
53      * we have an outstanding request. If when this time expires, we tear down this connection and attept to reconnect
54      * it.
55      */
56     @VisibleForTesting
57     static final long 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     @VisibleForTesting
64     static final long 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     @VisibleForTesting
70     static final long NO_PROGRESS_TIMEOUT_NANOS = TimeUnit.MINUTES.toNanos(15);
71
72     private final Lock lock = new ReentrantLock();
73     private final ClientActorContext context;
74     @GuardedBy("lock")
75     private final TransmitQueue queue;
76     private final Long cookie;
77
78     @GuardedBy("lock")
79     private boolean haveTimer;
80
81     /**
82      * Time reference when we saw any activity from the backend.
83      */
84     private long lastReceivedTicks;
85
86     private volatile RequestException poisoned;
87
88     // Do not allow subclassing outside of this package
89     AbstractClientConnection(final ClientActorContext context, final Long cookie,
90             final TransmitQueue queue) {
91         this.context = Preconditions.checkNotNull(context);
92         this.cookie = Preconditions.checkNotNull(cookie);
93         this.queue = Preconditions.checkNotNull(queue);
94         this.lastReceivedTicks = currentTime();
95     }
96
97     // Do not allow subclassing outside of this package
98     AbstractClientConnection(final AbstractClientConnection<T> oldConnection, final int targetQueueSize) {
99         this.context = oldConnection.context;
100         this.cookie = oldConnection.cookie;
101         this.queue = new TransmitQueue.Halted(targetQueueSize);
102         this.lastReceivedTicks = oldConnection.lastReceivedTicks;
103     }
104
105     public final ClientActorContext context() {
106         return context;
107     }
108
109     public final @Nonnull Long cookie() {
110         return cookie;
111     }
112
113     public final ActorRef localActor() {
114         return context.self();
115     }
116
117     public final long currentTime() {
118         return context.ticker().read();
119     }
120
121     /**
122      * Send a request to the backend and invoke a specified callback when it finishes. This method is safe to invoke
123      * from any thread.
124      *
125      * <p>This method may put the caller thread to sleep in order to throttle the request rate.
126      * The callback may be called before the sleep finishes.
127      *
128      * @param request Request to send
129      * @param callback Callback to invoke
130      */
131     public final void sendRequest(final Request<?, ?> request, final Consumer<Response<?, ?>> callback) {
132         final long now = currentTime();
133         final long delay = enqueueEntry(new ConnectionEntry(request, callback, now), now);
134         try {
135             TimeUnit.NANOSECONDS.sleep(delay);
136         } catch (InterruptedException e) {
137             Thread.currentThread().interrupt();
138             LOG.debug("Interrupted after sleeping {}ns", e, currentTime() - now);
139         }
140     }
141
142     /**
143      * Send a request to the backend and invoke a specified callback when it finishes. This method is safe to invoke
144      * from any thread.
145      *
146      * <p>
147      * Note that unlike {@link #sendRequest(Request, Consumer)}, this method does not exert backpressure, hence it
148      * should never be called from an application thread.
149      *
150      * @param request Request to send
151      * @param callback Callback to invoke
152      * @param enqueuedTicks Time (according to {@link #currentTime()} of request enqueue
153      */
154     public final void enqueueRequest(final Request<?, ?> request, final Consumer<Response<?, ?>> callback,
155             final long enqueuedTicks) {
156         enqueueEntry(new ConnectionEntry(request, callback, enqueuedTicks), currentTime());
157     }
158
159     public abstract Optional<T> getBackendInfo();
160
161     final Iterable<ConnectionEntry> startReplay() {
162         lock.lock();
163         return queue.asIterable();
164     }
165
166     @GuardedBy("lock")
167     final void finishReplay(final ReconnectForwarder forwarder) {
168         queue.setForwarder(forwarder);
169         lock.unlock();
170     }
171
172     @GuardedBy("lock")
173     final void setForwarder(final ReconnectForwarder forwarder) {
174         final long now = currentTime();
175         final Iterator<ConnectionEntry> it = queue.asIterable().iterator();
176         while (it.hasNext()) {
177             final ConnectionEntry e = it.next();
178             forwarder.forwardEntry(e, now);
179             it.remove();
180         }
181
182         queue.setForwarder(forwarder);
183     }
184
185     @GuardedBy("lock")
186     abstract ClientActorBehavior<T> lockedReconnect(ClientActorBehavior<T> current,
187             RequestException runtimeRequestException);
188
189     final long enqueueEntry(final ConnectionEntry entry, final long now) {
190         lock.lock();
191         try {
192             final RequestException maybePoison = poisoned;
193             if (maybePoison != null) {
194                 throw new IllegalStateException("Connection " + this + " has been poisoned", maybePoison);
195             }
196
197             if (queue.isEmpty()) {
198                 // The queue is becoming non-empty, schedule a timer.
199                 scheduleTimer(entry.getEnqueuedTicks() + REQUEST_TIMEOUT_NANOS - now);
200             }
201             return queue.enqueue(entry, now);
202         } finally {
203             lock.unlock();
204         }
205     }
206
207     final ClientActorBehavior<T> reconnect(final ClientActorBehavior<T> current, final RequestException cause) {
208         lock.lock();
209         try {
210             return lockedReconnect(current, cause);
211         } finally {
212             lock.unlock();
213         }
214     }
215
216     /**
217      * Schedule a timer to fire on the actor thread after a delay.
218      *
219      * @param delay Delay, in nanoseconds
220      */
221     @GuardedBy("lock")
222     private void scheduleTimer(final long delay) {
223         if (haveTimer) {
224             LOG.debug("{}: timer already scheduled", context.persistenceId());
225             return;
226         }
227         if (queue.hasSuccessor()) {
228             LOG.debug("{}: connection has successor, not scheduling timer", context.persistenceId());
229             return;
230         }
231
232         // If the delay is negative, we need to schedule an action immediately. While the caller could have checked
233         // for that condition and take appropriate action, but this is more convenient and less error-prone.
234         final long normalized =  delay <= 0 ? 0 : Math.min(delay, BACKEND_ALIVE_TIMEOUT_NANOS);
235
236         final FiniteDuration dur = FiniteDuration.fromNanos(normalized);
237         LOG.debug("{}: scheduling timeout in {}", context.persistenceId(), dur);
238         context.executeInActor(this::runTimer, dur);
239         haveTimer = true;
240     }
241
242     /**
243      * Check this queue for timeout and initiate reconnection if that happened. If the queue has not made progress
244      * in {@link #NO_PROGRESS_TIMEOUT_NANOS} nanoseconds, it will be aborted.
245      *
246      * @param current Current behavior
247      * @return Next behavior to use
248      */
249     @VisibleForTesting
250     final ClientActorBehavior<T> runTimer(final ClientActorBehavior<T> current) {
251         final Optional<Long> delay;
252
253         lock.lock();
254         try {
255             haveTimer = false;
256             final long now = currentTime();
257             // The following line is only reliable when queue is not forwarding, but such state should not last long.
258             // FIXME: BUG-8422: this may not be accurate w.r.t. replayed entries
259             final long ticksSinceProgress = queue.ticksStalling(now);
260             if (ticksSinceProgress >= NO_PROGRESS_TIMEOUT_NANOS) {
261                 LOG.error("Queue {} has not seen progress in {} seconds, failing all requests", this,
262                     TimeUnit.NANOSECONDS.toSeconds(ticksSinceProgress));
263
264                 lockedPoison(new NoProgressException(ticksSinceProgress));
265                 current.removeConnection(this);
266                 return current;
267             }
268
269             // Requests are always scheduled in sequence, hence checking for timeout is relatively straightforward.
270             // Note we use also inquire about the delay, so we can re-schedule if needed, hence the unusual tri-state
271             // return convention.
272             delay = lockedCheckTimeout(now);
273             if (delay == null) {
274                 // We have timed out. There is no point in scheduling a timer
275                 return lockedReconnect(current, new RuntimeRequestException("Backend connection timed out",
276                     new TimeoutException()));
277             }
278
279             if (delay.isPresent()) {
280                 // If there is new delay, schedule a timer
281                 scheduleTimer(delay.get());
282             }
283         } finally {
284             lock.unlock();
285         }
286
287         return current;
288     }
289
290     @VisibleForTesting
291     final Optional<Long> checkTimeout(final long now) {
292         lock.lock();
293         try {
294             return lockedCheckTimeout(now);
295         } finally {
296             lock.unlock();
297         }
298     }
299
300     /*
301      * We are using tri-state return here to indicate one of three conditions:
302      * - if there is no timeout to schedule, return Optional.empty()
303      * - if there is a timeout to schedule, return a non-empty optional
304      * - if this connections has timed out, return null
305      */
306     @SuppressFBWarnings(value = "NP_OPTIONAL_RETURN_NULL",
307             justification = "Returning null Optional is documented in the API contract.")
308     @GuardedBy("lock")
309     private Optional<Long> lockedCheckTimeout(final long now) {
310         if (queue.isEmpty()) {
311             return Optional.empty();
312         }
313
314         final long backendSilentTicks = now - lastReceivedTicks;
315         if (backendSilentTicks >= BACKEND_ALIVE_TIMEOUT_NANOS) {
316             LOG.debug("Connection {} has not seen activity from backend for {} nanoseconds, timing out", this,
317                 backendSilentTicks);
318             return null;
319         }
320
321         int tasksTimedOut = 0;
322         for (ConnectionEntry head = queue.peek(); head != null; head = queue.peek()) {
323             final long beenOpen = now - head.getEnqueuedTicks();
324             if (beenOpen < REQUEST_TIMEOUT_NANOS) {
325                 return Optional.of(REQUEST_TIMEOUT_NANOS - beenOpen);
326             }
327
328             tasksTimedOut++;
329             queue.remove(now);
330             LOG.debug("Connection {} timed out entryt {}", this, head);
331             head.complete(head.getRequest().toRequestFailure(
332                 new RequestTimeoutException("Timed out after " + beenOpen + "ns")));
333         }
334
335         LOG.debug("Connection {} timed out {} tasks", this, tasksTimedOut);
336         if (tasksTimedOut != 0) {
337             queue.tryTransmit(now);
338         }
339
340         return Optional.empty();
341     }
342
343     final void poison(final RequestException cause) {
344         lock.lock();
345         try {
346             lockedPoison(cause);
347         } finally {
348             lock.unlock();
349         }
350     }
351
352     @GuardedBy("lock")
353     private void lockedPoison(final RequestException cause) {
354         poisoned = enrichPoison(cause);
355         queue.poison(cause);
356     }
357
358     RequestException enrichPoison(final RequestException ex) {
359         return ex;
360     }
361
362     @VisibleForTesting
363     final RequestException poisoned() {
364         return poisoned;
365     }
366
367     final void receiveResponse(final ResponseEnvelope<?> envelope) {
368         final long now = currentTime();
369         lastReceivedTicks = now;
370
371         final Optional<TransmittedConnectionEntry> maybeEntry;
372         lock.lock();
373         try {
374             maybeEntry = queue.complete(envelope, now);
375         } finally {
376             lock.unlock();
377         }
378
379         if (maybeEntry.isPresent()) {
380             final TransmittedConnectionEntry entry = maybeEntry.get();
381             LOG.debug("Completing {} with {}", entry, envelope);
382             entry.complete(envelope.getMessage());
383         }
384     }
385
386     @Override
387     public final String toString() {
388         return addToStringAttributes(MoreObjects.toStringHelper(this).omitNullValues()).toString();
389     }
390
391     ToStringHelper addToStringAttributes(final ToStringHelper toStringHelper) {
392         return toStringHelper.add("client", context.getIdentifier()).add("cookie", cookie).add("poisoned", poisoned);
393     }
394 }