Ethanlm commented on a change in pull request #3402: URL: https://github.com/apache/storm/pull/3402#discussion_r664878945
########## File path: storm-client/src/jvm/org/apache/storm/messaging/DeserializingConnectionCallback.java ########## @@ -71,24 +74,26 @@ public void recv(List<TaskMessage> batch) { cb.transfer(ret); } - /** - * Returns serialized byte count traffic metrics. - * - * @return Map of metric counts, or null if disabled - */ @Override - public Object getValueAndReset() { + public void registerMetrics(StormMetricRegistry metricRegistry) { if (!sizeMetricsEnabled) { - return null; + return; } - HashMap<String, Long> outMap = new HashMap<>(); - for (Map.Entry<String, AtomicLong> ent : byteCounts.entrySet()) { - AtomicLong count = ent.getValue(); - if (count.get() > 0) { - outMap.put(ent.getKey(), count.getAndSet(0L)); + + Gauge<Map<String, Long>> sizes = new Gauge<Map<String, Long>>() { + @Override + public Map<String, Long> getValue() { + HashMap<String, Long> counts = new HashMap<>(); + for (Map.Entry<String, AtomicLong> ent : byteCounts.entrySet()) { + AtomicLong count = ent.getValue(); + if (count.get() > 0) { + counts.put(ent.getKey(), count.getAndSet(0L)); Review comment: Looks like this won't support multiple reporters? In the case of multiple reporters being configured, this `getAndSet` will return differently ########## File path: storm-client/src/jvm/org/apache/storm/messaging/netty/Server.java ########## @@ -118,6 +119,30 @@ } catch (InterruptedException e) { throw new RuntimeException(e); } + + if (metricRegistry != null) { + Gauge<Map<String, Integer>> enqueued = new Gauge<Map<String, Integer>>() { + @Override + public Map<String, Integer> getValue() { Review comment: Does it support having multiple reporters? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: dev-unsubscr...@storm.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org