[ 
https://issues.apache.org/jira/browse/FLINK-10252?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16672696#comment-16672696
 ] 

ASF GitHub Bot commented on FLINK-10252:
----------------------------------------

zentol commented on a change in pull request #6850: [FLINK-10252] Handle 
oversized metric messges
URL: https://github.com/apache/flink/pull/6850#discussion_r230287688
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricDumpSerialization.java
 ##########
 @@ -124,55 +160,135 @@ public MetricSerializationResult serialize(
                        Map<Counter, Tuple2<QueryScopeInfo, String>> counters,
                        Map<Gauge<?>, Tuple2<QueryScopeInfo, String>> gauges,
                        Map<Histogram, Tuple2<QueryScopeInfo, String>> 
histograms,
-                       Map<Meter, Tuple2<QueryScopeInfo, String>> meters) {
+                       Map<Meter, Tuple2<QueryScopeInfo, String>> meters,
+                       long maximumFramesize) {
+
+                       boolean markUnserializedMetrics = false;
 
-                       buffer.clear();
+                       Map<Counter, Tuple2<QueryScopeInfo, String>> 
unserializedCounters = new HashMap<>();
+                       Map<Gauge<?>, Tuple2<QueryScopeInfo, String>> 
unserializedGauges = new HashMap<>();
+                       Map<Histogram, Tuple2<QueryScopeInfo, String>> 
unserializedHistograms = new HashMap<>();
+                       Map<Meter, Tuple2<QueryScopeInfo, String>> 
unserializedMeters = new HashMap<>();
 
+                       countersBuffer.clear();
                        int numCounters = 0;
                        for (Map.Entry<Counter, Tuple2<QueryScopeInfo, String>> 
entry : counters.entrySet()) {
+                               if (markUnserializedMetrics) {
+                                       
unserializedCounters.put(entry.getKey(), entry.getValue());
+                                       continue;
+                               }
+
                                try {
-                                       serializeCounter(buffer, 
entry.getValue().f0, entry.getValue().f1, entry.getKey());
+                                       serializeCounter(countersBuffer, 
entry.getValue().f0, entry.getValue().f1, entry.getKey());
                                        numCounters++;
+                                       if (countersBuffer.length() > 
maximumFramesize) {
+                                               LOG.warn("The serialized 
counter metric is larger than the maximum frame size, " +
+                                                       " so maybe not all 
metrics would be reported.");
+                                               markUnserializedMetrics = true;
+                                               //clear all, because we can not 
revoke the latest metrics which caused overflow
+                                               
unserializedCounters.put(entry.getKey(), entry.getValue());
 
 Review comment:
   again, we're mixing concerns here. The serialization logic should only worry 
about faithfully serializing all metrics.
   
   We're just increasing complexity here for no real gain; the MQS is perfectly 
capable of determining whether the data fits, and we can add some basic utility 
functions to drop certain parts of a SerializationResult, which is functionally 
identical to the code here.

----------------------------------------------------------------
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.7.0
>
>
> 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)

Reply via email to