Propagate MountPointContext to NetconfMessageTransformer
[netconf.git] / netconf / sal-netconf-connector / src / main / java / org / opendaylight / netconf / sal / connect / netconf / sal / KeepaliveSalFacade.java
1 /*
2  * Copyright (c) 2015 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.netconf.sal.connect.netconf.sal;
9
10 import static com.google.common.base.Preconditions.checkState;
11 import static org.opendaylight.netconf.sal.connect.netconf.util.NetconfBaseOps.getSourceNode;
12 import static org.opendaylight.netconf.sal.connect.netconf.util.NetconfMessageTransformUtil.NETCONF_GET_CONFIG_NODEID;
13 import static org.opendaylight.netconf.sal.connect.netconf.util.NetconfMessageTransformUtil.NETCONF_GET_CONFIG_PATH;
14 import static org.opendaylight.netconf.sal.connect.netconf.util.NetconfMessageTransformUtil.NETCONF_RUNNING_QNAME;
15
16 import com.google.common.util.concurrent.FutureCallback;
17 import com.google.common.util.concurrent.Futures;
18 import com.google.common.util.concurrent.ListenableFuture;
19 import com.google.common.util.concurrent.MoreExecutors;
20 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
21 import java.util.concurrent.ScheduledExecutorService;
22 import java.util.concurrent.ScheduledFuture;
23 import java.util.concurrent.TimeUnit;
24 import java.util.concurrent.atomic.AtomicBoolean;
25 import org.opendaylight.mdsal.dom.api.DOMActionService;
26 import org.opendaylight.mdsal.dom.api.DOMNotification;
27 import org.opendaylight.mdsal.dom.api.DOMRpcAvailabilityListener;
28 import org.opendaylight.mdsal.dom.api.DOMRpcResult;
29 import org.opendaylight.mdsal.dom.api.DOMRpcService;
30 import org.opendaylight.netconf.sal.connect.api.RemoteDeviceHandler;
31 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfDeviceCommunicator;
32 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfSessionPreferences;
33 import org.opendaylight.netconf.sal.connect.netconf.util.NetconfMessageTransformUtil;
34 import org.opendaylight.netconf.sal.connect.util.RemoteDeviceId;
35 import org.opendaylight.yangtools.concepts.ListenerRegistration;
36 import org.opendaylight.yangtools.rfc8528.data.api.MountPointContext;
37 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
38 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
39 import org.opendaylight.yangtools.yang.model.api.SchemaPath;
40 import org.slf4j.Logger;
41 import org.slf4j.LoggerFactory;
42
43 /**
44  * SalFacade proxy that invokes keepalive RPCs to prevent session shutdown from remote device
45  * and to detect incorrect session drops (netconf session is inactive, but TCP/SSH connection is still present).
46  * The keepalive RPC is a get-config with empty filter.
47  */
48 public final class KeepaliveSalFacade implements RemoteDeviceHandler<NetconfSessionPreferences> {
49
50     private static final Logger LOG = LoggerFactory.getLogger(KeepaliveSalFacade.class);
51
52     // 2 minutes keepalive delay by default
53     private static final long DEFAULT_DELAY = TimeUnit.MINUTES.toSeconds(2);
54
55     // 1 minute transaction timeout by default
56     private static final long DEFAULT_TRANSACTION_TIMEOUT_MILLI = TimeUnit.MILLISECONDS.toMillis(60000);
57
58     private final RemoteDeviceId id;
59     private final RemoteDeviceHandler<NetconfSessionPreferences> salFacade;
60     private final ScheduledExecutorService executor;
61     private final long keepaliveDelaySeconds;
62     private final ResetKeepalive resetKeepaliveTask;
63     private final long defaultRequestTimeoutMillis;
64
65     private volatile NetconfDeviceCommunicator listener;
66     private volatile ScheduledFuture<?> currentKeepalive;
67     private volatile DOMRpcService currentDeviceRpc;
68     private final AtomicBoolean lastKeepAliveSucceeded = new AtomicBoolean(false);
69
70     public KeepaliveSalFacade(final RemoteDeviceId id, final RemoteDeviceHandler<NetconfSessionPreferences> salFacade,
71                               final ScheduledExecutorService executor, final long keepaliveDelaySeconds,
72                               final long defaultRequestTimeoutMillis) {
73         this.id = id;
74         this.salFacade = salFacade;
75         this.executor = executor;
76         this.keepaliveDelaySeconds = keepaliveDelaySeconds;
77         this.defaultRequestTimeoutMillis = defaultRequestTimeoutMillis;
78         this.resetKeepaliveTask = new ResetKeepalive();
79     }
80
81     public KeepaliveSalFacade(final RemoteDeviceId id, final RemoteDeviceHandler<NetconfSessionPreferences> salFacade,
82                               final ScheduledExecutorService executor) {
83         this(id, salFacade, executor, DEFAULT_DELAY, DEFAULT_TRANSACTION_TIMEOUT_MILLI);
84     }
85
86     /**
87      * Set the netconf session listener whenever ready.
88      *
89      * @param listener netconf session listener
90      */
91     public void setListener(final NetconfDeviceCommunicator listener) {
92         this.listener = listener;
93     }
94
95     /**
96      * Just cancel current keepalive task.
97      * If its already started, let it finish ... not such a big deal.
98      *
99      * <p>
100      * Then schedule next keepalive.
101      */
102     void resetKeepalive() {
103         LOG.trace("{}: Resetting netconf keepalive timer", id);
104         if (currentKeepalive != null) {
105             currentKeepalive.cancel(false);
106         }
107         scheduleKeepalives();
108     }
109
110     /**
111      * Cancel current keepalive and also reset current deviceRpc.
112      */
113     private void stopKeepalives() {
114         if (currentKeepalive != null) {
115             currentKeepalive.cancel(false);
116         }
117         currentDeviceRpc = null;
118     }
119
120     void reconnect() {
121         checkState(listener != null, "%s: Unable to reconnect, session listener is missing", id);
122         stopKeepalives();
123         LOG.info("{}: Reconnecting inactive netconf session", id);
124         listener.disconnect();
125     }
126
127     @Override
128     public void onDeviceConnected(final MountPointContext remoteSchemaContext,
129                           final NetconfSessionPreferences netconfSessionPreferences, final DOMRpcService deviceRpc) {
130         onDeviceConnected(remoteSchemaContext, netconfSessionPreferences, deviceRpc, null);
131     }
132
133     @Override
134     public void onDeviceConnected(final MountPointContext remoteSchemaContext,
135             final NetconfSessionPreferences netconfSessionPreferences, final DOMRpcService deviceRpc,
136             final DOMActionService deviceAction) {
137         this.currentDeviceRpc = deviceRpc;
138         final DOMRpcService deviceRpc1 =
139                 new KeepaliveDOMRpcService(deviceRpc, resetKeepaliveTask, defaultRequestTimeoutMillis, executor,
140                         new ResponseWaitingScheduler());
141
142         salFacade.onDeviceConnected(remoteSchemaContext, netconfSessionPreferences, deviceRpc1, deviceAction);
143
144         LOG.debug("{}: Netconf session initiated, starting keepalives", id);
145         scheduleKeepalives();
146     }
147
148     private void scheduleKeepalives() {
149         lastKeepAliveSucceeded.set(true);
150         checkState(currentDeviceRpc != null);
151         LOG.trace("{}: Scheduling keepalives every  {} {}", id, keepaliveDelaySeconds, TimeUnit.SECONDS);
152         currentKeepalive = executor.scheduleWithFixedDelay(new Keepalive(),
153           keepaliveDelaySeconds, keepaliveDelaySeconds, TimeUnit.SECONDS);
154     }
155
156     @Override
157     public void onDeviceDisconnected() {
158         stopKeepalives();
159         salFacade.onDeviceDisconnected();
160     }
161
162     @Override
163     public void onDeviceFailed(final Throwable throwable) {
164         stopKeepalives();
165         salFacade.onDeviceFailed(throwable);
166     }
167
168     @Override
169     public void onNotification(final DOMNotification domNotification) {
170         resetKeepalive();
171         salFacade.onNotification(domNotification);
172     }
173
174     @Override
175     public void close() {
176         stopKeepalives();
177         salFacade.close();
178     }
179
180     // Keepalive RPC static resources
181     private static final ContainerNode KEEPALIVE_PAYLOAD = NetconfMessageTransformUtil.wrap(NETCONF_GET_CONFIG_NODEID,
182             getSourceNode(NETCONF_RUNNING_QNAME), NetconfMessageTransformUtil.EMPTY_FILTER);
183
184     /**
185      * Invoke keepalive RPC and check the response. In case of any received response the keepalive
186      * is considered successful and schedules next keepalive with a fixed delay. If the response is unsuccessful (no
187      * response received, or the rcp could not even be sent) immediate reconnect is triggered as netconf session
188      * is considered inactive/failed.
189      */
190     private class Keepalive implements Runnable, FutureCallback<DOMRpcResult> {
191
192         @Override
193         public void run() {
194             LOG.trace("{}: Invoking keepalive RPC", id);
195
196             try {
197                 final boolean lastJobSucceeded = lastKeepAliveSucceeded.getAndSet(false);
198                 if (!lastJobSucceeded) {
199                     onFailure(new IllegalStateException("Previous keepalive timed out"));
200                 } else {
201                     Futures.addCallback(currentDeviceRpc.invokeRpc(NETCONF_GET_CONFIG_PATH, KEEPALIVE_PAYLOAD), this,
202                         MoreExecutors.directExecutor());
203                 }
204             } catch (final NullPointerException e) {
205                 LOG.debug("{}: Skipping keepalive while reconnecting", id);
206                 // Empty catch block intentional
207                 // Do nothing. The currentDeviceRpc was null and it means we hit the reconnect window and
208                 // attempted to send keepalive while we were reconnecting. Next keepalive will be scheduled
209                 // after reconnect so no action necessary here.
210             }
211         }
212
213         @SuppressFBWarnings(value = "RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE",
214                 justification = "Unrecognised NullableDecl")
215         @Override
216         public void onSuccess(final DOMRpcResult result) {
217             // No matter what response we got, rpc-reply or rpc-error,
218             // we got it from device so the netconf session is OK
219             if (result == null) {
220                 LOG.warn("{} Keepalive RPC returned null with response. Reconnecting netconf session", id);
221                 reconnect();
222                 return;
223             }
224
225             if (result.getResult() != null) {
226                 lastKeepAliveSucceeded.set(true);
227             }  else if (result.getErrors() != null) {
228                 LOG.warn("{}: Keepalive RPC failed with error: {}", id, result.getErrors());
229                 lastKeepAliveSucceeded.set(true);
230             } else {
231                 LOG.warn("{} Keepalive RPC returned null with response. Reconnecting netconf session", id);
232                 reconnect();
233             }
234         }
235
236         @Override
237         public void onFailure(final Throwable throwable) {
238             LOG.warn("{}: Keepalive RPC failed. Reconnecting netconf session.", id, throwable);
239             reconnect();
240         }
241     }
242
243     /**
244      * Reset keepalive after each RPC response received.
245      */
246     private class ResetKeepalive implements FutureCallback<DOMRpcResult> {
247         @Override
248         public void onSuccess(final DOMRpcResult result) {
249             // No matter what response we got,
250             // rpc-reply or rpc-error, we got it from device so the netconf session is OK.
251             resetKeepalive();
252         }
253
254         @Override
255         public void onFailure(final Throwable throwable) {
256             // User/Application RPC failed (The RPC did not reach the remote device or ..
257             // TODO what other reasons could cause this ?)
258             // There is no point in keeping this session. Reconnect.
259             LOG.warn("{}: Rpc failure detected. Reconnecting netconf session", id, throwable);
260             reconnect();
261         }
262     }
263
264     private final class ResponseWaitingScheduler {
265
266         private ScheduledFuture<?> schedule;
267
268         public void initScheduler(final Runnable runnable) {
269             if (currentKeepalive != null) {
270                 currentKeepalive.cancel(true);
271             } else {
272                 LOG.trace("Keepalive does not exist.");
273             }
274             scheduleKeepalives();
275             //Listening on the result should be done before the keepalive rpc will be send
276             final long delay = keepaliveDelaySeconds * 1000 - 500;
277             schedule = executor.schedule(runnable, delay, TimeUnit.MILLISECONDS);
278         }
279
280         public void stopScheduler() {
281             if (schedule != null) {
282                 schedule.cancel(true);
283             } else {
284                 LOG.trace("Scheduler does not exist.");
285             }
286         }
287     }
288
289     private static final class ResponseWaiting implements Runnable {
290
291         private final ListenableFuture<DOMRpcResult> rpcResultFuture;
292         private final ResponseWaitingScheduler responseWaitingScheduler;
293
294         ResponseWaiting(final ResponseWaitingScheduler responseWaitingScheduler,
295                 final ListenableFuture<DOMRpcResult> rpcResultFuture) {
296             this.responseWaitingScheduler = responseWaitingScheduler;
297             this.rpcResultFuture = rpcResultFuture;
298         }
299
300         public void start() {
301             LOG.trace("Start to waiting for result.");
302             responseWaitingScheduler.initScheduler(this);
303         }
304
305         public void stop() {
306             LOG.info("Stop to waiting for result.");
307             responseWaitingScheduler.stopScheduler();
308         }
309
310         @Override
311         public void run() {
312             if (!rpcResultFuture.isCancelled() && !rpcResultFuture.isDone()) {
313                 LOG.trace("Waiting for result");
314                 responseWaitingScheduler.initScheduler(this);
315             } else {
316                 LOG.trace("Result has been cancelled or done.");
317             }
318         }
319     }
320
321     /*
322      * Request timeout task is called once the defaultRequestTimeoutMillis is
323      * reached. At this moment, if the request is not yet finished, we cancel
324      * it.
325      */
326     private static final class RequestTimeoutTask implements Runnable {
327         private final ListenableFuture<DOMRpcResult> rpcResultFuture;
328         private final ResponseWaiting responseWaiting;
329
330         RequestTimeoutTask(final ListenableFuture<DOMRpcResult> rpcResultFuture,
331             final ResponseWaiting responseWaiting) {
332             this.rpcResultFuture = rpcResultFuture;
333             this.responseWaiting = responseWaiting;
334         }
335
336         @Override
337         public void run() {
338             if (!rpcResultFuture.isDone()) {
339                 rpcResultFuture.cancel(true);
340             }
341             if (responseWaiting != null) {
342                 responseWaiting.stop();
343             }
344         }
345     }
346
347     /**
348      * DOMRpcService proxy that attaches reset-keepalive-task and schedule
349      * request-timeout-task to each RPC invocation.
350      */
351     public static final class KeepaliveDOMRpcService implements DOMRpcService {
352
353         private final DOMRpcService deviceRpc;
354         private final ResetKeepalive resetKeepaliveTask;
355         private final long defaultRequestTimeoutMillis;
356         private final ScheduledExecutorService executor;
357         private final ResponseWaitingScheduler responseWaitingScheduler;
358
359         KeepaliveDOMRpcService(final DOMRpcService deviceRpc, final ResetKeepalive resetKeepaliveTask,
360                 final long defaultRequestTimeoutMillis, final ScheduledExecutorService executor,
361                 final ResponseWaitingScheduler responseWaitingScheduler) {
362             this.deviceRpc = deviceRpc;
363             this.resetKeepaliveTask = resetKeepaliveTask;
364             this.defaultRequestTimeoutMillis = defaultRequestTimeoutMillis;
365             this.executor = executor;
366             this.responseWaitingScheduler = responseWaitingScheduler;
367         }
368
369         public DOMRpcService getDeviceRpc() {
370             return deviceRpc;
371         }
372
373         @Override
374         public ListenableFuture<DOMRpcResult> invokeRpc(final SchemaPath type, final NormalizedNode<?, ?> input) {
375             final ListenableFuture<DOMRpcResult> rpcResultFuture = deviceRpc.invokeRpc(type, input);
376             final ResponseWaiting responseWaiting = new ResponseWaiting(responseWaitingScheduler, rpcResultFuture);
377             responseWaiting.start();
378             Futures.addCallback(rpcResultFuture, resetKeepaliveTask, MoreExecutors.directExecutor());
379
380             final RequestTimeoutTask timeoutTask = new RequestTimeoutTask(rpcResultFuture, responseWaiting);
381             executor.schedule(timeoutTask, defaultRequestTimeoutMillis, TimeUnit.MILLISECONDS);
382
383             return rpcResultFuture;
384         }
385
386         @Override
387         public <T extends DOMRpcAvailabilityListener> ListenerRegistration<T> registerRpcListener(final T listener) {
388             // There is no real communication with the device (yet), no reset here
389             return deviceRpc.registerRpcListener(listener);
390         }
391     }
392 }