Bug 1588 - OFConstants.java moved to openflowplugin-api module
[openflowplugin.git] / openflowplugin / src / main / java / org / opendaylight / openflowplugin / openflow / md / core / ConnectionConductorImpl.java
index cfc4011453ff8c9f4d79b1d5ac34cb7031d625b4..0b5ff69c10dece33a9b4170baab38e715d1c33ed 100644 (file)
@@ -8,16 +8,25 @@
 
 package org.opendaylight.openflowplugin.openflow.md.core;
 
-import java.util.List;
 import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
 import org.opendaylight.openflowjava.protocol.api.connection.ConnectionAdapter;
 import org.opendaylight.openflowjava.protocol.api.connection.ConnectionReadyListener;
+import org.opendaylight.openflowplugin.api.OFConstants;
 import org.opendaylight.openflowplugin.openflow.md.core.session.OFSessionUtil;
+import org.opendaylight.openflowplugin.openflow.md.core.session.PortFeaturesUtil;
 import org.opendaylight.openflowplugin.openflow.md.core.session.SessionContext;
 import org.opendaylight.openflowplugin.openflow.md.core.session.SessionManager;
 import org.opendaylight.openflowplugin.openflow.md.queue.QueueKeeper;
+import org.opendaylight.openflowplugin.openflow.md.queue.QueueKeeper.QueueType;
+import org.opendaylight.openflowplugin.openflow.md.queue.QueueKeeperFactory;
+import org.opendaylight.openflowplugin.openflow.md.queue.QueueProcessor;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.common.types.rev130731.MultipartRequestFlags;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.common.types.rev130731.MultipartType;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.common.types.rev130731.SwitchConfigFlag;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.EchoInputBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.EchoOutput;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.EchoReplyInputBuilder;
@@ -25,19 +34,25 @@ import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.ErrorMessage;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.ExperimenterMessage;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.FlowRemovedMessage;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.GetFeaturesInputBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.GetFeaturesOutput;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.HelloInput;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.HelloMessage;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.MultipartReplyMessage;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.MultipartRequestMessage;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.MultipartRequestInputBuilder;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.OfHeader;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.OpenflowProtocolListener;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.PacketInMessage;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.PortGrouping;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.PortStatus;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.PortStatusMessage;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.hello.Elements;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.SetConfigInputBuilder;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.multipart.request.multipart.request.body.MultipartRequestDescCaseBuilder;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.multipart.request.multipart.request.body.MultipartRequestGroupFeaturesCaseBuilder;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.multipart.request.multipart.request.body.MultipartRequestMeterFeaturesCaseBuilder;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.multipart.request.multipart.request.body.MultipartRequestPortDescCaseBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.system.rev130927.DisconnectEvent;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.system.rev130927.SwitchIdleEvent;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.system.rev130927.SystemNotificationsListener;
+import org.opendaylight.yangtools.yang.binding.DataObject;
 import org.opendaylight.yangtools.yang.common.RpcError;
 import org.opendaylight.yangtools.yang.common.RpcResult;
 import org.slf4j.Logger;
@@ -49,7 +64,10 @@ import com.google.common.util.concurrent.Futures;
  * @author mirehak
  */
 public class ConnectionConductorImpl implements OpenflowProtocolListener,
