[jira] [Commented] (KAFKA-16223) Replace EasyMock and PowerMock with Mockito for KafkaConfigBackingStoreTest

2024-02-14 Thread Chaitanya Mukka (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16223?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17817597#comment-17817597 ] Chaitanya Mukka commented on KAFKA-16223: - Hi [~hgeraldino]! I hope you are doing well. Are you

Re: [PR] MINOR: Fix KafkaAdminClientTest.testClientInstanceId [kafka]

2024-02-14 Thread via GitHub
dajac merged PR #15370: URL: https://github.com/apache/kafka/pull/15370 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

[jira] [Assigned] (KAFKA-16259) Immutable MetadataCache to improve client performance

2024-02-14 Thread Satish Duggana (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16259?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Satish Duggana reassigned KAFKA-16259: -- Assignee: Zhifeng Chen > Immutable MetadataCache to improve client performance >

Re: [PR] improve TopicCommandIntegrationTest to be less flaky [kafka]

2024-02-14 Thread via GitHub
showuon commented on PR #14891: URL: https://github.com/apache/kafka/pull/14891#issuecomment-1945384353 @jolshan , do you want to have another look? If no, I'm going to merge this PR this week. Thanks. -- This is an automated message from the Apache Git Service. To respond to the

Re: [PR] PROPOSAL: support async event execution in group coordinator [kafka]

2024-02-14 Thread via GitHub
github-actions[bot] commented on PR #14705: URL: https://github.com/apache/kafka/pull/14705#issuecomment-1945310660 This PR is being marked as stale since it has not had any activity in 90 days. If you would like to keep this PR alive, please ask a committer for review. If the PR has

Re: [PR] KAFKA-14569: Migrate Kafka Streams tests from Zookeeper to KRaft [kafka]

2024-02-14 Thread via GitHub
mdedetrich commented on PR #15341: URL: https://github.com/apache/kafka/pull/15341#issuecomment-194527 > Thanks for taking the comments into consideration. One other suggestion is that we can have a look into #13375 to see what is the common between both `EmbeddedKafkaCluster` in

Re: [PR] KAFKA-15625: Do not flush global state store at each commit [kafka]

2024-02-14 Thread via GitHub
jeffkbkim commented on code in PR #15361: URL: https://github.com/apache/kafka/pull/15361#discussion_r1490286153 ## streams/src/test/java/org/apache/kafka/streams/processor/internals/StateConsumerTest.java: ## @@ -161,6 +140,10 @@ public void update(final ConsumerRecord

[jira] [Updated] (KAFKA-16260) Remove window.size.ms from StreamsConfig

2024-02-14 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-16260: Component/s: streams > Remove window.size.ms from StreamsConfig >

[jira] [Updated] (KAFKA-16260) Remove window.size.ms from StreamsConfig

2024-02-14 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-16260: Labels: needs-kip (was: ) > Remove window.size.ms from StreamsConfig >

[jira] [Assigned] (KAFKA-16260) Remove window.size.ms from StreamsConfig

2024-02-14 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-16260: --- Assignee: Lucia Cerchie > Remove window.size.ms from StreamsConfig >

Re: [PR] KAFKA-15665: Enforce partition reassignment should complete when all target replicas are in ISR [kafka]

2024-02-14 Thread via GitHub
jolshan commented on PR #15359: URL: https://github.com/apache/kafka/pull/15359#issuecomment-1945185518 I will wait a day or to see if @cmccabe has any comments. If not I will merge.  -- This is an automated message from the Apache Git Service. To respond to the message, please log on

Re: [PR] KAFKA-16226 Reduce synchronization between producer threads [kafka]

2024-02-14 Thread via GitHub
ijuma commented on PR #15323: URL: https://github.com/apache/kafka/pull/15323#issuecomment-1945066461 @hachikuji @msn-tldr #15376 looks related. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to

Re: [PR] KAFKA-16259 Immutable MetadataCache to improve client performance [kafka]

2024-02-14 Thread via GitHub
ijuma commented on PR #15376: URL: https://github.com/apache/kafka/pull/15376#issuecomment-1945065055 Thanks for the PR. Is this similar to what was done in #15323? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the

Re: [PR] KAFKA-15770: IQv2 must return immutable position [kafka]

2024-02-14 Thread via GitHub
mjsax commented on PR #15219: URL: https://github.com/apache/kafka/pull/15219#issuecomment-1945045218 > Do we need to lock the implementations of AbstractDualSchemaRocksDBSegmentedBytesStore.getWriteBatches ? Outstanding catch! I believe yes. Updated the PR. -- This is an

[jira] [Updated] (KAFKA-16259) Immutable MetadataCache to improve client performance

2024-02-14 Thread Zhifeng Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16259?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhifeng Chen updated KAFKA-16259: - Description: TL;DR, A Kafka client produce latency issue is identified caused by synchronized

[jira] [Updated] (KAFKA-16259) Immutable MetadataCache to improve client performance

2024-02-14 Thread Zhifeng Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16259?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhifeng Chen updated KAFKA-16259: - Description: TL;DR, A Kafka client produce latency issue is identified caused by synchronized

[PR] KAFKA-16259 Immutable MetadataCache to improve client performance [kafka]

2024-02-14 Thread via GitHub
ericzhifengchen opened a new pull request, #15376: URL: https://github.com/apache/kafka/pull/15376 *More detailed description of your change, current MetadataCache is partially (but not fully) immutable, thus read/write requires synchronization and led to high produce latency with large

Re: [PR] KAFKA-16165: Fix invalid transition on poll timer expiration [kafka]

2024-02-14 Thread via GitHub
AndrewJSchofield commented on code in PR #15375: URL: https://github.com/apache/kafka/pull/15375#discussion_r1490086839 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java: ## @@ -731,10 +729,29 @@ private boolean

[jira] [Updated] (KAFKA-16260) Remove window.size.ms from StreamsConfig

2024-02-14 Thread Lucia Cerchie (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lucia Cerchie updated KAFKA-16260: -- Description: {{window.size.ms}}  is not a true KafkaStreams config, and results in an error

[jira] [Created] (KAFKA-16260) Remove window.size.ms from StreamsConfig

2024-02-14 Thread Lucia Cerchie (Jira)
Lucia Cerchie created KAFKA-16260: - Summary: Remove window.size.ms from StreamsConfig Key: KAFKA-16260 URL: https://issues.apache.org/jira/browse/KAFKA-16260 Project: Kafka Issue Type:

[jira] [Commented] (KAFKA-16260) Remove window.size.ms from StreamsConfig

2024-02-14 Thread Lucia Cerchie (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16260?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17817530#comment-17817530 ] Lucia Cerchie commented on KAFKA-16260: --- see discussion https://github.com/apache/kafka/pull/14360

[jira] [Commented] (KAFKA-15333) Flaky build failure throwing Connect Exception: Could not connect to server....

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15333?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17817529#comment-17817529 ] Greg Harris commented on KAFKA-15333: - Caused by this Gradle bug:

[jira] [Resolved] (KAFKA-15407) Not able to connect to kafka from the Private NLB from outside the VPC account

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15407?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris resolved KAFKA-15407. - Resolution: Invalid > Not able to connect to kafka from the Private NLB from outside the VPC >

[jira] [Commented] (KAFKA-15407) Not able to connect to kafka from the Private NLB from outside the VPC account

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17817524#comment-17817524 ] Greg Harris commented on KAFKA-15407: - Hi [~shivakumar] I don't think this is an appropriate place

Re: [PR] KAFKA-16165: Fix invalid transition on poll timer expiration [kafka]

2024-02-14 Thread via GitHub
kirktrue commented on code in PR #15375: URL: https://github.com/apache/kafka/pull/15375#discussion_r1490029659 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java: ## @@ -188,18 +188,18 @@ public HeartbeatRequestManager(

[jira] [Updated] (KAFKA-16165) Consumer invalid transition on expired poll interval

2024-02-14 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16165?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans updated KAFKA-16165: --- Description: Running system tests with the new async consumer revealed an invalid

[jira] [Commented] (KAFKA-15841) Add Support for Topic-Level Partitioning in Kafka Connect

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15841?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17817522#comment-17817522 ] Greg Harris commented on KAFKA-15841: - Hi [~henriquemota]! Can you share more details about your

[PR] KAFKA-16165: Fix invalid transition on poll timer expiration [kafka]

2024-02-14 Thread via GitHub
lianetm opened a new pull request, #15375: URL: https://github.com/apache/kafka/pull/15375 This fixes an invalid transition (leaving->stale) that was discovered in the system tests. The underlying issue was that the poll timer expiration logic was blindly forcing a transition to stale and

[jira] [Updated] (KAFKA-16259) Immutable MetadataCache to improve client performance

2024-02-14 Thread Zhifeng Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16259?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhifeng Chen updated KAFKA-16259: - Description: TL;DR, A Kafka client produce latency issue is identified caused by synchronized

[jira] [Updated] (KAFKA-16145) Windows Kafka Shutdown

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16145?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris updated KAFKA-16145: Component/s: core (was: connect) > Windows Kafka Shutdown >

[jira] [Created] (KAFKA-16259) Immutable MetadataCache to improve client performance

2024-02-14 Thread Zhifeng Chen (Jira)
Zhifeng Chen created KAFKA-16259: Summary: Immutable MetadataCache to improve client performance Key: KAFKA-16259 URL: https://issues.apache.org/jira/browse/KAFKA-16259 Project: Kafka Issue

[jira] [Updated] (KAFKA-9790) Wrong metric bean name for connect worker metrics

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9790?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris updated KAFKA-9790: --- Component/s: connect > Wrong metric bean name for connect worker metrics >

[jira] [Resolved] (KAFKA-10735) Kafka producer producing corrupted avro values when confluent cluster is recreated and producer application is not restarted

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10735?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris resolved KAFKA-10735. - Resolution: Cannot Reproduce Hi, since this doesn't contain any Apache Kafka code, and doesn't

[jira] [Updated] (KAFKA-12309) The revocation algorithm produces uneven distributions

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12309?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris updated KAFKA-12309: Component/s: connect > The revocation algorithm produces uneven distributions >

[jira] [Updated] (KAFKA-10719) MirrorMaker2 fails to update its runtime configuration

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10719?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris updated KAFKA-10719: Component/s: mirrormaker > MirrorMaker2 fails to update its runtime configuration >

[jira] [Resolved] (KAFKA-10719) MirrorMaker2 fails to update its runtime configuration

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10719?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris resolved KAFKA-10719. - Fix Version/s: 3.7.0 3.6.2 Resolution: Fixed > MirrorMaker2 fails to

[jira] [Updated] (KAFKA-10266) Fix connector configs in docs to mention the correct default value inherited from worker configs

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10266?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris updated KAFKA-10266: Component/s: connect > Fix connector configs in docs to mention the correct default value

[jira] [Updated] (KAFKA-14773) Make MirrorMaker startup synchronous

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14773?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris updated KAFKA-14773: Component/s: mirrormaker > Make MirrorMaker startup synchronous >

[jira] [Updated] (KAFKA-13656) Connect Transforms support for nested structures

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13656?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris updated KAFKA-13656: Component/s: connect > Connect Transforms support for nested structures >

[jira] [Updated] (KAFKA-14652) Improve MM2 logging by adding the flow information to the context (KIP-916)

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14652?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris updated KAFKA-14652: Component/s: mirrormaker > Improve MM2 logging by adding the flow information to the context

[jira] [Updated] (KAFKA-15892) Flaky test: testAlterSinkConnectorOffsets – org.apache.kafka.connect.integration.OffsetsApiIntegrationTest

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15892?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris updated KAFKA-15892: Component/s: connect > Flaky test: testAlterSinkConnectorOffsets – >

[jira] [Updated] (KAFKA-15914) Flaky test - testAlterSinkConnectorOffsetsOverriddenConsumerGroupId - OffsetsApiIntegrationTest

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15914?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris updated KAFKA-15914: Component/s: connect > Flaky test - testAlterSinkConnectorOffsetsOverriddenConsumerGroupId - >

[jira] [Updated] (KAFKA-14823) Clean up ConfigProvider API

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14823?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris updated KAFKA-14823: Component/s: connect > Clean up ConfigProvider API > --- > >

[jira] [Updated] (KAFKA-15891) Flaky test: testResetSinkConnectorOffsetsOverriddenConsumerGroupId – org.apache.kafka.connect.integration.OffsetsApiIntegrationTest

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15891?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris updated KAFKA-15891: Component/s: connect > Flaky test: testResetSinkConnectorOffsetsOverriddenConsumerGroupId – >

[jira] [Updated] (KAFKA-15918) Flaky test - OffsetsApiIntegrationTest.testResetSinkConnectorOffsets

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15918?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris updated KAFKA-15918: Component/s: connect > Flaky test - OffsetsApiIntegrationTest.testResetSinkConnectorOffsets >

[jira] [Updated] (KAFKA-13756) Connect validate endpoint should return proper response for invalid connector class

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13756?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris updated KAFKA-13756: Component/s: connect > Connect validate endpoint should return proper response for invalid

[jira] [Updated] (KAFKA-8314) Managing the doc field in case of schema projection - kafka connect

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8314?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris updated KAFKA-8314: --- Component/s: connect > Managing the doc field in case of schema projection - kafka connect >

[jira] [Updated] (KAFKA-16258) Stale member should trigger onPartitionsLost when leaving group

2024-02-14 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16258?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans updated KAFKA-16258: --- Labels: kip-848-client-support (was: ) > Stale member should trigger onPartitionsLost when

[jira] [Comment Edited] (KAFKA-16008) Fix PlaintextConsumerTest.testMaxPollIntervalMs

2024-02-14 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16008?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17817500#comment-17817500 ] Lianet Magrans edited comment on KAFKA-16008 at 2/14/24 7:25 PM: -

[jira] [Commented] (KAFKA-16008) Fix PlaintextConsumerTest.testMaxPollIntervalMs

2024-02-14 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16008?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17817500#comment-17817500 ] Lianet Magrans commented on KAFKA-16008: [~kirktrue] I just added the links to the issue that

[jira] [Updated] (KAFKA-16258) Stale member should trigger onPartitionsLost when leaving group

2024-02-14 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16258?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans updated KAFKA-16258: --- Description: When the poll timer expires, the new consumer proactively leaves the group and

[jira] [Created] (KAFKA-16258) Stale member should trigger onPartitionsLost when leaving group

2024-02-14 Thread Lianet Magrans (Jira)
Lianet Magrans created KAFKA-16258: -- Summary: Stale member should trigger onPartitionsLost when leaving group Key: KAFKA-16258 URL: https://issues.apache.org/jira/browse/KAFKA-16258 Project: Kafka

Re: [PR] KAFKA-16206 Fix unnecessary topic config deletion during ZK migration [kafka]

2024-02-14 Thread via GitHub
ahuang98 commented on PR #14206: URL: https://github.com/apache/kafka/pull/14206#issuecomment-193746 @mimaison @mumrah I moved unrelated test changes over to https://github.com/apache/kafka/pull/15373. The latest commit

[PR] [DRAFT] [KAFKA-16069] Source Tasks re-transform records after Retriable exceptions [kafka]

2024-02-14 Thread via GitHub
wasniksudesh opened a new pull request, #15374: URL: https://github.com/apache/kafka/pull/15374 If producer.send() throws a retriableException, exactly one single record would've been transformed (but not delivered). Simply storing SourceRecord and ProducerRecord (obtained after

[PR] Image test improvements [kafka]

2024-02-14 Thread via GitHub
ahuang98 opened a new pull request, #15373: URL: https://github.com/apache/kafka/pull/15373 More commit history and comments from https://github.com/apache/kafka/pull/14206 ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ]

[jira] [Comment Edited] (KAFKA-13505) Kafka Connect should respect Avro 1.10.X enum defaults spec

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13505?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17817494#comment-17817494 ] Greg Harris edited comment on KAFKA-13505 at 2/14/24 7:01 PM: -- Hi

[jira] [Commented] (KAFKA-13505) Kafka Connect should respect Avro 1.10.X enum defaults spec

2024-02-14 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13505?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17817494#comment-17817494 ] Greg Harris commented on KAFKA-13505: - Hi [~twbecker] Thanks for bringing this issue to our

[jira] [Created] (KAFKA-16257) SchemaProjector should be extensible to logical types

2024-02-14 Thread Greg Harris (Jira)
Greg Harris created KAFKA-16257: --- Summary: SchemaProjector should be extensible to logical types Key: KAFKA-16257 URL: https://issues.apache.org/jira/browse/KAFKA-16257 Project: Kafka Issue

Re: [PR] KAFKA-16206 Fix unnecessary topic config deletion during ZK migration [kafka]

2024-02-14 Thread via GitHub
ahuang98 commented on PR #14206: URL: https://github.com/apache/kafka/pull/14206#issuecomment-1944388740 @mimaison the PR was originally meant to introduce additional test changes, I believe @mumrah renamed it after I added the migration fix. I'll move the tests out and apply your

Re: [PR] 16069 - prevent re-transform of source-records after retriable exceptions [kafka]

2024-02-14 Thread via GitHub
wasniksudesh closed pull request #15140: 16069 - prevent re-transform of source-records after retriable exceptions URL: https://github.com/apache/kafka/pull/15140 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL

Re: [PR] MINOR: update leaderAndEpoch before initializing metadata publishers [kafka]

2024-02-14 Thread via GitHub
mumrah commented on code in PR #15366: URL: https://github.com/apache/kafka/pull/15366#discussion_r1489893040 ## metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java: ## @@ -197,13 +209,15 @@ private MetadataLoader( String threadNamePrefix,

Re: [PR] KAFKA-15586: Clean shutdown detection - server side [kafka]

2024-02-14 Thread via GitHub
CalvinConfluent commented on code in PR #14706: URL: https://github.com/apache/kafka/pull/14706#discussion_r1489894351 ## metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java: ## @@ -336,10 +342,14 @@ public ControllerResult registerBroker(

Re: [PR] KAFKA-15586: Clean shutdown detection - server side [kafka]

2024-02-14 Thread via GitHub
CalvinConfluent commented on code in PR #14706: URL: https://github.com/apache/kafka/pull/14706#discussion_r1489894070 ## metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java: ## @@ -780,4 +789,9 @@ public Entry> next() { } };

Re: [PR] [Draft] KIP-994 (Part 1) Minor Enhancements to ListTransactionsRequest [kafka]

2024-02-14 Thread via GitHub
RamanVerma commented on PR #14731: URL: https://github.com/apache/kafka/pull/14731#issuecomment-1944354242 > Hey @RamanVerma what is left here? We decided to bump the API version in the KIP. That and some testing. -- This is an automated message from the Apache Git Service. To

Re: [PR] [Draft] KIP-994 (Part 1) Minor Enhancements to ListTransactionsRequest [kafka]

2024-02-14 Thread via GitHub
jolshan commented on PR #14731: URL: https://github.com/apache/kafka/pull/14731#issuecomment-1944349380 Hey @RamanVerma what is left here? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the

Re: [PR] KAFKA-16226 Reduce synchronization between producer threads [kafka]

2024-02-14 Thread via GitHub
msn-tldr commented on code in PR #15323: URL: https://github.com/apache/kafka/pull/15323#discussion_r1489860692 ## clients/src/main/java/org/apache/kafka/clients/Metadata.java: ## @@ -279,7 +286,7 @@ synchronized Optional partitionMetadataIfCur * @return a mapping from

[jira] [Assigned] (KAFKA-16256) Update ConsumerConfig to validate use of group.remote.assignor and partition.assignment.strategy based on group.protocol

2024-02-14 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16256?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True reassigned KAFKA-16256: - Assignee: Kirk True > Update ConsumerConfig to validate use of group.remote.assignor and >

[jira] [Created] (KAFKA-16256) Update ConsumerConfig to validate use of group.remote.assignor and partition.assignment.strategy based on group.protocol

2024-02-14 Thread Kirk True (Jira)
Kirk True created KAFKA-16256: - Summary: Update ConsumerConfig to validate use of group.remote.assignor and partition.assignment.strategy based on group.protocol Key: KAFKA-16256 URL:

Re: [PR] KAFKA-16226 Reduce synchronization between producer threads [kafka]

2024-02-14 Thread via GitHub
msn-tldr commented on code in PR #15323: URL: https://github.com/apache/kafka/pull/15323#discussion_r1489832782 ## clients/src/main/java/org/apache/kafka/clients/Metadata.java: ## @@ -279,7 +286,7 @@ synchronized Optional partitionMetadataIfCur * @return a mapping from

[jira] [Updated] (KAFKA-16255) AsyncKafkaConsumer should not use partition.assignment.strategy

2024-02-14 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16255?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16255: -- Description: The {{partition.assignment.strategy}} configuration is used to specify a list of zero or

[jira] [Created] (KAFKA-16255) AsyncKafkaConsumer should not use partition.assignment.strategy

2024-02-14 Thread Kirk True (Jira)
Kirk True created KAFKA-16255: - Summary: AsyncKafkaConsumer should not use partition.assignment.strategy Key: KAFKA-16255 URL: https://issues.apache.org/jira/browse/KAFKA-16255 Project: Kafka

Re: [PR] KAFKA-15665: Enforce partition reassignment should complete when all target replicas are in ISR [kafka]

2024-02-14 Thread via GitHub
CalvinConfluent commented on PR #15359: URL: https://github.com/apache/kafka/pull/15359#issuecomment-1944225912 testIOExceptionDuringCheckpoint failure is irrelevant to the PR. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub

[jira] [Created] (KAFKA-16254) Allow MM2 to fully disable offset sync feature

2024-02-14 Thread Omnia Ibrahim (Jira)
Omnia Ibrahim created KAFKA-16254: - Summary: Allow MM2 to fully disable offset sync feature Key: KAFKA-16254 URL: https://issues.apache.org/jira/browse/KAFKA-16254 Project: Kafka Issue Type:

Re: [PR] KAFKA-15585: DescribeTopicPartitions client side change [kafka]

2024-02-14 Thread via GitHub
mumrah commented on code in PR #15265: URL: https://github.com/apache/kafka/pull/15265#discussion_r1489713255 ## clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java: ## @@ -2129,63 +2167,183 @@ private Map> handleDescribeTopicsByNames(f }

Re: [PR] KAFKA-15586: Clean shutdown detection - server side [kafka]

2024-02-14 Thread via GitHub
mumrah commented on code in PR #14706: URL: https://github.com/apache/kafka/pull/14706#discussion_r1489685922 ## metadata/src/main/java/org/apache/kafka/controller/BrokersToElrs.java: ## @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or

[jira] [Commented] (KAFKA-16212) Cache partitions by TopicIdPartition instead of TopicPartition

2024-02-14 Thread Omnia Ibrahim (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16212?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17817425#comment-17817425 ] Omnia Ibrahim commented on KAFKA-16212: --- Actually there is another proposal 4# which is to wait

Re: [PR] KAFKA-16206 Fix unnecessary topic config deletion during ZK migration [kafka]

2024-02-14 Thread via GitHub
mimaison commented on code in PR #14206: URL: https://github.com/apache/kafka/pull/14206#discussion_r1489636848 ## core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala: ## @@ -226,8 +226,8 @@ class ZkConfigMigrationClient( val (migrationZkVersion,

Re: [PR] KAFKA-14589: [2/3] Tests of ConsoleGroupCommand rewritten in java [kafka]

2024-02-14 Thread via GitHub
nizhikov commented on PR #15363: URL: https://github.com/apache/kafka/pull/15363#issuecomment-1944040321 Hello @showuon Can you, please, take a look? I've checked CI results and it seems OK for me. -- This is an automated message from the Apache Git Service. To respond to the

Re: [PR] KAFKA-14589: [3/3] Tests of ConsoleGroupCommand rewritten in java [kafka]

2024-02-14 Thread via GitHub
nizhikov commented on PR #15365: URL: https://github.com/apache/kafka/pull/15365#issuecomment-1944038174 Hello @showuon Can you, please, take a look? I've checked CI results and it seems OK for me. -- This is an automated message from the Apache Git Service. To respond

Re: [PR] KAFKA-16157: fix topic recreation handling with offline disks [kafka]

2024-02-14 Thread via GitHub
ijuma commented on code in PR #15263: URL: https://github.com/apache/kafka/pull/15263#discussion_r1489643087 ## core/src/main/scala/kafka/cluster/Partition.scala: ## @@ -874,7 +874,13 @@ class Partition(val topicPartition: TopicPartition, private def

Re: [PR] KAFKA-16155: Re-enable testAutoCommitIntercept [kafka]

2024-02-14 Thread via GitHub
lucasbru merged PR #15334: URL: https://github.com/apache/kafka/pull/15334 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

[jira] [Commented] (KAFKA-16212) Cache partitions by TopicIdPartition instead of TopicPartition

2024-02-14 Thread Omnia Ibrahim (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16212?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17817412#comment-17817412 ] Omnia Ibrahim commented on KAFKA-16212: --- The moment the cache in `ReplicaManager.allPartitions` is

Re: [PR] KAFKA-16226 Reduce synchronization between producer threads [kafka]

2024-02-14 Thread via GitHub
ijuma commented on code in PR #15323: URL: https://github.com/apache/kafka/pull/15323#discussion_r1489615650 ## clients/src/main/java/org/apache/kafka/clients/Metadata.java: ## @@ -279,7 +286,7 @@ synchronized Optional partitionMetadataIfCur * @return a mapping from

Re: [PR] KAFKA-16226 Reduce synchronization between producer threads [kafka]

2024-02-14 Thread via GitHub
ijuma commented on code in PR #15323: URL: https://github.com/apache/kafka/pull/15323#discussion_r1489615650 ## clients/src/main/java/org/apache/kafka/clients/Metadata.java: ## @@ -279,7 +286,7 @@ synchronized Optional partitionMetadataIfCur * @return a mapping from

[PR] KAFKA-16243: Make sure that we do not exceed max poll interval inside poll [kafka]

2024-02-14 Thread via GitHub
lucasbru opened a new pull request, #15372: URL: https://github.com/apache/kafka/pull/15372 The consumer keeps a poll timer, which is used to ensure liveness of the application thread. The poll timer automatically updates while the `Consumer.poll(Duration)` method is blocked, while the

Re: [PR] MINOR: Cleanup log.dirs in ReplicaManagerTest on JVM exit [kafka]

2024-02-14 Thread via GitHub
soarez commented on code in PR #15289: URL: https://github.com/apache/kafka/pull/15289#discussion_r1489605994 ## core/src/test/scala/unit/kafka/utils/TestUtils.scala: ## @@ -137,7 +137,18 @@ object TestUtils extends Logging { val parentFile = new File(parent)

Re: [PR] KAFKA-16225: Set `metadata.log.dir` to broker in KRAFT mode in integration test [kafka]

2024-02-14 Thread via GitHub
OmniaGM commented on code in PR #15354: URL: https://github.com/apache/kafka/pull/15354#discussion_r1489575856 ## core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala: ## @@ -168,6 +169,10 @@ class LogDirFailureTest extends IntegrationTestHarness { } def

Re: [PR] MINIOR: Remove accidentally logs [kafka]

2024-02-14 Thread via GitHub
OmniaGM commented on PR #15371: URL: https://github.com/apache/kafka/pull/15371#issuecomment-1943925566 cc: @jolshan can you merge this when you have a moment please? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use

[PR] MINIOR: Remove accidentally logs [kafka]

2024-02-14 Thread via GitHub
OmniaGM opened a new pull request, #15371: URL: https://github.com/apache/kafka/pull/15371 This logs was added during the troubleshooting for some flaky tests and was pushed by mistake in #15354 ### Committer Checklist (excluded from commit message) - [ ] Verify design and

Re: [PR] KAFKA-16225: Set `metadata.log.dir` to broker in KRAFT mode in integration test [kafka]

2024-02-14 Thread via GitHub
OmniaGM commented on code in PR #15354: URL: https://github.com/apache/kafka/pull/15354#discussion_r1489567388 ## core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala: ## @@ -168,6 +169,10 @@ class LogDirFailureTest extends IntegrationTestHarness { } def

Re: [PR] KAFKA-16225: Set `metadata.log.dir` to broker in KRAFT mode in integration test [kafka]

2024-02-14 Thread via GitHub
OmniaGM commented on code in PR #15354: URL: https://github.com/apache/kafka/pull/15354#discussion_r1489565890 ## core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala: ## @@ -168,6 +169,10 @@ class LogDirFailureTest extends IntegrationTestHarness { } def

Re: [PR] KAFKA-16225: Set `metadata.log.dir` to broker in KRAFT mode in integration test [kafka]

2024-02-14 Thread via GitHub
soarez commented on code in PR #15354: URL: https://github.com/apache/kafka/pull/15354#discussion_r1489560264 ## core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala: ## @@ -168,6 +169,10 @@ class LogDirFailureTest extends IntegrationTestHarness { } def

[PR] MINOR: Fix KafkaAdminClientTest.testClientInstanceId [kafka]

2024-02-14 Thread via GitHub
dajac opened a new pull request, #15370: URL: https://github.com/apache/kafka/pull/15370 This patch tries to address the

[jira] [Commented] (KAFKA-13505) Kafka Connect should respect Avro 1.10.X enum defaults spec

2024-02-14 Thread Tommy Becker (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13505?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17817385#comment-17817385 ] Tommy Becker commented on KAFKA-13505: -- I'm wondering if this bug is being overlooked because it's

[jira] [Updated] (KAFKA-16224) Fix handling of deleted topic when auto-committing before revocation

2024-02-14 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16224?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans updated KAFKA-16224: --- Labels: client-transitions-issues kip-848-client-support (was: kip-848-client-support) >

Re: [PR] KAFKA-16155: Re-enable testAutoCommitIntercept [kafka]

2024-02-14 Thread via GitHub
cadonna commented on code in PR #15334: URL: https://github.com/apache/kafka/pull/15334#discussion_r1489442547 ## core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala: ## @@ -1378,6 +1377,14 @@ class PlaintextConsumerTest extends BaseConsumerTest { //

Re: [PR] KAFKA-8977: Remove MockStreamsMetrics since it is not a mock [kafka]

2024-02-14 Thread via GitHub
cadonna commented on PR #13931: URL: https://github.com/apache/kafka/pull/13931#issuecomment-1943728460 @joobisb Could you please fix the compilation errors? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL

[jira] [Resolved] (KAFKA-16247) replica keep out-of-sync after migrating broker to KRaft

2024-02-14 Thread Luke Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16247?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Luke Chen resolved KAFKA-16247. --- Resolution: Fixed Fixed in 3.7.0 RC4 > replica keep out-of-sync after migrating broker to KRaft >

[jira] [Assigned] (KAFKA-16061) KRaft JBOD follow-ups and improvements

2024-02-14 Thread Igor Soarez (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16061?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Soarez reassigned KAFKA-16061: --- Assignee: Igor Soarez > KRaft JBOD follow-ups and improvements >

Re: [PR] KAFKA-16226 Reduce synchronization between producer threads [kafka]

2024-02-14 Thread via GitHub
msn-tldr commented on PR #15323: URL: https://github.com/apache/kafka/pull/15323#issuecomment-1943675980 Went through the test failures across all jdk/scala combos, they are unrelated, and have been failing before this PR as well

  1   2   >