Remove redundant code constructs
[netconf.git] / netconf / sal-netconf-connector / src / main / java / org / opendaylight / netconf / sal / connect / netconf / listener / NetconfDeviceCommunicator.java
index a3a6b60228f6bc8686ff658a6656f9209d2b0acd..7968fcda978255007baaf65ce23cecea3826f541 100644 (file)
@@ -14,8 +14,6 @@ import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
 import io.netty.util.concurrent.Future;
-import io.netty.util.concurrent.FutureListener;
-import io.netty.util.concurrent.GenericFutureListener;
 import java.util.ArrayDeque;
 import java.util.Iterator;
 import java.util.List;
@@ -24,12 +22,13 @@ import java.util.concurrent.Semaphore;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
-import org.opendaylight.controller.config.util.xml.XmlElement;
-import org.opendaylight.controller.config.util.xml.XmlUtil;
+import org.opendaylight.netconf.api.FailedNetconfMessage;
 import org.opendaylight.netconf.api.NetconfDocumentedException;
 import org.opendaylight.netconf.api.NetconfMessage;
 import org.opendaylight.netconf.api.NetconfTerminationReason;
+import org.opendaylight.netconf.api.xml.XmlElement;
 import org.opendaylight.netconf.api.xml.XmlNetconfConstants;
+import org.opendaylight.netconf.api.xml.XmlUtil;
 import org.opendaylight.netconf.client.NetconfClientDispatcher;
 import org.opendaylight.netconf.client.NetconfClientSession;
 import org.opendaylight.netconf.client.NetconfClientSessionListener;
