Cleanup RequestContextStack
[openflowplugin.git] / openflowplugin-impl / src / main / java / org / opendaylight / openflowplugin / impl / device / DeviceManagerImpl.java
index 1804492373ad153c5c3ee828df16a34923f63e36..c02764b2f761b30f049b8dde93e5d1ad4589d4ea 100644 (file)
@@ -10,9 +10,7 @@ package org.opendaylight.openflowplugin.impl.device;
 import com.google.common.base.Preconditions;
 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 com.google.common.util.concurrent.SettableFuture;
 import io.netty.util.HashedWheelTimer;
 import java.math.BigInteger;
 import java.util.ArrayList;
@@ -30,9 +28,12 @@ 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.WriteTransaction;
 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
+import org.opendaylight.openflowjava.protocol.api.connection.OutboundQueue;
 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.connection.ThrottledNotificationsOfferer;
 import org.opendaylight.openflowplugin.api.openflow.device.DeviceContext;
 import org.opendaylight.openflowplugin.api.openflow.device.DeviceManager;
 import org.opendaylight.openflowplugin.api.openflow.device.DeviceState;
@@ -47,10 +48,10 @@ import org.opendaylight.openflowplugin.api.openflow.md.core.TranslatorKey;
 import org.opendaylight.openflowplugin.api.openflow.statistics.ofpspecific.MessageIntelligenceAgency;
 import org.opendaylight.openflowplugin.impl.common.MultipartRequestInputFactory;
 import org.opendaylight.openflowplugin.impl.common.NodeStaticReplyTranslatorUtil;
+import org.opendaylight.openflowplugin.impl.connection.OutboundQueueProviderImpl;
+import org.opendaylight.openflowplugin.impl.connection.ThrottledNotificationsOffererImpl;
 import org.opendaylight.openflowplugin.impl.device.listener.OpenflowProtocolListenerFullImpl;
-import org.opendaylight.openflowplugin.impl.rpc.RequestContextImpl;
-import org.opendaylight.openflowplugin.impl.services.OFJResult2RequestCtxFuture;
-import org.opendaylight.openflowplugin.impl.statistics.ofpspecific.MessageIntelligenceAgencyImpl;
+import org.opendaylight.openflowplugin.impl.rpc.AbstractRequestContext;
 import org.opendaylight.openflowplugin.impl.util.DeviceStateUtil;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNodeBuilder;
@@ -73,6 +74,8 @@ import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.common.types.rev13
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.common.types.rev130731.CapabilitiesV10;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.common.types.rev130731.MultipartType;
 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.PacketInMessage;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.PortGrouping;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.multipart.reply.MultipartReplyBody;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.multipart.reply.multipart.reply.body.MultipartReplyDescCase;
