Bug 5692 - Direct statistics RPC
[openflowplugin.git] / openflowplugin-impl / src / main / java / org / opendaylight / openflowplugin / impl / device / DeviceContextImpl.java
index 0fb1b1d929bec9ba5a4aba7e27d03e9f322ab631..1d82ea0f2f89d7e7784f5e12fc33088842ca7718 100644 (file)
@@ -13,23 +13,19 @@ 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.FutureFallback;
 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.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
 import javax.annotation.CheckForNull;
 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.ReadOnlyTransaction;
 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
 import org.opendaylight.openflowjava.protocol.api.connection.ConnectionAdapter;
@@ -43,8 +39,8 @@ 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.DeviceTerminationPhaseHandler;
 import org.opendaylight.openflowplugin.api.openflow.device.handlers.MultiMsgCollector;
+import org.opendaylight.openflowplugin.api.openflow.lifecycle.LifecycleConductor;
 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;
@@ -63,7 +59,6 @@ import org.opendaylight.openflowplugin.extension.api.ExtensionConverterProviderK
 import org.opendaylight.openflowplugin.extension.api.core.extension.ExtensionConverterProvider;
 import org.opendaylight.openflowplugin.extension.api.exception.ConversionException;
 import org.opendaylight.openflowplugin.extension.api.path.MessagePath;
-import org.opendaylight.openflowplugin.impl.LifecycleConductor;
 import org.opendaylight.openflowplugin.impl.common.ItemLifeCycleSourceImpl;
 import org.opendaylight.openflowplugin.impl.common.NodeStaticReplyTranslatorUtil;
 import org.opendaylight.openflowplugin.impl.device.listener.MultiMsgCollectorImpl;
@@ -71,6 +66,7 @@ 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.rpc.RpcContextImpl;
 import org.opendaylight.openflowplugin.impl.util.DeviceInitializationUtils;
 import org.opendaylight.openflowplugin.impl.util.MdSalRegistrationUtils;
 import org.opendaylight.openflowplugin.openflow.md.core.session.SwitchConnectionCookieOFImpl;
@@ -127,18 +123,15 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     private final ConnectionContext primaryConnectionContext;
     private final DeviceState deviceState;
     private final DataBroker dataBroker;
-    private final HashedWheelTimer hashedWheelTimer;
     private final Map<SwitchConnectionDistinguisher, ConnectionContext> auxiliaryConnectionContexts;
     private final TransactionChainManager transactionChainManager;
     private final DeviceFlowRegistry deviceFlowRegistry;
     private final DeviceGroupRegistry deviceGroupRegistry;
     private final DeviceMeterRegistry deviceMeterRegistry;
-    private final Collection<DeviceTerminationPhaseHandler> closeHandlers = new HashSet<>();
     private final PacketInRateLimiter packetInLimiter;
     private final MessageSpy messageSpy;
     private final ItemLifeCycleKeeper flowLifeCycleKeeper;
     private NotificationPublishService notificationPublishService;
-    private NotificationService notificationService;
     private final OutboundQueue outboundQueueProvider;
     private Timeout barrierTaskTimeout;
     private final MessageTranslator<PortGrouping, FlowCapableNodeConnector> portStatusTranslator;
@@ -156,14 +149,14 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     private final NodeId nodeId;
 
     private volatile DEVICE_CONTEXT_STATE deviceCtxState;
