Re: Kafka retention bug (?)

2024-05-14 Thread Luke Chen
Hi Nicholas, I didn't know anything in v3.7.0 would cause this issue. It would be good if you could open a JIRA for it. Some info to be provided: 1. You said "in the past", what version of Kafka was it using? 2. What is your broker configuration? 3. KRaft mode? Combined mode? (controller + broker

Re: [ANNOUNCE] New committer: Igor Soarez

2024-04-24 Thread Luke Chen
Congrats, Igor! On Thu, Apr 25, 2024 at 6:10 AM Matthias J. Sax wrote: > Congrats! > > On 4/24/24 2:29 PM, Bill Bejeck wrote: > > Congrats Igor! > > > > -Bill > > > > On Wed, Apr 24, 2024 at 2:37 PM Tom Bentley wrote: > > > >> Congratulations Igor! > >> > >> On Thu, 25 Apr 2024 at 6:27 AM,

Re: Kraft controller readiness checks

2024-04-21 Thread Luke Chen
P readiness check > and the PDB, we are guaranteed not to cause any problem read or write > errors. Rotations are rather long, but we don’t really care about speed. > > Thanks, > > Frank > > -- > Francesco Burato | Software Development Engineer | Adobe | > bur...@adobe.

Re: Kraft controller readiness checks

2024-04-18 Thread Luke Chen
Hello Frank, That's a good question. I think we all know there is no "correct" answer for this question. But I can share with you what our team did for it. Readiness: controller is listening on the controller.listener.names The rationale behind it is: 1. The last step for the controller node

Re: production ready for zookeeper to kraft migration

2024-04-03 Thread Luke Chen
Hi Matthieu, Yes, the ZK migrating to KRaft feature is already GA in v3.6.0. Sorry, we forgot to update the document in the Kafka-site repo. I've filed a PR for it: https://github.com/apache/kafka-site/pull/594 Thanks. Luke On Thu, Apr 4, 2024 at 6:14 AM Matthieu Patou wrote: > I looked at

Re: Replicas not equally distributed within rack

2024-03-28 Thread Luke Chen
Hi Abhishek, For Zookeeper's mode, there's no workaround, unfortunately. But you can upgrade your cluster to the latest kafka version (v3.7.0) to migrate to KRaft mode. For your information, in KRaft mode, we use `StripedReplicaPlacer`, which will fix your problem. ref:

Re: Possible bug on Kafka documentation

2024-02-21 Thread Luke Chen
Hi Federico, Thanks for reporting the issue. We've fixed that in v3.5 and later in this PR: https://github.com/apache/kafka/pull/13115. But we didn't update for the older versions of docs. Are you willing to file a PR to kafka-site repo to fix that? Or create a JIRA issue for it? Thanks. Luke

Re: Can a controller in a kafka kraft cluster be a bootstrap server

2023-12-12 Thread Luke Chen
on down node is not routing to other running > node, It takes me always to restart the services, > Running on kafka version 3.2.1 (kraft mode) > > On Mon, Dec 11, 2023 at 12:33 PM Luke Chen wrote: > > > Hi Dima, > > > > You can set "process.roles=controller,broker" t

[ANNOUNCE] Apache Kafka 3.5.2

2023-12-11 Thread Luke Chen
, Levani Kokhreidze, Lucas Brutschy, Luke Chen, Manikumar Reddy, Matthias J. Sax, Mickael Maison, Nick Telford, Okada Haruki, Omnia G.H Ibrahim, Robert Wagner, Rohan, Said Boudjelda, sciclon2, Vincent Jiang, Xiaobing Fang, Yash Mayya We welcome your help and feedback. For more information on how

Re: Kafka frequent disconnection issue

2023-12-10 Thread Luke Chen
Hi Ankit, We can't see the log snippet. But it looks normal to disconnect when connections.max.idle.ms expires. When increasing the connections.max.idle.ms value, there might be some activities in the connection during this time (ex: at 5 min 10 sec), so the idle timer is reset. Thanks. Luke On

Re: Can a controller in a kafka kraft cluster be a bootstrap server

2023-12-10 Thread Luke Chen
Hi Dima, You can set "process.roles=controller,broker" to get what you want. Otherwise, the controller role cannot be served as a broker. Thanks. Luke On Sat, Dec 9, 2023 at 3:59 AM Dima Brodsky wrote: > Hello, > > Would the following configuration be valid in a kafka kraft cluster > > So

Re: [ANNOUNCE] Apache Kafka 3.6.1

2023-12-08 Thread Luke Chen
a, > Federico Valeri, Greg Harris, Guozhang Wang, Hao Li, hudeqi, > iit2009060, Ismael Juma, Jorge Esteban Quilcate Otoya, Josep Prat, > Jotaniya Jeel, Justine Olshan, Kamal Chandraprakash, kumarpritam863, > Levani Kokhreidze, Lucas Brutschy, Luke Chen, Manikumar Reddy, > Matthias J. Sax,

Re: Kafka 2.7.2 to 3.5.1 upgrade

