[jira] [Created] (KAFKA-17217) Clients : Optimise batching of requests per node in ShareConsumeRequestManager

2024-07-29 Thread Shivsundar R (Jira)
Shivsundar R created KAFKA-17217: Summary: Clients : Optimise batching of requests per node in ShareConsumeRequestManager Key: KAFKA-17217 URL: https://issues.apache.org/jira/browse/KAFKA-17217 Projec

Re: [PR] MINOR: Add 3.8.0 to system tests [kafka]

2024-07-29 Thread via GitHub
jlprat commented on PR #16714: URL: https://github.com/apache/kafka/pull/16714#issuecomment-2257606575 Packages are now in https://kafka-packages.s3.us-west-2.amazonaws.com/ Thanks @cmccabe -- This is an automated message from the Apache Git Service. To respond to the message, pleas

[jira] [Updated] (KAFKA-17137) Ensure Admin APIs are properly tested

2024-07-29 Thread Eric Chang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17137?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eric Chang updated KAFKA-17137: --- Description: A number of Admin client APIs don't have integration tests. While testing 3.8.0 RC0 we

[jira] [Updated] (KAFKA-17137) Ensure Admin APIs are properly tested

2024-07-29 Thread Eric Chang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17137?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eric Chang updated KAFKA-17137: --- Description: A number of Admin client APIs don't have integration tests. While testing 3.8.0 RC0 we

[jira] [Commented] (KAFKA-17116) New consumer may not send effective leave group if member ID received after close