+    private boolean isStatisticsRpcEnabled;
 
 
     @VisibleForTesting
     DeviceContextImpl(@Nonnull final ConnectionContext primaryConnectionContext,
                       @Nonnull final DeviceState deviceState,
                       @Nonnull final DataBroker dataBroker,
-                      @Nonnull final HashedWheelTimer hashedWheelTimer,
-                      @Nonnull final MessageSpy _messageSpy,
+                      @Nonnull final LifecycleConductor conductor,
                       @Nonnull final OutboundQueueProvider outboundQueueProvider,
                       @Nonnull final TranslatorLibrary translatorLibrary,
                       final boolean switchFeaturesMandatory) {
@@ -171,14 +164,14 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
         this.primaryConnectionContext = Preconditions.checkNotNull(primaryConnectionContext);
         this.deviceState = Preconditions.checkNotNull(deviceState);
         this.dataBroker = Preconditions.checkNotNull(dataBroker);
-        this.hashedWheelTimer = Preconditions.checkNotNull(hashedWheelTimer);
+        Preconditions.checkNotNull(conductor);
         this.outboundQueueProvider = Preconditions.checkNotNull(outboundQueueProvider);
-        this.transactionChainManager = new TransactionChainManager(dataBroker, deviceState);
+        this.transactionChainManager = new TransactionChainManager(dataBroker, deviceState, conductor);
         auxiliaryConnectionContexts = new HashMap<>();
         deviceFlowRegistry = new DeviceFlowRegistryImpl();
         deviceGroupRegistry = new DeviceGroupRegistryImpl();
         deviceMeterRegistry = new DeviceMeterRegistryImpl();
-        messageSpy = _messageSpy;
+        messageSpy = conductor.getMessageIntelligenceAgency();
 
         packetInLimiter = new PacketInRateLimiter(primaryConnectionContext.getConnectionAdapter(),
                 /*initial*/ 1000, /*initial*/2000, messageSpy, REJECTED_DRAIN_FACTOR);
@@ -211,7 +204,7 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     }
 
     @Override
-    public Long reservedXidForDeviceMessage() {
+    public Long reserveXidForDeviceMessage() {
         return outboundQueueProvider.reserveEntry();
     }
 
@@ -228,11 +221,9 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     @Override
     public void removeAuxiliaryConnectionContext(final ConnectionContext connectionContext) {
         final SwitchConnectionDistinguisher connectionDistinguisher = createConnectionDistinguisher(connectionContext);
-        if (null != connectionDistinguisher) {
-            LOG.debug("auxiliary connection dropped: {}, nodeId:{}", connectionContext.getConnectionAdapter()
-                    .getRemoteAddress(), nodeId);
-            auxiliaryConnectionContexts.remove(connectionDistinguisher);
-        }
+        LOG.debug("auxiliary connection dropped: {}, nodeId:{}", connectionContext.getConnectionAdapter()
+                .getRemoteAddress(), nodeId);
+        auxiliaryConnectionContexts.remove(connectionDistinguisher);
     }
 
     @Override
@@ -291,7 +282,11 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
         }
         /* Routed RPC registration */
         MdSalRegistrationUtils.registerMasterServices(getRpcContext(), DeviceContextImpl.this, OfpRole.BECOMEMASTER);
-        getRpcContext().registerStatCompatibilityServices();
+
+        if (isStatisticsRpcEnabled) {
+            MdSalRegistrationUtils.registerStatCompatibilityServices(getRpcContext(), this,
+                    notificationPublishService, new AtomicLong());
+        }
 
         /* Prepare init info collecting */
         getDeviceState().setDeviceSynchronized(false);
