Re: [VOTE] KIP-1037: Allow WriteTxnMarkers API with Alter Cluster Permission

2024-04-22 Thread Christo Lolov
Heya Nikhil, Thanks for the proposal, as mentioned before it makes sense to me! +1 (binding) Best, Christo On Sat, 20 Apr 2024 at 00:25, Justine Olshan wrote: > Hey Nikhil, > > I meant to comment on the discussion thread, but my draft took so long, you > opened the vote. > > Regardless, I jus

Re: [DISCUSS] KIP-1033: Add Kafka Streams exception handler for exceptions occuring during processing

2024-04-22 Thread Bruno Cadonna
Hi Damien, Thanks a lot for the updates! I have the following comments: (1) Could you rename ProcessingMetadata to ErrorHandlerContext or ErrorHandlerMetadata (I am preferring the former)? I think it makes it clearer for what this context/metadata is for. (2) Is there any reason you did no

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

2024-04-22 Thread Andrew Schofield
Hi Jun, Thanks for your comments. 120. Thanks. Fixed. 121. ShareUpdateValue.SnapshotEpoch indicates which snapshot the update applies to. It should of course be the snapshot that precedes it in the log. It’s just there to provide a consistency check. I also noticed that ShareSnapshotValue was mi

[jira] [Resolved] (KAFKA-15736) KRaft support in PlaintextConsumerTest

2024-04-22 Thread Walter Hernandez (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15736?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Walter Hernandez resolved KAFKA-15736. -- Resolution: Done > KRaft support in PlaintextConsumerTest > --

[jira] [Created] (KAFKA-16596) Flaky test – org.apache.kafka.clients.ClientUtilsTest.testParseAndValidateAddressesWithReverseLookup()

2024-04-22 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16596: --- Summary: Flaky test – org.apache.kafka.clients.ClientUtilsTest.testParseAndValidateAddressesWithReverseLookup() Key: KAFKA-16596 URL: https://issues.apache.org/jira/browse/KAFKA-1

[jira] [Created] (KAFKA-16597) Flaky test - org.apache.kafka.streams.integration.StoreQueryIntegrationTest.shouldQuerySpecificStalePartitionStoresMultiStreamThreads()

2024-04-22 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16597: --- Summary: Flaky test - org.apache.kafka.streams.integration.StoreQueryIntegrationTest.shouldQuerySpecificStalePartitionStoresMultiStreamThreads() Key: KAFKA-16597 URL: https://issues

[jira] [Resolved] (KAFKA-16596) Flaky test – org.apache.kafka.clients.ClientUtilsTest.testParseAndValidateAddressesWithReverseLookup()

2024-04-22 Thread Viktor Somogyi-Vass (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16596?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Viktor Somogyi-Vass resolved KAFKA-16596. - Fix Version/s: 3.8.0 Assignee: Andras Katona Resolution: Fixed >

Re: [VOTE] KIP-1037: Allow WriteTxnMarkers API with Alter Cluster Permission

2024-04-22 Thread Andrew Schofield
Hi Nikhil, Thanks for the KIP. Looks good to me. +1 (non-binding) Thanks, Andrew > On 22 Apr 2024, at 09:17, Christo Lolov wrote: > > Heya Nikhil, > > Thanks for the proposal, as mentioned before it makes sense to me! > > +1 (binding) > > Best, > Christo > > On Sat, 20 Apr 2024 at 00:25, Justin

Re: [DISCUSS] KIP-1034: Dead letter queue in Kafka Streams

2024-04-22 Thread Lucas Brutschy
Hi! Thanks for the KIP, great stuff. L1. I was a bit confused that the default configuration (once you set a DLQ topic) is going to be fail-and-send-to-DLQ, if I understood correctly. Is this something that will be a common use-case, and is it a configuration that we want to encourage? It expecte

[jira] [Resolved] (KAFKA-16549) suppress the warnings from RemoteLogManager

2024-04-22 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16549?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-16549. Fix Version/s: 3.8.0 Resolution: Fixed > suppress the warnings from RemoteLogManage

Re: [DISCUSS] KIP-1038: Add Custom Error Handler to Producer

2024-04-22 Thread Alieh Saeedi
Thank you all for the feedback! Addressing the main concern: The KIP is about giving the user the ability to handle producer exceptions, but to be more conservative and avoid future issues, we decided to be limited to a short list of exceptions. I included *RecordTooLargeExceptin* and *UnknownTopi

[jira] [Created] (KAFKA-16598) Mirgrate `ResetConsumerGroupOffsetTest` to new test infra

2024-04-22 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-16598: -- Summary: Mirgrate `ResetConsumerGroupOffsetTest` to new test infra Key: KAFKA-16598 URL: https://issues.apache.org/jira/browse/KAFKA-16598 Project: Kafka

Re: [DISCUSS] KIP-1033: Add Kafka Streams exception handler for exceptions occuring during processing

2024-04-22 Thread Damien Gasparina
Hi Bruno, 1) Good point, naming stuff is definitely hard, I renamed ProcessingMetadata to ErrorHandlerContext > Is there any reason you did not use something like > Record sourceRecord() 2) The record class is used in many locations and, I assume, could be expanded by new features. As the error

