Re: [PR] KAFKA-18024 ConcurrentModificationException in Kafka OffsetFetcher - Proposal for Thread-Safety Fix - Update OffsetFetcher.java [kafka]

2024-11-29 Thread via GitHub
kamil-adam-nowak commented on PR #17826: URL: https://github.com/apache/kafka/pull/17826#issuecomment-2507288877 @mimaison @frankvicky @FrankYang0529 @gongxuanzhang @chia7712 Can you please take a second look? -- This is an automated message from the Apache Git Service. To respond to the

Re: [PR] KAFKA-17750: Extend kafka-consumer-groups command line tool to support new consumer group (part 1) [kafka]

2024-11-29 Thread via GitHub
FrankYang0529 commented on PR #17958: URL: https://github.com/apache/kafka/pull/17958#issuecomment-2507327752 > Could you please extend tests in `ConsumerGroupDescribeRequestTest` to cover the new version of the API? Sorry, I missed this comment yesterday. I update the PR and CI resul

[jira] [Commented] (KAFKA-18119) Service loading loads incorrect plugin version.

2024-11-29 Thread Snehashis Pal (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18119?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901880#comment-17901880 ] Snehashis Pal commented on KAFKA-18119: --- With some more investigation, I think thi

Re: [PR] KAFKA-18084: Added write locks in SharePartition where locks were async calls were being made [kafka]

2024-11-29 Thread via GitHub
AndrewJSchofield commented on code in PR #17957: URL: https://github.com/apache/kafka/pull/17957#discussion_r1863193325 ## core/src/main/java/kafka/server/share/SharePartition.java: ## @@ -389,79 +389,84 @@ public CompletableFuture maybeInitialize() { .build

[jira] [Commented] (KAFKA-18119) Service loading loads incorrect plugin version.

2024-11-29 Thread Snehashis Pal (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18119?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901887#comment-17901887 ] Snehashis Pal commented on KAFKA-18119: --- Hi [~gharris1727], please take a look at

Re: [PR] KAFKA-10790: Add deadlock detection to producer#flush [kafka]

2024-11-29 Thread via GitHub
AndrewJSchofield commented on PR #17946: URL: https://github.com/apache/kafka/pull/17946#issuecomment-2507389932 The code looks good to me, but I don't think we are ready to merge yet. I've been thinking about the mechanics of delivering this into AK, and this is definitely a change i

[jira] [Created] (KAFKA-18120) KIP-891: Support for multiple versions of connect plugins.

2024-11-29 Thread Snehashis Pal (Jira)
Snehashis Pal created KAFKA-18120: - Summary: KIP-891: Support for multiple versions of connect plugins. Key: KAFKA-18120 URL: https://issues.apache.org/jira/browse/KAFKA-18120 Project: Kafka

[jira] [Commented] (KAFKA-18120) KIP-891: Support for multiple versions of connect plugins.

2024-11-29 Thread Snehashis Pal (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18120?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901893#comment-17901893 ] Snehashis Pal commented on KAFKA-18120: --- PRs [Pull requests · apache/kafka|https:

[jira] [Assigned] (KAFKA-18120) KIP-891: Support for multiple versions of connect plugins.

2024-11-29 Thread Snehashis Pal (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18120?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Snehashis Pal reassigned KAFKA-18120: - Assignee: Snehashis Pal > KIP-891: Support for multiple versions of connect plugins. >

[jira] [Created] (KAFKA-18121) Add support for duration-based offset reset strategy

2024-11-29 Thread Andrew Schofield (Jira)
Andrew Schofield created KAFKA-18121: Summary: Add support for duration-based offset reset strategy Key: KAFKA-18121 URL: https://issues.apache.org/jira/browse/KAFKA-18121 Project: Kafka

[PR] KAFKA-17544: Fix for loading big files while performing load tests [kafka]

2024-11-29 Thread via GitHub
manoj-mathivanan opened a new pull request, #17983: URL: https://github.com/apache/kafka/pull/17983 When performing perf tests, we can specify a payload using the --payloadFile parameter. This file is utilized during the load/performance testing process. However, if the file is large, it ma

Re: [PR] KAFKA-10790: Add deadlock detection to producer#flush [kafka]

2024-11-29 Thread via GitHub
chia7712 commented on PR #17946: URL: https://github.com/apache/kafka/pull/17946#issuecomment-2507407403 > but if someone has an application which is doing this, their code will break following this change. @frankvicky, do you agree? Can you create a KIP to make sure we have community alig

Re: [PR] KAFKA-10790: Add deadlock detection to producer#flush [kafka]

2024-11-29 Thread via GitHub
frankvicky commented on PR #17946: URL: https://github.com/apache/kafka/pull/17946#issuecomment-2507408503 Hello @AndrewJSchofield I think you are right, as Kafka usually consider Exception as a part of Public API, this changes will need a KIP. -- This is an automated message from

[jira] [Comment Edited] (KAFKA-18115) Issue loading big files for performance testing

2024-11-29 Thread Manoj Mathivanan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18115?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901894#comment-17901894 ] Manoj Mathivanan edited comment on KAFKA-18115 at 11/29/24 9:30 AM: --

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

2024-11-29 Thread via GitHub
m1a2st commented on PR #17440: URL: https://github.com/apache/kafka/pull/17440#issuecomment-2507420510 I think I have an idea of where the problem lies. With the current approach, the `completeExceptionally` inside `updateFetchPositions` cannot successfully propagate to the application thre

[jira] [Created] (KAFKA-18122) Add ShareConsumeBenchWorker for running trogdor workloads.

2024-11-29 Thread Shivsundar R (Jira)
Shivsundar R created KAFKA-18122: Summary: Add ShareConsumeBenchWorker for running trogdor workloads. Key: KAFKA-18122 URL: https://issues.apache.org/jira/browse/KAFKA-18122 Project: Kafka Is

[jira] [Created] (KAFKA-18123) Fix flaky DynamicBrokerReconfigurationTest#testThreadPoolResize

2024-11-29 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-18123: -- Summary: Fix flaky DynamicBrokerReconfigurationTest#testThreadPoolResize Key: KAFKA-18123 URL: https://issues.apache.org/jira/browse/KAFKA-18123 Project: Kafka

Re: [PR] MINOR: Convert DynamicBrokerReconfigurationTest to KRaft [kafka]

2024-11-29 Thread via GitHub
chia7712 commented on code in PR #17905: URL: https://github.com/apache/kafka/pull/17905#discussion_r1863235905 ## core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala: ## @@ -773,9 +761,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTe

[jira] [Commented] (KAFKA-18115) Issue loading big files for performance testing

2024-11-29 Thread Manoj Mathivanan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18115?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901894#comment-17901894 ] Manoj Mathivanan commented on KAFKA-18115: -- Fixed in PR: https://github.com/apa

[jira] [Commented] (KAFKA-18123) Fix flaky DynamicBrokerReconfigurationTest#testThreadPoolResize

2024-11-29 Thread PoAn Yang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901895#comment-17901895 ] PoAn Yang commented on KAFKA-18123: --- Hi [~chia7712], if you're not working on this, ma

[jira] [Assigned] (KAFKA-18123) Fix flaky DynamicBrokerReconfigurationTest#testThreadPoolResize

2024-11-29 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai reassigned KAFKA-18123: -- Assignee: PoAn Yang (was: Chia-Ping Tsai) > Fix flaky DynamicBrokerReconfigurationTe

[jira] [Assigned] (KAFKA-18122) Add ShareConsumeBenchWorker for running trogdor workloads.

2024-11-29 Thread Shivsundar R (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18122?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shivsundar R reassigned KAFKA-18122: Assignee: Shivsundar R > Add ShareConsumeBenchWorker for running trogdor workloads. > ---

Re: [PR] KIP-1071: GroupStore [kafka]

2024-11-29 Thread via GitHub
lucasbru commented on PR #17981: URL: https://github.com/apache/kafka/pull/17981#issuecomment-2507463702 Hi @aliehsaeedii. Thanks for the PR. A few high-level comments for now: - Could we already make use of `GroupStore`, i.e. remove the corresponding functionality from `GroupMeta

Re: [PR] [DRAFT] KAFKA 16720 : AdminClient support for ListShareGroupOffsets [kafka]

2024-11-29 Thread via GitHub
AndrewJSchofield commented on code in PR #17775: URL: https://github.com/apache/kafka/pull/17775#discussion_r1863265749 ## clients/src/main/java/org/apache/kafka/clients/admin/ForwardingAdmin.java: ## @@ -304,6 +304,12 @@ public DescribeShareGroupsResult describeShareGroups(Col

Re: [PR] MINOR: disable some rebootstrap tests, convert the others to KRaft [kafka]

2024-11-29 Thread via GitHub
chia7712 commented on PR #17765: URL: https://github.com/apache/kafka/pull/17765#issuecomment-2507473444 @cmccabe Could you please fix 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

[jira] [Updated] (KAFKA-18124) Remove zk migration from RaftManagerTest

2024-11-29 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18124?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai updated KAFKA-18124: --- Summary: Remove zk migration from RaftManagerTest (was: Remove zk migration from tests) >

[jira] [Updated] (KAFKA-18124) Remove zk migration from RaftManagerTest, BrokerLifecycleManagerTest

2024-11-29 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18124?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai updated KAFKA-18124: --- Summary: Remove zk migration from RaftManagerTest, BrokerLifecycleManagerTest (was: Remove

[jira] [Created] (KAFKA-18124) Remove zk migration from tests

2024-11-29 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-18124: -- Summary: Remove zk migration from tests Key: KAFKA-18124 URL: https://issues.apache.org/jira/browse/KAFKA-18124 Project: Kafka Issue Type: Sub-task

[jira] [Updated] (KAFKA-18124) Remove zk migration from RaftManagerTest, BrokerLifecycleManagerTest, KafkaConfigTest, and ReplicaManagerTest

2024-11-29 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18124?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai updated KAFKA-18124: --- Summary: Remove zk migration from RaftManagerTest, BrokerLifecycleManagerTest, KafkaConfigTe

[jira] [Updated] (KAFKA-18124) Remove zk migration from RaftManagerTest, BrokerLifecycleManagerTest

2024-11-29 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18124?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai updated KAFKA-18124: --- Description: * RaftManagerTest, BrokerLifecycleManagerTest.scala, KafkaConfigTest.scala, and

[jira] [Updated] (KAFKA-18124) Remove zk migration from RaftManagerTest, BrokerLifecycleManagerTest, KafkaConfigTest

2024-11-29 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18124?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai updated KAFKA-18124: --- Summary: Remove zk migration from RaftManagerTest, BrokerLifecycleManagerTest, KafkaConfigTe

[jira] [Assigned] (KAFKA-18124) Remove zk migration from RaftManagerTest, BrokerLifecycleManagerTest, KafkaConfigTest, and ReplicaManagerTest

2024-11-29 Thread Chia-Chuan Yu (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18124?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Chuan Yu reassigned KAFKA-18124: - Assignee: Chia-Chuan Yu (was: Chia-Ping Tsai) > Remove zk migration from RaftManagerTe

Re: [PR] Topology epochs and topic validation [kafka]

2024-11-29 Thread via GitHub
cadonna commented on code in PR #17721: URL: https://github.com/apache/kafka/pull/17721#discussion_r1863242293 ## clients/src/main/resources/common/message/StreamsGroupHeartbeatResponse.json: ## @@ -30,11 +30,11 @@ // - FENCED_MEMBER_EPOCH (version 0+) // - UNRELEASED_INST

[jira] [Created] (KAFKA-18125) Implement MirrorCheckpointConnector max task limit

2024-11-29 Thread Jarkko Jaakola (Jira)
Jarkko Jaakola created KAFKA-18125: -- Summary: Implement MirrorCheckpointConnector max task limit Key: KAFKA-18125 URL: https://issues.apache.org/jira/browse/KAFKA-18125 Project: Kafka Issue

[jira] [Commented] (KAFKA-18125) Implement MirrorCheckpointConnector max task limit

2024-11-29 Thread Jarkko Jaakola (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18125?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901900#comment-17901900 ] Jarkko Jaakola commented on KAFKA-18125: Pull request available: https://github.

Re: [PR] Topology epochs and topic validation [kafka]

2024-11-29 Thread via GitHub
lucasbru commented on code in PR #17721: URL: https://github.com/apache/kafka/pull/17721#discussion_r1863351962 ## streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java: ## @@ -73,32 +77,39 @@ @Timeout(600) @Tag("integra

[jira] [Commented] (KAFKA-18125) Implement MirrorCheckpointConnector max task limit

2024-11-29 Thread Jarkko Jaakola (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18125?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901904#comment-17901904 ] Jarkko Jaakola commented on KAFKA-18125: The PR can be backported to previous Ka

Re: [PR] Topology epochs and topic validation [kafka]

2024-11-29 Thread via GitHub
lucasbru merged PR #17721: URL: https://github.com/apache/kafka/pull/17721 -- 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-18086: Enable propagation of the error message when writing state [kafka]

2024-11-29 Thread via GitHub
Yunyung commented on code in PR #17980: URL: https://github.com/apache/kafka/pull/17980#discussion_r1863364362 ## core/src/main/java/kafka/server/share/SharePartitionManager.java: ## @@ -369,11 +383,23 @@ public CompletableFuture allFutures = CompletableFuture.allOf(

Re: [PR] KAFKA-18086: Enable propagation of the error message when writing state [kafka]

2024-11-29 Thread via GitHub
Yunyung commented on code in PR #17980: URL: https://github.com/apache/kafka/pull/17980#discussion_r1863365178 ## core/src/main/java/kafka/server/share/SharePartitionManager.java: ## @@ -300,11 +301,24 @@ public CompletableFuture allFutures = CompletableFuture.allOf(

Re: [PR] KAFKA-18086: Enable propagation of the error message when writing state [kafka]

2024-11-29 Thread via GitHub
Yunyung commented on code in PR #17980: URL: https://github.com/apache/kafka/pull/17980#discussion_r1863365600 ## core/src/main/java/kafka/server/share/SharePartitionManager.java: ## @@ -369,11 +383,23 @@ public CompletableFuture allFutures = CompletableFuture.allOf(

Re: [PR] KAFKA-18086: Enable propagation of the error message when writing state [kafka]

2024-11-29 Thread via GitHub
Yunyung commented on PR #17980: URL: https://github.com/apache/kafka/pull/17980#issuecomment-2507601231 These are great points we can improve on. Thanks, Andrew :) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the UR

Re: [PR] MINOR: Refactor configs in GroupMetadataManager [kafka]

2024-11-29 Thread via GitHub
chia7712 commented on code in PR #17982: URL: https://github.com/apache/kafka/pull/17982#discussion_r1863285991 ## server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java: ## @@ -43,31 +43,32 @@ */ public abstract class AbstractKafkaConfig extends Abstrac

Re: [PR] KAFKA-18084: Added write locks in SharePartition where locks were async calls were being made [kafka]

2024-11-29 Thread via GitHub
apoorvmittal10 commented on code in PR #17957: URL: https://github.com/apache/kafka/pull/17957#discussion_r1863416832 ## core/src/main/java/kafka/server/share/SharePartition.java: ## @@ -389,79 +389,84 @@ public CompletableFuture maybeInitialize() { .build()

Re: [PR] KAFKA-18086: Enable propagation of the error message when writing state [kafka]

2024-11-29 Thread via GitHub
apoorvmittal10 commented on code in PR #17980: URL: https://github.com/apache/kafka/pull/17980#discussion_r1863465386 ## core/src/main/java/kafka/server/share/SharePartitionManager.java: ## @@ -300,11 +301,25 @@ public CompletableFuture allFutures = CompletableFuture.allOf(

Re: [PR] KAFKA-17715 remove force_use_zk_connection from e2e [kafka]

2024-11-29 Thread via GitHub
mingdaoy commented on PR #17465: URL: https://github.com/apache/kafka/pull/17465#issuecomment-2507752032 PTAL @mimaison @chia7712 I limit the comments cleanup to `force_use_zk_connection` code block because other comments area still have **zk** or **zookeeper** related code to be removed l

Re: [PR] KAFKA-9366: Upgrade log4j to log4j2 [kafka]

2024-11-29 Thread via GitHub
frankvicky commented on PR #17373: URL: https://github.com/apache/kafka/pull/17373#issuecomment-2507753807 Since this issue is quite tricky for me, I went through a process of elimination to locate the bug. I found this line might be the root cause: https://github.com/apache/kafka

Re: [PR] KAFKA-17750: Extend kafka-consumer-groups command line tool to support new consumer group (part 1) [kafka]

2024-11-29 Thread via GitHub
dajac commented on code in PR #17958: URL: https://github.com/apache/kafka/pull/17958#discussion_r1863479048 ## core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala: ## @@ -213,4 +216,111 @@ class ConsumerGroupDescribeRequestTest(cluster: ClusterInstance

Re: [PR] KAFKA-18084: Added usage for rollback state while SharePartition acquires records [kafka]

2024-11-29 Thread via GitHub
apoorvmittal10 commented on PR #17965: URL: https://github.com/apache/kafka/pull/17965#issuecomment-2507770748 @adixitconfluent Thanks for the PR, but for my understanding when can this scenario happen? We take a lock in the `acquire` which means a single thread/client can have access. And

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

2024-11-29 Thread via GitHub
m1a2st commented on PR #17440: URL: https://github.com/apache/kafka/pull/17440#issuecomment-2507785075 Hello @apoorvmittal10, @AndrewJSchofield, @lianetm, @kirktrue The current PR has changed the way metadata errors are propagated, switching from `backgroundEventHandler.add(new Error

Re: [PR] KAFKA-18060: new coordinator does not handle TxnOffsetCommitRequest with empty member id when using CONSUMER group [kafka]

2024-11-29 Thread via GitHub
dajac commented on code in PR #17914: URL: https://github.com/apache/kafka/pull/17914#discussion_r1863485107 ## core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala: ## @@ -194,6 +209,104 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance)

Re: [PR] KAFKA-18086: Enable propagation of the error message when writing state [kafka]

2024-11-29 Thread via GitHub
apoorvmittal10 commented on code in PR #17980: URL: https://github.com/apache/kafka/pull/17980#discussion_r1863507302 ## core/src/main/java/kafka/server/share/SharePartitionManager.java: ## @@ -300,11 +301,25 @@ public CompletableFuture allFutures = CompletableFuture.allOf(

Re: [PR] KAFKA-18086: Enable propagation of the error message when writing state [kafka]

2024-11-29 Thread via GitHub
Yunyung commented on code in PR #17980: URL: https://github.com/apache/kafka/pull/17980#discussion_r1863520904 ## core/src/main/java/kafka/server/share/SharePartitionManager.java: ## @@ -300,11 +301,25 @@ public CompletableFuture allFutures = CompletableFuture.allOf(

Re: [PR] KAFKA-18086: Enable propagation of the error message when writing state [kafka]

2024-11-29 Thread via GitHub
Yunyung commented on code in PR #17980: URL: https://github.com/apache/kafka/pull/17980#discussion_r1863520904 ## core/src/main/java/kafka/server/share/SharePartitionManager.java: ## @@ -300,11 +301,25 @@ public CompletableFuture allFutures = CompletableFuture.allOf(

Re: [PR] MINOR: Refactor configs in GroupMetadataManager [kafka]

2024-11-29 Thread via GitHub
dajac commented on code in PR #17982: URL: https://github.com/apache/kafka/pull/17982#discussion_r1863528681 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java: ## @@ -367,6 +383,26 @@ public GroupCoordinatorConfig(AbstractConfig co

Re: [PR] MINOR: Refactor configs in GroupMetadataManager [kafka]

2024-11-29 Thread via GitHub
dajac commented on code in PR #17982: URL: https://github.com/apache/kafka/pull/17982#discussion_r1863529874 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java: ## @@ -1905,7 +1718,7 @@ private CoordinatorResult metr

Re: [PR] MINOR: Refactor configs in GroupMetadataManager [kafka]

2024-11-29 Thread via GitHub
dajac commented on code in PR #17982: URL: https://github.com/apache/kafka/pull/17982#discussion_r1863547223 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java: ## @@ -367,6 +383,26 @@ public GroupCoordinatorConfig(AbstractConfig co

Re: [PR] KAFKA-17715 remove force_use_zk_connection from e2e [kafka]

2024-11-29 Thread via GitHub
mimaison commented on PR #17465: URL: https://github.com/apache/kafka/pull/17465#issuecomment-2507850803 Thanks for the updates. Have you been able to run the updated system tests and validate they still pass? -- This is an automated message from the Apache Git Service. To respond to the

Re: [PR] KAFKA-9366: Upgrade log4j to log4j2 [kafka]

2024-11-29 Thread via GitHub
mimaison commented on PR #17373: URL: https://github.com/apache/kafka/pull/17373#issuecomment-2507855608 Thanks @frankvicky for investigating. I ran the system tests in our CI and confirm most the Connect tests are currently failing with your branch (I've not tried running other tests yet).

[PR] KAFKA-18025: Rework acquisition lock timeout test [kafka]

2024-11-29 Thread via GitHub
AndrewJSchofield opened a new pull request, #17985: URL: https://github.com/apache/kafka/pull/17985 Additional work on `ShareConsumerTest.testAcquisitionLockTimeoutOnConsumer`. First, mark the test as flaky since it fails occasionally and it would be best to get a decent number of passes be

[jira] [Created] (KAFKA-18126) Refactoring to split the GroupMetadataManager in AK

2024-11-29 Thread Alieh Saeedi (Jira)
Alieh Saeedi created KAFKA-18126: Summary: Refactoring to split the GroupMetadataManager in AK Key: KAFKA-18126 URL: https://issues.apache.org/jira/browse/KAFKA-18126 Project: Kafka Issue Typ

Re: [PR] KAFKA-1826 [1/N]: Introducing GroupStore [kafka]

2024-11-29 Thread via GitHub
aliehsaeedii commented on code in PR #17981: URL: https://github.com/apache/kafka/pull/17981#discussion_r1863574816 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupStore.java: ## @@ -0,0 +1,307 @@ +/* + * Licensed to the Apache Software Foundation (ASF

Re: [PR] KAFKA-15561 [5/N]: Integration tests for new subscribe API with Re2J pattern [kafka]

2024-11-29 Thread via GitHub
lianetm commented on code in PR #17964: URL: https://github.com/apache/kafka/pull/17964#discussion_r1863578362 ## core/src/test/scala/integration/kafka/api/PlaintextConsumerSubscriptionTest.scala: ## @@ -178,6 +178,49 @@ class PlaintextConsumerSubscriptionTest extends AbstractC

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

2024-11-29 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18014?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar updated KAFKA-18014: -- Fix Version/s: 4.1.0 (was: 4.0.0) > Add duration based offset reset option for

Re: [PR] KAFKA-17338 ConsumerConfig should prevent using partition assignors with CONSUMER group protocol [kafka]

2024-11-29 Thread via GitHub
lianetm merged PR #16899: URL: https://github.com/apache/kafka/pull/16899 -- 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-15561 [5/N]: Integration tests for new subscribe API with Re2J pattern [kafka]

2024-11-29 Thread via GitHub
dajac commented on code in PR #17964: URL: https://github.com/apache/kafka/pull/17964#discussion_r1863687478 ## core/src/test/scala/integration/kafka/api/PlaintextConsumerSubscriptionTest.scala: ## @@ -178,6 +178,49 @@ class PlaintextConsumerSubscriptionTest extends AbstractCon

Re: [PR] KIP 1071- Refactor GroupMetadataManager [kafka]

2024-11-29 Thread via GitHub
aliehsaeedii commented on PR #17941: URL: https://github.com/apache/kafka/pull/17941#issuecomment-2508033828 Thanks, @AndrewJSchofield. I should've closed this PR, since we decided to split it into multiple ones. This: https://github.com/apache/kafka/pull/17981 is the 1st one in the row. BT

Re: [PR] KIP 1071- Refactor GroupMetadataManager [kafka]

2024-11-29 Thread via GitHub
aliehsaeedii closed pull request #17941: KIP 1071- Refactor GroupMetadataManager URL: https://github.com/apache/kafka/pull/17941 -- 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.

Re: [PR] KAFKA-17750: Extend kafka-consumer-groups command line tool to support new consumer group (part 1) [kafka]

2024-11-29 Thread via GitHub
FrankYang0529 commented on code in PR #17958: URL: https://github.com/apache/kafka/pull/17958#discussion_r1863702678 ## core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala: ## @@ -213,4 +216,111 @@ class ConsumerGroupDescribeRequestTest(cluster: Cluster

Re: [PR] KAFKA-17750: Extend kafka-consumer-groups command line tool to support new consumer group (part 1) [kafka]

2024-11-29 Thread via GitHub
FrankYang0529 commented on code in PR #17958: URL: https://github.com/apache/kafka/pull/17958#discussion_r1863713049 ## core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala: ## @@ -213,4 +216,111 @@ class ConsumerGroupDescribeRequestTest(cluster: Cluster

Re: [PR] KAFKA-17750: Extend kafka-consumer-groups command line tool to support new consumer group (part 1) [kafka]

2024-11-29 Thread via GitHub
FrankYang0529 commented on code in PR #17958: URL: https://github.com/apache/kafka/pull/17958#discussion_r1863713283 ## core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala: ## @@ -213,4 +216,111 @@ class ConsumerGroupDescribeRequestTest(cluster: Cluster

Re: [PR] MINOR: Replaced Utils.join() with JDK API. [kafka]

2024-11-29 Thread via GitHub
marcelstoer commented on PR #15823: URL: https://github.com/apache/kafka/pull/15823#issuecomment-2508069034 IMO it's really bad practice to alter public methods in a minor release. Why would you not want to follow SemVer? -- This is an automated message from the Apache Git Service. To res

Re: [PR] KAFKA-18086: Enable propagation of the error message when writing state [kafka]

2024-11-29 Thread via GitHub
Yunyung commented on code in PR #17980: URL: https://github.com/apache/kafka/pull/17980#discussion_r1863744110 ## core/src/main/java/kafka/server/share/SharePartitionManager.java: ## @@ -300,11 +301,25 @@ public CompletableFuture allFutures = CompletableFuture.allOf(

Re: [PR] KAFKA-9366: Upgrade log4j to log4j2 [kafka]

2024-11-29 Thread via GitHub
chia7712 commented on PR #17373: URL: https://github.com/apache/kafka/pull/17373#issuecomment-2508123859 @frankvicky In the end-to-end tests, you must use the correct file extension when passing the log4j2 YAML configuration. If you use an incorrect extension - for example - the output of `

Re: [PR] KAFKA-18086: Enable propagation of the error message when writing state [kafka]

2024-11-29 Thread via GitHub
apoorvmittal10 commented on code in PR #17980: URL: https://github.com/apache/kafka/pull/17980#discussion_r1863751566 ## core/src/main/java/kafka/server/share/SharePartitionManager.java: ## @@ -294,17 +294,27 @@ public CompletableFuture allFutures = CompletableFuture.allOf(

[jira] [Updated] (KAFKA-18048) Consider to decouple AppEventHandler from ConsumerNetworkThread

2024-11-29 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18048?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans updated KAFKA-18048: --- Component/s: consumer > Consider to decouple AppEventHandler from ConsumerNetworkThread > --

[jira] [Updated] (KAFKA-18048) Consider to decouple AppEventHandler from ConsumerNetworkThread

2024-11-29 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18048?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans updated KAFKA-18048: --- Labels: consumer-threading-refactor (was: ) > Consider to decouple AppEventHandler from Con

Re: [PR] MINOR: Replaced Utils.join() with JDK API. [kafka]

2024-11-29 Thread via GitHub
chia7712 commented on PR #15823: URL: https://github.com/apache/kafka/pull/15823#issuecomment-2508133588 > IMO it's really bad practice to alter public methods in a minor release. Why would you not want to follow SemVer? Pardon me, could you clarify the details about the broken public

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

2024-11-29 Thread via GitHub
omkreddy commented on code in PR #17972: URL: https://github.com/apache/kafka/pull/17972#discussion_r1863012807 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoOffsetResetStrategy.java: ## @@ -39,29 +43,67 @@ public String toString() { public static

Re: [PR] MINOR: Replaced Utils.join() with JDK API. [kafka]

2024-11-29 Thread via GitHub
chia7712 commented on PR #15823: URL: https://github.com/apache/kafka/pull/15823#issuecomment-2508143972 As referenced in [this line](https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/utils/package-info.java#L19) > This package is not a supported

Re: [PR] KAFKA-15561 [5/N]: Integration tests for new subscribe API with Re2J pattern [kafka]

2024-11-29 Thread via GitHub
chia7712 merged PR #17964: URL: https://github.com/apache/kafka/pull/17964 -- 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] MINOR: Replaced Utils.join() with JDK API. [kafka]

2024-11-29 Thread via GitHub
marcelstoer commented on PR #15823: URL: https://github.com/apache/kafka/pull/15823#issuecomment-2508168927 Thanks for your feedback. When you stated in your first comment that the `utils` package be non-public I went looking for any sort of documented confirmation. That's when I foun

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

2024-11-29 Thread via GitHub
omkreddy merged PR #17972: URL: https://github.com/apache/kafka/pull/17972 -- 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

[PR] KAFKA-18123: Fix flaky DynamicBrokerReconfigurationTest#testThreadPoolResize (wip) [kafka]

2024-11-29 Thread via GitHub
FrankYang0529 opened a new pull request, #17986: URL: https://github.com/apache/kafka/pull/17986 ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ] Verify test coverage and CI build status - [ ] Verify documentation (including

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

2024-11-29 Thread Peter Lee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18014?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901965#comment-17901965 ] Peter Lee commented on KAFKA-18014: --- Hi [~omkreddy], may I take this issue? Thanks! >

Re: [PR] KAFKA-17534: Add configuration to disable the heartbeats topic replic… [kafka]

2024-11-29 Thread via GitHub
pedro-te commented on PR #17413: URL: https://github.com/apache/kafka/pull/17413#issuecomment-2508208968 @urbandan I can't thank you enough for this!! This will be super helpful! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub

Re: [PR] KAFKA-18086: Enable propagation of the error message when writing state [kafka]

2024-11-29 Thread via GitHub
Yunyung commented on code in PR #17980: URL: https://github.com/apache/kafka/pull/17980#discussion_r1863826675 ## core/src/main/java/kafka/server/share/SharePartitionManager.java: ## @@ -294,17 +294,27 @@ public CompletableFuture allFutures = CompletableFuture.allOf(

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

2024-11-29 Thread via GitHub
CefBoud opened a new pull request, #17987: URL: https://github.com/apache/kafka/pull/17987 # Introduction This PR proposes a change to add support for SSL hot reloading. # Motivation SSL certificates are typically short-lived, and while Kafka brokers support dynamic SSL certifi

Re: [PR] KAFKA-15737: KRaft Support in ConsumerBounceTest [kafka]

2024-11-29 Thread via GitHub
chia7712 commented on PR #17770: URL: https://github.com/apache/kafka/pull/17770#issuecomment-250846 > testConsumptionWithBrokerFailures any update for this test case? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub

Re: [PR] KAFKA-17834 Improvement the e2e Dockerfile [kafka]

2024-11-29 Thread via GitHub
chia7712 commented on PR #17554: URL: https://github.com/apache/kafka/pull/17554#issuecomment-2508335681 @m1a2st Could you please rebase code? -- 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

[jira] [Assigned] (KAFKA-10731) have kafka producer & consumer auto-reload ssl certificate

2024-11-29 Thread Moncef Abboud (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10731?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Moncef Abboud reassigned KAFKA-10731: - Assignee: Moncef Abboud > have kafka producer & consumer auto-reload ssl certificate

[jira] [Commented] (KAFKA-10731) have kafka producer & consumer auto-reload ssl certificate

2024-11-29 Thread Moncef Abboud (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10731?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901974#comment-17901974 ] Moncef Abboud commented on KAFKA-10731: --- I initiated a [PR|https://github.com/apac

Re: [PR] KAFKA-17554: Flaky testFutureCompletionOutsidePoll in ConsumerNetworkClientTest [kafka]

2024-11-29 Thread via GitHub
chia7712 commented on PR #17217: URL: https://github.com/apache/kafka/pull/17217#issuecomment-2508387154 @m1a2st could you please rebase code? -- 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-16617: Add KRaft info for the `advertised.listeners` doc description [kafka]

2024-11-29 Thread via GitHub
chia7712 merged PR #17552: URL: https://github.com/apache/kafka/pull/17552 -- 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

[jira] [Resolved] (KAFKA-16617) Add KRaft info for the `advertised.listeners` doc description

2024-11-29 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16617?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-16617. Fix Version/s: 4.0.0 Resolution: Fixed > Add KRaft info for the `advertised.listene

Re: [PR] KAFKA-1826 [1/N]: Introducing GroupStore [kafka]

2024-11-29 Thread via GitHub
dajac commented on PR #17981: URL: https://github.com/apache/kafka/pull/17981#issuecomment-2508422020 Thanks @aliehsaeedii! I will definitely review it next week. Would it be possible to describe the high level design that you're aiming for? I would like to ensure that we are on the same pa

Re: [PR] KAFKA-15561 [5/N]: Integration tests for new subscribe API with Re2J pattern [kafka]

2024-11-29 Thread via GitHub
chia7712 commented on code in PR #17964: URL: https://github.com/apache/kafka/pull/17964#discussion_r1863102966 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMetadata.java: ## @@ -68,7 +68,7 @@ public boolean allowAutoTopicCreation() { @Over

[jira] [Created] (KAFKA-18127) Handle subscription pattern used with v0 HB broker

2024-11-29 Thread Lianet Magrans (Jira)
Lianet Magrans created KAFKA-18127: -- Summary: Handle subscription pattern used with v0 HB broker Key: KAFKA-18127 URL: https://issues.apache.org/jira/browse/KAFKA-18127 Project: Kafka Issue

Re: [PR] [KAFKA-8830] KIP-512: make Record Headers available in onAcknowledgement [kafka]

2024-11-29 Thread via GitHub
chia7712 commented on code in PR #17099: URL: https://github.com/apache/kafka/pull/17099#discussion_r1863883205 ## clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java: ## @@ -81,12 +82,40 @@ public interface ProducerInterceptor extends Configurable,

Re: [PR] KAFKA-17783: Adding listeners to remove share partition on partition changes [kafka]

2024-11-29 Thread via GitHub
chia7712 commented on code in PR #17796: URL: https://github.com/apache/kafka/pull/17796#discussion_r1863901810 ## core/src/main/java/kafka/server/share/SharePartitionManager.java: ## @@ -609,6 +610,12 @@ private SharePartition getOrCreateSharePartition(SharePartitionKey shareP

  1   2   >