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

2024-04-30 Thread via GitHub
chia7712 commented on code in PR #15823: URL: https://github.com/apache/kafka/pull/15823#discussion_r1585914471 ## clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java: ## @@ -25,21 +25,21 @@ import

Re: [PR] KAFKA-16649: Remove lock from DynamicBrokerConfig.removeReconfigurable [kafka]

2024-04-30 Thread via GitHub
chia7712 commented on code in PR #15838: URL: https://github.com/apache/kafka/pull/15838#discussion_r1585911858 ## core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala: ## @@ -1541,6 +1541,36 @@ class KRaftClusterTest { cluster.close() } } + +

Re: [PR] KAFKA-16394: Fix null propagation in foreign key join result [kafka]

2024-04-30 Thread via GitHub
mjsax commented on PR #15607: URL: https://github.com/apache/kafka/pull/15607#issuecomment-2088000194 Can we update existing test to not use `Map` but `List` (as I mentioned on the duplicate ticket https://issues.apache.org/jira/browse/KAFKA-16644) which should expose the bug? Need

[jira] [Resolved] (KAFKA-16644) FK join emits duplicate tombstone on left-side delete

2024-04-30 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16644?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-16644. - Resolution: Duplicate > FK join emits duplicate tombstone on left-side delete >

Re: [PR] KIP-759 Mark as Partitioned [kafka]

2024-04-30 Thread via GitHub
mjsax commented on code in PR #15740: URL: https://github.com/apache/kafka/pull/15740#discussion_r1585886754 ## streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java: ## @@ -685,6 +685,41 @@ KStream flatMapValues(final ValueMapper KStream flatMapValues(final

[jira] [Assigned] (KAFKA-16650) add integration test for Admin#abortTransaction

2024-04-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16650?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai reassigned KAFKA-16650: -- Assignee: Kuan Po Tseng (was: Chia-Ping Tsai) > add integration test for

[jira] [Commented] (KAFKA-16650) add integration test for Admin#abortTransaction

