[DISCUSS] KIP-1110: Metadata of sensitive configuration

2024-11-14 Thread Luke Chen
Hi all, I've opened the KIP-1110: Metadata of sensitive configuration . Instead of returning null for sensitive configs, by returning metadata of the sensitive configs allowing the operators to hav

[jira] [Created] (KAFKA-18022) minBytes implementation for uncommon cases

2024-11-14 Thread Abhinav Dixit (Jira)
Abhinav Dixit created KAFKA-18022: - Summary: minBytes implementation for uncommon cases Key: KAFKA-18022 URL: https://issues.apache.org/jira/browse/KAFKA-18022 Project: Kafka Issue Type: Sub-

Re: [DISCUSS]KIP-1107: Adding record-level acks for producers

2024-11-14 Thread TaiJu Wu
Hi all, I have updated the contents of this KIP Please take a look and let me know what you think. Thanks, TaiJuWu On Thu, Nov 14, 2024 at 2:21 PM TaiJu Wu wrote: > Hi all, > > Thanks for your feeback and @Chia-Ping's help. > . > I also agree topic-level acks config is more reasonable and it c

Jenkins build is still unstable: Kafka » Kafka PowerPC Daily » test-powerpc #118

2024-11-14 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-17519) Define and validate correctness of Consumer.close() and its timeout when thread is interrupted

2024-11-14 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17519?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True resolved KAFKA-17519. --- Assignee: Kirk True (was: PoAn Yang) Resolution: Fixed > Define and validate correctness of C

Re: [VOTE] KIP-1103: Additional metrics for cooperative consumption

2024-11-14 Thread David Arthur
Apoorv, thanks for the KIP! I appreciate the rationale section, it made this quite easy to follow and understand. +1 binding -David A On Thu, Nov 14, 2024 at 1:43 AM Manikumar wrote: > Hi Apoorv, > > Thanks for the KIP. > > +1 (binding) > > Thanks > > On Thu, Nov 14, 2024 at 11:11 AM Chia-Pin

Re: [DISCUSS] KIP-890 Server Side Defense

2024-11-14 Thread Justine Olshan
Hey folks, just wanted to share another quick update with KIP-890. Making the change to bump epoch after every transaction means that we no longer need to call InitProducerId during the life of the producer to fence requests from previous transactions. With KIP-890 part 2, the new client will no l

[jira] [Created] (KAFKA-18013) Add duration based offset reset option for Kafka consumer

2024-11-14 Thread Manikumar (Jira)
Manikumar created KAFKA-18013: - Summary: Add duration based offset reset option for Kafka consumer Key: KAFKA-18013 URL: https://issues.apache.org/jira/browse/KAFKA-18013 Project: Kafka Issue Typ

[jira] [Created] (KAFKA-18021) Disabled MirrorCheckpointConnector throws RetriableException on task config generation

2024-11-14 Thread Greg Harris (Jira)
Greg Harris created KAFKA-18021: --- Summary: Disabled MirrorCheckpointConnector throws RetriableException on task config generation Key: KAFKA-18021 URL: https://issues.apache.org/jira/browse/KAFKA-18021

[jira] [Created] (KAFKA-18020) Encountered quorum controller fault: incrementalAlterConfigs .. RuntimeException: 'value' field is too long to be serialized

2024-11-14 Thread Edoardo Comar (Jira)
Edoardo Comar created KAFKA-18020: - Summary: Encountered quorum controller fault: incrementalAlterConfigs .. RuntimeException: 'value' field is too long to be serialized Key: KAFKA-18020 URL: https://issues.apach

[jira] [Created] (KAFKA-18019) Convert INVALID_PRODUCER_ID_MAPPING from abortable error to fatal error

2024-11-14 Thread Ritika Reddy (Jira)
Ritika Reddy created KAFKA-18019: Summary: Convert INVALID_PRODUCER_ID_MAPPING from abortable error to fatal error Key: KAFKA-18019 URL: https://issues.apache.org/jira/browse/KAFKA-18019 Project: Kafk

