Re: [PR] KAFKA-18129: Simplifying share partition maybeInitialize code [kafka]

2024-12-08 Thread via GitHub
apoorvmittal10 commented on code in PR #18093: URL: https://github.com/apache/kafka/pull/18093#discussion_r1874735278 ## core/src/main/java/kafka/server/share/SharePartition.java: ## @@ -371,52 +370,35 @@ public static RecordState forId(byte id) { */ public Completabl

Re: [PR] KAFKA-18129: Simplifying share partition maybeInitialize code [kafka]

2024-12-08 Thread via GitHub
apoorvmittal10 commented on code in PR #18093: URL: https://github.com/apache/kafka/pull/18093#discussion_r1874735359 ## core/src/main/java/kafka/server/share/SharePartition.java: ## @@ -426,55 +408,50 @@ public CompletableFuture maybeInitialize() { .build())

[PR] KAFKA-18021: Disabled MirrorCheckpointConnector throws RetriableException on task config generation [kafka]

2024-12-08 Thread via GitHub
frankvicky opened a new pull request, #18098: URL: https://github.com/apache/kafka/pull/18098 JIRA: KAFKA-18021 When a `MirrorCheckpointConnector` is disabled, the `start` method exits early without initiating `loadInitialConsumerGroups`. If the connector is restarted in a disable

Re: [PR] KAFKA-18021: Disabled MirrorCheckpointConnector throws RetriableException on task config generation [kafka]

2024-12-08 Thread via GitHub
frankvicky commented on PR #18098: URL: https://github.com/apache/kafka/pull/18098#issuecomment-2525655979 Hi @gharris1727 Could you please take a look when you have some time? Many thanks 🙇🏼 -- This is an automated message from the Apache Git Service. To respond to the message, pl

Re: [PR] KAFKA-18156: VerifiableConsumer should ignore "--session-timeout" when using CONSUMER protocol [kafka]

2024-12-08 Thread via GitHub
brandboat commented on code in PR #18036: URL: https://github.com/apache/kafka/pull/18036#discussion_r1874795473 ## tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java: ## @@ -649,8 +648,12 @@ public static VerifiableConsumer createFromArgs(ArgumentParser parser,

[jira] [Commented] (KAFKA-18183) ClusterInstance's helper should use byte array instead of Bytes in creating producer/consumer