-        SystemNotificationsListener, ConnectionConductor, ConnectionReadyListener {
+        SystemNotificationsListener, ConnectionConductor, ConnectionReadyListener, HandshakeListener {
+
+    /** ingress queue limit */
+    private static final int INGRESS_QUEUE_MAX_SIZE = 200;
 
     protected static final Logger LOG = LoggerFactory
             .getLogger(ConnectionConductorImpl.class);
@@ -58,45 +76,70 @@ public class ConnectionConductorImpl implements OpenflowProtocolListener,
      * it will help while testing and isolating issues related to processing of
      * BitMaps from switches.
      */
-    private static final boolean isBitmapNegotiationEnable = true;
-    private ErrorHandler errorHandler;
+    private boolean isBitmapNegotiationEnable = true;
+    protected ErrorHandler errorHandler;
 
     private final ConnectionAdapter connectionAdapter;
     private ConnectionConductor.CONDUCTOR_STATE conductorState;
     private Short version;
 
-    private SwitchConnectionDistinguisher auxiliaryKey;
+    protected SwitchConnectionDistinguisher auxiliaryKey;
 
-    private SessionContext sessionContext;
+    protected SessionContext sessionContext;
 
-    protected boolean isFirstHelloNegotiation = true;
+    private QueueProcessor<OfHeader, DataObject> queueProcessor;
+    private QueueKeeper<OfHeader> queue;
+    private ThreadPoolExecutor hsPool;
+    private HandshakeManager handshakeManager;
 
-    // TODO: use appropriate interface instead of Object
-    private QueueKeeper<Object> queueKeeper;
+    private boolean firstHelloProcessed;
+    
+    private PortFeaturesUtil portFeaturesUtils;
 
+    private int conductorId;
 
+    private int ingressMaxQueueSize;
 
+    
     /**
      * @param connectionAdapter
      */
     public ConnectionConductorImpl(ConnectionAdapter connectionAdapter) {
+        this(connectionAdapter, INGRESS_QUEUE_MAX_SIZE);
+    }
+
+    /**
+     * @param connectionAdapter
+     * @param ingressMaxQueueSize ingress queue limit (blocking)
+     */
+    public ConnectionConductorImpl(ConnectionAdapter connectionAdapter, int ingressMaxQueueSize) {
         this.connectionAdapter = connectionAdapter;
+        this.ingressMaxQueueSize = ingressMaxQueueSize;
         conductorState = CONDUCTOR_STATE.HANDSHAKING;
+        firstHelloProcessed = false;
+        handshakeManager = new HandshakeManagerImpl(connectionAdapter,
+                ConnectionConductor.versionOrder.get(0), ConnectionConductor.versionOrder);
+        handshakeManager.setUseVersionBitmap(isBitmapNegotiationEnable);
+        handshakeManager.setHandshakeListener(this);
+        portFeaturesUtils = PortFeaturesUtil.getInstance();
     }
 
     @Override
     public void init() {
+        int handshakeThreadLimit = 1;
+        hsPool = new ThreadPoolLoggingExecutor(handshakeThreadLimit , handshakeThreadLimit, 0L, 
+                TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>(), 
+                "OFHandshake-"+conductorId);
+        
         connectionAdapter.setMessageListener(this);
         connectionAdapter.setSystemListener(this);
         connectionAdapter.setConnectionReadyListener(this);
+        queue = QueueKeeperFactory.createFairQueueKeeper(queueProcessor, ingressMaxQueueSize);
     }
-    
-    /**
-     * @param queueKeeper the queueKeeper to set
-     */
+
     @Override
-    public void setQueueKeeper(QueueKeeper<Object> queueKeeper) {
-        this.queueKeeper = queueKeeper;
+    public void setQueueProcessor(QueueProcessor<OfHeader, DataObject> queueProcessor) {
+        this.queueProcessor = queueProcessor;
     }
 
     /**
@@ -105,33 +148,7 @@ public class ConnectionConductorImpl implements OpenflowProtocolListener,
     @Override
     public void setErrorHandler(ErrorHandler errorHandler) {
         this.errorHandler = errorHandler;
-    }
-
-    /**
-     * send first hello message to switch
-     */
-    protected void sendFirstHelloMessage() {
-        Short highestVersion = ConnectionConductor.versionOrder.get(0);
-        Long helloXid = 21L;
-        HelloInput helloInput = null;
-        
-        if (isBitmapNegotiationEnable) {
-            helloInput = MessageFactory.createHelloInput(highestVersion, helloXid, ConnectionConductor.versionOrder);
-            LOG.debug("sending first hello message: vertsion header={} , version bitmap={}", 
-                    highestVersion, MessageFactory.digVersions(helloInput.getElements()));
-        } else {
-            helloInput = MessageFactory.createHelloInput(highestVersion, helloXid);
-            LOG.debug("sending first hello message: version header={} ", highestVersion);
-        }
-        
-        try {
-            RpcResult<Void> helloResult = connectionAdapter.hello(helloInput).get(getMaxTimeout(), getMaxTimeoutUnit());
-            smokeRpc(helloResult);
-            LOG.debug("FIRST HELLO sent.");
-        } catch (Throwable e) {
-            LOG.debug("FIRST HELLO sending failed.");
-            errorHandler.handleException(e, getSessionContext());
-        }
+        handshakeManager.setErrorHandler(errorHandler);
     }
 
     @Override
@@ -144,25 +161,41 @@ public class ConnectionConductorImpl implements OpenflowProtocolListener,
                 builder.setVersion(echoRequestMessage.getVersion());
                 builder.setXid(echoRequestMessage.getXid());
                 builder.setData(echoRequestMessage.getData());
-                
+
                 getConnectionAdapter().echoReply(builder.build());
             }
-        }).start();            
+        }).start();
     }
 
     @Override
     public void onErrorMessage(ErrorMessage errorMessage) {
-        queueKeeper.push(ErrorMessage.class, errorMessage, this);
+        enqueueMessage(errorMessage);
+    }
+
+    
+    /**
+     * @param message
+     */
+    private void enqueueMessage(OfHeader message) {
+        enqueueMessage(message, QueueType.DEFAULT);
+    }
+
+    /**
+     * @param message
+     * @param queueType enqueue type
+     */
+    private void enqueueMessage(OfHeader message, QueueType queueType) {
+        queue.push(message, this, queueType);
     }
 
     @Override
     public void onExperimenterMessage(ExperimenterMessage experimenterMessage) {
-        queueKeeper.push(ExperimenterMessage.class, experimenterMessage, this);
+        enqueueMessage(experimenterMessage);
     }
 
     @Override
     public void onFlowRemovedMessage(FlowRemovedMessage message) {
-        queueKeeper.push(FlowRemovedMessage.class, message, this);
+        enqueueMessage(message);
     }
 
 
