Re: Apache Kafka 3.7.0 Release

2023-11-02 Thread Sophie Blee-Goldman
Looks great, thank you! +1 On Thu, Nov 2, 2023 at 10:21 AM David Jacot wrote: > +1 from me as well. Thanks, Stan! > > David > > On Thu, Nov 2, 2023 at 6:04 PM Ismael Juma wrote: > > > Thanks Stanislav, +1 > > > > Ismael > > > > On Thu, Nov 2, 2023 at 7:01 AM Stanislav Kozlovski > > wrote: > >

[VOTE] KIP-998: Give ProducerConfig(props, doLog) constructor protected access

2023-11-02 Thread Sophie Blee-Goldman
Hey all, This is a trivial one-liner change that it was determined should go through a KIP during the PR review process (see this thread for context). Since the change itself was already reviewed and approved I'm skipping the

[jira] [Created] (KAFKA-15782) Establish concrete project conventions to define public APIs that require a KIP

2023-11-02 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-15782: -- Summary: Establish concrete project conventions to define public APIs that require a KIP Key: KAFKA-15782 URL:

[jira] [Created] (KAFKA-15781) Change ProducerConfig(props, doLog) constructor to protected

2023-11-02 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-15781: -- Summary: Change ProducerConfig(props, doLog) constructor to protected Key: KAFKA-15781 URL: https://issues.apache.org/jira/browse/KAFKA-15781

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

2023-11-02 Thread Matthias J. Sax
Almog, can you explain how you intent to implement this change? It's not clear to me, how we could do this? When we call `StreasmBuilder.build()` it will give us a already fully wired `Topology`, including all store suppliers needed. I don't see a clean way how we could change the store

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

2023-11-02 Thread Almog Gavra
Hello everyone - I updated the KIP to also include the following modification: Both the new dsl.store.suppliers.class and the old default.dsl.store will now respect the configurations when passed in via KafkaStreams#new(Topology, StreamsConfig) (and other related constructors) instead of only

Re: [DISCUSS] Apache Kafka 3.5.2 release

2023-11-02 Thread Luke Chen
Hi Matthias, Is there any update about the test status for RocksDB versions bumps? Could I create a 3.5.2 RC build next week? Thanks. Luke On Sat, Oct 21, 2023 at 1:01 PM Luke Chen wrote: > Hi Matthias, > > I agree it's indeed a blocker for 3.5.2 to address CVE in RocksDB. > Please let me

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

2023-11-02 Thread Apache Jenkins Server
See Changes: -- [...truncated 430414 lines...] Gradle Test Run :streams:test > Gradle Test Executor 88 > SynchronizedPartitionGroupTest > testAddRawRecords() PASSED

[jira] [Created] (KAFKA-15780) Wait for consistent kraft metadata when creating topics in tests

2023-11-02 Thread David Mao (Jira)
David Mao created KAFKA-15780: - Summary: Wait for consistent kraft metadata when creating topics in tests Key: KAFKA-15780 URL: https://issues.apache.org/jira/browse/KAFKA-15780 Project: Kafka

Re: [VOTE] KIP-992 Proposal to introduce IQv2 Query Types: TimestampedKeyQuery and TimestampedRangeQuery

2023-11-02 Thread Hao Li
Hi Hanyu, Thanks for the KIP! +1 (non-binding) Hao On Thu, Nov 2, 2023 at 1:29 PM Bill Bejeck wrote: > Hi Hanyu, > > Thanks for the KIP this LGTM. > +1 (binding) > > Thanks, > Bill > > > > On Wed, Nov 1, 2023 at 1:07 PM Hanyu (Peter) Zheng > wrote: > > > Hello everyone, > > > > I would like

Re: [DISCUSS] KIP-992 Proposal to introduce IQv2 Query Types: TimestampedKeyQuery and TimestampedRangeQuery