@@ -46,7 +45,8 @@ import org.opendaylight.yangtools.yang.common.RpcResultBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class NetconfDeviceCommunicator implements NetconfClientSessionListener, RemoteDeviceCommunicator<NetconfMessage> {
+public class NetconfDeviceCommunicator
+        implements NetconfClientSessionListener, RemoteDeviceCommunicator<NetconfMessage> {
 
     private static final Logger LOG = LoggerFactory.getLogger(NetconfDeviceCommunicator.class);
 
@@ -59,7 +59,7 @@ public class NetconfDeviceCommunicator implements NetconfClientSessionListener,
     private final int concurentRpcMsgs;
 
     private final Queue<Request> requests = new ArrayDeque<>();
-    private NetconfClientSession session;
+    private NetconfClientSession currentSession;
 
     private Future<?> initFuture;
     private final SettableFuture<NetconfDeviceCapabilities> firstConnectionFuture;
@@ -75,19 +75,24 @@ public class NetconfDeviceCommunicator implements NetconfClientSessionListener,
         return isSessionClosing.get();
     }
 
-    public NetconfDeviceCommunicator(final RemoteDeviceId id, final RemoteDevice<NetconfSessionPreferences, NetconfMessage, NetconfDeviceCommunicator> remoteDevice,
-            final UserPreferences NetconfSessionPreferences, final int rpcMessageLimit) {
-        this(id, remoteDevice, Optional.of(NetconfSessionPreferences), rpcMessageLimit);
+    public NetconfDeviceCommunicator(
+            final RemoteDeviceId id,
+            final RemoteDevice<NetconfSessionPreferences, NetconfMessage, NetconfDeviceCommunicator> remoteDevice,
+            final UserPreferences netconfSessionPreferences, final int rpcMessageLimit) {
+        this(id, remoteDevice, Optional.of(netconfSessionPreferences), rpcMessageLimit);
     }
 
-    public NetconfDeviceCommunicator(final RemoteDeviceId id,
-                                     final RemoteDevice<NetconfSessionPreferences, NetconfMessage, NetconfDeviceCommunicator> remoteDevice,
-                                     final int rpcMessageLimit) {
-        this(id, remoteDevice, Optional.<UserPreferences>absent(), rpcMessageLimit);
+    public NetconfDeviceCommunicator(
+            final RemoteDeviceId id,
+            final RemoteDevice<NetconfSessionPreferences, NetconfMessage, NetconfDeviceCommunicator> remoteDevice,
+            final int rpcMessageLimit) {
+        this(id, remoteDevice, Optional.absent(), rpcMessageLimit);
     }
 
-    private NetconfDeviceCommunicator(final RemoteDeviceId id, final RemoteDevice<NetconfSessionPreferences, NetconfMessage, NetconfDeviceCommunicator> remoteDevice,
-                                      final Optional<UserPreferences> overrideNetconfCapabilities, final int rpcMessageLimit) {
+    private NetconfDeviceCommunicator(
+            final RemoteDeviceId id,
+            final RemoteDevice<NetconfSessionPreferences, NetconfMessage, NetconfDeviceCommunicator> remoteDevice,
+            final Optional<UserPreferences> overrideNetconfCapabilities, final int rpcMessageLimit) {
         this.concurentRpcMsgs = rpcMessageLimit;
         this.id = id;
         this.remoteDevice = remoteDevice;
@@ -101,58 +106,59 @@ public class NetconfDeviceCommunicator implements NetconfClientSessionListener,
         sessionLock.lock();
         try {
             LOG.debug("{}: Session established", id);
-            this.session = session;
+            currentSession = session;
 
             NetconfSessionPreferences netconfSessionPreferences =
                                              NetconfSessionPreferences.fromNetconfSession(session);
             LOG.trace("{}: Session advertised capabilities: {}", id,
                     netconfSessionPreferences);
 
-            if(overrideNetconfCapabilities.isPresent()) {
-                netconfSessionPreferences = overrideNetconfCapabilities.get().isOverride() ?
-                        netconfSessionPreferences.replaceModuleCaps(overrideNetconfCapabilities.get().getSessionPreferences()) :
-                        netconfSessionPreferences.addModuleCaps(overrideNetconfCapabilities.get().getSessionPreferences());
-                LOG.debug(
-                        "{}: Session capabilities overridden, capabilities that will be used: {}",
-                        id, netconfSessionPreferences);
+            if (overrideNetconfCapabilities.isPresent()) {
+                final NetconfSessionPreferences sessionPreferences = overrideNetconfCapabilities
+                        .get().getSessionPreferences();
+                netconfSessionPreferences = overrideNetconfCapabilities.get().moduleBasedCapsOverrided()
+                        ? netconfSessionPreferences.replaceModuleCaps(sessionPreferences)
+                        : netconfSessionPreferences.addModuleCaps(sessionPreferences);
+
+                netconfSessionPreferences = overrideNetconfCapabilities.get().nonModuleBasedCapsOverrided()
+                        ? netconfSessionPreferences.replaceNonModuleCaps(sessionPreferences)
+                        : netconfSessionPreferences.addNonModuleCaps(sessionPreferences);
+                LOG.debug("{}: Session capabilities overridden, capabilities that will be used: {}", id,
+                        netconfSessionPreferences);
             }
 
-
             remoteDevice.onRemoteSessionUp(netconfSessionPreferences, this);
             if (!firstConnectionFuture.isDone()) {
                 firstConnectionFuture.set(netconfSessionPreferences.getNetconfDeviceCapabilities());
             }
-        }
-        finally {
+        } finally {
             sessionLock.unlock();
         }
     }
 
     /**
+     * Initialize remote connection.
      *
-     * @param dispatcher
-     * @param config
+     * @param dispatcher {@code NetconfCLientDispatcher}
+     * @param config     {@code NetconfClientConfiguration}
      * @return future that returns succes on first succesfull connection and failure when the underlying
-     * reconnecting strategy runs out of reconnection attempts
+     *     reconnecting strategy runs out of reconnection attempts
      */
-    public ListenableFuture<NetconfDeviceCapabilities> initializeRemoteConnection(final NetconfClientDispatcher dispatcher, final NetconfClientConfiguration config) {
-        if(config instanceof NetconfReconnectingClientConfiguration) {
+    public ListenableFuture<NetconfDeviceCapabilities> initializeRemoteConnection(
+            final NetconfClientDispatcher dispatcher, final NetconfClientConfiguration config) {
+        if (config instanceof NetconfReconnectingClientConfiguration) {
             initFuture = dispatcher.createReconnectingClient((NetconfReconnectingClientConfiguration) config);
         } else {
             initFuture = dispatcher.createClient(config);
         }
 
 
-        initFuture.addListener(new GenericFutureListener<Future<Object>>(){
-
-            @Override
-            public void operationComplete(final Future<Object> future) throws Exception {
-                if (!future.isSuccess() && !future.isCancelled()) {
-                    LOG.debug("{}: Connection failed", id, future.cause());
-                    NetconfDeviceCommunicator.this.remoteDevice.onRemoteSessionFailed(future.cause());
-                    if (firstConnectionFuture.isDone()) {
-                        firstConnectionFuture.setException(future.cause());
-                    }
+        initFuture.addListener(future -> {
+            if (!future.isSuccess() && !future.isCancelled()) {
+                LOG.debug("{}: Connection failed", id, future.cause());
+                NetconfDeviceCommunicator.this.remoteDevice.onRemoteSessionFailed(future.cause());
+                if (firstConnectionFuture.isDone()) {
+                    firstConnectionFuture.setException(future.cause());
                 }
             }
         });
@@ -161,8 +167,8 @@ public class NetconfDeviceCommunicator implements NetconfClientSessionListener,
 
     public void disconnect() {
         // If session is already in closing, no need to close it again
-        if(session != null && isSessionClosing.compareAndSet(false, true)) {
-            session.close();
+        if (currentSession != null && isSessionClosing.compareAndSet(false, true)) {
+            currentSession.close();
         }
     }
 
@@ -174,9 +180,8 @@ public class NetconfDeviceCommunicator implements NetconfClientSessionListener,
         final List<UncancellableFuture<RpcResult<NetconfMessage>>> futuresToCancel = Lists.newArrayList();
         sessionLock.lock();
         try {
-            if( session != null ) {
-                session = null;
-
+            if (currentSession != null) {
+                currentSession = null;
                 /*
                  * Walk all requests, check if they have been executing
                  * or cancelled and remove them from the queue.
@@ -185,7 +190,7 @@ public class NetconfDeviceCommunicator implements NetconfClientSessionListener,
                 while (it.hasNext()) {
                     final Request r = it.next();
                     if (r.future.isUncancellable()) {
-                        futuresToCancel.add( r.future );
+                        futuresToCancel.add(r.future);
                         it.remove();
                     } else if (r.future.isCancelled()) {
                         // This just does some house-cleaning
@@ -195,18 +200,17 @@ public class NetconfDeviceCommunicator implements NetconfClientSessionListener,
 
                 remoteDevice.onRemoteSessionDown();
             }
-        }
-        finally {
+        } finally {
             sessionLock.unlock();
         }
 
         // Notify pending request futures outside of the sessionLock to avoid unnecessarily
         // blocking the caller.
         for (final UncancellableFuture<RpcResult<NetconfMessage>> future : futuresToCancel) {
-            if( Strings.isNullOrEmpty( reason ) ) {
-                future.set( createSessionDownRpcResult() );
+            if (Strings.isNullOrEmpty(reason)) {
+                future.set(createSessionDownRpcResult());
             } else {
-                future.set( createErrorRpcResult( RpcError.ErrorType.TRANSPORT, reason ) );
+                future.set(createErrorRpcResult(RpcError.ErrorType.TRANSPORT, reason));
             }
         }
 
@@ -214,21 +218,22 @@ public class NetconfDeviceCommunicator implements NetconfClientSessionListener,
     }
 
     private RpcResult<NetconfMessage> createSessionDownRpcResult() {
-        return createErrorRpcResult( RpcError.ErrorType.TRANSPORT,
-                             String.format( "The netconf session to %1$s is disconnected", id.getName() ) );
+        return createErrorRpcResult(RpcError.ErrorType.TRANSPORT,
+                String.format("The netconf session to %1$s is disconnected", id.getName()));
     }
 
-    private RpcResult<NetconfMessage> createErrorRpcResult(final RpcError.ErrorType errorType, final String message) {
+    private static RpcResult<NetconfMessage> createErrorRpcResult(final RpcError.ErrorType errorType,
+            final String message) {
         return RpcResultBuilder.<NetconfMessage>failed()
-                .withError(errorType, NetconfDocumentedException.ErrorTag.OPERATION_FAILED.getTagValue(), message).build();
+            .withError(errorType, NetconfDocumentedException.ErrorTag.OPERATION_FAILED.getTagValue(), message).build();
     }
 
     @Override
-    public void onSessionDown(final NetconfClientSession session, final Exception e) {
+    public void onSessionDown(final NetconfClientSession session, final Exception exception) {
         // If session is already in closing, no need to call tearDown again.
         if (isSessionClosing.compareAndSet(false, true)) {
-            LOG.warn("{}: Session went down", id, e);
-            tearDown( null );
+            LOG.warn("{}: Session went down", id, exception);
+            tearDown(null);
         }
     }
 
@@ -236,13 +241,13 @@ public class NetconfDeviceCommunicator implements NetconfClientSessionListener,
     public void onSessionTerminated(final NetconfClientSession session, final NetconfTerminationReason reason) {
         // onSessionTerminated is called directly by disconnect, no need to compare and set isSessionClosing.
         LOG.warn("{}: Session terminated {}", id, reason);
-        tearDown( reason.getErrorMessage() );
+        tearDown(reason.getErrorMessage());
     }
 
     @Override
     public void close() {
         // Cancel reconnect if in progress
-        if(initFuture != null) {
+        if (initFuture != null) {
             initFuture.cancel(false);
         }
         // Disconnect from device
@@ -270,10 +275,10 @@ public class NetconfDeviceCommunicator implements NetconfClientSessionListener,
         try {
             request = requests.peek();
             if (request != null && request.future.isUncancellable()) {
-                requests.poll();
+                request = requests.poll();
                 // we have just removed one request from the queue
                 // we can also release one permit
-                if(semaphore != null) {
+                if (semaphore != null) {
                     semaphore.release();
                 }
             } else {
@@ -281,28 +286,33 @@ public class NetconfDeviceCommunicator implements NetconfClientSessionListener,
                 LOG.warn("{}: Ignoring unsolicited message {}", id,
                         msgToS(message));
             }
-        }
-        finally {
+        } finally {
             sessionLock.unlock();
         }
 
-        if( request != null ) {
+        if (request != null) {
+
+            if (FailedNetconfMessage.class.isInstance(message)) {
+                request.future.set(NetconfMessageTransformUtil.toRpcResult((FailedNetconfMessage) message));
+                return;
+            }
 
             LOG.debug("{}: Message received {}", id, message);
 
-            if(LOG.isTraceEnabled()) {
-                LOG.trace( "{}: Matched request: {} to response: {}", id, msgToS( request.request ), msgToS( message ) );
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("{}: Matched request: {} to response: {}", id, msgToS(request.request), msgToS(message));
             }
 
             try {
-                NetconfMessageTransformUtil.checkValidReply( request.request, message );
+                NetconfMessageTransformUtil.checkValidReply(request.request, message);
             } catch (final NetconfDocumentedException e) {
                 LOG.warn(
-                        "{}: Invalid request-reply match, reply message contains different message-id, request: {}, response: {}",
+                        "{}: Invalid request-reply match,"
+                                + "reply message contains different message-id, request: {}, response: {}",
                         id, msgToS(request.request), msgToS(message), e);
 
-                request.future.set( RpcResultBuilder.<NetconfMessage>failed()
-                        .withRpcError( NetconfMessageTransformUtil.toRpcError( e ) ).build() );
+                request.future.set(RpcResultBuilder.<NetconfMessage>failed()
+                        .withRpcError(NetconfMessageTransformUtil.toRpcError(e)).build());
 
                 //recursively processing message to eventually find matching request
                 processMessage(message);
@@ -312,17 +322,17 @@ public class NetconfDeviceCommunicator implements NetconfClientSessionListener,
 
             try {
                 NetconfMessageTransformUtil.checkSuccessReply(message);
-            } catch(final NetconfDocumentedException e) {
+            } catch (final NetconfDocumentedException e) {
                 LOG.warn(
                         "{}: Error reply from remote device, request: {}, response: {}",
                         id, msgToS(request.request), msgToS(message), e);
 
-                request.future.set( RpcResultBuilder.<NetconfMessage>failed()
-                        .withRpcError( NetconfMessageTransformUtil.toRpcError( e ) ).build() );
+                request.future.set(RpcResultBuilder.<NetconfMessage>failed()
+                        .withRpcError(NetconfMessageTransformUtil.toRpcError(e)).build());
                 return;
             }
 
-            request.future.set( RpcResultBuilder.success( message ).build() );
+            request.future.set(RpcResultBuilder.success(message).build());
         }
     }
 
@@ -333,58 +343,52 @@ public class NetconfDeviceCommunicator implements NetconfClientSessionListener,
     @Override
     public ListenableFuture<RpcResult<NetconfMessage>> sendRequest(final NetconfMessage message, final QName rpc) {
         sessionLock.lock();
-
-        if (semaphore != null && !semaphore.tryAcquire()) {
-            LOG.warn("Limit of concurrent rpc messages was reached (limit :" +
-                    concurentRpcMsgs + "). Rpc reply message is needed. Discarding request of Netconf device with id" + id.getName());
-            sessionLock.unlock();
-            return Futures.immediateFailedFuture(new NetconfDocumentedException("Limit of rpc messages was reached (Limit :" +
-                    concurentRpcMsgs + ") waiting for emptying the queue of Netconf device with id" + id.getName()));
-        }
-
         try {
+            if (semaphore != null && !semaphore.tryAcquire()) {
+                LOG.warn("Limit of concurrent rpc messages was reached (limit :" + concurentRpcMsgs
+                    + "). Rpc reply message is needed. Discarding request of Netconf device with id" + id.getName());
+                return Futures.immediateFailedFuture(new NetconfDocumentedException(
+                        "Limit of rpc messages was reached (Limit :" + concurentRpcMsgs
+                        + ") waiting for emptying the queue of Netconf device with id" + id.getName()));
+            }
+
             return sendRequestWithLock(message, rpc);
         } finally {
             sessionLock.unlock();
         }
     }
 
-    private ListenableFuture<RpcResult<NetconfMessage>> sendRequestWithLock(
-                                               final NetconfMessage message, final QName rpc) {
-        if(LOG.isTraceEnabled()) {
+    private ListenableFuture<RpcResult<NetconfMessage>> sendRequestWithLock(final NetconfMessage message,
+                                                                            final QName rpc) {
+        if (LOG.isTraceEnabled()) {
             LOG.trace("{}: Sending message {}", id, msgToS(message));
         }
 
-        if (session == null) {
+        if (currentSession == null) {
             LOG.warn("{}: Session is disconnected, failing RPC request {}",
                     id, message);
-            return Futures.immediateFuture( createSessionDownRpcResult() );
+            return Futures.immediateFuture(createSessionDownRpcResult());
         }
 
-        final Request req = new Request( new UncancellableFuture<RpcResult<NetconfMessage>>(true),
-                                         message );
+        final Request req = new Request(new UncancellableFuture<>(true), message);
         requests.add(req);
 
-        session.sendMessage(req.request).addListener(new FutureListener<Void>() {
-            @Override
-            public void operationComplete(final Future<Void> future) throws Exception {
-                if( !future.isSuccess() ) {
-                    // We expect that a session down will occur at this point
-                    LOG.debug("{}: Failed to send request {}", id,
-                            XmlUtil.toString(req.request.getDocument()),
-                            future.cause());
-
-                    if( future.cause() != null ) {
-                        req.future.set( createErrorRpcResult( RpcError.ErrorType.TRANSPORT,
-                                                              future.cause().getLocalizedMessage() ) );
-                    } else {
-                        req.future.set( createSessionDownRpcResult() ); // assume session is down
-                    }
-                    req.future.setException( future.cause() );
-                }
-                else {
-                    LOG.trace("Finished sending request {}", req.request);
+        currentSession.sendMessage(req.request).addListener(future -> {
+            if (!future.isSuccess()) {
+                // We expect that a session down will occur at this point
+                LOG.debug("{}: Failed to send request {}", id,
+                        XmlUtil.toString(req.request.getDocument()),
+                        future.cause());
+
+                if (future.cause() != null) {
+                    req.future.set(createErrorRpcResult(RpcError.ErrorType.TRANSPORT,
+                            future.cause().getLocalizedMessage()));
+                } else {
+                    req.future.set(createSessionDownRpcResult()); // assume session is down
                 }
+                req.future.setException(future.cause());
+            } else {
+                LOG.trace("Finished sending request {}", req.request);
             }
         });
 
@@ -392,7 +396,7 @@ public class NetconfDeviceCommunicator implements NetconfClientSessionListener,
     }
 
     private void processNotification(final NetconfMessage notification) {
-        if(LOG.isTraceEnabled()) {
+        if (LOG.isTraceEnabled()) {
             LOG.trace("{}: Notification received: {}", id, notification);
         }
 
@@ -400,6 +404,10 @@ public class NetconfDeviceCommunicator implements NetconfClientSessionListener,
     }
 
     private static boolean isNotification(final NetconfMessage message) {
+        if (message.getDocument() == null) {
+            // We have no message, which mean we have a FailedNetconfMessage
+            return false;
+        }
         final XmlElement xmle = XmlElement.fromDomDocument(message.getDocument());
         return XmlNetconfConstants.NOTIFICATION_ELEMENT_NAME.equals(xmle.getName()) ;
     }