CVE-2024-56128: Apache Kafka: SCRAM authentication vulnerable to replay attacks when used without encryption

2024-12-18 Thread Manikumar
Severity: low Affected versions: - Apache Kafka 0.10.2.0 before 3.7.2 - Apache Kafka 3.8.0 Description: Incorrect Implementation of Authentication Algorithm in Apache Kafka's SCRAM implementation. Issue Summary: Apache Kafka's implementation of the Salted Challenge Response Authentication Mech

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

2024-12-11 Thread Manikumar
ated `with(Topology.AutoOffsetReset)` method - added new `with(TimestampExtractor, AutoOffsetReset)` method - added new `with(TimestampExtractor, AutoOffsetReset)` method Thanks On Tue, Nov 19, 2024 at 5:36 PM Manikumar wrote: > Hi All, > > I have made a minor update to the

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

2024-12-02 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18013?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-18013. --- Resolution: Fixed > Add duration based offset reset option for Kafka consu

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

2024-11-19 Thread Manikumar
Hi All, I have made a minor update to the KIP. Since we are deprecating the OffsetResetStrategy enum, I have deprecated the `MockConsumer(OffsetResetStrategy offsetResetStrategy)` constructor and added a new constructor. Thanks, On Thu, Nov 14, 2024 at 10:25 AM Manikumar wrote: > Hi

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

2024-11-18 Thread Manikumar
Thanks everyone for the votes. The vote passed with 6 binding votes (Kamal, David Arthur, Lianet, David Jacot, Jun, Matthias) and 2 non-binding votes (Andrew, Apoorv). Regards, Manikumar On Mon, Nov 18, 2024 at 2:41 PM Manikumar wrote: > Thanks Jun, > > Yes, Updated the KIP > &

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

2024-11-18 Thread Manikumar
Thanks Jun, Yes, Updated the KIP On Sat, Nov 16, 2024 at 1:21 AM Jun Rao wrote: > Hi, ManiKumar, > > Thanks for the KIP. +1 > > Should we document that duration can't be negative in the following too? > > AutoOffsetReset duration(Duration duration) > > Jun >

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

2024-11-15 Thread Manikumar
raprak...@gmail.com> wrote: > > > Hi Manikumar, > > > > Thanks for the KIP! Duration based offset reset is a useful addition when > > remote storage is enabled. > > > > +1 (binding) > > > > Thanks, > > Kamal > > > > On Thu, Nov 14, 2024