@@ -173,144 +206,16 @@ public class ConnectionConductorImpl implements OpenflowProtocolListener,
      * 2. If HelloMessage contains bitmap and common version found in bitmap
      *    then continue connection processing. if no common version found, just disconnect.
      * 3. If HelloMessage version is not supported, send HelloMessage with lower supported version.
-     *    If Hello message received again with not supported version, just disconnect.
-     *
-     *   TODO: Better to handle handshake into a maintainable innerclass which uses State-Pattern.
+     * 4. If Hello message received again with not supported version, just disconnect.
      */
     @Override
     public void onHelloMessage(final HelloMessage hello) {
-        // do handshake
-
-        new Thread(new Runnable() {
-            @Override
-            public void run() {
-                LOG.info("handshake STARTED");
-                checkState(CONDUCTOR_STATE.HANDSHAKING);
-                
-                Short remoteVersion = hello.getVersion();
-                List<Elements> elements = hello.getElements();
-                Long xid = hello.getXid();
-                Short proposedVersion;
-                LOG.debug("Hello message version={} and bitmap={}", remoteVersion, MessageFactory.digVersions(elements));
-                try {
-                    // find the version from header version field
-                    proposedVersion = proposeVersion(remoteVersion);
-                    
-                } catch (IllegalArgumentException e) {
-                    errorHandler.handleException(e, getSessionContext());
-                    getConnectionAdapter().disconnect();
-                    return;
-                }
-                
-                // sent version is equal to remote --> version is negotiated
-                if (proposedVersion == remoteVersion) {
-                    LOG.debug("sending helloReply as version in header is supported: {}", proposedVersion);
-                    sendHelloReply(proposedVersion, ++xid);
-                    postHandshake(proposedVersion, ++xid);
-                    
-                } else if (isBitmapNegotiationEnable && null != elements && 0 != elements.size()) {
-                    try {
-                        // hello contains version bitmap, checking highest common
-                        // version in bitmap
-                        proposedVersion = proposeBitmapVersion(elements);
-                    } catch (IllegalArgumentException ex) {
-                        errorHandler.handleException(ex, getSessionContext());
-                        getConnectionAdapter().disconnect();
-                        return;
-                    }
-                    LOG.debug("sending helloReply for common bitmap version : {}", proposedVersion);
-                    sendHelloReply(proposedVersion, ++xid);
-                    postHandshake(proposedVersion, ++xid);
-                } else {
-                    if (isFirstHelloNegotiation) {
-                        isFirstHelloNegotiation = false;
-                        LOG.debug("sending helloReply for lowest supported version : {}", proposedVersion);
-                        // send hello reply with lower version number supported
-                        sendHelloReply(proposedVersion, ++xid);
-                    } else {
-                        // terminate the connection.
-                        LOG.debug("Version negotiation failed. unsupported version : {}", remoteVersion);
-                        getConnectionAdapter().disconnect();
-                    }
-                }
-            }
-        }).start();
-    }
-
-    /**
-     * send hello reply
-     * @param proposedVersion
-     * @param hello
-     */
-    protected void sendHelloReply(Short proposedVersion, Long xid)
-    {
-        HelloInput helloMsg = MessageFactory.createHelloInput(proposedVersion, xid);
-        RpcResult<Void> result;
-        try {
-            result = connectionAdapter.hello(helloMsg).get(getMaxTimeout(), getMaxTimeoutUnit());
-            smokeRpc(result);
-        } catch (Throwable e) {
-            errorHandler.handleException(e, getSessionContext());
-        }
-    }
-
-
-    /**
-     * @param futureResult
-     * @throws Throwable 
-     */
-    private static void smokeRpc(RpcResult<?> result) throws Throwable {
-        if (!result.isSuccessful()) {
-            Throwable firstCause = null;
-            StringBuffer sb = new StringBuffer();
-            for (RpcError error : result.getErrors()) {
-                if (firstCause != null) {
-                    firstCause = error.getCause();
-                }
-                
-                sb.append("rpcError:").append(error.getCause().getMessage()).append(";");
-            }
-            throw new Exception(sb.toString(), firstCause);
-        }
-    }
-
-    /**
-     * after handshake set features, register to session
-     * @param proposedVersion
-     * @param xId
-     */
-    protected void postHandshake(Short proposedVersion, Long xid) {
-        // set version
-        version = proposedVersion;
-        LOG.debug("version set: " + proposedVersion);
-        // request features
-        GetFeaturesInputBuilder featuresBuilder = new GetFeaturesInputBuilder();
-        featuresBuilder.setVersion(version).setXid(xid);
-        LOG.debug("sending feature request for version={} and xid={}", version, xid);
-        Future<RpcResult<GetFeaturesOutput>> featuresFuture = connectionAdapter
-                .getFeatures(featuresBuilder.build());
-        LOG.debug("waiting for features");
-        try {
-            RpcResult<GetFeaturesOutput> rpcFeatures = 
-                    featuresFuture.get(getMaxTimeout(), getMaxTimeoutUnit());
-            smokeRpc(rpcFeatures);
-            
-            GetFeaturesOutput featureOutput =  rpcFeatures.getResult();
-            LOG.debug("obtained features: datapathId={}",
-                    featureOutput.getDatapathId());
-            LOG.debug("obtained features: auxiliaryId={}",
-                    featureOutput.getAuxiliaryId());
-            conductorState = CONDUCTOR_STATE.WORKING;
-
-            OFSessionUtil.registerSession(this,
-                    featureOutput, version);
-            LOG.info("handshake SETTLED: datapathId={}, auxiliaryId={}", featureOutput.getDatapathId(), featureOutput.getAuxiliaryId());
-        } catch (Throwable e) {
-            //handshake failed
-            LOG.error("issuing disconnect during handshake, reason: "+e.getMessage());
-            errorHandler.handleException(e, getSessionContext());
-            disconnect();
-        }
+        LOG.debug("processing HELLO.xid: {}", hello.getXid());
+        firstHelloProcessed = true;
+        checkState(CONDUCTOR_STATE.HANDSHAKING);
+        HandshakeStepWrapper handshakeStepWrapper = new HandshakeStepWrapper(
+                hello, handshakeManager, connectionAdapter);
+        hsPool.submit(handshakeStepWrapper);
     }
 
     /**
@@ -320,7 +225,7 @@ public class ConnectionConductorImpl implements OpenflowProtocolListener,
         // TODO:: get from configuration
         return 2000;
     }
-    
+
     /**
      * @return milliseconds
      */