[jira] [Created] (KAFKA-16599) Always await async commit callbacks in commitSync and close

2024-04-22 Thread Lucas Brutschy (Jira)
Lucas Brutschy created KAFKA-16599: -- Summary: Always await async commit callbacks in commitSync and close Key: KAFKA-16599 URL: https://issues.apache.org/jira/browse/KAFKA-16599 Project: Kafka

[jira] [Resolved] (KAFKA-16103) Review client logic for triggering offset commit callbacks

2024-04-22 Thread Lucas Brutschy (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16103?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lucas Brutschy resolved KAFKA-16103. Resolution: Fixed > Review client logic for triggering offset commit callbacks > -

Re: [VOTE] KIP-853: KRaft Controller Membership Changes

2024-04-22 Thread Jason Gustafson
Thanks Jose. +1. Great KIP! On Fri, Mar 29, 2024 at 11:16 AM Jun Rao wrote: > Hi, Jose, > > Thanks for the KIP. +1 > > Jun > > On Fri, Mar 29, 2024 at 9:55 AM José Armando García Sancio > wrote: > > > Hi all, > > > > I would like to call a vote to adopt KIP-853. > > > > KIP: https://cwiki.apach

Re: [VOTE] KIP-853: KRaft Controller Membership Changes

2024-04-22 Thread José Armando García Sancio
+1 binding. On Mon, Apr 22, 2024 at 9:28 AM Jason Gustafson wrote: > > Thanks Jose. +1. Great KIP! > > On Fri, Mar 29, 2024 at 11:16 AM Jun Rao wrote: > > > Hi, Jose, > > > > Thanks for the KIP. +1 > > > > Jun > > > > On Fri, Mar 29, 2024 at 9:55 AM José Armando García Sancio > > wrote: > > > >

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

2024-04-22 Thread Vedarth Sharma
Hey folks, Thanks a lot for reviewing the KIP and providing feedback. The discussion thread seems resolved and KIP has been updated accordingly. We will be starting the voting thread for this KIP in the next few days. Please take a look at the KIP and let us know if any further discussion is neede

Re: [DISCUSS] KIP-1023: Follower fetch from tiered offset

2024-04-22 Thread Abhijeet Kumar
Hi Luke, Thanks for your comments. Please find my responses inline. On Tue, Apr 9, 2024 at 2:08 PM Luke Chen wrote: > Hi Abhijeet, > > Thanks for the KIP to improve the tiered storage feature! > > Questions: > 1. We could also get the "pending-upload-offset" and epoch via remote log > metadata,

Re: [VOTE] KIP-853: KRaft Controller Membership Changes

2024-04-22 Thread José Armando García Sancio
I am going to close the vote tomorrow morning (PST). On Mon, Apr 22, 2024 at 10:06 AM José Armando García Sancio wrote: > > +1 binding. > > On Mon, Apr 22, 2024 at 9:28 AM Jason Gustafson > wrote: > > > > Thanks Jose. +1. Great KIP! > > > > On Fri, Mar 29, 2024 at 11:16 AM Jun Rao wrote: > > >

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

2024-04-22 Thread Jun Rao
Hi, Andrew, Thanks for the reply. 123. "The share group does not persist the target assignment." What's the impact of this? Everytime that GC fails over, it needs to recompute the assignment for every member. Do we expect the member assignment to change on every GC failover? 125. Should the GC

Re: [DISCUSS] KIP-1023: Follower fetch from tiered offset

2024-04-22 Thread Abhijeet Kumar
Hi Jun, Please find my comments inline. On Thu, Apr 18, 2024 at 3:26 AM Jun Rao wrote: > Hi, Abhijeet, > > Thanks for the reply. > > 1. I am wondering if we could achieve the same result by just lowering > local.retention.ms and local.retention.bytes. This also allows the newly > started follo

[jira] [Created] (KAFKA-16600) Periodically receive "Failed to transition to PENDING_SHUTDOWN, current state is PENDING_SHUTDOWN" during streams close

2024-04-22 Thread Alex Leung (Jira)
Alex Leung created KAFKA-16600: -- Summary: Periodically receive "Failed to transition to PENDING_SHUTDOWN, current state is PENDING_SHUTDOWN" during streams close Key: KAFKA-16600 URL: https://issues.apache.org/jira/b

[DISCUSS] KIP-1034: Dead letter queue in Kafka Streams

2024-04-22 Thread Sebastien Viale
Hi, Thanks for your remarks L1. I would say "who can do the most can do the least", even though most people will fail and stop, we found it interesting to offer the possibility to fail-and-send-to-DLQ L2: We did not consider extending the TimestampExtractor because we estimate it out of scope

Jenkins build is unstable: Kafka » Kafka Branch Builder » 3.7 #142

2024-04-22 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-16601) Flaky test – org.apache.kafka.controller.QuorumControllerMetricsIntegrationTest.testClosingQuorumControllerClosesMetrics()

