Hi ChangZhuo,
By low level process function you mean the Data Streams API, right?
How to disable the Kafka metrics when creating a Kafka source / sink is
described here:
https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/connectors/datastream/kafka/#additional-properties
The same pr
Nice work Peter! Looking forward to the fix.
@ChangZhou Kafka metrics are emitted from the source and the process
function would be a different operator. For the datastream API, you can set
`KafkaSourceOptions.REGISTER_KAFKA_CONSUMER_METRICS.key()` as `false` in
your consumer properties.
Best,
Ma
On Wed, May 04, 2022 at 01:53:01PM +0200, Chesnay Schepler wrote:
> Disabling the kafka metrics _should_ work.
Is there anyway to disable Kafka metrics when using low level process
function?
--
ChangZhuo Chen (陳昌倬) czchen@{czchen,debian}.org
http://czchen.info/
Key fingerprint = BA04 346D C2E1
Ah that's unfortunate. Yeah the feature freeze was quite a bit earlier
than I remembered :(
On 04/05/2022 15:31, Peter Schrott wrote:
Hi Chesnay,
Thanks again for the hints.
Unfortunately the metrics filtering feature is not part of 1.15.0. It
seems to be part of 1.16.0:
https://issues.apac
Hi Chesnay,
Thanks again for the hints.
Unfortunately the metrics filtering feature is not part of 1.15.0. It seems
to be part of 1.16.0: https://issues.apache.org/jira/browse/FLINK-21585
I was already wondering why I could not find the feature in the docs you
linked.
> Disabling the kafka metri
Disabling the kafka metrics _should_ work.
Alternatively you could use the new generic feature to filter metrics:
https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/deployment/metric_reporters/#filter-excludes
metrics.reporter..filter.excludes:
*KafkaProducer*;*KafkaConsumer*
Th
Allright! Thanks!
I tried to dig a bit deeper and see if there is any workaround for that
problem. I tried to switch off reporting the Kafka metrics, but I was not
quite successful. I am using the table api Kafka connector.
Do you have any suggestions on how to overcome this?
Could you also prov
https://issues.apache.org/jira/browse/FLINK-27487
On 04/05/2022 13:22, Chesnay Schepler wrote:
Yes, that looks like a new bug in 1.15.
The migration to the new non-deprecated Kafka API in the
KafkaMetricMutableWrapper was done incorrectly.
This should affect every job that uses the new kafka
Yes, that looks like a new bug in 1.15.
The migration to the new non-deprecated Kafka API in the
KafkaMetricMutableWrapper was done incorrectly.
This should affect every job that uses the new kafka connector.
Thank you for debugging the issue!
I will create a ticket.
On 04/05/2022 12:24, Pet
As the stracktrace says, class cast exception occurs here:
https://github.com/apache/flink/blob/master/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapper.java#L37
I found the following metrics to be affected
Sorry for the spamming!
Just after jumping into the debug-session I noticed that there are indeed
exceptions thrown when fetching the metrics on port 9200:
13657 INFO [ScalaTest-run] com.sun.net.httpserver - HttpServer
created http 0.0.0.0/0.0.0.0:9200
13658 INFO [ScalaTest-run] com.sun.net.ht
Hi Chesnay,
Thanks for that support! Just for compilation: Running the "Problem-Job"
locally as test in Intellij (as Chesney suggested above) reproduces the
described problem:
➜ ~ curl localhost:9200curl: (52) Empty reply from server
Doing the same with other jobs metrics are available on local
> I noticed that my config of the PrometheusReporter is different here.
I have: `metrics.reporter.prom.class:
org.apache.flink.metrics.prometheus.PrometheusReporter`. I will
investigate if this is a problem.
That's not a problem.
> Which trace logs are interesting?
The logging config I provi
Hi Chesnay,
Thanks for the code snipped. Which trace logs are interesting? Of "
org.apache.flink.metrics.prometheus.PrometheusReporter"?
I could also add this logger settings in the environment where the problem
is present.
Other than that, I am not sure how to reproduce this issue in a local
set
You'd help me out greatly if you could provide me with a sample job that
runs into the issue.
So far I wasn't able to reproduce the issue,
but it should be clear that there is some given 3 separate reports,
although it is strange that so far it was only reported for Prometheus.
If one of you is
On Tue, May 03, 2022 at 10:32:03AM +0200, Peter Schrott wrote:
> Hi!
>
> I also discovered problems with the PrometheusReporter on Flink 1.15.0,
> coming from 1.14.4. I already consulted the mailing list:
> https://lists.apache.org/thread/m8ohrfkrq1tqgq7lowr9p226z3yc0fgc
> I have not found the und
On Tue, May 03, 2022 at 10:28:18AM +0200, Chesnay Schepler wrote:
> Is there any warning in the logs containing "Error while handling metric"?
No, we don't find any "Error while handling metric"
--
ChangZhuo Chen (陳昌倬) czchen@{czchen,debian}.org
http://czchen.info/
Key fingerprint = BA04 346D C
Hi!
I also discovered problems with the PrometheusReporter on Flink 1.15.0,
coming from 1.14.4. I already consulted the mailing list:
https://lists.apache.org/thread/m8ohrfkrq1tqgq7lowr9p226z3yc0fgc
I have not found the underlying problem or a solution to it.
Actually, after re-checking, I see th
Is there any warning in the logs containing "Error while handling metric"?
On 03/05/2022 10:18, ChangZhuo Chen (陳昌倬) wrote:
On Tue, May 03, 2022 at 01:00:42AM -0700, Mason Chen wrote:
Hi ChangZhou,
The warning log indicates that the metric was previously defined and so the
runtime is handling
On Tue, May 03, 2022 at 01:00:42AM -0700, Mason Chen wrote:
> Hi ChangZhou,
>
> The warning log indicates that the metric was previously defined and so the
> runtime is handling the "duplicate" metric by ignoring it. This is
> typically a benign message unless you rely on this metric. Is it possib
Hi ChangZhou,
The warning log indicates that the metric was previously defined and so the
runtime is handling the "duplicate" metric by ignoring it. This is
typically a benign message unless you rely on this metric. Is it possible
that you are using the same task name for different tasks? It would
Hi,
We found that taskmanager Prometheus endpoint does not work after
upgrading from 1.14.3 to 1.15.0. Jobmanager Prometheus endpoint is okay
in 1.15.0, so we think the problem is not in image we used. Any idea how
to fix this problem?
Also, we found the following log in taskmanager, but not job
22 matches
Mail list logo