@@ -331,23 +236,47 @@ public class ConnectionConductorImpl implements OpenflowProtocolListener,
 
     @Override
     public void onMultipartReplyMessage(MultipartReplyMessage message) {
-        queueKeeper.push(MultipartReplyMessage.class, message, this);
-    }
-
-    @Override
-    public void onMultipartRequestMessage(MultipartRequestMessage message) {
-        queueKeeper.push(MultipartRequestMessage.class, message, this);
+        enqueueMessage(message);
     }
 
     @Override
     public void onPacketInMessage(PacketInMessage message) {
-        queueKeeper.push(PacketInMessage.class, message, this);
+        enqueueMessage(message, QueueKeeper.QueueType.UNORDERED);
     }
 
     @Override
     public void onPortStatusMessage(PortStatusMessage message) {
-        this.getSessionContext().processPortStatusMsg(message);
-        queueKeeper.push(PortStatusMessage.class, message, this);
+        processPortStatusMsg(message);
+        enqueueMessage(message);
+    }
+    
+    protected void processPortStatusMsg(PortStatus msg) {
+        if (msg.getReason().getIntValue() == 2) {
+            updatePort(msg);
+        } else if (msg.getReason().getIntValue() == 0) {
+            updatePort(msg);
+        } else if (msg.getReason().getIntValue() == 1) {
+            deletePort(msg);
+        }
+    }
+    
+    protected void updatePort(PortStatus msg) {
+        Long portNumber = msg.getPortNo();        
+        Boolean portBandwidth = portFeaturesUtils.getPortBandwidth(msg);
+        
+        if(portBandwidth == null) {
+            LOG.debug("can't get bandwidth info from port: {}, aborting port update", msg.toString());
+        } else {
+            this.getSessionContext().getPhysicalPorts().put(portNumber, msg);
+            this.getSessionContext().getPortsBandwidth().put(portNumber, portBandwidth);                   
+        }            
+    }
+    
+    protected void deletePort(PortGrouping port) {
+        Long portNumber = port.getPortNo();
+        
+        this.getSessionContext().getPhysicalPorts().remove(portNumber);
+        this.getSessionContext().getPortsBandwidth().remove(portNumber);
     }
 
     @Override