[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

[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

[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

[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: [VOTE] KIP-1103: Additional metrics for cooperative consumption

2024-11-13 Thread Manikumar
Hi Apoorv, Thanks for the KIP. +1 (binding) Thanks On Thu, Nov 14, 2024 at 11:11 AM Chia-Ping Tsai wrote: > hi Apoorv > > Could you please add the link of vote thread to the KIP-1103 page? > > thanks, > Chia-Ping > > On 2024/11/07 11:14:03 Apoorv Mittal wrote: > > Hi All, > > I would like to

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

2024-11-13 Thread Manikumar
Hi Jun, JR10: Yes, we can remove the name field. Updated the KIP. Thanks, On Thu, Nov 14, 2024 at 3:57 AM Jun Rao wrote: > Hi, Manikumar, > > Thanks for the explanation. We can follow the existing KStreams pattern. > > Could AutoOffsetReset(String name, long dura

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

2024-11-13 Thread Manikumar
d. We can follow your suggestions if we do not use Internal class approach, Thanks, On Wed, Nov 13, 2024 at 11:12 PM Jun Rao wrote: > Hi, Manikumar, > > Thanks for the updated KIP. > > JR10. The new AutoOffsetReset class. Could the following constructors be > private? >

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

2024-11-12 Thread Manikumar
mplementations), as it's not user facing, but only > have method signatures in the KIP. > > What I believe we need to add is a `protected` constructor (for the > internal sub-class): > > protected AutoOffsetReset(AutoOffsetReset autoOffsetReset); > > > > > -Ma

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

2024-11-12 Thread Manikumar
long as Optional, and use Optional.empty() for latest and earliest? > > Regards, > Apoorv Mittal > > > On Tue, Nov 12, 2024 at 12:06 PM Manikumar > wrote: > > > Thanks Ismael and Lianet for the reviews. > > > > Based on suggestions, I have updated the K

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

2024-11-12 Thread Manikumar
concerns. Thanks, On Sat, Nov 9, 2024 at 10:06 PM Lianet M. wrote: > > Hi all. Thanks Manikumar for the nice improvement, useful indeed. > > I also lean towards having a single config given that it's all about the > reset policy, seems all the same "what" (auto reset p

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

2024-11-08 Thread Manikumar
has the necessary getters. Of course, the KIP does not need to define > `AutoOffsetResetInternal` but only the public / user-facing > `AutoOffsetReset` class. > > > -Matthias > > On 11/7/24 10:23 PM, Manikumar wrote: > > Hi Matthias, > > > > Thanks for the comme

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

2024-11-08 Thread Manikumar
Hi All, I have updated the KIP with recent suggestions. Thanks. On Fri, Nov 8, 2024 at 11:53 AM Manikumar wrote: > > Hi Matthias, > > Thanks for the comments. > > > I am personally not convinced that adding a new config > `auto.offset.reset.by.duration` is the best

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

2024-11-07 Thread Manikumar
can add the new > class just for package `org.apache.kafka.streams` to avoid naming conflicts. > > Additionally, we would deprecate all methods taking existing enum > `AutoOffsetReset` (only two classed `Topology` and `Consumed` are > affected) and add new overload which accepts the

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

2024-11-07 Thread Manikumar
of range scenarios. Thanks, On Fri, Nov 8, 2024 at 3:27 AM Jun Rao wrote: > > Hi, Manikumar, > > Thanks for the KIP. A couple of comments. > > JR1. It doesn't seem that we need earliest_local. Intuitively, it makes > sense for a consumer app to bootstrap based on time.

Re: [VOTE] KIP-1102: Enable clients to rebootstrap based on timeout or error code

2024-11-06 Thread Manikumar
Hi Rajini, Thanks for the KIP. +1 (binding). Thanks, On Tue, Nov 5, 2024 at 2:55 PM Rajini Sivaram wrote: > > Hi everyone, > > I would like to start vote on KIP-1102 ( > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1102%3A+Enable+clients+to+rebootstrap+based+on+timeout+or+error+code).

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

2024-11-05 Thread Manikumar
, Nov 6, 2024 at 1:45 AM Apoorv Mittal wrote: > > Hi Manikumar, > Thanks for the changes. Just minor comment and a question: > > AM5: The description "How to initialize the share-partition start offset:" > for "*share.auto.offset.reset*" seems incomplete as it e

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

2024-11-05 Thread Manikumar
Hi Andrew, Thanks for the review. AS4, AS5: Updated the KIP. AS6: Yes, it applies to both. updated the KIP. Thanks, On Tue, Nov 5, 2024 at 10:58 PM Andrew Schofield wrote: > > Hi Manikumar, > Thanks for the updates. A couple of other comments. > > AS4: I think the AK co

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

2024-11-05 Thread Manikumar
> extend `bin/kafka-consumer-group.sh` accordingly (even if the tool works > slightly different, as it does commit an offset and there could be some > race condition between committing "latest-local", tiering, and when the > consumer is actually started? > > > -Matthias

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

2024-11-05 Thread Manikumar
etime( similar consumer group command). Thanks, On Mon, Nov 4, 2024 at 9:11 PM Apoorv Mittal wrote: > > Hi Manikumar, > Thanks for the KIP, this new strategy would be helpful in specifying fetch > behaviour. > > AM1: The config `auto.offset.reset` is currently applied

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

2024-11-05 Thread Manikumar
--by-duration reset option. Thanks, On Mon, Nov 4, 2024 at 9:08 PM Dániel Urbán wrote: > > Hi Manikumar, > > Thanks for the KIP! I have a few questions: > 1. Why did you choose the "minus-" prefix for the new value format? Can't > we just drop the prefix? If w

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

2024-11-05 Thread Manikumar
wrote: > > Hi Manikumar, > Thanks for the KIP. > > AS1: As Apoorv points out, the values of auto.offset.reset correspond to the > enum > OffsetResetStrategy. I would say that's a useful thing to maintain and I > suggest > adding a fourth value "by.dura

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

2024-11-04 Thread Manikumar
Hi everyone, I would like to start a discussion 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

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

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

Re: [ANNOUNCE] Apache Kafka 3.7.1

2024-07-01 Thread Manikumar
; Bruno Cadonna, Calvin Liu, Cameron Redpath, Cheng-Kai Zhang, Chia-Ping > > Tsai, Chris Egerton, Colin Patrick McCabe, David Arthur, David Jacot, Divij > > Vaidya, Dmitry Werner, Edoardo Comar, flashmouse, Florin Akermann, > > Gantigmaa Selenge, Gaurav Narula, Greg Harris,

Re: Requesting permissions to create KIP

2024-06-06 Thread Manikumar
Thank you for your interest in Apache Kafka. Updated the permissions. On Thu, Jun 6, 2024 at 1:15 PM Kaushik Raina wrote: > > Hello, > Please provide permissions to create KIP. I indent to contribute to Apache > kafka > > Wiki Id: kra...@confluent.io > Jira ID: k-raina

[jira] [Resolved] (KAFKA-16826) Integrate Native Kafka Docker Image with github Actions

2024-05-23 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16826?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-16826. --- Fix Version/s: 3.8.0 Resolution: Fixed > Integrate Native Kafka Docker Image with git

Re: [VOTE] KIP-1028: Docker Official Image for Apache Kafka

2024-05-14 Thread Manikumar
+1 (binding) Thanks for the KIP. On Tue, May 14, 2024, 9:46 PM Chris Egerton wrote: > +1 (binding), thanks for the KIP! > > On Tue, May 14, 2024, 12:13 Vedarth Sharma > wrote: > > > Hi everyone, > > > > I'd like to call a vote on KIP-1028 which aims to introduce a JVM based > > Docker Offi

Re: [VOTE] KIP-1041: Drop `offsets.commit.required.acks` config in 4.0 (deprecate in 3.8)

2024-05-13 Thread Manikumar
+1 (binding). Thanks for the KIP. Manikumar On Wed, May 8, 2024 at 9:55 PM Justine Olshan wrote: > > +1 (binding) > > Thanks, > Justine > > On Wed, May 8, 2024 at 8:36 AM Federico Valeri wrote: > > > +1 non binding > > > > Thanks > > >

Re: [VOTE] KIP-899: Allow producer and consumer clients to rebootstrap

2024-05-08 Thread Manikumar
Thanks for the KIP. +1 (binding). On Wed, Apr 17, 2024 at 7:50 PM Omnia Ibrahim wrote: > > Hi Ivan, > Thanks for the KIP this is a very nice feature to have. > +1(non-binding) > Omnia > > On 15 Apr 2024, at 14:33, Andrew Schofield > > wrote: > > > > Thanks for the KIP > > > > +1 (non-binding)

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

2024-05-08 Thread Manikumar
Hi Andrew, Thanks for the KIP. Great write-up! +1 (binding) Thanks, On Wed, May 8, 2024 at 12:17 PM Satish Duggana wrote: > > Hi Andrew, > Thanks for the nice KIP, it will allow other messaging use cases to be > onboarded to Kafka. > > +1 from me. > > Satish. > > On Tue, 7 May 2024 at 03:41,

Re: [DISCUSS] KIP-1028: Docker Official Image for Apache Kafka

2024-04-19 Thread Manikumar
Thanks Krish. KIP looks good to me. On Wed, Apr 17, 2024 at 1:38 PM Krish Vora wrote: > > Hi Manikumar, > > Thanks for the comments. > > Maybe as part of the release process, RM can create a JIRA for this > > task. This can be taken by RM or any comitter or any contrib

Re: [DISCUSS] KIP-1028: Docker Official Image for Apache Kafka

2024-04-15 Thread Manikumar
(or 3.7.1) Docker Official image. This will help us to validate the process and allow us to fix any changes suggested by Dockerhub before the 3.8.0 release. Thanks, On Mon, Apr 8, 2024 at 2:33 PM Krish Vora wrote: > > Hi Manikumar and Luke. > Thanks for the questions. > > 1. No, th

Re: [ANNOUNCE] New Kafka PMC Member: Greg Harris

2024-04-14 Thread Manikumar
Congratulations, Greg. On Mon, Apr 15, 2024 at 11:18 AM Bruno Cadonna wrote: > > Congratulations, Greg! > > Best, > Bruno > > On 4/15/24 7:33 AM, Claude Warren wrote: > > Congrats Greg! All the hard work paid off. > > > > On Mon, Apr 15, 2024 at 6:58 AM Ivan Yurchenko wrote: > > > >> Congrats G

[jira] [Resolved] (KAFKA-16473) KafkaDockerWrapper uses wrong cluster ID when formatting log dir

2024-04-11 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16473?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-16473. --- Fix Version/s: 3.8.0 3.7.1 Resolution: Fixed > KafkaDockerWrapper u

Re: [VOTE] KIP-1025: Optionally URL-encode clientID and clientSecret in authorization header

2024-04-07 Thread Manikumar
Thanks for the KIP. +1 (binding) On Mon, Apr 8, 2024 at 9:49 AM Kirk True wrote: > > +1 (non-binding) > > Apologies. I thought I’d already voted :( > > > On Apr 7, 2024, at 10:48 AM, Nelson B. wrote: > > > > Hi all, > > > > Just wanted to bump up this thread for visibility. > > > > Thanks! >

Re: [DISCUSS] Apache Kafka 3.6.2 release

2024-04-05 Thread Manikumar
announcement email Thanks, On Fri, Apr 5, 2024 at 5:34 PM Divij Vaidya wrote: > Hey Manikumar > > Are we planning to add the blog entry about the 3.6.2 release at > https://kafka.apache.org/blog ? Asking because I didn't see this included > in the release announcement. &

[ANNOUNCE] Apache Kafka 3.6.2

2024-04-04 Thread Manikumar
, David Mao, Divij Vaidya, Edoardo Comar, Emma Humber, Gaurav Narula, Greg Harris, hudeqi, Ismael Juma, Jason Gustafson, Jim Galasyn, Joel Hamill, Johnny Hsu, José Armando García Sancio, Justine Olshan, Luke Chen, Manikumar Reddy, Matthias J. Sax, Mayank Shekhar Narula, Mickael Maison, Mike Lloyd, Paolo

Re: [VOTE] 3.6.2 RC2

2024-04-04 Thread Manikumar
Thanks all for voting. I'm now closing the vote. The vote passes with - 3 +1 bindings votes from Divij Vaidya, Justine, and Manikumar - 3 +1 non-binding votes from Andrew, Jakub, and Lianet - 0 -1 votes I'll go ahead and finish the release process. Thanks, On Thu, Apr 4, 2024

Re: [VOTE] 3.6.2 RC2

2024-04-04 Thread Manikumar
+1 (binding) from my side also. - verified the signatures, artifacts - ran the tests on the source archive - verified the quickstarts Thanks On Thu, Apr 4, 2024 at 7:46 PM Justine Olshan wrote: > Thanks for clarifying Manikumar! > > +1 (binding) from me > > Justine > > O

Re: [VOTE] 3.6.2 RC2

2024-04-04 Thread Manikumar
some files that request fullDot version and that seemed closer to > what was expected: "fullDotVersion": "3.6.2-SNAPSHOT" > The upgrade.html file also looked ok. > > Thanks for running the release and answering my questions! > Justine > > On Wed, Apr 3, 20

Re: [VOTE] 3.6.2 RC2

2024-04-03 Thread Manikumar
Hi Justine, Yes, it is intended. For bug fix releases website docs will be updated during the final release process. We can verify the site-docs artifacts here: https://home.apache.org/~manikumar/kafka-3.6.2-rc2/kafka_2.12-3.6.2-site-docs.tgz These site-docs artifacts will be used to update

Re: [VOTE] 3.6.2 RC2

2024-04-02 Thread Manikumar
Gentle reminder. Please download, test and vote for the release. Thanks, On Fri, Mar 29, 2024 at 4:57 PM Manikumar wrote: > Hi All, > > System test runs are green. There were 13 test failures in the first run. > All the failed tests passed in the second run. > > System tes

Re: [DISCUSS] KIP-1028: Docker Official Image for Apache Kafka

2024-04-02 Thread Manikumar
Hi Krishna, Thanks for the KIP. I think Docker Official Images will be beneficial to the Kafka community. Few queries below. 1. Will the Docker inventory files/etc are the same for OSS Image and Docker Official Images 2. I am a bit worried about the new steps to the release process. Maybe we sho

Re: [VOTE] 3.6.2 RC2

2024-03-29 Thread Manikumar
Hi All, System test runs are green. There were 13 test failures in the first run. All the failed tests passed in the second run. System test results: https://gist.github.com/omkreddy/17d23d3eb36ef840011f2494d65bbd4f Thanks, On Thu, Mar 28, 2024 at 3:21 PM Manikumar wrote: > Hello Kafka us

[VOTE] 3.6.2 RC2

2024-03-28 Thread Manikumar
/~manikumar/kafka-3.6.2-rc2/RELEASE_NOTES.html *** Please download, test and vote by by Wednesday, April 3rd 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/~mani

[jira] [Reopened] (KAFKA-16310) ListOffsets doesn't report the offset with maxTimestamp anymore

2024-03-27 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16310?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar reopened KAFKA-16310: --- > ListOffsets doesn't report the offset with maxTimestamp

Re: [VOTE] 3.6.2 RC1

2024-03-25 Thread Manikumar
browse/KAFKA-16222 was fixed, and it > appears to still be an issue with 3.6.2 RC1. The quota on the default > resource is still getting translated improperly. > > I am looking into what the issue is here. > > best, > Colin > > > On Thu, Mar 21, 2024, at 19:32, Chia-Ping

Re: [VOTE] 3.6.2 RC1

2024-03-21 Thread Manikumar
is existent for a while, but it is not a true issue until > we allowed users to fetch offset of max timestamp. > > > Can we update the "Affects Version/s" field on JIRA? > > done. I attach the tags for active branches - 3.6.1 and 3.7.0 > > Manikumar 於 2024年3月21日

Re: [VOTE] 3.6.2 RC1

2024-03-21 Thread Manikumar
Hi Chia-Ping, Thanks for letting me know. Is this a regression from the 3.5.0 release? Can we update the "Affects Version/s" field on JIRA? Thanks, On Thu, Mar 21, 2024 at 5:06 PM Chia-Ping Tsai wrote: > hi Manikumar, > > There is a bug fix which needs to be backpo

[VOTE] 3.6.2 RC1

2024-03-21 Thread Manikumar
Hello Kafka users, developers and client-developers, This is the first candidate for release of Apache Kafka 3.6.2. This is a bugfix release with several fixes, including dependency version bumps for CVEs. Release notes for the 3.6.2 release: https://home.apache.org/~manikumar/kafka-3.6.2-rc1

Re: [DISCUSS] Apache Kafka 3.6.2 release

2024-03-20 Thread Manikumar
Hi, We have one non-blocker issue to be resolved. https://issues.apache.org/jira/browse/KAFKA-16073 I plan to generate the first release candidate later today/tomorrow. Thanks. On Mon, Mar 18, 2024 at 11:28 PM Edoardo Comar wrote: > Thanks Manikumar, done and marked the issue as resol

Re: [DISCUSS] Apache Kafka 3.6.2 release

2024-03-18 Thread Manikumar
Hi Edoardo, sure, pls go ahead and cherry-pick the changes to 3.7 and 3.6 branches. Thanks, On Mon, Mar 18, 2024 at 3:53 PM Edoardo Comar wrote: > Hi Manikumar, > https://issues.apache.org/jira/browse/KAFKA-16369 > is merged in trunk now. > can you please cherry-pick it to 3

Re: [DISCUSS] Apache Kafka 3.6.2 release

2024-03-18 Thread Manikumar
al > release on this branch. > > Ismael > > On Thu, Mar 14, 2024, 1:39 AM Divij Vaidya > wrote: > > > Hi Manikumar, > > > > 1. Can you please take a look at > > https://github.com/apache/kafka/pull/15490 > > which is a bug fix specific to the 3.6.x bra

Re: [DISCUSS] Apache Kafka 3.6.2 release

2024-03-14 Thread Manikumar
Hi Edoardo, Sure, we can include KAFKA-16369 <https://issues.apache.org/jira/browse/KAFKA-16369> based PR reviews. Thanks, On Thu, Mar 14, 2024 at 3:16 PM Edoardo Comar wrote: > Hi Manikumar, > can we please include > https://issues.apache.org/jira/browse/KAFKA-16369 > in 3.

Re: [DISCUSS] Apache Kafka 3.6.2 release

2024-03-14 Thread Manikumar
. Thanks, Manikumar On Thu, Mar 14, 2024 at 2:09 PM Divij Vaidya wrote: > Hi Manikumar, > > 1. Can you please take a look at > https://github.com/apache/kafka/pull/15490 > which is a bug fix specific to the 3.6.x branch? > 2. Should we do a one-time update of all dependencies in 3.6

Re: [DISCUSS] Apache Kafka 3.6.2 release

2024-03-13 Thread Manikumar
, Manikumar On Thu, Mar 14, 2024 at 6:24 AM Satish Duggana wrote: > +1, Thanks Mani for volunteering. > > On Thu, 14 Mar 2024 at 06:01, Luke Chen wrote: > > > > +1, Thanks Manikumar! > > > > On Thu, Mar 14, 2024 at 3:40 AM Bruno Cadonna > wrote: > &

[DISCUSS] Apache Kafka 3.6.2 release

2024-03-13 Thread Manikumar
Hi, I'd like to volunteer to be the release manager for a bug fix release of the 3.6 line. If there are no objections, I'll send out the release plan soon. Thanks, Manikumar

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

2024-03-06 Thread Manikumar
Hi Andrew, Thanks for the updated KIP. Few queries below: 1. What is the use-case of deliveryCount in ShareFetchResponse? 2. During delete share groups, Do we need to clean any in-memory state from share-partition leaders? 3. Any metrics for the share-coordinator? Thanks Manikumar On Wed, Feb

Re: [kafka-clients] Re: [ANNOUNCE] Apache Kafka 3.7.0

2024-02-28 Thread Manikumar
Crispin Bernier, David Arthur, David > > > Jacot, David Mao, Deqi Hu, Dimitar Dimitrov, Divij Vaidya, Dongnuo > > > Lyu, Eaugene Thomas, Eduwer Camacaro, Eike Thaden, Federico Valeri, > > > Florin Akermann, Gantigmaa Selenge, Gaurav Narula, gongzhongqiang, > > >

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

2024-02-20 Thread Manikumar
+1 (binding). Thanks for the KIP. Manikumar On Tue, Feb 20, 2024 at 2:31 PM Andrew Schofield < andrew_schofield_j...@outlook.com> wrote: > Hi Apoorv, > Thanks for the KIP. > > +1 (non-binding) > > Thanks, > Andrew > > > On 19 Feb 2024, at 22:31, Apoorv Mitta

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

2024-02-15 Thread Manikumar
LGTM, Thanks for the KIP. On Thu, Feb 15, 2024 at 8:50 PM Doğuşcan Namal wrote: > LGTM thanks for the KIP. > > +1(non-binding) > > On Wed, 14 Feb 2024 at 15:22, Andrew Schofield < > andrew_schofield_j...@outlook.com> wrote: > > > Hi Apoorv, > > Thanks for the KIP. Looks like a useful change to t

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

2024-02-14 Thread Manikumar
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

[jira] [Resolved] (KAFKA-15738) KRaft support in ConsumerWithLegacyMessageFormatIntegrationTest

2024-01-12 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15738?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-15738. --- Fix Version/s: 3.8.0 Resolution: Fixed > KRaft support

[jira] [Resolved] (KAFKA-15735) KRaft support in SaslMultiMechanismConsumerTest

2024-01-09 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15735?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-15735. --- Fix Version/s: 3.8.0 Resolution: Fixed > KRaft support in SaslMultiMechanismConsumerT

[jira] [Resolved] (KAFKA-15726) KRaft support in ProduceRequestTest

2024-01-09 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15726?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-15726. --- Fix Version/s: 3.8.0 Resolution: Fixed > KRaft support in ProduceRequestT

[jira] [Created] (KAFKA-15904) Downgrade tests are failing with directory.id 

2023-11-27 Thread Manikumar (Jira)
Manikumar created KAFKA-15904: - Summary: Downgrade tests are failing with directory.id  Key: KAFKA-15904 URL: https://issues.apache.org/jira/browse/KAFKA-15904 Project: Kafka Issue Type: Bug

Re: [VOTE] KIP-974: Docker Image for GraalVM based Native Kafka Broker

2023-11-22 Thread Manikumar
Hi Krishna, Thanks for KIP. +1 (binding). Thanks, Manikumar On Mon, Nov 20, 2023 at 11:57 AM Krishna Agarwal < krishna0608agar...@gmail.com> wrote: > Hi, > I'd like to call a vote on KIP-974 which aims to publish a docker image for > GraalVM based Native Kafka Broker.

Re: [VOTE] KIP-975: Docker Image for Apache Kafka

2023-10-28 Thread Manikumar
Hi, Thanks for the KIP. +1 (binding) Thanks On Fri, Oct 27, 2023 at 9:41 PM Ismael Juma wrote: > Thanks for the KIP Krishna - looking forward to this. +1 (binding). > > On Thu, Oct 26, 2023 at 9:36 PM Krishna Agarwal < > krishna0608agar...@gmail.com> wrote: > > > Hi, > > I'd like to call a vo

Re: [DISCUSS] KIP-974 Docker Image for GraalVM based Native Kafka Broker

2023-10-28 Thread Manikumar
Thanks for the explanation. I am fine with using ""kafka-local" as the image name. On Fri, Oct 27, 2023 at 11:47 AM Krishna Agarwal < krishna0608agar...@gmail.com> wrote: > Hi Manikumar, > Thanks for the feedback. > > This image signifies 2 things: > >

Re: [ANNOUNCE] New Kafka PMC Member: Satish Duggana

2023-10-27 Thread Manikumar
Congrats! On Fri, Oct 27, 2023 at 8:35 PM Jun Rao wrote: > Hi, Everyone, > > Satish Duggana has been a Kafka committer since 2022. He has been very > instrumental to the community since becoming a committer. It's my pleasure > to announce that Satish is now a member of Kafka PMC. > > Congratulat

Re: [VOTE] KIP-978: Allow dynamic reloading of certificates with different DN / SANs

2023-10-25 Thread Manikumar
Hi, Thanks for the KIP. +1 (binding) Thanks. On Wed, Oct 25, 2023 at 1:37 AM Jakub Scholz wrote: > Hi all, > > I would like to start a vote for the KIP-978: Allow dynamic reloading of > certificates with different DN / SANs > < > https://cwiki.apache.org/confluence/pages/viewpage.action?page

Re: [VOTE] KIP-982: Enhance Custom KafkaPrincipalBuilder to Access SslPrincipalMapper and KerberosShortNamer

2023-10-20 Thread Manikumar
Hi, Thanks for the KIP. +1 (binding) Thanks, Manikumar On Fri, Oct 20, 2023 at 4:26 AM Raghu B wrote: > Hi everyone, > > I would like to start a vote on KIP-982, which proposed enhancements to > the Custom KafkaPrincipalBuilder to allow access to SslPrincipalMapper and > Ker

Re: [DISCUSS] KIP-974 Docker Image for GraalVM based Native Kafka Broker

2023-10-20 Thread Manikumar
Hi, > For the native AK docker image, we are considering '*kafka-local*' as it clearly signifies that this image is intended exclusively for local I am not sure, if there is any naming pattern for graalvm based images. Can we include "graalvm" to the image name like "kafka-graalvm-native". This w

Re: [DISCUSS] KIP-975 Docker Image for Apache Kafka

2023-10-20 Thread Manikumar
ges to docker hub. I am mainly asking because PMC needs to request docker hub access/repos. I can help in getting required repos/accounts. https://infra.apache.org/docker-hub-policy.html Thanks, Manikumar On Thu, Oct 19, 2023 at 8:22 PM Krishna Agarwal < krishna0608agar...@gmail.com> wro

Re: [DISCUSS] KIP-982: Access SslPrincipalMapper and kerberosShortNamer in Custom KafkaPrincipalBuilder

2023-10-16 Thread Manikumar
Hi Raghu, Thanks for the KIP. Proposed changes look good to me. Thanks, Manikumar On Fri, Sep 22, 2023 at 11:44 PM Raghu B wrote: > Hi everyone, > > I would like to start the discussion on the KIP-982 to Access > SslPrincipalMapper and kerberosShortNamer in Custom KafkaPrin

[jira] [Resolved] (KAFKA-14927) Dynamic configs not validated when using kafka-configs and --add-config-file

2023-10-10 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14927?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-14927. --- Fix Version/s: 3.7.0 Assignee: Aman Singh (was: José Armando García Sancio

[jira] [Resolved] (KAFKA-15502) Handle large keystores in SslEngineValidator

2023-10-08 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15502?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-15502. --- Fix Version/s: 3.4.2 3.5.2 3.7.0 3.6.1

[jira] [Created] (KAFKA-15502) Handle large keystores in SslEngineValidator

2023-09-25 Thread Manikumar (Jira)
Manikumar created KAFKA-15502: - Summary: Handle large keystores in SslEngineValidator Key: KAFKA-15502 URL: https://issues.apache.org/jira/browse/KAFKA-15502 Project: Kafka Issue Type: Bug

[jira] [Resolved] (KAFKA-15273) Log common name of expired client certificate

2023-09-15 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15273?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-15273. --- Fix Version/s: 3.7.0 Resolution: Fixed > Log common name of expired client certific

[jira] [Resolved] (KAFKA-15243) User creation mismatch

2023-07-26 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15243?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-15243. --- Fix Version/s: 3.6.0 Resolution: Fixed > User creation misma

[jira] [Resolved] (KAFKA-15077) FileTokenRetriever doesn't trim the token before returning it.

2023-06-10 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15077?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-15077. --- Resolution: Fixed > FileTokenRetriever doesn't trim the token before retu

Re: [ANNOUNCE] New Kafka PMC Member: David Arthur

2023-03-09 Thread Manikumar
Congrats David! On Fri, Mar 10, 2023 at 12:24 AM Josep Prat wrote: > > Congrats David! > > ——— > Josep Prat > > Aiven Deutschland GmbH > > Alexanderufer 3-7, 10117 Berlin > > Amtsgericht Charlottenburg, HRB 209739 B > > Geschäftsführer: Oskari Saarenmaa & Hannu Valtonen > > m: +491715557497 > >

Re: [ANNOUNCE] New Kafka PMC Member: Chris Egerton

2023-03-09 Thread Manikumar
Congrats Chris!. Well deserved. On Fri, Mar 10, 2023 at 12:23 AM Josep Prat wrote: > > Congrats Chris! > > ——— > Josep Prat > > Aiven Deutschland GmbH > > Alexanderufer 3-7, 10117 Berlin > > Amtsgericht Charlottenburg, HRB 209739 B > > Geschäftsführer: Oskari Saarenmaa & Hannu Valtonen > > m: +49

Re: [VOTE] KIP-900: KRaft kafka-storage.sh API additions to support SCRAM for Kafka Brokers

2023-02-24 Thread Manikumar
+1 (binding) Thanks for the KIP. On Wed, Feb 22, 2023 at 3:48 AM José Armando García Sancio wrote: > > LGTM Proven. Thanks for the improvements. +1 (binding) > > -- > -José

Re: [Possible bug] Failing to use multiple client for multiple cluster using SASL channel.

2023-02-08 Thread Manikumar
Hi Sourav, Can you check if https://github.com/apache/kafka/pull/13211 can help to handle your scenario?. Thanks On Sun, Feb 5, 2023 at 6:23 AM Sourav Biswas wrote: > > Hello Kafka Dev, > Issue:Say, I need to configure multiple client (consumer/producer) listening > and publishing to different

CVE-2023-25194: Apache Kafka: Possible RCE/Denial of service attack via SASL JAAS JndiLoginModule configuration using Kafka Connect

2023-02-07 Thread Manikumar
Severity: important Description: A possible security vulnerability has been identified in Apache Kafka Connect. This requires access to a Kafka Connect worker, and the ability to create/modify connectors on it with an arbitrary Kafka client SASL JAAS config and a SASL-based security protocol, whi

Re: [VOTE] 3.3.2 RC1

2023-01-03 Thread Manikumar
Hi Chris, +1 (binding) - verified the signatures, artifacts - ran the tests on the source archive - verified the quickstarts Thanks for running the release! Thanks, Manikumar On Fri, Dec 23, 2022 at 4:44 PM Federico Valeri wrote: > > Hi, I did the following to validate the r

Re: [ANNOUNCE] New committer: Satish Duggana

2022-12-24 Thread Manikumar
Congrats, Satish! Well deserved. On Sat, Dec 24, 2022, 5:10 PM Tom Bentley wrote: > Congratulations! > > On Sat, 24 Dec 2022 at 05:05, Luke Chen wrote: > > > Congratulations, Satish! > > > > On Sat, Dec 24, 2022 at 4:12 AM Federico Valeri > > wrote: > > > > > Hi Satish, congrats! > > > > > >

[jira] [Resolved] (KAFKA-14320) Upgrade Jackson for CVE fix

2022-11-18 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14320?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-14320. --- Resolution: Fixed > Upgrade Jackson for CVE

[jira] [Resolved] (KAFKA-13518) Update gson dependency

2022-10-24 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13518?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-13518. --- Fix Version/s: 3.4.0 Resolution: Fixed > Update gson depende

Re: PR for CVE-2022-34917

2022-09-26 Thread Manikumar
https://issues.apache.org/jira/browse/KAFKA-14063?focusedCommentId=17608137&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17608137 On Mon, Sep 26, 2022 at 7:42 PM Swathi Mocharla wrote: > > Hi, > CVE: https://nvd.nist.gov/vuln/detail/CVE-2022-34917 > Could you pl

[jira] [Resolved] (KAFKA-14212) Fetch error response when hitting public OAuth/OIDC provider

2022-09-20 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14212?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-14212. --- Fix Version/s: 3.4.0 Resolution: Fixed > Fetch error response when hitting public OAuth/O

[jira] [Resolved] (KAFKA-14063) Kafka message parsing can cause ooms with small antagonistic payloads

2022-09-19 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14063?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-14063. --- Resolution: Fixed > Kafka message parsing can cause ooms with small antagonistic paylo

  1   2   3   4   5   6   7   8   9   10   >