2024-12-08 Thread Logan Zhu (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18183?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17903860#comment-17903860 ] Logan Zhu commented on KAFKA-18183: --- Sorry, [~suresh7]. I’m already working on this.

[PR] KAFKA-18135: ShareConsumer HB UnsupportedVersion msg mixed with Consumer HB [kafka]

2024-12-08 Thread via GitHub
peterxcli opened a new pull request, #18101: URL: https://github.com/apache/kafka/pull/18101 UnsupportedVersion error is handled in the parent AbstractHeartbeatRequestManager, so used by consumer and share consumer. If a share consumer gets the errors, it will end up with a msg that is curr

Re: [PR] KAFKA-17811: Separate modules to use different JDKs [kafka]

2024-12-08 Thread via GitHub
chia7712 commented on PR #17522: URL: https://github.com/apache/kafka/pull/17522#issuecomment-2526275233 open [KAFKA-18186](https://issues.apache.org/jira/browse/KAFKA-18186) to address this. @ableegoldman please feel free to share your thoughts on this approach. -- This is an auto

Re: [PR] [WIP] KIP-891: Connect Multiversioning Support (Configs and Validation changes for Connectors and Converters) [kafka]

2024-12-08 Thread via GitHub
gharris1727 commented on code in PR #17741: URL: https://github.com/apache/kafka/pull/17741#discussion_r1874983220 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/CachedConnectors.java: ## @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF

Re: [PR] KAFKA-16143: New JMX metrics for AsyncKafkaConsumer [kafka]

2024-12-08 Thread via GitHub
lianetm commented on code in PR #17199: URL: https://github.com/apache/kafka/pull/17199#discussion_r1872002514 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ## @@ -1893,25 +1901,30 @@ private void subscribeInternal(Collection topi

Re: [PR] KAFKA-16143: New JMX metrics for AsyncKafkaConsumer [kafka]

2024-12-08 Thread via GitHub
lianetm commented on code in PR #17199: URL: https://github.com/apache/kafka/pull/17199#discussion_r1875064838 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ## @@ -1893,25 +1901,30 @@ private void subscribeInternal(Collection topi

Re: [PR] KAFKA-18164: Clear existing acknowledgements on share session epoch reset. [kafka]

2024-12-08 Thread via GitHub
AndrewJSchofield commented on code in PR #18063: URL: https://github.com/apache/kafka/pull/18063#discussion_r1875029001 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManager.java: ## @@ -990,16 +990,18 @@ UnsentRequest buildRequest() {

Re: [PR] MINOR: Extract some codeblocks as methods to simplify readability [kafka]

2024-12-08 Thread via GitHub
AndrewJSchofield commented on code in PR #18017: URL: https://github.com/apache/kafka/pull/18017#discussion_r1875032057 ## core/src/main/java/kafka/server/TierStateMachine.java: ## @@ -229,12 +229,8 @@ private Long buildRemoteLogAuxState(TopicPartition topicPartition,

Re: [PR] KAFKA-18135: ShareConsumer HB UnsupportedVersion msg mixed with Consumer HB [kafka]

2024-12-08 Thread via GitHub
AndrewJSchofield commented on PR #18101: URL: https://github.com/apache/kafka/pull/18101#issuecomment-2526360864 We should get @lianetm to make sure she's happy too. -- 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-18058: Share group state record pruning impl. [kafka]

2024-12-08 Thread via GitHub
AndrewJSchofield commented on PR #18014: URL: https://github.com/apache/kafka/pull/18014#issuecomment-2526362182 @smjn Please resolve the conflicts. -- 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 t

Re: [PR] MINOR: Extract some codeblocks as methods to simplify readability [kafka]

2024-12-08 Thread via GitHub
overpathz commented on code in PR #18017: URL: https://github.com/apache/kafka/pull/18017#discussion_r1875034438 ## core/src/main/java/kafka/server/TierStateMachine.java: ## @@ -229,12 +229,8 @@ private Long buildRemoteLogAuxState(TopicPartition topicPartition, }

Re: [PR] KAFKA-18135: ShareConsumer HB UnsupportedVersion msg mixed with Consumer HB [kafka]

2024-12-08 Thread via GitHub
AndrewJSchofield commented on code in PR #18101: URL: https://github.com/apache/kafka/pull/18101#discussion_r1875034076 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java: ## @@ -100,6 +100,9 @@ public abstract class Abstrac

Re: [PR] KAFKA-18014: Add duration based offset reset option for ShareConsumer [kafka]

2024-12-08 Thread via GitHub
AndrewJSchofield commented on code in PR #18096: URL: https://github.com/apache/kafka/pull/18096#discussion_r1875037969 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/ShareGroupAutoOffsetResetStrategy.java: ## @@ -0,0 +1,162 @@ +/* + * Licensed to the Apac

Re: [PR] [WIP] KIP-891: Connect Multiversioning Support (Configs and Validation changes for Connectors and Converters) [kafka]

2024-12-08 Thread via GitHub
snehashisp commented on PR #17741: URL: https://github.com/apache/kafka/pull/17741#issuecomment-2526364647 Thanks for the review @gharris1727. Will get to work on the soon, in my daytime. -- This is an automated message from the Apache Git Service. To respond to the message, please log o

Re: [PR] MINOR: Extract some codeblocks as methods to simplify readability [kafka]

2024-12-08 Thread via GitHub
overpathz commented on code in PR #18017: URL: https://github.com/apache/kafka/pull/18017#discussion_r1875039076 ## core/src/main/java/kafka/server/TierStateMachine.java: ## @@ -229,12 +229,8 @@ private Long buildRemoteLogAuxState(TopicPartition topicPartition, }

[jira] [Assigned] (KAFKA-18186) add sourceCompatibility back to build.gradle to allow idea to configure suitable language level automatically

2024-12-08 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18186?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai reassigned KAFKA-18186: -- Assignee: 黃竣陽 (was: Chia-Ping Tsai) > add sourceCompatibility back to build.gradle t

[jira] [Updated] (KAFKA-18186) add sourceCompatibility back to build.gradle to allow idea to configure suitable language level automatically

2024-12-08 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18186?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai updated KAFKA-18186: --- Parent: KAFKA-16096 Issue Type: Sub-task (was: Improvement) > add sourceCompatibili

Re: [PR] KAFKA-18058: Share group state record pruning impl. [kafka]

2024-12-08 Thread via GitHub
smjn commented on PR #18014: URL: https://github.com/apache/kafka/pull/18014#issuecomment-2526373284 > @smjn Please resolve the conflicts. @AndrewJSchofield done -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use t

[jira] [Commented] (KAFKA-18186) add sourceCompatibility back to build.gradle to allow idea to configure suitable language level automatically

2024-12-08 Thread Jira
[ https://issues.apache.org/jira/browse/KAFKA-18186?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17903938#comment-17903938 ] 黃竣陽 commented on KAFKA-18186: - Hello, [~chia7712] , if you wont work on this, may I take the

[jira] [Created] (KAFKA-18186) add sourceCompatibility back to build.gradle to allow idea to configure suitable language level automatically

2024-12-08 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-18186: -- Summary: add sourceCompatibility back to build.gradle to allow idea to configure suitable language level automatically Key: KAFKA-18186 URL: https://issues.apache.org/jira/bro

Re: [PR] KAFKA-17811: Separate modules to use different JDKs [kafka]

2024-12-08 Thread via GitHub
ijuma commented on PR #17522: URL: https://github.com/apache/kafka/pull/17522#issuecomment-2526306797 I see some confusion here and I'll try to clarify it. > Are you using JDK 11 to run the streams tests? If so, that could be an issue since the generator module requires JDK 17. I assu

Re: [PR] KAFKA-18129: Simplifying share partition maybeInitialize code [kafka]

2024-12-08 Thread via GitHub
chia7712 commented on code in PR #18093: URL: https://github.com/apache/kafka/pull/18093#discussion_r1875006046 ## core/src/main/java/kafka/server/share/SharePartition.java: ## @@ -371,52 +370,35 @@ public static RecordState forId(byte id) { */ public CompletableFutur

Re: [PR] KAFKA-17811: Separate modules to use different JDKs [kafka]

2024-12-08 Thread via GitHub
chia7712 commented on PR #17522: URL: https://github.com/apache/kafka/pull/17522#issuecomment-2526274179 @ableegoldman Thanks for your feedback. I agree that we should prioritize making our developers' experience better. I plan to add sourceCompatibility back to build.gradle since IntelliJ

[jira] [Resolved] (KAFKA-18101) Merge duplicate assertFutureThrows and assertFutureExceptionTypeEquals

2024-12-08 Thread Peter Lee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18101?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Lee resolved KAFKA-18101. --- Resolution: Done > Merge duplicate assertFutureThrows and assertFutureExceptionTypeEquals >

[jira] [Assigned] (KAFKA-13560) Load indexes and data in async manner in the critical path of replica fetcher threads.

2024-12-08 Thread Peter Lee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13560?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Lee reassigned KAFKA-13560: - Assignee: Peter Lee > Load indexes and data in async manner in the critical path of replica fet

Re: [PR] KAFKA-18129: Simplifying share partition maybeInitialize code [kafka]

2024-12-08 Thread via GitHub
chia7712 merged PR #18093: URL: https://github.com/apache/kafka/pull/18093 -- 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-unsubscr...@kafka.apache

Re: [PR] KAFKA-18135: ShareConsumer HB UnsupportedVersion msg mixed with Consumer HB [kafka]

2024-12-08 Thread via GitHub
lianetm commented on code in PR #18101: URL: https://github.com/apache/kafka/pull/18101#discussion_r1875074269 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerHeartbeatRequestManager.java: ## @@ -97,6 +99,18 @@ public boolean handleSpecificError(fina

Re: [PR] KAFKA-18015: Add support for duration based offset reset strategy to Kafka Streams [kafka]

2024-12-08 Thread via GitHub
mjsax commented on code in PR #17973: URL: https://github.com/apache/kafka/pull/17973#discussion_r1875208289 ## streams/src/main/java/org/apache/kafka/streams/AutoOffsetReset.java: ## @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more +

Re: [PR] KAFKA-17696 New consumer background operations unaware of metadata errors [kafka]

2024-12-08 Thread via GitHub
lianetm commented on code in PR #17440: URL: https://github.com/apache/kafka/pull/17440#discussion_r1875198457 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java: ## @@ -325,4 +333,21 @@ void cleanup() { log.debug("Closed

Re: [PR] KAFKA-17696 New consumer background operations unaware of metadata errors [kafka]

2024-12-08 Thread via GitHub
lianetm commented on code in PR #17440: URL: https://github.com/apache/kafka/pull/17440#discussion_r1875210056 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java: ## @@ -150,7 +155,11 @@ private void maybePropagateMetadataError() {

Re: [PR] KAFKA-17696 New consumer background operations unaware of metadata errors [kafka]

2024-12-08 Thread via GitHub
lianetm commented on code in PR #17440: URL: https://github.com/apache/kafka/pull/17440#discussion_r1875210056 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java: ## @@ -150,7 +155,11 @@ private void maybePropagateMetadataError() {

Re: [PR] KAFKA-18015: Add support for duration based offset reset strategy to Kafka Streams [kafka]

2024-12-08 Thread via GitHub
mjsax commented on code in PR #17973: URL: https://github.com/apache/kafka/pull/17973#discussion_r1875208289 ## streams/src/main/java/org/apache/kafka/streams/AutoOffsetReset.java: ## @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more +

Re: [PR] KAFKA-17696 New consumer background operations unaware of metadata errors [kafka]

2024-12-08 Thread via GitHub
lianetm commented on code in PR #17440: URL: https://github.com/apache/kafka/pull/17440#discussion_r1875210056 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java: ## @@ -150,7 +155,11 @@ private void maybePropagateMetadataError() {

[PR] KAFKA-18184:Remove the unnecessary project path check from build.gradle [kafka]

2024-12-08 Thread via GitHub
Rancho-7 opened a new pull request, #18102: URL: https://github.com/apache/kafka/pull/18102 jira:https://issues.apache.org/jira/browse/KAFKA-18184 In the build.gradle file, the project path is not initialized, so the project.path should always refer to the root path ':' The con

Re: [PR] KAFKA-17696 New consumer background operations unaware of metadata errors [kafka]

2024-12-08 Thread via GitHub
lianetm commented on code in PR #17440: URL: https://github.com/apache/kafka/pull/17440#discussion_r1875232483 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java: ## @@ -212,9 +213,27 @@ public void testPropagateMetadataError() {

Re: [PR] KAFKA-17696 New consumer background operations unaware of metadata errors [kafka]

2024-12-08 Thread via GitHub
m1a2st commented on code in PR #17440: URL: https://github.com/apache/kafka/pull/17440#discussion_r1875248156 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java: ## @@ -212,9 +213,27 @@ public void testPropagateMetadataError() {

Re: [PR] KAFKA-18015: Add support for duration based offset reset strategy to Kafka Streams [kafka]

2024-12-08 Thread via GitHub
mjsax commented on code in PR #17973: URL: https://github.com/apache/kafka/pull/17973#discussion_r1875208289 ## streams/src/main/java/org/apache/kafka/streams/AutoOffsetReset.java: ## @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more +

Re: [PR] KAFKA-18015: Add support for duration based offset reset strategy to Kafka Streams [kafka]

2024-12-08 Thread via GitHub
mjsax commented on code in PR #17973: URL: https://github.com/apache/kafka/pull/17973#discussion_r1875212818 ## streams/src/main/java/org/apache/kafka/streams/AutoOffsetReset.java: ## @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more +

[jira] [Updated] (KAFKA-12843) KIP-740 follow up: clean up TaskMetadata

2024-12-08 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12843?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-12843: Parent: KAFKA-12822 Issue Type: Sub-task (was: Task) > KIP-740 follow up: clean u

Re: [PR] KAFKA-16339: [4/4 KStream#flatTransformValues] Remove Deprecated "transformer" methods and classes [kafka]

2024-12-08 Thread via GitHub
mjsax commented on PR #17882: URL: https://github.com/apache/kafka/pull/17882#issuecomment-2526845764 Thanks for the PR. Merged to `trunk`. -- 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 spe

Re: [PR] KAFKA-18014: Add duration based offset reset option for ShareConsumer [kafka]

2024-12-08 Thread via GitHub
peterxcli commented on code in PR #18096: URL: https://github.com/apache/kafka/pull/18096#discussion_r1875313096 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupConfig.java: ## @@ -53,8 +50,14 @@ public final class GroupConfig extends AbstractConfig {

[jira] [Resolved] (KAFKA-12843) KIP-740 follow up: clean up TaskMetadata

2024-12-08 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12843?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-12843. - Resolution: Duplicate (was: Fixed) > KIP-740 follow up: clean up TaskMetadata > ---

Re: [PR] KAFKA-16339: [4/4 KStream#flatTransformValues] Remove Deprecated "transformer" methods and classes [kafka]

2024-12-08 Thread via GitHub
mjsax merged PR #17882: URL: https://github.com/apache/kafka/pull/17882 -- 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-unsubscr...@kafka.apache.or

[jira] [Commented] (KAFKA-12843) KIP-740 follow up: clean up TaskMetadata

2024-12-08 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12843?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17904014#comment-17904014 ] Matthias J. Sax commented on KAFKA-12843: - KIP-740 overlaps with KIP-744 which d

Re: [PR] KAFKA-12829: Remove deprecated Topology#addProcessor of old Processor API [kafka]

2024-12-08 Thread via GitHub
mjsax commented on PR #17190: URL: https://github.com/apache/kafka/pull/17190#issuecomment-2526857196 Thanks for your understanding. And thanks for contributing! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL

[PR] MINOR: remove old procesor API MockInternalProcessorContext [kafka]

2024-12-08 Thread via GitHub
mjsax opened a new pull request, #18103: URL: https://github.com/apache/kafka/pull/18103 (no comment) -- 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

Re: [PR] MINOR: remove old procesor API MockInternalProcessorContext [kafka]

2024-12-08 Thread via GitHub
mjsax commented on code in PR #18103: URL: https://github.com/apache/kafka/pull/18103#discussion_r1875348859 ## streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBBlockCacheMetricsTest.java: ## @@ -79,17 +80,22 @@ static void withStore(final RocksDBSto

Re: [PR] MINOR: remove old procesor API MockInternalProcessorContext [kafka]

2024-12-08 Thread via GitHub
mjsax commented on code in PR #18103: URL: https://github.com/apache/kafka/pull/18103#discussion_r1875350982 ## streams/src/test/java/org/apache/kafka/test/MockInternalNewProcessorContext.java: ## @@ -108,12 +112,12 @@ public void setHeaders(final Headers headers) { }

Re: [PR] MINOR: remove old procesor API MockInternalProcessorContext [kafka]

2024-12-08 Thread via GitHub
mjsax commented on code in PR #18103: URL: https://github.com/apache/kafka/pull/18103#discussion_r1875348688 ## streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java: ## @@ -1049,11 +1048,12 @@ private static void putRecord(final Time

Re: [PR] MINOR: remove old procesor API MockInternalProcessorContext [kafka]

2024-12-08 Thread via GitHub
mjsax commented on code in PR #18103: URL: https://github.com/apache/kafka/pull/18103#discussion_r1875350717 ## streams/src/test/java/org/apache/kafka/test/MockInternalNewProcessorContext.java: ## @@ -41,12 +41,16 @@ import org.apache.kafka.streams.state.internals.ThreadCache.

Re: [PR] KAFKA-18164: Clear existing acknowledgements on share session epoch reset. [kafka]

2024-12-08 Thread via GitHub
ShivsundarR commented on code in PR #18063: URL: https://github.com/apache/kafka/pull/18063#discussion_r1875353133 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManager.java: ## @@ -990,16 +990,18 @@ UnsentRequest buildRequest() {

Re: [PR] MINOR: remove old procesor API MockInternalProcessorContext [kafka]

2024-12-08 Thread via GitHub
mjsax commented on code in PR #18103: URL: https://github.com/apache/kafka/pull/18103#discussion_r1875349000 ## streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBBlockCacheMetricsTest.java: ## @@ -98,11 +104,14 @@ public void shouldRecordCorrectBlockC

[jira] [Commented] (KAFKA-18187) Replicas that receive EndQuorum should grant preVotes in that epoch

2024-12-08 Thread Peter Lee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18187?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17904027#comment-17904027 ] Peter Lee commented on KAFKA-18187: --- Hi [~alyssahuang], does this depend on [https://

[jira] [Created] (KAFKA-18187) Replicas that receive EndQuorum should grant preVotes in that epoch

2024-12-08 Thread Alyssa Huang (Jira)
Alyssa Huang created KAFKA-18187: Summary: Replicas that receive EndQuorum should grant preVotes in that epoch Key: KAFKA-18187 URL: https://issues.apache.org/jira/browse/KAFKA-18187 Project: Kafka

Re: [PR] KAFKA-18134: Disallow group upgrades when custom assignors are used [kafka]

2024-12-08 Thread via GitHub
squah-confluent commented on code in PR #18046: URL: https://github.com/apache/kafka/pull/18046#discussion_r1875402835 ## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java: ## @@ -10077,6 +10082,116 @@ barTopicName, new TopicMetadat

[jira] [Assigned] (KAFKA-18187) Replicas that receive EndQuorum should grant preVotes in that epoch

2024-12-08 Thread Peter Lee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18187?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Lee reassigned KAFKA-18187: - Assignee: Peter Lee > Replicas that receive EndQuorum should grant preVotes in that epoch > ---

Re: [PR] KAFKA-18134: Disallow group upgrades when custom assignors are used [kafka]

2024-12-08 Thread via GitHub
squah-confluent commented on code in PR #18046: URL: https://github.com/apache/kafka/pull/18046#discussion_r1875401373 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java: ## @@ -1053,6 +1055,12 @@ ConsumerGroup convertToConsumerGroup(

Re: [PR] KAFKA-18134: Disallow group upgrades when custom assignors are used [kafka]

2024-12-08 Thread via GitHub
squah-confluent commented on code in PR #18046: URL: https://github.com/apache/kafka/pull/18046#discussion_r1875401870 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java: ## @@ -1053,6 +1055,13 @@ ConsumerGroup convertToConsumerGroup(

Re: [PR] [WIP] KIP-891: Connect Multiversioning Support (Configs and Validation changes for Connectors and Converters) [kafka]

2024-12-08 Thread via GitHub
snehashisp commented on code in PR #17741: URL: https://github.com/apache/kafka/pull/17741#discussion_r1875404360 ## .gitignore: ## @@ -61,3 +61,6 @@ storage/kafka-tiered-storage/ docker/test/report_*.html kafka.Kafka __pycache__ +/connect/runtime/src/main/java/org/apache/kaf

Re: [PR] KAFKA-18156: VerifiableConsumer should ignore "--session-timeout" when using CONSUMER protocol [kafka]

2024-12-08 Thread via GitHub
brandboat commented on code in PR #18036: URL: https://github.com/apache/kafka/pull/18036#discussion_r1874816095 ## tests/kafkatest/services/verifiable_consumer.py: ## @@ -417,10 +417,13 @@ def start_cmd(self, node): else: cmd += " --bootstrap-server %s" %

Re: [PR] KAFKA-18156: VerifiableConsumer should ignore "--session-timeout" when using CONSUMER protocol [kafka]

2024-12-08 Thread via GitHub
brandboat commented on code in PR #18036: URL: https://github.com/apache/kafka/pull/18036#discussion_r1874823531 ## tests/kafkatest/services/verifiable_consumer.py: ## @@ -251,8 +251,6 @@ def __init__(self, context, num_nodes, kafka, topic, group_id, self.session_timeo

[jira] [Commented] (KAFKA-18014) Add duration based offset reset option for ShareConsumer

2024-12-08 Thread Peter Lee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18014?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17903916#comment-17903916 ] Peter Lee commented on KAFKA-18014: --- Hi [~omkreddy], PR is ready, PTAL. Thanks! > Add

[jira] [Closed] (KAFKA-12843) KIP-740 follow up: clean up TaskMetadata

2024-12-08 Thread Jira
[ https://issues.apache.org/jira/browse/KAFKA-12843?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] João Pedro Fonseca closed KAFKA-12843. -- > KIP-740 follow up: clean up TaskMetadata > > >

[PR] [WIP] KAFKA-10409: Refactor Kafka Streams RocksDb iterators [kafka]

2024-12-08 Thread via GitHub
fonsdant opened a new pull request, #18099: URL: https://github.com/apache/kafka/pull/18099 (no comment) -- 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-m

Re: [PR] KAFKA-18015: Add support for duration based offset reset strategy to Kafka Streams [kafka]

2024-12-08 Thread via GitHub
peterxcli commented on code in PR #17973: URL: https://github.com/apache/kafka/pull/17973#discussion_r1874837232 ## streams/src/main/java/org/apache/kafka/streams/AutoOffsetReset.java: ## @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or mor

Re: [PR] KAFKA-18014: Add duration based offset reset option for ShareConsumer [kafka]

2024-12-08 Thread via GitHub
peterxcli commented on code in PR #18096: URL: https://github.com/apache/kafka/pull/18096#discussion_r1874838353 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/ShareGroupAutoOffsetResetStrategy.java: ## @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Soft

Re: [PR] KAFKA-18015: Add support for duration based offset reset strategy to Kafka Streams [kafka]

2024-12-08 Thread via GitHub
peterxcli commented on code in PR #17973: URL: https://github.com/apache/kafka/pull/17973#discussion_r1874837232 ## streams/src/main/java/org/apache/kafka/streams/AutoOffsetReset.java: ## @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or mor

Re: [PR] KAFKA-18014: Add duration based offset reset option for ShareConsumer [kafka]

2024-12-08 Thread via GitHub
peterxcli commented on PR #18096: URL: https://github.com/apache/kafka/pull/18096#issuecomment-2526019079 Hi @omkreddy, PR is ready, PTAL. Thanks! Sorry for mentioning you both side, just to make sure you do receive that 😁~ -- This is an automated message from the Apache Git Service. To

Re: [PR] KAFKA-18014: Add duration based offset reset option for ShareConsumer [kafka]

2024-12-08 Thread via GitHub
peterxcli commented on code in PR #18096: URL: https://github.com/apache/kafka/pull/18096#discussion_r1874838353 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/ShareGroupAutoOffsetResetStrategy.java: ## @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Soft

[jira] [Resolved] (KAFKA-12843) KIP-740 follow up: clean up TaskMetadata

2024-12-08 Thread Jira
[ https://issues.apache.org/jira/browse/KAFKA-12843?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] João Pedro Fonseca resolved KAFKA-12843. Resolution: Fixed > KIP-740 follow up: clean up TaskMetadata > ---

Re: [PR] KAFKA-18156: VerifiableConsumer should ignore "--session-timeout" when using CONSUMER protocol [kafka]

2024-12-08 Thread via GitHub
chia7712 commented on code in PR #18036: URL: https://github.com/apache/kafka/pull/18036#discussion_r1874846202 ## tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java: ## @@ -649,8 +648,12 @@ public static VerifiableConsumer createFromArgs(ArgumentParser parser,

Re: [PR] KAFKA-18164: Clear existing acknowledgements on share session epoch reset. [kafka]

2024-12-08 Thread via GitHub
ShivsundarR commented on code in PR #18063: URL: https://github.com/apache/kafka/pull/18063#discussion_r1874846243 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManager.java: ## @@ -990,16 +990,17 @@ UnsentRequest buildRequest() {

Re: [PR] KAFKA-18156: VerifiableConsumer should ignore "--session-timeout" when using CONSUMER protocol [kafka]

2024-12-08 Thread via GitHub
chia7712 commented on code in PR #18036: URL: https://github.com/apache/kafka/pull/18036#discussion_r1874952912 ## tests/kafkatest/tests/verifiable_consumer_test.py: ## @@ -56,7 +55,7 @@ def min_cluster_size(self): def setup_consumer(self, topic, static_membership=False, e

[PR] KAFKA-18180: Move OffsetResultHolder to storage module [kafka]

2024-12-08 Thread via GitHub
m1a2st opened a new pull request, #18100: URL: https://github.com/apache/kafka/pull/18100 Jira: https://issues.apache.org/jira/browse/KAFKA-18180 ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ] Verify test coverage and CI build

Re: [PR] KAFKA-18180: Move OffsetResultHolder to storage module [kafka]

2024-12-08 Thread via GitHub
m1a2st commented on code in PR #18100: URL: https://github.com/apache/kafka/pull/18100#discussion_r1874955519 ## storage/src/main/java/org/apache/kafka/storage/log/OffsetResultHolder.java: ## @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or

Re: [PR] KAFKA-10731: add support for SSL hot reload [kafka]

2024-12-08 Thread via GitHub
CefBoud commented on PR #17987: URL: https://github.com/apache/kafka/pull/17987#issuecomment-2526238613 @TaiJuWu I've drafted a [KIP](https://cwiki.apache.org/confluence/x/eIrREw). If you have any feedback, I’d really appreciate it! -- This is an automated message from the Apache Git Ser