2024-07-29 Thread TengYao Chi (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17116?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17869536#comment-17869536 ] TengYao Chi commented on KAFKA-17116: - Hello folks, This is a very interesting disc

Re: [PR] MINOR: simplify code which calles `Punctuator.punctuate()` [kafka]

2024-07-29 Thread via GitHub
mjsax commented on code in PR #16725: URL: https://github.com/apache/kafka/pull/16725#discussion_r1696297327 ## streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java: ## @@ -203,7 +203,7 @@ public void process(final Record record) { } cat

Re: [PR] KAFKA-16448: Handle fatal user exception during processing error [kafka]

2024-07-29 Thread via GitHub
mjsax commented on code in PR #16675: URL: https://github.com/apache/kafka/pull/16675#discussion_r1696289377 ## streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java: ## @@ -214,9 +214,13 @@ public void process(final Record record) {

Re: [PR] KAFKA-16448: Handle fatal user exception during processing error [kafka]

2024-07-29 Thread via GitHub
mjsax commented on code in PR #16675: URL: https://github.com/apache/kafka/pull/16675#discussion_r1696289377 ## streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java: ## @@ -214,9 +214,13 @@ public void process(final Record record) {

Re: [PR] KAFKA-16448: Handle fatal user exception during processing error [kafka]

2024-07-29 Thread via GitHub
mjsax commented on code in PR #16675: URL: https://github.com/apache/kafka/pull/16675#discussion_r1696289377 ## streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java: ## @@ -214,9 +214,13 @@ public void process(final Record record) {

Re: [PR] KAFKA-16448: Handle processing exceptions in punctuate [kafka]

2024-07-29 Thread via GitHub
mjsax commented on code in PR #16300: URL: https://github.com/apache/kafka/pull/16300#discussion_r1696286131 ## streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java: ## @@ -916,7 +920,17 @@ public void punctuate(final ProcessorNode node, }

Re: [PR] KAFKA-16448: Add ErrorHandlerContext in production exception handler [kafka]

2024-07-29 Thread via GitHub
mjsax commented on PR #16433: URL: https://github.com/apache/kafka/pull/16433#issuecomment-2257434274 Thanks for the PR. Merged to `trunk` and cherry-picked to `3.9` branch. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitH

Re: [PR] KAFKA-16448: Add ErrorHandlerContext in deserialization exception handler [kafka]

2024-07-29 Thread via GitHub
mjsax commented on PR #16432: URL: https://github.com/apache/kafka/pull/16432#issuecomment-2257434209 Thanks for the PR. Merged to `trunk` and cherry-picked to `3.9` branch. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitH

Re: [PR] KAFKA-16448: Remove source raw key and source raw value from ErrorHandlerContext [kafka]

2024-07-29 Thread via GitHub
mjsax commented on PR #16716: URL: https://github.com/apache/kafka/pull/16716#issuecomment-2257425665 Just seeing this PR now. Seems it was already included in the other two PRs which I just merged. I believe we can close this one? -- This is an automated message from the Apache Git Servi

Re: [PR] KAFKA-16448: Handle processing exceptions in punctuate [kafka]

2024-07-29 Thread via GitHub
mjsax commented on code in PR #16300: URL: https://github.com/apache/kafka/pull/16300#discussion_r1696256477 ## streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java: ## @@ -916,7 +920,17 @@ public void punctuate(final ProcessorNode node, }

Re: [PR] KAFKA-16066: Upgrade apacheds to 2.0.0.AM27 With apache kerby [kafka]

2024-07-29 Thread via GitHub
highluck commented on PR #15277: URL: https://github.com/apache/kafka/pull/15277#issuecomment-2257418930 @mimaison I fixed the conflict thanks -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to

Re: [PR] KAFKA-13898 : Updated docs for metrics.recording.level [kafka]

2024-07-29 Thread via GitHub
Nancy-ksolves commented on PR #16402: URL: https://github.com/apache/kafka/pull/16402#issuecomment-2257413970 Hi @chia7712, Can you please review the PR? -- 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

Re: [PR] KAFKA-16057 : Change the configuration for adminclient connections.max.idle.ms to 9 minutes [kafka]

2024-07-29 Thread via GitHub
Nancy-ksolves commented on PR #16548: URL: https://github.com/apache/kafka/pull/16548#issuecomment-2257410972 Hi @mimaison , I wanted to inform you that our Confluence account has not been created yet. we are currently unable to create the KIP (Kafka Improvement Proposal) for this ticke

Re: [PR] KAFKA-17188: Ensure login and callback handler are closed upon encountering LoginException [kafka]

2024-07-29 Thread via GitHub
philipnee commented on PR #16724: URL: https://github.com/apache/kafka/pull/16724#issuecomment-2257402065 Hello @rajinisivaram - Here is the PR for the issue for not closing the login and handler upon LoginException. If you get time, would you be kindly reviewing this? Thanks! -- This i

Re: [PR] KAFKA-17188: Ensure login and callback handler are closed upon encountering LoginException [kafka]

2024-07-29 Thread via GitHub
philipnee commented on code in PR #16724: URL: https://github.com/apache/kafka/pull/16724#discussion_r1696234213 ## clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java: ## @@ -168,6 +172,11 @@ public String toString() { ", refC

[PR] KAFKA-17188: Ensure login and callback handler are closed upon encountering LoginException [kafka]

2024-07-29 Thread via GitHub
philipnee opened a new pull request, #16724: URL: https://github.com/apache/kafka/pull/16724 Adding a try-catch in the LoginManager constructor to ensure `Login` and `AuthenticateCallbackHandler` are closed when login.login() throws an exception. -- This is an automated message from the

Re: [PR] KAFKA-16448: Fix raw record not being cached in store [kafka]

2024-07-29 Thread via GitHub
mjsax commented on PR #16684: URL: https://github.com/apache/kafka/pull/16684#issuecomment-2257392840 > Got it, should we update KIP-1033 to remove any link with sourceRawKey and sourceRawValue? Yes, we should update the KIP accordingly. > We'll update existing PRs, and open a

Re: [PR] KAFKA-16448: Handle fatal user exception during processing error [kafka]

2024-07-29 Thread via GitHub
mjsax commented on PR #16675: URL: https://github.com/apache/kafka/pull/16675#issuecomment-2257391601 @sebastienviale -- The PR has merge conflicts and needs to be rebased. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and u

Re: [PR] KAFKA-16448: Add ErrorHandlerContext in deserialization exception handler [kafka]

2024-07-29 Thread via GitHub
mjsax commented on code in PR #16432: URL: https://github.com/apache/kafka/pull/16432#discussion_r1696221005 ## streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java: ## @@ -80,12 +81,29 @@ public static void handleDeserializationFailure(fina

Re: [PR] KAFKA-16448: Add ErrorHandlerContext in deserialization exception handler [kafka]

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

Re: [PR] KAFKA-16448: Add ErrorHandlerContext in production exception handler [kafka]

2024-07-29 Thread via GitHub
mjsax commented on code in PR #16433: URL: https://github.com/apache/kafka/pull/16433#discussion_r1696213764 ## streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java: ## @@ -285,15 +280,95 @@ public void send(final String topic,

[PR] KAFKA-17201: SelectorTest.testInboundConnectionsCountInConnectionCreationMetric leaks sockets and threads [kafka]

2024-07-29 Thread via GitHub
frankvicky opened a new pull request, #16723: URL: https://github.com/apache/kafka/pull/16723 Currently, the sockets and threads created in `testInboundConnectionsCountInConnectionCreationMetric` have not been closed, and this issue needs to be fixed. ### Committer Checklist (exclude

Re: [PR] KAFKA-16448: Add ErrorHandlerContext in production exception handler [kafka]

2024-07-29 Thread via GitHub
mjsax merged PR #16433: URL: https://github.com/apache/kafka/pull/16433 -- 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] [Created] (KAFKA-17216) StreamsConfig STATE_DIR_CONFIG

2024-07-29 Thread raphaelauv (Jira)
raphaelauv created KAFKA-17216: -- Summary: StreamsConfig STATE_DIR_CONFIG Key: KAFKA-17216 URL: https://issues.apache.org/jira/browse/KAFKA-17216 Project: Kafka Issue Type: Bug Affects Versio

Re: [PR] KAFKA-16448: Add ErrorHandlerContext in production exception handler [kafka]

2024-07-29 Thread via GitHub
mjsax commented on code in PR #16433: URL: https://github.com/apache/kafka/pull/16433#discussion_r1696207479 ## streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java: ## @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Founda

Re: [PR] KAFKA-14957: Update-Description-String [kafka]

2024-07-29 Thread via GitHub
raphaelauv commented on code in PR #13909: URL: https://github.com/apache/kafka/pull/13909#discussion_r1696207016 ## streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java: ## @@ -839,7 +839,8 @@ public class StreamsConfig extends AbstractConfig {

Re: [PR] KAFKA-17175: Remove interface `BrokerNode` and `ControllerNode` [kafka]

2024-07-29 Thread via GitHub
frankvicky commented on PR #1: URL: https://github.com/apache/kafka/pull/1#issuecomment-2257364095 Hello @chia7712 I have redefine the scope of method, PTAL 😺 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and us

Re: [PR] KAFKA-16390: consume_bench_test.py failed using AsyncKafkaConsumer [kafka]

2024-07-29 Thread via GitHub
frankvicky commented on code in PR #16715: URL: https://github.com/apache/kafka/pull/16715#discussion_r1696187170 ## tests/kafkatest/services/kafka/kafka.py: ## @@ -782,7 +782,8 @@ def prop_file(self, node): if self.use_new_coordinator: override_configs[c

[jira] [Commented] (KAFKA-17214) Add 3.8.0 Streams and Core to system tests

2024-07-29 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17214?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17869487#comment-17869487 ] Matthias J. Sax commented on KAFKA-17214: - As we already have a 3.9 branch, PR f

[jira] [Updated] (KAFKA-17214) Add 3.8.0 Streams and Core to system tests

2024-07-29 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17214?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-17214: Component/s: clients consumer core produ

[PR] MINOR: remove get prefix for internal Kafka Streams methods [kafka]

2024-07-29 Thread via GitHub
mjsax opened a new pull request, #16722: URL: https://github.com/apache/kafka/pull/16722 Browsing through the code base to file https://issues.apache.org/jira/browse/KAFKA-17215, I saw that these methods also use `get` prefix, and though might be worth to clean it up. This is only a

[jira] [Updated] (KAFKA-17215) Remove get-prefix for all getters

2024-07-29 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17215?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-17215: Description: Kafka traditionally does not use a `get` prefix for getter methods. However,

[jira] [Resolved] (KAFKA-17186) Cannot receive message after stopping Source Mirror Maker 2

2024-07-29 Thread George Yang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17186?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] George Yang resolved KAFKA-17186. - Resolution: Not A Problem configuration issues, fixed > Cannot receive message after stopping S

