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

Theo Diefenthal commented on FLINK-12579:
-----------------------------------------

Prometheus can deal with Infinity/NaN values and the Prometheus Java Client 
(Which is used by Flink Prometheus reporter) deals with it like so:
{code:java}
Collector.java
public static String doubleToGoString(double d) {
  if (d == Double.POSITIVE_INFINITY) {
    return "+Inf";
  } 
  if (d == Double.NEGATIVE_INFINITY) {
    return "-Inf";
  }
  if (Double.isNaN(d)) {
    return "NaN";
  }
  return Double.toString(d);
{code}
I think, a similar catch to the metric values is required for the influx 
reporter, but as Influx doesn't support positive/negative inifinity or NaN, 
those metrics should just be ignored. (The behavior will remain the same as 
now, but the log won't be polluted with warnings)

See here for a tracking of InfluxDB to become Prometheus compliant by adding 
support for infinity and NaN (Which is still open)
https://github.com/influxdata/flux/issues/1040

> Influx Metric exporter throws a lot of negative infinity errors
> ---------------------------------------------------------------
>
>                 Key: FLINK-12579
>                 URL: https://issues.apache.org/jira/browse/FLINK-12579
>             Project: Flink
>          Issue Type: Bug
>            Reporter: Theo Diefenthal
>            Priority: Major
>
> When using the InfluxDB metrics, the logs are polluted by lots of warnings 
> due to negative infinity values which influxDB can't handle:
> {code:java}
> 2019-05-21 18:24:40,410 WARN 
> org.apache.flink.runtime.metrics.MetricRegistryImpl - Error while reporting 
> metrics
> org.apache.flink.metrics.influxdb.shaded.org.influxdb.InfluxDBException$UnableToParseException:
>  partial write: unable to parse 
> 'taskmanager_job_task_operator_sync-time-max,host=..,job_id=d7306cf5af0cf386a9259845d2a79f7c,job_name=..,operator_id=0be27db0efe2375e2766b48026cbee26,operator_name=Source:\
>  
> ...kafka_source...,subtask_index=0,task_attempt_id=bbff973b4e71d377745f0f2e3bf884ef,task_attempt_num=0,task_id=0be27db0efe2375e2766b48026cbee26,task_name=Source:\
>  ..._kafka_source_...,tm_id=container_e101_1557348638026_49850_01_000002 
> value=∞ 1558463080133000000': invalid number
> unable to parse 
> 'taskmanager_job_task_operator_KafkaConsumer_join-time-max,host=...,job_id=d7306cf5af0cf386a9259845d2a79f7c,job_name=...,operator_id=0be27db0efe2375e2766b48026cbee26,operator_name=Source:\
>  
> ...kafka_source...,subtask_index=0,task_attempt_id=bbff973b4e71d377745f0f2e3bf884ef,task_attempt_num=0,task_id=0be27db0efe2375e2766b48026cbee26,task_name=Source:\
>  ...kafka_source...,tm_id=container_e101_1557348638026_49850_01_000002 
> value=-∞ 1558463080133000000': invalid number
>  ....
>   unable to parse 
> 'taskmanager_job_task_operator_KafkaConsumer_heartbeat-response-time-max,host=...,job_id=d7306cf5af0cf386a9259845d2a79f7c,job_name=...,operator_id=0be27db0efe2375e2766b48026cbee26,operator_name=Source:\
>  
> ..kafka_source..,subtask_index=0,task_attempt_id=bbff973b4e71d377745f0f2e3bf884ef,task_attempt_num=0,task_id=0be27db0efe2375e2766b48026cbee26,task_name=Source:\
>  ..kafka_source...,tm_id=container_e101_1557348638026_49850_01_000002 
> value=-∞ 1558463080133000000': invalid number dropped=0
>  at 
> org.apache.flink.metrics.influxdb.shaded.org.influxdb.InfluxDBException.buildExceptionFromErrorMessage(InfluxDBException.java:147)
>  at 
> org.apache.flink.metrics.influxdb.shaded.org.influxdb.InfluxDBException.buildExceptionForErrorState(InfluxDBException.java:173)
>  at 
> org.apache.flink.metrics.influxdb.shaded.org.influxdb.impl.InfluxDBImpl.execute(InfluxDBImpl.java:796)
>  at 
> org.apache.flink.metrics.influxdb.shaded.org.influxdb.impl.InfluxDBImpl.write(InfluxDBImpl.java:455)
>  at 
> org.apache.flink.metrics.influxdb.InfluxdbReporter.report(InfluxdbReporter.java:97)
>  at 
> org.apache.flink.runtime.metrics.MetricRegistryImpl$ReporterTask.run(MetricRegistryImpl.java:430)
>  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>  at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
>  at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
>  at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at java.lang.Thread.run(Thread.java:748)
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to