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