2023-12-05 Thread Luke Chen
Hi Lud, This is a known issue(KAFKA-15353 ) and I've fixed it in v3.5.2 (will get released soon) and v3.6.0. Thanks. Luke On Mon, Dec 4, 2023 at 6:01 PM Lud Antonie wrote: > Hi Megh, > > No, the number of partitions haven't increased. > The

[VOTE] 3.5.2 RC1

2023-11-21 Thread Luke Chen
Hello Kafka users, developers and client-developers, This is the first candidate for release of Apache Kafka 3.5.2. This is a bugfix release with several fixes since the release of 3.5.1, including dependency version bumps for CVEs. Release notes for the 3.5.2 release:

Re: How to dynamically change configurations in the controllers

2023-11-15 Thread Luke Chen
Hi Jesus, KIP-919 is what you're looking for: https://cwiki.apache.org/confluence/display/KAFKA/KIP-919%3A+Allow+AdminClient+to+Talk+Directly+with+the+KRaft+Controller+Quorum+and+add+Controller+Registration This feature will be included in next release (i.e. Kafka v3.7.0). Thanks. Luke On Thu,

Re: [ANNOUNCE] Apache Kafka 3.6.0

2023-10-12 Thread Luke Chen
Selenge, Gaurav Narula, GeunJae Jeon, Greg Harris, Guozhang > > Wang, Hailey Ni, Hao Li, Hector Geraldino, hudeqi, hzh0425, Iblis Lin, > > iit2009060, Ismael Juma, Ivan Yurchenko, James Shaw, Jason Gustafson, > > Jeff Kim, Jim Galasyn, John Roesler, Joobi S B, Jorge Esteban

Re: [ANNOUNCE] Apache Kafka 3.6.0

2023-10-11 Thread Luke Chen
Gustafson, > Jeff Kim, Jim Galasyn, John Roesler, Joobi S B, Jorge Esteban Quilcate > Otoya, Josep Prat, Joseph (Ting-Chou) Lin, José Armando García Sancio, > Jun Rao, Justine Olshan, Kamal Chandraprakash, Keith Wall, Kirk True, > Lianet Magrans, LinShunKang, Liu Zeyu, lixy, Lucas Bradstre

Re: Kafka Protocol : Compact Array or Array ?

2023-10-02 Thread Luke Chen
Hi Neeraj, Yes, for MetadataRequest, version 0 ~ 8, the topic is ARRAY type. After version 9, it'll be COMPACT_ARRAY. It's because of this definition: "flexibleVersions": "9+". You can check KIP-482 for more information: https://cwiki.apache.org/confluence/display/KAFKA/KIP-482 Thanks. Luke On

Re: [VOTE] 3.6.0 RC2

2023-10-02 Thread Luke Chen
Hi Satish, I verified with: 1. Ran quick start in KRaft for scala 2.12 artifact 2. Making sure the checksum are correct 3. Browsing release notes, documents, javadocs, protocols. 4. Verified the tiered storage feature works well. +1 (binding). Thanks. Luke On Mon, Oct 2, 2023 at 5:23 AM

Re: [kafka-clients] [VOTE] 3.6.0 RC1

2023-09-25 Thread Luke Chen
Hi Jose, Sounds good to me. Let's have further discussion in JIRA/PR, and target to v3.6.1/v3.7.0. Thanks. Luke On Tue, Sep 26, 2023 at 1:35 AM José Armando García Sancio wrote: > On Sat, Sep 23, 2023 at 3:08 AM Luke Chen wrote: > > > > Hi Satish, > > > >

Re: [kafka-clients] [VOTE] 3.6.0 RC1

2023-09-24 Thread Luke Chen
s? > > > > 1. https://issues.apache.org/jira/browse/KAFKA-15489 > > > > Thanks, > > Satish. > > > > On Sat, 23 Sept 2023 at 03:08, Luke Chen wrote: > > > > > > Hi Satish, > > > > > > I found the current KRaft implemen

Re: [kafka-clients] [VOTE] 3.6.0 RC1

2023-09-23 Thread Luke Chen
Hi Satish, I found the current KRaft implementation will have "split brain" issue when network partition happens, which will cause inconsistent metadata returned from the controller. Filed KAFKA-15489 for this issue, and PR

Re: Ruquest for Jira assign permissions

2023-08-10 Thread Luke Chen
Hi Romulo, You should be able to assign tasks to yourselves. Thanks. Luke On Sat, Aug 5, 2023 at 9:10 PM Romulo Oliveira wrote: > I added a new Jira task in order to contribute to a new feature for Kafka > Connectors TimestampConverter transformer: >

Re: Unable to launch KRaft

2023-07-28 Thread Luke Chen
Hi Adrian, Please follow the quick start guide in Kafka doc: https://kafka.apache.org/quickstart Thanks. Luke On Fri, Jul 28, 2023 at 3:53 PM adrien ruffie wrote: > dear all, > > I would like to try KRaft in but I didn't arrived to finish the > installation bellow: > > >

Re: [VOTE] 3.5.1 RC1