[jira] [Commented] (KAFKA-17186) Cannot receive message after stopping Source Mirror Maker 2

2024-07-29 Thread George Yang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17186?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17869474#comment-17869474 ] George Yang commented on KAFKA-17186: - OK,[~gharris1727]  Thank you for your explana

Re: [PR] KAFKA-16745: Implemented handleShareFetchRequest RPC including unit tests [kafka]

2024-07-29 Thread via GitHub
junrao commented on code in PR #16456: URL: https://github.com/apache/kafka/pull/16456#discussion_r1696094681 ## core/src/test/scala/unit/kafka/server/KafkaApisTest.scala: ## @@ -4406,6 +4414,2341 @@ class KafkaApisTest extends Logging { assertEquals("broker2", node.host)

Re: [PR] KAFKA-16745: Implemented handleShareFetchRequest RPC including unit tests [kafka]

2024-07-29 Thread via GitHub
junrao commented on code in PR #16456: URL: https://github.com/apache/kafka/pull/16456#discussion_r1696066333 ## core/src/test/scala/unit/kafka/server/KafkaApisTest.scala: ## @@ -4406,6 +4414,2341 @@ class KafkaApisTest extends Logging { assertEquals("broker2", node.host)

Re: [PR] KAFKA-16745: Implemented handleShareFetchRequest RPC including unit tests [kafka]