@@ -304,7 +299,7 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
                 new AsyncFunction<Void, Boolean>() {
 
                     @Override
-                    public ListenableFuture<Boolean> apply(final Void input) throws Exception {
+                    public ListenableFuture<Boolean> apply(@Nonnull final Void input) throws Exception {
                         getStatisticsContext().statListForCollectingInitialization();
                         return getStatisticsContext().gatherDynamicData();
                     }
@@ -320,7 +315,7 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
                     LOG.warn(errMsg);
                     throw new IllegalStateException(errMsg);
                 }
-                if (!input.booleanValue()) {
+                if (!input) {
                     final String errMsg = String.format("Get Initial Device %s information fails",
                             getDeviceState().getNodeId());
                     LOG.warn(errMsg);
@@ -338,7 +333,12 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     @Override
     public <T extends DataObject> void writeToTransaction(final LogicalDatastoreType store,
                                                           final InstanceIdentifier<T> path, final T data) throws Exception {
-        transactionChainManager.writeToTransaction(store, path, data);
+        transactionChainManager.writeToTransaction(store, path, data, false);
+    }
+
+    @Override
+    public <T extends DataObject> void writeToTransactionWithParentsSlow(LogicalDatastoreType store, InstanceIdentifier<T> path, T data) throws Exception {
+        transactionChainManager.writeToTransaction(store, path, data, true);
     }
 
     @Override
@@ -398,7 +398,7 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
         if (itemLifecycleListener != null) {
             //1. translate to general flow (table, priority, match, cookie)
             final org.opendaylight.yang.gen.v1.urn.opendaylight.flow.service.rev130819.FlowRemoved flowRemovedNotification =
-                    flowRemovedTranslator.translate(flowRemoved, this, null);
+                    flowRemovedTranslator.translate(flowRemoved, this.getDeviceState(), null);
             //2. create registry key
             final FlowRegistryKey flowRegKey = FlowRegistryKeyFactory.create(flowRemovedNotification);
             //3. lookup flowId
@@ -422,7 +422,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, this, null);
+        final FlowCapableNodeConnector flowCapableNodeConnector = portStatusTranslator.translate(portStatus, this.getDeviceState(), null);
 
         final KeyedInstanceIdentifier<NodeConnector, NodeConnectorKey> iiToNodeConnector = provideIIToNodeConnector(portStatus.getPortNo(), portStatus.getVersion());
         try {
@@ -436,7 +436,7 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
                 addDeleteToTxChain(LogicalDatastoreType.OPERATIONAL, iiToNodeConnector);
             }
             submitTransaction();
-        } catch (Exception e) {
+        } catch (final Exception e) {
             LOG.warn("Error processing port status message: {}", e.getMessage());
         }
     }
@@ -452,7 +452,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, this, null);
+        final PacketReceived packetReceived = packetInTranslator.translate(packetInMessage, this.getDeviceState(), null);
 
         if (packetReceived == null) {
             LOG.debug("Received a null packet from switch {}", connectionAdapter.getRemoteAddress());
@@ -519,7 +519,7 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
             // publish
             notificationPublishService.offerNotification(experimenterMessageFromDevBld.build());
         } catch (final ConversionException e) {
-            LOG.warn("Conversion of experimenter notification failed", e);
+            LOG.error("Conversion of experimenter notification failed", e);
         }
     }
 
@@ -528,11 +528,6 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
         return translatorLibrary;
     }
 
-    @Override
-    public HashedWheelTimer getTimer() {
-        return hashedWheelTimer;
-    }
-
     @Override
     public synchronized void close() {
         LOG.debug("closing deviceContext: {}, nodeId:{}",
@@ -552,11 +547,6 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
         return barrierTaskTimeout;
     }
 
-    @Override
-    public void setNotificationService(final NotificationService notificationService) {
-        this.notificationService = notificationService;
-    }
-
     @Override
     public void setNotificationPublishService(final NotificationPublishService notificationPublishService) {
         this.notificationPublishService = notificationPublishService;
@@ -588,7 +578,7 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
     }
 
     @Override
-    public void storeNodeConnectorRef(final Long portNumber, final NodeConnectorRef nodeConnectorRef) {
+    public void storeNodeConnectorRef(@Nonnull final Long portNumber, @Nonnull final NodeConnectorRef nodeConnectorRef) {
         nodeConnectorCache.put(
                 Preconditions.checkNotNull(portNumber),
                 Preconditions.checkNotNull(nodeConnectorRef));
@@ -661,6 +651,11 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
         deviceMeterRegistry.close();
     }
 
+    @Override
+    public void setStatisticsRpcEnabled(boolean isStatisticsRpcEnabled) {
+        this.isStatisticsRpcEnabled = isStatisticsRpcEnabled;
+    }
+
     @Override
     public DEVICE_CONTEXT_STATE getDeviceContextState() {
         return deviceCtxState;
@@ -671,4 +666,9 @@ public class DeviceContextImpl implements DeviceContext, ExtensionConverterProvi
         deviceState.setValid(false);
         return transactionChainManager.shuttingDown();
     }
+
+    @VisibleForTesting
+    TransactionChainManager getTransactionChainManager() {
+        return this.transactionChainManager;
+    }
 }