Disconnection improvements.
[openflowplugin.git] / openflowplugin-impl / src / main / java / org / opendaylight / openflowplugin / impl / device / DeviceContextImpl.java
index 6b53858e870a6f4d5e7b80441b82a704fab2ebe7..c07b67aa91615bb8e319193d3f148a6f7ceb7d55 100644 (file)
@@ -8,27 +8,39 @@
 package org.opendaylight.openflowplugin.impl.device;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Verify;
+import com.google.common.util.concurrent.AsyncFunction;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.JdkFutureAdapters;
 import com.google.common.util.concurrent.ListenableFuture;
+import io.netty.util.HashedWheelTimer;
 import io.netty.util.Timeout;
+import io.netty.util.TimerTask;
 import java.math.BigInteger;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
 import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 import org.opendaylight.controller.md.sal.binding.api.DataBroker;
 import org.opendaylight.controller.md.sal.binding.api.NotificationPublishService;
 import org.opendaylight.controller.md.sal.binding.api.ReadOnlyTransaction;
 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
-import org.opendaylight.controller.md.sal.common.api.data.TransactionChainClosedException;
 import org.opendaylight.mdsal.singleton.common.api.ServiceGroupIdentifier;
 import org.opendaylight.openflowjava.protocol.api.connection.ConnectionAdapter;
+import org.opendaylight.openflowjava.protocol.api.connection.OutboundQueueHandlerRegistration;
 import org.opendaylight.openflowjava.protocol.api.keys.MessageTypeKey;
+import org.opendaylight.openflowplugin.api.ConnectionException;
+import org.opendaylight.openflowplugin.api.OFConstants;
 import org.opendaylight.openflowplugin.api.openflow.connection.ConnectionContext;
+import org.opendaylight.openflowplugin.api.openflow.connection.OutboundQueueProvider;
 import org.opendaylight.openflowplugin.api.openflow.device.DeviceContext;
 import org.opendaylight.openflowplugin.api.openflow.device.DeviceInfo;
 import org.opendaylight.openflowplugin.api.openflow.device.DeviceManager;
@@ -37,10 +49,13 @@ import org.opendaylight.openflowplugin.api.openflow.device.MessageTranslator;
 import org.opendaylight.openflowplugin.api.openflow.device.RequestContext;
 import org.opendaylight.openflowplugin.api.openflow.device.TranslatorLibrary;
 import org.opendaylight.openflowplugin.api.openflow.device.Xid;
+import org.opendaylight.openflowplugin.api.openflow.device.handlers.ClusterInitializationPhaseHandler;
 import org.opendaylight.openflowplugin.api.openflow.device.handlers.MultiMsgCollector;
 import org.opendaylight.openflowplugin.api.openflow.lifecycle.LifecycleService;
+import org.opendaylight.openflowplugin.api.openflow.lifecycle.MastershipChangeListener;
 import org.opendaylight.openflowplugin.api.openflow.md.core.SwitchConnectionDistinguisher;
 import org.opendaylight.openflowplugin.api.openflow.md.core.TranslatorKey;
+import org.opendaylight.openflowplugin.api.openflow.md.util.OpenflowVersion;
 import org.opendaylight.openflowplugin.api.openflow.registry.ItemLifeCycleRegistry;
 import org.opendaylight.openflowplugin.api.openflow.registry.flow.DeviceFlowRegistry;
 import org.opendaylight.openflowplugin.api.openflow.registry.flow.FlowDescriptor;
@@ -56,15 +71,22 @@ import org.opendaylight.openflowplugin.extension.api.core.extension.ExtensionCon
 import org.opendaylight.openflowplugin.extension.api.exception.ConversionException;
 import org.opendaylight.openflowplugin.extension.api.path.MessagePath;
 import org.opendaylight.openflowplugin.impl.common.ItemLifeCycleSourceImpl;
-import org.opendaylight.openflowplugin.impl.common.NodeStaticReplyTranslatorUtil;
+import org.opendaylight.openflowplugin.impl.connection.OutboundQueueProviderImpl;
+import org.opendaylight.openflowplugin.impl.datastore.MultipartWriterProvider;
+import org.opendaylight.openflowplugin.impl.datastore.MultipartWriterProviderFactory;
+import org.opendaylight.openflowplugin.impl.device.initialization.AbstractDeviceInitializer;
+import org.opendaylight.openflowplugin.impl.device.initialization.DeviceInitializerProvider;
 import org.opendaylight.openflowplugin.impl.device.listener.MultiMsgCollectorImpl;
