Move queue/meter/flow listeners into their trackers
[controller.git] / opendaylight / md-sal / statistics-manager / src / main / java / org / opendaylight / controller / md / statistics / manager / NodeStatisticsHandler.java
index 5d5d17230fda1d9efa458deddc3d5fbb44626c86..691b9c0b15f58a9d6a01d05e7146c8a69251e332 100644 (file)
@@ -9,8 +9,6 @@ package org.opendaylight.controller.md.statistics.manager;
 
 import java.util.Collection;
 import java.util.List;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
 import org.opendaylight.controller.md.statistics.manager.MultipartMessageManager.StatsRequestType;
@@ -20,29 +18,15 @@ import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.Fl
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.tables.Table;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.tables.TableBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.tables.TableKey;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.tables.table.Flow;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.statistics.rev130819.AggregateFlowStatisticsData;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.statistics.rev130819.AggregateFlowStatisticsDataBuilder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.statistics.rev130819.GetAggregateFlowStatisticsFromFlowTableForAllFlowsInputBuilder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.statistics.rev130819.GetAggregateFlowStatisticsFromFlowTableForAllFlowsOutput;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.statistics.rev130819.GetAllFlowsStatisticsFromAllFlowTablesInputBuilder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.statistics.rev130819.GetAllFlowsStatisticsFromAllFlowTablesOutput;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.statistics.rev130819.GetFlowStatisticsFromFlowTableInputBuilder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.statistics.rev130819.GetFlowStatisticsFromFlowTableOutput;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.statistics.rev130819.OpendaylightFlowStatisticsService;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.statistics.rev130819.aggregate.flow.statistics.AggregateFlowStatisticsBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.statistics.rev130819.flow.and.statistics.map.list.FlowAndStatisticsMapList;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.table.statistics.rev131215.GetFlowTablesStatisticsInputBuilder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.table.statistics.rev131215.GetFlowTablesStatisticsOutput;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.table.statistics.rev131215.OpendaylightFlowTableStatisticsService;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.table.statistics.rev131215.flow.table.and.statistics.map.FlowTableAndStatisticsMap;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.transaction.rev131103.TransactionAware;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.transaction.rev131103.TransactionId;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.types.queue.rev130925.QueueId;
-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.NodeGroupFeatures;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.group.statistics.rev131111.NodeGroupFeaturesBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.group.statistics.rev131111.OpendaylightGroupStatisticsService;
@@ -50,16 +34,11 @@ import org.opendaylight.yang.gen.v1.urn.opendaylight.group.statistics.rev131111.
 import org.opendaylight.yang.gen.v1.urn.opendaylight.group.types.rev131018.GroupFeatures;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.group.types.rev131018.group.desc.stats.reply.GroupDescStats;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.group.types.rev131018.group.statistics.reply.GroupStats;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeConnectorId;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.NodeRef;
 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.inventory.rev130819.nodes.NodeBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.inventory.rev130819.nodes.NodeKey;
-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.NodeMeterFeatures;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.meter.statistics.rev131111.NodeMeterFeaturesBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.meter.statistics.rev131111.OpendaylightMeterStatisticsService;
@@ -68,22 +47,18 @@ import org.opendaylight.yang.gen.v1.urn.opendaylight.meter.types.rev130918.Meter
 import org.opendaylight.yang.gen.v1.urn.opendaylight.meter.types.rev130918.meter.config.stats.reply.MeterConfigStats;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.meter.types.rev130918.meter.statistics.reply.MeterStats;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.model.statistics.types.rev130925.AggregateFlowStatistics;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.port.statistics.rev131214.GetAllNodeConnectorsStatisticsInputBuilder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.port.statistics.rev131214.GetAllNodeConnectorsStatisticsOutput;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.port.statistics.rev131214.OpendaylightPortStatisticsService;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.port.statistics.rev131214.node.connector.statistics.and.port.number.map.NodeConnectorStatisticsAndPortNumberMap;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.queue.statistics.rev131216.GetAllQueuesStatisticsFromAllPortsInputBuilder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.queue.statistics.rev131216.GetAllQueuesStatisticsFromAllPortsOutput;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.queue.statistics.rev131216.GetQueueStatisticsFromGivenPortInputBuilder;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.queue.statistics.rev131216.GetQueueStatisticsFromGivenPortOutput;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.queue.statistics.rev131216.OpendaylightQueueStatisticsService;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.queue.statistics.rev131216.queue.id.and.statistics.map.QueueIdAndStatisticsMap;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
