Hi Romand and Hongshun,

> 1. The scope of the variables added - is it only the last transformation?
> Do I understand correctly, that chaining does NOT affect this scoping?

> Could you please give a more detailed example how to do it? Currently, the
> example only shows source.

Setting the scope will work exactly the same way as setting `uid`, `name`,
`parallelism` or `maxParallelism`. So it will be applied only to the last
defined transformation.
For example:

        DataStream<Tuple2<String, Integer>> dataStream = env
                .socketTextStream("localhost", 9999)
                .addMetricVariable("foo1", "bar1")
                .flatMap(new Splitter())
                .addMetricVariable("foo2", "bar2")
                .keyBy(value -> value.f0)
                .window(TumblingProcessingTimeWindows.of(Time.seconds(5)))
                .sum(1)
                .addMetricVariable("foo3", "bar3");

Chaining doesn't affect additional metric variables.

> 2. Is Python API going to be supported as well?

Unfortunately not initially.

Best,
Piotrek

wt., 3 gru 2024 o 03:27 Hongshun Wang <loserwang1...@gmail.com> napisał(a):

> Hi Piotr,
>   thanks for the proposal.
>
> >  it would be helpful to group metrics from each source/sink instance
> together,
> Could you please give a more detailed example how to do it? Currently, the
> example only shows source.
>
> Thanks,
> Hongshun
>
> On Thu, Nov 14, 2024 at 8:36 PM Roman Khachatryan <ro...@apache.org>
> wrote:
>
> > Hi Piotr, thanks for the proposal,
> >
> > Can you please clarify
> > 1. The scope of the variables added - is it only the last transformation?
> > Do I understand correctly, that chaining does NOT affect this scoping?
> >
> > 2. Is Python API going to be supported as well?
> >
> > Thanks
> >
> > Regards,
> > Roman
> >
> >
> > On Thu, Nov 7, 2024 at 2:52 PM Piotr Nowojski <pnowoj...@apache.org>
> > wrote:
> >
> > > Hi all!
> > >
> > > I would like to open up for discussion a new FLIP-484 [1].
> > >
> > > Motivation
> > > When running a Flink job that reads data from multiple sources and
> writes
> > > to multiple sinks, it would be helpful to group metrics from each
> > > source/sink instance together, for example based on the underlying name
> > of
> > > the source/sink table/topic.
> > >
> > > Proposed Changes
> > > This flip proposes to allow users to set custom metric variables to
> > > operators/transformations, that will be later passed on to the metrics
> > and
> > > trace reporters.
> > >
> > > This will allow users to label all of the reported metrics from
> operators
> > > with for example the table name (SQL), allowing to easily report the
> > number
> > > of records/watermark values per each source/sink table in the same
> Flink
> > > Job.
> > >
> > > For more information please look into the FLIP-484 [1].
> > >
> > > I'm looking forward to your thoughts on this.
> > >
> > > Best,
> > > Piotrek
> > >
> > > [1] https://cwiki.apache.org/confluence/x/44yMEw
> > > <https://cwiki.apache.org/confluence/x/4IyMEw>
> > >
> >
>

Reply via email to