+import org.opendaylight.openflowplugin.impl.device.listener.OpenflowProtocolListenerFullImpl;
 import org.opendaylight.openflowplugin.impl.registry.flow.DeviceFlowRegistryImpl;
 import org.opendaylight.openflowplugin.impl.registry.flow.FlowRegistryKeyFactory;
 import org.opendaylight.openflowplugin.impl.registry.group.DeviceGroupRegistryImpl;
 import org.opendaylight.openflowplugin.impl.registry.meter.DeviceMeterRegistryImpl;
-import org.opendaylight.openflowplugin.impl.util.DeviceInitializationUtils;
+import org.opendaylight.openflowplugin.impl.rpc.AbstractRequestContext;
+import org.opendaylight.openflowplugin.impl.services.sal.SalRoleServiceImpl;
 import org.opendaylight.openflowplugin.openflow.md.core.sal.convertor.ConvertorExecutor;
 import org.opendaylight.openflowplugin.openflow.md.core.session.SwitchConnectionCookieOFImpl;
+import org.opendaylight.openflowplugin.openflow.md.util.InventoryDataServiceUtil;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.experimenter.message.service.rev151020.ExperimenterMessageFromDevBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNodeConnector;
@@ -73,6 +95,8 @@ import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.ta
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.tables.table.FlowKey;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeConnectorId;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeRef;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeRemovedBuilder;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeUpdatedBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.node.NodeConnector;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.node.NodeConnectorBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.node.NodeConnectorKey;
@@ -81,7 +105,6 @@ import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.common.types.rev13
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.Error;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.ExperimenterMessage;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.FlowRemoved;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.MultipartReply;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.OfHeader;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.PacketIn;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.PacketInMessage;
@@ -92,16 +115,19 @@ import org.opendaylight.yang.gen.v1.urn.opendaylight.openflowplugin.experimenter
 import org.opendaylight.yang.gen.v1.urn.opendaylight.packet.service.rev130709.PacketReceived;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.port.statistics.rev131214.FlowCapableNodeConnectorStatisticsData;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.port.statistics.rev131214.FlowCapableNodeConnectorStatisticsDataBuilder;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.role.service.rev150727.OfpRole;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.role.service.rev150727.SalRoleService;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.role.service.rev150727.SetRoleInput;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.role.service.rev150727.SetRoleInputBuilder;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.role.service.rev150727.SetRoleOutput;
 import org.opendaylight.yangtools.yang.binding.DataObject;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
 import org.opendaylight.yangtools.yang.binding.KeyedInstanceIdentifier;