2023-11-02 Thread Hao Li
Thanks Hanyu! Make sense. I was mistaken that timestamp is also in the key for these queries. Hao On Mon, Oct 30, 2023 at 1:46 PM Hanyu (Peter) Zheng wrote: > Hi, Hao, > > For TimestampedKeyQuery, it only returns the value of the key, and the > value should be ValueAndTimestamp. > If you want

Build failed in Jenkins: Kafka » Kafka Branch Builder » 3.6 #104

2023-11-02 Thread Apache Jenkins Server
See Changes: -- [...truncated 302605 lines...] Gradle Test Run :connect:runtime:test > Gradle Test Executor 44 > org.apache.kafka.connect.storage.OffsetUtilsTest >

Re: [DISCUSS] KIP-992 Proposal to introduce IQv2 Query Types: TimestampedKeyQuery and TimestampedRangeQuery

2023-11-02 Thread Bill Bejeck
Hi Hanyu, One minor comment. Can you update the KIP to add the behavior when developer uses a TimestampedXQuery against a non-timestamped store? Thanks! Bill On Mon, Oct 30, 2023 at 4:46 PM Hanyu (Peter) Zheng wrote: > Hi, Hao, > > For TimestampedKeyQuery, it only returns the value of the

Re: [VOTE] KIP-992 Proposal to introduce IQv2 Query Types: TimestampedKeyQuery and TimestampedRangeQuery

2023-11-02 Thread Bill Bejeck
Hi Hanyu, Thanks for the KIP this LGTM. +1 (binding) Thanks, Bill On Wed, Nov 1, 2023 at 1:07 PM Hanyu (Peter) Zheng wrote: > Hello everyone, > > I would like to start a vote for KIP-992: Proposal to introduce IQv2 Query > Types: TimestampedKeyQuery and TimestampedRangeQuery. > > Sincerely,

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

2023-11-02 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-15779) Define broker configurations and exceptions

2023-11-02 Thread Apoorv Mittal (Jira)
Apoorv Mittal created KAFKA-15779: - Summary: Define broker configurations and exceptions Key: KAFKA-15779 URL: https://issues.apache.org/jira/browse/KAFKA-15779 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-15778) Implement ClientMetricsManager to process request

2023-11-02 Thread Apoorv Mittal (Jira)
Apoorv Mittal created KAFKA-15778: - Summary: Implement ClientMetricsManager to process request Key: KAFKA-15778 URL: https://issues.apache.org/jira/browse/KAFKA-15778 Project: Kafka Issue

[jira] [Resolved] (KAFKA-15317) Fix for async consumer access to committed offsets with multiple consumers

2023-11-02 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15317?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans resolved KAFKA-15317. Resolution: Fixed Failing test consistently passing after PR

[jira] [Created] (KAFKA-15777) Configurable remote fetch bytes per partition from Consumer

2023-11-02 Thread Kamal Chandraprakash (Jira)
Kamal Chandraprakash created KAFKA-15777: Summary: Configurable remote fetch bytes per partition from Consumer Key: KAFKA-15777 URL: https://issues.apache.org/jira/browse/KAFKA-15777 Project:

[jira] [Created] (KAFKA-15776) Configurable delay timeout for DelayedRemoteFetch request

2023-11-02 Thread Kamal Chandraprakash (Jira)
Kamal Chandraprakash created KAFKA-15776: Summary: Configurable delay timeout for DelayedRemoteFetch request Key: KAFKA-15776 URL: https://issues.apache.org/jira/browse/KAFKA-15776 Project:

Re: [DISCUSS] KIP-968: Support single-key_multi-timestamp interactive queries (IQv2) for versioned state stores

2023-11-02 Thread Bruno Cadonna
Hi Alieh, First of all, I like the examples. Is validTo in VersionedRecord exclusive or inclusive? In the javadocs you write: "@param validTothe latest timestamp that value is valid" I think that is not true because the validity is defined by the start time of the new version. The new

Re: Apache Kafka 3.7.0 Release