Re: Apply to contribute Kafka

2024-11-14 Thread Justine Olshan
Thanks Josep for answering! Kevin, let the mailing list know if you have any further questions or issues Justine On Thu, Nov 14, 2024 at 4:01 AM Josep Prat wrote: > Hi Kevin, > > Thanks for your interest in Apache Kafka! In order to contribute to Apache > Kafka you need to first create a fork

Build failed in Jenkins: Kafka » Kafka Branch Builder » 3.9 #107

2024-11-14 Thread Apache Jenkins Server
See Changes: -- [...truncated 410365 lines...] [2024-11-14T16:21:52.933Z] Gradle Test Run :core:test > Gradle Test Executor 85 > ZkMigrationClientTest > testEmptyWrite() STAR

Re: [DISCUSS] Cannot pull 3.9.0, missing tag

2024-11-14 Thread Josep Prat
Hi Anton, I was looking a bit at what you mentioned and I realized the following: Tag is created in GH: https://github.com/apache/kafka/releases/tag/3.9.0 This Tag is pointing to the commit: https://github.com/apache/kafka/commit/84caaa6e9da06435411510a81fa321d4f99c351f However, this commit seems

[DISCUSS] Cannot pull 3.9.0, missing tag

2024-11-14 Thread Anton Agestam
Hi there, I seem to be unable to properly pull the 3.9.0 tag, and the Github UI says its commit is not found within the repository (see the screenshot attached). Invoking any of the below does not work: git fetch -p upstream && git checkout -b scratch upstream/3.9.0 git fetch -p upstream && git

Re: [DISCUSS] KIP-1098: Reverse Checkpointing in MirrorMaker

2024-11-14 Thread Viktor Somogyi-Vass
Hi Daniel, SVV3. Kind of an implementation detail. So I think using TopicFilter is good, however I was wondering if we should provide a default implementation instead of null? We have to implement the pass-through behavior anyways, and it makes sense to me to do it in a filter. SVV4. Also, an alte

[jira] [Resolved] (KAFKA-17943) Error handling when calling replicaManager.getPartitionOrException