2023-07-18 Thread Luke Chen
Hi Divij, I've run: 1. Download kafka_2.12-3.5.1.tgz 2. Run quick start using KRaft mode 3. Verified the checksum 4. Sanity check the javadoc All looks good. +1 (binding) Thanks. Luke On Tue, Jul 18, 2023 at 5:15 AM Chris Egerton wrote: > Hi Divij, > > Thanks for running this release! > > To

Re: Replication factor for a topic Increase

2023-07-13 Thread Luke Chen
, > GP > > On Thu, 13 Jul, 2023, 13:54 Luke Chen, wrote: > > > Hi Gaurav, > > > > > Why do I see topic ReplicationFactor:1 during description of a topic ? > > I think you should set this config: `default.replication.factor` > > > > > And how can w

Re: Replication factor for a topic Increase

2023-07-13 Thread Luke Chen
Hi Gaurav, > Why do I see topic ReplicationFactor:1 during description of a topic ? I think you should set this config: `default.replication.factor` > And how can we increase replication factor? I have 3 Kafka brokers running on 2.7.0 version. Please check this doc:

Re: [KRaft] Clarification about Deploying Considerations (Kafka docs)

2023-07-06 Thread Luke Chen
Hi Grigorios I've asked the author in the JIRA: https://issues.apache.org/jira/browse/KAFKA-14207 Thanks. Luke On Wed, Jul 5, 2023 at 10:31 PM Grigorios Avgitidis wrote: > Hi, > > On the KRaft section of the Kafka documentation, related with `Deploying > Considerations`, in one of the points

Re: [ANNOUNCE] Apache Kafka 3.5.0

2023-06-15 Thread Luke Chen
a, Ivan Yurchenko, Jakub Scholz, Jason Gustafson, Jeff > Kim, Jim Galasyn, Jorge Esteban Quilcate Otoya, Josep Prat, José Armando > García Sancio, Juan José Ramos, Junyang Liu, Justine Olshan, Kamal > Chandraprakash, Kirk True, Kowshik Prakasam, littlehorse-eng, liuzc9, Lucas > Brutschy, Luci

Re: Process to Upgrade Zookeeper from 2.7.0 to 3.4.1

2023-06-14 Thread Luke Chen
Hi Gaurav, Please check Zookeeper's doc for upgrading guide. Thanks. Luke On Wed, Jun 14, 2023 at 12:03 PM Gaurav Pande wrote: > Hi Guys, > > Could anyone help on this query? > > Regards, > Gaurav > > On Tue, 13 Jun, 2023, 11:40 Gaurav Pande, wrote: > > > Hello Guys, > > > > Iam new in this

[ANNOUNCE] Apache Kafka 3.4.1

2023-06-06 Thread Luke Chen
, José Armando García Sancio, Lucia Cerchie, Luke Chen, Manikumar Reddy, Matthias J. Sax, Mickael Maison, Philip Nee, Purshotam Chauhan, Rajini Sivaram, Ron Dagostino, Terry, Victoria Xia, Viktor Somogyi-Vass, Yash Mayya We welcome your help and feedback. For more information on how to report problems

Re: kafka 3.4.0 sasl_PLAINTEXT kafka-metadata-quorum.sh Unexpected Kafka request of type METADATA during SASL handshake

2023-06-06 Thread Luke Chen
Hi, I've replied you in your previous email. https://lists.apache.org/thread/ogzpxtnsyklp7q82xh5t0y58rgtpc18x Please take a look. Thanks. Luke On Tue, Jun 6, 2023 at 5:13 PM Yj Jia wrote: > > > kafka 3.4.0 sasl_PLAINTEXT exec

Re: [VOTE] 3.5.0 RC1

2023-06-06 Thread Luke Chen
Hi Mickael, I ran the following validation steps: - Built from source with Java 17 and Scala 2.13 - Signatures and hashes of the artifacts generated - Navigated through Javadoc including links to JDK classes - Run the quickstart in KRaft and Zookeeper mode +1 (binding) from me. Thanks for

Re: [VOTE] 3.4.1 RC3

2023-06-05 Thread Luke Chen
ldn't find an existing issue for this one. I note that > others > > seem to have run the integration tests without problems, so I don't think > > this is a blocker. I also did the Kafka, Connect and Streams quickstarts. > > > > +1 binding. > > > > Tom > > >

Re: kafka 3.4.0 sasl_PLAINTEXT exec kafka-metadata-quorum.sh Unexpected Kafka request of type METADATA during SASL handshake.

2023-06-02 Thread Luke Chen
Hi Yj, Thanks for reporting the issue. This is a known issue will be fixed in v3.4.1, which is planning to release next week (hopefully). JIRA: https://issues.apache.org/jira/browse/KAFKA-14711 Thanks. Luke On Thu, Jun 1, 2023 at 9:02 PM Yj Jia wrote: > kafka 3.4.0 sasl_PLAINTEXT exec

Re: [VOTE] 3.4.1 RC3

2023-06-01 Thread Luke Chen
> On Sat, May 27, 2023 at 12:58 PM Jakub Scholz wrote: > > > > > > +1 (non-binding) ... I used the staged binaries and Maven artifacts to > > run > > > my tests and all seems to work fine. > > > > > > Thanks for running the release. > >

