Re: Apache Kafka 3.6.0 release

2023-07-21 Thread Satish Duggana
Thanks Hao for the update on KIP-925. On Thu, 20 Jul 2023 at 23:05, Hao Li wrote: > > Hi Satish, > > KIP-925 was accepted and currently under implementation. I just added it to > the release plan. > >

Re: [DISCUSS] KIP-953: partition method to be overloaded to accept headers as well.

2023-07-21 Thread Jack Tomy
Hey @Sagar, Thank you again for the response and feedback. 1. Though the ask wasn't very clear to me I have attached the Javadoc as per your suggestion. Please have a look and let me know if this meets the expectations. 2. Done. 3. Done 4. Done Hey @Sagar and everyone, Please

Jenkins build is unstable: Kafka » Kafka Branch Builder » trunk #2027

2023-07-21 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-759: Unneeded repartition canceling

2023-07-21 Thread Matthias J. Sax
I agree that it could easily be misused. There is a few Jira tickets for cases when people want to "cancel" a repartition step. I would hope those tickets are linked to the KIP (if not, we should do this, and maybe even c those cases as motivation into the KIP itself)? It's always a tricky

Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #2026

2023-07-21 Thread Apache Jenkins Server
See Changes: -- [...truncated 393019 lines...] Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > RestoreIntegrationTest >

Re: [DISCUSS] KIP-759: Unneeded repartition canceling

2023-07-21 Thread Sophie Blee-Goldman
I guess I felt a bit uneasy about how this could be used/abused while reading the KIP, but if we truly believe this is an advanced feature, I'm fine with the way things currently are. It doesn't feel like the best API, but it does seem to be the best *possible* API given the way things are. W.r.t

Re: [DISCUSS] KIP-954: expand default DSL store configuration to custom types

2023-07-21 Thread Sophie Blee-Goldman
I agree with everything Almog said above, and will just add on to two points: 1. Regarding whether to block this KIP on the completion of any or all future implementations of in-memory version stores (or persist suppression buffers), I feel that would be unfair to this feature which is completely

Re: [DISCUSS] KIP-759: Unneeded repartition canceling

2023-07-21 Thread Matthias J. Sax
Some thought about the API question. A. kstream.groupBy(...).aggregate(...) This can be re-writtten as kstream.selectKey(...) .markAsRepartitioned() .groupByKey() .aggregate() Given that `markAsRepartitoned` is an advanced feature, I think it would be ok? B.

[jira] [Resolved] (KAFKA-14950) Implement assign() and assignment()

2023-07-21 Thread Jun Rao (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14950?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-14950. - Fix Version/s: 3.6.0 Resolution: Fixed merged the PR to trunk. > Implement assign() and

Re: [DISCUSS] KIP-954: expand default DSL store configuration to custom types

2023-07-21 Thread Almog Gavra
Thanks for all the feedback folk! Responses inline. > Basically, I'm suggesting two things: first, call out in some way (perhaps the StoreTypeSpec javadocs) that each StoreTypeSpec is considered a public contract in itself and should outline any semantic guarantees it does, or does not, make.

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-07-21 Thread Nick Telford
One more thing: I noted John's suggestion in the KIP, under "Rejected Alternatives". I still think it's an idea worth pursuing, but I believe that it's out of the scope of this KIP, because it solves a different set of problems to this KIP, and the scope of this one has already grown quite large!

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-07-21 Thread Nick Telford
Hi everyone, I've updated the KIP ( https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores) with the latest changes; mostly bringing back "Atomic Checkpointing" (for what feels like the 10th time!). I think the one thing missing is some changes to

[DISCUSS] KIP-950: Tiered Storage Disablement

2023-07-21 Thread Beyene, Mehari
Hi everyone, I would like to start a discussion on KIP-950: Tiered Storage Disablement (https://cwiki.apache.org/confluence/display/KAFKA/KIP-950%3A++Tiered+Storage+Disablement). This KIP proposes adding the ability to disable and re-enable tiered storage on a topic. Thanks, Mehari

Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #2025

2023-07-21 Thread Apache Jenkins Server
See Changes: -- [...truncated 392388 lines...] Gradle Test Run :streams:integrationTest > Gradle Test Executor 181 > RestoreIntegrationTest >

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

2023-07-21 Thread Colin McCabe
On Fri, Jul 21, 2023, at 09:43, José Armando García Sancio wrote: > Thanks for the KIP Colin. Apologies if some of these points have > already been made. I have not followed the discussion closely: > > 1. Re: Periodically, each controller will check that the controller > registration for its ID is

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

2023-07-21 Thread Colin McCabe
On Fri, Jul 21, 2023, at 08:14, David Arthur wrote: > Hey Colin, thanks for the KIP! Some questions > Hi David, Thanks for reviewing. > 1) "This registration will include information about the endpoints which > they possess" Will this include all endpoints, or only those configured in >

Re: [VOTE] KIP-944 Support async runtimes in consumer, votes needed!

2023-07-21 Thread Matthias J. Sax
I am not a clients (or threading) expert, but I tend to agree to Colin's concerns. In particular, it would be nice to see an example how you intent to use the API (I am not familiar with Kotlin or it's co-routins), to better understand what this changes help to solve to begin with. Opening

