Merge "Bug 6380 lldp-speaker - DTCL instead of DTL"
[openflowplugin.git] / openflowplugin-impl / src / main / java / org / opendaylight / openflowplugin / impl / device / DeviceContextImpl.java
index baf8c9e106f5e815725651a53c27c8278b6d99c0..96acf077a969620cd8ccaeef7d571fdc33f3a83c 100644 (file)
@@ -18,17 +18,17 @@ import java.math.BigInteger;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ExecutionException;
 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.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.OutboundQueue;
 import org.opendaylight.openflowjava.protocol.api.keys.MessageTypeKey;
 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;
@@ -38,7 +38,7 @@ 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.MultiMsgCollector;
-import org.opendaylight.openflowplugin.api.openflow.lifecycle.LifecycleConductor;
+import org.opendaylight.openflowplugin.api.openflow.lifecycle.LifecycleService;
 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;
@@ -62,6 +62,8 @@ 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.openflow.md.core.sal.convertor.ConvertorExecutor;
 import org.opendaylight.openflowplugin.openflow.md.core.session.SwitchConnectionCookieOFImpl;
 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;
@@ -99,7 +101,7 @@ 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);
 
@@ -110,7 +112,7 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     // TODO: high water mark factor should be parametrized
     private static final float HIGH_WATERMARK_FACTOR = 0.95f;
 
-    private final ConnectionContext primaryConnectionContext;
+    private ConnectionContext primaryConnectionContext;
     private final DeviceState deviceState;
     private final DataBroker dataBroker;
     private final Map<SwitchConnectionDistinguisher, ConnectionContext> auxiliaryConnectionContexts;
@@ -122,7 +124,6 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     private final MessageSpy messageSpy;
     private final ItemLifeCycleKeeper flowLifeCycleKeeper;
     private NotificationPublishService notificationPublishService;
-    private final OutboundQueue outboundQueueProvider;
     private Timeout barrierTaskTimeout;
     private final MessageTranslator<PortGrouping, FlowCapableNodeConnector> portStatusTranslator;
     private final MessageTranslator<PacketInMessage, PacketReceived> packetInTranslator;
@@ -132,35 +133,32 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     private ExtensionConverterProvider extensionConverterProvider;
     private final DeviceManager deviceManager;
 
+    private boolean switchFeaturesMandatory;
     private final DeviceInfo deviceInfo;