Re: client.dns.lookup=”use_all_dns_ips” cache ttl

2023-06-01 Thread Luke Chen
Hi George, I believe the JVM will have cache to this DNS records. And maybe OS will also cache it. Had a quick look, Kafka doesn't cache it. Thanks. Luke On Thu, Jun 1, 2023 at 1:21 AM George Goh wrote: > Hi! > > I’m considering to use the following configuration for my producers and >

[VOTE] 3.4.1 RC3

2023-05-26 Thread Luke Chen
Hello Kafka users, developers and client-developers, This is the 4th candidate for release of Apache Kafka 3.4.1. This is a bugfix release with several fixes since the release of 3.4.0. A few of the major issues include: - core KAFKA-14644

[VOTE] 3.4.1 RC2

2023-05-24 Thread Luke Chen
Hello Kafka users, developers and client-developers, This is the 3rd candidate for release of Apache Kafka 3.4.1. This is a bugfix release with several fixes since the release of 3.4.0. A few of the major issues include: - core KAFKA-14644

Re: Re: Re: Kafka 3.2.1 performance issue with JDK 11

2023-05-22 Thread Luke Chen
Hi Vic, Yes, JDK 8 is still supported in kafka 3.x.x. It'll become unsupported after kafka 4.0.0. Thanks. Luke On Tue, May 23, 2023 at 9:56 AM Vic Xu wrote: > Hi Greg > > I know JDK 8 will solve the problem certainly, but I wondering if JDK8 has > been deprecated after Kafka 3? Can I use JDK

[VOTE] 3.4.1 RC1

2023-05-22 Thread Luke Chen
Hello Kafka users, developers and client-developers, This is the 2nd candidate for release of Apache Kafka 3.4.1. This is a bugfix release with several fixes since the release of 3.4.0. A few of the major issues include: - core KAFKA-14644

Re: [VOTE] 3.4.1 RC0

2023-05-21 Thread Luke Chen
> >>> Looks like the Maven artifacts are not in staging: > >>> > >>> > https://repository.apache.org/content/groups/staging/org/apache/kafka/kafka-clients/3.4.1/ > >>> > >>> Documentation still has 3.4.0, instead of 3.4.1 (not sure if this will

Re: kafka acl issue

2023-05-19 Thread Luke Chen
Hi Hari, You might want to ask in the client repo (kafkajs?) They should be able to help you. Thanks. Luke On Fri, May 19, 2023 at 3:00 PM HariBabu kuruva wrote: > Hi All, > > I am trying to implement kafka acl for one of the topics. > it's a kafka cluster with 1 broker. > > Below are the

[VOTE] 3.4.1 RC0

2023-05-16 Thread Luke Chen
Hello Kafka users, developers and client-developers, This is the first candidate for release of Apache Kafka 3.4.1. This is a bugfix release with several fixes since the release of 3.4.0. A few of the major issues include: - core KAFKA-14644

Re: Trying to reduce the number of replicates from 9 to 3

2023-05-15 Thread Luke Chen
Hi Mich, You might want to take a look at this section: "Increasing replication factor" in the doc: https://kafka.apache.org/documentation/#basic_ops_increase_replication_factor Simply put, the json file provided in kafka-reassign-partitions.sh should show the final replicas assignment after

Re: CVEs related to Kafka

2023-05-09 Thread Luke Chen
Hi Sahil, > in which version of Kafka these will be fixed https://issues.apache.org/jira/browse/KAFKA-14320 https://issues.apache.org/jira/browse/KAFKA-14107 https://issues.apache.org/jira/browse/KAFKA-14256 Maybe you can try to search the JIRA first next time. :) Thank you. Luke On Wed, May

Re: KRaft with even number of controllers

2023-05-05 Thread Luke Chen
Hi De Gao, Usually, we won't compare the performance with even/odd cases. I think the most important reason we don't recommend using even number of nodes is the node failure tolerance. In 1 nodes case, the majority is 1, so we can't lose any node In 2 nodes case, the majority is 2, so we can't

Re: Kafka topic __consumer_offsets replication issue

2023-05-01 Thread Luke Chen
Hi kiran, I would check the log end offset of the in-sync partition first, and check the lags of the offsets with the leader offset. (You can check by metric: `kafka.log:type=Log,name=LogEndOffset,topic=xxx, partition=xx`) Then, I would check if the follower is doing fetching without error by

Re: Kafka support for IPV6 only Networking stack

2023-04-27 Thread Luke Chen
Hi Senjoorian, I've never tested it, but AFAIK, Kafka supports IPv6. There's also a KIP to allow Kafka to accept duplicate listener on port for IPv4/IPv6: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=195726330 Maybe you'll be interested. Thank you. Luke On Thu, Apr 27, 2023

Re: Addition to Kafka contributor group