2023-11-02 Thread Justine Olshan
This makes sense to me. Thanks for following up, Stan. On Thu, Nov 2, 2023 at 7:02 AM Stanislav Kozlovski wrote: > Hi all, > > Given the discussion here and the lack of any pushback, I have changed the > dates of the release: > - KIP Freeze - *November 22 *(moved 4 days later) > - Feature

Re: [DISCUSS] KIP-963: Upload and delete lag metrics in Tiered Storage

2023-11-02 Thread Kamal Chandraprakash
Hi Christo, Thanks for expanding the scope of the KIP! We should also cover the time taken to read data from remote storage. This will give our users a fair idea about the P99, P95, and P50 Fetch latency to read data from remote storage. The Fetch API request metrics currently provides a

[jira] [Created] (KAFKA-15775) Implement listTopics() and partitionFor() for the AsyncKafkaConsumer

2023-11-02 Thread Philip Nee (Jira)
Philip Nee created KAFKA-15775: -- Summary: Implement listTopics() and partitionFor() for the AsyncKafkaConsumer Key: KAFKA-15775 URL: https://issues.apache.org/jira/browse/KAFKA-15775 Project: Kafka

Re: Apache Kafka 3.7.0 Release

2023-11-02 Thread David Jacot
+1 from me as well. Thanks, Stan! David On Thu, Nov 2, 2023 at 6:04 PM Ismael Juma wrote: > Thanks Stanislav, +1 > > Ismael > > On Thu, Nov 2, 2023 at 7:01 AM Stanislav Kozlovski > wrote: > > > Hi all, > > > > Given the discussion here and the lack of any pushback, I have changed > the > >

Re: [DISCUSS] KIP-967: Support custom SSL configuration for Kafka Connect RestServer

2023-11-02 Thread Chris Egerton
Hi Taras, Thanks for the changes to the KIP! Regarding item 4: I think some background may be helpful for people without context on the Connect code base. The current parsing logic for SSL-related properties used with the REST API is to use all worker properties prefixed with "listeners.https."

Re: Apache Kafka 3.7.0 Release

2023-11-02 Thread Ismael Juma
Thanks Stanislav, +1 Ismael On Thu, Nov 2, 2023 at 7:01 AM Stanislav Kozlovski wrote: > Hi all, > > Given the discussion here and the lack of any pushback, I have changed the > dates of the release: > - KIP Freeze - *November 22 *(moved 4 days later) > - Feature Freeze - *December 6 *(moved 2

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

2023-11-02 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-15679) Client support for new consumer configs

2023-11-02 Thread Philip Nee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15679?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Nee resolved KAFKA-15679. Resolution: Fixed > Client support for new consumer configs >

[jira] [Resolved] (KAFKA-15531) Ensure coordinator node is removed upon disconnection exception

2023-11-02 Thread Philip Nee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15531?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Nee resolved KAFKA-15531. Resolution: Fixed > Ensure coordinator node is removed upon disconnection exception >

[jira] [Resolved] (KAFKA-15694) New integration tests to have full coverage for preview

2023-11-02 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15694?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True resolved KAFKA-15694. --- Assignee: Kirk True Resolution: Duplicate > New integration tests to have full coverage for

[jira] [Created] (KAFKA-15774) Respect default.dsl.store Configuration Without Passing it to StreamsBuilder

2023-11-02 Thread Almog Gavra (Jira)
Almog Gavra created KAFKA-15774: --- Summary: Respect default.dsl.store Configuration Without Passing it to StreamsBuilder Key: KAFKA-15774 URL: https://issues.apache.org/jira/browse/KAFKA-15774 Project:

[jira] [Resolved] (KAFKA-15316) CommitRequestManager not calling RequestState callbacks

2023-11-02 Thread Philip Nee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15316?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Nee resolved KAFKA-15316. Resolution: Fixed > CommitRequestManager not calling RequestState callbacks >

[jira] [Resolved] (KAFKA-15562) Ensure fetch offset and commit offset handler handles both timeout and various error types