2024-04-22 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16601: --- Summary: Flaky test – org.apache.kafka.controller.QuorumControllerMetricsIntegrationTest.testClosingQuorumControllerClosesMetrics() Key: KAFKA-16601 URL: https://issues.apache.org/j

[jira] [Created] (KAFKA-16602) Flaky test – org.apache.kafka.controller.QuorumControllerTest.testBootstrapZkMigrationRecord()

2024-04-22 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16602: --- Summary: Flaky test – org.apache.kafka.controller.QuorumControllerTest.testBootstrapZkMigrationRecord() Key: KAFKA-16602 URL: https://issues.apache.org/jira/browse/KAFKA-16602

[jira] [Created] (KAFKA-16603) Data loss when kafka connect sending data to Kafka

2024-04-22 Thread Anil Dasari (Jira)
Anil Dasari created KAFKA-16603: --- Summary: Data loss when kafka connect sending data to Kafka Key: KAFKA-16603 URL: https://issues.apache.org/jira/browse/KAFKA-16603 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-1033: Add Kafka Streams exception handler for exceptions occuring during processing

2024-04-22 Thread Matthias J. Sax
Thanks for all the updates. Great discussion. Few follow up questions from my side: 99: I agree with Damien about Bruno's point (2). We should not return `Record` (cf point 103 below why). 100: Can we imagine a case, for which the `ProcessingExceptionHandler` would want/need to have access

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

2024-04-22 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-1035: StateStore managed changelog offsets

2024-04-22 Thread Matthias J. Sax
Thanks for splitting out this KIP. The discussion shows, that it is a complex beast by itself, so worth to discuss by its own. Couple of question / comment: 100 `StateStore#commit()`: The JavaDoc says "must not be called by users" -- I would propose to put a guard in place for this, by eithe

Re: [DISCUSS] KIP-1038: Add Custom Error Handler to Producer

2024-04-22 Thread Matthias J. Sax
Thanks Alieh! A few nits: 1) The new config we add for the producer should be mentioned in the "Public Interfaces" section. 2) Why do we use `producer.` prefix for a *producer* config? Should it be `exception.handler` only? -Matthias On 4/22/24 7:38 AM, Alieh Saeedi wrote: Thank you al

Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-22 Thread Andrew Schofield
Hi Fred, Just reviewing the KIP again now that the discussion has quietened down a little. It will soon be ready for a vote I think. I have a few comments about details. A1) The signature of the new constructor for RecordDeserializationException needs to be updated accord to the discussion. I see