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 > > > > > >> > > > > > > > > > > > > > > > > > > > > >