2023-11-02 Thread Philip Nee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15562?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Nee resolved KAFKA-15562. Resolution: Fixed > Ensure fetch offset and commit offset handler handles both timeout and >

[jira] [Created] (KAFKA-15773) Group protocol configuration should be validated

2023-11-02 Thread Philip Nee (Jira)
Philip Nee created KAFKA-15773: -- Summary: Group protocol configuration should be validated Key: KAFKA-15773 URL: https://issues.apache.org/jira/browse/KAFKA-15773 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-967: Support custom SSL configuration for Kafka Connect RestServer

2023-11-02 Thread Taras Ledkov
Hi Chris, Thanks a lot for such a close review. > 1. The "ssl.engine.factory.class" property was originally added for Kafka > brokers in KIP-519 [1]. It'd be nice to link to that KIP (possibly in a > "Background" section? Added "Background" section. > 2. Can we clarify that the new

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

2023-11-02 Thread Mickael Maison
Hi Raghu, Thanks for the KIP. Have you considered retrieving these values using AuthenticationContext? For example SslAuthenticationContext could have a getter for SslPrincipalMapper. For kerberosShortNamer we could have a new subclass of SaslAuthenticationContext, for example

Re: Apache Kafka 3.7.0 Release

2023-11-02 Thread Stanislav Kozlovski
Hi all, Given the discussion here and the lack of any pushback, I have changed the dates of the release: - KIP Freeze - *November 22 *(moved 4 days later) - Feature Freeze - *December 6 *(moved 2 days earlier) - Code Freeze - *December 20* If anyone has any thoughts against this proposal -

[jira] [Created] (KAFKA-15772) Flay test TransactionsWithTieredStoreTest

2023-11-02 Thread Divij Vaidya (Jira)
Divij Vaidya created KAFKA-15772: Summary: Flay test TransactionsWithTieredStoreTest Key: KAFKA-15772 URL: https://issues.apache.org/jira/browse/KAFKA-15772 Project: Kafka Issue Type: Test

[jira] [Resolved] (KAFKA-15326) Decouple Processing Thread from Polling Thread

2023-11-02 Thread Lucas Brutschy (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15326?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lucas Brutschy resolved KAFKA-15326. Resolution: Implemented > Decouple Processing Thread from Polling Thread >

[jira] [Resolved] (KAFKA-15461) Add integration test for the ListGroup API

2023-11-02 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15461?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-15461. - Fix Version/s: 3.7.0 Reviewer: David Jacot Assignee: Dongnuo Lyu

[jira] [Resolved] (KAFKA-15598) Add integration tests for DescribeGroups API, DeleteGroups API and OffsetDelete API

2023-11-02 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15598?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-15598. - Fix Version/s: 3.7.0 Resolution: Fixed > Add integration tests for DescribeGroups API,

Re: Requesting permission to contribute to Apache Kafka

2023-11-02 Thread Yash Mayya
Hi Vedarth, I've granted you the necessary permissions. Thanks for your interest in contributing to Apache Kafka! Cheers, Yash On Thu, Nov 2, 2023 at 1:22 PM Vedarth Sharma wrote: > Hi, > > Please grant me permission to contribute to KIPs and assign jira tickets to > myself. > > wiki id and

Re: [DISCUSS] KIP-936: Throttle number of active PIDs

2023-11-02 Thread Claude Warren
I don't know why I missed this message. You don't have to update the max entries for the shape. Set the max entries to be the highest quota. Then you can use the BloomFilter.estimateN() method to determine how many PIDs have been inserted into the filter. On Wed, Aug 30, 2023 at 1:19 PM Omnia

Requesting permission to contribute to Apache Kafka

2023-11-02 Thread Vedarth Sharma
Hi, Please grant me permission to contribute to KIPs and assign jira tickets to myself. wiki id and jira id :- vedarth email :- vedarth.sha...@gmail.com Regards, Vedarth