Close device's source resolution on teardown
[netconf.git] / netconf / sal-netconf-connector / src / main / java / org / opendaylight / netconf / sal / connect / netconf / NetconfDevice.java
index 22fa4cb92ed49cbd25c38ed9fa119b3f814b0783..ca821798b1f724536fb3e46fa91329dc0becde13 100644 (file)
@@ -7,35 +7,41 @@
  */
 package org.opendaylight.netconf.sal.connect.netconf;
 
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
+import static com.google.common.base.Preconditions.checkState;
+import static java.util.Objects.requireNonNull;
+import static org.opendaylight.netconf.sal.connect.netconf.util.NetconfMessageTransformUtil.NETCONF_GET_NODEID;
+
 import com.google.common.base.Predicates;
 import com.google.common.collect.Collections2;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.CheckedFuture;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
+import com.google.common.util.concurrent.SettableFuture;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import io.netty.util.concurrent.EventExecutor;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
-import javax.annotation.concurrent.GuardedBy;
-import org.opendaylight.controller.md.sal.dom.api.DOMNotification;
-import org.opendaylight.controller.md.sal.dom.api.DOMRpcException;
-import org.opendaylight.controller.md.sal.dom.api.DOMRpcResult;
-import org.opendaylight.controller.md.sal.dom.api.DOMRpcService;
+import org.checkerframework.checker.lock.qual.GuardedBy;
+import org.opendaylight.mdsal.dom.api.DOMRpcResult;
+import org.opendaylight.mdsal.dom.api.DOMRpcService;
 import org.opendaylight.netconf.api.NetconfMessage;
+import org.opendaylight.netconf.api.xml.XmlNetconfConstants;
+import org.opendaylight.netconf.sal.connect.api.DeviceActionFactory;
 import org.opendaylight.netconf.sal.connect.api.MessageTransformer;
-import org.opendaylight.netconf.sal.connect.api.NetconfDeviceSchemas;
 import org.opendaylight.netconf.sal.connect.api.NetconfDeviceSchemasResolver;
 import org.opendaylight.netconf.sal.connect.api.RemoteDevice;
 import org.opendaylight.netconf.sal.connect.api.RemoteDeviceCommunicator;
@@ -44,31 +50,37 @@ import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfDeviceCapabi
 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfDeviceCommunicator;
 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfSessionPreferences;
 import org.opendaylight.netconf.sal.connect.netconf.sal.NetconfDeviceRpc;
-import org.opendaylight.netconf.sal.connect.netconf.schema.NetconfRemoteSchemaYangSourceProvider;
-import org.opendaylight.netconf.sal.connect.netconf.schema.YangLibrarySchemaYangSourceProvider;
+import org.opendaylight.netconf.sal.connect.netconf.schema.mapping.BaseNetconfSchemas;
 import org.opendaylight.netconf.sal.connect.netconf.schema.mapping.BaseSchema;
 import org.opendaylight.netconf.sal.connect.netconf.schema.mapping.NetconfMessageTransformer;
 import org.opendaylight.netconf.sal.connect.netconf.util.NetconfMessageTransformUtil;
 import org.opendaylight.netconf.sal.connect.util.RemoteDeviceId;
 import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.netconf.notifications.rev120206.NetconfCapabilityChange;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.optional.rev190614.NetconfNodeAugmentedOptional;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.NetconfNode;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.connection.status.available.capabilities.AvailableCapabilityBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.connection.status.unavailable.capabilities.UnavailableCapability;
+import org.opendaylight.yangtools.rfc8528.data.api.MountPointContext;
+import org.opendaylight.yangtools.rfc8528.data.util.EmptyMountPointContext;
+import org.opendaylight.yangtools.rfc8528.model.api.SchemaMountConstants;
 import org.opendaylight.yangtools.yang.common.QName;
-import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.opendaylight.yangtools.yang.common.RpcError;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.NodeIdentifier;
+import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
+import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
+import org.opendaylight.yangtools.yang.data.impl.schema.Builders;
+import org.opendaylight.yangtools.yang.model.api.EffectiveModelContext;
+import org.opendaylight.yangtools.yang.model.repo.api.EffectiveModelContextFactory;
 import org.opendaylight.yangtools.yang.model.repo.api.MissingSchemaSourceException;
-import org.opendaylight.yangtools.yang.model.repo.api.RevisionSourceIdentifier;
-import org.opendaylight.yangtools.yang.model.repo.api.SchemaContextFactory;
 import org.opendaylight.yangtools.yang.model.repo.api.SchemaRepository;
 import org.opendaylight.yangtools.yang.model.repo.api.SchemaResolutionException;
-import org.opendaylight.yangtools.yang.model.repo.api.SchemaSourceException;
-import org.opendaylight.yangtools.yang.model.repo.api.SchemaSourceRepresentation;
 import org.opendaylight.yangtools.yang.model.repo.api.SourceIdentifier;
 import org.opendaylight.yangtools.yang.model.repo.api.YangTextSchemaSource;
 import org.opendaylight.yangtools.yang.model.repo.spi.PotentialSchemaSource;
 import org.opendaylight.yangtools.yang.model.repo.spi.SchemaSourceProvider;
 import org.opendaylight.yangtools.yang.model.repo.spi.SchemaSourceRegistration;
 import org.opendaylight.yangtools.yang.model.repo.spi.SchemaSourceRegistry;
