Hi Yu,

   Thanks for your suggestion. We are currently using a flink runner web
submission for running the beam program. When we tried to execute the beam
metrics like this, we faced an issue that custom metrics (querymetrics and
metricspusher) are not supported by detached mode flink runner ( apparently
all the flink web submissions are only in detached mode by default now)
with the following error :


Could not execute application. at org.apache.flink.client.
deployment.application.DetachedApplicationRunner.tryExecuteJobs(
DetachedApplicationRunner.java:81) at org.apache.flink.client.
deployment.application.DetachedApplicationRunner.run(
DetachedApplicationRunner.java:67) at org.apache.flink.runtime.
webmonitor.handlers.JarRunHandler.lambda$handleRequest$0(JarRunHandler.java:100)
at java.util.concurrent.CompletableFuture$AsyncSupply.
run(CompletableFuture.java:1604) ... 7 more Caused by:
org.apache.flink.client.program.ProgramInvocationException: The main method
caused an error: The FlinkRunner does not currently support metrics. at
org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:330)
at org.apache.flink.client.program.PackagedProgram.
invokeInteractiveModeForExecution(PackagedProgram.java:198) at
org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:114) at
org.apache.flink.client.deployment.application.DetachedApplicationRunner.
tryExecuteJobs(DetachedApplicationRunner.java:78) ... 10 more Caused by:
java.lang.UnsupportedOperationException: The FlinkRunner does not currently
support metrics. at org.apache.beam.runners.flink.FlinkDetachedRunnerResult.
metrics(FlinkDetachedRunnerResult.java:40)

 On further analysis, we also found certain discussions supporting the
same. Below are the links for that.

https://github.com/apache/beam/pull/4548

https://ask.csdn.net/questions/7180575

 Is there any workaround for this so that we could successfully export the
beam custom metrics to Prometheus through the metrics reporter which is
running through a flink web submission (flink in detached mode)

Thanks and regards
Feba Fathima


On Thu, 29 Apr, 2021, 10:59 pm Yu Zhang, <[email protected]> wrote:

> Hi Feba,
>
> I did a POC based on Beam’s doc 10.5 [1] and Flink metrics reporter [2].
> You can add metrics reporter configuration in flink-conf.yaml so that
> metrics can be scraped from port you configured. Be sure to set
> disable_metrics [3] to false.
>
>
> [1] https://beam.apache.org/documentation/programming-guide/#metrics
> [2]
> https://ci.apache.org/projects/flink/flink-docs-stable/deployment/metric_reporters.html#prometheus-orgapacheflinkmetricsprometheusprometheusreporter
> [3] https://beam.apache.org/documentation/runners/flink/
>
> Thanks,
> Yu
>
> On Apr 28, 2021, at 22:58, Feba Fathima <[email protected]> wrote:
>
> Hi,
>
>  Thanks Yu. Can you please let me know how you are displaying the counter
> metrics in Prometheus. Are you using any metrics sink to push the metrics
> from beam and is there any kind of mechanism to pull the metrics into
> Prometheus?
>
> Thanks in advance.
>
> Thanks & Regards,
> Feba Fathima
>
> On Wed, 28 Apr, 2021, 11:28 pm Yu Zhang, <[email protected]> wrote:
>
>> Hi,
>>
>> Having similar issues. I tried with Beam 2.27.0, Flink 1.12.1 and
>> Prometheus 2.25.0. The distribution and gauge metrics are always 0 in both
>> Flink UI and Prometheus. Counter can be displayed correctly. This should be
>> related with issue 10928.
>>
>> Thanks,
>> Yu
>>
>>
>> On 2021/04/28 17:39:29, Alexey Romanenko <[email protected]>
>> wrote:
>> > Hi,
>> >
>> > Could it be related [1][2]?
>> >
>> > [1] https://issues.apache.org/jira/browse/BEAM-7438
>> > [2] https://issues.apache.org/jira/browse/BEAM-10928
>> >
>> > —
>> > Alexey
>> >
>> > > On 28 Apr 2021, at 08:45, Feba Fathima <[email protected]>
>> wrote:
>> > >
>> > > Hi,
>> > >
>> > >    I have a java beam pipeline which is run using a flink runner. I
>> tried adding custom metrics to it which I want to export to Prometheus. I
>> would like to know how to export the beam custom metrics to Prometheus.
>> Please help me with this.
>> > >
>> > > Thanks in advance
>> > > Feba Fathima
>> >
>> >
>>
>
>

Reply via email to