@@ -361,14 +290,14 @@ public class ConnectionConductorImpl implements OpenflowProtocolListener,
                     disconnect();
                     OFSessionUtil.getSessionManager().invalidateOnDisconnect(ConnectionConductorImpl.this);
                 } else {
-                    LOG.debug("first idle state occured");
+                    LOG.debug("first idle state occured, sessionCtx={}|auxId={}", sessionContext, auxiliaryKey);
                     EchoInputBuilder builder = new EchoInputBuilder();
                     builder.setVersion(getVersion());
                     builder.setXid(getSessionContext().getNextXid());
-                    
+
                     Future<RpcResult<EchoOutput>> echoReplyFuture = getConnectionAdapter()
                             .echo(builder.build());
-                    
+
                     try {
                         RpcResult<EchoOutput> echoReplyValue = echoReplyFuture.get(getMaxTimeout(),
                                 getMaxTimeoutUnit());
@@ -386,14 +315,15 @@ public class ConnectionConductorImpl implements OpenflowProtocolListener,
                         }
                     } catch (Exception e) {
                         LOG.error("while waiting for echoReply in TIMEOUTING state: "
-                                + e.getMessage(), e);
+                                + e.getMessage());
+                        errorHandler.handleException(e, sessionContext);
                         //switch is not responding
                         disconnect();
                         OFSessionUtil.getSessionManager().invalidateOnDisconnect(ConnectionConductorImpl.this);
                     }
                 }
             }
-            
+
         }).start();
     }
 