2024-07-29 Thread via GitHub
junrao commented on code in PR #16456: URL: https://github.com/apache/kafka/pull/16456#discussion_r169611 ## core/src/main/scala/kafka/server/KafkaApis.scala: ## @@ -3955,11 +3948,484 @@ class KafkaApis(val requestChannel: RequestChannel, } } + /** + * Handle a

Re: [PR] KAFKA-15203 Use Classgraph since org.reflections is no longer under maintainence [kafka]

2024-07-29 Thread via GitHub
gharris1727 commented on code in PR #16604: URL: https://github.com/apache/kafka/pull/16604#discussion_r1696102323 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginUtils.java: ## @@ -453,4 +457,58 @@ private static class DirectoryEntry {

[jira] [Updated] (KAFKA-17215) Remove get-prefix for all getters

2024-07-29 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17215?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-17215: Description: Kafka traditionally does not use a `get` prefix for getter methods. However,

[jira] [Updated] (KAFKA-17215) Remove get-prefix for all getters

2024-07-29 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17215?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-17215: Description: Kafka traditionally does not use a `get` prefix for getter methods. However,

[jira] [Commented] (KAFKA-17186) Cannot receive message after stopping Source Mirror Maker 2

2024-07-29 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17186?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17869468#comment-17869468 ] Greg Harris commented on KAFKA-17186: - I don't think i've ever thought about it like

[jira] [Created] (KAFKA-17215) Remove get-prefix for all getters

2024-07-29 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-17215: --- Summary: Remove get-prefix for all getters Key: KAFKA-17215 URL: https://issues.apache.org/jira/browse/KAFKA-17215 Project: Kafka Issue Type: Improveme

[jira] [Updated] (KAFKA-17211) kafka producer get stuck by Uncaught error in kafka producer I/O thread

2024-07-29 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17211?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-17211: -- Component/s: producer > kafka producer get stuck by Uncaught error in kafka producer I/O thread > ---

[jira] [Commented] (KAFKA-17116) New consumer may not send effective leave group if member ID received after close

2024-07-29 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17116?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17869467#comment-17869467 ] Kirk True commented on KAFKA-17116: --- My understanding is: # This is an edge case # T

Re: [PR] KAFKA-17209: Revisit testCurrentLag for asyncConsumer [kafka]

2024-07-29 Thread via GitHub
kirktrue commented on code in PR #16703: URL: https://github.com/apache/kafka/pull/16703#discussion_r1696033289 ## clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java: ## @@ -2491,7 +2508,8 @@ public void testCurrentLag(GroupProtocol groupProtocol) {

Re: [PR] KAFKA-17192 Fix MirrorMaker2 worker config does not pass config.provi… [kafka]

2024-07-29 Thread via GitHub
k0b3rIT commented on PR #16678: URL: https://github.com/apache/kafka/pull/16678#issuecomment-2257117723 > Thanks @k0b3rIT! Looks good for the most part, just one small comment request. Can you also add a description for the changes to your PR (where the "_More detailed description of your c

