BUG-4084: Li:Save flows in operational based on barrier success
[openflowplugin.git] / openflowplugin-impl / src / main / java / org / opendaylight / openflowplugin / impl / device / DeviceContextImpl.java
index 69a6e2f1788a4c87904154d8ab3512d7d7525f48..16ae98f5996fd690e030f2bd7e4359cb4f90eb86 100644 (file)
@@ -9,48 +9,52 @@ package org.opendaylight.openflowplugin.impl.device;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import com.google.common.util.concurrent.SettableFuture;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import io.netty.util.HashedWheelTimer;
 import io.netty.util.Timeout;
 import java.math.BigInteger;
-import java.util.ArrayList;
-import java.util.Collections;
+import java.util.Collection;
 import java.util.HashMap;
-import java.util.Iterator;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.TreeMap;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.ConcurrentHashMap;
 import javax.annotation.Nonnull;
 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.NotificationService;
 import org.opendaylight.controller.md.sal.binding.api.ReadTransaction;
 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
-import org.opendaylight.controller.sal.binding.api.NotificationProviderService;
+import org.opendaylight.openflowjava.protocol.api.connection.ConnectionAdapter;
+import org.opendaylight.openflowjava.protocol.api.connection.OutboundQueue;
 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.DeviceState;
 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.exception.DeviceDataException;
 import org.opendaylight.openflowplugin.api.openflow.device.handlers.DeviceContextClosedHandler;
-import org.opendaylight.openflowplugin.api.openflow.device.handlers.DeviceDisconnectedHandler;
-import org.opendaylight.openflowplugin.api.openflow.device.listener.OpenflowMessageListenerFacade;
+import org.opendaylight.openflowplugin.api.openflow.device.handlers.MultiMsgCollector;
 import org.opendaylight.openflowplugin.api.openflow.md.core.SwitchConnectionDistinguisher;
 import org.opendaylight.openflowplugin.api.openflow.md.core.TranslatorKey;
+import org.opendaylight.openflowplugin.api.openflow.registry.ItemLifeCycleRegistry;
 import org.opendaylight.openflowplugin.api.openflow.registry.flow.DeviceFlowRegistry;
 import org.opendaylight.openflowplugin.api.openflow.registry.group.DeviceGroupRegistry;
 import org.opendaylight.openflowplugin.api.openflow.registry.meter.DeviceMeterRegistry;
 import org.opendaylight.openflowplugin.api.openflow.statistics.ofpspecific.MessageSpy;
 import org.opendaylight.openflowplugin.impl.common.NodeStaticReplyTranslatorUtil;
+import org.opendaylight.openflowplugin.impl.device.listener.MultiMsgCollectorImpl;
 import org.opendaylight.openflowplugin.impl.registry.flow.DeviceFlowRegistryImpl;
 import org.opendaylight.openflowplugin.impl.registry.group.DeviceGroupRegistryImpl;
 import org.opendaylight.openflowplugin.impl.registry.meter.DeviceMeterRegistryImpl;
-import org.opendaylight.openflowplugin.impl.services.RequestContextUtil;
 import org.opendaylight.openflowplugin.openflow.md.core.session.SwitchConnectionCookieOFImpl;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNodeConnector;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeConnectorId;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeConnectorRef;
 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;
@@ -71,9 +75,6 @@ import org.opendaylight.yangtools.yang.binding.ChildOf;
 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.RpcError;