-import org.opendaylight.yangtools.yang.parser.util.ASTSchemaSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -78,57 +90,72 @@ import org.slf4j.LoggerFactory;
 public class NetconfDevice
         implements RemoteDevice<NetconfSessionPreferences, NetconfMessage, NetconfDeviceCommunicator> {
 
-    private static final Logger LOG = LoggerFactory.getLogger(NetconfDevice.class);
+    @SuppressFBWarnings(value = "SLF4J_LOGGER_SHOULD_BE_PRIVATE",
+            justification = "Needed for common logging of related classes")
+    static final Logger LOG = LoggerFactory.getLogger(NetconfDevice.class);
+
+    private static final QName RFC8528_SCHEMA_MOUNTS_QNAME = QName.create(
+        SchemaMountConstants.RFC8528_MODULE, "schema-mounts").intern();
+    private static final YangInstanceIdentifier RFC8528_SCHEMA_MOUNTS = YangInstanceIdentifier.create(
+        NodeIdentifier.create(RFC8528_SCHEMA_MOUNTS_QNAME));
 
     protected final RemoteDeviceId id;
-    private final boolean reconnectOnSchemasChange;
+    protected final EffectiveModelContextFactory schemaContextFactory;
+    protected final SchemaSourceRegistry schemaRegistry;
+    protected final SchemaRepository schemaRepository;
+
+    protected final List<SchemaSourceRegistration<?>> sourceRegistrations = new ArrayList<>();
 
-    protected final SchemaContextFactory schemaContextFactory;
     private final RemoteDeviceHandler<NetconfSessionPreferences> salFacade;
     private final ListeningExecutorService processingExecutor;
-    protected final SchemaSourceRegistry schemaRegistry;
-    protected final SchemaRepository schemaRepository;
+    private final DeviceActionFactory deviceActionFactory;
     private final NetconfDeviceSchemasResolver stateSchemasResolver;
     private final NotificationHandler notificationHandler;
-    protected final List<SchemaSourceRegistration<? extends SchemaSourceRepresentation>> sourceRegistrations =
-            new ArrayList<>();
+    private final boolean reconnectOnSchemasChange;
+    private final BaseNetconfSchemas baseSchemas;
+    private final NetconfNode node;
+    private final EventExecutor eventExecutor;
+    private final NetconfNodeAugmentedOptional nodeOptional;
+
+    @GuardedBy("this")
+    private ListenableFuture<List<Object>> schemaFuturesList;
     @GuardedBy("this")
     private boolean connected = false;
 
     // Message transformer is constructed once the schemas are available
     private MessageTransformer<NetconfMessage> messageTransformer;
 
-    /**
-     * Create rpc implementation capable of handling RPC for monitoring and notifications
-     * even before the schemas of remote device are downloaded.
-     */
-    static NetconfDeviceRpc getRpcForInitialization(final NetconfDeviceCommunicator listener,
-                                                    final boolean notificationSupport) {
-        final BaseSchema baseSchema = notificationSupport
-                ? BaseSchema.BASE_NETCONF_CTX_WITH_NOTIFICATIONS
-                : BaseSchema.BASE_NETCONF_CTX;
-
-        return new NetconfDeviceRpc(baseSchema.getSchemaContext(), listener,
-                new NetconfMessageTransformer(baseSchema.getSchemaContext(), false, baseSchema));
+    public NetconfDevice(final SchemaResourcesDTO schemaResourcesDTO, final BaseNetconfSchemas baseSchemas,
+            final RemoteDeviceId id, final RemoteDeviceHandler<NetconfSessionPreferences> salFacade,
+            final ListeningExecutorService globalProcessingExecutor, final boolean reconnectOnSchemasChange) {
+        this(schemaResourcesDTO, baseSchemas, id, salFacade, globalProcessingExecutor, reconnectOnSchemasChange, null,
+            null, null, null);
     }
 
-    public NetconfDevice(final SchemaResourcesDTO schemaResourcesDTO, final RemoteDeviceId id,
-                         final RemoteDeviceHandler<NetconfSessionPreferences> salFacade,
-                         final ExecutorService globalProcessingExecutor, final boolean reconnectOnSchemasChange) {
+    public NetconfDevice(final SchemaResourcesDTO schemaResourcesDTO, final BaseNetconfSchemas baseSchemas,
+            final RemoteDeviceId id, final RemoteDeviceHandler<NetconfSessionPreferences> salFacade,
+            final ListeningExecutorService globalProcessingExecutor, final boolean reconnectOnSchemasChange,
+            final DeviceActionFactory deviceActionFactory, final NetconfNode node, final EventExecutor eventExecutor,
+            final NetconfNodeAugmentedOptional nodeOptional) {
+        this.baseSchemas = requireNonNull(baseSchemas);
         this.id = id;
         this.reconnectOnSchemasChange = reconnectOnSchemasChange;
-        this.schemaRegistry = schemaResourcesDTO.getSchemaRegistry();
-        this.schemaRepository = schemaResourcesDTO.getSchemaRepository();
-        this.schemaContextFactory = schemaResourcesDTO.getSchemaContextFactory();
+        this.deviceActionFactory = deviceActionFactory;
+        this.node = node;
+        this.eventExecutor = eventExecutor;
+        this.nodeOptional = nodeOptional;
+        schemaRegistry = schemaResourcesDTO.getSchemaRegistry();
+        schemaRepository = schemaResourcesDTO.getSchemaRepository();
+        schemaContextFactory = schemaResourcesDTO.getSchemaContextFactory();
         this.salFacade = salFacade;
-        this.stateSchemasResolver = schemaResourcesDTO.getStateSchemasResolver();
-        this.processingExecutor = MoreExecutors.listeningDecorator(globalProcessingExecutor);
-        this.notificationHandler = new NotificationHandler(salFacade, id);
+        stateSchemasResolver = schemaResourcesDTO.getStateSchemasResolver();
+        processingExecutor = requireNonNull(globalProcessingExecutor);
+        notificationHandler = new NotificationHandler(salFacade, id);
     }
 
     @Override
-    public void onRemoteSessionUp(final NetconfSessionPreferences remoteSessionCapabilities,
-                                  final NetconfDeviceCommunicator listener) {
+    public synchronized void onRemoteSessionUp(final NetconfSessionPreferences remoteSessionCapabilities,
+            final NetconfDeviceCommunicator listener) {
         // SchemaContext setup has to be performed in a dedicated thread since
         // we are in a netty thread in this method
         // Yang models are being downloaded in this method and it would cause a
@@ -137,35 +164,59 @@ public class NetconfDevice
         setConnected(true);
         LOG.debug("{}: Session to remote device established with {}", id, remoteSessionCapabilities);
 
-        final NetconfDeviceRpc initRpc =
-                getRpcForInitialization(listener, remoteSessionCapabilities.isNotificationsSupported());
-        final DeviceSourcesResolver task =
-                new DeviceSourcesResolver(remoteSessionCapabilities, id, stateSchemasResolver, initRpc);
-        final ListenableFuture<DeviceSources> sourceResolverFuture = processingExecutor.submit(task);
+        final BaseSchema baseSchema = resolveBaseSchema(remoteSessionCapabilities.isNotificationsSupported());
+        final NetconfDeviceRpc initRpc = new NetconfDeviceRpc(baseSchema.getEffectiveModelContext(), listener,
+            new NetconfMessageTransformer(baseSchema.getMountPointContext(), false, baseSchema));
+        final ListenableFuture<DeviceSources> sourceResolverFuture = processingExecutor.submit(
+            new DeviceSourcesResolver(id, baseSchema, initRpc, remoteSessionCapabilities, stateSchemasResolver));
 
         if (shouldListenOnSchemaChange(remoteSessionCapabilities)) {
             registerToBaseNetconfStream(initRpc, listener);
         }
 
-        final FutureCallback<DeviceSources> resolvedSourceCallback = new FutureCallback<DeviceSources>() {
-            @Override
-            public void onSuccess(final DeviceSources result) {
-                addProvidedSourcesToSchemaRegistry(result);
-                setUpSchema(result);
-            }
+        // Set up the SchemaContext for the device
+        final ListenableFuture<EffectiveModelContext> futureSchema = Futures.transformAsync(sourceResolverFuture,
+            deviceSources -> assembleSchemaContext(deviceSources, remoteSessionCapabilities), processingExecutor);
 
-            private void setUpSchema(final DeviceSources result) {
-                processingExecutor.submit(new SchemaSetup(result, remoteSessionCapabilities, listener));
-            }
+        // Potentially acquire mount point list and interpret it
+        final ListenableFuture<MountPointContext> futureContext = Futures.transformAsync(futureSchema,
+            schemaContext -> createMountPointContext(schemaContext, baseSchema, listener), processingExecutor);
+        schemaFuturesList = Futures.allAsList(sourceResolverFuture, futureSchema, futureContext);
 
+        Futures.addCallback(futureContext, new FutureCallback<MountPointContext>() {
             @Override
-            public void onFailure(final Throwable throwable) {
-                LOG.warn("{}: Unexpected error resolving device sources: {}", id, throwable);
-                handleSalInitializationFailure(throwable, listener);
+            public void onSuccess(final MountPointContext result) {
+                handleSalInitializationSuccess(result, remoteSessionCapabilities,
+                        getDeviceSpecificRpc(result, listener, baseSchema), listener);
             }
-        };
 
-        Futures.addCallback(sourceResolverFuture, resolvedSourceCallback, MoreExecutors.directExecutor());
+            @Override
+            public void onFailure(final Throwable cause) {
+                if (cause instanceof CancellationException) {
+                    LOG.warn("{}: Device communicator was tear down since the schema setup started", id);
+                    return;
+                }
+
+                LOG.warn("{}: Unexpected error resolving device sources", id, cause);
+                // No more sources, fail or try to reconnect
+                if (cause instanceof EmptySchemaContextException) {
+                    if (nodeOptional != null && nodeOptional.getIgnoreMissingSchemaSources().getAllowed()) {
+                        eventExecutor.schedule(() -> {
+                            LOG.warn("Reconnection is allowed! This can lead to unexpected errors at runtime.");
+                            LOG.warn("{} : No more sources for schema context.", id);
+                            LOG.info("{} : Try to remount device.", id);
+                            onRemoteSessionDown();
+                            salFacade.onDeviceReconnected(remoteSessionCapabilities, node);
+                        }, nodeOptional.getIgnoreMissingSchemaSources().getReconnectTime().toJava(),
+                            TimeUnit.MILLISECONDS);
+                        return;
+                    }
+                }
+
+                handleSalInitializationFailure(cause, listener);
+                salFacade.onDeviceFailed(cause);
+            }
+        }, MoreExecutors.directExecutor());
     }
 
     private void registerToBaseNetconfStream(final NetconfDeviceRpc deviceRpc,
@@ -173,32 +224,23 @@ public class NetconfDevice
         // TODO check whether the model describing create subscription is present in schema
         // 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)
-        final CheckedFuture<DOMRpcResult, DOMRpcException> rpcResultListenableFuture = deviceRpc.invokeRpc(
-                NetconfMessageTransformUtil.toPath(NetconfMessageTransformUtil.CREATE_SUBSCRIPTION_RPC_QNAME),
+        final ListenableFuture<DOMRpcResult> rpcResultListenableFuture = deviceRpc.invokeRpc(
+                NetconfMessageTransformUtil.CREATE_SUBSCRIPTION_RPC_QNAME,
                 NetconfMessageTransformUtil.CREATE_SUBSCRIPTION_RPC_CONTENT);
 
-        final NotificationHandler.NotificationFilter filter = new NotificationHandler.NotificationFilter() {
-            @Override
-            public Optional<DOMNotification> filterNotification(final DOMNotification notification) {
-                if (isCapabilityChanged(notification)) {
-                    LOG.info("{}: Schemas change detected, reconnecting", id);
-                    // Only disconnect is enough,
-                    // the reconnecting nature of the connector will take care of reconnecting
-                    listener.disconnect();
-                    return Optional.absent();
-                }
-                return Optional.of(notification);
-            }
-
-            private boolean isCapabilityChanged(final DOMNotification notification) {
-                return notification.getBody().getNodeType().equals(NetconfCapabilityChange.QNAME);
-            }
-        };
-
         Futures.addCallback(rpcResultListenableFuture, new FutureCallback<DOMRpcResult>() {
             @Override
             public void onSuccess(final DOMRpcResult domRpcResult) {
-                notificationHandler.addNotificationFilter(filter);
+                notificationHandler.addNotificationFilter(notification -> {
+                    if (NetconfCapabilityChange.QNAME.equals(notification.getBody().getIdentifier().getNodeType())) {
+                        LOG.info("{}: Schemas change detected, reconnecting", id);
+                        // Only disconnect is enough,
+                        // the reconnecting nature of the connector will take care of reconnecting
+                        listener.disconnect();
+                        return Optional.empty();
+                    }
+                    return Optional.of(notification);
+                });
             }
 
             @Override
@@ -213,20 +255,20 @@ public class NetconfDevice
         return remoteSessionCapabilities.isNotificationsSupported() && reconnectOnSchemasChange;
     }
 
-    private synchronized void handleSalInitializationSuccess(final SchemaContext result,
+    private synchronized void handleSalInitializationSuccess(final MountPointContext result,
                                         final NetconfSessionPreferences remoteSessionCapabilities,
-                                        final DOMRpcService deviceRpc) {
+                                        final DOMRpcService deviceRpc,
+                                        final RemoteDeviceCommunicator<NetconfMessage> listener) {
         //NetconfDevice.SchemaSetup can complete after NetconfDeviceCommunicator was closed. In that case do nothing,
         //since salFacade.onDeviceDisconnected was already called.
         if (connected) {
-            final BaseSchema baseSchema =
-                remoteSessionCapabilities.isNotificationsSupported()
-                        ? BaseSchema.BASE_NETCONF_CTX_WITH_NOTIFICATIONS : BaseSchema.BASE_NETCONF_CTX;
-            messageTransformer = new NetconfMessageTransformer(result, true, baseSchema);
+            messageTransformer = new NetconfMessageTransformer(result, true,
+                resolveBaseSchema(remoteSessionCapabilities.isNotificationsSupported()));
 
-            updateTransformer(messageTransformer);
             // salFacade.onDeviceConnected has to be called before the notification handler is initialized
-            salFacade.onDeviceConnected(result, remoteSessionCapabilities, deviceRpc);
+            salFacade.onDeviceConnected(result, remoteSessionCapabilities, deviceRpc,
+                    deviceActionFactory == null ? null : deviceActionFactory.createDeviceAction(
+                            messageTransformer, listener, result.getEffectiveModelContext()));
             notificationHandler.onRemoteSchemaUp(messageTransformer);
 
             LOG.info("{}: Netconf connector initialized successfully", id);
@@ -250,7 +292,7 @@ public class NetconfDevice
         updateTransformer(null);
     }
 
-    private void updateTransformer(final MessageTransformer<NetconfMessage> transformer) {
+    private synchronized void updateTransformer(final MessageTransformer<NetconfMessage> transformer) {
         messageTransformer = transformer;
     }
 
@@ -258,25 +300,67 @@ public class NetconfDevice
         this.connected = connected;
     }
 
-    private void addProvidedSourcesToSchemaRegistry(final DeviceSources deviceSources) {
+    private ListenableFuture<EffectiveModelContext> assembleSchemaContext(final DeviceSources deviceSources,
+            final NetconfSessionPreferences remoteSessionCapabilities) {
+        LOG.debug("{}: Resolved device sources to {}", id, deviceSources);
         final SchemaSourceProvider<YangTextSchemaSource> yangProvider = deviceSources.getSourceProvider();
         for (final SourceIdentifier sourceId : deviceSources.getProvidedSources()) {
             sourceRegistrations.add(schemaRegistry.registerSchemaSource(yangProvider,
-                    PotentialSchemaSource.create(
-                            sourceId, YangTextSchemaSource.class, PotentialSchemaSource.Costs.REMOTE_IO.getValue())));
+                PotentialSchemaSource.create(sourceId, YangTextSchemaSource.class,
+                    PotentialSchemaSource.Costs.REMOTE_IO.getValue())));
+        }
+
+        return new SchemaSetup(deviceSources, remoteSessionCapabilities).startResolution();
+    }
+
+    private ListenableFuture<MountPointContext> createMountPointContext(final EffectiveModelContext schemaContext,
+            final BaseSchema baseSchema, final NetconfDeviceCommunicator listener) {
+        final MountPointContext emptyContext = new EmptyMountPointContext(schemaContext);
+        if (schemaContext.findModule(SchemaMountConstants.RFC8528_MODULE).isEmpty()) {
+            return Futures.immediateFuture(emptyContext);
         }
+
+        // Create a temporary RPC invoker and acquire the mount point tree
+        LOG.debug("{}: Acquiring available mount points", id);
+        final NetconfDeviceRpc deviceRpc = new NetconfDeviceRpc(schemaContext, listener,
+            new NetconfMessageTransformer(emptyContext, false, baseSchema));
+
+        return Futures.transform(deviceRpc.invokeRpc(NetconfMessageTransformUtil.NETCONF_GET_QNAME,
+            Builders.containerBuilder().withNodeIdentifier(NETCONF_GET_NODEID)
+                .withChild(NetconfMessageTransformUtil.toFilterStructure(RFC8528_SCHEMA_MOUNTS, schemaContext))
+                .build()), rpcResult -> processSchemaMounts(rpcResult, emptyContext), MoreExecutors.directExecutor());
+    }
+
+    private MountPointContext processSchemaMounts(final DOMRpcResult rpcResult, final MountPointContext emptyContext) {
+        final Collection<? extends RpcError> errors = rpcResult.getErrors();
+        if (!errors.isEmpty()) {
+            LOG.warn("{}: Schema-mounts acquisition resulted in errors {}", id, errors);
+        }
+        final NormalizedNode schemaMounts = rpcResult.getResult();
+        if (schemaMounts == null) {
+            LOG.debug("{}: device does not define any schema mounts", id);
+            return emptyContext;
+        }
+        if (!(schemaMounts instanceof ContainerNode)) {
+            LOG.warn("{}: ignoring non-container schema mounts {}", id, schemaMounts);
+            return emptyContext;
+        }
+
+        return DeviceMountPointContext.create(emptyContext, (ContainerNode) schemaMounts);
     }
 
     @Override
-    public void onRemoteSessionDown() {
+    public synchronized void onRemoteSessionDown() {
         setConnected(false);
+        if (schemaFuturesList != null && !schemaFuturesList.isDone()) {
+            if (!schemaFuturesList.cancel(true)) {
+                LOG.warn("The cleanup of Schema Futures for device {} was unsuccessful.", id);
+            }
+        }
         notificationHandler.onRemoteSchemaDown();
-
         salFacade.onDeviceDisconnected();
-        for (final SchemaSourceRegistration<? extends SchemaSourceRepresentation> sourceRegistration
-                : sourceRegistrations) {
-            sourceRegistration.close();
-        }
+        sourceRegistrations.forEach(SchemaSourceRegistration::close);
+        sourceRegistrations.clear();
         resetMessageTransformer();
     }
 
@@ -291,23 +375,33 @@ public class NetconfDevice
         notificationHandler.handleNotification(notification);
     }
 
+    private BaseSchema resolveBaseSchema(final boolean notificationSupport) {
+        return notificationSupport ? baseSchemas.getBaseSchemaWithNotifications() : baseSchemas.getBaseSchema();
+    }
+
+    protected NetconfDeviceRpc getDeviceSpecificRpc(final MountPointContext result,
+            final RemoteDeviceCommunicator<NetconfMessage> listener, final BaseSchema schema) {
+        return new NetconfDeviceRpc(result.getEffectiveModelContext(), listener,
+            new NetconfMessageTransformer(result, true, schema));
+    }
+
     /**
      * Just a transfer object containing schema related dependencies. Injected in constructor.
      */
     public static class SchemaResourcesDTO {
         private final SchemaSourceRegistry schemaRegistry;
         private final SchemaRepository schemaRepository;
-        private final SchemaContextFactory schemaContextFactory;
+        private final EffectiveModelContextFactory schemaContextFactory;
         private final NetconfDeviceSchemasResolver stateSchemasResolver;
 
         public SchemaResourcesDTO(final SchemaSourceRegistry schemaRegistry,
                                   final SchemaRepository schemaRepository,
-                                  final SchemaContextFactory schemaContextFactory,
+                                  final EffectiveModelContextFactory schemaContextFactory,
                                   final NetconfDeviceSchemasResolver deviceSchemasResolver) {
-            this.schemaRegistry = Preconditions.checkNotNull(schemaRegistry);
-            this.schemaRepository = Preconditions.checkNotNull(schemaRepository);
-            this.schemaContextFactory = Preconditions.checkNotNull(schemaContextFactory);
-            this.stateSchemasResolver = Preconditions.checkNotNull(deviceSchemasResolver);
+            this.schemaRegistry = requireNonNull(schemaRegistry);
+            this.schemaRepository = requireNonNull(schemaRepository);
+            this.schemaContextFactory = requireNonNull(schemaContextFactory);
+            stateSchemasResolver = requireNonNull(deviceSchemasResolver);
         }
 
         public SchemaSourceRegistry getSchemaRegistry() {
@@ -318,7 +412,7 @@ public class NetconfDevice
             return schemaRepository;
         }
 
-        public SchemaContextFactory getSchemaContextFactory() {
+        public EffectiveModelContextFactory getSchemaContextFactory() {
             return schemaContextFactory;
         }
 
@@ -328,225 +422,142 @@ public class NetconfDevice
     }
 
     /**
-     * Schema building callable.
+     * A dedicated exception to indicate when we fail to setup a SchemaContext.
+     *
+     * @author Robert Varga
      */
-    private static class DeviceSourcesResolver implements Callable<DeviceSources> {
+    private static final class EmptySchemaContextException extends Exception {
+        private static final long serialVersionUID = 1L;
 
-        private final NetconfDeviceRpc deviceRpc;
-        private final NetconfSessionPreferences remoteSessionCapabilities;
-        private final RemoteDeviceId id;
-        private final NetconfDeviceSchemasResolver stateSchemasResolver;
-
-        DeviceSourcesResolver(final NetconfDeviceRpc deviceRpc,
-                              final NetconfSessionPreferences remoteSessionCapabilities,
-                              final RemoteDeviceId id, final NetconfDeviceSchemasResolver stateSchemasResolver) {
-            this.deviceRpc = deviceRpc;
-            this.remoteSessionCapabilities = remoteSessionCapabilities;
-            this.id = id;
-            this.stateSchemasResolver = stateSchemasResolver;
-        }
-
-        DeviceSourcesResolver(final NetconfSessionPreferences remoteSessionCapabilities, final RemoteDeviceId id,
-                                     final NetconfDeviceSchemasResolver stateSchemasResolver,
-                                     final NetconfDeviceRpc rpcForMonitoring) {
-            this(rpcForMonitoring, remoteSessionCapabilities, id, stateSchemasResolver);
-        }
-
-        @Override
-        public DeviceSources call() throws Exception {
-            final NetconfDeviceSchemas availableSchemas =
-                    stateSchemasResolver.resolve(deviceRpc, remoteSessionCapabilities, id);
-            LOG.debug("{}: Schemas exposed by ietf-netconf-monitoring: {}", id,
-                    availableSchemas.getAvailableYangSchemasQNames());
-
-            final Set<QName> requiredSources = Sets.newHashSet(remoteSessionCapabilities.getModuleBasedCaps());
-            final Set<QName> providedSources = availableSchemas.getAvailableYangSchemasQNames();
-
-            final Set<QName> requiredSourcesNotProvided = Sets.difference(requiredSources, providedSources);
-            if (!requiredSourcesNotProvided.isEmpty()) {
-                LOG.warn("{}: Netconf device does not provide all yang models reported in hello message capabilities,"
-                        + " required but not provided: {}", id, requiredSourcesNotProvided);
-                LOG.warn("{}: Attempting to build schema context from required sources", id);
-            }
-
-            // Here all the sources reported in netconf monitoring are merged with those reported in hello.
-            // It is necessary to perform this since submodules are not mentioned in hello but still required.
-            // This clashes with the option of a user to specify supported yang models manually in configuration
-            // for netconf-connector and as a result one is not able to fully override yang models of a device.
-            // It is only possible to add additional models.
-            final Set<QName> providedSourcesNotRequired = Sets.difference(providedSources, requiredSources);
-            if (!providedSourcesNotRequired.isEmpty()) {
-                LOG.warn("{}: Netconf device provides additional yang models not reported in "
-                        + "hello message capabilities: {}", id, providedSourcesNotRequired);
-                LOG.warn("{}: Adding provided but not required sources as required to prevent failures", id);
-                LOG.debug("{}: Netconf device reported in hello: {}", id, requiredSources);
-                requiredSources.addAll(providedSourcesNotRequired);
-            }
-
-            final SchemaSourceProvider<YangTextSchemaSource> sourceProvider;
-            if (availableSchemas instanceof LibraryModulesSchemas) {
-                sourceProvider = new YangLibrarySchemaYangSourceProvider(id,
-                        ((LibraryModulesSchemas) availableSchemas).getAvailableModels());
-            } else {
-                sourceProvider = new NetconfRemoteSchemaYangSourceProvider(id, deviceRpc);
-            }
-
-            return new DeviceSources(requiredSources, providedSources, sourceProvider);
-        }
-    }
-
-    /**
-     * Contains RequiredSources - sources from capabilities.
-     */
-    private static final class DeviceSources {
-        private final Set<QName> requiredSources;
-        private final Set<QName> providedSources;
-        private final SchemaSourceProvider<YangTextSchemaSource> sourceProvider;
-
-        DeviceSources(final Set<QName> requiredSources, final Set<QName> providedSources,
-                             final SchemaSourceProvider<YangTextSchemaSource> sourceProvider) {
-            this.requiredSources = requiredSources;
-            this.providedSources = providedSources;
-            this.sourceProvider = sourceProvider;
-        }
-
-        public Set<QName> getRequiredSourcesQName() {
-            return requiredSources;
-        }
-
-        public Set<QName> getProvidedSourcesQName() {
-            return providedSources;
-        }
-
-        public Collection<SourceIdentifier> getRequiredSources() {
-            return Collections2.transform(requiredSources, DeviceSources::toSourceId);
-        }
-
-        public Collection<SourceIdentifier> getProvidedSources() {
-            return Collections2.transform(providedSources, DeviceSources::toSourceId);
-        }
-
-        public SchemaSourceProvider<YangTextSchemaSource> getSourceProvider() {
-            return sourceProvider;
-        }
-
-        private static SourceIdentifier toSourceId(final QName input) {
-            return RevisionSourceIdentifier.create(input.getLocalName(),
-                Optional.fromNullable(input.getFormattedRevision()));
+        EmptySchemaContextException(final String message) {
+            super(message);
         }
     }
 
     /**
      * Schema builder that tries to build schema context from provided sources or biggest subset of it.
      */
-    private final class SchemaSetup implements Runnable {
+    private final class SchemaSetup implements FutureCallback<EffectiveModelContext> {
+        private final SettableFuture<EffectiveModelContext> resultFuture = SettableFuture.create();
+
         private final DeviceSources deviceSources;
         private final NetconfSessionPreferences remoteSessionCapabilities;
-        private final RemoteDeviceCommunicator<NetconfMessage> listener;
         private final NetconfDeviceCapabilities capabilities;
 
-        SchemaSetup(final DeviceSources deviceSources, final NetconfSessionPreferences remoteSessionCapabilities,
-                           final RemoteDeviceCommunicator<NetconfMessage> listener) {
+        private Collection<SourceIdentifier> requiredSources;
+
+        SchemaSetup(final DeviceSources deviceSources, final NetconfSessionPreferences remoteSessionCapabilities) {
             this.deviceSources = deviceSources;
             this.remoteSessionCapabilities = remoteSessionCapabilities;
-            this.listener = listener;
-            this.capabilities = remoteSessionCapabilities.getNetconfDeviceCapabilities();
-        }
-
-        @Override
-        public void run() {
+            capabilities = remoteSessionCapabilities.getNetconfDeviceCapabilities();
+
+            // If device supports notifications and does not contain necessary modules, add them automatically
+            if (remoteSessionCapabilities.containsNonModuleCapability(
+                    XmlNetconfConstants.URN_IETF_PARAMS_NETCONF_CAPABILITY_NOTIFICATION_1_0)) {
+                deviceSources.getRequiredSourcesQName().addAll(
+                        Arrays.asList(
+                                org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.netconf.notification._1._0.rev080714
+                                        .$YangModuleInfoImpl.getInstance().getName(),
+                                org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.yang.types.rev130715
+                                        .$YangModuleInfoImpl.getInstance().getName()
+                        )
+                );
+            }
 
-            final Collection<SourceIdentifier> requiredSources = deviceSources.getRequiredSources();
+            requiredSources = deviceSources.getRequiredSources();
             final Collection<SourceIdentifier> missingSources = filterMissingSources(requiredSources);
 
             capabilities.addUnresolvedCapabilities(getQNameFromSourceIdentifiers(missingSources),
                     UnavailableCapability.FailureReason.MissingSource);
-
             requiredSources.removeAll(missingSources);
-            setUpSchema(requiredSources);
         }
 
-        private Collection<SourceIdentifier> filterMissingSources(final Collection<SourceIdentifier> requiredSources) {
+        ListenableFuture<EffectiveModelContext> startResolution() {
+            trySetupSchema();
+            return resultFuture;
+        }
 
-            return requiredSources.parallelStream().filter(sourceIdentifier -> {
-                boolean remove = false;
-                try {
-                    schemaRepository.getSchemaSource(sourceIdentifier, ASTSchemaSource.class).checkedGet();
-                } catch (SchemaSourceException e) {
-                    remove = true;
-                }
-                return remove;
-            }).collect(Collectors.toList());
+        @Override
+        public void onSuccess(final EffectiveModelContext result) {
+            LOG.debug("{}: Schema context built successfully from {}", id, requiredSources);
+
+            final Collection<QName> filteredQNames = Sets.difference(deviceSources.getRequiredSourcesQName(),
+                    capabilities.getUnresolvedCapabilites().keySet());
+            capabilities.addCapabilities(filteredQNames.stream().map(entry -> new AvailableCapabilityBuilder()
+                    .setCapability(entry.toString()).setCapabilityOrigin(
+                            remoteSessionCapabilities.getModuleBasedCapsOrigin().get(entry)).build())
+                    .collect(Collectors.toList()));
+
+            capabilities.addNonModuleBasedCapabilities(remoteSessionCapabilities
+                    .getNonModuleCaps().stream().map(entry -> new AvailableCapabilityBuilder()
+                            .setCapability(entry).setCapabilityOrigin(
+                                    remoteSessionCapabilities.getNonModuleBasedCapsOrigin().get(entry)).build())
+                    .collect(Collectors.toList()));
+
+            resultFuture.set(result);
         }
 
-        /**
-         * Build schema context, in case of success or final failure notify device.
-         */
-        @SuppressWarnings("checkstyle:IllegalCatch")
-        private void setUpSchema(Collection<SourceIdentifier> requiredSources) {
-            while (!requiredSources.isEmpty()) {
+        @Override
+        public void onFailure(final Throwable cause) {
+            // schemaBuilderFuture.checkedGet() throws only SchemaResolutionException
+            // that might be wrapping a MissingSchemaSourceException so we need to look
+            // at the cause of the exception to make sure we don't misinterpret it.
+            if (cause instanceof MissingSchemaSourceException) {
+                requiredSources = handleMissingSchemaSourceException((MissingSchemaSourceException) cause);
+            } else if (cause instanceof SchemaResolutionException) {
+                requiredSources = handleSchemaResolutionException((SchemaResolutionException) cause);
+            } else {
+                LOG.debug("Unhandled failure", cause);
+                resultFuture.setException(cause);
+                // No more trying...
+                return;
+            }
+
+            trySetupSchema();
+        }
+
+        private void trySetupSchema() {
+            if (!requiredSources.isEmpty()) {
+                // Initiate async resolution, drive it back based on the result
                 LOG.trace("{}: Trying to build schema context from {}", id, requiredSources);
+                Futures.addCallback(schemaContextFactory.createEffectiveModelContext(requiredSources), this,
+                    MoreExecutors.directExecutor());
+            } else {
+                LOG.debug("{}: no more sources for schema context", id);
+                resultFuture.setException(new EmptySchemaContextException(id + ": No more sources for schema context"));
+            }
+        }
+
+        private Collection<SourceIdentifier> filterMissingSources(final Collection<SourceIdentifier> origSources) {
+            return origSources.parallelStream().filter(sourceIdentifier -> {
                 try {
-                    final CheckedFuture<SchemaContext, SchemaResolutionException> schemaBuilderFuture =
-                            schemaContextFactory.createSchemaContext(requiredSources);
-                    final SchemaContext result = schemaBuilderFuture.checkedGet();
-                    LOG.debug("{}: Schema context built successfully from {}", id, requiredSources);
-                    final Collection<QName> filteredQNames = Sets.difference(deviceSources.getRequiredSourcesQName(),
-                            capabilities.getUnresolvedCapabilites().keySet());
-                    capabilities.addCapabilities(filteredQNames.stream().map(entry -> new AvailableCapabilityBuilder()
-                            .setCapability(entry.toString()).setCapabilityOrigin(
-                                    remoteSessionCapabilities.getModuleBasedCapsOrigin().get(entry)).build())
-                            .collect(Collectors.toList()));
-
-                    capabilities.addNonModuleBasedCapabilities(remoteSessionCapabilities
-                            .getNonModuleCaps().stream().map(entry -> new AvailableCapabilityBuilder()
-                                    .setCapability(entry).setCapabilityOrigin(
-                                            remoteSessionCapabilities.getNonModuleBasedCapsOrigin().get(entry)).build())
-                            .collect(Collectors.toList()));
-
-                    handleSalInitializationSuccess(result, remoteSessionCapabilities, getDeviceSpecificRpc(result));
-                    return;
-                } catch (final SchemaResolutionException e) {
-                    // schemaBuilderFuture.checkedGet() throws only SchemaResolutionException
-                    // that might be wrapping a MissingSchemaSourceException so we need to look
-                    // at the cause of the exception to make sure we don't misinterpret it.
-                    if (e.getCause() instanceof MissingSchemaSourceException) {
-                        requiredSources = handleMissingSchemaSourceException(
-                                requiredSources, (MissingSchemaSourceException) e.getCause());
-                        continue;
-                    }
-                    requiredSources = handleSchemaResolutionException(requiredSources, e);
-                } catch (final Exception e) {
-                    // unknown error, fail
-                    handleSalInitializationFailure(e, listener);
-                    return;
+                    schemaRepository.getSchemaSource(sourceIdentifier, YangTextSchemaSource.class).get();
+                    return false;
+                } catch (InterruptedException | ExecutionException e) {
+                    return true;
                 }
-            }
-            // No more sources, fail
-            final IllegalStateException cause = new IllegalStateException(id + ": No more sources for schema context");
-            handleSalInitializationFailure(cause, listener);
-            salFacade.onDeviceFailed(cause);
+            }).collect(Collectors.toList());
         }
 
         private Collection<SourceIdentifier> handleMissingSchemaSourceException(
-                final Collection<SourceIdentifier> requiredSources, final MissingSchemaSourceException exception) {
+                final MissingSchemaSourceException exception) {
             // In case source missing, try without it
             final SourceIdentifier missingSource = exception.getSourceId();
-            LOG.warn("{}: Unable to build schema context, missing source {}, will reattempt without it", id,
-                    missingSource);
+            LOG.warn("{}: Unable to build schema context, missing source {}, will reattempt without it",
+                id, missingSource);
             LOG.debug("{}: Unable to build schema context, missing source {}, will reattempt without it",
-                    exception);
+                id, missingSource, exception);
             final Collection<QName> qNameOfMissingSource =
-                    getQNameFromSourceIdentifiers(Sets.newHashSet(missingSource));
+                getQNameFromSourceIdentifiers(Sets.newHashSet(missingSource));
             if (!qNameOfMissingSource.isEmpty()) {
                 capabilities.addUnresolvedCapabilities(
                         qNameOfMissingSource, UnavailableCapability.FailureReason.MissingSource);
             }
-            return stripUnavailableSource(requiredSources, missingSource);
+            return stripUnavailableSource(missingSource);
         }
 
         private Collection<SourceIdentifier> handleSchemaResolutionException(
-            final Collection<SourceIdentifier> requiredSources, final SchemaResolutionException resolutionException) {
+                final SchemaResolutionException resolutionException) {
             // In case resolution error, try only with resolved sources
             // There are two options why schema resolution exception occurred : unsatisfied imports or flawed model
             // FIXME Do we really have assurance that these two cases cannot happen at once?
@@ -554,34 +565,29 @@ public class NetconfDevice
                 // flawed model - exclude it
                 final SourceIdentifier failedSourceId = resolutionException.getFailedSource();
                 LOG.warn("{}: Unable to build schema context, failed to resolve source {}, will reattempt without it",
-                        id, failedSourceId);
+                    id, failedSourceId);
                 LOG.warn("{}: Unable to build schema context, failed to resolve source {}, will reattempt without it",
-                        id, resolutionException);
-                capabilities.addUnresolvedCapabilities(getQNameFromSourceIdentifiers(
-                        Collections.singleton(failedSourceId)), UnavailableCapability.FailureReason.UnableToResolve);
-                return stripUnavailableSource(requiredSources, resolutionException.getFailedSource());
+                    id, failedSourceId, resolutionException);
+                capabilities.addUnresolvedCapabilities(
+                        getQNameFromSourceIdentifiers(Collections.singleton(failedSourceId)),
+                        UnavailableCapability.FailureReason.UnableToResolve);
+                return stripUnavailableSource(resolutionException.getFailedSource());
             }
             // unsatisfied imports
             final Set<SourceIdentifier> unresolvedSources = resolutionException.getUnsatisfiedImports().keySet();
-            capabilities.addUnresolvedCapabilities(
-                getQNameFromSourceIdentifiers(unresolvedSources), UnavailableCapability.FailureReason.UnableToResolve);
+            capabilities.addUnresolvedCapabilities(getQNameFromSourceIdentifiers(unresolvedSources),
+                UnavailableCapability.FailureReason.UnableToResolve);
             LOG.warn("{}: Unable to build schema context, unsatisfied imports {}, will reattempt with resolved only",
-                    id, resolutionException.getUnsatisfiedImports());
+                id, resolutionException.getUnsatisfiedImports());
             LOG.debug("{}: Unable to build schema context, unsatisfied imports {}, will reattempt with resolved only",
-                    resolutionException);
+                id, resolutionException.getUnsatisfiedImports(), resolutionException);
             return resolutionException.getResolvedSources();
         }
 
-        protected NetconfDeviceRpc getDeviceSpecificRpc(final SchemaContext result) {
-            return new NetconfDeviceRpc(result, listener, new NetconfMessageTransformer(result, true));
-        }
-
-        private Collection<SourceIdentifier> stripUnavailableSource(final Collection<SourceIdentifier> requiredSources,
-                                                                    final SourceIdentifier sourceIdToRemove) {
-            final LinkedList<SourceIdentifier> sourceIdentifiers = Lists.newLinkedList(requiredSources);
-            final boolean removed = sourceIdentifiers.remove(sourceIdToRemove);
-            Preconditions.checkState(
-                    removed, "{}: Trying to remove {} from {} failed", id, sourceIdToRemove, requiredSources);
+        private Collection<SourceIdentifier> stripUnavailableSource(final SourceIdentifier sourceIdToRemove) {
+            final LinkedList<SourceIdentifier> sourceIdentifiers = new LinkedList<>(requiredSources);
+            checkState(sourceIdentifiers.remove(sourceIdToRemove),
+                    "%s: Trying to remove %s from %s failed", id, sourceIdToRemove, requiredSources);
             return sourceIdentifiers;
         }
 
@@ -598,16 +604,11 @@ public class NetconfDevice
         private QName getQNameFromSourceIdentifier(final SourceIdentifier identifier) {
             // Required sources are all required and provided merged in DeviceSourcesResolver
             for (final QName qname : deviceSources.getRequiredSourcesQName()) {
-                if (!qname.getLocalName().equals(identifier.getName())) {
+                if (!qname.getLocalName().equals(identifier.name().getLocalName())) {
                     continue;
                 }
 
-                final String rev = getNullableRev(identifier);
-                if (rev == null) {
-                    if (qname.getRevision() == null) {
-                        return qname;
-                    }
-                } else if (qname.getFormattedRevision().equals(rev)) {
+                if (Objects.equals(identifier.revision(), qname.getRevision().orElse(null))) {
                     return qname;
                 }
             }
@@ -617,10 +618,5 @@ public class NetconfDevice
             // this capability will be removed from required sources and not reported as unresolved-capability
             return null;
         }
-
-        private String getNullableRev(final SourceIdentifier identifier) {
-            final String rev = identifier.getRevision();
-            return rev == null || SourceIdentifier.NOT_PRESENT_FORMATTED_REVISION.equals(rev) ? null : rev;
-        }
     }
 }