This could be related to FLINK-12579 <https://issues.apache.org/jira/browse/FLINK-12579>, some kafka metrics use Infinity as a value which Influx doesn't support. For 1.10 we updated our influxdb-java dependency which automatically filters out these values, but we haven't released this yet.

We'll have to rely on a SNAPSHOT jar instead for testing this theory; can you try replacing the flink-metrics-influxdb jar in your distribution with this <https://repository.apache.org/content/repositories/snapshots/org/apache/flink/flink-metrics-influxdb_2.11/1.11-SNAPSHOT/flink-metrics-influxdb_2.11-1.11-20200130.041903-45.jar> one and try again?

On 06/02/2020 11:13, Morgan Geldenhuys wrote:

The only difference that i can tell is the Kubernetes cluster was upgraded from 1.14 to 1.17, however I rolled this back to test and the same result on the older version.

Ive created a stripped down Job which produces the errors:

// create a testing sink private static class CollectSinkimplements 
SinkFunction<String> {

     // must be static public static final List<String>values =new 
ArrayList<>();

     @Override public synchronized void invoke(String value)throws Exception {
         values.add(value);
     }
}

public static void main(String[] args)throws Exception {

     Properties props = Resources.GET.read("iot_traffic_processor.properties", 
Properties.class);

     // setup Kafka consumer Properties kafkaConsumerProps =new Properties();

     kafkaConsumerProps.setProperty("bootstrap.servers", 
props.getProperty("kafka.brokerList"));kafkaConsumerProps.setProperty("group.id", 
props.getProperty("kafka.group"));FlinkKafkaConsumer<String> myConsumer =
             new FlinkKafkaConsumer<>(
                     props.getProperty("kafka.consumerTopic"),
                     new SimpleStringSchema(),
                     kafkaConsumerProps);


     // set up streaming execution environment StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
     env.getConfig().setAutoWatermarkInterval(1000L);

     // create direct kafka stream DataStream<String> trafficEventStream = 
env.addSource(myConsumer);

     DataStream<String> mapStream =
         trafficEventStream
             .map(new RichMapFunction<String, String>() {
         @Override public String map(String value)throws Exception {
             return value;
         }
     });

     mapStream.addSink(new CollectSink());

     env.execute("Traffic");

Dependencies have also been stripped down to a minimum:

<!-- flink dependencies --> <dependency>
     <groupId>org.apache.flink</groupId>
     <artifactId>flink-java</artifactId>
     <version>1.9.0</version>
     <scope>provided</scope>
</dependency>
<dependency>
     <groupId>org.apache.flink</groupId>
     <artifactId>flink-streaming-scala_2.12</artifactId>
     <version>1.9.0</version>
     <scope>provided</scope>
</dependency>
<dependency>
     <groupId>org.apache.flink</groupId>
     <artifactId>flink-streaming-java_2.12</artifactId>
     <version>1.9.0</version>
     <scope>provided</scope>
</dependency>
<dependency>
     <groupId>org.apache.flink</groupId>
     <artifactId>flink-table-uber_2.12</artifactId>
     <version>1.9.0</version>
     <scope>provided</scope>
</dependency>
<dependency>
     <groupId>org.apache.flink</groupId>
     <artifactId>flink-connector-kafka_2.12</artifactId>
     <version>1.9.0</version>
</dependency>

<!-- general dependencies --> <dependency>
     <groupId>org.apache.commons</groupId>
     <artifactId>commons-lang3</artifactId>
     <version>3.9</version>
</dependency>
<dependency>
     <groupId>log4j</groupId>
     <artifactId>log4j</artifactId>
     <version>1.2.17</version>
</dependency>
<dependency>
     <groupId>junit</groupId>
     <artifactId>junit</artifactId>
     <version>4.12</version>
</dependency>


On 06.02.20 10:58, Chesnay Schepler wrote:
Setup-wise, are there any differences to what you had a few months ago?

On 06/02/2020 10:40, Morgan Geldenhuys wrote:

Further info, the flink cluster (1.9) is running on Kubernetes (1.17) with InfluxDB.

I have tried the following images for InfluxDB: docker.io/influxdb:1.6.4 and influxdb:latest

When going into the database and showing the series, there are really weird results:

> show series
key
---
jobmanager_Status_JVM_CPU_Load,host=flink-jobmanager
jobmanager_Status_JVM_CPU_Time,host=flink-jobmanager
jobmanager_Status_JVM_ClassLoader_ClassesLoaded,host=flink-jobmanager
jobmanager_Status_JVM_ClassLoader_ClassesUnloaded,host=flink-jobmanager
jobmanager_Status_JVM_GarbageCollector_Copy_Count,host=flink-jobmanager
jobmanager_Status_JVM_GarbageCollector_Copy_Time,host=flink-jobmanager
jobmanager_Status_JVM_GarbageCollector_MarkSweepCompact_Count,host=flink-jobmanager
jobmanager_Status_JVM_GarbageCollector_MarkSweepCompact_Time,host=flink-jobmanager
jobmanager_Status_JVM_Memory_Direct_Count,host=flink-jobmanager
jobmanager_Status_JVM_Memory_Direct_MemoryUsed,host=flink-jobmanager
jobmanager_Status_JVM_Memory_Direct_TotalCapacity,host=flink-jobmanager
jobmanager_Status_JVM_Memory_Heap_Committed,host=flink-jobmanager
jobmanager_Status_JVM_Memory_Heap_Max,host=flink-jobmanager
jobmanager_Status_JVM_Memory_Heap_Used,host=flink-jobmanager
jobmanager_Status_JVM_Memory_Mapped_Count,host=flink-jobmanager
jobmanager_Status_JVM_Memory_Mapped_MemoryUsed,host=flink-jobmanager
jobmanager_Status_JVM_Memory_Mapped_TotalCapacity,host=flink-jobmanager
jobmanager_Status_JVM_Memory_NonHeap_Committed,host=flink-jobmanager
jobmanager_Status_JVM_Memory_NonHeap_Max,host=flink-jobmanager
jobmanager_Status_JVM_Memory_NonHeap_Used,host=flink-jobmanager
jobmanager_Status_JVM_Threads_Count,host=flink-jobmanager
jobmanager_numRegisteredTaskManagers,host=flink-jobmanager
jobmanager_numRunningJobs,host=flink-jobmanager
jobmanager_taskSlotsAvailable,host=flink-jobmanager
jobmanager_taskSlotsTotal,host=flink-jobmanager
taskmanager_Status_JVM_CPU_Load,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_CPU_Time,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_ClassLoader_ClassesLoaded,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_ClassLoader_ClassesUnloaded,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_GarbageCollector_G1_Old_Generation_Count,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_GarbageCollector_G1_Old_Generation_Time,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_GarbageCollector_G1_Young_Generation_Count,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_GarbageCollector_G1_Young_Generation_Time,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_Memory_Direct_Count,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_Memory_Direct_MemoryUsed,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_Memory_Direct_TotalCapacity,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_Memory_Heap_Committed,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_Memory_Heap_Max,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_Memory_Heap_Used,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_Memory_Mapped_Count,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_Memory_Mapped_MemoryUsed,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_Memory_Mapped_TotalCapacity,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_Memory_NonHeap_Committed,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_Memory_NonHeap_Max,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_Memory_NonHeap_Used,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_JVM_Threads_Count,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_Network_AvailableMemorySegments,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_Network_TotalMemorySegments,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_Shuffle_Netty_AvailableMemorySegments,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717
taskmanager_Status_Shuffle_Netty_TotalMemorySegments,host=flink-taskmanager-6484bdf6c5-6v5q2,tm_id=4f62a2c1108e00ca0fdbcbb49f068717

As the job continues this list of series just gets longer and longer.

It was working perfectly fine a few months ago, but no idea what is happening now. Any ideas?

On 06.02.20 10:23, Chesnay Schepler wrote:
What InfluxDB version are you using?

On 05/02/2020 19:41, Morgan Geldenhuys wrote:

I am trying to setup metrics reporting for Flink using InflixDB, however I am receiving tons of exceptions (listed right at the bottom).

Reporting is setup as recommended by the documentation:
metrics.reporter.influxdb.class: 
org.apache.flink.metrics.influxdb.InfluxdbReporter
metrics.reporter.influxdb.host: influxdb
metrics.reporter.influxdb.port: 8086
metrics.reporter.influxdb.db: flink
metrics.reporter.influxdb.username: ***
metrics.reporter.influxdb.password: ***
Any hints at what would cause all these exceptions?

2020-02-0518:15:17,777WARNorg.apache.flink.runtime.metrics.MetricRegistryImpl- Errorwhilereporting metrics org.apache.flink.metrics.influxdb.shaded.org.influxdb.InfluxDBException$UnableToParseException: partial write: unable to parse 'taskmanager_job_task_operator_partition-revoked-latency-max,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_KafkaConsumer_heartbeat-response-time-max,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_KafkaConsumer_sync-time-max,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_KafkaConsumer_reauthentication-latency-avg,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_rebalance-latency-avg,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_reauthentication-latency-avg,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_KafkaConsumer_reauthentication-latency-max,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_partition-revoked-latency-avg,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_KafkaConsumer_join-time-avg,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_KafkaConsumer_rebalance-latency-avg,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_partition-assigned-latency-max,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_sync-time-avg,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_partition-assigned-latency-avg,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_KafkaConsumer_sync-time-avg,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_KafkaConsumer_partition-lost-latency-avg,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_rebalance-latency-max,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_reauthentication-latency-max,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_heartbeat-response-time-max,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_KafkaConsumer_rebalance-latency-max,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_KafkaConsumer_join-time-max,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_join-time-avg,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_KafkaConsumer_partition-revoked-latency-avg,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_partition-lost-latency-max,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_KafkaConsumer_partition-lost-latency-max,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_sync-time-max,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_join-time-max,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_partition-lost-latency-avg,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_KafkaConsumer_partition-assigned-latency-max,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_KafkaConsumer_partition-revoked-latency-max,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid boolean unable to parse 'taskmanager_job_task_operator_KafkaConsumer_partition-assigned-latency-avg,host=flink-taskmanager-6484bdf6c5-sctng,job_id=6fe7d657f2a2f989115752d7e1a5cf5b,job_name=Traffic,operator_id=cbc357ccb763df2852fee8c4fc7d55f2,operator_name=Source:\ Custom\ Source,subtask_index=0,task_attempt_id=2087c37093ea7c1d6896caf6e3768b88,task_attempt_num=0,task_id=cbc357ccb763df2852fee8c4fc7d55f2,task_name=Source:\ Custom\ Source\ ->\ Filter,tm_id=7adf9ad0bf8edc167cfafbe9bc22f2f2 value=� 1580926517720000000': invalid booleandropped=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:101)     at org.apache.flink.runtime.metrics.MetricRegistryImpl$ReporterTask.run(MetricRegistryImpl.java:436)     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)






Reply via email to