-import org.opendaylight.yangtools.yang.common.RpcResult;
-import org.opendaylight.yangtools.yang.common.RpcResultBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -83,27 +84,35 @@ import org.slf4j.LoggerFactory;
 public class DeviceContextImpl implements DeviceContext {
 
     private static final Logger LOG = LoggerFactory.getLogger(DeviceContextImpl.class);
-    public static final String DEVICE_DISCONNECTED = "Device disconnected.";
+
+    // TODO: drain factor should be parametrized
+    public static final float REJECTED_DRAIN_FACTOR = 0.25f;
+    // TODO: low water mark factor should be parametrized
+    private static final float LOW_WATERMARK_FACTOR = 0.75f;
+    // TODO: high water mark factor should be parametrized
+    private static final float HIGH_WATERMARK_FACTOR = 0.95f;
 
     private final ConnectionContext primaryConnectionContext;
     private final DeviceState deviceState;
     private final DataBroker dataBroker;
-    private final XidGenerator xidGenerator;
     private final HashedWheelTimer hashedWheelTimer;
-    private Map<Long, RequestContext> requests = Collections.synchronizedMap(new TreeMap());
-
     private final Map<SwitchConnectionDistinguisher, ConnectionContext> auxiliaryConnectionContexts;
-    private final TransactionChainManager txChainManager;
-    private TranslatorLibrary translatorLibrary;
-    private OpenflowMessageListenerFacade openflowMessageListenerFacade;
+    private final TransactionChainManager transactionChainManager;
     private final DeviceFlowRegistry deviceFlowRegistry;
     private final DeviceGroupRegistry deviceGroupRegistry;
     private final DeviceMeterRegistry deviceMeterRegistry;
+    private final Collection<DeviceContextClosedHandler> closeHandlers = new HashSet<>();
+    private final PacketInRateLimiter packetInLimiter;
+    private final MessageSpy messageSpy;
+    private NotificationPublishService notificationPublishService;
+    private NotificationService notificationService;
+    private final OutboundQueue outboundQueueProvider;
     private Timeout barrierTaskTimeout;
-    private NotificationProviderService notificationService;
-    private final MessageSpy<Class> messageSpy;
-    private DeviceDisconnectedHandler deviceDisconnectedHandler;
-    private List<DeviceContextClosedHandler> closeHandlers = new ArrayList<>();
+    private final MessageTranslator<PortGrouping, FlowCapableNodeConnector> portStatusTranslator;
+    private final MessageTranslator<PacketInMessage, PacketReceived> packetInTranslator;
+    private final TranslatorLibrary translatorLibrary;
+    private Map<Long, NodeConnectorRef> nodeConnectorCache;
+    private ItemLifeCycleRegistry itemLifeCycleSourceRegistry;
 
 
     @VisibleForTesting
@@ -111,42 +120,63 @@ public class DeviceContextImpl implements DeviceContext {
                       @Nonnull final DeviceState deviceState,
                       @Nonnull final DataBroker dataBroker,
                       @Nonnull final HashedWheelTimer hashedWheelTimer,
-                      @Nonnull final MessageSpy _messageSpy) {
+                      @Nonnull final MessageSpy _messageSpy,
+                      @Nonnull final OutboundQueueProvider outboundQueueProvider,
+                      @Nonnull final TranslatorLibrary translatorLibrary,
+                      @Nonnull final TransactionChainManager transactionChainManager) {
         this.primaryConnectionContext = Preconditions.checkNotNull(primaryConnectionContext);
         this.deviceState = Preconditions.checkNotNull(deviceState);
         this.dataBroker = Preconditions.checkNotNull(dataBroker);
         this.hashedWheelTimer = Preconditions.checkNotNull(hashedWheelTimer);
-        xidGenerator = new XidGenerator();
-        txChainManager = new TransactionChainManager(dataBroker, hashedWheelTimer, 500L, 500L);
+        this.outboundQueueProvider = Preconditions.checkNotNull(outboundQueueProvider);
+        this.transactionChainManager = Preconditions.checkNotNull(transactionChainManager);
         auxiliaryConnectionContexts = new HashMap<>();
-        requests = new HashMap<>();
         deviceFlowRegistry = new DeviceFlowRegistryImpl();
         deviceGroupRegistry = new DeviceGroupRegistryImpl();
         deviceMeterRegistry = new DeviceMeterRegistryImpl();
         messageSpy = _messageSpy;
+
+        packetInLimiter = new PacketInRateLimiter(primaryConnectionContext.getConnectionAdapter(),
+                /*initial*/ 1000, /*initial*/2000, messageSpy, REJECTED_DRAIN_FACTOR);
+
+        this.translatorLibrary = translatorLibrary;
+        portStatusTranslator = translatorLibrary.lookupTranslator(
+                new TranslatorKey(deviceState.getVersion(), PortGrouping.class.getName()));
+        packetInTranslator = translatorLibrary.lookupTranslator(
+                new TranslatorKey(deviceState.getVersion(), PacketIn.class.getName()));
+        nodeConnectorCache = new ConcurrentHashMap<>();
+
+        itemLifeCycleSourceRegistry = new ItemLifeCycleRegistryImpl();
     }
 
     /**
      * 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).
      */
-    void submitTransaction() {
-        txChainManager.submitTransaction();
-        txChainManager.enableCounter();
+    void initialSubmitTransaction() {
+        transactionChainManager.initialSubmitWriteTransaction();
     }
 
     @Override
-    public <M extends ChildOf<DataObject>> void onMessage(final M message, final RequestContext requestContext) {
-        // TODO Auto-generated method stub
+    public Long getReservedXid() {
+        return outboundQueueProvider.reserveEntry();
+    }
 
+    @Override
+    public <M extends ChildOf<DataObject>> void onMessage(final M message, final RequestContext<?> requestContext) {
+        // TODO Auto-generated method stub
     }
 
     @Override
     public void addAuxiliaryConenctionContext(final ConnectionContext connectionContext) {
-        final SwitchConnectionDistinguisher connectionDistinguisher = new SwitchConnectionCookieOFImpl(connectionContext.getFeatures().getAuxiliaryId());
+        final SwitchConnectionDistinguisher connectionDistinguisher = createConnectionDistinguisher(connectionContext);
         auxiliaryConnectionContexts.put(connectionDistinguisher, connectionContext);
     }
 
+    private static SwitchConnectionDistinguisher createConnectionDistinguisher(final ConnectionContext connectionContext) {
+        return new SwitchConnectionCookieOFImpl(connectionContext.getFeatures().getAuxiliaryId());
+    }
+
     @Override
     public void removeAuxiliaryConenctionContext(final ConnectionContext connectionContext) {
         // TODO Auto-generated method stub
@@ -165,12 +195,17 @@ public class DeviceContextImpl implements DeviceContext {
     @Override
     public <T extends DataObject> void writeToTransaction(final LogicalDatastoreType store,
                                                           final InstanceIdentifier<T> path, final T data) {
-        txChainManager.writeToTransaction(store, path, data);
+        transactionChainManager.writeToTransaction(store, path, data);
     }
 
     @Override
     public <T extends DataObject> void addDeleteToTxChain(final LogicalDatastoreType store, final InstanceIdentifier<T> path) {
-        txChainManager.addDeleteOperationTotTxChain(store, path);
+        transactionChainManager.addDeleteOperationTotTxChain(store, path);
+    }
+
+    @Override
+    public boolean submitTransaction() {
+        return transactionChainManager.submitWriteTransaction();
     }
 
     @Override
@@ -183,42 +218,6 @@ public class DeviceContextImpl implements DeviceContext {
         return auxiliaryConnectionContexts.get(new SwitchConnectionCookieOFImpl(cookie.longValue()));
     }
 
-    @Override
-    public Xid getNextXid() {
-        return xidGenerator.generate();
-    }
-
-    @Override
-    public RequestContext lookupRequest(Xid xid) {
-        return requests.get(xid.getValue());
-    }
-
-    @Override
-    public int getNumberOfOutstandingRequests() {
-        return requests.size();
-    }
-
-    @Override
-    public void hookRequestCtx(final Xid xid, final RequestContext requestFutureContext) {
-        requests.put(xid.getValue(), requestFutureContext);
-    }
-
-    @Override
-    public RequestContext unhookRequestCtx(Xid xid) {
-        return requests.remove(xid.getValue());
-    }
-
-    @Override
-    public void attachOpenflowMessageListener(final OpenflowMessageListenerFacade openflowMessageListenerFacade) {
-        this.openflowMessageListenerFacade = openflowMessageListenerFacade;
-        primaryConnectionContext.getConnectionAdapter().setMessageListener(openflowMessageListenerFacade);
-    }
-
-    @Override
-    public OpenflowMessageListenerFacade getOpenflowMessageListenerFacade() {
-        return openflowMessageListenerFacade;
-    }
-
     @Override
     public DeviceFlowRegistry getDeviceFlowRegistry() {
         return deviceFlowRegistry;
@@ -236,90 +235,17 @@ public class DeviceContextImpl implements DeviceContext {
 
     @Override
     public void processReply(final OfHeader ofHeader) {
-        final RequestContext requestContext = requests.get(ofHeader.getXid());
-        if (null != requestContext) {
-            final SettableFuture replyFuture = requestContext.getFuture();
-            requests.remove(ofHeader.getXid());
-            RpcResult<OfHeader> rpcResult;
-            if (ofHeader instanceof Error) {
-                //TODO : this is the point, where we can discover that add flow operation failed and where we should
-                //TODO : remove this flow from deviceFlowRegistry
-                final Error error = (Error) ofHeader;
-                final String message = "Operation on device failed";
-                rpcResult = RpcResultBuilder
-                        .<OfHeader>failed()
-                        .withError(RpcError.ErrorType.APPLICATION, message, new DeviceDataException(message, error))
-                        .build();
-                messageSpy.spyMessage(ofHeader.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_FAILURE);
-            } else {
-                rpcResult = RpcResultBuilder
-                        .<OfHeader>success()
-                        .withResult(ofHeader)
-                        .build();
-                messageSpy.spyMessage(ofHeader.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_SUCCESS);
-            }
-
-            replyFuture.set(rpcResult);
-            try {
-                requestContext.close();
-            } catch (final Exception e) {
-                LOG.error("Closing RequestContext failed: ", e);
-            }
+        if (ofHeader instanceof Error) {
+            messageSpy.spyMessage(ofHeader.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_FAILURE);
         } else {
-            LOG.error("Can't find request context registered for xid : {}. Type of reply: {}. From address: {}", ofHeader.getXid(), ofHeader.getClass().getName(),
-                    getPrimaryConnectionContext().getConnectionAdapter().getRemoteAddress());
+            messageSpy.spyMessage(ofHeader.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_SUCCESS);
         }
     }
 
     @Override
     public void processReply(final Xid xid, final List<MultipartReply> ofHeaderList) {
-        final RequestContext requestContext = requests.get(xid.getValue());
-        if (null != requestContext) {
-            final SettableFuture replyFuture = requestContext.getFuture();
-            requests.remove(xid.getValue());
-            final RpcResult<List<MultipartReply>> rpcResult = RpcResultBuilder
-                    .<List<MultipartReply>>success()
-                    .withResult(ofHeaderList)
-                    .build();
-            replyFuture.set(rpcResult);
-            for (MultipartReply multipartReply : ofHeaderList) {
-                messageSpy.spyMessage(multipartReply.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_FAILURE);
-            }
-
-            try {
-                requestContext.close();
-            } catch (final Exception e) {
-                LOG.error("Closing RequestContext failed: ", e);
-            }
-        } else {
-            LOG.error("Can't find request context registered for xid : {}. Type of reply: MULTIPART. From address: {}", xid.getValue(),
-                    getPrimaryConnectionContext().getConnectionAdapter().getRemoteAddress());
-        }
-    }
-
-    @Override
-    public void processException(final Xid xid, final DeviceDataException deviceDataException) {
-
-        LOG.trace("Processing exception for xid : {}", xid.getValue());
-
-        final RequestContext requestContext = requests.get(xid.getValue());
-
-        if (null != requestContext) {
-            final SettableFuture replyFuture = requestContext.getFuture();
-            requests.remove(xid.getValue());
-            final RpcResult<List<OfHeader>> rpcResult = RpcResultBuilder
-                    .<List<OfHeader>>failed()
-                    .withError(RpcError.ErrorType.APPLICATION, String.format("Message processing failed : %s", deviceDataException.getError()), deviceDataException)
-                    .build();
-            replyFuture.set(rpcResult);
-            messageSpy.spyMessage(deviceDataException.getClass(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_FAILURE);
-            try {
-                requestContext.close();
-            } catch (final Exception e) {
-                LOG.error("Closing RequestContext failed: ", e);
-            }
-        } else {
-            LOG.error("Can't find request context registered for xid : {}", xid.getValue());
+        for (final MultipartReply multipartReply : ofHeaderList) {
+            messageSpy.spyMessage(multipartReply.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_FAILURE);
         }
     }
 
@@ -329,11 +255,9 @@ public class DeviceContextImpl implements DeviceContext {
     }
 
     @Override
-    public synchronized void processPortStatusMessage(final PortStatusMessage portStatus) {
+    public void processPortStatusMessage(final PortStatusMessage portStatus) {
         messageSpy.spyMessage(portStatus.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_SUCCESS);
-        final TranslatorKey translatorKey = new TranslatorKey(portStatus.getVersion(), PortGrouping.class.getName());
-        final MessageTranslator<PortGrouping, FlowCapableNodeConnector> messageTranslator = translatorLibrary.lookupTranslator(translatorKey);
-        final FlowCapableNodeConnector flowCapableNodeConnector = messageTranslator.translate(portStatus, this, null);
+        final FlowCapableNodeConnector flowCapableNodeConnector = portStatusTranslator.translate(portStatus, this, null);
 
         final KeyedInstanceIdentifier<NodeConnector, NodeConnectorKey> iiToNodeConnector = provideIIToNodeConnector(portStatus.getPortNo(), portStatus.getVersion());
         if (portStatus.getReason().equals(PortReason.OFPPRADD) || portStatus.getReason().equals(PortReason.OFPPRMODIFY)) {
@@ -345,6 +269,7 @@ public class DeviceContextImpl implements DeviceContext {
         } else if (portStatus.getReason().equals(PortReason.OFPPRDELETE)) {
             addDeleteToTxChain(LogicalDatastoreType.OPERATIONAL, iiToNodeConnector);
         }
+        submitTransaction();
     }
 
     private KeyedInstanceIdentifier<NodeConnector, NodeConnectorKey> provideIIToNodeConnector(final long portNo, final short version) {
@@ -356,11 +281,47 @@ public class DeviceContextImpl implements DeviceContext {
 
     @Override
     public void processPacketInMessage(final PacketInMessage packetInMessage) {
-        messageSpy.spyMessage(packetInMessage.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_SUCCESS);
-        final TranslatorKey translatorKey = new TranslatorKey(packetInMessage.getVersion(), PacketIn.class.getName());
-        final MessageTranslator<PacketInMessage, PacketReceived> messageTranslator = translatorLibrary.lookupTranslator(translatorKey);
-        final PacketReceived packetReceived = messageTranslator.translate(packetInMessage, this, null);
-        notificationService.publish(packetReceived);
+        messageSpy.spyMessage(packetInMessage.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH);
+        final ConnectionAdapter connectionAdapter = getPrimaryConnectionContext().getConnectionAdapter();
+        final PacketReceived packetReceived = packetInTranslator.translate(packetInMessage, this, null);
+
+        if (packetReceived == null) {
+            LOG.debug("Received a null packet from switch {}", connectionAdapter.getRemoteAddress());
+            return;
+        }
+
+        if (!packetInLimiter.acquirePermit()) {
+            LOG.debug("Packet limited");
+            // TODO: save packet into emergency slot if possible
+            // FIXME: some other counter
+            messageSpy.spyMessage(packetReceived.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH_TRANSLATE_SRC_FAILURE);
+            return;
+        }
+
+        final ListenableFuture<? extends Object> offerNotification = notificationPublishService.offerNotification(packetReceived);
+        if (NotificationPublishService.REJECTED.equals(offerNotification)) {
+            LOG.debug("notification offer rejected");
+            messageSpy.spyMessage(packetReceived.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH_TRANSLATE_SRC_FAILURE);
+            packetInLimiter.drainLowWaterMark();
+            packetInLimiter.releasePermit();
+            return;
+        }
+
+        Futures.addCallback(offerNotification, new FutureCallback<Object>() {
+            @Override
+            public void onSuccess(final Object result) {
+                messageSpy.spyMessage(packetReceived.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_SUCCESS);
+                packetInLimiter.releasePermit();
+            }
+
+            @Override
+            public void onFailure(final Throwable t) {
+                messageSpy.spyMessage(packetReceived.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH_NOTIFICATION_REJECTED);
+                LOG.debug("notification offer failed: {}", t.getMessage());
+                LOG.trace("notification offer failed..", t);
+                packetInLimiter.releasePermit();
+            }
+        });
     }
 
     @Override
@@ -369,110 +330,121 @@ public class DeviceContextImpl implements DeviceContext {
     }
 
     @Override
-    public void setTranslatorLibrary(final TranslatorLibrary translatorLibrary) {
-        this.translatorLibrary = translatorLibrary;
+    public HashedWheelTimer getTimer() {
+        return hashedWheelTimer;
     }
 
     @Override
-    public synchronized HashedWheelTimer getTimer() {
-        return hashedWheelTimer;
+    public void close() {
+        LOG.debug("closing deviceContext: {}, nodeId:{}",
+                getPrimaryConnectionContext().getConnectionAdapter().getRemoteAddress(),
+                getDeviceState().getNodeId());
+
+        tearDown();
+
+        primaryConnectionContext.closeConnection(false);
     }
 
-    @Override
-    public void close() throws Exception {
+    private void tearDown() {
         deviceState.setValid(false);
 
-        LOG.trace("Removing node {} from operational DS.", getDeviceState().getNodeId());
-        addDeleteToTxChain(LogicalDatastoreType.OPERATIONAL, getDeviceState().getNodeInstanceIdentifier());
+        for (final ConnectionContext connectionContext : auxiliaryConnectionContexts.values()) {
+            connectionContext.closeConnection(false);
+        }
 
         deviceGroupRegistry.close();
         deviceFlowRegistry.close();
         deviceMeterRegistry.close();
 
-        if (primaryConnectionContext.getConnectionAdapter().isAlive()) {
-            primaryConnectionContext.setConnectionState(ConnectionContext.CONNECTION_STATE.RIP);
-            primaryConnectionContext.getConnectionAdapter().disconnect();
-        }
-        for (Map.Entry<Long, RequestContext> entry : requests.entrySet()) {
-            RequestContextUtil.closeRequestContextWithRpcError(entry.getValue(), DEVICE_DISCONNECTED);
-        }
-        for (ConnectionContext connectionContext : auxiliaryConnectionContexts.values()) {
-            if (connectionContext.getConnectionAdapter().isAlive()) {
-                connectionContext.getConnectionAdapter().disconnect();
-            }
-        }
-        for (DeviceContextClosedHandler deviceContextClosedHandler : closeHandlers) {
+        itemLifeCycleSourceRegistry.clear();
+
+
+        for (final DeviceContextClosedHandler deviceContextClosedHandler : closeHandlers) {
             deviceContextClosedHandler.onDeviceContextClosed(this);
         }
 
+        transactionChainManager.close();
     }
 
     @Override
-    public synchronized void onDeviceDisconnected(final ConnectionContext connectionContext) {
-        if (this.getPrimaryConnectionContext().equals(connectionContext)) {
+    public void onDeviceDisconnected(final ConnectionContext connectionContext) {
+        if (getPrimaryConnectionContext().equals(connectionContext)) {
             try {
-                close();
-            } catch (Exception e) {
+                tearDown();
+            } catch (final Exception e) {
                 LOG.trace("Error closing device context.");
             }
-            if (null != deviceDisconnectedHandler) {
-                deviceDisconnectedHandler.onDeviceDisconnected(connectionContext);
-            }
         } else {
-            auxiliaryConnectionContexts.remove(connectionContext);
+            LOG.debug("auxiliary connection dropped: {}, nodeId:{}",
+                    connectionContext.getConnectionAdapter().getRemoteAddress(),
+                    getDeviceState().getNodeId());
+            final SwitchConnectionDistinguisher connectionDistinguisher = createConnectionDistinguisher(connectionContext);
+            auxiliaryConnectionContexts.remove(connectionDistinguisher);
         }
     }
 
+    @Override
+    public void setCurrentBarrierTimeout(final Timeout timeout) {
+        barrierTaskTimeout = timeout;
+    }
+
+    @Override
+    public Timeout getBarrierTaskTimeout() {
+        return barrierTaskTimeout;
+    }
 
-    private class XidGenerator {
+    @Override
+    public void setNotificationService(final NotificationService notificationServiceParam) {
+        notificationService = notificationServiceParam;
+    }
 
-        private final AtomicLong xid = new AtomicLong(0);
+    @Override
+    public void setNotificationPublishService(final NotificationPublishService notificationPublishService) {
+        this.notificationPublishService = notificationPublishService;
+    }
 
-        public Xid generate() {
-            return new Xid(xid.incrementAndGet());
-        }
+    @Override
+    public MessageSpy getMessageSpy() {
+        return messageSpy;
     }
 
     @Override
-    public RequestContext extractNextOutstandingMessage(final long barrierXid) {
-        RequestContext nextMessage = null;
-        final Iterator<Long> keyIterator = requests.keySet().iterator();
-        if (keyIterator.hasNext()) {
-            final Long oldestXid = keyIterator.next();
-            if (oldestXid < barrierXid) {
-                nextMessage = requests.remove(oldestXid);
-            }
-        }
-        return nextMessage;
+    public void addDeviceContextClosedHandler(final DeviceContextClosedHandler deviceContextClosedHandler) {
+        closeHandlers.add(deviceContextClosedHandler);
     }
 
     @Override
-    public void setCurrentBarrierTimeout(final Timeout timeout) {
-        barrierTaskTimeout = timeout;
+    public void onPublished() {
+        primaryConnectionContext.getConnectionAdapter().setPacketInFiltering(false);
+        for (final ConnectionContext switchAuxConnectionContext : auxiliaryConnectionContexts.values()) {
+            switchAuxConnectionContext.getConnectionAdapter().setPacketInFiltering(false);
+        }
     }
 
     @Override
-    public Timeout getBarrierTaskTimeout() {
-        return barrierTaskTimeout;
+    public MultiMsgCollector getMultiMsgCollector(final RequestContext<List<MultipartReply>> requestContext) {
+        return new MultiMsgCollectorImpl(this, requestContext);
     }
 
     @Override
-    public void setNotificationService(final NotificationProviderService notificationServiceParam) {
-        notificationService = notificationServiceParam;
+    public NodeConnectorRef lookupNodeConnectorRef(Long portNumber) {
+        return nodeConnectorCache.get(portNumber);
     }
 
     @Override
-    public MessageSpy getMessageSpy() {
-        return messageSpy;
+    public void storeNodeConnectorRef(final Long portNumber, final NodeConnectorRef nodeConnectorRef) {
+        nodeConnectorCache.put(
+                Preconditions.checkNotNull(portNumber),
+                Preconditions.checkNotNull(nodeConnectorRef));
     }
 
     @Override
-    public void setDeviceDisconnectedHandler(final DeviceDisconnectedHandler deviceDisconnectedHandler) {
-        this.deviceDisconnectedHandler = deviceDisconnectedHandler;
+    public void updatePacketInRateLimit(long upperBound) {
+        packetInLimiter.changeWaterMarks((int) (LOW_WATERMARK_FACTOR * upperBound), (int) (HIGH_WATERMARK_FACTOR * upperBound));
     }
 
     @Override
-    public void addDeviceContextClosedHandler(final DeviceContextClosedHandler deviceContextClosedHandler) {
-        this.closeHandlers.add(deviceContextClosedHandler);
+    public ItemLifeCycleRegistry getItemLifeCycleSourceRegistry() {
+        return itemLifeCycleSourceRegistry;
     }
 }