2023-04-22 Thread Luke Chen
Hi Atul, I've added you to the contribution group in JIRA. I didn't find your account in wiki system. Let me know if you need wiki permission to create a KIP. Thank you. Luke On Fri, Apr 21, 2023 at 10:15 PM Atul Sharma wrote: > Hi Team, > > I would like to contribute to open source Kafka,

Re: Kafka Node Shutting Down Automatically

2023-04-22 Thread Luke Chen
Hi Akshay, Thanks for reporting the issue. It looks like a bug. Could you open a JIRA ticket to track it? Thank you. Luke On Fri, Apr 21, 2023 at 10:16 PM Akshay Kumar wrote: > Hello team, > >- We are using the zookeeper less Kafka (kafka

Re: About CVE-2023-25194

2023-03-29 Thread Luke Chen
Hi, This is the commit to fix the CVE: https://github.com/apache/kafka/commit/ae22ec1a0ea005664439c3f45111aa34390ecaa1 2.x upgrades to 3.x includes a major version upgrade, so it'll have some compatibility issues. Please check the notable changes for v3.0 here:

Re: Questions about creating jira account

2023-03-19 Thread Luke Chen
Hi Jimmy, You can create an account request via https://selfserve.apache.org/jira-account.html Thank you. Luke On Sat, Mar 18, 2023 at 12:03 AM zw wrote: > Hi, > It seems that JIRA access are disabled by default and I can't create an > account by myself. > > I would appreciate that if anyone

Re: Question about KRaft

2023-03-09 Thread Luke Chen
For questions related to confluent, I think you'd better ask in their channel. Luke On Fri, Mar 10, 2023 at 12:54 PM sunil chaudhari < sunilmchaudhar...@gmail.com> wrote: > Hi Luke, > This docu is good. > Does it apply for confluent as well? > > > > On Fri, 10 Mar

Re: Question about KRaft

2023-03-09 Thread Luke Chen
Hi Zhenyu, Answering your question: > Should I simply 1. download 3.4 binary 2. stop ZK & Kafka service 3. upgrade Kafka to 3.4 4. start only Kafka service with KRaft server.properties That is not migrating, actually. That is just creating another kafka cluster in KRaft mode. The point for

Re: Kafka metrics - Requests.

2023-02-24 Thread Luke Chen
Hi David, It did look like a bug. Could you file a bug in JIRA? And if you have time, welcome to investigate and submit PR for it. (My guess is there are some internal topics are included in `TotalFetchRequestsPerSec`, but not included in another one, but not sure) Thank you. Luke On Sat, Feb

Re: kafka-producer-perf-test.sh maximum throughput limits for broker on single node

2023-02-24 Thread Luke Chen
Hi Tomasz, There are some configurations needed to be tuned in producer side. Try searching kafka producer tuning, you should find many good articles describing it. Good luck. Luke On Fri, Feb 24, 2023 at 6:16 AM Tomasz Sąciński < tomasz.sacin...@linuxpolska.pl> wrote: > Hello users. > > I

Re: Release plan for kafka-python

2023-02-14 Thread Luke Chen
Hi Sahil, You should ask in kafka-python repo, not here. Thank you. Luke On Tue, Feb 14, 2023 at 7:02 PM Sahil Sharma D wrote: > Hi team, > > As per understanding its last release was released in 2020, after that > there is no release. We are planning to use v2.0.2, if we face any issue in >

Re: Server running issue

2023-02-13 Thread Luke Chen
Hi Shivannad, It said there's some exception during reading/writing data from/to /tmp/kafka-logs dir. There should be some errors in the log above these lines. That should be the root cause. Luke On Mon, Feb 13, 2023 at 11:51 PM Shivanand K Badiger wrote: > > Hi Team, > > I am facing some

Re: RequestsPerSec version

2023-02-10 Thread Luke Chen
> Do I understand correctly that messages from 9+ do not follow the same conversion as before? No, there's no change for messages 9+. So, I think your understanding is correct. When client and server are using the same compression codec, it won't do conversion. What you need to do now is to make

Re: Re: RequestsPerSec version

2023-02-09 Thread Luke Chen
ion Availability > "transaction.state.log.replication.factor" = 3 > "transaction.state.log.min.isr"= 1 > > # Storage Optimization > "compression.type" = "zstd" > ``` > > I tried disabling zstd, but ProduceMessageConversionsPer

Re: RequestsPerSec version

2023-02-08 Thread Luke Chen
Hi Gonzalo, For the produce request record version, you should refer to this file: https://github.com/apache/kafka/blob/trunk/clients/src/main/resources/common/message/ProduceRequest.json#L35 But you're right, basically the message conversion happened in a very old produce request version (ex:

Re: Certificate renewal issue

2023-01-10 Thread Luke Chen
Hi Sandeep, Maybe it'd be better to ask in strimzi community? Luke On Tue, Jan 10, 2023 at 5:57 PM Sandeep M wrote: > Hi all, > > We are using strimzi operator version 0.21.1 and Kubernetes 1.23.1. Broker > certs are not getting renewed as a result we are facing issue. Let me know > how to

Re: Topic partition Leader: none

2023-01-10 Thread Luke Chen
Hi, That sounds similar to this issue: KAFKA-14190 . Could you help confirm it and comment in the JIRA? That will let contributors know this is an important issue bothering many users. Thank you. Luke On Tue, Jan 10, 2023 at 10:08 AM megh

Re: log.cleaner.io.max.bytes.per.second

2022-12-28 Thread Luke Chen
% of total > time) > Start size: 1 372,1 MB (10 329 108 messages) > End size: 230,0 MB (1 000 105 messages) > 83,2% size reduction (90,3% fewer messages) > > Le mer. 28 déc. 2022 à 09:41, Luke Chen a écrit : > > > Hi Nicolas, > > > > The

