Thank you for voting. Voting is closed.

wt., 10 gru 2024 o 14:34 Piotr Nowojski <pnowoj...@apache.org> napisał(a):

> > It sounds like we are using this mechanism to cover all three cases. If
> there is an appetite to do the first 2 in a more
> > metadata driven way then I guess they would appear in the main scopes as
> User Variables.
>
> Yes, this FLIP will be re-using the pre-existing mechanism, so for
> example, no changes to metric reporters will be needed.
> Those custom metric variables, that I'm proposing in this FLIP, will be
> attached to and returned from the pre-existing
> `org.apache.flink.metrics.MetricGroup#getAllVariables` method.  That will
> be handled/done inside
> `org.apache.flink.runtime.metrics.groups.InternalOperatorMetricGroup`
>
> > I am curious, does this effect all scopes, for example would the
> JobManager /connectors / IO metrics?
>
> Additional metric variables defined on the metric reporter level are
> attached to all metrics reported by this metric reporter.
> That includes JobManager, connector and IO Metrics as well.
>
> Additional metric variables defined on the operator level, as proposed in
> this FLIP, would be only applicable to that
> operator's metrics.
>
> > Are these like the User variables defined for the connectors.
>
> In principle those custom metric variables can be used for any operator,
> not only for the sources/sinks.
>
> >  What happen if there is a name clash with the user variable?
>
> Hmm, good question, I would just overwrite one with the other and maybe
> print a warning message.
>
> Best,
> Piotrek
>
> pon., 9 gru 2024 o 10:28 David Radley <david_rad...@uk.ibm.com>
> napisał(a):
>
>> Hi Piotr,
>> Thanks your you quick response and for the extra details.
>>
>> I am thinking that if this were use case driven then
>>
>>   *   there would be some use cases like the table name that could be
>> derived from existing metadata, either names, tags<
>> https://egeria-project.org/concepts/informal-tag/> or classifications<
>> https://egeria-project.org/types/4/0421-Governance-Classification-Levels/
>> >.
>>   *   others like region that sound like reference data (other tags that
>> are constant for the whole Flink Cluster) – i.e. an enumeration of strings
>> , country codes , region names etc that would be useful and be consistent
>> outside of metrics.
>>   *   other freeform text, to make an entry more meaningful to humans,
>> like comments descriptions, short names.
>>
>>
>>
>> It sounds like we are using this mechanism to cover all three cases. If
>> there is an appetite to do the first 2 in a more metadata driven way then I
>> guess they would appear in the main scopes as User Variables.
>>
>>
>>
>> I am curious, does this effect all scopes, for example would the
>> JobManager /connectors / IO metrics?
>>
>>
>>
>> Are these like the User variables defined for the connectors. What happen
>> if there is a name clash with the user variable?
>>
>>
>>
>> I am in favour of this Flip , but would like the above clarifications
>> please,
>>
>>
>>
>>     Kind regards, David.
>>
>>
>>
>> From: Piotr Nowojski <pnowoj...@apache.org>
>> Date: Friday, 6 December 2024 at 16:18
>> To: dev@flink.apache.org <dev@flink.apache.org>
>> Subject: [EXTERNAL] Re: [VOTE] FLIP-484: Add custom metric variables to
>> operators
>> Hi David,
>>
>> no worries and thanks for the feedback!
>>
>> While I agree that the "scope" feels more intuitive, and I think in the
>> first version that's the name I used.
>> However the issue is that there is already a pre-existing naming
>> convention
>> in Flink, where "variable" [1]
>> and "additional variables" [2] names are already used for the exact thing
>> that I'm adding here. While
>> "scope" [3] is used for a higher level concept.
>>
>> Scope is for example "Task" in the "Task.numRecordsIn" metric or
>> "TaskManager.Status.JVM.CPU" in
>> the "TaskManager.Status.JVM.CPU.Load" metric.
>>
>> Metric variables are those extra tags attached to the reported metric.
>> Like
>> `<job_id>`, `<task_name>`.
>> "additional variables" that user can configure in the config for each
>> metric/trace reporter, is
>> commonly used to add variables for things like cloud region (us_west,
>> central_europe etc) or some
>> other tags that are constant for the whole Flink Cluster.
>>
>> > If the main use case for this is to put the table name as the metric
>> scope – could we not do this
>> > automatically in the code without any additional need to add code?
>>
>> Maybe that could be some valuable follow up work. Especially since I'm not
>> very familiar with those
>> areas of the code myself. And AFAIK it's very common for users to use
>> custom table registries/meta
>> stores.
>>
>> Anyway there are also other good potential use cases, also in the
>> DataStream API, so either way
>> I think it would be good to start with this FLIP
>>
>> Best,
>> Piotrek
>>
>> [1]
>>
>> https://nightlies.apache.org/flink/flink-docs-master/docs/ops/metrics/#system-scope
>> [2]
>>
>> https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/metric_reporters/#scope-variables-additional
>> [3]
>>
>> https://nightlies.apache.org/flink/flink-docs-master/docs/ops/metrics/#scope
>>
>>
>> pt., 6 gru 2024 o 12:34 David Radley <david_rad...@uk.ibm.com>
>> napisał(a):
>>
>> > Hi,
>> > Sorry I am late to this. I have some comments.
>> >
>> > If I am understanding this correctly, the custom metric is not a new
>> > metric that has been customized, it is the ability to add readable
>> useful
>> > names / tags as scopes to existing metrics. If this is the case I
>> suggest
>> > changing the method name from “ .addMetricVariable(” to something like
>> > “.addMetricScope(“ and docs should refer to metric scope / tag. The name
>> > metric variable, to me, implies that it is a variable that is measuring
>> > something.
>> >
>> > I assume these variables would be most useful if they were unique; is
>> > there a way to make enforce uniqueness.
>> >
>> > From your example, I assume all the “table_name” could be any literal.
>> If
>> > the main use case for this is to put the table name as the metric scope
>> –
>> > could we not do this automatically in the code without any additional
>> need
>> > to add code?
>> >
>> > Kind regards, David.
>> >
>> >
>> > From: Yuepeng Pan <panyuep...@apache.org>
>> > Date: Friday, 6 December 2024 at 10:27
>> > To: dev@flink.apache.org <dev@flink.apache.org>
>> > Subject: [EXTERNAL] Re: [VOTE] FLIP-484: Add custom metric variables to
>> > operators
>> > +1 (non-binding)
>> >
>> > Thanks Piotr for driving it !
>> >
>> > Best,
>> > Yuepeng Pan
>> >
>> >
>> > On 2024/12/06 09:19:46 Piotr Nowojski wrote:
>> > > Hi all!
>> > >
>> > > I would like to open the vote for FLIP-484 [1]. It has been discussed
>> > here
>> > > [2].
>> > >
>> > > The vote will remain open for at least 72 hours (excluding the
>> weekend).
>> > >
>> > > Best,
>> > > Piotrek
>> > >
>> > > [1] https://cwiki.apache.org/confluence/x/44yMEw
>> > > [2] https://lists.apache.org/thread/1g1n2l8bc27zvw88ljw4l2gqzx6rg5zy
>> > >
>> >
>> > Unless otherwise stated above:
>> >
>> > IBM United Kingdom Limited
>> > Registered in England and Wales with number 741598
>> > Registered office: Building C, IBM Hursley Office, Hursley Park Road,
>> > Winchester, Hampshire SO21 2JN
>> >
>>
>> Unless otherwise stated above:
>>
>> IBM United Kingdom Limited
>> Registered in England and Wales with number 741598
>> Registered office: Building C, IBM Hursley Office, Hursley Park Road,
>> Winchester, Hampshire SO21 2JN
>>
>

Reply via email to