Merge "Fixed for bug 1168 : Issue while update subnet"
[controller.git] / opendaylight / md-sal / statistics-manager / src / main / java / org / opendaylight / controller / md / statistics / manager / StatisticsProvider.java
index 5218d051fa2cdb40ec9fa47f58114abf5c334dc6..8c9b60e43f0b7dc373b3365039ab950f86df6bfc 100644 (file)
  */
 package org.opendaylight.controller.md.statistics.manager;
 
-import java.util.List;
+import java.util.Collection;
+import java.util.Timer;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.Future;
 
-import org.eclipse.xtext.xbase.lib.Exceptions;
 import org.opendaylight.controller.sal.binding.api.NotificationProviderService;
-import org.opendaylight.controller.sal.binding.api.data.DataModificationTransaction;
+import org.opendaylight.controller.sal.binding.api.RpcConsumerRegistry;
+import org.opendaylight.controller.sal.binding.api.data.DataChangeListener;
 import org.opendaylight.controller.sal.binding.api.data.DataProviderService;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.group.statistics.rev131111.GetAllGroupStatisticsInputBuilder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.group.statistics.rev131111.GetAllGroupStatisticsOutput;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.group.statistics.rev131111.GetGroupDescriptionInputBuilder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.group.statistics.rev131111.GetGroupDescriptionOutput;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.group.statistics.rev131111.GetGroupFeaturesInputBuilder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.group.statistics.rev131111.GetGroupFeaturesOutput;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowCapableNode;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.statistics.rev130819.OpendaylightFlowStatisticsService;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.table.statistics.rev131215.OpendaylightFlowTableStatisticsService;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.group.statistics.rev131111.OpendaylightGroupStatisticsService;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeId;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.Nodes;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.nodes.Node;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.meter.statistics.rev131111.GetAllMeterConfigStatisticsInputBuilder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.meter.statistics.rev131111.GetAllMeterConfigStatisticsOutput;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.meter.statistics.rev131111.GetAllMeterStatisticsInputBuilder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.meter.statistics.rev131111.GetAllMeterStatisticsOutput;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.meter.statistics.rev131111.GetMeterFeaturesInputBuilder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.meter.statistics.rev131111.GetMeterFeaturesOutput;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.nodes.NodeKey;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.meter.statistics.rev131111.OpendaylightMeterStatisticsService;
-import org.opendaylight.yangtools.concepts.Registration;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.port.statistics.rev131214.OpendaylightPortStatisticsService;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.queue.statistics.rev131216.OpendaylightQueueStatisticsService;
+import org.opendaylight.yangtools.concepts.ListenerRegistration;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
 import org.opendaylight.yangtools.yang.binding.NotificationListener;
