BUG-5280: handle NotLeaderException
[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.Preconditions;
13 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
14 import java.util.Optional;
15 import java.util.concurrent.TimeUnit;
16 import java.util.concurrent.locks.Lock;
17 import java.util.concurrent.locks.ReentrantLock;
18 import java.util.function.Consumer;
19 import javax.annotation.Nonnull;
20 import javax.annotation.concurrent.GuardedBy;
21 import javax.annotation.concurrent.NotThreadSafe;
22 import org.opendaylight.controller.cluster.access.concepts.Request;
23 import org.opendaylight.controller.cluster.access.concepts.RequestException;
24 import org.opendaylight.controller.cluster.access.concepts.Response;
25 import org.opendaylight.controller.cluster.access.concepts.ResponseEnvelope;
26 import org.slf4j.Logger;
27 import org.slf4j.LoggerFactory;
28 import scala.concurrent.duration.FiniteDuration;
29
30 /**
31  * Base class for a connection to the backend. Responsible to queueing and dispatch of requests toward the backend.
32  * Can be in three conceptual states: Connecting, Connected and Reconnecting, which are represented by public final
33  * classes exposed from this package.
34  *
35  * @author Robert Varga
36  */
37 @NotThreadSafe
38 public abstract class AbstractClientConnection<T extends BackendInfo> {
39     private static final Logger LOG = LoggerFactory.getLogger(AbstractClientConnection.class);
40
41     // Keep these constants in nanoseconds, as that prevents unnecessary conversions in the fast path
42     @VisibleForTesting
43     static final long NO_PROGRESS_TIMEOUT_NANOS = TimeUnit.MINUTES.toNanos(15);
44     @VisibleForTesting
45     static final long REQUEST_TIMEOUT_NANOS = TimeUnit.SECONDS.toNanos(30);
46
47     private static final FiniteDuration REQUEST_TIMEOUT_DURATION = FiniteDuration.apply(REQUEST_TIMEOUT_NANOS,
48         TimeUnit.NANOSECONDS);
49
50     private final Lock lock = new ReentrantLock();
51     private final ClientActorContext context;
52     @GuardedBy("lock")
53     private final TransmitQueue queue;
54     private final Long cookie;
55
56     @GuardedBy("lock")
57     private boolean haveTimer;
58
59     private volatile RequestException poisoned;
60
61     // Do not allow subclassing outside of this package
62     AbstractClientConnection(final ClientActorContext context, final Long cookie,
63             final TransmitQueue queue) {
64         this.context = Preconditions.checkNotNull(context);
65         this.cookie = Preconditions.checkNotNull(cookie);
66         this.queue = Preconditions.checkNotNull(queue);
67     }
68
69     // Do not allow subclassing outside of this package
70     AbstractClientConnection(final AbstractClientConnection<T> oldConnection, final int targetQueueSize) {
71         this.context = oldConnection.context;
72         this.cookie = oldConnection.cookie;
73         this.queue = new TransmitQueue.Halted(targetQueueSize);
74     }
75
76     public final ClientActorContext context() {
77         return context;
78     }
79
80     public final @Nonnull Long cookie() {
81         return cookie;
82     }
83
84     public final ActorRef localActor() {
85         return context.self();
86     }
87
88     /**
89      * Send a request to the backend and invoke a specified callback when it finishes. This method is safe to invoke
90      * from any thread.
91      *
92      * <p>This method may put the caller thread to sleep in order to throttle the request rate.
93      * The callback may be called before the sleep finishes.
94      *
95      * @param request Request to send
96      * @param callback Callback to invoke
97      */
98     public final void sendRequest(final Request<?, ?> request, final Consumer<Response<?, ?>> callback) {
99         final RequestException maybePoison = poisoned;
100         if (maybePoison != null) {
101             throw new IllegalStateException("Connection " + this + " has been poisoned", maybePoison);
102         }
103
104         final ConnectionEntry entry = new ConnectionEntry(request, callback, readTime());
105         enqueueAndWait(entry, entry.getEnqueuedTicks());
106     }
107
108     public abstract Optional<T> getBackendInfo();
109
110     final Iterable<ConnectionEntry> startReplay() {
111         lock.lock();
112         return queue.asIterable();
113     }
114
115     @GuardedBy("lock")
116     final void finishReplay(final ReconnectForwarder forwarder) {
117         setForwarder(forwarder);
118         lock.unlock();
119     }
120
121     @GuardedBy("lock")
122     final void setForwarder(final ReconnectForwarder forwarder) {
123         queue.setForwarder(forwarder, readTime());
124     }
125
126     @GuardedBy("lock")
127     abstract ClientActorBehavior<T> lockedReconnect(ClientActorBehavior<T> current);
128
129     private long readTime() {
130         return context.ticker().read();
131     }
132
133     final long enqueueEntry(final ConnectionEntry entry, final long now) {
134         lock.lock();
135         try {
136             if (queue.isEmpty()) {
137                 // The queue is becoming non-empty, schedule a timer
138                 scheduleTimer(REQUEST_TIMEOUT_DURATION);
139             }
140             return queue.enqueue(entry, now);
141         } finally {
142             lock.unlock();
143         }
144     }
145
146     final void enqueueAndWait(final ConnectionEntry entry, final long now) {
147         final long delay = enqueueEntry(entry, now);
148         try {
149             TimeUnit.NANOSECONDS.sleep(delay);
150         } catch (InterruptedException e) {
151             LOG.debug("Interrupted while sleeping", e);
152         }
153     }
154
155     final ClientActorBehavior<T> reconnect(final ClientActorBehavior<T> current) {
156         lock.lock();
157         try {
158             return lockedReconnect(current);
159         } finally {
160             lock.unlock();
161         }
162     }
163
164     /**
165      * Schedule a timer to fire on the actor thread after a delay.
166      *
167      * @param delay Delay, in nanoseconds
168      */
169     @GuardedBy("lock")
170     private void scheduleTimer(final FiniteDuration delay) {
171         if (haveTimer) {
172             LOG.debug("{}: timer already scheduled", context.persistenceId());
173             return;
174         }
175         if (queue.hasSuccessor()) {
176             LOG.debug("{}: connection has successor, not scheduling timer", context.persistenceId());
177             return;
178         }
179         LOG.debug("{}: scheduling timeout in {}", context.persistenceId(), delay);
180         context.executeInActor(this::runTimer, delay);
181         haveTimer = true;
182     }
183
184     /**
185      * Check this queue for timeout and initiate reconnection if that happened. If the queue has not made progress
186      * in {@link #NO_PROGRESS_TIMEOUT_NANOS} nanoseconds, it will be aborted.
187      *
188      * @param current Current behavior
189      * @return Next behavior to use
190      */
191     @VisibleForTesting
192     final ClientActorBehavior<T> runTimer(final ClientActorBehavior<T> current) {
193         final Optional<FiniteDuration> delay;
194
195         lock.lock();
196         try {
197             haveTimer = false;
198             final long now = readTime();
199             // The following line is only reliable when queue is not forwarding, but such state should not last long.
200             final long ticksSinceProgress = queue.ticksStalling(now);
201             if (ticksSinceProgress >= NO_PROGRESS_TIMEOUT_NANOS) {
202                 LOG.error("Queue {} has not seen progress in {} seconds, failing all requests", this,
203                     TimeUnit.NANOSECONDS.toSeconds(ticksSinceProgress));
204
205                 lockedPoison(new NoProgressException(ticksSinceProgress));
206                 current.removeConnection(this);
207                 return current;
208             }
209
210             // Requests are always scheduled in sequence, hence checking for timeout is relatively straightforward.
211             // Note we use also inquire about the delay, so we can re-schedule if needed, hence the unusual tri-state
212             // return convention.
213             delay = lockedCheckTimeout(now);
214             if (delay == null) {
215                 // We have timed out. There is no point in scheduling a timer
216                 return lockedReconnect(current);
217             }
218
219             if (delay.isPresent()) {
220                 // If there is new delay, schedule a timer
221                 scheduleTimer(delay.get());
222             }
223         } finally {
224             lock.unlock();
225         }
226
227         return current;
228     }
229
230     @VisibleForTesting
231     final Optional<FiniteDuration> checkTimeout(final long now) {
232         lock.lock();
233         try {
234             return lockedCheckTimeout(now);
235         } finally {
236             lock.unlock();
237         }
238     }
239
240     /*
241      * We are using tri-state return here to indicate one of three conditions:
242      * - if there is no timeout to schedule, return Optional.empty()
243      * - if there is a timeout to schedule, return a non-empty optional
244      * - if this connections has timed out, return null
245      */
246     @SuppressFBWarnings(value = "NP_OPTIONAL_RETURN_NULL",
247             justification = "Returning null Optional is documented in the API contract.")
248     @GuardedBy("lock")
249     private Optional<FiniteDuration> lockedCheckTimeout(final long now) {
250         final ConnectionEntry head = queue.peek();
251         if (head == null) {
252             return Optional.empty();
253         }
254
255         final long beenOpen = now - head.getEnqueuedTicks();
256         if (beenOpen >= REQUEST_TIMEOUT_NANOS) {
257             LOG.debug("Connection {} has a request not completed for {} nanoseconds, timing out", this, beenOpen);
258             return null;
259         }
260
261         return Optional.of(FiniteDuration.apply(REQUEST_TIMEOUT_NANOS - beenOpen, TimeUnit.NANOSECONDS));
262     }
263
264     final void poison(final RequestException cause) {
265         lock.lock();
266         try {
267             lockedPoison(cause);
268         } finally {
269             lock.unlock();
270         }
271     }
272
273     @GuardedBy("lock")
274     private void lockedPoison(final RequestException cause) {
275         poisoned = cause;
276         queue.poison(cause);
277     }
278
279     @VisibleForTesting
280     final RequestException poisoned() {
281         return poisoned;
282     }
283
284     final void receiveResponse(final ResponseEnvelope<?> envelope) {
285         final long now = readTime();
286
287         final Optional<TransmittedConnectionEntry> maybeEntry;
288         lock.lock();
289         try {
290             maybeEntry = queue.complete(envelope, now);
291         } finally {
292             lock.unlock();
293         }
294
295         if (maybeEntry.isPresent()) {
296             final TransmittedConnectionEntry entry = maybeEntry.get();
297             LOG.debug("Completing {} with {}", entry, envelope);
298             entry.complete(envelope.getMessage());
299         }
300     }
301 }