@@ -101,7 +104,7 @@ public class DeviceManagerImpl implements DeviceManager, AutoCloseable {
 
     private static final Logger LOG = LoggerFactory.getLogger(DeviceManagerImpl.class);
 
-    private static final long TICK_DURATION = 500; // 0.5 sec.
+    private static final long TICK_DURATION = 10; // 0.5 sec.
     private ScheduledThreadPoolExecutor spyPool;
     private final int spyRate = 10;
 
@@ -112,36 +115,36 @@ public class DeviceManagerImpl implements DeviceManager, AutoCloseable {
     private DeviceInitializationPhaseHandler deviceInitPhaseHandler;
     private NotificationService notificationService;
     private NotificationPublishService notificationPublishService;
+    private ThrottledNotificationsOfferer<PacketInMessage> throttledNotificationsOfferer;
 
-    private final List<DeviceContext> synchronizedDeviceContextsList = new ArrayList<DeviceContext>();
-    private final MessageIntelligenceAgency messageIntelligenceAgency = new MessageIntelligenceAgencyImpl();
+    private final List<DeviceContext> deviceContexts = new ArrayList<DeviceContext>();
+    private final MessageIntelligenceAgency messageIntelligenceAgency;
 
-    public DeviceManagerImpl(@Nonnull final DataBroker dataBroker) {
+    private final long barrierNanos = TimeUnit.MILLISECONDS.toNanos(500);
+    private final int maxQueueDepth = 25600;
+
+    public DeviceManagerImpl(@Nonnull final DataBroker dataBroker,
+                             @Nonnull final MessageIntelligenceAgency messageIntelligenceAgency) {
         this.dataBroker = Preconditions.checkNotNull(dataBroker);
-        hashedWheelTimer = new HashedWheelTimer(TICK_DURATION, TimeUnit.MILLISECONDS, 10);
+        hashedWheelTimer = new HashedWheelTimer(TICK_DURATION, TimeUnit.MILLISECONDS, 500);
         /* merge empty nodes to oper DS to predict any problems with missing parent for Node */
         final WriteTransaction tx = dataBroker.newWriteOnlyTransaction();
         tx.merge(LogicalDatastoreType.OPERATIONAL, InstanceIdentifier.create(Nodes.class), new NodesBuilder().build());
         tx.submit();
 
-        emptyRequestContextStack = new RequestContextStack() {
-            @Override
-            public <T> void forgetRequestContext(final RequestContext<T> requestContext) {
-                //NOOP
-            }
-
-            @Override
-            public <T> SettableFuture<RpcResult<T>> storeOrFail(final RequestContext<T> data) {
-                return data.getFuture();
-            }
+        this.messageIntelligenceAgency = messageIntelligenceAgency;
 
+        emptyRequestContextStack = new RequestContextStack() {
             @Override
             public <T> RequestContext<T> createRequestContext() {
-                return new RequestContextImpl<>(this);
+                return new AbstractRequestContext<T>() {
+                    @Override
+                    public void close() {
+                        //NOOP
+                    }
+                };
             }
         };
-        spyPool = new ScheduledThreadPoolExecutor(1);
-        spyPool.scheduleAtFixedRate(messageIntelligenceAgency, spyRate, spyRate, TimeUnit.SECONDS);
     }
 
     @Override
@@ -154,16 +157,20 @@ public class DeviceManagerImpl implements DeviceManager, AutoCloseable {
         // final phase - we have to add new Device to MD-SAL DataStore
         Preconditions.checkNotNull(deviceContext);
         ((DeviceContextImpl) deviceContext).submitTransaction();
-        new BarrierTaskBuilder(deviceContext).buildAndFireBarrierTask();
     }
 
     @Override
     public void deviceConnected(@CheckForNull final ConnectionContext connectionContext) {
         Preconditions.checkArgument(connectionContext != null);
 
+        final Short version = connectionContext.getFeatures().getVersion();
+        OutboundQueueProvider outboundQueueProvider = new OutboundQueueProviderImpl(version);
+        connectionContext.getConnectionAdapter().registerOutboundQueueHandler(outboundQueueProvider, maxQueueDepth, barrierNanos);
+        connectionContext.setOutboundQueueProvider(outboundQueueProvider);
+
         final DeviceState deviceState = new DeviceStateImpl(connectionContext.getFeatures(), connectionContext.getNodeId());
 
-        final DeviceContext deviceContext = new DeviceContextImpl(connectionContext, deviceState, dataBroker, hashedWheelTimer, messageIntelligenceAgency);
+        final DeviceContext deviceContext = new DeviceContextImpl(connectionContext, deviceState, dataBroker, hashedWheelTimer, messageIntelligenceAgency, throttledNotificationsOfferer);
 
         deviceContext.setNotificationService(notificationService);
         deviceContext.setNotificationPublishService(notificationPublishService);
@@ -176,17 +183,16 @@ public class DeviceManagerImpl implements DeviceManager, AutoCloseable {
         final OpenflowProtocolListenerFullImpl messageListener = new OpenflowProtocolListenerFullImpl(
                 connectionContext.getConnectionAdapter(), deviceContext);
 
-        deviceContext.attachOpenflowMessageListener(messageListener);
-
         final ListenableFuture<List<RpcResult<List<MultipartReply>>>> deviceFeaturesFuture;
 
-        final Short version = connectionContext.getFeatures().getVersion();
+
+
         if (OFConstants.OFP_VERSION_1_0 == version) {
             final CapabilitiesV10 capabilitiesV10 = connectionContext.getFeatures().getCapabilitiesV10();
 
             DeviceStateUtil.setDeviceStateBasedOnV10Capabilities(deviceState, capabilitiesV10);
 
-            deviceFeaturesFuture = createDeviceFeaturesForOF10(messageListener, deviceContext, deviceState);
+            deviceFeaturesFuture = createDeviceFeaturesForOF10(deviceContext, deviceState);
             // create empty tables after device description is processed
             chainTableTrunkWriteOF10(deviceContext, deviceFeaturesFuture);
 
@@ -207,8 +213,9 @@ public class DeviceManagerImpl implements DeviceManager, AutoCloseable {
             }
         } else if (OFConstants.OFP_VERSION_1_3 == version) {
             final Capabilities capabilities = connectionContext.getFeatures().getCapabilities();
+            LOG.debug("Setting capabilities for device {}", deviceContext.getDeviceState().getNodeId());
             DeviceStateUtil.setDeviceStateBasedOnV13Capabilities(deviceState, capabilities);
-            deviceFeaturesFuture = createDeviceFeaturesForOF13(messageListener, deviceContext, deviceState);
+            deviceFeaturesFuture = createDeviceFeaturesForOF13(deviceContext, deviceState);
         } else {
             deviceFeaturesFuture = Futures.immediateFailedFuture(new ConnectionException("Unsupported version " + version));
         }
@@ -229,7 +236,7 @@ public class DeviceManagerImpl implements DeviceManager, AutoCloseable {
         });
     }
 
-    private void chainTableTrunkWriteOF10(final DeviceContext deviceContext, final ListenableFuture<List<RpcResult<List<MultipartReply>>>> deviceFeaturesFuture) {
+    private static void chainTableTrunkWriteOF10(final DeviceContext deviceContext, final ListenableFuture<List<RpcResult<List<MultipartReply>>>> deviceFeaturesFuture) {
         Futures.addCallback(deviceFeaturesFuture, new FutureCallback<List<RpcResult<List<MultipartReply>>>>() {
             @Override
             public void onSuccess(final List<RpcResult<List<MultipartReply>>> results) {
@@ -238,11 +245,9 @@ public class DeviceManagerImpl implements DeviceManager, AutoCloseable {
                     allSucceeded &= rpcResult.isSuccessful();
                 }
                 if (allSucceeded) {
-                    synchronized (deviceContext) {
-                        createEmptyFlowCapableNodeInDs(deviceContext);
-                        makeEmptyTables(deviceContext, deviceContext.getDeviceState().getNodeInstanceIdentifier(),
-                                deviceContext.getDeviceState().getFeatures().getTables());
-                    }
+                    createEmptyFlowCapableNodeInDs(deviceContext);
+                    makeEmptyTables(deviceContext, deviceContext.getDeviceState().getNodeInstanceIdentifier(),
+                            deviceContext.getDeviceState().getFeatures().getTables());
                 }
             }
 
@@ -253,37 +258,44 @@ public class DeviceManagerImpl implements DeviceManager, AutoCloseable {
         });
     }
 
-
-    private ListenableFuture<RpcResult<List<MultipartReply>>> processReplyDesc(final OpenflowProtocolListenerFullImpl messageListener,
-                                                                               final DeviceContext deviceContext,
+    private ListenableFuture<RpcResult<List<MultipartReply>>> processReplyDesc(final DeviceContext deviceContext,
                                                                                final DeviceState deviceState) {
-        final ListenableFuture<RpcResult<List<MultipartReply>>> replyDesc = getNodeStaticInfo(messageListener,
-                MultipartType.OFPMPDESC, deviceContext, deviceState.getNodeInstanceIdentifier(), deviceState.getVersion());
+
+        final ListenableFuture<RpcResult<List<MultipartReply>>> replyDesc = getNodeStaticInfo(MultipartType.OFPMPDESC,
+                deviceContext,
+                deviceState.getNodeInstanceIdentifier(),
+                deviceState.getVersion());
         return replyDesc;
     }
 
-    private ListenableFuture<List<RpcResult<List<MultipartReply>>>> createDeviceFeaturesForOF10(final OpenflowProtocolListenerFullImpl messageListener,
-                                                                                                final DeviceContext deviceContext,
+    private ListenableFuture<List<RpcResult<List<MultipartReply>>>> createDeviceFeaturesForOF10(final DeviceContext deviceContext,
                                                                                                 final DeviceState deviceState) {
-        return Futures.allAsList(Arrays.asList(processReplyDesc(messageListener, deviceContext, deviceState)));
+        return Futures.allAsList(Arrays.asList(processReplyDesc(deviceContext, deviceState)));
     }
 
-    private ListenableFuture<List<RpcResult<List<MultipartReply>>>> createDeviceFeaturesForOF13(final OpenflowProtocolListenerFullImpl messageListener,
-                                                                                                final DeviceContext deviceContext,
+    private ListenableFuture<List<RpcResult<List<MultipartReply>>>> createDeviceFeaturesForOF13(final DeviceContext deviceContext,
                                                                                                 final DeviceState deviceState) {
-        final ListenableFuture<RpcResult<List<MultipartReply>>> replyDesc = processReplyDesc(messageListener, deviceContext, deviceState);
+        final ListenableFuture<RpcResult<List<MultipartReply>>> replyDesc = processReplyDesc(deviceContext, deviceState);
 
-        final ListenableFuture<RpcResult<List<MultipartReply>>> replyMeterFeature = getNodeStaticInfo(messageListener,
-                MultipartType.OFPMPMETERFEATURES, deviceContext, deviceState.getNodeInstanceIdentifier(), deviceState.getVersion());
+        final ListenableFuture<RpcResult<List<MultipartReply>>> replyMeterFeature = getNodeStaticInfo(MultipartType.OFPMPMETERFEATURES,
+                deviceContext,
+                deviceState.getNodeInstanceIdentifier(),
+                deviceState.getVersion());
 
-        final ListenableFuture<RpcResult<List<MultipartReply>>> replyGroupFeatures = getNodeStaticInfo(messageListener,
-                MultipartType.OFPMPGROUPFEATURES, deviceContext, deviceState.getNodeInstanceIdentifier(), deviceState.getVersion());
+        final ListenableFuture<RpcResult<List<MultipartReply>>> replyGroupFeatures = getNodeStaticInfo(MultipartType.OFPMPGROUPFEATURES,
+                deviceContext,
+                deviceState.getNodeInstanceIdentifier(),
+                deviceState.getVersion());
 
-        final ListenableFuture<RpcResult<List<MultipartReply>>> replyTableFeatures = getNodeStaticInfo(messageListener,
-                MultipartType.OFPMPTABLEFEATURES, deviceContext, deviceState.getNodeInstanceIdentifier(), deviceState.getVersion());
+        final ListenableFuture<RpcResult<List<MultipartReply>>> replyTableFeatures = getNodeStaticInfo(MultipartType.OFPMPTABLEFEATURES,
+                deviceContext,
+                deviceState.getNodeInstanceIdentifier(),
+                deviceState.getVersion());
 
-        final ListenableFuture<RpcResult<List<MultipartReply>>> replyPortDescription = getNodeStaticInfo(messageListener,
-                MultipartType.OFPMPPORTDESC, deviceContext, deviceState.getNodeInstanceIdentifier(), deviceState.getVersion());
+        final ListenableFuture<RpcResult<List<MultipartReply>>> replyPortDescription = getNodeStaticInfo(MultipartType.OFPMPPORTDESC,
+                deviceContext,
+                deviceState.getNodeInstanceIdentifier(),
+                deviceState.getVersion());
 
         return Futures.allAsList(Arrays.asList(replyDesc,
                 replyMeterFeature,
@@ -303,23 +315,51 @@ public class DeviceManagerImpl implements DeviceManager, AutoCloseable {
         this.translatorLibrary = translatorLibrary;
     }
 
-    private ListenableFuture<RpcResult<List<MultipartReply>>> getNodeStaticInfo(final MultiMsgCollector multiMsgCollector, final MultipartType type, final DeviceContext deviceContext,
+    private ListenableFuture<RpcResult<List<MultipartReply>>> getNodeStaticInfo(final MultipartType type, final DeviceContext deviceContext,
                                                                                 final InstanceIdentifier<Node> nodeII, final short version) {
 
-        final Xid xid = deviceContext.getNextXid();
+        final OutboundQueue queue = deviceContext.getPrimaryConnectionContext().getOutboundQueueProvider();
+        Long reservedXid = queue.reserveEntry();
+        final Xid xid = new Xid(reservedXid);
+
         final RequestContext<List<MultipartReply>> requestContext = emptyRequestContextStack.createRequestContext();
         requestContext.setXid(xid);
 
         LOG.trace("Hooking xid {} to device context - precaution.", requestContext.getXid().getValue());
         deviceContext.hookRequestCtx(requestContext.getXid(), requestContext);
 
+        final ListenableFuture<RpcResult<List<MultipartReply>>> requestContextFuture = requestContext.getFuture();
 
+        final MultiMsgCollector multiMsgCollector = deviceContext.getMultiMsgCollector();
         multiMsgCollector.registerMultipartXid(xid.getValue());
-        Futures.addCallback(requestContext.getFuture(), new FutureCallback<RpcResult<List<MultipartReply>>>() {
+        queue.commitEntry(reservedXid, MultipartRequestInputFactory.makeMultipartRequestInput(xid.getValue(), version, type), new FutureCallback<OfHeader>() {
+            @Override
+            public void onSuccess(final OfHeader ofHeader) {
+                if (ofHeader instanceof MultipartReply) {
+                    MultipartReply multipartReply = (MultipartReply) ofHeader;
+                    multiMsgCollector.addMultipartMsg(multipartReply);
+                } else {
+                    if (null != ofHeader) {
+                        LOG.info("Unexpected response type received {}.", ofHeader.getClass());
+                    } else {
+                        LOG.info("Response received is null.");
+                    }
+                }
+
+            }
+
+            @Override
+            public void onFailure(final Throwable t) {
+                LOG.info("Fail response from OutboundQueue.");
+            }
+        });
+
+        Futures.addCallback(requestContextFuture, new FutureCallback<RpcResult<List<MultipartReply>>>() {
             @Override
             public void onSuccess(final RpcResult<List<MultipartReply>> rpcResult) {
                 final List<MultipartReply> result = rpcResult.getResult();
                 if (result != null) {
+                    LOG.info("Static node {} info: {} collected", nodeII.toString(), type);
                     translateAndWriteReply(type, deviceContext, nodeII, result);
                 } else {
                     final Iterator<RpcError> rpcErrorIterator = rpcResult.getErrors().iterator();
@@ -337,15 +377,18 @@ public class DeviceManagerImpl implements DeviceManager, AutoCloseable {
             }
 
             @Override
-            public void onFailure(final Throwable t) {
-                LOG.info("Failed to retrieve static node {} info: {}", type, t.getMessage());
+            public void onFailure(final Throwable throwable) {
+                LOG.info("Request of type {} for static info of node {} failed.", type, nodeII);
             }
         });
 
+
+/*
         final ListenableFuture<RpcResult<Void>> rpcFuture = JdkFutureAdapters.listenInPoolThread(deviceContext.getPrimaryConnectionContext().getConnectionAdapter()
                 .multipartRequest(MultipartRequestInputFactory.makeMultipartRequestInput(xid.getValue(), version, type)));
         final OFJResult2RequestCtxFuture OFJResult2RequestCtxFuture = new OFJResult2RequestCtxFuture(requestContext, deviceContext);
         OFJResult2RequestCtxFuture.processResultFromOfJava(rpcFuture);
+*/
 
         return requestContext.getFuture();
     }
@@ -446,9 +489,12 @@ public class DeviceManagerImpl implements DeviceManager, AutoCloseable {
 
     @Override
     public void close() throws Exception {
-        for (final DeviceContext deviceContext : synchronizedDeviceContextsList) {
+        for (final DeviceContext deviceContext : deviceContexts) {
             deviceContext.close();
         }
+        if (throttledNotificationsOfferer != null) {
+            throttledNotificationsOfferer.close();
+        }
     }
 
     private static void createEmptyFlowCapableNodeInDs(final DeviceContext deviceContext) {
@@ -459,6 +505,14 @@ public class DeviceManagerImpl implements DeviceManager, AutoCloseable {
 
     @Override
     public void onDeviceContextClosed(final DeviceContext deviceContext) {
-        synchronizedDeviceContextsList.remove(deviceContext);
+        deviceContexts.remove(deviceContext);
+    }
+
+    @Override
+    public void initialize() {
+        spyPool = new ScheduledThreadPoolExecutor(1);
+        spyPool.scheduleAtFixedRate(messageIntelligenceAgency, spyRate, spyRate, TimeUnit.SECONDS);
+
+        throttledNotificationsOfferer = new ThrottledNotificationsOffererImpl<>(notificationPublishService, messageIntelligenceAgency);
     }
 }