-import org.opendaylight.yangtools.yang.common.RpcResult;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Preconditions;
+
+/**
+ * Following are main responsibilities of the class:
+ * 1) Invoke statistics request thread to send periodic statistics request to all the
+ * flow capable switch connected to the controller. It sends statistics request for
+ * Group,Meter,Table,Flow,Queue,Aggregate stats.
+ *
+ * 2) Invoke statistics ager thread, to clean up all the stale statistics data from
+ * operational data store.
+ *
+ * @author avishnoi@in.ibm.com
+ *
+ */
 public class StatisticsProvider implements AutoCloseable {
+    private static final Logger spLogger = LoggerFactory.getLogger(StatisticsProvider.class);
 
-    public final static Logger spLogger = LoggerFactory.getLogger(StatisticsProvider.class);
-    
-    private DataProviderService dps;
+    private final ConcurrentMap<NodeId, NodeStatisticsHandler> handlers = new ConcurrentHashMap<>();
+    private final Timer timer = new Timer("statistics-manager", true);
+    private final DataProviderService dps;
 
-    private NotificationProviderService nps;
-    
     private OpendaylightGroupStatisticsService groupStatsService;
-    
+
     private OpendaylightMeterStatisticsService meterStatsService;
-    
-    private Thread statisticsRequesterThread;
-    
-    private final  InstanceIdentifier<Nodes> nodesIdentifier = InstanceIdentifier.builder().node(Nodes.class).toInstance();
-    
-    //Local caching of stats
-    
-    private final ConcurrentMap<NodeId,NodeStatistics> statisticsCache = 
-            new ConcurrentHashMap<NodeId,NodeStatistics>();
-    
-    public DataProviderService getDataService() {
-      return this.dps;
-    }
-    
-    public void setDataService(final DataProviderService dataService) {
-      this.dps = dataService;
-    }
-    
-    public NotificationProviderService getNotificationService() {
-      return this.nps;
-    }
-    
-    public void setNotificationService(final NotificationProviderService notificationService) {
-      this.nps = notificationService;
+
+    private OpendaylightFlowStatisticsService flowStatsService;
+
+    private OpendaylightPortStatisticsService portStatsService;
+
+    private OpendaylightFlowTableStatisticsService flowTableStatsService;
+
+    private OpendaylightQueueStatisticsService queueStatsService;
+
+    private final StatisticsRequestScheduler srScheduler;
+
+    public StatisticsProvider(final DataProviderService dataService) {
+        this.dps = Preconditions.checkNotNull(dataService);
+        this.srScheduler = new StatisticsRequestScheduler();
     }
 
-    private final StatisticsUpdateCommiter updateCommiter = new StatisticsUpdateCommiter(StatisticsProvider.this);
-    
-    private Registration<NotificationListener> listenerRegistration;
-    
-    public void start() {
-        
-        NotificationProviderService nps = this.getNotificationService();
-        Registration<NotificationListener> registerNotificationListener = nps.registerNotificationListener(this.updateCommiter);
-        this.listenerRegistration = registerNotificationListener;
-        
-        // Get Group/Meter statistics service instance
-        groupStatsService = StatisticsManagerActivator.getProviderContext().
-                getRpcService(OpendaylightGroupStatisticsService.class);
-        
-        meterStatsService = StatisticsManagerActivator.getProviderContext().
-                getRpcService(OpendaylightMeterStatisticsService.class);
-
-        statisticsRequesterThread = new Thread( new Runnable(){
-
-            @Override
-            public void run() {
-                while(true){
-                    try {
-                        spLogger.info("Statistics requester thread started with timer interval : {}",5000);
-                        
-                        statsRequestSender();
-                        
-                        Thread.sleep(5000);
-                    }catch (Exception e){
-                        spLogger.error("Exception occurred while sending stats request : {}",e);
-                    }
-                }
-            }
-        });
+    private final StatisticsListener updateCommiter = new StatisticsListener(StatisticsProvider.this);
+
+    private ListenerRegistration<NotificationListener> listenerRegistration;
+
+    private ListenerRegistration<DataChangeListener> flowCapableTrackerRegistration;
+
+    public void start(final NotificationProviderService nps, final RpcConsumerRegistry rpcRegistry) {
+
+        // Get Group/Meter statistics service instances
+        groupStatsService = rpcRegistry.getRpcService(OpendaylightGroupStatisticsService.class);
+        meterStatsService = rpcRegistry.getRpcService(OpendaylightMeterStatisticsService.class);
+        flowStatsService = rpcRegistry.getRpcService(OpendaylightFlowStatisticsService.class);
+        portStatsService = rpcRegistry.getRpcService(OpendaylightPortStatisticsService.class);
+        flowTableStatsService = rpcRegistry.getRpcService(OpendaylightFlowTableStatisticsService.class);
+        queueStatsService = rpcRegistry.getRpcService(OpendaylightQueueStatisticsService.class);
+        this.srScheduler.start();
+
+        // Start receiving notifications
+        this.listenerRegistration = nps.registerNotificationListener(this.updateCommiter);
+
+        // Register for switch connect/disconnect notifications
+        final InstanceIdentifier<FlowCapableNode> fcnId = InstanceIdentifier.builder(Nodes.class)
+                .child(Node.class).augmentation(FlowCapableNode.class).build();
+        spLogger.debug("Registering FlowCapable tracker to {}", fcnId);
+        this.flowCapableTrackerRegistration = dps.registerDataChangeListener(fcnId,
+                new FlowCapableTracker(this, fcnId));
+
         spLogger.info("Statistics Provider started.");
     }
-    
-    protected DataModificationTransaction startChange() {
-        
-        DataProviderService dps = this.getDataService();
-        return dps.beginTransaction();
-    }
-    
-    private void statsRequestSender(){
-        
-        //Need to call API to receive all the nodes connected to controller.
-        List<Node> targetNodes = getAllConnectedNodes();
-
-        for (Node targetNode : targetNodes){
-            spLogger.info("Send request for stats collection to node : {})",targetNode.getId());
-            
-            //We need to add check, so see if groups/meters are supported
-            //by the target node. Below check doesn't look good.
-            if(targetNode.getId().getValue().contains("openflow:")){
-                sendAllGroupStatisticsRequest(targetNode);
-                
-                sendAllMeterStatisticsRequest(targetNode);
-                
-                sendGroupDescriptionRequest(targetNode);
-                
-                sendGroupFeaturesRequest(targetNode);
-                
-                sendMeterConfigStatisticsRequest(targetNode);
-                
-                sendMeterFeaturesRequest(targetNode);
-            }
+
+    /**
+     * Get the handler for a particular node.
+     *
+     * @param nodeId source node
+     * @return Node statistics handler for that node. Null if the statistics should
+     *         not handled.
+     */
+    public final NodeStatisticsHandler getStatisticsHandler(final NodeId nodeId) {
+        Preconditions.checkNotNull(nodeId);
+        NodeStatisticsHandler handler = handlers.get(nodeId);
+        if (handler == null) {
+            spLogger.info("Attempted to get non-existing handler for {}", nodeId);
         }
-    }
-    
-    private void sendAllGroupStatisticsRequest(Node targetNode){
-        
-        final GetAllGroupStatisticsInputBuilder input = new GetAllGroupStatisticsInputBuilder();
-        
-        input.setId(targetNode.getId());
-
-        Future<RpcResult<GetAllGroupStatisticsOutput>> response = 
-                groupStatsService.getAllGroupStatistics(input.build());
-    }
-    
-    private void sendGroupDescriptionRequest(Node targetNode){
-        final GetGroupDescriptionInputBuilder input = new GetGroupDescriptionInputBuilder();
-        
-        input.setId(targetNode.getId());
-        
-        Future<RpcResult<GetGroupDescriptionOutput>> response = 
-                groupStatsService.getGroupDescription(input.build());
-    }
-    
-    private void sendGroupFeaturesRequest(Node targetNode){
-        
-        GetGroupFeaturesInputBuilder input = new GetGroupFeaturesInputBuilder();
-        
-        input.setId(targetNode.getId());
-        
-        Future<RpcResult<GetGroupFeaturesOutput>> response = 
-                groupStatsService.getGroupFeatures(input.build());
-    }
-    
-    private void sendAllMeterStatisticsRequest(Node targetNode){
-        
-        GetAllMeterStatisticsInputBuilder input = new GetAllMeterStatisticsInputBuilder();
-        
-        input.setId(targetNode.getId());
-        
-        Future<RpcResult<GetAllMeterStatisticsOutput>> response = 
-                meterStatsService.getAllMeterStatistics(input.build());
-    }
-    
-    private void sendMeterConfigStatisticsRequest(Node targetNode){
-        
-        GetAllMeterConfigStatisticsInputBuilder input = new GetAllMeterConfigStatisticsInputBuilder();
-        
-        input.setId(targetNode.getId());
-        
-        Future<RpcResult<GetAllMeterConfigStatisticsOutput>> response = 
-                meterStatsService.getAllMeterConfigStatistics(input.build());
-        
-    }
-    private void sendMeterFeaturesRequest(Node targetNode){
-     
-        GetMeterFeaturesInputBuilder input = new GetMeterFeaturesInputBuilder();
-        
-        input.setId(targetNode.getId());
-        
-        Future<RpcResult<GetMeterFeaturesOutput>> response = 
-                meterStatsService.getMeterFeatures(input.build());
-    }
-    
-    public ConcurrentMap<NodeId, NodeStatistics> getStatisticsCache() {
-        return statisticsCache;
-    }
-    
-    private List<Node> getAllConnectedNodes(){
-        
-        Nodes nodes = (Nodes) dps.readOperationalData(nodesIdentifier);
-        spLogger.info("Number of connected nodes : {}",nodes.getNode().size());
-        return nodes.getNode();
+        return handler;
     }
 
-    @SuppressWarnings("deprecation")
     @Override
-    public void close(){
-        
+    public void close() {
         try {
-            spLogger.info("Statistics Provider stopped.");
             if (this.listenerRegistration != null) {
-              
                 this.listenerRegistration.close();
-                
-                this.statisticsRequesterThread.destroy();
-            
+                this.listenerRegistration = null;
+            }
+            if (this.flowCapableTrackerRegistration != null) {
+                this.flowCapableTrackerRegistration.close();
+                this.flowCapableTrackerRegistration = null;
+            }
+            timer.cancel();
+        } catch (Exception e) {
+            spLogger.warn("Failed to stop Statistics Provider completely", e);
+        } finally {
+            spLogger.info("Statistics Provider stopped.");
+        }
+    }
+
+    void startNodeHandlers(final Collection<NodeKey> addedNodes) {
+        for (NodeKey key : addedNodes) {
+            if (handlers.containsKey(key.getId())) {
+                spLogger.warn("Attempted to start already-existing handler for {}, very strange", key.getId());
+                continue;
             }
-          } catch (Throwable e) {
-            throw Exceptions.sneakyThrow(e);
-          }
 
+            final NodeStatisticsHandler h = new NodeStatisticsHandler(dps, key,
+                    flowStatsService, flowTableStatsService, groupStatsService,
+                    meterStatsService, portStatsService, queueStatsService,srScheduler);
+            final NodeStatisticsHandler old = handlers.putIfAbsent(key.getId(), h);
+            if (old == null) {
+                spLogger.debug("Started node handler for {}", key.getId());
+                h.start(timer);
+            } else {
+                spLogger.debug("Prevented race on handler for {}", key.getId());
+            }
+        }
     }
 
+    void stopNodeHandlers(final Collection<NodeKey> removedNodes) {
+        for (NodeKey key : removedNodes) {
+            final NodeStatisticsHandler s = handlers.remove(key.getId());
+            if (s != null) {
+                spLogger.debug("Stopping node handler for {}", key.getId());
+                s.close();
+            } else {
+                spLogger.warn("Attempted to remove non-existing handler for {}, very strange", key.getId());
+            }
+        }
+    }
 }