Re: [PR] KAFKA-17192 Fix MirrorMaker2 worker config does not pass config.provi… [kafka]

2024-07-29 Thread via GitHub
k0b3rIT commented on code in PR #16678: URL: https://github.com/apache/kafka/pull/16678#discussion_r1696031526 ## clients/src/main/java/org/apache/kafka/common/utils/Utils.java: ## @@ -1648,9 +1648,13 @@ public static Map entriesWithPrefix(Map map, String pr * @param th

[jira] [Updated] (KAFKA-16390) consume_bench_test.py failed using AsyncKafkaConsumer

2024-07-29 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16390?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16390: -- Component/s: clients > consume_bench_test.py failed using AsyncKafkaConsumer > ---

[jira] [Commented] (KAFKA-16390) consume_bench_test.py failed using AsyncKafkaConsumer

2024-07-29 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16390?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17869464#comment-17869464 ] Kirk True commented on KAFKA-16390: --- [~frankvicky] transferring the Jira to you and ma

[jira] [Assigned] (KAFKA-16390) consume_bench_test.py failed using AsyncKafkaConsumer

2024-07-29 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16390?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True reassigned KAFKA-16390: - Assignee: TengYao Chi (was: Kirk True) > consume_bench_test.py failed using AsyncKafkaConsumer

Re: [PR] KAFKA-16899 MembershipManagerImpl: rebalanceTimeoutMs variable name changed to commitTimeoutDuringReconciliation [kafka]

2024-07-29 Thread via GitHub
kirktrue commented on code in PR #16334: URL: https://github.com/apache/kafka/pull/16334#discussion_r1696026161 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java: ## @@ -138,11 +138,11 @@ public class MembershipManagerImpl implements

[PR] MINOR: Flaky test reduced timeouts to help eliminate race conditions on timeout [kafka]

2024-07-29 Thread via GitHub
bbejeck opened a new pull request, #16721: URL: https://github.com/apache/kafka/pull/16721 Flaky test - seems there could be a race condition with timeouts in this test, I've reduced the pauses at various points in the test to speed it up and see if that helps with flakyness

Re: [PR] KAFKA-16450: Add instance methods to Exit to replace static methods [kafka]

2024-07-29 Thread via GitHub
gharris1727 commented on code in PR #15598: URL: https://github.com/apache/kafka/pull/15598#discussion_r1695952150 ## clients/src/test/java/org/apache/kafka/common/utils/MockExit.java: ## @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or mor

Re: [PR] KAFKA-16450: Add instance methods to Exit to replace static methods [kafka]

2024-07-29 Thread via GitHub
gharris1727 commented on PR #15598: URL: https://github.com/apache/kafka/pull/15598#issuecomment-2257077548 > One higher-level thought I had: is it better to keep all of this new logic in the existing Exit class, or would it make sense to separate the two? IMO the final state we work toward

Re: [PR] KAFKA-16450: Add instance methods to Exit to replace static methods [kafka]

2024-07-29 Thread via GitHub
gharris1727 commented on code in PR #15598: URL: https://github.com/apache/kafka/pull/15598#discussion_r1695995911 ## clients/src/main/java/org/apache/kafka/common/utils/Exit.java: ## @@ -119,4 +134,138 @@ public static void resetHaltProcedure() { public static void resetSh

Re: [PR] KAFKA-16450: Add instance methods to Exit to replace static methods [kafka]

2024-07-29 Thread via GitHub
gharris1727 commented on code in PR #15598: URL: https://github.com/apache/kafka/pull/15598#discussion_r1695952150 ## clients/src/test/java/org/apache/kafka/common/utils/MockExit.java: ## @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or mor

Re: [PR] KAFKA-17190: AssignmentsManager gets stuck retrying on deleted topics [kafka]

2024-07-29 Thread via GitHub
cmccabe commented on code in PR #16672: URL: https://github.com/apache/kafka/pull/16672#discussion_r1695820312 ## server/src/test/java/org/apache/kafka/server/AssignmentsManagerTest.java: ## @@ -14,451 +14,506 @@ * See the License for the specific language governing permission

