Hi Mathias and Andrew,

Thanks for taking the time to read and comment on the KIP.  I'll address
each comment below:

MS1 - I agree that logging a WARN here should be sufficient.  I'll update
the interface in KIP to the original `void` return type.
AS1 - I was considering throwing an exception, but you make a compelling
argument about the expected behavior and external dynamic factors.
So as stated above, the KIP will specify a `void` return type.

MS2 - Good point. I'll update the KIP with a brief statement on supported
types and a table of Kafka Streams metrics displaying what's supported and
which ones aren't.
AS2 - I think the above addresses your second comment.

MS3 - We could explore adding new metrics, but I wouldn't want that to
block the progress of this KIP.
So I'd prefer to defer the discussion of new metrics to a follow-on KIP.
Having said that, if you (or anyone else!) has an idea of a specific metric
to add, we can consider it now.

MS4 - Unsupported metrics are silently filtered out.  Although it's an
implementation detail, I propose to log a WARN statement for each filtered
metric passed to `registerMetricsForSubscription`.
I also update the KIP with the behavior of unsupported metric types.

MS5 - Setting `enable.metric.push = false` completely disables the
telemetry pipeline and any provided Kafka Streams metrics are ignored (I
have this in the KIP).
Although this is implementation detail, this is a valid condition to throw
a `ConfigException` on Kafka Streams startup should this inconsistency in
telemetry configs exist (at least in the main and admin consumer).
I'll update the KIP to be more clear on this point as well.

Thanks,
Bill

On Thu, Aug 22, 2024 at 10:54 AM Andrew Schofield <andrew_schofi...@live.com>
wrote:

> Hi Bill (and Matthias),
> Thanks for the KIP. This looks like a valuable extension to KIP-714.
>
> AS1: Personally, I think that registerMetricsForSubscription should return
> void and not throw an exception if metrics push is not enabled. Otherwise,
> you end up with an application whose behaviour is markedly different
> depending
> on external factors. For example, if you connect to a broker without a
> plugin
> that supports the ClientTelemetry interface, no metrics are going to be
> pushed.
> Similarly, if the enable.metrics.push is false, no metrics are going to be
> pushed.
> Neither of those seem to me to have implications on whether this method
> executes successfully. Even if everything is enabled, the metrics will
> only flow
> if there’s a telemetry subscription which matches the metric names. Those
> are
> dynamic things which are intended to be modified by operators seeking to
> diagnose problems. We certainly don’t want the
> registerMetricsForSubscription
> method to behave differently depending on the current state of its
> telemetry
> subscriptions.
>
> AS2: Good catch from Matthias for metrics whose type is incompatible
> with OTLP. We need to define the behaviour here.
>
> Thanks,
> Andrew
>
>
> > On 22 Aug 2024, at 05:35, Matthias J. Sax <mj...@apache.org> wrote:
> >
> > Thanks for the KIP Bill.
> >
> > Just for brainstorming: the newly added methods have `boolean` return
> type. Is this necessary or would logging a WARN be sufficient? Or maybe
> throw an exception if reporting is disabled?
> >
> > As we intend to report KS metrics, I am wondering if we should talk
> about this in some more detail? As discussed offline, KIP-714 is based on
> open-telemetry and supports only certain metric types, and not all KS
> metrics can be reported.
> >
> > Should we maybe even add new metrics to KS to close this gap (could also
> be a follow up KIP).
> >
> > What happens if a registered metric is of an unsupported type?
> >
> > What happens if users set KS level `enable.metric.push = true` but a
> client level config `enable.maetric.push = false`? Should we throw a
> ConfigException or similar for this case (at least for admin and
> main.consumer -- as we don't intent to use the producer, it would not
> matter if producer metric push is enabled or not). To be fair, this might
> be more of an impl detail? Just curious.
> >
> >
> > -Matthias
> >
> >
> > On 8/21/24 1:03 PM, Apoorv Mittal wrote:
> >> Hi Bill,
> >> Thanks for addressing, looks good to me. Looking forward to
> implementation.
> >> Regards,
> >> Apoorv Mittal
> >> On Wed, Aug 21, 2024 at 8:14 PM Bill Bejeck <bbej...@gmail.com> wrote:
> >>> Hi Apoorv,
> >>>
> >>> Good call, I've updated the compatibility section with text about using
> >>> older brokers.
> >>>
> >>> -Bill
> >>>
> >>> On Wed, Aug 21, 2024 at 1:30 PM Apoorv Mittal <
> apoorvmitta...@gmail.com>
> >>> wrote:
> >>>
> >>>> Hi Bill,
> >>>> Thanks for the details. Sounds good to me.
> >>>>
> >>>> One last question:
> >>>>
> >>>> AM5: Do we need to capture any details around compatibility with older
> >>>> brokers which don't support KIP-714 or when telemetry APIs are not
> >>> enabled
> >>>> on brokers (no configured receiver plugin)?
> >>>>
> >>>> Regards,
> >>>> Apoorv Mittal
> >>>>
> >>>>
> >>>> On Tue, Aug 20, 2024 at 10:22 PM Bill Bejeck <bbej...@apache.org>
> wrote:
> >>>>
> >>>>> Hi Apoorv,
> >>>>>
> >>>>> Thank you for the discussion! Here are my responses to your follow-up
> >>>>> questions:
> >>>>>
> >>>>> AM2: My apologies for the previous response; I misunderstood the
> >>>> question.
> >>>>> When executing the `registerMetricsForSubscription,`  it is up to the
> >>>>> developer to provide all metrics that could possibly be in a
> >>> subscription
> >>>>> request. For example, Kafka Streams will provide all stream metrics,
> >>> thus
> >>>>> supporting any subscription changes at runtime. I'll clarify this in
> >>> the
> >>>>> KIP.
> >>>>>
> >>>>> AM3: I understand your point, but I'm not convinced this will be an
> >>> issue
> >>>>> right now.   But given it is an implementation detail, I'd like to
> >>> defer
> >>>>> addressing this issue in the PR phase, where we can thoroughly test
> and
> >>>>> adjust accordingly.  Is that acceptable for you?
> >>>>>
> >>>>> Thanks,
> >>>>> Bill
> >>>>>
> >>>>>
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Tue, Aug 20, 2024 at 3:34 PM Apoorv Mittal <
> >>> apoorvmitta...@gmail.com>
> >>>>> wrote:
> >>>>>
> >>>>>> Hi Bill,
> >>>>>> Thanks for the answers, some follow ups.
> >>>>>>
> >>>>>> AM2: If we will only register subscribed metrics, as per the
> >>>> subscription
> >>>>>> received from the broker, then are we going to call the
> >>>>>> `registerMetricsForSubscription` post to GetTelemetrySubscription
> >>>>> response
> >>>>>> from the broker? Also how do we manage the dynamic update of
> >>> subscribed
> >>>>>> metrics with this approach i.e. subscribed metrics can change at
> >>>> runtime
> >>>>>> then are we going to re-register additional metrics from external
> >>>>>> applications?
> >>>>>>
> >>>>>> AM3: Sounds good. I understand that Streams JMXReporter will emit
> >>>> Streams
> >>>>>> metrics and individual client JMXReporter will emit respective
> client
> >>>>>> metrics. But currently in the clients, all the metrics in mbean are
> >>>>>> evaluated by JMXReporter and KafkaMetricsCollector, though it's an
> >>>>>> implementation detail but I am not sure if we register external
> >>>>> application
> >>>>>> metrics in client, by registerMetricsForSubscription, then how we
> >>> will
> >>>>>> differentiate with external application metrics while emitting
> >>> through
> >>>>>> JMXReporter of clients.
> >>>>>>
> >>>>>> Regards,
> >>>>>> Apoorv Mittal
> >>>>>>
> >>>>>>
> >>>>>> On Tue, Aug 20, 2024 at 8:09 PM Bill Bejeck <bbej...@apache.org>
> >>>> wrote:
> >>>>>>
> >>>>>>> Hi Apoorv,
> >>>>>>>
> >>>>>>> Thanks for reading the KIP and commenting.  Here are my answers to
> >>>> your
> >>>>>>> comments/questions:
> >>>>>>>
> >>>>>>> AM1: Good point; I'll update the Javadoc in the KIP to be more
> >>>>> specific.
> >>>>>>>
> >>>>>>> AM2: I'm not sure I follow, but IIUC, then yes, this method will
> >>> only
> >>>>>>> subscribe telemetry metrics.
> >>>>>>>
> >>>>>>> AM3: The `registerMetricsForSubscription` method only subscribes
> >>>>> metrics
> >>>>>>> for telemetry reporting; it does not give them to the JMX reporter.
> >>>>> The
> >>>>>>> application implementer is responsible for providing metrics to any
> >>>>> other
> >>>>>>> reporter.  For example, when starting an application, Kafka Streams
> >>>>>>> separately exposes all its metrics to a JMX reporter.
> >>>>>>>
> >>>>>>> AM4: Great question.  I'll assert we still need the
> >>>>> `enable.metrics.push`
> >>>>>>> configuration to toggle pushing Kafka Streams telemetry metrics.
> >>> To
> >>>>>> change
> >>>>>>> the metrics of the embedded Kafka clients within Kafka Streams,
> >>> users
> >>>>>> will
> >>>>>>> use the current mechanism to update client configuration settings.
> >>>>> I'll
> >>>>>>> update the KIP to clarify these details.
> >>>>>>>
> >>>>>>> Thanks,
> >>>>>>> Bill
> >>>>>>>
> >>>>>>> On Tue, Aug 20, 2024 at 6:30 AM Apoorv Mittal <
> >>>>> apoorvmitta...@gmail.com>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>>> Hi Bill,
> >>>>>>>> Thanks for the KIP and it's great to see a way applications can
> >>>> also
> >>>>>> send
> >>>>>>>> telemetry metrics over the KIP-714 pipeline.
> >>>>>>>>
> >>>>>>>> A couple of questions:
> >>>>>>>> AM1: The Javadoc for the method seems to be a bit vague, it talks
> >>>>> about
> >>>>>>>> "subscription" but do we need to be a bit more detailed regarding
> >>>>> which
> >>>>>>>> subscription we refer to?
> >>>>>>>>
> >>>>>>>> AM2: Are we going to only register KIP-714 subscribed metrics
> >>> with
> >>>>> this
> >>>>>>>> method?
> >>>>>>>>
> >>>>>>>> AM3: Are the additional registered metrics available on JMX
> >>>> reporter
> >>>>> as
> >>>>>>>> well or are we going to only register additional metrics for
> >>>> KIP-714
> >>>>>>>> telemetry pipeline?
> >>>>>>>>
> >>>>>>>> AM4: StreamsConfig also defines `enable.metrics.push`, do we need
> >>>>> that
> >>>>>>>> config or rely on client metric push config? Or all the clients
> >>>> will
> >>>>>>>> inherit the `enable.metrics.push` defined for Streams?
> >>>>>>>>
> >>>>>>>> Regards,
> >>>>>>>> Apoorv Mittal
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Tue, Aug 20, 2024 at 10:21 AM Bruno Cadonna <
> >>> cado...@apache.org
> >>>>>
> >>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Hi Bill,
> >>>>>>>>>
> >>>>>>>>> Thanks for the KIP!
> >>>>>>>>>
> >>>>>>>>> BC1
> >>>>>>>>> I find the name registerAdditionalMetrics() not specific
> >>> enough.
> >>>> It
> >>>>>> is
> >>>>>>>>> not clear that the passed-in metrics are made available for
> >>>>>>>>> subscription. What about registerMetricsForSubscription()?
> >>>>>>>>> The Additional part is confusing in my opinion: additional to
> >>>> what?
> >>>>>>>>>
> >>>>>>>>> BC2
> >>>>>>>>> Does the KIP need a transformation rule from MetricName to
> >>>>>>>>> OpenTelemetry-compatible names that are exposed at the broker
> >>> for
> >>>>>>>>> subscription and retrieval?
> >>>>>>>>>
> >>>>>>>>> Best,
> >>>>>>>>> Bruno
> >>>>>>>>>
> >>>>>>>>> On 8/20/24 12:43 AM, Bill Bejeck wrote:
> >>>>>>>>>> Hi Lucas, thanks for the comments.
> >>>>>>>>>>
> >>>>>>>>>>> LB1
> >>>>>>>>>> Yes, I'll update the KIP signature to `? extends Metric`
> >>>>>>>>>>
> >>>>>>>>>>> LB2
> >>>>>>>>>> If `enable.metrics.push` is disabled, the metrics passed in
> >>> are
> >>>>>>>> ignored.
> >>>>>>>>>> Thinking about this now, I'm considering updating the return
> >>>> type
> >>>>>> of
> >>>>>>>>>> `registerAdditionalMetrics` to a `boolean` from `void`, true
> >>> if
> >>>>> the
> >>>>>>>>> metrics
> >>>>>>>>>> have been successfully applied and false otherwise.
> >>>>>>>>>> WDYT?
> >>>>>>>>>>
> >>>>>>>>>> Cheers,
> >>>>>>>>>> Bill
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Mon, Aug 12, 2024 at 11:33 AM Lucas Brutschy
> >>>>>>>>>> <lbruts...@confluent.io.invalid> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Hi Bill!
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks for the KIP! I like this solution, it's general
> >>> enough
> >>>>> for
> >>>>>>> many
> >>>>>>>>>>> kinds of application.
> >>>>>>>>>>>
> >>>>>>>>>>> LB1: Could we accept an instance of the Metric interface
> >>>> instead
> >>>>>> of
> >>>>>>>>>>> KafkaMetric? No strong reason to do it, it would just make
> >>> the
> >>>>>>>>>>> interface slightly more generic.
> >>>>>>>>>>>
> >>>>>>>>>>> LB2: What is the behavior is `enable.metrics.push` is
> >>>> disabled?
> >>>>>>>>>>>
> >>>>>>>>>>> Cheers,
> >>>>>>>>>>> Lucas
> >>>>>>>>>>>
> >>>>>>>>>>> On Fri, Aug 2, 2024 at 9:30 PM Bill Bejeck <
> >>>> bbej...@apache.org>
> >>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>> Hi all,
> >>>>>>>>>>>>
> >>>>>>>>>>>> I would like to start a discussion thread on KIP-1076 to
> >>>> enable
> >>>>>>>> metrics
> >>>>>>>>>>>> collection for applications that embed a Kafka client.
> >>>>>>>>>>>>
> >>>>>>>>>>>> The KIP can be found here:
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1076%3A++Metrics+for+client+applications+KIP-714+extension
> >>>>>>>>>>>>
> >>>>>>>>>>>> I look forward to the discussion and your feedback.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thanks,
> >>>>>>>>>>>> Bill
>
>
>

Reply via email to