-import org.opendaylight.yangtools.yang.common.RpcResult;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 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.ListenableFuture;
 
 /**
  * This class handles the lifecycle of per-node statistics. It receives data
@@ -92,17 +67,10 @@ import com.google.common.base.Preconditions;
  *
  * @author avishnoi@in.ibm.com
  */
-public final class NodeStatisticsHandler implements AutoCloseable {
+public final class NodeStatisticsHandler implements AutoCloseable, FlowCapableContext {
     private static final Logger logger = LoggerFactory.getLogger(NodeStatisticsHandler.class);
     private static final int NUMBER_OF_WAIT_CYCLES = 2;
 
-    private final OpendaylightFlowStatisticsService flowStatsService;
-    private final OpendaylightFlowTableStatisticsService flowTableStatsService;
-    private final OpendaylightGroupStatisticsService groupStatsService;
-    private final OpendaylightMeterStatisticsService meterStatsService;
-    private final OpendaylightPortStatisticsService portStatsService;
-    private final OpendaylightQueueStatisticsService queueStatsService;
-
     private final MultipartMessageManager msgManager = new MultipartMessageManager();
     private final InstanceIdentifier<Node> targetNodeIdentifier;
     private final FlowStatsTracker flowStats;
@@ -129,40 +97,37 @@ public final class NodeStatisticsHandler implements AutoCloseable {
         this.targetNodeIdentifier = InstanceIdentifier.builder(Nodes.class).child(Node.class, targetNodeKey).build();
         this.targetNodeRef = new NodeRef(targetNodeIdentifier);
 
-        this.flowStatsService = flowStatsService;
-        this.flowTableStatsService = flowTableStatsService;
-        this.groupStatsService = groupStatsService;
-        this.meterStatsService = meterStatsService;
-        this.portStatsService = portStatsService;
-        this.queueStatsService = queueStatsService;
-
         final long lifetimeNanos = TimeUnit.MILLISECONDS.toNanos(StatisticsProvider.STATS_COLLECTION_MILLIS * NUMBER_OF_WAIT_CYCLES);
-        flowStats = new FlowStatsTracker(targetNodeIdentifier, dps, lifetimeNanos);
-        flowTableStats = new FlowTableStatsTracker(targetNodeIdentifier, dps, lifetimeNanos);
-        groupDescStats = new GroupDescStatsTracker(targetNodeIdentifier, dps, lifetimeNanos);
-        groupStats = new GroupStatsTracker(targetNodeIdentifier, dps, lifetimeNanos);
-        meterConfigStats = new MeterConfigStatsTracker(targetNodeIdentifier, dps, lifetimeNanos);
-        meterStats = new MeterStatsTracker(targetNodeIdentifier, dps, lifetimeNanos);
-        nodeConnectorStats = new NodeConnectorStatsTracker(targetNodeIdentifier, dps, lifetimeNanos);
-        queueStats = new QueueStatsTracker(targetNodeIdentifier, dps, lifetimeNanos);
+
+        flowStats = new FlowStatsTracker(flowStatsService, this, lifetimeNanos);
+        flowTableStats = new FlowTableStatsTracker(flowTableStatsService, this, lifetimeNanos);
+        groupDescStats = new GroupDescStatsTracker(groupStatsService, this, lifetimeNanos);
+        groupStats = new GroupStatsTracker(groupStatsService, this, lifetimeNanos);
+        meterConfigStats = new MeterConfigStatsTracker(meterStatsService, this, lifetimeNanos);
+        meterStats = new MeterStatsTracker(meterStatsService, this, lifetimeNanos);
+        nodeConnectorStats = new NodeConnectorStatsTracker(portStatsService, this, lifetimeNanos);
+        queueStats = new QueueStatsTracker(queueStatsService, this, lifetimeNanos);
     }
 
     public NodeKey getTargetNodeKey() {
         return targetNodeKey;
     }
 
-    public Collection<TableKey> getKnownTables() {
-        return flowTableStats.getTables();
-    }
-
-    public InstanceIdentifier<Node> getTargetNodeIdentifier() {
+    @Override
+    public InstanceIdentifier<Node> getNodeIdentifier() {
         return targetNodeIdentifier;
     }
 
-    public NodeRef getTargetNodeRef() {
+    @Override
+    public NodeRef getNodeRef() {
         return targetNodeRef;
     }
 
+    @Override
+    public DataModificationTransaction startDataModification() {
+        return dps.beginTransaction();
+    }
+
     public synchronized void updateGroupDescStats(TransactionAware transaction, Boolean more, List<GroupDescStats> list) {
         if (msgManager.isExpectedTransaction(transaction, more)) {
             groupDescStats.updateStats(list);
@@ -290,176 +255,78 @@ public final class NodeStatisticsHandler implements AutoCloseable {
     public synchronized void requestPeriodicStatistics() {
         logger.debug("Send requests for statistics collection to node : {}", targetNodeKey);
 
-        try{
-            if(flowTableStatsService != null){
-                final GetFlowTablesStatisticsInputBuilder input = new GetFlowTablesStatisticsInputBuilder();
-                input.setNode(targetNodeRef);
+        flowTableStats.request();
 
-                Future<RpcResult<GetFlowTablesStatisticsOutput>> response = flowTableStatsService.getFlowTablesStatistics(input.build());
-                recordExpectedTransaction(response.get().getResult().getTransactionId(), StatsRequestType.ALL_FLOW_TABLE);
-            }
-            if(flowStatsService != null){
-                // FIXME: it does not make sense to trigger this before sendAllFlowTablesStatisticsRequest()
-                //        comes back -- we do not have any tables anyway.
-                sendAggregateFlowsStatsFromAllTablesRequest();
-
-                sendAllFlowsStatsFromAllTablesRequest();
-            }
-            if(portStatsService != null){
-                sendAllNodeConnectorsStatisticsRequest();
-            }
-            if(groupStatsService != null){
-                sendAllGroupStatisticsRequest();
-                sendGroupDescriptionRequest();
-            }
-            if(meterStatsService != null){
-                sendAllMeterStatisticsRequest();
-                sendMeterConfigStatisticsRequest();
-            }
-            if(queueStatsService != null){
-                sendAllQueueStatsFromAllNodeConnector();
-            }
-        } catch(Exception e) {
-            logger.error("Exception occured while sending statistics requests", e);
+        // FIXME: it does not make sense to trigger this before sendAllFlowTablesStatisticsRequest()
+        //        comes back -- we do not have any tables anyway.
+        final Collection<TableKey> tables = flowTableStats.getTables();
+        logger.debug("Node {} supports {} table(s)", targetNodeKey, tables.size());
+        for (final TableKey key : tables) {
+            logger.debug("Send aggregate stats request for flow table {} to node {}", key.getId(), targetNodeKey);
+            flowStats.requestAggregateFlows(key);
         }
+
+        flowStats.requestAllFlowsAllTables();
+        nodeConnectorStats.request();
+        groupStats.request();
+        groupDescStats.request();
+        meterStats.request();
+        meterConfigStats.request();
+        queueStats.request();
     }
 
     public synchronized void start() {
+        flowStats.start(dps);
+        groupDescStats.start(dps);
+        groupStats.start(dps);
+        meterConfigStats.start(dps);
+        meterStats.start(dps);
+        queueStats.start(dps);
+
         requestPeriodicStatistics();
     }
 
     @Override
     public synchronized void close() {
-        // FIXME: cleanup any resources we hold (registrations, etc.)
-        logger.debug("Statistics handler for {} shut down", targetNodeKey.getId());
-    }
-
-    synchronized void sendFlowStatsFromTableRequest(Flow flow) throws InterruptedException, ExecutionException{
-        final GetFlowStatisticsFromFlowTableInputBuilder input =
-                new GetFlowStatisticsFromFlowTableInputBuilder(flow);
-
-        input.setNode(targetNodeRef);
-
-        Future<RpcResult<GetFlowStatisticsFromFlowTableOutput>> response =
-                flowStatsService.getFlowStatisticsFromFlowTable(input.build());
-
-        recordExpectedTransaction(response.get().getResult().getTransactionId(), StatsRequestType.ALL_FLOW);
-    }
-
-    synchronized void sendGroupDescriptionRequest() throws InterruptedException, ExecutionException{
-        final GetGroupDescriptionInputBuilder input = new GetGroupDescriptionInputBuilder();
-
-        input.setNode(targetNodeRef);
-
-        Future<RpcResult<GetGroupDescriptionOutput>> response =
-                groupStatsService.getGroupDescription(input.build());
-
-        recordExpectedTransaction(response.get().getResult().getTransactionId(), StatsRequestType.GROUP_DESC);
-    }
-
-    synchronized void sendMeterConfigStatisticsRequest() throws InterruptedException, ExecutionException{
-
-        GetAllMeterConfigStatisticsInputBuilder input = new GetAllMeterConfigStatisticsInputBuilder();
-
-        input.setNode(targetNodeRef);
-
-        Future<RpcResult<GetAllMeterConfigStatisticsOutput>> response =
-                meterStatsService.getAllMeterConfigStatistics(input.build());
-
-        recordExpectedTransaction(response.get().getResult().getTransactionId(), StatsRequestType.METER_CONFIG);
-    }
-
-    synchronized void sendQueueStatsFromGivenNodeConnector(NodeConnectorId nodeConnectorId, QueueId queueId) throws InterruptedException, ExecutionException {
-        GetQueueStatisticsFromGivenPortInputBuilder input = new GetQueueStatisticsFromGivenPortInputBuilder();
+        flowStats.close();
+        groupDescStats.close();
+        groupStats.close();
+        meterConfigStats.close();
+        meterStats.close();
+        queueStats.close();
 
-        input.setNode(targetNodeRef);
-        input.setNodeConnectorId(nodeConnectorId);
-        input.setQueueId(queueId);
-        Future<RpcResult<GetQueueStatisticsFromGivenPortOutput>> response =
-                queueStatsService.getQueueStatisticsFromGivenPort(input.build());
-
-        recordExpectedTransaction(response.get().getResult().getTransactionId(), StatsRequestType.ALL_QUEUE_STATS);;
-    }
-
-    private void sendAllMeterStatisticsRequest() throws InterruptedException, ExecutionException{
-
-        GetAllMeterStatisticsInputBuilder input = new GetAllMeterStatisticsInputBuilder();
-
-        input.setNode(targetNodeRef);
-
-        Future<RpcResult<GetAllMeterStatisticsOutput>> response =
-                meterStatsService.getAllMeterStatistics(input.build());
-
-        recordExpectedTransaction(response.get().getResult().getTransactionId(), StatsRequestType.ALL_METER);
-    }
-
-    private void sendAllFlowsStatsFromAllTablesRequest() throws InterruptedException, ExecutionException{
-        final GetAllFlowsStatisticsFromAllFlowTablesInputBuilder input = new GetAllFlowsStatisticsFromAllFlowTablesInputBuilder();
-        input.setNode(targetNodeRef);
-
-        Future<RpcResult<GetAllFlowsStatisticsFromAllFlowTablesOutput>> response = flowStatsService.getAllFlowsStatisticsFromAllFlowTables(input.build());
-
-        recordExpectedTransaction(response.get().getResult().getTransactionId(), StatsRequestType.ALL_FLOW);
-    }
-
-    private void sendAggregateFlowsStatsFromAllTablesRequest() throws InterruptedException, ExecutionException{
-        final Collection<TableKey> tables = getKnownTables();
-        logger.debug("Node {} supports {} table(s)", targetNodeKey, tables.size());
-
-        for (TableKey key : tables) {
-            sendAggregateFlowsStatsFromTableRequest(key.getId().shortValue());
-        }
-    }
-
-    private void sendAggregateFlowsStatsFromTableRequest(Short tableId) throws InterruptedException, ExecutionException{
-        logger.debug("Send aggregate stats request for flow table {} to node {}",tableId, targetNodeKey);
-        GetAggregateFlowStatisticsFromFlowTableForAllFlowsInputBuilder input =
-                new GetAggregateFlowStatisticsFromFlowTableForAllFlowsInputBuilder();
-
-        input.setNode(new NodeRef(InstanceIdentifier.builder(Nodes.class).child(Node.class, targetNodeKey).toInstance()));
-        input.setTableId(new org.opendaylight.yang.gen.v1.urn.opendaylight.table.types.rev131026.TableId(tableId));
-        Future<RpcResult<GetAggregateFlowStatisticsFromFlowTableForAllFlowsOutput>> response =
-                flowStatsService.getAggregateFlowStatisticsFromFlowTableForAllFlows(input.build());
-
-        recordExpectedTableTransaction(response.get().getResult().getTransactionId(), tableId);
-    }
-
-    private void sendAllQueueStatsFromAllNodeConnector() throws InterruptedException, ExecutionException {
-        GetAllQueuesStatisticsFromAllPortsInputBuilder input = new GetAllQueuesStatisticsFromAllPortsInputBuilder();
-
-        input.setNode(targetNodeRef);
-
-        Future<RpcResult<GetAllQueuesStatisticsFromAllPortsOutput>> response =
-                queueStatsService.getAllQueuesStatisticsFromAllPorts(input.build());
-
-        recordExpectedTransaction(response.get().getResult().getTransactionId(), StatsRequestType.ALL_QUEUE_STATS);
-    }
-
-    private void sendAllNodeConnectorsStatisticsRequest() throws InterruptedException, ExecutionException{
-        final GetAllNodeConnectorsStatisticsInputBuilder input = new GetAllNodeConnectorsStatisticsInputBuilder();
-
-        input.setNode(targetNodeRef);
-
-        Future<RpcResult<GetAllNodeConnectorsStatisticsOutput>> response =
-                portStatsService.getAllNodeConnectorsStatistics(input.build());
-        recordExpectedTransaction(response.get().getResult().getTransactionId(), StatsRequestType.ALL_PORT);
+        logger.debug("Statistics handler for {} shut down", targetNodeKey.getId());
     }
 
-    private void sendAllGroupStatisticsRequest() throws InterruptedException, ExecutionException{
-        final GetAllGroupStatisticsInputBuilder input = new GetAllGroupStatisticsInputBuilder();
-        input.setNode(targetNodeRef);
-
-        Future<RpcResult<GetAllGroupStatisticsOutput>> response =
-                groupStatsService.getAllGroupStatistics(input.build());
+    @Override
+    public void registerTransaction(final ListenableFuture<TransactionId> future, final StatsRequestType type) {
+        Futures.addCallback(future, new FutureCallback<TransactionId>() {
+            @Override
+            public void onSuccess(TransactionId result) {
+                msgManager.recordExpectedTransaction(result, type);
+                logger.debug("Transaction {} for node {} sent successfully", result, targetNodeKey);
+            }
 
-        recordExpectedTransaction(response.get().getResult().getTransactionId(), StatsRequestType.ALL_GROUP);
+            @Override
+            public void onFailure(Throwable t) {
+                logger.warn("Failed to send statistics request for node {}", targetNodeKey, t);
+            }
+        });
     }
 
-    private void recordExpectedTransaction(TransactionId transactionId, StatsRequestType reqType) {
-        msgManager.recordExpectedTransaction(transactionId, reqType);
-    }
+    @Override
+    public void registerTableTransaction(final ListenableFuture<TransactionId> future, final Short id) {
+        Futures.addCallback(future, new FutureCallback<TransactionId>() {
+            @Override
+            public void onSuccess(TransactionId result) {
+                msgManager.recordExpectedTableTransaction(result, StatsRequestType.AGGR_FLOW, id);
+                logger.debug("Transaction {} for node {} table {} sent successfully", result, targetNodeKey, id);
+            }
 
-    private void recordExpectedTableTransaction(TransactionId transactionId, Short tableId) {
-        msgManager.recordExpectedTableTransaction(transactionId, StatsRequestType.AGGR_FLOW, tableId);
+            @Override
+            public void onFailure(Throwable t) {
+                logger.warn("Failed to send table statistics request for node {} table {}", targetNodeKey, id, t);
+            }
+        });
     }
 }