-
-    private volatile CONTEXT_STATE contextState;
-
-
-    @VisibleForTesting
-    DeviceContextImpl(@Nonnull final ConnectionContext primaryConnectionContext,
-                      @Nonnull final DeviceState deviceState,
-                      @Nonnull final DataBroker dataBroker,
-                      @Nonnull final LifecycleConductor conductor,
-                      @Nonnull final OutboundQueueProvider outboundQueueProvider,
-                      @Nonnull final TranslatorLibrary translatorLibrary,
-                      final DeviceManager manager) {
+    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) {
         this.primaryConnectionContext = Preconditions.checkNotNull(primaryConnectionContext);
-        this.deviceState = Preconditions.checkNotNull(deviceState);
+        this.deviceInfo = primaryConnectionContext.getDeviceInfo();
+        this.deviceState = new DeviceStateImpl();
         this.dataBroker = Preconditions.checkNotNull(dataBroker);
-        Preconditions.checkNotNull(conductor);
-        this.outboundQueueProvider = Preconditions.checkNotNull(outboundQueueProvider);
-        deviceInfo = primaryConnectionContext.getDeviceInfo();
-        this.transactionChainManager = new TransactionChainManager(dataBroker, deviceInfo, conductor);
+        this.transactionChainManager = new TransactionChainManager(dataBroker, deviceInfo);
         auxiliaryConnectionContexts = new HashMap<>();
-        deviceFlowRegistry = new DeviceFlowRegistryImpl(dataBroker);
+        deviceFlowRegistry = new DeviceFlowRegistryImpl(dataBroker, deviceInfo.getNodeInstanceIdentifier());
         deviceGroupRegistry = new DeviceGroupRegistryImpl();
         deviceMeterRegistry = new DeviceMeterRegistryImpl();
-        messageSpy = conductor.getMessageIntelligenceAgency();
+        this.messageSpy = Preconditions.checkNotNull(messageSpy);
         this.deviceManager = Preconditions.checkNotNull(manager);
 
         packetInLimiter = new PacketInRateLimiter(primaryConnectionContext.getConnectionAdapter(),
-                /*initial*/ 1000, /*initial*/2000, messageSpy, REJECTED_DRAIN_FACTOR);
+                /*initial*/ 1000, /*initial*/2000, this.messageSpy, REJECTED_DRAIN_FACTOR);
 
         this.translatorLibrary = translatorLibrary;
         portStatusTranslator = translatorLibrary.lookupTranslator(
@@ -173,20 +171,17 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
         itemLifeCycleSourceRegistry = new ItemLifeCycleRegistryImpl();
         flowLifeCycleKeeper = new ItemLifeCycleSourceImpl();
         itemLifeCycleSourceRegistry.registerLifeCycleSource(flowLifeCycleKeeper);
-        contextState = CONTEXT_STATE.INITIALIZATION;
+        this.state = CONTEXT_STATE.INITIALIZATION;
+        this.convertorExecutor = convertorExecutor;
     }
 
     /**
      * 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 initialSubmitTransaction() {
-        transactionChainManager.initialSubmitWriteTransaction();
-    }
-
     @Override
-    public Long reserveXidForDeviceMessage() {
-        return outboundQueueProvider.reserveEntry();
+    public void initialSubmitTransaction() {
+        transactionChainManager.initialSubmitWriteTransaction();
     }
 
     @Override
@@ -203,7 +198,7 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     public void removeAuxiliaryConnectionContext(final ConnectionContext connectionContext) {
         final SwitchConnectionDistinguisher connectionDistinguisher = createConnectionDistinguisher(connectionContext);
         LOG.debug("auxiliary connection dropped: {}, nodeId:{}", connectionContext.getConnectionAdapter()
-                .getRemoteAddress(), deviceInfo.getNodeId());
+                .getRemoteAddress(), getDeviceInfo().getLOGValue());
         auxiliaryConnectionContexts.remove(connectionDistinguisher);
     }
 
@@ -212,11 +207,6 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
         return deviceState;
     }
 
-    @Override
-    public DeviceInfo getDeviceInfo() {
-        return this.deviceInfo;
-    }
-
     @Override
     public ReadOnlyTransaction getReadTransaction() {
         return dataBroker.newReadOnlyTransaction();
@@ -224,12 +214,15 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
 
     @Override
     public <T extends DataObject> void writeToTransaction(final LogicalDatastoreType store,
-                                                          final InstanceIdentifier<T> path, final T data) throws TransactionChainClosedException {
+                                                          final InstanceIdentifier<T> path,
+                                                          final T data){
         transactionChainManager.writeToTransaction(store, path, data, false);
     }
 
     @Override
-    public <T extends DataObject> void writeToTransactionWithParentsSlow(LogicalDatastoreType store, InstanceIdentifier<T> path, T data) throws TransactionChainClosedException {
+    public <T extends DataObject> void writeToTransactionWithParentsSlow(final LogicalDatastoreType store,
+                                                                         final InstanceIdentifier<T> path,
+                                                                         final T data){
         transactionChainManager.writeToTransaction(store, path, data, true);
     }
 
@@ -294,7 +287,7 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
             // Trigger off a notification
             notificationPublishService.offerNotification(flowRemovedNotification);
         } else if(LOG.isDebugEnabled()) {
-            LOG.debug("For nodeId={} isNotificationFlowRemovedOff={}", getDeviceInfo().getNodeId(), deviceManager.getIsNotificationFlowRemovedOff());
+            LOG.debug("For nodeId={} isNotificationFlowRemovedOff={}", getDeviceInfo().getLOGValue(), deviceManager.getIsNotificationFlowRemovedOff());
         }
 
         final ItemLifecycleListener itemLifecycleListener = flowLifeCycleKeeper.getItemLifecycleListener();
@@ -322,7 +315,7 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     @Override
     public void processPortStatusMessage(final PortStatusMessage portStatus) {
         messageSpy.spyMessage(portStatus.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH_PUBLISHED_SUCCESS);
-        final FlowCapableNodeConnector flowCapableNodeConnector = portStatusTranslator.translate(portStatus, deviceInfo, null);
+        final FlowCapableNodeConnector flowCapableNodeConnector = portStatusTranslator.translate(portStatus, getDeviceInfo(), null);
 
         final KeyedInstanceIdentifier<NodeConnector, NodeConnectorKey> iiToNodeConnector = provideIIToNodeConnector(portStatus.getPortNo(), portStatus.getVersion());
         try {
@@ -337,13 +330,13 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
             }
             submitTransaction();
         } catch (final Exception e) {
-            LOG.warn("Error processing port status message: {}", e.getMessage());
+            LOG.warn("Error processing port status message: ", e);
         }
     }
 
     private KeyedInstanceIdentifier<NodeConnector, NodeConnectorKey> provideIIToNodeConnector(final long portNo, final short version) {
-        final InstanceIdentifier<Node> iiToNodes = deviceInfo.getNodeInstanceIdentifier();
-        final BigInteger dataPathId = deviceInfo.getDatapathId();
+        final InstanceIdentifier<Node> iiToNodes = getDeviceInfo().getNodeInstanceIdentifier();
+        final BigInteger dataPathId = getDeviceInfo().getDatapathId();
         final NodeConnectorId nodeConnectorId = NodeStaticReplyTranslatorUtil.nodeConnectorId(dataPathId.toString(), portNo, version);
         return iiToNodes.child(NodeConnector.class, new NodeConnectorKey(nodeConnectorId));
     }
@@ -352,7 +345,7 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     public void processPacketInMessage(final PacketInMessage packetInMessage) {
         messageSpy.spyMessage(packetInMessage.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.FROM_SWITCH);
         final ConnectionAdapter connectionAdapter = getPrimaryConnectionContext().getConnectionAdapter();
-        final PacketReceived packetReceived = packetInTranslator.translate(packetInMessage, deviceInfo, null);
+        final PacketReceived packetReceived = packetInTranslator.translate(packetInMessage, getDeviceInfo(), null);
 
         if (packetReceived == null) {
             LOG.debug("Received a null packet from switch {}", connectionAdapter.getRemoteAddress());
@@ -400,13 +393,13 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
         // lookup converter
         final ExperimenterDataOfChoice vendorData = notification.getExperimenterDataOfChoice();
         final MessageTypeKey<? extends ExperimenterDataOfChoice> key = new MessageTypeKey<>(
-                deviceInfo.getVersion(),
+                getDeviceInfo().getVersion(),
                 (Class<? extends ExperimenterDataOfChoice>) vendorData.getImplementedInterface());
         final ConvertorMessageFromOFJava<ExperimenterDataOfChoice, MessagePath> messageConverter = extensionConverterProvider.getMessageConverter(key);
         if (messageConverter == null) {
             LOG.warn("custom converter for {}[OF:{}] not found",
                     notification.getExperimenterDataOfChoice().getImplementedInterface(),
-                    deviceInfo.getVersion());
+                    getDeviceInfo().getVersion());
             return;
         }
         // build notification
@@ -414,7 +407,7 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
         try {
             messageOfChoice = messageConverter.convert(vendorData, MessagePath.MESSAGE_NOTIFICATION);
             final ExperimenterMessageFromDevBuilder experimenterMessageFromDevBld = new ExperimenterMessageFromDevBuilder()
-                .setNode(new NodeRef(deviceInfo.getNodeInstanceIdentifier()))
+                .setNode(new NodeRef(getDeviceInfo().getNodeInstanceIdentifier()))
                     .setExperimenterMessageOfChoice(messageOfChoice);
             // publish
             notificationPublishService.offerNotification(experimenterMessageFromDevBld.build());
@@ -432,7 +425,7 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     public synchronized void close() {
         LOG.debug("closing deviceContext: {}, nodeId:{}",
                 getPrimaryConnectionContext().getConnectionAdapter().getRemoteAddress(),
-                getDeviceInfo().getNodeId());
+                getDeviceInfo().getLOGValue());
         // NOOP
         throw new UnsupportedOperationException("Autocloseble.close will be removed soon");
     }
@@ -459,8 +452,8 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
 
     @Override
     public void onPublished() {
-        Verify.verify(CONTEXT_STATE.INITIALIZATION.equals(contextState));
-        contextState = CONTEXT_STATE.WORKING;
+        Verify.verify(CONTEXT_STATE.INITIALIZATION.equals(getState()));
+        setState(CONTEXT_STATE.WORKING);
         primaryConnectionContext.getConnectionAdapter().setPacketInFiltering(false);
         for (final ConnectionContext switchAuxConnectionContext : auxiliaryConnectionContexts.values()) {
             switchAuxConnectionContext.getConnectionAdapter().setPacketInFiltering(false);
@@ -494,15 +487,15 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
 
     @Override
     public synchronized void shutdownConnection() {
-        LOG.debug("Shutdown method for node {}", deviceInfo.getNodeId());
-        if (CONTEXT_STATE.TERMINATION.equals(contextState)) {
-            LOG.debug("DeviceCtx for Node {} is in termination process.", deviceInfo.getNodeId());
+        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;
         }
-        contextState = CONTEXT_STATE.TERMINATION;
+        setState(CONTEXT_STATE.TERMINATION);
 
         if (ConnectionContext.CONNECTION_STATE.RIP.equals(getPrimaryConnectionContext().getConnectionState())) {
-            LOG.debug("ConnectionCtx for Node {} is in RIP state.", deviceInfo.getNodeId());
+            LOG.debug("ConnectionCtx for Node {} is in RIP state.", getDeviceInfo().getLOGValue());
             return;
         }
         /* Terminate Auxiliary Connection */
@@ -528,8 +521,54 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
         return this.transactionChainManager;
     }
 
+    @Override
+    public void setSwitchFeaturesMandatory(boolean switchFeaturesMandatory) {
+        this.switchFeaturesMandatory = switchFeaturesMandatory;
+    }
+
     @Override
     public CONTEXT_STATE getState() {
-        return this.contextState;
+        return this.state;
+    }
+
+    @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);
+    }
+
+    @Override
+    public ListenableFuture<Void> stopClusterServices(boolean deviceDisconnected) {
+        return this.transactionChainManager.deactivateTransactionManager();
+    }
+
+    @Override
+    public ServiceGroupIdentifier getServiceIdentifier() {
+        return this.deviceInfo.getServiceIdentifier();
+    }
+
+    @Override
+    public DeviceInfo getDeviceInfo() {
+        return this.deviceInfo;
+    }
+
+    @Override
+    public void putLifecycleServiceIntoTxChainManager(final LifecycleService lifecycleService){
+        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();
     }
 }