[ https://issues.apache.org/jira/browse/FLINK-10252?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16709834#comment-16709834 ]
ASF GitHub Bot commented on FLINK-10252: ---------------------------------------- yanghua commented on a change in pull request #6850: [FLINK-10252] Handle oversized metric messges URL: https://github.com/apache/flink/pull/6850#discussion_r238988939 ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricQueryService.java ########## @@ -119,6 +121,101 @@ public void onReceive(Object message) { } } + public void unregister(Metric metric) { + if (metric instanceof Counter) { + this.counters.remove(metric); + } else if (metric instanceof Gauge) { + this.gauges.remove(metric); + } else if (metric instanceof Histogram) { + this.histograms.remove(metric); + } else if (metric instanceof Meter) { + this.meters.remove(metric); + } + } + + private MetricDumpSerialization.MetricSerializationResult verifyResultAndUnregisterOversizedMetrics( + MetricDumpSerialization.MetricSerializationResult serializationResult) { + + byte[] serializedCounters; + int numCounters; + if (serializationResult.serializedCounters.length > maximumFramesize) { + LOG.warn("The serialized counter metric is larger than the maximum frame size, " + + " so maybe not all metrics would be reported."); + serializedCounters = new byte[0]; + numCounters = 0; + + for (Metric metric : this.counters.keySet()) { + unregister(metric); Review comment: @zentol I agree with you. Oversize may be just a short-term phenomenon (in particular, `maximum-frame-size` is still configurable), the unregister operation may be overreacting. Subsequent optimizations can use statistical oversize times to estimate whether this phenomenon is long-term. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > Handle oversized metric messges > ------------------------------- > > Key: FLINK-10252 > URL: https://issues.apache.org/jira/browse/FLINK-10252 > Project: Flink > Issue Type: Sub-task > Components: Metrics > Affects Versions: 1.5.3, 1.6.0, 1.7.0 > Reporter: Till Rohrmann > Assignee: vinoyang > Priority: Critical > Labels: pull-request-available > Fix For: 1.5.6, 1.6.3, 1.8.0, 1.7.1 > > > Since the {{MetricQueryService}} is implemented as an Akka actor, it can only > send messages of a smaller size then the current {{akka.framesize}}. We > should check similarly to FLINK-10251 whether the payload exceeds the maximum > framesize and fail fast if it is true. -- This message was sent by Atlassian JIRA (v7.6.3#76005)