Re: [DISCUSS] KIP-954: expand default DSL store configuration to custom types

2023-07-21 Thread Matthias J. Sax
Thanks for the KIP. Overall I like the idea to close this gap. However, I am wondering if we should close others gaps first? In particular, IIRC, we have a few cases for which we only have a RocksDB implementation for a store, and thus, adding an in-memory version for these stores first, to

Re: How to request review for pull request

2023-07-21 Thread Boudjelda Mohamed Said
Hello Wilma, This is a good contribution guide to start with, https://kafka.apache.org/contributing Yes you can start a merge request, first ensures well it builds locally and then the reviewers will check your merge request the of course you will get a feedback if something went wrong, you

RE: Re: [DISCUSS] KIP-951: Leader discovery optimisations for the client

2023-07-21 Thread Emanuele Sabellico
The downsides of bumping the version is that clients have to have all the latest features implemented before being able to benefit from this performance improvement. One of the benefits of using a tagged field is to make the field available to previous versions too. Choosing a minimum value

How to request review for pull request

2023-07-21 Thread Willma
Hi team, I just made my first pull request to the kafka repo. I wonder how can I request review for my pr, also do I need to have all 4 builds success before my pr can be reviewed? Thanks and looking forward to your reply.

Request contributor permissions to kafka

2023-07-21 Thread Boudjelda Mohamed Said
Hi I would like to have a contributor permission to kafka, this is my jira id: *bmscomp* Best regards

[jira] [Created] (KAFKA-15233) Add public documentation for plugin.discovery migration steps

2023-07-21 Thread Greg Harris (Jira)
Greg Harris created KAFKA-15233: --- Summary: Add public documentation for plugin.discovery migration steps Key: KAFKA-15233 URL: https://issues.apache.org/jira/browse/KAFKA-15233 Project: Kafka

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

2023-07-21 Thread José Armando García Sancio
Thanks for the KIP Colin. Apologies if some of these points have already been made. I have not followed the discussion closely: 1. Re: Periodically, each controller will check that the controller registration for its ID is as expected Does this need to be periodic? Can't the controller schedule

Re: Re: [DISCUSS] KIP-951: Leader discovery optimisations for the client

2023-07-21 Thread Mayank Shekhar Narula
David 03. Fixed as well, to remove ignorable. In MetadataResponse, Rack came a version later, hence was marked ignorable. Thanks. On Fri, Jul 21, 2023 at 1:38 PM Mayank Shekhar Narula < mayanks.nar...@gmail.com> wrote: > Hi David > > 01. My reasoning noted in the KIP is that CurrentLeader was

Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #2024

2023-07-21 Thread Apache Jenkins Server
See Changes: -- [...truncated 385594 lines...] * What went wrong: Execution failed for task ':storage:unitTest'. > Process 'Gradle Test Executor 140' finished with

[jira] [Resolved] (KAFKA-13431) Sink Connectors: Support topic-mutating SMTs for async connectors (preCommit users)

2023-07-21 Thread Chris Egerton (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13431?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chris Egerton resolved KAFKA-13431. --- Fix Version/s: 3.6.0 Resolution: Done > Sink Connectors: Support topic-mutating SMTs

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

2023-07-21 Thread David Arthur
Hey Colin, thanks for the KIP! Some questions 1) "This registration will include information about the endpoints which they possess" Will this include all endpoints, or only those configured in "advertised.listeners" 2) "Periodically, each controller will check that the controller registration

[jira] [Created] (KAFKA-15232) Move ToolsUtils to tools

2023-07-21 Thread Federico Valeri (Jira)
Federico Valeri created KAFKA-15232: --- Summary: Move ToolsUtils to tools Key: KAFKA-15232 URL: https://issues.apache.org/jira/browse/KAFKA-15232 Project: Kafka Issue Type: Sub-task

Re: [DISCUSS] KIP-949: Add flag to enable the usage of topic separator in MM2 DefaultReplicationPolicy

2023-07-21 Thread Federico Valeri
Hi, the point that the legacy approach can only be taken once is valid, so LGTM. Thanks. Cheers Fede On Fri, Jul 21, 2023 at 4:28 PM Chris Egerton wrote: > > Hi Omnia, > > LGTM, thanks! We may want to note the LegacyReplicationPolicy option in the > rejected alternatives section in case others

Re: [DISCUSS] KIP-949: Add flag to enable the usage of topic separator in MM2 DefaultReplicationPolicy

