Re: [VOTE] 3.7.0 RC4

2024-02-14 Thread Paolo Patierno
+1 (non-binding). I used the staged binaries with Scala 2.13 and mostly
focused on the ZooKeeper to KRaft migration with multiple tests. Everything
works fine.

Thanks
Paolo

On Mon, 12 Feb 2024, 22:06 Jakub Scholz,  wrote:

> +1 (non-binding). I used the staged binaries with Scala 2.13 and the staged
> Maven artifacts to run my tests. All seems to work fine. Thanks.
>
> Jakub
>
> On Fri, Feb 9, 2024 at 4:20 PM Stanislav Kozlovski
>  wrote:
>
> > Hello Kafka users, developers and client-developers,
> >
> > This is the second candidate we are considering for release of Apache
> Kafka
> > 3.7.0.
> >
> > Major changes include:
> > - Early Access to KIP-848 - the next generation of the consumer rebalance
> > protocol
> > - Early Access to KIP-858: Adding JBOD support to KRaft
> > - KIP-714: Observability into Client metrics via a standardized interface
> >
> > Release notes for the 3.7.0 release:
> >
> >
> https://home.apache.org/~stanislavkozlovski/kafka-3.7.0-rc4/RELEASE_NOTES.html
> >
> > *** Please download, test and vote by Thursday, February 15th, 9AM PST
> ***
> >
> > Kafka's KEYS file containing PGP keys we use to sign the release:
> > https://kafka.apache.org/KEYS
> >
> > * Release artifacts to be voted upon (source and binary):
> > https://home.apache.org/~stanislavkozlovski/kafka-3.7.0-rc4/
> >
> > * Docker release artifact to be voted upon:
> > apache/kafka:3.7.0-rc4
> >
> > * Maven artifacts to be voted upon:
> > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> >
> > * Javadoc:
> > https://home.apache.org/~stanislavkozlovski/kafka-3.7.0-rc4/javadoc/
> >
> > * Tag to be voted upon (off 3.7 branch) is the 3.7.0 tag:
> > https://github.com/apache/kafka/releases/tag/3.7.0-rc4
> >
> > * Documentation:
> > https://kafka.apache.org/37/documentation.html
> >
> > * Protocol:
> > https://kafka.apache.org/37/protocol.html
> >
> > * Successful Jenkins builds for the 3.7 branch:
> >
> > Unit/integration tests: I am in the process of running and analyzing
> these.
> > System tests: I am in the process of running these.
> >
> > Expect a follow-up over the weekend
> >
> > * Successful Docker Image Github Actions Pipeline for 3.7 branch:
> > Docker Build Test Pipeline:
> > https://github.com/apache/kafka/actions/runs/7845614846
> >
> > /**
> >
> > Best,
> > Stanislav
> >
>


Re: DISCUSS KIP-984 Add pluggable compression interface to Kafka

2024-02-14 Thread Greg Harris
Hi Assane,

Thanks for the KIP!
Looking back, it appears that the project has only ever added
compression types twice: lz4 in 2014 and zstd in 2018, and perhaps
Kafka has fallen behind the state-of-the-art compression algorithms.
Thanks for working to fix that!

I do have some concerns:

1. I think this is a very "java centric" proposal, and doesn't take
non-java clients into enough consideration. librdkafka [1] is a great
example of an implementation of the Kafka protocol which doesn't have
the same classloading and plugin infrastructure that Java has, which
would make implementing this feature much more difficult.

2. By making the interface pluggable, it puts the burden of
maintaining individual compression codecs onto external developers,
which may not be willing to maintain a codec for the service-lifetime
of such a codec.
An individual developer can easily implement a plugin to allow them to
use a cutting-edge compression algorithm without consulting the Kafka
project, but as soon as data is compressed using that algorithm, they
are on the hook to support that plugin going forward by the
organizations using their implementation.
Part of the collective benefits of the Kafka project is to ensure the
ongoing maintenance of such codecs, and provide a long deprecation
window when a codec reaches EOL. I think the Kafka project is
well-equipped to evaluate the maturity and properties of compression
codecs and then maintain them going forward.

3. Also by making the interface pluggable, it reduces the scope of
individual compression codecs. No longer is there a single lineage of
Kafka protocols, where vN+1 of a protocol supports a codec that vN
does not. Now there will be "flavors" of the protocol, and operators
will need to ensure that their servers and their clients support the
same "flavors" or else encounter errors.
This is the sort of protocol forking which can be dangerous to the
Kafka community going forward. If there is a single lineage of codecs
such that the upstream Kafka vX.Y supports codec Z, it is much simpler
for other implementations to check and specify "Kafka vX.Y
compatible", than it is to check & specify "Kafka vX.Y & Z
compatible".

4. The Java class namespace is distributed, as anyone can name their
class anything. It achieves this by being very verbose, with long
fully-qualified names for classes. This is in conflict with a binary
protocol, where it is desirable for the overhead to be as small as
possible.
This may incentivise developers to keep their class names short, which
also makes conflict more likely. If you have the option of naming your
class "B" instead of
"org.example.blah.BrotlCompressionCodecVersionOne", and meaningfully
save a flat 47 bytes on every request, somebody/everybody is going to
do that.
This now increases the likelihood for conflict, as perhaps two
developers want the same short name. Yes there are 52 one-letter class
names, but to ensure that no two codecs ever conflict requires global
coordination that a pluggable interface tries to avoid.
Operators then take on the burden of ensuring that the "B" codec on
the other machine is indeed the "B" codec that they have installed on
their machines, or else encounter errors.

I think that having contributors propose that Kafka support their
favorite compression type in order to get assigned a globally unique
number is much healthier for the ecosystem than making this a
pluggable interface and leaving the namespace to be wrangled by
operators and client libraries.

Thanks,
Greg

[1] https://github.com/confluentinc/librdkafka
[2] 
https://github.com/apache/kafka/blob/e8c70fce26626ed2ab90f2728a45f6e55e907ec1/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java#L130

On Wed, Feb 14, 2024 at 12:59 PM Diop, Assane  wrote:
>
> Hi Divij, Mickael,
> Since Mickael KIP-390 was accepted, I did not want to respond in that thread 
> to not confuse the work.
>
> As mentioned in the thread, the KIP-390 and KIP-984 do not supercede each 
> other. However the scope of KIP-984 goes beyond the scope of KIP-390. 
> Pluggable compression interface is added as a new codec. The other codecs 
> already implemented are not affected by this change.  I believe these 2 KIP 
> are not the same but they compliment each other.
>
> As I stated before, the motivation is to give the users the ability to  use 
> different compressors without needing future changes in Kafka.
> Kafka currently supports zstd, snappy, gzip and lz4. However, other 
> opensource compression projects like the Brotli algorithm are also gaining 
> traction. For example the HTTP servers Apache and nginx offer Brotli 
> compression as an option. With a pluggable interface, any Kafka developer 
> could integrate and test Brotli with Kafka simply by writing a plugin. This 
> same motivation can be applied to any other compression algorithm including 
> hardware accelerated compression. There are hardware companies including 
> intel and AMD that are working o

[jira] [Created] (KAFKA-16260) Remove window.size.ms from StreamsConfig

2024-02-14 Thread Lucia Cerchie (Jira)
Lucia Cerchie created KAFKA-16260:
-

 Summary: Remove window.size.ms from StreamsConfig
 Key: KAFKA-16260
 URL: https://issues.apache.org/jira/browse/KAFKA-16260
 Project: Kafka
  Issue Type: Improvement
Reporter: Lucia Cerchie






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


RE: DISCUSS KIP-984 Add pluggable compression interface to Kafka

2024-02-14 Thread Diop, Assane
Hi Divij, Mickael, 
Since Mickael KIP-390 was accepted, I did not want to respond in that thread to 
not confuse the work. 

As mentioned in the thread, the KIP-390 and KIP-984 do not supercede each 
other. However the scope of KIP-984 goes beyond the scope of KIP-390. Pluggable 
compression interface is added as a new codec. The other codecs already 
implemented are not affected by this change.  I believe these 2 KIP are not the 
same but they compliment each other. 

As I stated before, the motivation is to give the users the ability to  use 
different compressors without needing future changes in Kafka. 
Kafka currently supports zstd, snappy, gzip and lz4. However, other opensource 
compression projects like the Brotli algorithm are also gaining traction. For 
example the HTTP servers Apache and nginx offer Brotli compression as an 
option. With a pluggable interface, any Kafka developer could integrate and 
test Brotli with Kafka simply by writing a plugin. This same motivation can be 
applied to any other compression algorithm including hardware accelerated 
compression. There are hardware companies including intel and AMD that are 
working on accelerating compression.

The main change in itself is an update in the message format to allow for 
metadata to be passed indicating the which plugin to use  to the broker. This 
only happens if the user selects the pluggable codec. The metadata adds on an 
additional 52 bytes to the message format. 

Broker recompression is taking care of when producer and brokers have different 
codec because it is just another codec being added as far as Kafka. 
I have added more information to the  
https://cwiki.apache.org/confluence/display/KAFKA/KIP-984%3A+Add+pluggable+compression+interface+to+Kafka
 I am ready for a PR if this KIP gets accepted

Assane

-Original Message-
From: Diop, Assane  
Sent: Wednesday, January 31, 2024 10:24 AM
To: dev@kafka.apache.org
Subject: RE: DISCUSS KIP-984 Add pluggable compression interface to Kafka

Hi Divij,
Thank you for your response!
  
Although compression is not a new problem, it has continued to be an important 
research topic.
The integration and testing of new compression algorithms into Kafka currently 
requires significant code changes and rebuilding of the distribution package 
for Kafka. 
This KIP will allow for any compression algorithm to be seamlessly integrated 
into Kafka by writing a plugin that would bind into the wrapForInput and 
wrapForOutput methods in Kafka.

As you mentioned, Kafka currently supports zstd, snappy, gzip and lz4. However, 
other opensource compression projects like the Brotli algorithm are also 
gaining traction. For example the HTTP servers Apache and nginx offer Brotli 
compression as an option. With a pluggable interface, any Kafka developer could 
integrate and test Brotli with Kafka simply by writing a plugin. This same 
motivation can be applied to any other compression algorithm including hardware 
accelerated compression. There are hardware companies including intel and AMD 
that are working on accelerating compression. 

This KIP would certainly complement the current 
https://issues.apache.org/jira/browse/KAFKA-7632 by adding even more 
flexibility for the users. 
A plugin could be tailored to arbitrary datasets in response to a user's 
specific resource requirements. 
 
For reference, other opensource projects have already started or implemented 
this type of plugin technology such as: 
1. Cassandra, which has implemented the same concept of pluggable 
interface. 
2. OpenSearch is also working on enabling the same type of plugin 
framework.
 
With respect to message recompression, the plugin interface would handle this 
use case on the broker side similar to the current recompression process. 
 
Assane  

-Original Message-
From: Divij Vaidya 
Sent: Friday, December 22, 2023 2:27 AM
To: dev@kafka.apache.org
Subject: Re: DISCUSS KIP-984 Add pluggable compression interface to Kafka

Thank you for writing the KIP Assane.

In general, exposing a "pluggable" interface is not a decision made lightly 
because it limits our ability to remove / change that interface in future.
Any future changes to the interface will have to remain compatible with 
existing plugins which limits the flexibility of changes we can make inside 
Kafka. Hence, we need a strong motivation for adding a pluggable interface.

1\ May I ask the motivation for this KIP? Are the current compression codecs 
(zstd, gzip, lz4, snappy) not sufficient for your use case? Would proving fine 
grained compression options as proposed in
https://issues.apache.org/jira/browse/KAFKA-7632 and 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-390%3A+Support+Compression+Level
address your use case?
2\ "This option impacts the following processes" -> This should also include 
the decompression and compression that occurs during message version 
transformation, i.e. when client send message with V1 and b

[jira] [Resolved] (KAFKA-15407) Not able to connect to kafka from the Private NLB from outside the VPC account

2024-02-14 Thread Greg Harris (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15407?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Greg Harris resolved KAFKA-15407.
-
Resolution: Invalid

> Not able to connect to kafka from the Private NLB from outside the VPC 
> account 
> ---
>
> Key: KAFKA-15407
> URL: https://issues.apache.org/jira/browse/KAFKA-15407
> Project: Kafka
>  Issue Type: Bug
>  Components: clients, connect, consumer, producer , protocol
> Environment: Staging, PROD
>Reporter: Shivakumar
>Priority: Blocker
> Attachments: image-2023-08-28-12-37-33-100.png
>
>
> !image-2023-08-28-12-37-33-100.png|width=768,height=223!
> Problem statement : 
> We are trying to connect Kafka from another account/VPC account
> Our kafka is in EKS cluster , we have service pointing to these pods for 
> connection
> We tried to create private link endpoint form Account B to connect to our NLB 
> to connect to our Kafka in Account A
> We see the connection reset from both client and target(kafka) in the NLB 
> monitoring tab of AWS.
> We tried various combo of listeners and advertised listeners which did not 
> help us.
> We are assuming we are missing some combination of Listeners and Network 
> level configs with which this connection can be made 
> Can you please guide us with this as we are blocked with a major migration. 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] KIP-1010: Topic Partition Quota

2024-02-14 Thread Afshin Moazami
Thanks Viktor,

Hi folks,
I would like to propose a new feature to extend the quota management in
Kafka to support topic-partition based quotas. The following is the link to
the KIP
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1010%3A+Topic+Partition+Quota


Best,
Afshin Moazami

On Wed, Feb 7, 2024 at 5:25 AM Viktor Somogyi-Vass
 wrote:

> Hi Afshin,
>
> We keep KIP discussions on dev@kafka.apache.org so please post this over
> there too. I'll go over this later this week but devs usually monitor that
> list more frequently and you'll have better chances of getting a reply
> there.
>
> Regards,
> Viktor
>
> On Wed, Jan 17, 2024 at 12:03 AM Afshin Moazami
>  wrote:
>
> > Hi folks,
> > I am not sure what is the KIP life-cycle and how we can get more
> attention
> > on them, so I just reply to this thread with the hope to get some
> > discussion started.
> >
> > Thanks,
> > Afshin
> >
> > On Mon, Dec 11, 2023 at 10:43 AM Afshin Moazami  >
> > wrote:
> >
> > > Hi folks,
> > > I would like to propose a new feature to extend the quota management in
> > > Kafka to support topic-partition based quotas. The following is the
> link
> > to
> > > the KIP
> > >
> > >
> >
> https://urldefense.com/v3/__https://cwiki.apache.org/confluence/display/KAFKA/KIP-1010*3A*Topic*Partition*Quota__;JSsrKw!!DCbAVzZNrAf4!BK-888ZjIeh53cmPcRZ_ZIpA6-02xIk5LXsT4cl82ieHRjWN31a-xsi36sN9I3P3LOhhpYCJU2FpbYkfg2YpGX2RXtPFAIjsHv0$
> > >
> > >
> > > Best,
> > > Afshin Moazami
> > >
> >
>


[jira] [Created] (KAFKA-16259) Immutable MetadataCache to improve client performance

2024-02-14 Thread Zhifeng Chen (Jira)
Zhifeng Chen created KAFKA-16259:


 Summary: Immutable MetadataCache to improve client performance
 Key: KAFKA-16259
 URL: https://issues.apache.org/jira/browse/KAFKA-16259
 Project: Kafka
  Issue Type: Improvement
  Components: clients
Affects Versions: 2.8.0
Reporter: Zhifeng Chen
 Attachments: image-2024-02-14-12-11-07-366.png

TL;DR, A Kafka client produce latency issue is identified caused by 
synchronized lock contention of metadata cache read/write in the native kafka 
producer.

Trigger Condition: A producer need to produce to large number of topics. such 
as in kafka rest-proxy

 

 

What is producer metadata cache

Kafka producer maintains a in-memory copy of cluster metadata, and it avoided 
fetch metadata every time when produce message to reduce latency

 

What’s the synchronized lock contention problem

Kafka producer metadata cache is a *mutable* object, read/write are isolated by 
a synchronized lock. Which means when the metadata cache is being updated, all 
read requests are blocked. 

Topic metadata expiration frequency increase liner with number of topics. In a 
kafka cluster with large number of topic partitions, topic metadata expiration 
and refresh triggers high frequent metadata update. When read operation blocked 
by update, producer threads are blocked and caused high produce latency issue.

 

*Proposed solution*

TL;DR Optimize performance of metadata cache read operation of native kafka 
producer with copy-on-write strategy

What is copy-on-write strategy

It’s a solution to reduce synchronized lock contention by making the object 
immutable, and always create a new instance when updating, but since the object 
is immutable, read operation will be free from waiting, thus produce latency 
reduced significantly

Besides performance, it can also make the metadata cache immutable from 
unexpected modification, reduce occurrence of code bugs due to incorrect 
synchronization 

 

Test result:

Environment: Kafka-rest-proxy

Client version: 2.8.0

Number of topic partitions: 250k

test result show 90%+ latency reduction on test cluster

!image-2024-02-14-12-11-07-366.png!

P99 produce latency on deployed instances reduced from 200ms -> 5ms (upper part 
show latency after the improvement, lower part show before improvement)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [VOTE] 3.7.0 RC4

2024-02-14 Thread Justine Olshan
I just merged a PR to trunk that fixes some of the LogDirFailureTests.

I believe the issue is mostly related to the tests and not the code. Not
sure if we think it is necessary to hotfix since the change was in the
tests.
https://github.com/apache/kafka/commit/be6653c8bc25717e25a7db164527635a6579b4cc

Thanks for the results and the gist!

Justine

On Wed, Feb 14, 2024 at 11:53 AM Stanislav Kozlovski
 wrote:

> Hey Justine,
>
> Good question. I have been running the system tests and I have been
> preparing a doc to share. I got one run completed and was waiting for a
> second one to cross-check, but unfortunately I had a build take 35+ hours
> and canceled it. I'm currently 6h into the second one, expecting it to
> complete in 11h or so.
>
> There were quite a bit, so I was looking forward to get another run in to
> cross-check the flakes:
> -
>
> https://gist.githubusercontent.com/stanislavkozlovski/3a97fc7602f3fee40cb374f1343d46b6/raw/b9de6e0eb975e8234d43bc725982096e47fd0457/rc4_test_failures_system_test_run_1.md
>
> I also checked the integration tests and saw
> - kafka.server.LogDirFailureTest.testProduceErrorFromFailureOnLogRoll
> - kafka.server.LogDirFailureTest.testIOExceptionDuringLogRoll
> failed three timed in a row.
>
> Here is the gist with the results:
> https://gist.github.com/stanislavkozlovski/820976fc7bfb5f4dcdf9742fd96a9982
>
>
>
> On Wed, Feb 14, 2024 at 6:31 PM Justine Olshan
> 
> wrote:
>
> > Hey Stan,
> > Did we ever get system tests results? I can also start a run
> >
> > On Mon, Feb 12, 2024 at 1:06 PM Jakub Scholz  wrote:
> >
> > > +1 (non-binding). I used the staged binaries with Scala 2.13 and the
> > staged
> > > Maven artifacts to run my tests. All seems to work fine. Thanks.
> > >
> > > Jakub
> > >
> > > On Fri, Feb 9, 2024 at 4:20 PM Stanislav Kozlovski
> > >  wrote:
> > >
> > > > Hello Kafka users, developers and client-developers,
> > > >
> > > > This is the second candidate we are considering for release of Apache
> > > Kafka
> > > > 3.7.0.
> > > >
> > > > Major changes include:
> > > > - Early Access to KIP-848 - the next generation of the consumer
> > rebalance
> > > > protocol
> > > > - Early Access to KIP-858: Adding JBOD support to KRaft
> > > > - KIP-714: Observability into Client metrics via a standardized
> > interface
> > > >
> > > > Release notes for the 3.7.0 release:
> > > >
> > > >
> > >
> >
> https://home.apache.org/~stanislavkozlovski/kafka-3.7.0-rc4/RELEASE_NOTES.html
> > > >
> > > > *** Please download, test and vote by Thursday, February 15th, 9AM
> PST
> > > ***
> > > >
> > > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > > https://kafka.apache.org/KEYS
> > > >
> > > > * Release artifacts to be voted upon (source and binary):
> > > > https://home.apache.org/~stanislavkozlovski/kafka-3.7.0-rc4/
> > > >
> > > > * Docker release artifact to be voted upon:
> > > > apache/kafka:3.7.0-rc4
> > > >
> > > > * Maven artifacts to be voted upon:
> > > >
> https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > > >
> > > > * Javadoc:
> > > > https://home.apache.org/~stanislavkozlovski/kafka-3.7.0-rc4/javadoc/
> > > >
> > > > * Tag to be voted upon (off 3.7 branch) is the 3.7.0 tag:
> > > > https://github.com/apache/kafka/releases/tag/3.7.0-rc4
> > > >
> > > > * Documentation:
> > > > https://kafka.apache.org/37/documentation.html
> > > >
> > > > * Protocol:
> > > > https://kafka.apache.org/37/protocol.html
> > > >
> > > > * Successful Jenkins builds for the 3.7 branch:
> > > >
> > > > Unit/integration tests: I am in the process of running and analyzing
> > > these.
> > > > System tests: I am in the process of running these.
> > > >
> > > > Expect a follow-up over the weekend
> > > >
> > > > * Successful Docker Image Github Actions Pipeline for 3.7 branch:
> > > > Docker Build Test Pipeline:
> > > > https://github.com/apache/kafka/actions/runs/7845614846
> > > >
> > > > /**
> > > >
> > > > Best,
> > > > Stanislav
> > > >
> > >
> >
>
>
> --
> Best,
> Stanislav
>


Re: [VOTE] 3.7.0 RC4

2024-02-14 Thread Stanislav Kozlovski
Hey Justine,

Good question. I have been running the system tests and I have been
preparing a doc to share. I got one run completed and was waiting for a
second one to cross-check, but unfortunately I had a build take 35+ hours
and canceled it. I'm currently 6h into the second one, expecting it to
complete in 11h or so.

There were quite a bit, so I was looking forward to get another run in to
cross-check the flakes:
-
https://gist.githubusercontent.com/stanislavkozlovski/3a97fc7602f3fee40cb374f1343d46b6/raw/b9de6e0eb975e8234d43bc725982096e47fd0457/rc4_test_failures_system_test_run_1.md

I also checked the integration tests and saw
- kafka.server.LogDirFailureTest.testProduceErrorFromFailureOnLogRoll
- kafka.server.LogDirFailureTest.testIOExceptionDuringLogRoll
failed three timed in a row.

Here is the gist with the results:
https://gist.github.com/stanislavkozlovski/820976fc7bfb5f4dcdf9742fd96a9982



On Wed, Feb 14, 2024 at 6:31 PM Justine Olshan 
wrote:

> Hey Stan,
> Did we ever get system tests results? I can also start a run
>
> On Mon, Feb 12, 2024 at 1:06 PM Jakub Scholz  wrote:
>
> > +1 (non-binding). I used the staged binaries with Scala 2.13 and the
> staged
> > Maven artifacts to run my tests. All seems to work fine. Thanks.
> >
> > Jakub
> >
> > On Fri, Feb 9, 2024 at 4:20 PM Stanislav Kozlovski
> >  wrote:
> >
> > > Hello Kafka users, developers and client-developers,
> > >
> > > This is the second candidate we are considering for release of Apache
> > Kafka
> > > 3.7.0.
> > >
> > > Major changes include:
> > > - Early Access to KIP-848 - the next generation of the consumer
> rebalance
> > > protocol
> > > - Early Access to KIP-858: Adding JBOD support to KRaft
> > > - KIP-714: Observability into Client metrics via a standardized
> interface
> > >
> > > Release notes for the 3.7.0 release:
> > >
> > >
> >
> https://home.apache.org/~stanislavkozlovski/kafka-3.7.0-rc4/RELEASE_NOTES.html
> > >
> > > *** Please download, test and vote by Thursday, February 15th, 9AM PST
> > ***
> > >
> > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > https://kafka.apache.org/KEYS
> > >
> > > * Release artifacts to be voted upon (source and binary):
> > > https://home.apache.org/~stanislavkozlovski/kafka-3.7.0-rc4/
> > >
> > > * Docker release artifact to be voted upon:
> > > apache/kafka:3.7.0-rc4
> > >
> > > * Maven artifacts to be voted upon:
> > > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > >
> > > * Javadoc:
> > > https://home.apache.org/~stanislavkozlovski/kafka-3.7.0-rc4/javadoc/
> > >
> > > * Tag to be voted upon (off 3.7 branch) is the 3.7.0 tag:
> > > https://github.com/apache/kafka/releases/tag/3.7.0-rc4
> > >
> > > * Documentation:
> > > https://kafka.apache.org/37/documentation.html
> > >
> > > * Protocol:
> > > https://kafka.apache.org/37/protocol.html
> > >
> > > * Successful Jenkins builds for the 3.7 branch:
> > >
> > > Unit/integration tests: I am in the process of running and analyzing
> > these.
> > > System tests: I am in the process of running these.
> > >
> > > Expect a follow-up over the weekend
> > >
> > > * Successful Docker Image Github Actions Pipeline for 3.7 branch:
> > > Docker Build Test Pipeline:
> > > https://github.com/apache/kafka/actions/runs/7845614846
> > >
> > > /**
> > >
> > > Best,
> > > Stanislav
> > >
> >
>


-- 
Best,
Stanislav


[jira] [Resolved] (KAFKA-10735) Kafka producer producing corrupted avro values when confluent cluster is recreated and producer application is not restarted

2024-02-14 Thread Greg Harris (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-10735?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Greg Harris resolved KAFKA-10735.
-
Resolution: Cannot Reproduce

Hi, since this doesn't contain any Apache Kafka code, and doesn't provide any 
configurations, we are unable to reproduce this failure. Closing as cannot 
reproduce.

> Kafka producer producing corrupted avro values when confluent cluster is 
> recreated and producer application is not restarted
> 
>
> Key: KAFKA-10735
> URL: https://issues.apache.org/jira/browse/KAFKA-10735
> Project: Kafka
>  Issue Type: Bug
>Reporter: Tim Tattersall
>Priority: Major
>
> *Our Environment (AWS):*
> 1 x EC2 instance running 4 docker containers (using docker-compose)
>  * cp-kafka 5.5.1
>  * cp-zookeeper 5.5.1
>  * cp-schema-registry 5.5.1
>  * cp-enterprise-control-center 5.5.1
> 1 x ECS service running a single java application with spring-kafka producer
> Topics are using String key and Avro value
>  
> *Problem:*
>  * Avro values published after confluent cluster is recreated are corrupted. 
> Expecting Avro json structure, received string value with corrupted Avro 
> details
>  ** Expected: {"metadata":{"nabEventVersion":"1.0","type":"Kafka IBMMQ sink 
> connector","schemaUrl": ...*ongoing*
>  ** Actual: 1.08Kafka IBMMQ source 
> connector^kafka-conector-ibm-mq-source-entitlements-check\Kafka IBMMQ source 
> connector - sourced*ongoing*
>  
> *How to Reproduce*
>  # Using an existing confluent cluster
>  # Start a kafka producer java application (ours running with spring-kafka)
>  # Destroy the existing confluent cluster (using docker-compose down)
>  # Recreate the confluent cluster (using docker-compose up)
>  # Add the topic back onto the new cluster
>  # Trigger a message to be produced by the running Kafka producer
>  
> *Current Workaround*
>  * Killing running tasks on ECS service and allowing AWS to start new ones



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-10719) MirrorMaker2 fails to update its runtime configuration

2024-02-14 Thread Greg Harris (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-10719?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Greg Harris resolved KAFKA-10719.
-
Fix Version/s: 3.7.0
   3.6.2
   Resolution: Fixed

> MirrorMaker2 fails to update its runtime configuration
> --
>
> Key: KAFKA-10719
> URL: https://issues.apache.org/jira/browse/KAFKA-10719
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.6.0
>Reporter: Peter Sinoros-Szabo
>Priority: Major
> Fix For: 3.7.0, 3.6.2
>
>
> I was running successfully the MM2 cluster with the following configuration, 
> I simplified it a little: {code:java} clusters = main, backup 
> main.bootstrap.servers = kafkaA:9202,kafkaB:9092,kafkaB:9092 
> backup.bootstrap.servers = backupA:9092,backupB:9092,backupC:9092 
> main->backup.enabled = true main->backup.topics = .*{code} I wanted to change 
> the bootstrap.address list of the destination cluster to a different list 
> that is pointing to the *same* cluster, just a different listener with a 
> different routing. So I changed it to: {code:java} backup.bootstrap.servers = 
> backupSecA:1234,backupSecB:1234,backupSecC:1234{code} I did a rolling restart 
> on the MM2 nodes and say that some tasks were still using the old bootstrap 
> addresses, some of them were using the new one. I don't have the logs, so 
> unfortunately I don't know which one picked up the good values and which 
> didn't. I even stopped the cluster completely, but it didn't help. Ryanne 
> adviced to delete the mm2-config and mm2-status topics, so I did delete those 
> on the destination cluster, that solved this issue.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-16258) Stale member should trigger onPartitionsLost when leaving group

2024-02-14 Thread Lianet Magrans (Jira)
Lianet Magrans created KAFKA-16258:
--

 Summary: Stale member should trigger onPartitionsLost when leaving 
group
 Key: KAFKA-16258
 URL: https://issues.apache.org/jira/browse/KAFKA-16258
 Project: Kafka
  Issue Type: Sub-task
  Components: clients, consumer
Affects Versions: 3.7.0
Reporter: Lianet Magrans
 Fix For: 3.8.0


When the poll timer expires, the new consumer proactively leaves the group and 
clears its assignments, but it should also invoke the onPartitionsLost 
callback. The legacy coordinator does the following sequence on poll timer 
expiration: send leave group request 
([here|https://github.com/apache/kafka/blob/e8c70fce26626ed2ab90f2728a45f6e55e907ec1/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java#L1517]),
 invoke onPartitionsLost, and when it completes it clears the assignment 
(onJoinPrepare 
[here|https://github.com/apache/kafka/blob/e8c70fce26626ed2ab90f2728a45f6e55e907ec1/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java#L779]).



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #2645

2024-02-14 Thread Apache Jenkins Server
See 




Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-02-14 Thread José Armando García Sancio
Hi Jun,

Thanks for the feedback. Excuse the delay, it took me a while to
properly address your detailed feedback. See my comments below.

I am going to update the KIP as outlined in this email. I will send
another email when I have made all of the changes.

On Fri, Feb 2, 2024 at 10:54 AM Jun Rao  wrote:
> 10. kraft.version: Functionality wise, this seems very similar to
> metadata.version, which is to make sure that all brokers/controllers are on
> a supported version before enabling a new feature. Could you explain why we
> need a new one instead of just relying on metadata.version?

Yes, they are trying to solve similar problems but they are doing them
at a different abstraction layer. "metadata.version" uses
FeatureLevelRecord which is a "data" record. I am trying to design the
KRaft protocol (KafkaRaftClient implementation) to not assume a
specific application (metadata and the controller). To support
multiple applications possibly using KRaft means that we need to use
control records to define KRaft's behavior. Instead of using
FeatureLevelRecord which is a cluster metadata record, this KIP uses
KRaftVersionRecord which will be a control record.

The reason why we need to consider observers (brokers) when
determining the kraft.version is because of snapshot (compaction).
Since snapshots need to include all logical records in the log
segments and observers (brokers) maintain the log on disk, this means
that they need to be able to decode and encode both KRaftVersionRecord
and VotersRecord before the leader is allowed to write them.

> 11. Both the quorum-state file and controller.quorum.bootstrap.servers
> contain endpoints. Which one takes precedence?

I updated the KIP (section "Reference explanation / Endpoints
information") after Jack's email. KRaft is going to keep two sets of
endpoints.

1. Voters set. This set of endpoints is used by voters to establish
leadership with the Vote, BeginQuorumEpoch and EndQuorumEpoch RPCs.
The precedence order is the VotersRecord in the snapshot and log. If
there are no VotersRecords in the snapshot or log, it will use the
configuration in controller.quorum.voters.

2. Bootstrap servers. This is mainly used by observers (brokers) to
discover the leader through Fetch RPCs. The precedence order is
controller.quorum.bootstrap.servers first, controller.quorum.voters
second. Voters won't use this property as they discover the leader and
its endpoint from the BeginQuorumEpoch RPC from the leader.

I believe that the original intent of the voters in quorum state file
was as a cache of the controller.quorum.voters configuration and to
identify invalid changes in the configuration. I was trying to keep
this functionality in the new version of the quorum state file.

The more I think about the implementation, I don't think this is
useful or even implementable. KRaft needs to keep all voters sets from
the latest snapshot to the LEO so that it could include the correct
voters set when generating a snapshot. I am going to update the KIP to
remove voter information from the quorum state file.

> 12. It seems that downgrading from this KIP is not supported? Could we have
> a section to make it explicit?

Yes. Downgrades will not be supported. I updated the "Compatibility,
deprecation and migration plan". There is a sentence about this in the
"Public interfaces / Command line interface / kafka-features" section.
I'll also update the "Proposed changes / Reference explanation /
Supported features" and "Public interfaces / RPCs / UpdateFeatures /
Handling" sections.

> 13. controller.quorum.auto.join.enable: If this is set true, when does the
> controller issue the addVoter RPC? Does it need to wait until it's caught
> up? Does it issue the addVoter RPC on every restart?

The controller will monitor the voters set. If the controller finds
that the voters set doesn't contain itself, it will send an AddVoter
RPC to the leader. To avoid the quorum adding a node and becoming
unavailable, I will change the handling of the AddVoter RPC to not
allow duplicate replica id. If there is any replica with the same id
but a different uuid, the old uuid must be removed first (with the
RemoveVoter RPC) before the new uuid can be added.

Here is an example that shows how a KRaft partition can become
unavailable if we allow automatically adding duplicate replica ids.
Assume that the partition starts with voters v1 and v2. Operator
starts controller 3 as v3 (replica id 3, some generated replica uuid)
and it attempts to automatically join the cluster by sending the
AddVoter RPC. Assume that the voter set (v1, v2, v3) gets committed by
v1 and v2. Controller 3 restarts with a new disk as v3' and sends an
AddVoter RPC. Because (v1, v2, v3) was committed the leader is able to
change the quorum to (v1, v2, v3, v3') but won't be able to commit it
if the controller 3 restarts again and comes back with a new disk as
v3''.

This case is avoided if the KRaft leader rejects any AddVoter RPC that
duplicates the r

[jira] [Created] (KAFKA-16257) SchemaProjector should be extensible to logical types

2024-02-14 Thread Greg Harris (Jira)
Greg Harris created KAFKA-16257:
---

 Summary: SchemaProjector should be extensible to logical types
 Key: KAFKA-16257
 URL: https://issues.apache.org/jira/browse/KAFKA-16257
 Project: Kafka
  Issue Type: New Feature
  Components: connect
Reporter: Greg Harris


The SchemaProjector currently only supports projecting primitive Number types, 
and cannot handle common logical types as have proliferated in the Connect 
ecosystem.

The SchemaProjector or a replacement should have the ability to extend it's 
functionality to support these logical types.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] KIP-932: Queues for Kafka

2024-02-14 Thread Manikumar
Hi Andrew,

Thanks for the KIP. A few comments below.

1. kafka-configs.sh (incrementalAlterConfigs) allows you to dynamically
change the configs. Maybe in this case, we should not allow the user to
change `group.type` if it's already set.
2. What's the behaviour after a group transitions into DEAD state. Do we
add new control records to reset the state? Can we reuse the group again?
3. Are we going to write new control records after the
AlterShareGroupOffsets API to reset the state?
4. Is there any API for DeleteShareGroups? I assume, group co-ordinator is
going to handle the API. If so, Does this mean the group co-ordinator also
needs to write control records?
5. How about using "org.apache.kafka.clients.consumer.share" package for
new interfaces/classes?


Thanks,
Manikumar


Re: [VOTE] 3.7.0 RC4

2024-02-14 Thread Justine Olshan
Hey Stan,
Did we ever get system tests results? I can also start a run

On Mon, Feb 12, 2024 at 1:06 PM Jakub Scholz  wrote:

> +1 (non-binding). I used the staged binaries with Scala 2.13 and the staged
> Maven artifacts to run my tests. All seems to work fine. Thanks.
>
> Jakub
>
> On Fri, Feb 9, 2024 at 4:20 PM Stanislav Kozlovski
>  wrote:
>
> > Hello Kafka users, developers and client-developers,
> >
> > This is the second candidate we are considering for release of Apache
> Kafka
> > 3.7.0.
> >
> > Major changes include:
> > - Early Access to KIP-848 - the next generation of the consumer rebalance
> > protocol
> > - Early Access to KIP-858: Adding JBOD support to KRaft
> > - KIP-714: Observability into Client metrics via a standardized interface
> >
> > Release notes for the 3.7.0 release:
> >
> >
> https://home.apache.org/~stanislavkozlovski/kafka-3.7.0-rc4/RELEASE_NOTES.html
> >
> > *** Please download, test and vote by Thursday, February 15th, 9AM PST
> ***
> >
> > Kafka's KEYS file containing PGP keys we use to sign the release:
> > https://kafka.apache.org/KEYS
> >
> > * Release artifacts to be voted upon (source and binary):
> > https://home.apache.org/~stanislavkozlovski/kafka-3.7.0-rc4/
> >
> > * Docker release artifact to be voted upon:
> > apache/kafka:3.7.0-rc4
> >
> > * Maven artifacts to be voted upon:
> > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> >
> > * Javadoc:
> > https://home.apache.org/~stanislavkozlovski/kafka-3.7.0-rc4/javadoc/
> >
> > * Tag to be voted upon (off 3.7 branch) is the 3.7.0 tag:
> > https://github.com/apache/kafka/releases/tag/3.7.0-rc4
> >
> > * Documentation:
> > https://kafka.apache.org/37/documentation.html
> >
> > * Protocol:
> > https://kafka.apache.org/37/protocol.html
> >
> > * Successful Jenkins builds for the 3.7 branch:
> >
> > Unit/integration tests: I am in the process of running and analyzing
> these.
> > System tests: I am in the process of running these.
> >
> > Expect a follow-up over the weekend
> >
> > * Successful Docker Image Github Actions Pipeline for 3.7 branch:
> > Docker Build Test Pipeline:
> > https://github.com/apache/kafka/actions/runs/7845614846
> >
> > /**
> >
> > Best,
> > Stanislav
> >
>


[jira] [Created] (KAFKA-16256) Update ConsumerConfig to validate use of group.remote.assignor and partition.assignment.strategy based on group.protocol

2024-02-14 Thread Kirk True (Jira)
Kirk True created KAFKA-16256:
-

 Summary: Update ConsumerConfig to validate use of 
group.remote.assignor and partition.assignment.strategy based on group.protocol
 Key: KAFKA-16256
 URL: https://issues.apache.org/jira/browse/KAFKA-16256
 Project: Kafka
  Issue Type: Improvement
  Components: clients, consumer
Affects Versions: 3.7.0
Reporter: Kirk True
 Fix For: 3.8.0


{{ConsumerConfig}} supports both the {{group.remote.assignor}} and 
{{partition.assignment.strategy}} configuration options. These, however, should 
not be used together; the former is applicable only when the {{group.protocol}} 
is set to {{consumer}} and the latter when the {{group.protocol}} is set to 
{{{}classic{}}}. We should emit a warning if the user specifies the incorrect 
configuration based on the value of {{{}group.protocol{}}}.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-16255) AsyncKafkaConsumer should not use partition.assignment.strategy

2024-02-14 Thread Kirk True (Jira)
Kirk True created KAFKA-16255:
-

 Summary: AsyncKafkaConsumer should not use 
partition.assignment.strategy
 Key: KAFKA-16255
 URL: https://issues.apache.org/jira/browse/KAFKA-16255
 Project: Kafka
  Issue Type: Bug
  Components: clients, consumer
Affects Versions: 3.7.0
Reporter: Kirk True
Assignee: Kirk True
 Fix For: 3.8.0


The partition.assignment.strategy configuration is used to specify a list of 
zero or more 
ConsumerPartitionAssignor instances. However, that interface is not applicable 
for the KIP-848-based protocol on top of which AsyncKafkaConsumer is built. 
Therefore, the use of ConsumerPartitionAssignor is in appropriate and should be 
removed.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-16254) Allow MM2 to fully disable offset sync feature

2024-02-14 Thread Omnia Ibrahim (Jira)
Omnia Ibrahim created KAFKA-16254:
-

 Summary: Allow MM2 to fully disable offset sync feature
 Key: KAFKA-16254
 URL: https://issues.apache.org/jira/browse/KAFKA-16254
 Project: Kafka
  Issue Type: Bug
  Components: mirrormaker
Affects Versions: 3.6.0, 3.5.0, 3.7.0
Reporter: Omnia Ibrahim
Assignee: Omnia Ibrahim


*Background:* 
At the moment syncing offsets feature in MM2 is broken to 2 parts
 # One is in `MirrorSourceTask` where we store the new recored's offset on 
target cluster to {{offset_syncs}} internal topic after mirroring the record. 
Before KAFKA-14610 in 3.5 MM2 used to just queue the offsets and publish them 
later but since 3.5 this behaviour changed we now publish any offset syncs that 
we've queued up, but have not yet been able to publish when 
`MirrorSourceTask.commit` get invoked. This introduced an over head to commit 
process.
 # The second part is in checkpoints source task where we use the new record 
offsets from {{offset_syncs}} and update {{checkpoints}} and 
{{__consumer_offsets}} topics.

*Problem:*
For customers who only use MM2 for mirroring data and not interested in syncing 
offsets feature they now can disable the second part of this feature which is 
by disabling {{emit.checkpoints.enabled}} and/or {{sync.group.offsets.enabled}} 
to disable emitting {{__consumer_offsets}} topic but nothing disabling 1st part 
of the feature. 

The problem get worse if they disabled MM2 from creating offset syncs internal 
topic as 
1. this will increase throughput as MM2 will try to force trying to update the 
offset with every mirrored batch which impacting the performance of our MM2.
2. Get too many error logs because they don't create the sync offset topic as 
they don't use the feature.

*Possible solution:*
Allow customers to fully disable the feature if they don't really need it 
similar to how we fully can disable other MM2 features like heartbeat feature 
by adding a new config.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [Discuss] KIP-1019: Expose method to determine Metric Measurability

2024-02-14 Thread Andrew Schofield
Hi Apoorv,
Thanks for the KIP. Looks like a useful change to tidy up the metrics code.

Thanks,
Andrew

> On 14 Feb 2024, at 14:55, Apoorv Mittal  wrote:
>
> Hi,
> I would like to start discussion of a small KIP which fills a gap in
> determining Kafka Metric measurability.
>
> KIP-1019: Expose method to determine Metric Measurability
> 
>
> Regards,
> Apoorv Mittal



[Discuss] KIP-1019: Expose method to determine Metric Measurability

2024-02-14 Thread Apoorv Mittal
Hi,
I would like to start discussion of a small KIP which fills a gap in
determining Kafka Metric measurability.

KIP-1019: Expose method to determine Metric Measurability


Regards,
Apoorv Mittal


[jira] [Resolved] (KAFKA-16247) replica keep out-of-sync after migrating broker to KRaft

2024-02-14 Thread Luke Chen (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-16247?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Luke Chen resolved KAFKA-16247.
---
Resolution: Fixed

Fixed in 3.7.0 RC4

> replica keep out-of-sync after migrating broker to KRaft
> 
>
> Key: KAFKA-16247
> URL: https://issues.apache.org/jira/browse/KAFKA-16247
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.7.0
>Reporter: Luke Chen
>Priority: Major
> Attachments: KAFKA-16247.zip
>
>
> We are deploying 3 controllers and 3 brokers, and following the steps in 
> [doc|https://kafka.apache.org/documentation/#kraft_zk_migration]. When we're 
> moving from "Enabling the migration on the brokers" state to "Migrating 
> brokers to KRaft" state, the first rolled broker becomes out-of-sync and 
> never become in-sync. 
> From the log, we can see some "reject alterPartition" errors, but it just 
> happen 2 times. Theoretically, the leader should add the follower  into ISR 
> as long as the follower is fetching since we don't have client writing data. 
> But can't figure out why it didn't fetch. 
> Logs: https://gist.github.com/showuon/64c4dcecb238a317bdbdec8db17fd494
> ===
> update Feb. 14
> After further investigating the logs, I think the reason why the replica is 
> not added into ISR is because the alterPartition request got non-retriable 
> error from controller:
> {code:java}
> Failed to alter partition to PendingExpandIsr(newInSyncReplicaId=0, 
> sentLeaderAndIsr=LeaderAndIsr(leader=1, leaderEpoch=4, 
> isrWithBrokerEpoch=List(BrokerState(brokerId=1, brokerEpoch=-1), 
> BrokerState(brokerId=2, brokerEpoch=-1), BrokerState(brokerId=0, 
> brokerEpoch=-1)), leaderRecoveryState=RECOVERED, partitionEpoch=7), 
> leaderRecoveryState=RECOVERED, 
> lastCommittedState=CommittedPartitionState(isr=Set(1, 2), 
> leaderRecoveryState=RECOVERED)) because the partition epoch is invalid. 
> Partition state may be out of sync, awaiting new the latest metadata. 
> (kafka.cluster.Partition) 
> [zk-broker-1-to-controller-alter-partition-channel-manager]
> {code}
> Since it's a non-retriable error, we'll keep the state as pending, and 
> waiting for later leaderAndISR update as described 
> [here|https://github.com/apache/kafka/blob/d24abe0edebad37e554adea47408c3063037f744/core/src/main/scala/kafka/cluster/Partition.scala#L1876C1-L1876C41].
> Log analysis: https://gist.github.com/showuon/5514cbb995fc2ae6acd5858f69c137bb
> So the question becomes:
> 1. Why does the controller increase the partition epoch?
> 2. When the leader receives the leaderAndISR request from the controller, it 
> ignored the request because the leader epoch is identical, even though the 
> partition epoch is updated. Is the behavior expected? Will it impact the 
> alterPartition request later?



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #2644

2024-02-14 Thread Apache Jenkins Server
See 




[jira] [Created] (KAFKA-16253) kafka_server_alterPartition_metrics_network_io_total is not supported in kafka 3.5.1

2024-02-14 Thread Janardhana Gopalachar (Jira)
Janardhana Gopalachar created KAFKA-16253:
-

 Summary: kafka_server_alterPartition_metrics_network_io_total is 
not supported in kafka 3.5.1
 Key: KAFKA-16253
 URL: https://issues.apache.org/jira/browse/KAFKA-16253
 Project: Kafka
  Issue Type: Bug
  Components: core
Affects Versions: 3.5.1
Reporter: Janardhana Gopalachar


Hi 

The metrics  below was supported in kafka 3.2.3 but in 3.5.1 it is not sent . I 
tried to search the source code to see if this is present, but couldnt find a 
reference. 

Is the below metrics deprecated or not supported. ?
kafka_server_alterPartition_metrics_network_io_total



--
This message was sent by Atlassian Jira
(v8.20.10#820010)