Re: [PR] KAFKA-17190: AssignmentsManager gets stuck retrying on deleted topics [kafka]

2024-07-29 Thread via GitHub
cmccabe commented on code in PR #16672: URL: https://github.com/apache/kafka/pull/16672#discussion_r1695816959 ## server/src/test/java/org/apache/kafka/server/AssignmentsManagerTest.java: ## @@ -14,451 +14,506 @@ * See the License for the specific language governing permission

Re: [PR] KAFKA-16312, KAFKA-16185: Local epochs in reconciliation [kafka]

2024-07-29 Thread via GitHub
lianetm commented on code in PR #15511: URL: https://github.com/apache/kafka/pull/15511#discussion_r1695813884 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java: ## @@ -568,7 +570,11 @@ public void transitionToFenced() { public

[jira] [Updated] (KAFKA-17209) Revisit testCurrentLag for AsyncKafkaConsumer

2024-07-29 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17209?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-17209: -- Labels: kip-848-client-support (was: ) > Revisit testCurrentLag for AsyncKafkaConsumer >

[jira] [Updated] (KAFKA-17209) Revisit testCurrentLag for AsyncKafkaConsumer

2024-07-29 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17209?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-17209: -- Component/s: clients unit tests > Revisit testCurrentLag for AsyncKafkaConsumer > ---

Re: [PR] MINOR: Fix consumer log on fatal error & improve memberId logging [kafka]

2024-07-29 Thread via GitHub
lianetm commented on PR #16720: URL: https://github.com/apache/kafka/pull/16720#issuecomment-2256827278 Hey @chia7712 , here's the fix for the log with/without member ID on `transitionToFatal` (applied generally to cover us in all logs). Could you take a look when you have a chance? Thanks!

[PR] KAFKA-14562: Implement epoch bump after every transaction [kafka]

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

Re: [PR] KAFKA-16308 [3/N]: Introduce feature dependency validation to UpdateFeatures command [kafka]

2024-07-29 Thread via GitHub
jolshan commented on PR #16443: URL: https://github.com/apache/kafka/pull/16443#issuecomment-2256803549 I will probably just include this in 4.0 so we can drop the ZK support as well. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to

Re: [PR] KAFKA-17210: Broker fixes for smooth concurrent fetches on share partition [kafka]

2024-07-29 Thread via GitHub
omkreddy merged PR #16711: URL: https://github.com/apache/kafka/pull/16711 -- 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: update CachingPersistentWindowStoreTest [kafka]

2024-07-29 Thread via GitHub
mjsax merged PR #16701: URL: https://github.com/apache/kafka/pull/16701 -- 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-16883) Zookeeper-Kraft failing migration - RPC got timed out before it could be sent