2023-07-21 Thread Chris Egerton
Hi Omnia, LGTM, thanks! We may want to note the LegacyReplicationPolicy option in the rejected alternatives section in case others prefer that option. Given that we'd like this to land in time for 3.6.0, you may want to start a vote thread soon. Cheers, Chris On Fri, Jul 21, 2023 at 10:08 AM

Re: [DISCUSS] KIP-949: Add flag to enable the usage of topic separator in MM2 DefaultReplicationPolicy

2023-07-21 Thread Omnia Ibrahim
Hi Chris and Federico, thinking about I think Chris's concern is valid and the bigger concern here is that having this `LegacyReplicationPolicy` will eventually open the door for diversion at some point between this `LegacyReplicationPolicy` and the default one. For now, let's have the flag

Re: [DISCUSS] KIP-954: expand default DSL store configuration to custom types

2023-07-21 Thread Colt McNealy
Sophie— Thanks for chiming in here. +1 to the idea of specifying the ordering guarantees that we make in the StorageTypeSpec javadocs. Quick question then. Is the javadoc that says: > Order is not guaranteed as bytes lexicographical ordering might not represent key order. no longer correct,

[jira] [Created] (KAFKA-15231) Add ability to pause/resume Remote Log Manager tasks

2023-07-21 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-15231: Summary: Add ability to pause/resume Remote Log Manager tasks Key: KAFKA-15231 URL: https://issues.apache.org/jira/browse/KAFKA-15231

Re: Re: [DISCUSS] KIP-951: Leader discovery optimisations for the client

2023-07-21 Thread Mayank Shekhar Narula
Hi David 01. My reasoning noted in the KIP is that CurrentLeader was first added in version 12, so 12 is the least version where clients could get these optimisations. So any client can now choose to implement this with version 12 of the protocol itself. If the version is bumped to X(say 16),

[jira] [Created] (KAFKA-15230) ApiVersions data between controllers is not reliable

2023-07-21 Thread David Arthur (Jira)
David Arthur created KAFKA-15230: Summary: ApiVersions data between controllers is not reliable Key: KAFKA-15230 URL: https://issues.apache.org/jira/browse/KAFKA-15230 Project: Kafka Issue

Request permission to contribute

2023-07-21 Thread Taras Ledkov
Hi, Kafka Team. I'm following this wiki to request permission to contribute to Apache Kafka https://cwiki.apache.org/confluence/display/kafka/kafka+improvement+proposals#KafkaImprovementProposals-GettingStarted I'll propose custom SSL factory for Kafka Connect REST server

[jira] [Created] (KAFKA-15229) Increase default value of task.shutdown.graceful.timeout.ms

2023-07-21 Thread Sagar Rao (Jira)
Sagar Rao created KAFKA-15229: - Summary: Increase default value of task.shutdown.graceful.timeout.ms Key: KAFKA-15229 URL: https://issues.apache.org/jira/browse/KAFKA-15229 Project: Kafka Issue

Re: Re: [DISCUSS] KIP-951: Leader discovery optimisations for the client

2023-07-21 Thread Ismael Juma
Thanks for the update Crispin - very helpful to have actual performance data. 2-5% for the default configuration is a bit on the low side for this kind of proposal. Ismael On Thu, Jul 20, 2023 at 11:33 PM Crispin Bernier wrote: > Benchmark numbers have been posted on the KIP, please review. >

Re: Re: [DISCUSS] KIP-951: Leader discovery optimisations for the client

2023-07-21 Thread David Jacot
Hi Mayank, Thanks for the KIP. This is an interesting idea that I have been thinking about for a long time so I am happy to see it. The gain is smaller than I expected but still worth it in my opinion. 01. In the FetchResponse, what's the reason for using version `12+` for the new tagged field

Re: [DISCUSS] KIP-943: Add independent "offset.storage.segment.bytes" for connect-distributed.properties

2023-07-21 Thread Sagar
Hey Hudeqi, Thanks for the KIP! After reading the KIP and the comments by Yash and Greg I agree with these aspects: 1) While I agree that having a high value for segment.btes config can lead to higher startup times, we don't necessarily need to expose a separate config for it(as Yash suggested).

[jira] [Reopened] (KAFKA-14581) Move GetOffsetShell to tools

2023-07-21 Thread Federico Valeri (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14581?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Federico Valeri reopened KAFKA-14581: - > Move GetOffsetShell to tools > > > Key:

[jira] [Resolved] (KAFKA-15222) Upgrade zinc scala incremental compiler plugin version to a latests stable fit version (1.9.2)

2023-07-21 Thread Divij Vaidya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15222?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Divij Vaidya resolved KAFKA-15222. -- Fix Version/s: 3.6.0 Resolution: Fixed > Upgrade zinc scala incremental compiler