@@ -412,7 +342,7 @@ public class ConnectionConductorImpl implements OpenflowProtocolListener,
     }
 
     /**
-     * @param handshaking
+     * @param expectedState
      */
     protected void checkState(CONDUCTOR_STATE expectedState) {
         if (!conductorState.equals(expectedState)) {
@@ -427,60 +357,6 @@ public class ConnectionConductorImpl implements OpenflowProtocolListener,
         sessionManager.invalidateOnDisconnect(this);
     }
 
-    /**
-     * find supported version based on remoteVersion
-     * @param remoteVersion
-     * @return
-     */
-    protected short proposeVersion(short remoteVersion) {
-        Short proposal = null;
-        for (short offer : ConnectionConductor.versionOrder) {
-            if (offer <= remoteVersion) {
-                proposal = offer;
-                break;
-            }
-        }
-        if (proposal == null) {
-            throw new IllegalArgumentException("unsupported version: "
-                    + remoteVersion);
-        }
-        return proposal;
-    }
-
-    /**
-     * find common highest supported bitmap version
-     * @param list
-     * @return
-     */
-    protected Short proposeBitmapVersion(List<Elements> list)
-    {
-        Short supportedHighestVersion = null;
-        if((null != list) && (0 != list.size()))
-        {
-           for(Elements element : list)
-           {
-              List<Boolean> bitmap = element.getVersionBitmap();
-              // check for version bitmap
-              for(short bitPos : ConnectionConductor.versionOrder)
-              {
-                  // with all the version it should work.
-                  if(bitmap.get(bitPos % Integer.SIZE))
-                  {
-                      supportedHighestVersion = bitPos;
-                      break;
-                  }
-              }
-           }
-           if(null == supportedHighestVersion)
-            {
-                throw new IllegalArgumentException("unsupported bitmap version.");
-            }
-
-        }
-
-        return supportedHighestVersion;
-    }
-
     @Override
     public Short getVersion() {
         return version;
@@ -488,8 +364,8 @@ public class ConnectionConductorImpl implements OpenflowProtocolListener,
 
     @Override
     public Future<Boolean> disconnect() {
-        LOG.info("disconnecting: sessionCtx="+sessionContext+"|auxId="+auxiliaryKey);
-        
+        LOG.trace("disconnecting: sessionCtx={}|auxId={}", sessionContext, auxiliaryKey);
+
         Future<Boolean> result = null;
         if (connectionAdapter.isAlive()) {
             result = connectionAdapter.disconnect();
@@ -497,8 +373,8 @@ public class ConnectionConductorImpl implements OpenflowProtocolListener,
             LOG.debug("connection already disconnected");
             result = Futures.immediateFuture(true);
         }
-        
-        return result; 
+
+        return result;
     }
 
     @Override
@@ -520,7 +396,7 @@ public class ConnectionConductorImpl implements OpenflowProtocolListener,
     public SessionContext getSessionContext() {
         return sessionContext;
     }
-    
+
     @Override
     public ConnectionAdapter getConnectionAdapter() {
         return connectionAdapter;
@@ -529,12 +405,128 @@ public class ConnectionConductorImpl implements OpenflowProtocolListener,
     @Override
     public void onConnectionReady() {
         LOG.debug("connection is ready-to-use");
-        new Thread(new Runnable() {
-            @Override
-            public void run() {
-                sendFirstHelloMessage();
-            }
-        }).start();
+        if (! firstHelloProcessed) {
+            HandshakeStepWrapper handshakeStepWrapper = new HandshakeStepWrapper(
+                    null, handshakeManager, connectionAdapter);
+            hsPool.execute(handshakeStepWrapper);
+            firstHelloProcessed = true;
+        } else {
+            LOG.debug("already touched by hello message");
+        }
+    }
+
+    @Override
+    public void onHandshakeSuccessfull(GetFeaturesOutput featureOutput,
+            Short negotiatedVersion) {
+        postHandshakeBasic(featureOutput, negotiatedVersion);
+        
+        // post-handshake actions
+        if(version == OFConstants.OFP_VERSION_1_3){
+            setDefaultConfig();
+            requestPorts();
+            requestGroupFeatures();
+            requestMeterFeatures();
+        } else if (version == OFConstants.OFP_VERSION_1_0) {
+            //  Because the GetFeaturesOutput contains information about the port
+            //  in OF1.0 (that we would otherwise get from the PortDesc) we have to pass
+            //  it up for parsing to convert into a NodeConnectorUpdate
+            enqueueMessage(featureOutput);
+        }
+        
+        requestDesc();
+    }
+
+    /**
+     * used by tests
+     * @param featureOutput
+     * @param negotiatedVersion
+     */
+    protected void postHandshakeBasic(GetFeaturesOutput featureOutput,
+            Short negotiatedVersion) {
+        version = negotiatedVersion;
+        conductorState = CONDUCTOR_STATE.WORKING;
+        OFSessionUtil.registerSession(this, featureOutput, negotiatedVersion);
+        hsPool.shutdown();
+        hsPool.purge();
+    }
+
+    private void setDefaultConfig(){
+        SetConfigInputBuilder builder = new SetConfigInputBuilder();
+        builder.setVersion(getVersion());
+        builder.setXid(getSessionContext().getNextXid());
+        SwitchConfigFlag flag = SwitchConfigFlag.FRAGNORMAL;
+        builder.setFlags(flag);
+        builder.setMissSendLen(OFConstants.OFPCML_NO_BUFFER);
+        getConnectionAdapter().setConfig(builder.build());
+    }
+
+    /*
+     *  Send an OFPMP_DESC request message to the switch
+     */
+    private void requestDesc() {
+        MultipartRequestInputBuilder builder = new MultipartRequestInputBuilder();
+        builder.setType(MultipartType.OFPMPDESC);
+        builder.setVersion(getVersion());
+        builder.setFlags(new MultipartRequestFlags(false));
+        builder.setMultipartRequestBody(new MultipartRequestDescCaseBuilder().build());
+        builder.setXid(getSessionContext().getNextXid());
+        getConnectionAdapter().multipartRequest(builder.build());
+    }
+
+    private void requestPorts() {
+        MultipartRequestInputBuilder builder = new MultipartRequestInputBuilder();
+        builder.setType(MultipartType.OFPMPPORTDESC);
+        builder.setVersion(getVersion());
+        builder.setFlags(new MultipartRequestFlags(false));
+        builder.setMultipartRequestBody(new MultipartRequestPortDescCaseBuilder().build());
+        builder.setXid(getSessionContext().getNextXid());
+        getConnectionAdapter().multipartRequest(builder.build());
+    }
+    private void requestGroupFeatures(){
+        MultipartRequestInputBuilder mprInput = new MultipartRequestInputBuilder();
+        mprInput.setType(MultipartType.OFPMPGROUPFEATURES);
+        mprInput.setVersion(getVersion());
+        mprInput.setFlags(new MultipartRequestFlags(false));
+        mprInput.setXid(getSessionContext().getNextXid());
+
+        MultipartRequestGroupFeaturesCaseBuilder mprGroupFeaturesBuild = 
+                new MultipartRequestGroupFeaturesCaseBuilder();
+        mprInput.setMultipartRequestBody(mprGroupFeaturesBuild.build());
+
+        LOG.debug("Send group features statistics request :{}",mprGroupFeaturesBuild);
+        getConnectionAdapter().multipartRequest(mprInput.build());
+        
+    }
+    private void requestMeterFeatures(){
+        MultipartRequestInputBuilder mprInput = new MultipartRequestInputBuilder();
+        mprInput.setType(MultipartType.OFPMPMETERFEATURES);
+        mprInput.setVersion(getVersion());
+        mprInput.setFlags(new MultipartRequestFlags(false));
+        mprInput.setXid(getSessionContext().getNextXid());
+
+        MultipartRequestMeterFeaturesCaseBuilder mprMeterFeaturesBuild =
+                new MultipartRequestMeterFeaturesCaseBuilder();
+        mprInput.setMultipartRequestBody(mprMeterFeaturesBuild.build());
+
+        LOG.debug("Send meter features statistics request :{}",mprMeterFeaturesBuild);
+        getConnectionAdapter().multipartRequest(mprInput.build());
+        
+    }
+    /**
+     * @param isBitmapNegotiationEnable the isBitmapNegotiationEnable to set
+     */
+    public void setBitmapNegotiationEnable(
+            boolean isBitmapNegotiationEnable) {
+        this.isBitmapNegotiationEnable = isBitmapNegotiationEnable;
+    }
+
+    protected void shutdownPool() {
+        hsPool.shutdownNow();
+        LOG.debug("pool is terminated: {}", hsPool.isTerminated());
     }
     
+    @Override
+    public void setId(int conductorId) {
+        this.conductorId = conductorId;
+    }
 }