2024-11-14 Thread Apoorv Mittal (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17943?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apoorv Mittal resolved KAFKA-17943. --- Resolution: Fixed Fixed with PR: https://github.com/apache/kafka/pull/17709 > Error handlin

Re: [VOTE] KIP-1074: Allow the replication of user internal topics

2024-11-14 Thread Mickael Maison
Hi, +1 (binding) Thanks for the KIP! Mickael On Thu, Nov 14, 2024 at 3:42 PM Patrik Marton wrote: > > Hi all, > > We currently have 2 binding and one non-binding vote. To pass the KIP, we > need one more binding vote. > I created a PR with the proposed changes to help the reviews: > https://git

[jira] [Resolved] (KAFKA-17510) Refactor code between SharePartitionManager and DelayedShareFetch for share partition initialization

2024-11-14 Thread Jun Rao (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17510?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-17510. - Fix Version/s: 4.0.0 Resolution: Fixed merged the PR to trunk > Refactor code between ShareParti

Re: [VOTE]: KIP-1050: Consistent error handling for Transactions

2024-11-14 Thread Kaushik Raina
Thanks all for reviewing KIP, I'm closing the vote. The vote has passed with + 3 binding votes from Justine, Matthias and Lianet. Kaushik On Thu, Nov 14, 2024 at 8:28 PM Lianet M. wrote: > Thanks Kaushik for the KIP, and for addressing all my comments on the > discussion thread. LGTM now. > >

Re: [VOTE] KIP-1074: Allow the replication of user internal topics

2024-11-14 Thread Patrik Marton
Hi all, We currently have 2 binding and one non-binding vote. To pass the KIP, we need one more binding vote. I created a PR with the proposed changes to help the reviews: https://github.com/apache/kafka/pull/17815 Thanks, Patrik On Wed, Nov 6, 2024 at 4:53 PM Viktor Somogyi-Vass wrote: > Hey

Re: [VOTE]: KIP-1050: Consistent error handling for Transactions

2024-11-14 Thread Lianet M.
Thanks Kaushik for the KIP, and for addressing all my comments on the discussion thread. LGTM now. +1 (binding) Lianet On Thu, Oct 3, 2024 at 8:51 PM Matthias J. Sax wrote: > Thanks for updating the KIP. Very happy to see this moving forward! > > +1 (binding). > > On 8/26/24 9:54 AM, Justine O

Re: [DISCUSS] KIP-1050: Consistent error handling for Transactions

2024-11-14 Thread Kaushik Raina
Thanks Lianet, Updated KIP to include LM5 On Wed, Nov 13, 2024 at 9:14 PM Lianet M. wrote: > Hello, thanks for the updates. Just a minor comment left > > LM5. Should we add the change to the AuthorizationException parent to the > public interfaces section? Same for the InvalidMetadataException.

[jira] [Created] (KAFKA-18018) Consumption failing with DefaultStatePersister for 1 million records

2024-11-14 Thread Abhinav Dixit (Jira)
Abhinav Dixit created KAFKA-18018: - Summary: Consumption failing with DefaultStatePersister for 1 million records Key: KAFKA-18018 URL: https://issues.apache.org/jira/browse/KAFKA-18018 Project: Kafka

[jira] [Created] (KAFKA-18017) New consumer should notify HB errors to group manager after errors have been handled/propagated

2024-11-14 Thread Lianet Magrans (Jira)
Lianet Magrans created KAFKA-18017: -- Summary: New consumer should notify HB errors to group manager after errors have been handled/propagated Key: KAFKA-18017 URL: https://issues.apache.org/jira/browse/KAFKA-1801

Re: [VOTE] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-14 Thread Apoorv Mittal
Hi Manikumar, Thanks for the KIP. It will be a valuable addition. +1 (non-binding) Regards, Apoorv Mittal On Thu, Nov 14, 2024 at 2:11 PM Hector Geraldino (BLOOMBERG/ 919 3RD A) < hgerald...@bloomberg.net> wrote: > Thanks for the KIP > > +1 (non-binding) > > From: dev@kafka.apache.org At: 11/1

Re: [VOTE] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-14 Thread Hector Geraldino (BLOOMBERG/ 919 3RD A)
Thanks for the KIP +1 (non-binding) From: dev@kafka.apache.org At: 11/14/24 09:00:52 UTC-5:00To: dev@kafka.apache.org Subject: Re: [VOTE] KIP-1106: Add duration based offset reset option for consumer clients Hi Manikumar, Thanks for the KIP. +1 (non-binding) Thanks, Andrew _

Re: [VOTE] KIP-877: Mechanism for plugins and connectors to register metrics

2024-11-14 Thread Mickael Maison
Hi, While implementing the KIP I noticed the Kafka Connect specifics had not been updated correctly based on the latest discussions. The consensus was that connector and tasks should use their context to retrieve the PluginMetrics instance if they want to register metrics. While it's a slightly d

[jira] [Created] (KAFKA-18016) Fix bugs in handling piggyback acknowledgements in ShareFetch

2024-11-14 Thread Shivsundar R (Jira)
Shivsundar R created KAFKA-18016: Summary: Fix bugs in handling piggyback acknowledgements in ShareFetch Key: KAFKA-18016 URL: https://issues.apache.org/jira/browse/KAFKA-18016 Project: Kafka

Re: [VOTE] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-14 Thread Andrew Schofield
Hi Manikumar, Thanks for the KIP. +1 (non-binding) Thanks, Andrew From: Manikumar Sent: 14 November 2024 13:30 To: dev Subject: [VOTE] KIP-1106: Add duration based offset reset option for consumer clients Hi All, I would like to start start vote on KI

[jira] [Created] (KAFKA-18015) Add duration based offset reset option for Kafka Streams

2024-11-14 Thread Manikumar (Jira)
Manikumar created KAFKA-18015: - Summary: Add duration based offset reset option for Kafka Streams Key: KAFKA-18015 URL: https://issues.apache.org/jira/browse/KAFKA-18015 Project: Kafka Issue Type

[jira] [Created] (KAFKA-18014) Add duration based offset reset option for ShareConsumer

2024-11-14 Thread Manikumar (Jira)
Manikumar created KAFKA-18014: - Summary: Add duration based offset reset option for ShareConsumer Key: KAFKA-18014 URL: https://issues.apache.org/jira/browse/KAFKA-18014 Project: Kafka Issue Type

[VOTE] KIP-1106: Add duration based offset reset option for consumer clients

2024-11-14 Thread Manikumar
Hi All, I would like to start start vote on KIP-1106: https://cwiki.apache.org/confluence/display/KAFKA/KIP-1106%3A+Add+duration+based+offset+reset+option+for+consumer+clients This KIP proposes to add an additional auto offset reset strategy for consumer clients. Regards, Manikumar

Re: [DISCUSS] KIP-1099: Extend kafka-consumer-groups command line tool to support new consumer group

2024-11-14 Thread Andrew Schofield
Hi PoAn, DJ2: I was just going to comment that "-" would be a more appropriate missing value, but you got there first. AS3: The examples for kafka-share-groups.sh include kafka-consumer-groups.sh in the command line. If this is accepted in time, I'm happy to pick up the implementation of the s

Re: How to push forward KIP-1011 complete implementation

2024-11-14 Thread ziming deng
Hi David, In the original scheme of KIP-1011, we were going to maintain a deprecation period before 4.X, which means we won’t fail directly before 4.X, this is consistent with what we have done in KIP-894, currently the PR is not landed onto release-3.9, so there would be no deprecation period

Re: Apply to contribute Kafka

2024-11-14 Thread Josep Prat
Hi Kevin, Thanks for your interest in Apache Kafka! In order to contribute to Apache Kafka you need to first create a fork on your own GitHub space. There is plenty of information on how to do this on the internet, but long story short: go to github.com/apache/kafka and click on fork on the top ri

Re: Apply to contribute Kafka

2024-11-14 Thread Kevin Apolinario
Hello Justine, I am emailing as I saw you helped another contributor with a similar problem. I am new to the project and currently working on KAFKA-17934, however I cannot push to the branch I created as I am getting 403 error messages. Github: https://github.com/KApolinario1120 Please let me k

[jira] [Created] (KAFKA-18012) Update the Scram configuration section for KRaft

2024-11-14 Thread Mickael Maison (Jira)
Mickael Maison created KAFKA-18012: -- Summary: Update the Scram configuration section for KRaft Key: KAFKA-18012 URL: https://issues.apache.org/jira/browse/KAFKA-18012 Project: Kafka Issue Ty

[jira] [Created] (KAFKA-18011) Remove ZooKeeper from the docs

2024-11-14 Thread Mickael Maison (Jira)
Mickael Maison created KAFKA-18011: -- Summary: Remove ZooKeeper from the docs Key: KAFKA-18011 URL: https://issues.apache.org/jira/browse/KAFKA-18011 Project: Kafka Issue Type: Sub-task

[jira] [Resolved] (KAFKA-17888) e2e should support to run <2.3 kafka under JDK 11

2024-11-14 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17888?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-17888. Fix Version/s: 4.0.0 Resolution: Fixed > e2e should support to run <2.3 kafka under

Re: [ANNOUNCE] Apache Kafka 3.9.0

2024-11-14 Thread Chia-Ping Tsai
hi Divij Nice find. I've updated the fix version to 3.9.1, and I'll test it on the trunk and 3.9 branch tomorrow. Best, Chia-Ping Divij Vaidya 於 2024年11月14日 週四 下午6:46寫道: > FYI folks - 3.9.0 is showing up as "unreleased" in JIRA right now. Before > moving the version to released, I wanted to ch

[jira] [Resolved] (KAFKA-17995) Large value for `retention.ms` could prevent remote data cleanup in Tiered Storage

2024-11-14 Thread Divij Vaidya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17995?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Divij Vaidya resolved KAFKA-17995. -- Reviewer: Divij Vaidya Resolution: Fixed > Large value for `retention.ms` could prevent

Re: [DISCUSS] KIP-1099: Extend kafka-consumer-groups command line tool to support new consumer group

2024-11-14 Thread Frank Yang
Hi David, Thanks for the review and suggestion! I would like to get this in AK 4.0 as well. I will do my best. DJ1: Update KIP to put GROUP-EPOCH and TARGET-ASSIGNMENT-EPOCH before #MEMBERS. DJ2: I prefer to follow current missing column value “-“. (reference

Re: [ANNOUNCE] Apache Kafka 3.9.0

2024-11-14 Thread Divij Vaidya
FYI folks - 3.9.0 is showing up as "unreleased" in JIRA right now. Before moving the version to released, I wanted to check the status of https://issues.apache.org/jira/browse/KAFKA-16949 which was targeted for 3.9.0. I assume the ship has already sailed now and the earliest we can fix this is 3.9.

[jira] [Resolved] (KAFKA-18010) Update compatibility/upgrade system tests to test with 3.9.0

2024-11-14 Thread Bruno Cadonna (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18010?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bruno Cadonna resolved KAFKA-18010. --- Resolution: Duplicate Duplicate of https://issues.apache.org/jira/browse/KAFKA-18006 > Upda

[jira] [Resolved] (KAFKA-17992) Remove `getUnderlying` and `isKRaftTest` from ClusterInstance

2024-11-14 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17992?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-17992. Fix Version/s: 4.0.0 Resolution: Fixed > Remove `getUnderlying` and `isKRaftTest` f

[jira] [Created] (KAFKA-18010) Update compatibility/upgrade system tests to test with 3.9.0

2024-11-14 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-18010: - Summary: Update compatibility/upgrade system tests to test with 3.9.0 Key: KAFKA-18010 URL: https://issues.apache.org/jira/browse/KAFKA-18010 Project: Kafka

[jira] [Resolved] (KAFKA-17991) Timed calls to future.get in DefaultStatePersister and test improvements

2024-11-14 Thread Sushant Mahajan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17991?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sushant Mahajan resolved KAFKA-17991. - Resolution: Fixed > Timed calls to future.get in DefaultStatePersister and test improvem

Re: How to push forward KIP-1011 complete implementation

2024-11-14 Thread David Jacot
Hi Ziming, I am not sure to understand the concern. What's wrong with keeping the original plan? If I understood it correctly, the plan was to fail directly if the incrementalAlterConfigs API is not available in 4.x versions. The new API was introduced in AK 2.3. The likelihood of using the update

[jira] [Resolved] (KAFKA-17632) Custom `partitioner.class` with an even number of partitions always writes to even partitions if use RoundRobinPartitioner

2024-11-14 Thread Daniel Urban (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17632?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Daniel Urban resolved KAFKA-17632. -- Fix Version/s: 4.0.0 Resolution: Fixed [~thanhlv] The RR partitioner has been fixed, an

Re: [DISCUSS] KIP-1099: Extend kafka-consumer-groups command line tool to support new consumer group

2024-11-14 Thread David Jacot
Hi PoAn, Thanks for the KIP! I have a few minor comments/suggestions: DJ1: In the output of `--describe --verbose`, I would suggest putting `GROUP-EPOCH` and `TARGET-ASSIGNMENT-EPOCH` before `#MEMBERS`. DJ2: Continuing on the above, I assume that we will print out N/A for the fields not supported