2024-07-29 Thread David Arthur (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16883?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17869447#comment-17869447 ] David Arthur commented on KAFKA-16883: -- Thanks for confirming your success hre [~ni

[jira] [Closed] (KAFKA-16883) Zookeeper-Kraft failing migration - RPC got timed out before it could be sent

2024-07-29 Thread David Arthur (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16883?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Arthur closed KAFKA-16883. > Zookeeper-Kraft failing migration - RPC got timed out before it could be sent >

Re: [PR] MINOR: update flaky CustomHandlerIntegrationTest [kafka]

2024-07-29 Thread via GitHub
mjsax commented on PR #16710: URL: https://github.com/apache/kafka/pull/16710#issuecomment-2256762537 Re-triggered Jenkins to make sure... -- 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 spec

[jira] [Commented] (KAFKA-17146) ZK to KRAFT migration stuck in pre-migration mode

2024-07-29 Thread David Arthur (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17869445#comment-17869445 ] David Arthur commented on KAFKA-17146: -- [~saimon46] thanks for the report. > More

Re: [PR] KAFKA-16953: Properly implement the sending of DescribeQuorumResponse [kafka]

2024-07-29 Thread via GitHub
jsancio merged PR #16637: URL: https://github.com/apache/kafka/pull/16637 -- 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-15522) Flaky test org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithFrequentOffsetSyncs

2024-07-29 Thread Chris Egerton (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15522?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chris Egerton resolved KAFKA-15522. --- Resolution: Fixed > Flaky test > org.apache.kafka.connect.mirror.integration.MirrorConnecto

[jira] [Commented] (KAFKA-15522) Flaky test org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationExactlyOnceTest.testOneWayReplicationWithFrequentOffsetSyncs

2024-07-29 Thread Chris Egerton (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15522?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17869429#comment-17869429 ] Chris Egerton commented on KAFKA-15522: --- We haven't seen this failure in a while a

[jira] [Created] (KAFKA-17214) Add 3.8.0 Streams and Core to system tests

2024-07-29 Thread Josep Prat (Jira)
Josep Prat created KAFKA-17214: -- Summary: Add 3.8.0 Streams and Core to system tests Key: KAFKA-17214 URL: https://issues.apache.org/jira/browse/KAFKA-17214 Project: Kafka Issue Type: Bug

[jira] [Updated] (KAFKA-17213) Change ControllerMovedException from ApiException to InvalidMetadataException.

2024-07-29 Thread Josep Prat (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17213?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Josep Prat updated KAFKA-17213: --- Fix Version/s: (was: 3.8.0) > Change ControllerMovedException from ApiException to InvalidMetada

Re: [PR] KAFKA-16312, KAFKA-16185: Local epochs in reconciliation [kafka]

2024-07-29 Thread via GitHub
chia7712 commented on code in PR #15511: URL: https://github.com/apache/kafka/pull/15511#discussion_r1695624583 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java: ## @@ -568,7 +570,11 @@ public void transitionToFenced() { public

Re: [PR] KAFKA-16312, KAFKA-16185: Local epochs in reconciliation [kafka]

2024-07-29 Thread via GitHub
lianetm commented on code in PR #15511: URL: https://github.com/apache/kafka/pull/15511#discussion_r1695597540 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java: ## @@ -568,7 +570,11 @@ public void transitionToFenced() { public

Re: [PR] KAFKA-16900 kafka-producer-perf-test reports error when using transaction. [kafka]

2024-07-29 Thread via GitHub
chia7712 commented on code in PR #16646: URL: https://github.com/apache/kafka/pull/16646#discussion_r1695591590 ## tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java: ## @@ -494,4 +473,60 @@ public void onCompletion(RecordMetadata metadata, Exception exception)

[jira] [Resolved] (KAFKA-17202) EosIntegrationTest.verifyChangelogMaxRecordOffsetMatchesCheckpointedOffset leaks consumers

2024-07-29 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17202?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-17202. Fix Version/s: 3.9.0 Resolution: Fixed > EosIntegrationTest.verifyChangelogMaxRecor

Re: [PR] KAFKA-17202: EosIntegrationTest.verifyChangelogMaxRecordOffsetMatchesCheckpointedOffset leaks consumers [kafka]

2024-07-29 Thread via GitHub
chia7712 merged PR #16702: URL: https://github.com/apache/kafka/pull/16702 -- 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: update flaky CustomHandlerIntegrationTest [kafka]

2024-07-29 Thread via GitHub
bbejeck commented on PR #16710: URL: https://github.com/apache/kafka/pull/16710#issuecomment-2256448024 Test failures are unrelated, but it seems build 21 didn't run - so it may be worth running again, but it's up to you. -- This is an automated message from the Apache Git Service. To res

Re: [PR] KAFKA-17151: Remove waitForCondition when detecting thread leak [kafka]

2024-07-29 Thread via GitHub
chia7712 commented on code in PR #16661: URL: https://github.com/apache/kafka/pull/16661#discussion_r1695547956 ## core/src/test/java/kafka/testkit/KafkaClusterThreadFactory.java: ## @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + *

[jira] [Commented] (KAFKA-4928) Add integration test for DumpLogSegments

2024-07-29 Thread Volk Huang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-4928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17869406#comment-17869406 ] Volk Huang commented on KAFKA-4928: --- [~chia7712] No problem, thank you for the tips! >

Re: [PR] KAFKA-17151: Remove waitForCondition when detecting thread leak [kafka]

2024-07-29 Thread via GitHub
chia7712 commented on PR #16661: URL: https://github.com/apache/kafka/pull/16661#issuecomment-2256415831 @FrankYang0529 please update related jiras if you try to fix them in this single PR -- This is an automated message from the Apache Git Service. To respond to the message, please log o

Re: [PR] KAFKA-17175: Remove interface `BrokerNode` and `ControllerNode` [kafka]

2024-07-29 Thread via GitHub
chia7712 commented on code in PR #1: URL: https://github.com/apache/kafka/pull/1#discussion_r1695540825 ## core/src/test/java/kafka/testkit/TestKitNodes.java: ## @@ -216,4 +218,90 @@ public ListenerName externalListenerName() { public ListenerName controllerListener

Re: [PR] KAFKA-17198: Partition imbalance after alter due to inconsistent order of broker list [kafka]

2024-07-29 Thread via GitHub
chia7712 commented on PR #16712: URL: https://github.com/apache/kafka/pull/16712#issuecomment-2256403019 @popsunlake thanks for this patch. I will take a look tomorrow :) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use

[jira] [Commented] (KAFKA-4928) Add integration test for DumpLogSegments

2024-07-29 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-4928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17869400#comment-17869400 ] Chia-Ping Tsai commented on KAFKA-4928: --- [~soravolk] I have assigned this jira to y

[jira] [Assigned] (KAFKA-4928) Add integration test for DumpLogSegments

2024-07-29 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-4928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai reassigned KAFKA-4928: - Assignee: Volk Huang > Add integration test for DumpLogSegments > --

[jira] [Assigned] (KAFKA-17201) SelectorTest.testInboundConnectionsCountInConnectionCreationMetric leaks sockets and threads

2024-07-29 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17201?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai reassigned KAFKA-17201: -- Assignee: TengYao Chi > SelectorTest.testInboundConnectionsCountInConnectionCreationM

[jira] [Commented] (KAFKA-4928) Add integration test for DumpLogSegments

2024-07-29 Thread Volk Huang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-4928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17869399#comment-17869399 ] Volk Huang commented on KAFKA-4928: --- Hi [~ijuma], can I try this one? Thank you so much

[jira] [Commented] (KAFKA-17116) New consumer may not send effective leave group if member ID received after close

2024-07-29 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17116?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17869396#comment-17869396 ] Chia-Ping Tsai commented on KAFKA-17116: [~dajac] thanks for sharing the idea an

Re: [PR] KAFKA-16450: Add instance methods to Exit to replace static methods [kafka]

2024-07-29 Thread via GitHub
gharris1727 commented on code in PR #15598: URL: https://github.com/apache/kafka/pull/15598#discussion_r1695512242 ## clients/src/main/java/org/apache/kafka/common/utils/Exit.java: ## @@ -55,22 +55,37 @@ public interface ShutdownHookAdder { private static volatile Procedure

Re: [PR] KAFKA-16450: Add instance methods to Exit to replace static methods [kafka]

2024-07-29 Thread via GitHub
gharris1727 commented on code in PR #15598: URL: https://github.com/apache/kafka/pull/15598#discussion_r1695510458 ## clients/src/test/java/org/apache/kafka/common/utils/MockExit.java: ## @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or mor

Re: [PR] KAFKA-16450: Add instance methods to Exit to replace static methods [kafka]

2024-07-29 Thread via GitHub
C0urante commented on code in PR #15598: URL: https://github.com/apache/kafka/pull/15598#discussion_r1695447336 ## clients/src/test/java/org/apache/kafka/common/utils/MockExit.java: ## @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more +

Re: [PR] KAFKA-16390: consume_bench_test.py failed using AsyncKafkaConsumer [kafka]

2024-07-29 Thread via GitHub
lianetm commented on code in PR #16715: URL: https://github.com/apache/kafka/pull/16715#discussion_r1695486461 ## tests/kafkatest/services/kafka/kafka.py: ## @@ -782,7 +782,8 @@ def prop_file(self, node): if self.use_new_coordinator: override_configs[conf

Re: [PR] KAFKA-16312, KAFKA-16185: Local epochs in reconciliation [kafka]

2024-07-29 Thread via GitHub
chia7712 commented on code in PR #15511: URL: https://github.com/apache/kafka/pull/15511#discussion_r1695461084 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java: ## @@ -568,7 +570,11 @@ public void transitionToFenced() { public

  1   2   >