+import org.opendaylight.yangtools.yang.common.RpcResult;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- *
- */
-public class DeviceContextImpl implements DeviceContext, ExtensionConverterProviderKeeper{
+public class DeviceContextImpl implements DeviceContext, ExtensionConverterProviderKeeper {
 
     private static final Logger LOG = LoggerFactory.getLogger(DeviceContextImpl.class);
 
@@ -112,15 +138,25 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     // TODO: high water mark factor should be parametrized
     private static final float HIGH_WATERMARK_FACTOR = 0.95f;
 
-    private ConnectionContext primaryConnectionContext;
+    // Timeout in seconds after what we will give up on propagating role
+    private static final int SET_ROLE_TIMEOUT = 10;
+
+    private static final int LOW_WATERMARK = 1000;
+    private static final int HIGH_WATERMARK = 2000;
+
+    private boolean initialized;
+
+    private SalRoleService salRoleService = null;
+    private final HashedWheelTimer hashedWheelTimer;
+    private volatile ConnectionContext primaryConnectionContext;
     private final DeviceState deviceState;
     private final DataBroker dataBroker;
     private final Map<SwitchConnectionDistinguisher, ConnectionContext> auxiliaryConnectionContexts;
-    private final TransactionChainManager transactionChainManager;
-    private final DeviceFlowRegistry deviceFlowRegistry;
-    private final DeviceGroupRegistry deviceGroupRegistry;
-    private final DeviceMeterRegistry deviceMeterRegistry;
-    private final PacketInRateLimiter packetInLimiter;
+    private TransactionChainManager transactionChainManager;
+    private DeviceFlowRegistry deviceFlowRegistry;
+    private DeviceGroupRegistry deviceGroupRegistry;
+    private DeviceMeterRegistry deviceMeterRegistry;
+    private PacketInRateLimiter packetInLimiter;
     private final MessageSpy messageSpy;
     private final ItemLifeCycleKeeper flowLifeCycleKeeper;
     private NotificationPublishService notificationPublishService;
@@ -131,59 +167,69 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     private final TranslatorLibrary translatorLibrary;
     private final ItemLifeCycleRegistry itemLifeCycleSourceRegistry;
     private ExtensionConverterProvider extensionConverterProvider;
-    private final DeviceManager deviceManager;
     private boolean skipTableFeatures;
     private boolean switchFeaturesMandatory;
-    private final DeviceInfo deviceInfo;
+    private DeviceInfo deviceInfo;
     private final ConvertorExecutor convertorExecutor;
     private volatile CONTEXT_STATE state;
-
-    public DeviceContextImpl(
-            @Nonnull final ConnectionContext primaryConnectionContext,
-            @Nonnull final DataBroker dataBroker,
-            @Nonnull final MessageSpy messageSpy,
-            @Nonnull final TranslatorLibrary translatorLibrary,
-            @Nonnull final DeviceManager manager,
-            final ConvertorExecutor convertorExecutor,
-            final boolean skipTableFeatures) {
-        this.primaryConnectionContext = Preconditions.checkNotNull(primaryConnectionContext);
+    private ClusterInitializationPhaseHandler clusterInitializationPhaseHandler;
+    private final DeviceManager myManager;
+    private final DeviceInitializerProvider deviceInitializerProvider;
+    private final boolean useSingleLayerSerialization;
+    private Boolean isAddNotificationSent = false;
+    private OutboundQueueProvider outboundQueueProvider;
+    private boolean wasOnceMaster;
+
+    DeviceContextImpl(
+        @Nonnull final ConnectionContext primaryConnectionContext,
+        @Nonnull final DataBroker dataBroker,
+        @Nonnull final MessageSpy messageSpy,
+        @Nonnull final TranslatorLibrary translatorLibrary,
+        @Nonnull final DeviceManager contextManager,
+        final ConvertorExecutor convertorExecutor,
+        final boolean skipTableFeatures,
+        final HashedWheelTimer hashedWheelTimer,
+        final boolean useSingleLayerSerialization,
+        final DeviceInitializerProvider deviceInitializerProvider) {
+
+        this.primaryConnectionContext = primaryConnectionContext;
+        this.outboundQueueProvider = (OutboundQueueProvider) primaryConnectionContext.getOutboundQueueProvider();
         this.deviceInfo = primaryConnectionContext.getDeviceInfo();
+        this.hashedWheelTimer = hashedWheelTimer;
+        this.deviceInitializerProvider = deviceInitializerProvider;
+        this.myManager = contextManager;
         this.deviceState = new DeviceStateImpl();
-        this.dataBroker = Preconditions.checkNotNull(dataBroker);
-        this.transactionChainManager = new TransactionChainManager(dataBroker, deviceInfo);
-        auxiliaryConnectionContexts = new HashMap<>();
-        deviceFlowRegistry = new DeviceFlowRegistryImpl(dataBroker, deviceInfo.getNodeInstanceIdentifier());
-        deviceGroupRegistry = new DeviceGroupRegistryImpl();
-        deviceMeterRegistry = new DeviceMeterRegistryImpl();
+        this.dataBroker = dataBroker;
+        this.auxiliaryConnectionContexts = new HashMap<>();
         this.messageSpy = Preconditions.checkNotNull(messageSpy);
-        this.deviceManager = Preconditions.checkNotNull(manager);
 
-        packetInLimiter = new PacketInRateLimiter(primaryConnectionContext.getConnectionAdapter(),
-                /*initial*/ 1000, /*initial*/2000, this.messageSpy, REJECTED_DRAIN_FACTOR);
+        this.packetInLimiter = new PacketInRateLimiter(primaryConnectionContext.getConnectionAdapter(),
+                /*initial*/ LOW_WATERMARK, /*initial*/HIGH_WATERMARK, this.messageSpy, REJECTED_DRAIN_FACTOR);
 
         this.translatorLibrary = translatorLibrary;
-        portStatusTranslator = translatorLibrary.lookupTranslator(
+        this.portStatusTranslator = translatorLibrary.lookupTranslator(
                 new TranslatorKey(deviceInfo.getVersion(), PortGrouping.class.getName()));
-        packetInTranslator = translatorLibrary.lookupTranslator(
+        this.packetInTranslator = translatorLibrary.lookupTranslator(
                 new TranslatorKey(deviceInfo.getVersion(), PacketIn.class.getName()));
-        flowRemovedTranslator = translatorLibrary.lookupTranslator(
+        this.flowRemovedTranslator = translatorLibrary.lookupTranslator(
                 new TranslatorKey(deviceInfo.getVersion(), FlowRemoved.class.getName()));
 
-        itemLifeCycleSourceRegistry = new ItemLifeCycleRegistryImpl();
-        flowLifeCycleKeeper = new ItemLifeCycleSourceImpl();
-        itemLifeCycleSourceRegistry.registerLifeCycleSource(flowLifeCycleKeeper);
+        this.itemLifeCycleSourceRegistry = new ItemLifeCycleRegistryImpl();
+        this.flowLifeCycleKeeper = new ItemLifeCycleSourceImpl();
+        this.itemLifeCycleSourceRegistry.registerLifeCycleSource(flowLifeCycleKeeper);
         this.state = CONTEXT_STATE.INITIALIZATION;
         this.convertorExecutor = convertorExecutor;
         this.skipTableFeatures = skipTableFeatures;
+        this.useSingleLayerSerialization = useSingleLayerSerialization;
+        this.initialized = false;
+        this.wasOnceMaster = false;
     }
 
-    /**
-     * This method is called from {@link DeviceManagerImpl} only. So we could say "posthandshake process finish"
-     * and we are able to set a scheduler for an automatic transaction submitting by time (0,5sec).
-     */
     @Override
     public void initialSubmitTransaction() {
-        transactionChainManager.initialSubmitWriteTransaction();
+        if (initialized) {
+            transactionChainManager.initialSubmitWriteTransaction();
+        }
     }
 
     @Override
@@ -218,24 +264,30 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     public <T extends DataObject> void writeToTransaction(final LogicalDatastoreType store,
                                                           final InstanceIdentifier<T> path,
                                                           final T data){
-        transactionChainManager.writeToTransaction(store, path, data, false);
+        if (initialized) {
+            transactionChainManager.writeToTransaction(store, path, data, false);
+        }
     }
 
     @Override
     public <T extends DataObject> void writeToTransactionWithParentsSlow(final LogicalDatastoreType store,
                                                                          final InstanceIdentifier<T> path,
                                                                          final T data){
-        transactionChainManager.writeToTransaction(store, path, data, true);
+        if (initialized) {
+            transactionChainManager.writeToTransaction(store, path, data, true);
+        }
     }
 
     @Override
-    public <T extends DataObject> void addDeleteToTxChain(final LogicalDatastoreType store, final InstanceIdentifier<T> path) throws TransactionChainClosedException {
-        transactionChainManager.addDeleteOperationTotTxChain(store, path);
+    public <T extends DataObject> void addDeleteToTxChain(final LogicalDatastoreType store, final InstanceIdentifier<T> path) {
+        if (initialized) {
+            transactionChainManager.addDeleteOperationTotTxChain(store, path);
+        }
     }
 
     @Override
     public boolean submitTransaction() {
-        return transactionChainManager.submitWriteTransaction();
+        return initialized && transactionChainManager.submitWriteTransaction();
     }
 
     @Override
@@ -244,7 +296,7 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     }
 
     @Override
-    public ConnectionContext getAuxiliaryConnectiobContexts(final BigInteger cookie) {
+    public ConnectionContext getAuxiliaryConnectionContexts(final BigInteger cookie) {
         return auxiliaryConnectionContexts.get(new SwitchConnectionCookieOFImpl(cookie.longValue()));
     }
 
@@ -265,18 +317,20 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
 
     @Override
     public void processReply(final OfHeader ofHeader) {
-        if (ofHeader instanceof Error) {
-            messageSpy.spyMessage(ofHeader.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_FAILURE);
-        } else {
-            messageSpy.spyMessage(ofHeader.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_SUCCESS);
-        }
+        messageSpy.spyMessage(
+            ofHeader.getImplementedInterface(),
+            (ofHeader instanceof Error)
+                ? MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_FAILURE
+                : MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_SUCCESS);
     }
 
     @Override
-    public void processReply(final Xid xid, final List<MultipartReply> ofHeaderList) {
-        for (final MultipartReply multipartReply : ofHeaderList) {
-            messageSpy.spyMessage(multipartReply.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_FAILURE);
-        }
+    public void processReply(final Xid xid, final List<? extends OfHeader> ofHeaderList) {
+        ofHeaderList.forEach(header -> messageSpy.spyMessage(
+            header.getImplementedInterface(),
+            (header instanceof Error)
+                ? MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_FAILURE
+                : MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_SUCCESS));
     }
 
     @Override
@@ -285,19 +339,19 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
         final org.opendaylight.yang.gen.v1.urn.opendaylight.flow.service.rev130819.FlowRemoved flowRemovedNotification =
                 flowRemovedTranslator.translate(flowRemoved, deviceInfo, null);
 
-        if(!deviceManager.getIsNotificationFlowRemovedOff()) {
+        if(!myManager.isFlowRemovedNotificationOn()) {
             // Trigger off a notification
             notificationPublishService.offerNotification(flowRemovedNotification);
         } else if(LOG.isDebugEnabled()) {
-            LOG.debug("For nodeId={} isNotificationFlowRemovedOff={}", getDeviceInfo().getLOGValue(), deviceManager.getIsNotificationFlowRemovedOff());
+            LOG.debug("For nodeId={} isNotificationFlowRemovedOn={}", getDeviceInfo().getLOGValue(), myManager.isFlowRemovedNotificationOn());
         }
 
         final ItemLifecycleListener itemLifecycleListener = flowLifeCycleKeeper.getItemLifecycleListener();
         if (itemLifecycleListener != null) {
             //2. create registry key
-            final FlowRegistryKey flowRegKey = FlowRegistryKeyFactory.create(flowRemovedNotification);
+            final FlowRegistryKey flowRegKey = FlowRegistryKeyFactory.create(getDeviceInfo().getVersion(), flowRemovedNotification);
             //3. lookup flowId
-            final FlowDescriptor flowDescriptor = deviceFlowRegistry.retrieveIdForFlow(flowRegKey);
+            final FlowDescriptor flowDescriptor = deviceFlowRegistry.retrieveDescriptor(flowRegKey);
             //4. if flowId present:
             if (flowDescriptor != null) {
                 // a) construct flow path
@@ -314,6 +368,26 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
         }
     }
 
+    @Override
+    public void sendNodeAddedNotification() {
+        if (!isAddNotificationSent) {
+            isAddNotificationSent = true;
+            NodeUpdatedBuilder builder = new NodeUpdatedBuilder();
+            builder.setId(getDeviceInfo().getNodeId());
+            builder.setNodeRef(new NodeRef(getDeviceInfo().getNodeInstanceIdentifier()));
+            LOG.debug("Publishing node added notification for {}", builder.build());
+            notificationPublishService.offerNotification(builder.build());
+        }
+    }
+
+    @Override
+    public void sendNodeRemovedNotification() {
+        NodeRemovedBuilder builder = new NodeRemovedBuilder();
+        builder.setNodeRef(new NodeRef(getDeviceInfo().getNodeInstanceIdentifier()));
+        LOG.debug("Publishing node removed notification for {}", builder.build());
+        notificationPublishService.offerNotification(builder.build());
+    }
+
     @Override
     public void processPortStatusMessage(final PortStatusMessage portStatus) {
         messageSpy.spyMessage(portStatus.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_SUCCESS);
@@ -332,14 +406,15 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
             }
             submitTransaction();
         } catch (final Exception e) {
-            LOG.warn("Error processing port status message: ", e);
+            LOG.warn("Error processing port status message for port {} on device {} : {}", portStatus.getPortNo(),
+                    getDeviceInfo().getNodeId().toString(), e);
         }
     }
 
     private KeyedInstanceIdentifier<NodeConnector, NodeConnectorKey> provideIIToNodeConnector(final long portNo, final short version) {
         final InstanceIdentifier<Node> iiToNodes = getDeviceInfo().getNodeInstanceIdentifier();
         final BigInteger dataPathId = getDeviceInfo().getDatapathId();
-        final NodeConnectorId nodeConnectorId = NodeStaticReplyTranslatorUtil.nodeConnectorId(dataPathId.toString(), portNo, version);
+        final NodeConnectorId nodeConnectorId = InventoryDataServiceUtil.nodeConnectorIdfromDatapathPortNo(dataPathId, portNo, OpenflowVersion.get(version));
         return iiToNodes.child(NodeConnector.class, new NodeConnectorKey(nodeConnectorId));
     }
 
@@ -423,15 +498,6 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
         return translatorLibrary;
     }
 
-    @Override
-    public synchronized void close() {
-        LOG.debug("closing deviceContext: {}, nodeId:{}",
-                getPrimaryConnectionContext().getConnectionAdapter().getRemoteAddress(),
-                getDeviceInfo().getLOGValue());
-        // NOOP
-        throw new UnsupportedOperationException("Autocloseble.close will be removed soon");
-    }
-
     @Override
     public void setCurrentBarrierTimeout(final Timeout timeout) {
         barrierTaskTimeout = timeout;
@@ -455,16 +521,18 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     @Override
     public void onPublished() {
         Verify.verify(CONTEXT_STATE.INITIALIZATION.equals(getState()));
-        setState(CONTEXT_STATE.WORKING);
-        primaryConnectionContext.getConnectionAdapter().setPacketInFiltering(false);
+        this.state = CONTEXT_STATE.WORKING;
+        synchronized (primaryConnectionContext) {
+            primaryConnectionContext.getConnectionAdapter().setPacketInFiltering(false);
+        }
         for (final ConnectionContext switchAuxConnectionContext : auxiliaryConnectionContexts.values()) {
             switchAuxConnectionContext.getConnectionAdapter().setPacketInFiltering(false);
         }
     }
 
     @Override
-    public MultiMsgCollector getMultiMsgCollector(final RequestContext<List<MultipartReply>> requestContext) {
-        return new MultiMsgCollectorImpl(this, requestContext);
+    public <T extends OfHeader> MultiMsgCollector<T> getMultiMsgCollector(final RequestContext<List<T>> requestContext) {
+        return new MultiMsgCollectorImpl<>(this, requestContext);
     }
 
     @Override
@@ -489,33 +557,41 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
 
     @Override
     public synchronized void shutdownConnection() {
-        LOG.debug("Shutdown method for node {}", getDeviceInfo().getLOGValue());
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Shutdown method for node {}", getDeviceInfo().getLOGValue());
+        }
         if (CONTEXT_STATE.TERMINATION.equals(getState())) {
             LOG.debug("DeviceCtx for Node {} is in termination process.", getDeviceInfo().getLOGValue());
             return;
         }
-        setState(CONTEXT_STATE.TERMINATION);
 
         if (ConnectionContext.CONNECTION_STATE.RIP.equals(getPrimaryConnectionContext().getConnectionState())) {
             LOG.debug("ConnectionCtx for Node {} is in RIP state.", getDeviceInfo().getLOGValue());
             return;
         }
-        /* Terminate Auxiliary Connection */
+
+        // Terminate Auxiliary Connection
         for (final ConnectionContext connectionContext : auxiliaryConnectionContexts.values()) {
             LOG.debug("Closing auxiliary connection {}", connectionContext.getNodeId());
             connectionContext.closeConnection(false);
         }
-        /* Terminate Primary Connection */
+
+        // Terminate Primary Connection
         getPrimaryConnectionContext().closeConnection(true);
-        /* Close all Group Registry */
-        deviceGroupRegistry.close();
-        deviceFlowRegistry.close();
-        deviceMeterRegistry.close();
+
+        // Close all datastore registries
+        if (initialized) {
+            deviceGroupRegistry.close();
+            deviceFlowRegistry.close();
+            deviceMeterRegistry.close();
+        }
     }
 
     @Override
     public ListenableFuture<Void> shuttingDownDataStoreTransactions() {
-        return transactionChainManager.shuttingDown();
+        return initialized
+                ? this.transactionChainManager.shuttingDown()
+                : Futures.immediateFuture(null);
     }
 
     @VisibleForTesting
@@ -529,26 +605,79 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     }
 
     @Override
-    public CONTEXT_STATE getState() {
-        return this.state;
-    }
+    public synchronized void replaceConnection(final ConnectionContext connectionContext) {
+
+        primaryConnectionContext = null;
+        deviceInfo = null;
+        packetInLimiter = null;
+
+        primaryConnectionContext = connectionContext;
+        deviceInfo = primaryConnectionContext.getDeviceInfo();
+
+        packetInLimiter = new PacketInRateLimiter(primaryConnectionContext.getConnectionAdapter(),
+                /*initial*/ LOW_WATERMARK, /*initial*/HIGH_WATERMARK, messageSpy, REJECTED_DRAIN_FACTOR);
+
+        primaryConnectionContext.setOutboundQueueProvider(outboundQueueProvider);
+
+        final OutboundQueueHandlerRegistration<OutboundQueueProvider> outboundQueueHandlerRegistration =
+                primaryConnectionContext.getConnectionAdapter().registerOutboundQueueHandler(
+                        outboundQueueProvider,
+                        myManager.getBarrierCountLimit(),
+                        myManager.getBarrierIntervalNanos());
+
+        primaryConnectionContext.setOutboundQueueHandleRegistration(outboundQueueHandlerRegistration);
+
+        final OpenflowProtocolListenerFullImpl messageListener = new OpenflowProtocolListenerFullImpl(
+                primaryConnectionContext.getConnectionAdapter(), this);
+
+        primaryConnectionContext.getConnectionAdapter().setMessageListener(messageListener);
+
+        LOG.info("ConnectionEvent: Connection on device:{}, NodeId:{} switched.",
+                primaryConnectionContext.getConnectionAdapter().getRemoteAddress(),
+                primaryConnectionContext.getDeviceInfo().getNodeId());
 
-    @Override
-    public void setState(CONTEXT_STATE state) {
-        this.state = state;
     }
 
     @Override
-    public void startupClusterServices() throws ExecutionException, InterruptedException {
-        LOG.debug("Initializing transaction chain manager for node {}", getDeviceInfo().getLOGValue());
-        this.transactionChainManager.activateTransactionManager();
-        LOG.debug("Waiting to get node {} information", getDeviceInfo().getLOGValue());
-        DeviceInitializationUtils.initializeNodeInformation(this, switchFeaturesMandatory, this.convertorExecutor);
+    public CONTEXT_STATE getState() {
+        return this.state;
     }
 
     @Override
-    public ListenableFuture<Void> stopClusterServices(boolean deviceDisconnected) {
-        return this.transactionChainManager.deactivateTransactionManager();
+    public ListenableFuture<Void> stopClusterServices(boolean connectionInterrupted) {
+        final ListenableFuture<Void> deactivateTxManagerFuture = initialized
+                ? transactionChainManager.deactivateTransactionManager()
+                : Futures.immediateFuture(null);
+
+        if (!connectionInterrupted) {
+            final ListenableFuture<Void> makeSlaveFuture
+                    = Futures.transform(makeDeviceSlave(), new Function<RpcResult<SetRoleOutput>, Void>() {
+                @Nullable
+                @Override
+                public Void apply(@Nullable RpcResult<SetRoleOutput> setRoleOutputRpcResult) {
+                    return null;
+                }
+            });
+
+            Futures.addCallback(makeSlaveFuture, new FutureCallback<Void>() {
+                @Override
+                public void onSuccess(@Nullable Void aVoid) {
+                    if (LOG.isDebugEnabled()) {
+                        LOG.debug("Role SLAVE was successfully propagated on device, node {}", deviceInfo.getLOGValue());
+                    }
+                    sendNodeAddedNotification();
+                }
+
+                @Override
+                public void onFailure(final Throwable throwable) {
+                    LOG.warn("Was not able to set role SLAVE to device on node {} ", deviceInfo.getLOGValue());
+                    LOG.trace("Error occurred on device role setting, probably connection loss: ", throwable);
+                }
+            });
+
+        }
+
+        return deactivateTxManagerFuture;
     }
 
     @Override
@@ -561,21 +690,189 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
         return this.deviceInfo;
     }
 
+    @Override
+    public void close() {
+        if (CONTEXT_STATE.TERMINATION.equals(getState())){
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("DeviceContext for node {} is already in TERMINATION state.", getDeviceInfo().getLOGValue());
+            }
+        } else {
+            this.state = CONTEXT_STATE.TERMINATION;
+        }
+        sendNodeRemovedNotification();
+    }
+
     @Override
     public void putLifecycleServiceIntoTxChainManager(final LifecycleService lifecycleService){
-        this.transactionChainManager.setLifecycleService(lifecycleService);
+        if (initialized) {
+            this.transactionChainManager.setLifecycleService(lifecycleService);
+        }
     }
 
     @Override
-    public void replaceConnectionContext(final ConnectionContext connectionContext){
-        // Act like we are initializing the context
-        setState(CONTEXT_STATE.INITIALIZATION);
-        this.primaryConnectionContext = connectionContext;
-        this.onPublished();
+    public boolean canUseSingleLayerSerialization() {
+        return useSingleLayerSerialization && getDeviceInfo().getVersion() >= OFConstants.OFP_VERSION_1_3;
     }
 
     @Override
     public boolean isSkipTableFeatures() {
         return this.skipTableFeatures;
     }
+
+    @Override
+    public void setSalRoleService(@Nonnull SalRoleService salRoleService) {
+        this.salRoleService = salRoleService;
+    }
+
+    @Override
+    public void setLifecycleInitializationPhaseHandler(final ClusterInitializationPhaseHandler handler) {
+        this.clusterInitializationPhaseHandler = handler;
+    }
+
+    @Override
+    public void masterSuccessful(){
+        this.wasOnceMaster = true;
+    }
+
+    @Override
+    public boolean onContextInstantiateService(final MastershipChangeListener mastershipChangeListener) {
+
+        LOG.info("Starting device context cluster services for node {}", deviceInfo.getLOGValue());
+
+        lazyTransactionManagerInitialization();
+
+        this.transactionChainManager.activateTransactionManager();
+
+        try {
+            final java.util.Optional<AbstractDeviceInitializer> initializer = deviceInitializerProvider
+                .lookup(deviceInfo.getVersion());
+
+            if (initializer.isPresent()) {
+                final MultipartWriterProvider writerProvider = MultipartWriterProviderFactory.createDefaultProvider(this);
+                initializer.get().initialize(this, switchFeaturesMandatory, writerProvider, convertorExecutor);
+            } else {
+                throw new ExecutionException(new ConnectionException("Unsupported version " + deviceInfo.getVersion()));
+            }
+        } catch (ExecutionException | InterruptedException e) {
+            LOG.warn("Device {} cannot be initialized: ", deviceInfo.getLOGValue(), e);
+            return false;
+        }
+
+        Futures.addCallback(sendRoleChangeToDevice(OfpRole.BECOMEMASTER),
+                new RpcResultFutureCallback(mastershipChangeListener));
+
+        final ListenableFuture<List<Optional<FlowCapableNode>>> deviceFlowRegistryFill = getDeviceFlowRegistry().fill();
+        Futures.addCallback(deviceFlowRegistryFill, new DeviceFlowRegistryCallback(deviceFlowRegistryFill));
+
+        return this.clusterInitializationPhaseHandler.onContextInstantiateService(mastershipChangeListener);
+    }
+
+    @VisibleForTesting
+    void lazyTransactionManagerInitialization() {
+        if (!this.initialized) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Transaction chain manager for node {} created", deviceInfo.getLOGValue());
+            }
+            this.transactionChainManager = new TransactionChainManager(dataBroker, deviceInfo);
+            this.deviceFlowRegistry = new DeviceFlowRegistryImpl(deviceInfo.getVersion(), dataBroker, deviceInfo.getNodeInstanceIdentifier());
+            this.deviceGroupRegistry = new DeviceGroupRegistryImpl();
+            this.deviceMeterRegistry = new DeviceMeterRegistryImpl();
+            this.initialized = true;
+        }
+    }
+
+    @Nullable
+    @Override
+    public <T> RequestContext<T> createRequestContext() {
+        return new AbstractRequestContext<T>(deviceInfo.reserveXidForDeviceMessage()) {
+            @Override
+            public void close() {
+            }
+        };
+
+    }
+
+    ListenableFuture<RpcResult<SetRoleOutput>> sendRoleChangeToDevice(final OfpRole newRole) {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Sending new role {} to device {}", newRole, deviceInfo.getNodeId());
+        }
+
+        final Future<RpcResult<SetRoleOutput>> setRoleOutputFuture;
+
+        if (deviceInfo.getVersion() >= OFConstants.OFP_VERSION_1_3) {
+            final SetRoleInput setRoleInput = (new SetRoleInputBuilder()).setControllerRole(newRole)
+                    .setNode(new NodeRef(deviceInfo.getNodeInstanceIdentifier())).build();
+
+            setRoleOutputFuture = this.salRoleService.setRole(setRoleInput);
+
+            final TimerTask timerTask = timeout -> {
+                if (!setRoleOutputFuture.isDone()) {
+                    LOG.warn("New role {} was not propagated to device {} during {} sec", newRole, deviceInfo.getLOGValue(), SET_ROLE_TIMEOUT);
+                    setRoleOutputFuture.cancel(true);
+                }
+            };
+
+            hashedWheelTimer.newTimeout(timerTask, SET_ROLE_TIMEOUT, TimeUnit.SECONDS);
+        } else {
+            LOG.info("Device: {} with version: {} does not support role", deviceInfo.getLOGValue(), deviceInfo.getVersion());
+            return Futures.immediateFuture(null);
+        }
+
+        return JdkFutureAdapters.listenInPoolThread(setRoleOutputFuture);
+    }
+
+    @Override
+    public ListenableFuture<RpcResult<SetRoleOutput>> makeDeviceSlave() {
+        return sendRoleChangeToDevice(OfpRole.BECOMESLAVE);
+    }
+
+    private class RpcResultFutureCallback implements FutureCallback<RpcResult<SetRoleOutput>> {
+
+        private final MastershipChangeListener mastershipChangeListener;
+
+        RpcResultFutureCallback(final MastershipChangeListener mastershipChangeListener) {
+            this.mastershipChangeListener = mastershipChangeListener;
+        }
+
+        @Override
+        public void onSuccess(@Nullable RpcResult<SetRoleOutput> setRoleOutputRpcResult) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Role MASTER was successfully set on device, node {}", deviceInfo.getLOGValue());
+            }
+            sendNodeAddedNotification();
+        }
+
+        @Override
+        public void onFailure(final Throwable throwable) {
+            LOG.warn("Was not able to set MASTER role on device, node {}", deviceInfo.getLOGValue());
+            mastershipChangeListener.onNotAbleToStartMastership(deviceInfo);
+        }
+    }
+
+    private class DeviceFlowRegistryCallback implements FutureCallback<List<Optional<FlowCapableNode>>> {
+        private final ListenableFuture<List<Optional<FlowCapableNode>>> deviceFlowRegistryFill;
+
+        DeviceFlowRegistryCallback(ListenableFuture<List<Optional<FlowCapableNode>>> deviceFlowRegistryFill) {
+            this.deviceFlowRegistryFill = deviceFlowRegistryFill;
+        }
+
+        @Override
+        public void onSuccess(@Nullable List<Optional<FlowCapableNode>> result) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Finished filling flow registry with flows for node: {}", deviceInfo.getLOGValue());
+            }
+        }
+
+        @Override
+        public void onFailure(Throwable t) {
+            if (deviceFlowRegistryFill.isCancelled()) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("Cancelled filling flow registry with flows for node: {}", deviceInfo.getLOGValue());
+                }
+            } else {
+                LOG.warn("Failed filling flow registry with flows for node: {} with exception: {}", deviceInfo.getLOGValue(), t);
+            }
+        }
+    }
+
 }