Simplify base schema lookups
[netconf.git] / netconf / sal-netconf-connector / src / main / java / org / opendaylight / netconf / sal / connect / netconf / NetconfDevice.java
1 /*
2  * Copyright (c) 2014 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;
9
10 import static com.google.common.base.Preconditions.checkState;
11 import static java.util.Objects.requireNonNull;
12
13 import com.google.common.base.Predicates;
14 import com.google.common.collect.Collections2;
15 import com.google.common.collect.Lists;
16 import com.google.common.collect.Sets;
17 import com.google.common.util.concurrent.FutureCallback;
18 import com.google.common.util.concurrent.Futures;
19 import com.google.common.util.concurrent.ListenableFuture;
20 import com.google.common.util.concurrent.ListeningExecutorService;
21 import com.google.common.util.concurrent.MoreExecutors;
22 import io.netty.util.concurrent.EventExecutor;
23 import java.util.ArrayList;
24 import java.util.Collection;
25 import java.util.Collections;
26 import java.util.LinkedList;
27 import java.util.List;
28 import java.util.Optional;
29 import java.util.Set;
30 import java.util.concurrent.Callable;
31 import java.util.concurrent.ExecutionException;
32 import java.util.concurrent.TimeUnit;
33 import java.util.stream.Collectors;
34 import org.checkerframework.checker.lock.qual.GuardedBy;
35 import org.opendaylight.mdsal.dom.api.DOMRpcResult;
36 import org.opendaylight.mdsal.dom.api.DOMRpcService;
37 import org.opendaylight.netconf.api.NetconfMessage;
38 import org.opendaylight.netconf.sal.connect.api.DeviceActionFactory;
39 import org.opendaylight.netconf.sal.connect.api.MessageTransformer;
40 import org.opendaylight.netconf.sal.connect.api.NetconfDeviceSchemas;
41 import org.opendaylight.netconf.sal.connect.api.NetconfDeviceSchemasResolver;
42 import org.opendaylight.netconf.sal.connect.api.RemoteDevice;
43 import org.opendaylight.netconf.sal.connect.api.RemoteDeviceCommunicator;
44 import org.opendaylight.netconf.sal.connect.api.RemoteDeviceHandler;
45 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfDeviceCapabilities;
46 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfDeviceCommunicator;
47 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfSessionPreferences;
48 import org.opendaylight.netconf.sal.connect.netconf.sal.NetconfDeviceRpc;
49 import org.opendaylight.netconf.sal.connect.netconf.schema.NetconfRemoteSchemaYangSourceProvider;
50 import org.opendaylight.netconf.sal.connect.netconf.schema.YangLibrarySchemaYangSourceProvider;
51 import org.opendaylight.netconf.sal.connect.netconf.schema.mapping.BaseSchema;
52 import org.opendaylight.netconf.sal.connect.netconf.schema.mapping.NetconfMessageTransformer;
53 import org.opendaylight.netconf.sal.connect.netconf.util.NetconfMessageTransformUtil;
54 import org.opendaylight.netconf.sal.connect.util.RemoteDeviceId;
55 import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.netconf.notifications.rev120206.NetconfCapabilityChange;
56 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.optional.rev190614.NetconfNodeAugmentedOptional;
57 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.NetconfNode;
58 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.connection.status.available.capabilities.AvailableCapabilityBuilder;
59 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.connection.status.unavailable.capabilities.UnavailableCapability;
60 import org.opendaylight.yangtools.rcf8528.data.util.EmptyMountPointContext;
61 import org.opendaylight.yangtools.rfc8528.data.api.MountPointContext;
62 import org.opendaylight.yangtools.yang.common.QName;
63 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
64 import org.opendaylight.yangtools.yang.model.repo.api.MissingSchemaSourceException;
65 import org.opendaylight.yangtools.yang.model.repo.api.RevisionSourceIdentifier;
66 import org.opendaylight.yangtools.yang.model.repo.api.SchemaContextFactory;
67 import org.opendaylight.yangtools.yang.model.repo.api.SchemaRepository;
68 import org.opendaylight.yangtools.yang.model.repo.api.SchemaResolutionException;
69 import org.opendaylight.yangtools.yang.model.repo.api.SourceIdentifier;
70 import org.opendaylight.yangtools.yang.model.repo.api.YangTextSchemaSource;
71 import org.opendaylight.yangtools.yang.model.repo.spi.PotentialSchemaSource;
72 import org.opendaylight.yangtools.yang.model.repo.spi.SchemaSourceProvider;
73 import org.opendaylight.yangtools.yang.model.repo.spi.SchemaSourceRegistration;
74 import org.opendaylight.yangtools.yang.model.repo.spi.SchemaSourceRegistry;
75 import org.slf4j.Logger;
76 import org.slf4j.LoggerFactory;
77
78 /**
79  *  This is a mediator between NetconfDeviceCommunicator and NetconfDeviceSalFacade.
80  */
81 public class NetconfDevice
82         implements RemoteDevice<NetconfSessionPreferences, NetconfMessage, NetconfDeviceCommunicator> {
83
84     private static final Logger LOG = LoggerFactory.getLogger(NetconfDevice.class);
85
86     protected final RemoteDeviceId id;
87     protected final SchemaContextFactory schemaContextFactory;
88     protected final SchemaSourceRegistry schemaRegistry;
89     protected final SchemaRepository schemaRepository;
90
91     protected final List<SchemaSourceRegistration<?>> sourceRegistrations = new ArrayList<>();
92
93     private final RemoteDeviceHandler<NetconfSessionPreferences> salFacade;
94     private final ListeningExecutorService processingExecutor;
95     private final DeviceActionFactory deviceActionFactory;
96     private final NetconfDeviceSchemasResolver stateSchemasResolver;
97     private final NotificationHandler notificationHandler;
98     private final boolean reconnectOnSchemasChange;
99     private final NetconfNode node;
100     private final EventExecutor eventExecutor;
101     private final NetconfNodeAugmentedOptional nodeOptional;
102
103     @GuardedBy("this")
104     private boolean connected = false;
105
106     // Message transformer is constructed once the schemas are available
107     private MessageTransformer<NetconfMessage> messageTransformer;
108
109     public NetconfDevice(final SchemaResourcesDTO schemaResourcesDTO, final RemoteDeviceId id,
110                          final RemoteDeviceHandler<NetconfSessionPreferences> salFacade,
111                          final ListeningExecutorService globalProcessingExecutor,
112                          final boolean reconnectOnSchemasChange) {
113         this(schemaResourcesDTO, id, salFacade, globalProcessingExecutor, reconnectOnSchemasChange, null, null, null,
114                 null);
115     }
116
117     public NetconfDevice(final SchemaResourcesDTO schemaResourcesDTO, final RemoteDeviceId id,
118             final RemoteDeviceHandler<NetconfSessionPreferences> salFacade,
119             final ListeningExecutorService globalProcessingExecutor, final boolean reconnectOnSchemasChange,
120             final DeviceActionFactory deviceActionFactory, final NetconfNode node, final EventExecutor eventExecutor,
121             final NetconfNodeAugmentedOptional nodeOptional) {
122         this.id = id;
123         this.reconnectOnSchemasChange = reconnectOnSchemasChange;
124         this.deviceActionFactory = deviceActionFactory;
125         this.node = node;
126         this.eventExecutor = eventExecutor;
127         this.nodeOptional = nodeOptional;
128         this.schemaRegistry = schemaResourcesDTO.getSchemaRegistry();
129         this.schemaRepository = schemaResourcesDTO.getSchemaRepository();
130         this.schemaContextFactory = schemaResourcesDTO.getSchemaContextFactory();
131         this.salFacade = salFacade;
132         this.stateSchemasResolver = schemaResourcesDTO.getStateSchemasResolver();
133         this.processingExecutor = requireNonNull(globalProcessingExecutor);
134         this.notificationHandler = new NotificationHandler(salFacade, id);
135     }
136
137     @Override
138     public void onRemoteSessionUp(final NetconfSessionPreferences remoteSessionCapabilities,
139                                   final NetconfDeviceCommunicator listener) {
140         // SchemaContext setup has to be performed in a dedicated thread since
141         // we are in a netty thread in this method
142         // Yang models are being downloaded in this method and it would cause a
143         // deadlock if we used the netty thread
144         // http://netty.io/wiki/thread-model.html
145         setConnected(true);
146         LOG.debug("{}: Session to remote device established with {}", id, remoteSessionCapabilities);
147
148         final BaseSchema baseSchema = resolveBaseSchema(remoteSessionCapabilities.isNotificationsSupported());
149         final NetconfDeviceRpc initRpc = new NetconfDeviceRpc(baseSchema.getSchemaContext(), listener,
150             new NetconfMessageTransformer(baseSchema.getMountPointContext(), false, baseSchema));
151         final DeviceSourcesResolver task = new DeviceSourcesResolver(remoteSessionCapabilities, id,
152             stateSchemasResolver, initRpc, baseSchema.getSchemaContext());
153         final ListenableFuture<DeviceSources> sourceResolverFuture = processingExecutor.submit(task);
154
155         if (shouldListenOnSchemaChange(remoteSessionCapabilities)) {
156             registerToBaseNetconfStream(initRpc, listener);
157         }
158
159         final FutureCallback<DeviceSources> resolvedSourceCallback = new FutureCallback<DeviceSources>() {
160             @Override
161             public void onSuccess(final DeviceSources result) {
162                 addProvidedSourcesToSchemaRegistry(result);
163                 setUpSchema(result);
164             }
165
166             private void setUpSchema(final DeviceSources result) {
167                 processingExecutor.submit(new SchemaSetup(result, remoteSessionCapabilities, listener));
168             }
169
170             @Override
171             public void onFailure(final Throwable throwable) {
172                 LOG.warn("{}: Unexpected error resolving device sources", id, throwable);
173                 handleSalInitializationFailure(throwable, listener);
174             }
175         };
176
177         Futures.addCallback(sourceResolverFuture, resolvedSourceCallback, MoreExecutors.directExecutor());
178     }
179
180     private void registerToBaseNetconfStream(final NetconfDeviceRpc deviceRpc,
181                                              final NetconfDeviceCommunicator listener) {
182         // TODO check whether the model describing create subscription is present in schema
183         // Perhaps add a default schema context to support create-subscription if the model was not provided
184         // (same as what we do for base netconf operations in transformer)
185         final ListenableFuture<DOMRpcResult> rpcResultListenableFuture = deviceRpc.invokeRpc(
186                 NetconfMessageTransformUtil.CREATE_SUBSCRIPTION_RPC_PATH,
187                 NetconfMessageTransformUtil.CREATE_SUBSCRIPTION_RPC_CONTENT);
188
189         Futures.addCallback(rpcResultListenableFuture, new FutureCallback<DOMRpcResult>() {
190             @Override
191             public void onSuccess(final DOMRpcResult domRpcResult) {
192                 notificationHandler.addNotificationFilter(notification -> {
193                     if (NetconfCapabilityChange.QNAME.equals(notification.getBody().getNodeType())) {
194                         LOG.info("{}: Schemas change detected, reconnecting", id);
195                         // Only disconnect is enough,
196                         // the reconnecting nature of the connector will take care of reconnecting
197                         listener.disconnect();
198                         return Optional.empty();
199                     }
200                     return Optional.of(notification);
201                 });
202             }
203
204             @Override
205             public void onFailure(final Throwable throwable) {
206                 LOG.warn("Unable to subscribe to base notification stream. Schemas will not be reloaded on the fly",
207                         throwable);
208             }
209         }, MoreExecutors.directExecutor());
210     }
211
212     private boolean shouldListenOnSchemaChange(final NetconfSessionPreferences remoteSessionCapabilities) {
213         return remoteSessionCapabilities.isNotificationsSupported() && reconnectOnSchemasChange;
214     }
215
216     private synchronized void handleSalInitializationSuccess(final MountPointContext result,
217                                         final NetconfSessionPreferences remoteSessionCapabilities,
218                                         final DOMRpcService deviceRpc,
219                                         final RemoteDeviceCommunicator<NetconfMessage> listener) {
220         //NetconfDevice.SchemaSetup can complete after NetconfDeviceCommunicator was closed. In that case do nothing,
221         //since salFacade.onDeviceDisconnected was already called.
222         if (connected) {
223             final BaseSchema baseSchema =
224                 remoteSessionCapabilities.isNotificationsSupported()
225                         ? BaseSchema.BASE_NETCONF_CTX_WITH_NOTIFICATIONS : BaseSchema.BASE_NETCONF_CTX;
226             this.messageTransformer = new NetconfMessageTransformer(result, true, baseSchema);
227
228             // salFacade.onDeviceConnected has to be called before the notification handler is initialized
229             this.salFacade.onDeviceConnected(result, remoteSessionCapabilities, deviceRpc,
230                     this.deviceActionFactory == null ? null : this.deviceActionFactory.createDeviceAction(
231                             this.messageTransformer, listener, result.getSchemaContext()));
232             this.notificationHandler.onRemoteSchemaUp(this.messageTransformer);
233
234             LOG.info("{}: Netconf connector initialized successfully", id);
235         } else {
236             LOG.warn("{}: Device communicator was closed before schema setup finished.", id);
237         }
238     }
239
240     private void handleSalInitializationFailure(final Throwable throwable,
241                                                 final RemoteDeviceCommunicator<NetconfMessage> listener) {
242         LOG.error("{}: Initialization in sal failed, disconnecting from device", id, throwable);
243         listener.close();
244         onRemoteSessionDown();
245         resetMessageTransformer();
246     }
247
248     /**
249      * Set the transformer to null as is in initial state.
250      */
251     private void resetMessageTransformer() {
252         updateTransformer(null);
253     }
254
255     private synchronized void updateTransformer(final MessageTransformer<NetconfMessage> transformer) {
256         messageTransformer = transformer;
257     }
258
259     private synchronized void setConnected(final boolean connected) {
260         this.connected = connected;
261     }
262
263     private void addProvidedSourcesToSchemaRegistry(final DeviceSources deviceSources) {
264         final SchemaSourceProvider<YangTextSchemaSource> yangProvider = deviceSources.getSourceProvider();
265         for (final SourceIdentifier sourceId : deviceSources.getProvidedSources()) {
266             sourceRegistrations.add(schemaRegistry.registerSchemaSource(yangProvider,
267                     PotentialSchemaSource.create(
268                             sourceId, YangTextSchemaSource.class, PotentialSchemaSource.Costs.REMOTE_IO.getValue())));
269         }
270     }
271
272     @Override
273     public void onRemoteSessionDown() {
274         setConnected(false);
275         notificationHandler.onRemoteSchemaDown();
276
277         salFacade.onDeviceDisconnected();
278         sourceRegistrations.forEach(SchemaSourceRegistration::close);
279         sourceRegistrations.clear();
280         resetMessageTransformer();
281     }
282
283     @Override
284     public void onRemoteSessionFailed(final Throwable throwable) {
285         setConnected(false);
286         salFacade.onDeviceFailed(throwable);
287     }
288
289     @Override
290     public void onNotification(final NetconfMessage notification) {
291         notificationHandler.handleNotification(notification);
292     }
293
294     private static BaseSchema resolveBaseSchema(final boolean notificationSupport) {
295         return notificationSupport ? BaseSchema.BASE_NETCONF_CTX_WITH_NOTIFICATIONS : BaseSchema.BASE_NETCONF_CTX;
296     }
297
298     /**
299      * Just a transfer object containing schema related dependencies. Injected in constructor.
300      */
301     public static class SchemaResourcesDTO {
302         private final SchemaSourceRegistry schemaRegistry;
303         private final SchemaRepository schemaRepository;
304         private final SchemaContextFactory schemaContextFactory;
305         private final NetconfDeviceSchemasResolver stateSchemasResolver;
306
307         public SchemaResourcesDTO(final SchemaSourceRegistry schemaRegistry,
308                                   final SchemaRepository schemaRepository,
309                                   final SchemaContextFactory schemaContextFactory,
310                                   final NetconfDeviceSchemasResolver deviceSchemasResolver) {
311             this.schemaRegistry = requireNonNull(schemaRegistry);
312             this.schemaRepository = requireNonNull(schemaRepository);
313             this.schemaContextFactory = requireNonNull(schemaContextFactory);
314             this.stateSchemasResolver = requireNonNull(deviceSchemasResolver);
315         }
316
317         public SchemaSourceRegistry getSchemaRegistry() {
318             return schemaRegistry;
319         }
320
321         public SchemaRepository getSchemaRepository() {
322             return schemaRepository;
323         }
324
325         public SchemaContextFactory getSchemaContextFactory() {
326             return schemaContextFactory;
327         }
328
329         public NetconfDeviceSchemasResolver getStateSchemasResolver() {
330             return stateSchemasResolver;
331         }
332     }
333
334     /**
335      * Schema building callable.
336      */
337     private static class DeviceSourcesResolver implements Callable<DeviceSources> {
338
339         private final NetconfDeviceRpc deviceRpc;
340         private final NetconfSessionPreferences remoteSessionCapabilities;
341         private final RemoteDeviceId id;
342         private final NetconfDeviceSchemasResolver stateSchemasResolver;
343         private final SchemaContext schemaContext;
344
345         DeviceSourcesResolver(final NetconfDeviceRpc deviceRpc,
346                               final NetconfSessionPreferences remoteSessionCapabilities,
347                               final RemoteDeviceId id, final NetconfDeviceSchemasResolver stateSchemasResolver,
348                               final SchemaContext schemaContext) {
349             this.deviceRpc = deviceRpc;
350             this.remoteSessionCapabilities = remoteSessionCapabilities;
351             this.id = id;
352             this.stateSchemasResolver = stateSchemasResolver;
353             this.schemaContext = schemaContext;
354         }
355
356         DeviceSourcesResolver(final NetconfSessionPreferences remoteSessionCapabilities, final RemoteDeviceId id,
357                                      final NetconfDeviceSchemasResolver stateSchemasResolver,
358                                      final NetconfDeviceRpc rpcForMonitoring, final SchemaContext schemaCtx) {
359             this(rpcForMonitoring, remoteSessionCapabilities, id, stateSchemasResolver, schemaCtx);
360         }
361
362         @Override
363         public DeviceSources call() {
364             final NetconfDeviceSchemas availableSchemas =
365                     stateSchemasResolver.resolve(deviceRpc, remoteSessionCapabilities, id, schemaContext);
366             LOG.debug("{}: Schemas exposed by ietf-netconf-monitoring: {}", id,
367                     availableSchemas.getAvailableYangSchemasQNames());
368
369             final Set<QName> requiredSources = Sets.newHashSet(remoteSessionCapabilities.getModuleBasedCaps());
370             final Set<QName> providedSources = availableSchemas.getAvailableYangSchemasQNames();
371
372             final Set<QName> requiredSourcesNotProvided = Sets.difference(requiredSources, providedSources);
373             if (!requiredSourcesNotProvided.isEmpty()) {
374                 LOG.warn("{}: Netconf device does not provide all yang models reported in hello message capabilities,"
375                         + " required but not provided: {}", id, requiredSourcesNotProvided);
376                 LOG.warn("{}: Attempting to build schema context from required sources", id);
377             }
378
379             // Here all the sources reported in netconf monitoring are merged with those reported in hello.
380             // It is necessary to perform this since submodules are not mentioned in hello but still required.
381             // This clashes with the option of a user to specify supported yang models manually in configuration
382             // for netconf-connector and as a result one is not able to fully override yang models of a device.
383             // It is only possible to add additional models.
384             final Set<QName> providedSourcesNotRequired = Sets.difference(providedSources, requiredSources);
385             if (!providedSourcesNotRequired.isEmpty()) {
386                 LOG.warn("{}: Netconf device provides additional yang models not reported in "
387                         + "hello message capabilities: {}", id, providedSourcesNotRequired);
388                 LOG.warn("{}: Adding provided but not required sources as required to prevent failures", id);
389                 LOG.debug("{}: Netconf device reported in hello: {}", id, requiredSources);
390                 requiredSources.addAll(providedSourcesNotRequired);
391             }
392
393             final SchemaSourceProvider<YangTextSchemaSource> sourceProvider;
394             if (availableSchemas instanceof LibraryModulesSchemas) {
395                 sourceProvider = new YangLibrarySchemaYangSourceProvider(id,
396                         ((LibraryModulesSchemas) availableSchemas).getAvailableModels());
397             } else {
398                 sourceProvider = new NetconfRemoteSchemaYangSourceProvider(id, deviceRpc);
399             }
400
401             return new DeviceSources(requiredSources, providedSources, sourceProvider);
402         }
403     }
404
405     /**
406      * Contains RequiredSources - sources from capabilities.
407      */
408     private static final class DeviceSources {
409         private final Set<QName> requiredSources;
410         private final Set<QName> providedSources;
411         private final SchemaSourceProvider<YangTextSchemaSource> sourceProvider;
412
413         DeviceSources(final Set<QName> requiredSources, final Set<QName> providedSources,
414                              final SchemaSourceProvider<YangTextSchemaSource> sourceProvider) {
415             this.requiredSources = requiredSources;
416             this.providedSources = providedSources;
417             this.sourceProvider = sourceProvider;
418         }
419
420         public Set<QName> getRequiredSourcesQName() {
421             return requiredSources;
422         }
423
424         public Set<QName> getProvidedSourcesQName() {
425             return providedSources;
426         }
427
428         public Collection<SourceIdentifier> getRequiredSources() {
429             return Collections2.transform(requiredSources, DeviceSources::toSourceId);
430         }
431
432         public Collection<SourceIdentifier> getProvidedSources() {
433             return Collections2.transform(providedSources, DeviceSources::toSourceId);
434         }
435
436         public SchemaSourceProvider<YangTextSchemaSource> getSourceProvider() {
437             return sourceProvider;
438         }
439
440         private static SourceIdentifier toSourceId(final QName input) {
441             return RevisionSourceIdentifier.create(input.getLocalName(), input.getRevision());
442         }
443     }
444
445     /**
446      * Schema builder that tries to build schema context from provided sources or biggest subset of it.
447      */
448     private final class SchemaSetup implements Runnable {
449         private final DeviceSources deviceSources;
450         private final NetconfSessionPreferences remoteSessionCapabilities;
451         private final RemoteDeviceCommunicator<NetconfMessage> listener;
452         private final NetconfDeviceCapabilities capabilities;
453
454         SchemaSetup(final DeviceSources deviceSources, final NetconfSessionPreferences remoteSessionCapabilities,
455                            final RemoteDeviceCommunicator<NetconfMessage> listener) {
456             this.deviceSources = deviceSources;
457             this.remoteSessionCapabilities = remoteSessionCapabilities;
458             this.listener = listener;
459             this.capabilities = remoteSessionCapabilities.getNetconfDeviceCapabilities();
460         }
461
462         @Override
463         public void run() {
464
465             final Collection<SourceIdentifier> requiredSources = deviceSources.getRequiredSources();
466             final Collection<SourceIdentifier> missingSources = filterMissingSources(requiredSources);
467
468             capabilities.addUnresolvedCapabilities(getQNameFromSourceIdentifiers(missingSources),
469                     UnavailableCapability.FailureReason.MissingSource);
470
471             requiredSources.removeAll(missingSources);
472             setUpSchema(requiredSources);
473         }
474
475         private Collection<SourceIdentifier> filterMissingSources(final Collection<SourceIdentifier> requiredSources) {
476             return requiredSources.parallelStream().filter(sourceIdentifier -> {
477                 try {
478                     schemaRepository.getSchemaSource(sourceIdentifier, YangTextSchemaSource.class).get();
479                     return false;
480                 } catch (InterruptedException | ExecutionException e) {
481                     return true;
482                 }
483             }).collect(Collectors.toList());
484         }
485
486         /**
487          * Build schema context, in case of success or final failure notify device.
488          *
489          * @param requiredSources required sources
490          */
491         @SuppressWarnings("checkstyle:IllegalCatch")
492         private void setUpSchema(Collection<SourceIdentifier> requiredSources) {
493             while (!requiredSources.isEmpty()) {
494                 LOG.trace("{}: Trying to build schema context from {}", id, requiredSources);
495                 try {
496                     final ListenableFuture<SchemaContext> schemaBuilderFuture = schemaContextFactory
497                             .createSchemaContext(requiredSources);
498                     final SchemaContext result = schemaBuilderFuture.get();
499                     LOG.debug("{}: Schema context built successfully from {}", id, requiredSources);
500                     final Collection<QName> filteredQNames = Sets.difference(deviceSources.getRequiredSourcesQName(),
501                             capabilities.getUnresolvedCapabilites().keySet());
502                     capabilities.addCapabilities(filteredQNames.stream().map(entry -> new AvailableCapabilityBuilder()
503                             .setCapability(entry.toString()).setCapabilityOrigin(
504                                     remoteSessionCapabilities.getModuleBasedCapsOrigin().get(entry)).build())
505                             .collect(Collectors.toList()));
506
507                     capabilities.addNonModuleBasedCapabilities(remoteSessionCapabilities
508                             .getNonModuleCaps().stream().map(entry -> new AvailableCapabilityBuilder()
509                                     .setCapability(entry).setCapabilityOrigin(
510                                             remoteSessionCapabilities.getNonModuleBasedCapsOrigin().get(entry)).build())
511                             .collect(Collectors.toList()));
512
513                     final MountPointContext mountContext = new EmptyMountPointContext(result);
514                     handleSalInitializationSuccess(mountContext, remoteSessionCapabilities,
515                         getDeviceSpecificRpc(mountContext), listener);
516                     return;
517                 } catch (final ExecutionException e) {
518                     // schemaBuilderFuture.checkedGet() throws only SchemaResolutionException
519                     // that might be wrapping a MissingSchemaSourceException so we need to look
520                     // at the cause of the exception to make sure we don't misinterpret it.
521                     final Throwable cause = e.getCause();
522
523                     if (cause instanceof MissingSchemaSourceException) {
524                         requiredSources = handleMissingSchemaSourceException(
525                                 requiredSources, (MissingSchemaSourceException) cause);
526                         continue;
527                     }
528                     if (cause instanceof SchemaResolutionException) {
529                         requiredSources = handleSchemaResolutionException(requiredSources,
530                             (SchemaResolutionException) cause);
531                     } else {
532                         handleSalInitializationFailure(e, listener);
533                         return;
534                     }
535                 } catch (final Exception e) {
536                     // unknown error, fail
537                     handleSalInitializationFailure(e, listener);
538                     return;
539                 }
540             }
541             // No more sources, fail or try to reconnect
542             if (nodeOptional != null && nodeOptional.getIgnoreMissingSchemaSources().isAllowed()) {
543                 eventExecutor.schedule(() -> {
544                     LOG.warn("Reconnection is allowed! This can lead to unexpected errors at runtime.");
545                     LOG.warn("{} : No more sources for schema context.", id);
546                     LOG.info("{} : Try to remount device.", id);
547                     onRemoteSessionDown();
548                     salFacade.onDeviceReconnected(remoteSessionCapabilities, node);
549                 }, nodeOptional.getIgnoreMissingSchemaSources().getReconnectTime(), TimeUnit.MILLISECONDS);
550             } else {
551                 final IllegalStateException cause =
552                         new IllegalStateException(id + ": No more sources for schema context");
553                 handleSalInitializationFailure(cause, listener);
554                 salFacade.onDeviceFailed(cause);
555             }
556         }
557
558         private Collection<SourceIdentifier> handleMissingSchemaSourceException(
559                 final Collection<SourceIdentifier> requiredSources, final MissingSchemaSourceException exception) {
560             // In case source missing, try without it
561             final SourceIdentifier missingSource = exception.getSourceId();
562             LOG.warn("{}: Unable to build schema context, missing source {}, will reattempt without it",
563                 id, missingSource);
564             LOG.debug("{}: Unable to build schema context, missing source {}, will reattempt without it",
565                 id, missingSource, exception);
566             final Collection<QName> qNameOfMissingSource =
567                 getQNameFromSourceIdentifiers(Sets.newHashSet(missingSource));
568             if (!qNameOfMissingSource.isEmpty()) {
569                 capabilities.addUnresolvedCapabilities(
570                         qNameOfMissingSource, UnavailableCapability.FailureReason.MissingSource);
571             }
572             return stripUnavailableSource(requiredSources, missingSource);
573         }
574
575         private Collection<SourceIdentifier> handleSchemaResolutionException(
576             final Collection<SourceIdentifier> requiredSources, final SchemaResolutionException resolutionException) {
577             // In case resolution error, try only with resolved sources
578             // There are two options why schema resolution exception occurred : unsatisfied imports or flawed model
579             // FIXME Do we really have assurance that these two cases cannot happen at once?
580             if (resolutionException.getFailedSource() != null) {
581                 // flawed model - exclude it
582                 final SourceIdentifier failedSourceId = resolutionException.getFailedSource();
583                 LOG.warn("{}: Unable to build schema context, failed to resolve source {}, will reattempt without it",
584                     id, failedSourceId);
585                 LOG.warn("{}: Unable to build schema context, failed to resolve source {}, will reattempt without it",
586                     id, failedSourceId, resolutionException);
587                 capabilities.addUnresolvedCapabilities(
588                         getQNameFromSourceIdentifiers(Collections.singleton(failedSourceId)),
589                         UnavailableCapability.FailureReason.UnableToResolve);
590                 return stripUnavailableSource(requiredSources, resolutionException.getFailedSource());
591             }
592             // unsatisfied imports
593             final Set<SourceIdentifier> unresolvedSources = resolutionException.getUnsatisfiedImports().keySet();
594             capabilities.addUnresolvedCapabilities(getQNameFromSourceIdentifiers(unresolvedSources),
595                 UnavailableCapability.FailureReason.UnableToResolve);
596             LOG.warn("{}: Unable to build schema context, unsatisfied imports {}, will reattempt with resolved only",
597                 id, resolutionException.getUnsatisfiedImports());
598             LOG.debug("{}: Unable to build schema context, unsatisfied imports {}, will reattempt with resolved only",
599                 id, resolutionException.getUnsatisfiedImports(), resolutionException);
600             return resolutionException.getResolvedSources();
601         }
602
603         protected NetconfDeviceRpc getDeviceSpecificRpc(final MountPointContext result) {
604             return new NetconfDeviceRpc(result.getSchemaContext(), listener,
605                 new NetconfMessageTransformer(result, true));
606         }
607
608         private Collection<SourceIdentifier> stripUnavailableSource(final Collection<SourceIdentifier> requiredSources,
609                                                                     final SourceIdentifier sourceIdToRemove) {
610             final LinkedList<SourceIdentifier> sourceIdentifiers = Lists.newLinkedList(requiredSources);
611             checkState(sourceIdentifiers.remove(sourceIdToRemove),
612                     "%s: Trying to remove %s from %s failed", id, sourceIdToRemove, requiredSources);
613             return sourceIdentifiers;
614         }
615
616         private Collection<QName> getQNameFromSourceIdentifiers(final Collection<SourceIdentifier> identifiers) {
617             final Collection<QName> qNames = Collections2.transform(identifiers, this::getQNameFromSourceIdentifier);
618
619             if (qNames.isEmpty()) {
620                 LOG.debug("{}: Unable to map any source identifiers to a capability reported by device : {}", id,
621                         identifiers);
622             }
623             return Collections2.filter(qNames, Predicates.notNull());
624         }
625
626         private QName getQNameFromSourceIdentifier(final SourceIdentifier identifier) {
627             // Required sources are all required and provided merged in DeviceSourcesResolver
628             for (final QName qname : deviceSources.getRequiredSourcesQName()) {
629                 if (!qname.getLocalName().equals(identifier.getName())) {
630                     continue;
631                 }
632
633                 if (identifier.getRevision().equals(qname.getRevision())) {
634                     return qname;
635                 }
636             }
637             LOG.warn("Unable to map identifier to a devices reported capability: {} Available: {}",identifier,
638                     deviceSources.getRequiredSourcesQName());
639             // return null since we cannot find the QName,
640             // this capability will be removed from required sources and not reported as unresolved-capability
641             return null;
642         }
643     }
644 }