Thank you for updating the KIP Qichao.

I don't have any more questions or suggestions. Looks good to move forward
from my perspective.



--
Divij Vaidya



On Fri, Nov 10, 2023 at 2:25 PM Qichao Chu <qic...@uber.com.invalid> wrote:

> Thank you again for the nice suggestions, Jorge!
> I will wait for Divij's response and move it to the vote stage once the
> generic filter part reached concensus.
>
> Qichao Chu
> Software Engineer | Data - Kafka
> [image: Uber] <https://uber.com/>
>
>
> On Fri, Nov 10, 2023 at 6:49 AM Jorge Esteban Quilcate Otoya <
> quilcate.jo...@gmail.com> wrote:
>
> > Hi Qichao,
> >
> > Thanks for updating the KIP, all updates look good to me.
> >
> > Looking forward to see this KIP moving forward!
> >
> > Cheers,
> > Jorge.
> >
> >
> >
> > On Wed, 8 Nov 2023 at 08:55, Qichao Chu <qic...@uber.com.invalid> wrote:
> >
> > > Hi Divij,
> > >
> > > Thank you for the feedback. I updated the KIP to make it a little bit
> > more
> > > generic: filters will stay in an array instead of different top-level
> > > objects. In this way, if we need language filters in the future. The
> > logic
> > > relationship of filters is also added.
> > >
> > > Hi Jorge,
> > >
> > > Thank you for the review and great comments. Here is the reply for each
> > of
> > > the suggestions:
> > >
> > > 1) The words describing the property are now updated to include more
> > > details of the keys in the JSON. It also explicitly mentions the JSON
> > > nature of the config now.
> > > 2) The JSON entries should be non-conflict so the order is not
> relevant.
> > If
> > > there's conflict, the conflict resolution rules are stated in the KIP.
> To
> > > make it more clear, ordering and duplication rules are updated in the
> > > Restrictions section of the *level* property.
> > > 3) Yeah we did take a look at the RecordingLevel config and it does not
> > > work for this case. The RecodingLevel config does not offer the
> > capability
> > > of filtering and it has a drawback of needing to be added to all the
> > future
> > > sensors. To reduce the duplication, I propose we merge the
> RecordingLevel
> > > to this more generic config in the future. Please take a look into the
> > > *Using
> > > the Existing RecordingLevel Config* section under *Rejected
> Alternatives*
> > > for more details.
> > > 4) This suggestion makes a lot of sense. My idea is to create a
> > > table/form/doc in the documentation for the verbosity levels of all
> > metric
> > > series. If it's too verbose to be in the docs, I will update the KIP to
> > > include this info. I will create a JIRA for this effort once the KIP is
> > > approved.
> > > 5) Sure we can expand to all other series, added to the KIP.
> > > 6) Added a new section(*Working with the Configuration via CLI)* with
> the
> > > user experience details
> > > 7) Links are updated.
> > >
> > > Please take another look and let me know if you have any more concerns.
> > >
> > > Best,
> > > Qichao Chu
> > > Software Engineer | Data - Kafka
> > > [image: Uber] <https://uber.com/>
> > >
> > >
> > > On Wed, Nov 8, 2023 at 6:29 AM Jorge Esteban Quilcate Otoya <
> > > quilcate.jo...@gmail.com> wrote:
> > >
> > > > Hi Qichao,
> > > >
> > > > Thanks for the KIP! This will be a valuable contribution and improve
> > the
> > > > tooling for troubleshooting.
> > > >
> > > > I have a couple of comments:
> > > >
> > > > 1. It's unclear from the `metrics.verbosity` description what the
> > > supported
> > > > values are. In the description mentions "If the value is high ... In
> > the
> > > > low settings" but I think it's referring to the `level` property
> > > > specifically instead of the whole value that is now JSON. Could you
> > > clarify
> > > > this?
> > > >
> > > > 2. Could we state in which order the JSON entries are going to be
> > > > evaluated? I guess the last entry wins if it overlaps previous
> values,
> > > but
> > > > better to make this explicit.
> > > >
> > > > 3. Kafka metrics library has a `RecordingLevel` configuration -- have
> > we
> > > > considered aligning these concepts and maybe reuse it instead of
> > > > `verbosityLevel`? Then we can reuse the levels: INFO, DEBUG, TRACE.
> > > >
> > > > 4. Not sure if within the scope of the KIP, but would be helpful to
> > > > document the metrics with the verbosity level attached to the
> metrics.
> > > > Maybe creating a JIRA ticket to track this would be enough if we
> can't
> > > > cover it as part of the KIP.
> > > >
> > > > 5. Could we consider the following client-related metrics as well:
> > > >   - BytesRejectedPerSec
> > > >   - TotalProduceRequestsPerSec
> > > >   - TotalFetchRequestsPerSec
> > > >   - FailedProduceRequestsPerSec
> > > >   - FailedFetchRequestsPerSec
> > > >   - FetchMessageConversionsPerSec
> > > >   - ProduceMessageConversionsPerSec
> > > > Would be great to have these from day 1 instead of requiring a
> > following
> > > > KIP to extend this. Could be implemented in separate PRs if needed.
> > > >
> > > > 6. To make it clearer how the user experience would be, could we
> > provide
> > > an
> > > > example of:
> > > > - how the broker configuration will be provided by default, and
> > > > - how the CLI tooling would be used to change the configuration?
> > > > - Maybe a couple of scenarios: adding a new metric config, a second
> one
> > > > with overlapping values, and
> > > > - describing the expected metrics to be mapped
> > > >
> > > > A couple of nits:
> > > > - The first link "MessagesInPerSec metrics" is pointing to
> > > > https://kafka.apache.org/documentation/#uses_metrics -- is this the
> > > > correct
> > > > reference? It doesn't seem too relevant.
> > > > - Also, the link to ReplicaManager points to a line that has change
> > > > already; better to have a permalink to a specific commit: e.g.
> > > >
> > > >
> > >
> >
> https://github.com/apache/kafka/blob/edc7e10a745c350ad1efa9e4866370dc8ea0e034/core/src/main/scala/kafka/server/ReplicaManager.scala#L1218
> > > >
> > > > Cheers,
> > > > Jorge.
> > > >
> > > > On Tue, 7 Nov 2023 at 17:06, Qichao Chu <qic...@uber.com.invalid>
> > wrote:
> > > >
> > > > > Hi Divij,
> > > > >
> > > > > It would be very nice if you could take a look at the recent
> changes,
> > > > thank
> > > > > you!
> > > > > If there's no more required changes, shall we move to vote stage?
> > > > >
> > > > > Best,
> > > > > Qichao Chu
> > > > > Software Engineer | Data - Kafka
> > > > > [image: Uber] <https://uber.com/>
> > > > >
> > > > >
> > > > > On Thu, Nov 2, 2023 at 12:06 AM Qichao Chu <qic...@uber.com>
> wrote:
> > > > >
> > > > > > Hi Divij,
> > > > > >
> > > > > > Thank you for the very quick response and the nice suggestions. I
> > > have
> > > > > > updated the KIP with the following thoughts.
> > > > > >
> > > > > > 1. I checked the Java documentation and it seems the regex engine
> > in
> > > > > utils
> > > > > > <
> > > > >
> > > >
> > >
> >
> https://docs.oracle.com/en/java/javase/11/docs/api/java.base/java/util/regex/Pattern.html
> > > > >
> > > > > is
> > > > > > not 100% compatible with PCRE, though it is very close. I stated
> > > > > > the Java implementation as the requirement since we are most
> likely
> > > to
> > > > > > target a JVM language.
> > > > > > 2. Agreed with the filter limitation. For now, let's keep it
> topic
> > > > only.
> > > > > > With that in mind, I feel we do have cases where a user wants to
> > list
> > > > > many
> > > > > > topics. Although regex is also possible, an array will make
> things
> > > > > faster.
> > > > > > This makes me add two options for the topic filter.
> > > > > > 3. It seems not many configs are using JSON, this was the
> intention
> > > for
> > > > > me
> > > > > > to use a compound string. However since JSON is used widely in
> the
> > > > > project,
> > > > > > and given the benefits you mentioned earlier, I tried to make the
> > > > config
> > > > > a
> > > > > > JSON array. The change is to make it compatible with multi-level
> > > > > settings.
> > > > > >
> > > > > > Let me know what you think. Many thanks!
> > > > > >
> > > > > > Best,
> > > > > > Qichao Chu
> > > > > > Software Engineer | Data - Kafka
> > > > > > [image: Uber] <https://uber.com/>
> > > > > >
> > > > > >
> > > > > > On Wed, Nov 1, 2023 at 9:43 PM Divij Vaidya <
> > divijvaidy...@gmail.com
> > > >
> > > > > > wrote:
> > > > > >
> > > > > >> Thank you for making the changes Qichao.
> > > > > >>
> > > > > >> We are now entering in the territory of defining a declarative
> > > schema
> > > > > for
> > > > > >> filters. In the new input format, the type is string but we are
> > > > > imposing a
> > > > > >> schema for the string and we should clearly call out the schema.
> > You
> > > > can
> > > > > >> perhaps choose to adopt a schema such as below:
> > > > > >>
> > > > > >> metricLevel = High | Low (default: Low)
> > > > > >> metricNameRegEx = regEx (default: .*)
> > > > > >> nameOfDimension = string
> > > > > >> dimensionRegEx = regEx
> > > > > >> dimensionFilter = [<nameOfDimension>=<dimensionRegEx>] (default:
> > [])
> > > > > >>
> > > > > >> Final Value schema = "level"=$metricLevel,
> > "name"=$metricNameRegEx,
> > > > > >> $dimensionFilter
> > > > > >>
> > > > > >> Further we need to answer questions such as :
> > > > > >> 1. which regEx format do we support (it should probably be
> > > > > Perl-compatible
> > > > > >> regular expressions (PCRE) because Java's regEx is compatible
> with
> > > it)
> > > > > >> 2. should we restrict the dimensionFilter to at max length 1 and
> > > value
> > > > > >> "topic" only for now. Later when we want to expand, we can
> expand
> > > > > filters
> > > > > >> for other dimensions as well such as partitions.
> > > > > >> 3. if we are coming up with our stringified-schema, why not use
> > > json?
> > > > It
> > > > > >> would save us from building a parsing utility for the schema. (I
> > > like
> > > > it
> > > > > >> in
> > > > > >> its current format but there is a case to be made for json as
> > well)
> > > > > >> 4. what happens when there are contradictory regEx rules, e.g. a
> > > topic
> > > > > >> defined in high as well as low. It is generally solved by
> defining
> > > > > >> precedence. In our case, we can choose that high has more
> > precedence
> > > > > than
> > > > > >> low.
> > > > > >>
> > > > > >> What do you think?
> > > > > >>
> > > > > >> --
> > > > > >> Divij Vaidya
> > > > > >>
> > > > > >>
> > > > > >>
> > > > > >> On Wed, Nov 1, 2023 at 2:07 PM Qichao Chu
> <qic...@uber.com.invalid
> > >
> > > > > >> wrote:
> > > > > >>
> > > > > >> > Hi Divij,
> > > > > >> >
> > > > > >> > Thank you for the review and the great suggestions, again. I
> > have
> > > > > >> updated
> > > > > >> > the corresponding content, can you take another look?
> > > > > >> > Regarding the KIP-544 style regex, I have added it to the new
> > > > property
> > > > > >> too.
> > > > > >> > It's expanded to include multiple sections for better future
> > > > > extension.
> > > > > >> >
> > > > > >> > Best,
> > > > > >> > Qichao Chu
> > > > > >> > Software Engineer | Data - Kafka
> > > > > >> > [image: Uber] <https://uber.com/>
> > > > > >> >
> > > > > >> >
> > > > > >> > On Mon, Oct 30, 2023 at 6:26 PM Divij Vaidya <
> > > > divijvaidy...@gmail.com
> > > > > >
> > > > > >> > wrote:
> > > > > >> >
> > > > > >> > > Hey *Qichao*
> > > > > >> > >
> > > > > >> > > Thank you for the update on the KIP. I like the idea of
> > > > incremental
> > > > > >> > > delivery and adding which metrics support this verbosity as
> a
> > > > later
> > > > > >> KIP.
> > > > > >> > > But I also want to ensure that we wouldn't have to change
> the
> > > > > current
> > > > > >> > > config when adding that in future. Hence, we need some
> > > discussion
> > > > on
> > > > > >> it
> > > > > >> > in
> > > > > >> > > the scope of the KIP.
> > > > > >> > >
> > > > > >> > > About the dynamic configuration:
> > > > > >> > > Do we need to add the "default" mode? I am asking because it
> > may
> > > > > >> inhibit
> > > > > >> > us
> > > > > >> > > from adding the allowList option in future. Instead if we
> > could
> > > > > >> rephrase
> > > > > >> > > the config as: "metric.verbosity.high" which takes values
> as a
> > > > regEx
> > > > > >> > > (default will be empty), then we wouldn't have to worry
> about
> > > > > >> > > future-proofness of this KIP. Notably this is an existing
> > > pattern
> > > > > >> used by
> > > > > >> > > KIP-544.
> > > > > >> > > Alternatively, if you choose to stick to the current
> > > configuration
> > > > > >> > pattern,
> > > > > >> > > please provide information on how this config will look like
> > > when
> > > > we
> > > > > >> add
> > > > > >> > > allow listing in future.
> > > > > >> > >
> > > > > >> > > About the perf test:
> > > > > >> > > Motivation - The motivation of perf test is to provide users
> > > with
> > > > a
> > > > > >> hint
> > > > > >> > on
> > > > > >> > > what perf penalty they can expect and whether default has
> > > degraded
> > > > > >> perf
> > > > > >> > > (due to additional "empty" labels).
> > > > > >> > > Dimensions of the test could be - scrape interval,
> utilization
> > > of
> > > > > >> broker
> > > > > >> > > (no traffic vs. heavy traffic), number of partitions
> > (small/200
> > > to
> > > > > >> > > large/2k).
> > > > > >> > > Things to collect during perf test - number of mbeans
> > registered
> > > > > with
> > > > > >> > JMX,
> > > > > >> > > CPU, heap utilization
> > > > > >> > > Expected results - As long as we can prove that there is no
> > > > > additional
> > > > > >> > > usage (significant) of CPU or heap after this change for the
> > > > > "default
> > > > > >> > > mode", we should be good. For the "high" mode, we should
> > > document
> > > > > the
> > > > > >> > > expected increase for users but it is not a blocker to
> > implement
> > > > > this
> > > > > >> > KIP.
> > > > > >> > >
> > > > > >> > >
> > > > > >> > > *Kirk*, I have tried to clarify the expectation on
> > performance,
> > > > does
> > > > > >> that
> > > > > >> > > address your question earlier? Also, I am happy with having
> a
> > > > Kafka
> > > > > >> level
> > > > > >> > > dynamic config that we can use to filter our
> > > metric/dimensionality
> > > > > >> since
> > > > > >> > we
> > > > > >> > > have a precedence at KIP-544. Hence, my suggestion to push
> > this
> > > > > >> filtering
> > > > > >> > > to metric library can be ignored.
> > > > > >> > >
> > > > > >> > > --
> > > > > >> > > Divij Vaidya
> > > > > >> > >
> > > > > >> > >
> > > > > >> > >
> > > > > >> > > On Sat, Oct 28, 2023 at 11:37 AM Qichao Chu
> > > > <qic...@uber.com.invalid
> > > > > >
> > > > > >> > > wrote:
> > > > > >> > >
> > > > > >> > > > Hello Everyone,
> > > > > >> > > >
> > > > > >> > > > Can I ask for some feedback regarding KIP-977
> > > > > >> > > > <
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-977%3A+Partition-Level+Throughput+Metrics
> > > > > >> > > > >
> > > > > >> > > > ?
> > > > > >> > > >
> > > > > >> > > > Best,
> > > > > >> > > > Qichao Chu
> > > > > >> > > > Software Engineer | Data - Kafka
> > > > > >> > > > [image: Uber] <https://uber.com/>
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > > On Mon, Oct 16, 2023 at 7:34 PM Qichao Chu <
> qic...@uber.com
> > >
> > > > > wrote:
> > > > > >> > > >
> > > > > >> > > > > Hi Divij and Kirk,
> > > > > >> > > > >
> > > > > >> > > > > Thank you both for providing the valuable feedback and
> > sorry
> > > > for
> > > > > >> the
> > > > > >> > > > > delay. I have just updated the KIP to address the
> > comments.
> > > > > >> > > > >
> > > > > >> > > > >    1. Instead of using a topic-level control, global
> > > verbosity
> > > > > >> > control
> > > > > >> > > > >    makes more sense if we want to extend it in the
> future.
> > > It
> > > > > >> would
> > > > > >> > be
> > > > > >> > > > very
> > > > > >> > > > >    difficult if we want to apply the topic allowlist
> > > > everywhere
> > > > > >> > > > >    2. Also, the topic allowlist was not dynamic which
> > makes
> > > > > >> > everything
> > > > > >> > > > >    quite complex, especially for the topic lifecycle
> > > > management.
> > > > > >> By
> > > > > >> > > > using the
> > > > > >> > > > >    dynamic global config, debugging could be easier, and
> > > > > >> management
> > > > > >> > of
> > > > > >> > > > the
> > > > > >> > > > >    config is also made easier.
> > > > > >> > > > >    3. More details are included in the test section.
> > > > > >> > > > >
> > > > > >> > > > > One thing that still misses is the performance numbers.
> I
> > > will
> > > > > >> get it
> > > > > >> > > > > ready with our internal clusters and share out soon.
> > > > > >> > > > >
> > > > > >> > > > > Many thanks for the review!
> > > > > >> > > > > Qichao
> > > > > >> > > > >
> > > > > >> > > > > On Tue, Sep 12, 2023 at 8:31 AM Kirk True <
> > > k...@kirktrue.pro>
> > > > > >> wrote:
> > > > > >> > > > >
> > > > > >> > > > >> Oh, and does metrics.partition.level.reporting.topics
> > allow
> > > > for
> > > > > >> > regex?
> > > > > >> > > > >>
> > > > > >> > > > >> > On Sep 12, 2023, at 8:26 AM, Kirk True <
> > > k...@kirktrue.pro>
> > > > > >> wrote:
> > > > > >> > > > >> >
> > > > > >> > > > >> > Hi Qichao,
> > > > > >> > > > >> >
> > > > > >> > > > >> > Thanks for the KIP!
> > > > > >> > > > >> >
> > > > > >> > > > >> > Divij—questions/comments inline...
> > > > > >> > > > >> >
> > > > > >> > > > >> >> On Sep 11, 2023, at 4:32 AM, Divij Vaidya <
> > > > > >> > divijvaidy...@gmail.com
> > > > > >> > > >
> > > > > >> > > > >> wrote:
> > > > > >> > > > >> >>
> > > > > >> > > > >> >> Thank you for the proposal Qichao.
> > > > > >> > > > >> >>
> > > > > >> > > > >> >> I agree with the motivation here and understand the
> > > > tradeoff
> > > > > >> here
> > > > > >> > > > >> >> between observability vs. increased metric
> dimensions
> > > > > (metric
> > > > > >> > > fan-out
> > > > > >> > > > >> >> as you say in the KIP).
> > > > > >> > > > >> >>
> > > > > >> > > > >> >> High level comments:
> > > > > >> > > > >> >>
> > > > > >> > > > >> >> 1. I would urge you to consider the extensibility of
> > the
> > > > > >> proposal
> > > > > >> > > for
> > > > > >> > > > >> >> other types of metrics. Tomorrow, if we want to
> > > > selectively
> > > > > >> add
> > > > > >> > > > >> >> "partition" dimension to another metric, would we
> have
> > > to
> > > > > >> modify
> > > > > >> > > the
> > > > > >> > > > >> >> code where each metric is emitted? Alternatively,
> > could
> > > we
> > > > > >> > abstract
> > > > > >> > > > >> >> out this config in a "Kafka Metrics" library. The
> code
> > > > > >> provides
> > > > > >> > all
> > > > > >> > > > >> >> information about this library and this library can
> > > choose
> > > > > >> which
> > > > > >> > > > >> >> dimensions it wants to add to the final metrics that
> > are
> > > > > >> emitted
> > > > > >> > > > based
> > > > > >> > > > >> >> on declarative configuration.
> > > > > >> > > > >> >
> > > > > >> > > > >> > I’d agree with this if it doesn’t place a burden on
> the
> > > > > >> callers.
> > > > > >> > Are
> > > > > >> > > > >> there any potential call sites that don’t have the
> > > partition
> > > > > >> > > information
> > > > > >> > > > >> readily available?
> > > > > >> > > > >> >
> > > > > >> > > > >> >> 2. Can we offload the handling of this dimension
> > > filtering
> > > > > to
> > > > > >> the
> > > > > >> > > > >> >> metric framework? Have you explored whether
> prometheus
> > > or
> > > > > >> other
> > > > > >> > > > >> >> libraries provide the ability to dynamically change
> > > > > dimensions
> > > > > >> > > > >> >> associated with metrics?
> > > > > >> > > > >> >
> > > > > >> > > > >> > I’m not familiar with the downstream metrics
> providers’
> > > > > >> > > capabilities.
> > > > > >> > > > >> This is a greatest common denominator scenario, right?
> > We’d
> > > > > have
> > > > > >> to
> > > > > >> > be
> > > > > >> > > > >> reasonable sure that the heavily used providers *all*
> > > support
> > > > > >> such
> > > > > >> > > > dynamic
> > > > > >> > > > >> filtering.
> > > > > >> > > > >> >
> > > > > >> > > > >> > Also—and correct me as needed as I’m not familiar
> with
> > > the
> > > > > >> area—if
> > > > > >> > > we
> > > > > >> > > > >> relegate partition filtering to a lower layer, we’d
> still
> > > > need
> > > > > to
> > > > > >> > > store
> > > > > >> > > > the
> > > > > >> > > > >> metric data in memory until it’s flushed, yes? If so,
> is
> > > that
> > > > > >> > overhead
> > > > > >> > > > of
> > > > > >> > > > >> any concern?
> > > > > >> > > > >> >
> > > > > >> > > > >> >> Implementation level comments:
> > > > > >> > > > >> >>
> > > > > >> > > > >> >> 1. In the test plan section, please mention what
> kind
> > of
> > > > > integ
> > > > > >> > > and/or
> > > > > >> > > > >> >> unit tests will be added and what they will assert.
> As
> > > an
> > > > > >> > example,
> > > > > >> > > > you
> > > > > >> > > > >> >> can add a section, "functionality tests", which
> would
> > > > assert
> > > > > >> that
> > > > > >> > > new
> > > > > >> > > > >> >> metric config is being respected and another
> section,
> > > > > >> > "performance
> > > > > >> > > > >> >> tests", which could be a system test and assert that
> > no
> > > > > >> > regression
> > > > > >> > > > >> >> caused wrt resources occupied by metrics from one
> > > version
> > > > to
> > > > > >> > > another.
> > > > > >> > > > >> >> 2. Please mention why or why not are we considering
> > > > > >> dynamically
> > > > > >> > > > >> >> setting the configuration (i.e. without a broker
> > > > restart)? I
> > > > > >> > would
> > > > > >> > > > >> >> imagine that the ability to dynamically configure
> for
> > a
> > > > > >> specific
> > > > > >> > > > topic
> > > > > >> > > > >> >> will be very useful especially to debug production
> > > > > situations
> > > > > >> > that
> > > > > >> > > > you
> > > > > >> > > > >> >> mention in the motivation.
> > > > > >> > > > >> >
> > > > > >> > > > >> > +1
> > > > > >> > > > >> >
> > > > > >> > > > >> >> 3. You mention that we want to start with metrics
> > > closely
> > > > > >> related
> > > > > >> > > to
> > > > > >> > > > >> >> producer & consumers first, which is fair. Could you
> > > > please
> > > > > >> add a
> > > > > >> > > > >> >> statement on the work required to extend this to
> other
> > > > > >> metrics in
> > > > > >> > > > >> >> future?
> > > > > >> > > > >> >
> > > > > >> > > > >> > +1
> > > > > >> > > > >> >
> > > > > >> > > > >> >> 4. In the compatibility section, you mention that
> this
> > > > > change
> > > > > >> is
> > > > > >> > > > >> >> backward compatible. I don't fully understand that.
> > > > During a
> > > > > >> > > version
> > > > > >> > > > >> >> upgrade, we will start with an empty list of topics
> to
> > > > > >> maintain
> > > > > >> > > > >> >> backward compatibility. I assume after the upgrade,
> we
> > > > will
> > > > > >> > update
> > > > > >> > > > the
> > > > > >> > > > >> >> new config with topic names that we desire to
> monitor.
> > > But
> > > > > >> > updating
> > > > > >> > > > >> >> the config will require a broker restart (a rolling
> > > > restart
> > > > > >> since
> > > > > >> > > > >> >> config is read-only). We will be in a situation
> where
> > > some
> > > > > >> > brokers
> > > > > >> > > > are
> > > > > >> > > > >> >> sending metrics with a new "partition" dimension and
> > > some
> > > > > >> brokers
> > > > > >> > > are
> > > > > >> > > > >> >> sending metrics with no partition dimension. Is that
> > > > > >> acceptable
> > > > > >> > to
> > > > > >> > > > JMX
> > > > > >> > > > >> >> / prometheus collectors? Would it break them? Please
> > > > clarify
> > > > > >> how
> > > > > >> > > > >> >> upgrades will work in the compatibility section.
> > > > > >> > > > >> >> 5. Could you please quantify (with an experiment)
> the
> > > > > expected
> > > > > >> > perf
> > > > > >> > > > >> >> impact of adding the partition dimension? This could
> > be
> > > > done
> > > > > >> as
> > > > > >> > > part
> > > > > >> > > > >> >> of "test plan" section and would serve as a data
> point
> > > for
> > > > > >> users
> > > > > >> > to
> > > > > >> > > > >> >> understand the potential impact if they decide to
> turn
> > > it
> > > > > on.
> > > > > >> > > > >> >
> > > > > >> > > > >> > Is there some guidance on the level of precision and
> > > detail
> > > > > >> > expected
> > > > > >> > > > >> when providing the performance numbers in the KIP?
> > > > > >> > > > >> >
> > > > > >> > > > >> > This notion of proving out the performance impact is
> > > > > >> important, I
> > > > > >> > > > >> agree. Anecdotally, there was another KIP I was
> following
> > > for
> > > > > >> which
> > > > > >> > > > >> performance numbers were requested, as is reasonable.
> But
> > > > that
> > > > > >> > caused
> > > > > >> > > > the
> > > > > >> > > > >> KIP to go a bit sideways as a result because it wasn’t
> > able
> > > > to
> > > > > >> get
> > > > > >> > > > >> consensus on a) the different scenarios to test, and b)
> > the
> > > > > >> > > quantitative
> > > > > >> > > > >> goal for each. I’m not really sure the rigo(u)r that’s
> > > > expected
> > > > > >> at
> > > > > >> > > this
> > > > > >> > > > >> stage in the development of a new feature.
> > > > > >> > > > >> >
> > > > > >> > > > >> > Thanks,
> > > > > >> > > > >> > Kirk
> > > > > >> > > > >> >
> > > > > >> > > > >> >>
> > > > > >> > > > >> >> --
> > > > > >> > > > >> >> Divij Vaidya
> > > > > >> > > > >> >>
> > > > > >> > > > >> >>
> > > > > >> > > > >> >> On Sat, Sep 9, 2023 at 8:18 PM Qichao Chu
> > > > > >> > <qic...@uber.com.invalid
> > > > > >> > > >
> > > > > >> > > > >> wrote:
> > > > > >> > > > >> >>>
> > > > > >> > > > >> >>> Hi All,
> > > > > >> > > > >> >>>
> > > > > >> > > > >> >>> Although this has been discussed many times, I
> would
> > > like
> > > > > to
> > > > > >> > > start a
> > > > > >> > > > >> new
> > > > > >> > > > >> >>> discussion regarding the introduction of
> > > partition-level
> > > > > >> > > throughput
> > > > > >> > > > >> >>> metrics. Please review the KIP and I'm eager to
> know
> > > > > >> everyone's
> > > > > >> > > > >> thoughts:
> > > > > >> > > > >> >>>
> > > > > >> > > > >>
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-977%3A+Partition-Level+Throughput+Metrics
> > > > > >> > > > >> >>>
> > > > > >> > > > >> >>> TL;DR: The KIP proposes to add partition-level
> > > throughput
> > > > > >> > metrics
> > > > > >> > > > and
> > > > > >> > > > >> a new
> > > > > >> > > > >> >>> configuration to control the fan-out rate.
> > > > > >> > > > >> >>>
> > > > > >> > > > >> >>> Thank you all for the review and have a nice
> weekend!
> > > > > >> > > > >> >>>
> > > > > >> > > > >> >>> Best,
> > > > > >> > > > >> >>> Qichao
> > > > > >> > > > >>
> > > > > >> > > > >>
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
>

Reply via email to