Re: log.cleaner.io.max.bytes.per.second

2022-12-28 Thread Luke Chen
Hi Nicolas, The throttler in log cleaner is just a simple throttler to control the IO speed by periodically (every 300ms) check the IO rate so far, and sleep some time to slow it down. That is, in your case, it could be: log cleaner runs for 300ms with full IO speed -> check -> throttle (sleep to

Re: [ANNOUNCE] New committer: Josep Prat

2022-12-20 Thread Luke Chen
Congratulations, Josep! Luke On Wed, Dec 21, 2022 at 6:26 AM Viktor Somogyi-Vass wrote: > Congrats Josep! > > On Tue, Dec 20, 2022, 21:56 Matthias J. Sax wrote: > > > Congrats! > > > > On 12/20/22 12:01 PM, Josep Prat wrote: > > > Thank you all! > > > > > > ——— > > > Josep Prat > > > > > >

Re: KRaft? cannot recreate a particular topic

2022-12-20 Thread Luke Chen
Hi Simon, This is a known issue in kafka v3.3.1, and will be fixed in kafka v3.3.2 and v3.4.0. Here's the JIRA issue: https://issues.apache.org/jira/browse/KAFKA-14337 Thank you Luke On Tue, Dec 20, 2022 at 7:13 PM Simon Dahlbacka wrote: > I am running a single node cluster in KRaft mode on >

Re: Granting permission for Create KIP and contribute to kafka

2022-11-27 Thread Luke Chen
Hi t-mac, I've granted your JIRA account. But I can't find the wiki id: "ws" in the wiki system. Are your sure you registered with "ws" here ? Any more info you can provide to us, like account full name? Thank you.

Re: Leader election strategy

2022-11-14 Thread Luke Chen
Hi Pierre, Try using kafka-reassign-partitions.sh to reassign partitions to different replicas you like. ref: https://kafka.apache.org/documentation/#basic_ops_automigrate Luke On Mon, Nov 14, 2022 at 3:55 PM Pierre Coquentin wrote: > Hello, > We have a Kafka cluster (2.4.1) with a

Re: Connection issue

2022-11-14 Thread Luke Chen
Hi shikha, I think you should ask in Spark community. Thanks Luke On Tue, Nov 15, 2022 at 3:17 AM shikha sharma wrote: > Hello, > > I am trying to connect to kafka using this command: > orderRawData = spark.readStream \ > .format("kafka") \ > .option("kafka.bootstrap.servers",

Re: [ANNOUNCE] New Kafka PMC Member: Bruno Cadonna

2022-11-01 Thread Luke Chen
Congrats Bruno! Well deserved! Luke On Wed, Nov 2, 2022 at 10:07 AM John Roesler wrote: > Congratulations, Bruno!!! > > On Tue, Nov 1, 2022, at 15:16, Lucas Brutschy wrote: > > Wow, congratulations! > > > > On Tue, Nov 1, 2022 at 8:55 PM Chris Egerton > wrote: > >> > >> Congrats! > >> > >> On

Re: Help with MM2 active/passive configuration

2022-10-24 Thread Luke Chen
This is what Ryanne mentioned doc: https://github.com/apache/kafka/tree/trunk/connect/mirror Thanks. Luke On Mon, Oct 24, 2022 at 6:38 PM Chris Peart wrote: > > > Hi Ryanne, > > I cannot find what you mentioned, all i'm looking for are some example > configurations for and active/passive

Re: in case of disk failure,why not recover from middle position of the log file

2022-10-13 Thread Luke Chen
Hi Zhang, Yes, when doing log recovery, we'll start from the checkpoint for the partition in `recovery-point-offset-checkpoint` file, which will be updated at runtime. Is that what you expected? Check LogManager#loadLogs method for more implementation detail. Thanks. Luke On Thu, Oct 13, 2022

Re: consumer loses offset

2022-10-10 Thread Luke Chen
abled every 5 seconds? > We are checking daily and the offset is always set to NULL. > > > > Lorenzo Rovere > > Technology Reply > Via Avogadri, 2 > 31057 - Silea (TV) - ITALY > phone: +39 0422 1836521 > l.rov...@reply.it > www.reply.it > -Original Message--

Re: Apply to be a contributor for Kafka

2022-10-10 Thread Luke Chen
Hi Divya, Done. Thanks for the interest in Apache Kafka. Luke On Mon, Oct 10, 2022 at 6:20 AM Divya A L wrote: > I’m a developer of kafka, and I want to contribute for the project. Can I > be added as a contributor, as I would like to pick up an issue from the > jira board. > > jira ID :

Re: consumer loses offset

2022-10-07 Thread Luke Chen
Hi Lorenzo, Sounds like it is caused by this bug: https://issues.apache.org/jira/browse/KAFKA-13636 If you're not in the versions of fixed version list or newer, please try to upgrade it. Thanks. Luke On Fri, Oct 7, 2022 at 5:36 PM Lorenzo Rovere wrote: > Hi everyone, I have a simple question

Re: Metadata Refresh and TimeoutException when MAX_BLOCK_MS_CONFIG set 0

2022-09-25 Thread Luke Chen
Hi Bhavesh, I understand your point. There was an old KIP with the similar idea which was not accepted by the community in the end. Maybe you can try to bring it back to the community again, or try to propose your own KIP for this idea?

Re: Kafka consumer ensure offsets committed before partition rebalance to avoid duplicates

2022-09-20 Thread Luke Chen
eStickyAssignore help here to > wait for process of 100 records and commit their offsets before moving the > partition to new consumer? Looks like i am missing something > > On Fri, Sep 16, 2022 at 7:59 AM Luke Chen wrote: > > > Hi Pushkar, > > > > Here's the answer to

Re: CVE-2022-34917: Unauthenticated clients may cause OutOfMemoryError on Apache Kafka Brokers

2022-09-20 Thread Luke Chen
What a great finding! Thanks Mickael Maison, Tom Bentley and Daniel Collins! And thanks for all the release managers who help drive all these security patch releases! Luke On Mon, Sep 19, 2022 at 11:53 PM Manikumar wrote: > Severity: High > > Description: > > A security vulnerability has

Re: Kafka consumer ensure offsets committed before partition rebalance to avoid duplicates

2022-09-15 Thread Luke Chen
Hi Pushkar, Here's the answer to your questions: > 1. During scale-down operation, I am adding a shutdown hook to the Java Runtime, and calling close on the consumer. As per kafka docs, close provides 30 sec to commit current offsets if auto.commit is enabled: so, i assume that it will process

Re: Add to kafka contributor list

2022-09-05 Thread Luke Chen
Hi Vinay, Done. Thanks for the interest in Apache Kafka. Luke On Mon, Sep 5, 2022 at 1:42 PM vinay kumar wrote: > Hi Bill, > > My jira user : vvakati > > Can you add me to contributor list please? > > Thanks, > Vinay > > On Fri, 2 Sep 2022 at 12:09 AM, Bill Bejeck > wrote: > > > Hi Vinay, >

Re: Unable to reset kafka offset in a consumer group

2022-08-29 Thread Luke Chen
mer group was not inactive even > with all the consumers stopped. > Managed to work around this now. > > Many Thanks > Chris > > > On 25 Aug 2022, at 10:12 am, Luke Chen wrote: > > Hi Chris, > > > >> is there a way to force the offset forward by an incr

Re: Unable to reset kafka offset in a consumer group

2022-08-25 Thread Luke Chen
Hi Chris, > is there a way to force the offset forward by an increment of 1 just for this topic? kafka-consumer-groups.sh script has an option `--shift-by` to shift current offset by 'n'. I think this is what you are looking for. Thank you Luke On Thu, Aug 25, 2022 at 4:25 PM Chris Peart

Re: Regarding kafka 2.3.0

2022-08-25 Thread Luke Chen
1. Is kafka 2.3.0 going end of life ,If yes then what is the expected date? -> Kafka supports last 3 releases. REF: https://cwiki.apache.org/confluence/display/KAFKA/Time+Based+Release+Plan#TimeBasedReleasePlan-WhatIsOurEOLPolicy ? 2. Is kafka 3.1.0 backward compatible to 2.3.0? -> Since 2.3 to

Re: Fw: An existing connection was forcibly closed by the remote host

2022-08-03 Thread Luke Chen
Hi Podunk, If you're saying the error: ERROR Exiting JVM with code 0 (org.apache.zookeeper.util.ServiceUtils), I think this is a bug in Kafka. I guess it's because we didn't close ZooKeeperAdmin before exit. Please open a JIRA ticket for it, and welcome to file a PR for it! But for the ZK shell,

Re: [ANNOUNCE] New Kafka PMC Member: A. Sophie Blee-Goldman

2022-08-01 Thread Luke Chen
Congrats Sophie! :) Luke On Tue, Aug 2, 2022 at 7:56 AM Adam Bellemare wrote: > Congratulations Sophie! I’m glad to see you made as a PMC member! Well > earned. > > > On Aug 1, 2022, at 6:42 PM, Guozhang Wang wrote: > > > > Hi everyone, > > > > I'd like to introduce our new Kafka PMC member,

Re: Kafka certificate monitoring

2022-07-27 Thread Luke Chen
Hi Sandeep, AFAIK, Kafka doesn't expose this kind of metrics. I did a quick search, and found there's a similar request in Strimzi. https://github.com/strimzi/strimzi-kafka-operator/issues/3761 Maybe you can help contribute it? Either to Kafka or to Strimzi? :) Thank you. Luke On Wed, Jul 27,

Re: [ANNOUNCE] New Committer: Chris Egerton

2022-07-25 Thread Luke Chen
Congratulations Chris! Well deserved! Luke On Tue, Jul 26, 2022 at 5:39 AM Anna McDonald wrote: > Congratulations Chris! Time to Cellobrate! > > anna > > On Mon, Jul 25, 2022 at 4:23 PM Martin Gainty wrote: > > > Congratulations Chris! > > > > martin~ > > > >

Re: NoAuth for /brokers/ids

2022-07-04 Thread Luke Chen
Hi Evgeny You can check the zookeeper log to see if there are logs about why the error happened. Thanks. Luke On Tue, Jul 5, 2022 at 1:34 AM Ivanov, Evgeny wrote: > Hi everyone, > > could you please advise how to fix the problem below ? > > I'm trying to run Zookeeper with mTLS to

Re: a little problem in quickstart

2022-06-26 Thread Luke Chen
rton wrote: > Hi Mason, > > You're correct that the quickstart should use 'libs' instead of 'lib'. > This has already been fixed in the docs for the upcoming 3.3.0 release with > https://github.com/apache/kafka/pull/12252. We might consider backporting > that change; I've CC'd

Re: Granting contributor permission for Jira

2022-06-19 Thread Luke Chen
Hi, Done. Thanks for the interest in Apache Kafka. Luke On Sat, Jun 18, 2022 at 10:17 AM lith_angelo wrote: > Please add my Jira ID to contributor list, so that I'd be able to handle > Jira ticket, thanks. > Jira ID: lith_angelo

Re: Broker allows transactions with generation.id -1 and could lead to duplicates

2022-06-10 Thread Luke Chen
Hi Gabriel, Sounds like a bug to me (although we didn't document anywhere about the generation id will always start from 0). You can file a jira and we can discuss it there. Thank you. Luke On Fri, Jun 10, 2022 at 9:35 PM Gabriel Giussi wrote: > I did the following test that allowed me to

Re: Random continuous TimeoutException with Topic not present on one KafkaProducer out of many in multithreaded env

2022-06-07 Thread Luke Chen
oot cause and resolving it. > > Regards, > Deepak > > -Original Message- > From: 张晓寅 > Sent: 06 June 2022 19:10 > To: users@kafka.apache.org > Cc: Luke Chen > Subject: Re: Random continuous TimeoutException with Topic not present on > one KafkaPro

Re: Request to include me to contributors list

2022-05-22 Thread Luke Chen
Hi, I've added you to the contribution list, and assigned this ticket to you. Thanks for the interest in Apache Kafka. Luke On Mon, May 23, 2022 at 12:52 AM Kumud Kumar Srivatsava Tirupati < kumudkumartirup...@gmail.com> wrote: > Hi team, > I am willing to work on

Re: Increased latency when consuming from the closest ISR

2022-05-11 Thread Luke Chen
Hi, We have some improvement for the preferred read replica configured case. Ex: https://github.com/apache/kafka/pull/11942 https://github.com/apache/kafka/pull/11965 I know one improvement will be included in the v3.2.0 release, which will be released soon. Maybe you can give it a try to see if

Re: Topic without Leader / ISR

2022-05-10 Thread Luke Chen
er(3, 1, 2), 1 -> ArrayBuffer(1, 2, 3), 2 > -> ArrayBuffer(2, 3, 1), 3 -> ArrayBuffer(3, 2, 1), 4 -> ArrayBuffer(1, 3, > 2), 5 -> ArrayBuffer(2, 1, 3), 6 -> ArrayBuffer(3, 1, 2), 7 -> > ArrayBuffer(1, 2, 3), 8 -> ArrayBuffer(2, 3, 1), 9 -> ArrayBuffer(3, 2, 1)) &g

Re: Topic without Leader / ISR

2022-05-09 Thread Luke Chen
Hi Nicolas, Could you check the logs in broker side? There must be some errors while electing leaders or something. Thank you. Luke On Mon, May 9, 2022 at 11:31 PM Nicolas Carlot wrote: > Hello, > > I have a situation where when creating a new topic it stays in the > following state, having

Re: [VOTE] 3.1.1 RC1

2022-05-04 Thread Luke Chen
Hi Tom, I did: 1. check the signature and checksums 2. ran quick start with java17 + sacla2.12 3. browse java docs/documentations +1 (non-binding) Thanks for running the release. Luke On Thu, May 5, 2022 at 12:43 AM Bill Bejeck wrote: > Hi Tom, > > Thanks for running the release! > > I did

Re: [VOTE] 3.2.0 RC1

2022-05-04 Thread Luke Chen
Hi Bruno, I did: 1. check the signature and checksums 2. ran quick start with java17 + sacla2.12 3. browse java docs/documentations +1 (non-binding) Thanks for running the release. Luke On Thu, May 5, 2022 at 1:23 AM Mickael Maison wrote: > Hi Bruno, > > I checked signatures and checksums,

  1   2   3   >