2024-04-30 Thread Kuan Po Tseng (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16650?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842542#comment-17842542 ] Kuan Po Tseng commented on KAFKA-16650: --- May I take over this issue ? :) > add integration test

Re: [PR] KAFKA-16608: Honour interrupted thread state on KafkaConsumer.poll [kafka]

2024-04-30 Thread via GitHub
AndrewJSchofield commented on code in PR #15803: URL: https://github.com/apache/kafka/pull/15803#discussion_r1585852808 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java: ## @@ -1789,6 +1791,33 @@ public void

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

2024-04-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16223?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842538#comment-17842538 ] Chia-Ping Tsai commented on KAFKA-16223: [~cmukka20] Could we take over remaining tasks?

[jira] [Commented] (KAFKA-16027) Refactor MetadataTest#testUpdatePartitionLeadership

2024-04-30 Thread Johnny Hsu (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16027?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842537#comment-17842537 ] Johnny Hsu commented on KAFKA-16027: hey [~alexanderaghili] may I know if we have any updates on

Re: [PR] KAFKA-16572: allow defining number of disks per broker in ClusterTest [kafka]

2024-04-30 Thread via GitHub
FrankYang0529 commented on PR #15745: URL: https://github.com/apache/kafka/pull/15745#issuecomment-2087904101 Hi @chia7712, thanks for the review. I address all comments and add some test cases for ClusterConfig. -- This is an automated message from the Apache Git Service. To respond to

Re: [PR] KAFKA-16207; KRaft's internal log listener to update voter set [kafka]

2024-04-30 Thread via GitHub
jsancio commented on code in PR #15671: URL: https://github.com/apache/kafka/pull/15671#discussion_r1580155319 ## clients/src/main/java/org/apache/kafka/common/record/ControlRecordType.java: ## @@ -44,11 +44,15 @@ public enum ControlRecordType { ABORT((short) 0),

[jira] [Commented] (KAFKA-16637) KIP-848 does not work well

2024-04-30 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16637?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842530#comment-17842530 ] Kirk True commented on KAFKA-16637: --- [~chickenchickenlove]—thanks for filing this. There are two

Re: [PR] KAFKA-15713: KRaft support in AclCommandTest [kafka]

2024-04-30 Thread via GitHub
pasharik commented on code in PR #15830: URL: https://github.com/apache/kafka/pull/15830#discussion_r1585713585 ## core/src/main/scala/kafka/admin/AclCommand.scala: ## @@ -115,8 +115,6 @@ object AclCommand extends Logging { val aclBindings = acls.map(acl => new

Re: [PR] KAFKA-15713: KRaft support in AclCommandTest [kafka]

2024-04-30 Thread via GitHub
pasharik commented on code in PR #15830: URL: https://github.com/apache/kafka/pull/15830#discussion_r1585704666 ## core/src/main/scala/kafka/admin/AclCommand.scala: ## @@ -115,8 +115,6 @@ object AclCommand extends Logging { val aclBindings = acls.map(acl => new

Re: [PR] KAFKA-15713: KRaft support in AclCommandTest [kafka]

2024-04-30 Thread via GitHub
pasharik commented on PR #15830: URL: https://github.com/apache/kafka/pull/15830#issuecomment-2087745006 > Hi @pasharik. Thanks for the change. > > > In the original implementation, listAcls() method was called directly from addAcls() and removeAcls() methods, which caused a race

Re: [PR] KAFKA-15713: KRaft support in AclCommandTest [kafka]

2024-04-30 Thread via GitHub
pasharik commented on code in PR #15830: URL: https://github.com/apache/kafka/pull/15830#discussion_r1585704666 ## core/src/main/scala/kafka/admin/AclCommand.scala: ## @@ -115,8 +115,6 @@ object AclCommand extends Logging { val aclBindings = acls.map(acl => new

[jira] [Commented] (KAFKA-16644) FK join emits duplicate tombstone on left-side delete

2024-04-30 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16644?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842514#comment-17842514 ] Matthias J. Sax commented on KAFKA-16644: - Sorry. Wrong link. Fixed ->

[jira] [Updated] (KAFKA-16644) FK join emits duplicate tombstone on left-side delete

2024-04-30 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16644?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-16644: Description: We introduced a regression bug in 3.7.0 release via KAFKA-14748. When a

Re: [PR] KAFKA-16475: add more tests to TopicImageNodeTest [kafka]

2024-04-30 Thread via GitHub
cmccabe merged PR #15735: URL: https://github.com/apache/kafka/pull/15735 -- 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] KAFKA-16356 RemoteLogMetadataSerde: Serializer via class-name dispatch removed and replaced with if-elseif-else conditions [kafka]

2024-04-30 Thread via GitHub
linu-shibu commented on PR #15620: URL: https://github.com/apache/kafka/pull/15620#issuecomment-2087468237 @gharris1727 I do not have permission/write access to merge the PR. Will I get permission/right to merge? -- This is an automated message from the Apache Git Service. To respond to

Re: [PR] KAFKA-16356 RemoteLogMetadataSerde: Serializer via class-name dispatch removed and replaced with if-elseif-else conditions [kafka]

2024-04-30 Thread via GitHub
linu-shibu commented on PR #15620: URL: https://github.com/apache/kafka/pull/15620#issuecomment-2087467721 > Test failures appear unrelated, there's a targeted RemoteLogMetadataSerdeTest for this logic, and the storage tests appear to pass for me locally. Yes, in local, the tests

Re: [PR] KAFKA-15974: Enforce that event processing respects user-provided timeout [kafka]

2024-04-30 Thread via GitHub
philipnee commented on code in PR #15640: URL: https://github.com/apache/kafka/pull/15640#discussion_r1585558712 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java: ## @@ -30,12 +29,7 @@ public abstract class CommitEvent extends

Re: [PR] KAFKA-15974: Enforce that event processing respects user-provided timeout [kafka]

2024-04-30 Thread via GitHub
philipnee commented on code in PR #15640: URL: https://github.com/apache/kafka/pull/15640#discussion_r1585179214 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ## @@ -1273,6 +1228,22 @@ private void close(Duration timeout, boolean

Re: [PR] KAFKA-16207; KRaft's internal log listener to update voter set [kafka]

2024-04-30 Thread via GitHub
hachikuji commented on code in PR #15671: URL: https://github.com/apache/kafka/pull/15671#discussion_r1583487039 ## core/src/main/scala/kafka/raft/RaftManager.scala: ## @@ -181,20 +181,12 @@ class KafkaRaftManager[T]( private val clientDriver = new

[jira] [Created] (KAFKA-16650) add integration test for Admin#abortTransaction

2024-04-30 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-16650: -- Summary: add integration test for Admin#abortTransaction Key: KAFKA-16650 URL: https://issues.apache.org/jira/browse/KAFKA-16650 Project: Kafka Issue

Re: [PR] KAFKA-15974: Enforce that event processing respects user-provided timeout [kafka]

2024-04-30 Thread via GitHub
lianetm commented on PR #15640: URL: https://github.com/apache/kafka/pull/15640#issuecomment-2087359722 Hey @kirktrue , thanks a lot for the PR, this is a big piece! I completed a pass of all the non-test files, left some comments. -- This is an automated message from the Apache Git

Re: [PR] KAFKA-15974: Enforce that event processing respects user-provided timeout [kafka]

2024-04-30 Thread via GitHub
lianetm commented on PR #15640: URL: https://github.com/apache/kafka/pull/15640#issuecomment-2087355371 > > Here I have a comment, I could not put at the right location in the code: > > > > On line 1362, in commitSync() the consumer waits on the commitFuture with a timer. I think, it

[PR] KAFKA-16649: Remove lock from DynamicBrokerConfig.removeReconfigurable [kafka]

2024-04-30 Thread via GitHub
cmccabe opened a new pull request, #15838: URL: https://github.com/apache/kafka/pull/15838 Do not acquire the DynamicBrokerConfig lock in DynamicBrokerConfig.removeReconfigurable. It's not necessary, because the list that these functions are modifying is a thread-safe CopyOnWriteArrayList.

Re: [PR] KAFKA-14588 UserScramCredentialsCommandTest rewritten in java [kafka]

2024-04-30 Thread via GitHub
chia7712 commented on code in PR #15832: URL: https://github.com/apache/kafka/pull/15832#discussion_r1585527416 ## core/src/test/java/kafka/admin/UserScramCredentialsCommandTest.java: ## @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or

[jira] [Updated] (KAFKA-16649) Remove lock from DynamicBrokerConfig.removeReconfigurable

2024-04-30 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16649?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe updated KAFKA-16649: - Summary: Remove lock from DynamicBrokerConfig.removeReconfigurable (was: Fix potential

[jira] [Updated] (KAFKA-16649) Remove lock from DynamicBrokerConfig.removeReconfigurable

2024-04-30 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16649?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe updated KAFKA-16649: - Description: Do not acquire the DynamicBrokerConfig lock in

[jira] [Created] (KAFKA-16649) Fix potential deadlock in DynamicBrokerConfig

2024-04-30 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-16649: Summary: Fix potential deadlock in DynamicBrokerConfig Key: KAFKA-16649 URL: https://issues.apache.org/jira/browse/KAFKA-16649 Project: Kafka Issue Type:

[jira] [Comment Edited] (KAFKA-16644) FK join emits duplicate tombstone on left-side delete

2024-04-30 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16644?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842504#comment-17842504 ] A. Sophie Blee-Goldman edited comment on KAFKA-16644 at 4/30/24 8:57 PM:

[jira] [Commented] (KAFKA-16644) FK join emits duplicate tombstone on left-side delete

2024-04-30 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16644?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842504#comment-17842504 ] A. Sophie Blee-Goldman commented on KAFKA-16644: [~mjsax] is KAFKA-14778 the correct

Re: [PR] KAFKA-15307: Update/errors for deprecated config [kafka]

2024-04-30 Thread via GitHub
Cerchie commented on PR #14448: URL: https://github.com/apache/kafka/pull/14448#issuecomment-2087251352 tagging @mjsax in for re-review -- 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-16572: allow defining number of disks per broker in ClusterTest [kafka]

2024-04-30 Thread via GitHub
chia7712 commented on code in PR #15745: URL: https://github.com/apache/kafka/pull/15745#discussion_r1585496531 ## core/src/test/java/kafka/test/ClusterConfig.java: ## @@ -319,7 +302,10 @@ public Builder setPerBrokerProperties(Map> perBroke } public

Re: [PR] KAFKA-16614:Disallow @ClusterTemplate("") [kafka]

2024-04-30 Thread via GitHub
chia7712 commented on code in PR #15800: URL: https://github.com/apache/kafka/pull/15800#discussion_r1585492894 ## core/src/test/java/kafka/test/junit/ClusterTestExtensions.java: ## @@ -91,9 +91,6 @@ public Stream provideTestTemplateInvocationContex ClusterTemplate

Re: [PR] KAFKA-15307: update/note deprecated configs [kafka]

2024-04-30 Thread via GitHub
Cerchie commented on PR #14360: URL: https://github.com/apache/kafka/pull/14360#issuecomment-2087132901 tagging @mjsax here, made some edits in response to the last roung -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and

Re: [PR] KAFKA-15307: update/note deprecated configs [kafka]

2024-04-30 Thread via GitHub
Cerchie commented on code in PR #14360: URL: https://github.com/apache/kafka/pull/14360#discussion_r1585489940 ## docs/streams/developer-guide/config-streams.html: ## @@ -257,7 +258,12 @@ num.standby.replicasThe maximum number of records to buffer per partition.

Re: [PR] KAFKA-9401: Reduce contention for Fetch requests [kafka]

2024-04-30 Thread via GitHub
chia7712 commented on code in PR #15836: URL: https://github.com/apache/kafka/pull/15836#discussion_r1585489570 ## core/src/main/scala/kafka/server/FetchSession.scala: ## @@ -603,14 +619,16 @@ class FetchSessionCache(private val maxEntries: Int, // A map containing sessions

Re: [PR] KAFKA-16614:Disallow @ClusterTemplate("") [kafka]

2024-04-30 Thread via GitHub
TaiJuWu commented on code in PR #15800: URL: https://github.com/apache/kafka/pull/15800#discussion_r1585486875 ## core/src/test/java/kafka/test/junit/ClusterTestExtensions.java: ## @@ -91,9 +91,6 @@ public Stream provideTestTemplateInvocationContex ClusterTemplate

[jira] [Comment Edited] (KAFKA-16637) KIP-848 does not work well

2024-04-30 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16637?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842470#comment-17842470 ] Lianet Magrans edited comment on KAFKA-16637 at 4/30/24 8:25 PM: - Hey

Re: [PR] KAFKA-16614:Disallow @ClusterTemplate("") [kafka]

2024-04-30 Thread via GitHub
TaiJuWu commented on code in PR #15800: URL: https://github.com/apache/kafka/pull/15800#discussion_r1585486875 ## core/src/test/java/kafka/test/junit/ClusterTestExtensions.java: ## @@ -91,9 +91,6 @@ public Stream provideTestTemplateInvocationContex ClusterTemplate

Re: [PR] KAFKA-15307: update/note deprecated configs [kafka]

2024-04-30 Thread via GitHub
Cerchie commented on code in PR #14360: URL: https://github.com/apache/kafka/pull/14360#discussion_r1585481048 ## docs/streams/developer-guide/config-streams.html: ## @@ -257,7 +258,12 @@ num.standby.replicasThe maximum number of records to buffer per partition.

Re: [PR] KAFKA-16027: MINOR Refactor MetadataTest#testUpdatePartitionLeadership [kafka]

2024-04-30 Thread via GitHub
chia7712 commented on PR #15055: URL: https://github.com/apache/kafka/pull/15055#issuecomment-2087068199 > Okay I messed up the git a little here (still have much to learn), the current trunk does exist on my [Kafka-16027](https://issues.apache.org/jira/browse/KAFKA-16027) branch but I

Re: [PR] MINOR: Clean up TestUtils.scala [kafka]

2024-04-30 Thread via GitHub
chia7712 merged PR #15808: URL: https://github.com/apache/kafka/pull/15808 -- 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] KAFKA-15974: Enforce that event processing respects user-provided timeout [kafka]

2024-04-30 Thread via GitHub
lianetm commented on code in PR #15640: URL: https://github.com/apache/kafka/pull/15640#discussion_r1585466853 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java: ## @@ -16,9 +16,118 @@ */ package

Re: [PR] KAFKA-16593: Rewrite DeleteConsumerGroupsTest by ClusterTestExtensions [kafka]

2024-04-30 Thread via GitHub
chia7712 commented on code in PR #15766: URL: https://github.com/apache/kafka/pull/15766#discussion_r1585460093 ## tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java: ## @@ -17,279 +17,332 @@ package org.apache.kafka.tools.consumer.group;

Re: [PR] KAFKA-16593: Rewrite DeleteConsumerGroupsTest by ClusterTestExtensions [kafka]

2024-04-30 Thread via GitHub
chia7712 commented on code in PR #15766: URL: https://github.com/apache/kafka/pull/15766#discussion_r1585460093 ## tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java: ## @@ -17,279 +17,332 @@ package org.apache.kafka.tools.consumer.group;

Re: [PR] KAFKA-16593: Rewrite DeleteConsumerGroupsTest by ClusterTestExtensions [kafka]

2024-04-30 Thread via GitHub
chia7712 commented on code in PR #15766: URL: https://github.com/apache/kafka/pull/15766#discussion_r1585460093 ## tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java: ## @@ -17,279 +17,332 @@ package org.apache.kafka.tools.consumer.group;

[jira] [Updated] (KAFKA-9401) High lock contention for kafka.server.FetchManager.newContext

2024-04-30 Thread Gaurav Narula (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9401?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gaurav Narula updated KAFKA-9401: - Fix Version/s: 3.8.0 3.7.1 > High lock contention for

Re: [PR] KAFKA-15974: Enforce that event processing respects user-provided timeout [kafka]

2024-04-30 Thread via GitHub
lianetm commented on code in PR #15640: URL: https://github.com/apache/kafka/pull/15640#discussion_r1585435927 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java: ## @@ -30,12 +29,7 @@ public abstract class CommitEvent extends

Re: [PR] KAFKA-16593: Rewrite DeleteConsumerGroupsTest by ClusterTestExtensions [kafka]

2024-04-30 Thread via GitHub
chia7712 commented on code in PR #15766: URL: https://github.com/apache/kafka/pull/15766#discussion_r1585435466 ## tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java: ## @@ -17,279 +17,332 @@ package org.apache.kafka.tools.consumer.group;

Re: [PR] KAFKA-15974: Enforce that event processing respects user-provided timeout [kafka]

2024-04-30 Thread via GitHub
lianetm commented on code in PR #15640: URL: https://github.com/apache/kafka/pull/15640#discussion_r1585390985 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java: ## @@ -273,9 +310,18 @@ void cleanup() {

Re: [PR] KAFKA-15974: Enforce that event processing respects user-provided timeout [kafka]

2024-04-30 Thread via GitHub
lianetm commented on code in PR #15640: URL: https://github.com/apache/kafka/pull/15640#discussion_r1585390985 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java: ## @@ -273,9 +310,18 @@ void cleanup() {

Re: [PR] KAFKA-16593: Rewrite DeleteConsumerGroupsTest by ClusterTestExtensions [kafka]

2024-04-30 Thread via GitHub
lianetm commented on code in PR #15766: URL: https://github.com/apache/kafka/pull/15766#discussion_r1585359721 ## tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java: ## @@ -17,279 +17,332 @@ package org.apache.kafka.tools.consumer.group;

[jira] [Assigned] (KAFKA-13447) Consumer should not reuse committed offset after topic recreation

2024-04-30 Thread Philip Nee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13447?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Nee reassigned KAFKA-13447: -- Assignee: Philip Nee > Consumer should not reuse committed offset after topic recreation >

[PR] Allowing WriteTxnMarkers API to run with AlterCluster permissions [kafka]

2024-04-30 Thread via GitHub
sidyag opened a new pull request, #15837: URL: https://github.com/apache/kafka/pull/15837 Allowing WriteTxnMarkers API to run with AlterCluster permissions https://issues.apache.org/jira/browse/KAFKA-16513

[jira] [Updated] (KAFKA-16110) Document and publicize performance test results for AsyncKafkaConsumer

2024-04-30 Thread Philip Nee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16110?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Nee updated KAFKA-16110: --- Issue Type: Task (was: New Feature) > Document and publicize performance test results for

[jira] [Updated] (KAFKA-16110) Document and publicize performance test results for AsyncKafkaConsumer

2024-04-30 Thread Philip Nee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16110?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Nee updated KAFKA-16110: --- Summary: Document and publicize performance test results for AsyncKafkaConsumer (was: Implement

[jira] [Updated] (KAFKA-16290) Investigate propagating subscription state updates via queues

2024-04-30 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16290?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16290: -- Priority: Major (was: Critical) > Investigate propagating subscription state updates via queues >

Re: [PR] KAFKA-13328, KAFKA-13329 (2): Add custom preflight validation support for connector header, key, and value converters [kafka]

2024-04-30 Thread via GitHub
gharris1727 commented on code in PR #14309: URL: https://github.com/apache/kafka/pull/14309#discussion_r1585294415 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java: ## @@ -655,27 +811,38 @@ private static ConfigInfos

Re: [PR] KAFKA-9401: Reduce contention for Fetch requests [kafka]

2024-04-30 Thread via GitHub
gaurav-narula commented on PR #15836: URL: https://github.com/apache/kafka/pull/15836#issuecomment-2086478984 The following images show lock profiles collected using async-profiler before and after this change with numCacheShards = numIoThreads = 64 and demonstrates significant reduction

[jira] [Updated] (KAFKA-16110) Implement consumer performance tests

2024-04-30 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16110?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16110: -- Priority: Major (was: Blocker) > Implement consumer performance tests >

Re: [PR] KAFKA-16593: Rewrite DeleteConsumerGroupsTest by ClusterTestExtensions [kafka]

2024-04-30 Thread via GitHub
chia7712 commented on code in PR #15766: URL: https://github.com/apache/kafka/pull/15766#discussion_r1585308784 ## tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java: ## @@ -17,279 +17,332 @@ package org.apache.kafka.tools.consumer.group;

[PR] KAFKA-9401: Reduce contention for Fetch requests [kafka]

2024-04-30 Thread via GitHub
gaurav-narula opened a new pull request, #15836: URL: https://github.com/apache/kafka/pull/15836 KIP-227 introduced in-memory caching of FetchSessions. Brokers with a large number of Fetch requests suffer from contention on trying to acquire a lock on FetchSessionCache. This change

[jira] [Comment Edited] (KAFKA-16637) KIP-848 does not work well

2024-04-30 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16637?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842470#comment-17842470 ] Lianet Magrans edited comment on KAFKA-16637 at 4/30/24 6:10 PM: - Hey

[jira] [Comment Edited] (KAFKA-16637) KIP-848 does not work well

2024-04-30 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16637?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842470#comment-17842470 ] Lianet Magrans edited comment on KAFKA-16637 at 4/30/24 6:10 PM: - Hey

[jira] [Commented] (KAFKA-16637) KIP-848 does not work well

2024-04-30 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16637?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842470#comment-17842470 ] Lianet Magrans commented on KAFKA-16637: Hey [~chickenchickenlove], just to rule out the basics,

[jira] [Updated] (KAFKA-16557) Fix OffsetFetchRequestState.toString()

2024-04-30 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16557?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16557: -- Priority: Minor (was: Major) > Fix OffsetFetchRequestState.toString() >

[jira] [Updated] (KAFKA-16558) Implement HeartbeatRequestState.toStringBase()

2024-04-30 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16558?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16558: -- Priority: Minor (was: Major) > Implement HeartbeatRequestState.toStringBase() >

Re: [PR] KAFKA-16307: fix coordinator thread idle ratio [kafka]

2024-04-30 Thread via GitHub
jeffkbkim commented on code in PR #15835: URL: https://github.com/apache/kafka/pull/15835#discussion_r1585255667 ## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java: ## @@ -461,26 +459,30 @@ public void

[PR] KAFKA-16307: fix coordinator thread idle ratio [kafka]

2024-04-30 Thread via GitHub
jeffkbkim opened a new pull request, #15835: URL: https://github.com/apache/kafka/pull/15835 This PR fixes the thread idle ratio. We take a similar approach to the kafka request handler idle ratio:

[jira] [Assigned] (KAFKA-9401) High lock contention for kafka.server.FetchManager.newContext

2024-04-30 Thread Gaurav Narula (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9401?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gaurav Narula reassigned KAFKA-9401: Assignee: Gaurav Narula > High lock contention for kafka.server.FetchManager.newContext >

Re: [PR] KAFKA-16027: MINOR Refactor MetadataTest#testUpdatePartitionLeadership [kafka]

2024-04-30 Thread via GitHub
Alexander-Aghili commented on PR #15055: URL: https://github.com/apache/kafka/pull/15055#issuecomment-2086139400 Okay I messed up the git a little here (still have much to learn), the current trunk does exist on my Kafka-16027 branch but I had to redo it force push which led to

Re: [PR] KAFKA-16027: MINOR Refactor MetadataTest#testUpdatePartitionLeadership [kafka]

2024-04-30 Thread via GitHub
Alexander-Aghili closed pull request #15055: KAFKA-16027: MINOR Refactor MetadataTest#testUpdatePartitionLeadership URL: https://github.com/apache/kafka/pull/15055 -- 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] KAFKA-16000: Migrate MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-04-30 Thread via GitHub
kirktrue closed pull request #14950: KAFKA-16000: Migrate MembershipManagerImplTest away from ConsumerTestBuilder URL: https://github.com/apache/kafka/pull/14950 -- 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] [Updated] (KAFKA-16623) KafkaAsyncConsumer system tests warn about revoking partitions that weren't previously assigned

2024-04-30 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16623?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16623: -- Description: When running system tests for the KafkaAsyncConsumer, we occasionally see this warning:

[jira] [Updated] (KAFKA-16623) KafkaAsyncConsumer system tests warn about revoking partitions that weren't previously assigned

2024-04-30 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16623?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16623: -- Description: When running system tests for the KafkaAsyncConsumer, we occasionally see this warning:

[jira] [Commented] (KAFKA-16623) KafkaAsyncConsumer system tests warn about revoking partitions that weren't previously assigned

2024-04-30 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16623?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842453#comment-17842453 ] Kirk True commented on KAFKA-16623: --- If I add the following code to {{test_fencing_static_consumer}}

Re: [PR] KAFKA-16555: Consumer's RequestState has incorrect logic to determine if inflight [kafka]

2024-04-30 Thread via GitHub
kirktrue commented on PR #15723: URL: https://github.com/apache/kafka/pull/15723#issuecomment-2086027819 Thanks everyone for the reviews and @lucasbru for the merge! -- 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] MINOR: Replaced Utils.join() with JDK API. [kafka]

2024-04-30 Thread via GitHub
chiacyu commented on code in PR #15823: URL: https://github.com/apache/kafka/pull/15823#discussion_r1585188342 ## clients/src/main/java/org/apache/kafka/common/utils/Utils.java: ## @@ -584,27 +584,6 @@ public static String formatBytes(long bytes) { } } -/**

Re: [PR] KAFKA-13328, KAFKA-13329 (2): Add custom preflight validation support for connector header, key, and value converters [kafka]

2024-04-30 Thread via GitHub
C0urante commented on PR #14309: URL: https://github.com/apache/kafka/pull/14309#issuecomment-2085912341 @gharris1727 I've resolved the merge conflicts again; can you please take a look when you get a chance? -- This is an automated message from the Apache Git Service. To respond to the

[PR] KAFKA-16647: Remove setMetadataDirectory from BrokerNode/ControllerNode [kafka]

2024-04-30 Thread via GitHub
brandboat opened a new pull request, #15833: URL: https://github.com/apache/kafka/pull/15833 related to https://issues.apache.org/jira/browse/KAFKA-16647 ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ] Verify test coverage and

Re: [PR] KAFKA-16587: Add subscription model information to group state [kafka]

2024-04-30 Thread via GitHub
rreddy-22 commented on code in PR #15785: URL: https://github.com/apache/kafka/pull/15785#discussion_r1585160959 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java: ## @@ -1307,13 +1307,14 @@ private CoordinatorResult consumerGr

Re: [PR] KAFKA-16615: JoinGroup API for upgrading ConsumerGroup [kafka]

2024-04-30 Thread via GitHub
dajac commented on PR #15798: URL: https://github.com/apache/kafka/pull/15798#issuecomment-2085866313 @dongnuo123 Be aware of https://github.com/apache/kafka/pull/15785. The PR changes code that you have refactored or reused in this one. We will need to adapt when we merge it. -- This

Re: [PR] KAFKA-16587: Add subscription model information to group state [kafka]

2024-04-30 Thread via GitHub
rreddy-22 commented on code in PR #15785: URL: https://github.com/apache/kafka/pull/15785#discussion_r1585158392 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java: ## @@ -1307,13 +1307,14 @@ private CoordinatorResult consumerGr

Re: [PR] KAFKA-16587: Add subscription model information to group state [kafka]

2024-04-30 Thread via GitHub
dajac commented on code in PR #15785: URL: https://github.com/apache/kafka/pull/15785#discussion_r1585155736 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java: ## @@ -966,6 +979,55 @@ private static void

Re: [PR] KAFKA-16555: Consumer's RequestState has incorrect logic to determine if inflight [kafka]

2024-04-30 Thread via GitHub
philipnee commented on PR #15723: URL: https://github.com/apache/kafka/pull/15723#issuecomment-2085841809 Hey sorry for the delay, the changes look good to me. -- 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] KAFKA-15974: Enforce that event processing respects user-provided timeout [kafka]

2024-04-30 Thread via GitHub
lianetm commented on code in PR #15640: URL: https://github.com/apache/kafka/pull/15640#discussion_r1585152310 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ## @@ -1892,13 +1897,13 @@ private void subscribeInternal(Collection

Re: [PR] MINOR: Add replayRecords to CoordinatorResult [kafka]

2024-04-30 Thread via GitHub
dajac merged PR #15818: URL: https://github.com/apache/kafka/pull/15818 -- 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] KAFKA-15974: Enforce that event processing respects user-provided timeout [kafka]

2024-04-30 Thread via GitHub
lianetm commented on code in PR #15640: URL: https://github.com/apache/kafka/pull/15640#discussion_r1585035057 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ## @@ -1853,6 +1824,40 @@ private void subscribeInternal(Collection

Re: [PR] KAFKA-15974: Enforce that event processing respects user-provided timeout [kafka]

2024-04-30 Thread via GitHub
lianetm commented on code in PR #15640: URL: https://github.com/apache/kafka/pull/15640#discussion_r1585091606 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ## @@ -1892,13 +1897,13 @@ private void subscribeInternal(Collection

Re: [PR] KAFKA-15974: Enforce that event processing respects user-provided timeout [kafka]

2024-04-30 Thread via GitHub
lianetm commented on code in PR #15640: URL: https://github.com/apache/kafka/pull/15640#discussion_r1585083395 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ## @@ -1892,13 +1897,13 @@ private void subscribeInternal(Collection

Re: [PR] KAFKA-15974: Enforce that event processing respects user-provided timeout [kafka]

2024-04-30 Thread via GitHub
lianetm commented on code in PR #15640: URL: https://github.com/apache/kafka/pull/15640#discussion_r1585035057 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ## @@ -1853,6 +1824,40 @@ private void subscribeInternal(Collection

[jira] [Commented] (KAFKA-16382) Kafka Streams drop NULL values after reset

2024-04-30 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16382?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842423#comment-17842423 ] Matthias J. Sax commented on KAFKA-16382: - Not yet from our side... Working on other things atm.

Re: [PR] KAFKA-15974: Enforce that event processing respects user-provided timeout [kafka]

2024-04-30 Thread via GitHub
lianetm commented on code in PR #15640: URL: https://github.com/apache/kafka/pull/15640#discussion_r1585004533 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java: ## @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software

Re: [PR] KAFKA-15974: Enforce that event processing respects user-provided timeout [kafka]

2024-04-30 Thread via GitHub
lianetm commented on code in PR #15640: URL: https://github.com/apache/kafka/pull/15640#discussion_r1585003046 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java: ## @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software

Re: [PR] KAFKA-15974: Enforce that event processing respects user-provided timeout [kafka]

2024-04-30 Thread via GitHub
lianetm commented on code in PR #15640: URL: https://github.com/apache/kafka/pull/15640#discussion_r1585003046 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java: ## @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software

  1   2   >