[jira] [Commented] (KAFKA-18084) Null and leaked AcquisitionLockTimerTask causes hanging AcknowledgeRequest and corrupted state of batch

2024-11-26 Thread Andrew Schofield (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901402#comment-17901402 ] Andrew Schofield commented on KAFKA-18084: -- Thanks, [~adixitconfluent]. I think

Re: [PR] KAFKA-18100: `Using` block suppresses all errors [kafka]

2024-11-26 Thread via GitHub
frankvicky commented on PR #17954: URL: https://github.com/apache/kafka/pull/17954#issuecomment-2503114950 Hi @chia7712 I have file a PR for it #17954 -- 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-18068: Fixing typo in ProducerConfig [kafka]

2024-11-26 Thread via GitHub
AndrewJSchofield commented on PR #17908: URL: https://github.com/apache/kafka/pull/17908#issuecomment-2503121184 Actually, thinking more about this, KIP-794 https://cwiki.apache.org/confluence/plugins/servlet/mobile?contentId=191336857#content/view/191336857 defines the config without the s

[PR] KAFKA-18100: Fix testDeserializationErrorFailsTheLoading [kafka]

2024-11-26 Thread via GitHub
frankvicky opened a new pull request, #17956: URL: https://github.com/apache/kafka/pull/17956 JIRA: KAFKA-18100 This test fails due to the error flow change in #16898. To fix it, we should update the assertion message. ### Committer Checklist (excluded from commit message) -

Re: [PR] KAFKA-17864: add descriptions to fields in the agreement [kafka]

2024-11-26 Thread via GitHub
yx9o commented on code in PR #17681: URL: https://github.com/apache/kafka/pull/17681#discussion_r1860046772 ## clients/src/main/resources/common/message/DescribeClientQuotasResponse.json: ## @@ -36,7 +36,7 @@ "about": "The entity name, or null if the default." }

[jira] [Commented] (KAFKA-18084) Null and leaked AcquisitionLockTimerTask causes hanging AcknowledgeRequest and corrupted state of batch

2024-11-26 Thread Abhinav Dixit (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901385#comment-17901385 ] Abhinav Dixit commented on KAFKA-18084: --- [~chia7712], thanks for pointing out thes

[jira] [Commented] (KAFKA-17893) Support record keys in the foreignKeyExtractor argument of KTable foreign join

2024-11-26 Thread Peter Lee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17893?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901377#comment-17901377 ] Peter Lee commented on KAFKA-17893: --- Hi [~mjsax]  PR is ready. PTAL. Thanks [https://g

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

2024-11-26 Thread Deng Ziming (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18101?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Deng Ziming reassigned KAFKA-18101: --- Assignee: Peter Lee > Merge duplicate assertFutureThrows and assertFutureExceptionTypeEqual

[PR] KAFKA-18098: add kraft support to testReplicaPlacementAllServers and testReplicaPlacementPartialServers [kafka]

2024-11-26 Thread via GitHub
peterxcli opened a new pull request, #17955: URL: https://github.com/apache/kafka/pull/17955 In kraft the [replicas algorithm](https://github.com/apache/kafka/blob/trunk/metadata/src/main/java/org/apache/kafka/metadata/placement/StripedReplicaPlacer.java#L156) includes some randomization, s

Re: [PR] KAFKA-14563: RemoveClient-Side AddPartitionsToTxn Requests [kafka]

2024-11-26 Thread via GitHub
CalvinConfluent commented on code in PR #17698: URL: https://github.com/apache/kafka/pull/17698#discussion_r1859947357 ## core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala: ## @@ -525,6 +558,7 @@ class TransactionCoordinator(txnConfig: TransactionCon

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

2024-11-26 Thread Peter Lee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901373#comment-17901373 ] Peter Lee commented on KAFKA-18101: --- HI [~dengziming], I would like to work on this, m

Re: [PR] KAFKA-18100: `Using` block suppresses all errors [kafka]

2024-11-26 Thread via GitHub
chia7712 commented on PR #17954: URL: https://github.com/apache/kafka/pull/17954#issuecomment-2502771967 @frankvicky Could you please a PR to fix the ``CoordinatorLoaderImplTest` ` first? -- This is an automated message from the Apache Git Service. To respond to the message, please log on

Re: [PR] KAFKA-18100: `Using` block suppresses all errors [kafka]

2024-11-26 Thread via GitHub
chia7712 commented on PR #17954: URL: https://github.com/apache/kafka/pull/17954#issuecomment-2502763639 @TaiJuWu Could you replace all `Using` by "try-release" ? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL

[PR] KAFKA-18100: https://issues.apache.org/jira/browse/KAFKA-18100 [kafka]

2024-11-26 Thread via GitHub
TaiJuWu opened a new pull request, #17954: URL: https://github.com/apache/kafka/pull/17954 JIRA: https://issues.apache.org/jira/browse/KAFKA-18100/ Only add `match` after `Scala.Using` ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementati

[jira] [Assigned] (KAFKA-18100) `Using` block suppresses all errors

2024-11-26 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18100?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai reassigned KAFKA-18100: -- Assignee: Chia-Ping Tsai > `Using` block suppresses all errors >

Re: [PR] KAFKA-16640: Replace TestUtils#resource by scala.util.Using [kafka]

2024-11-26 Thread via GitHub
chia7712 commented on PR #15881: URL: https://github.com/apache/kafka/pull/15881#issuecomment-2502688514 Sorry for the trouble. I’ll take over the Jira. 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 a

Re: [PR] KAFKA-16640: Replace TestUtils#resource by scala.util.Using [kafka]

2024-11-26 Thread via GitHub
frankvicky commented on PR #15881: URL: https://github.com/apache/kafka/pull/15881#issuecomment-2502685865 Oops, I will take a look -- 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 c

[jira] [Commented] (KAFKA-18100) `Using` block suppresses all errors

2024-11-26 Thread TengYao Chi (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18100?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901370#comment-17901370 ] TengYao Chi commented on KAFKA-18100: - Hello [~jolshan]  I will take a look and fix

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

2024-11-26 Thread via GitHub
TaiJuWu commented on code in PR #17946: URL: https://github.com/apache/kafka/pull/17946#discussion_r1859537038 ## clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java: ## @@ -1223,6 +1223,11 @@ private void ensureValidRecordSize(int size) { */ @O

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

2024-11-26 Thread Deng Ziming (Jira)
Deng Ziming created KAFKA-18101: --- Summary: Merge duplicate assertFutureThrows and assertFutureExceptionTypeEquals Key: KAFKA-18101 URL: https://issues.apache.org/jira/browse/KAFKA-18101 Project: Kafka

Re: [PR] KAFKA-18039: Test all versions of quorum responses in RequestResponseTest [kafka]

2024-11-26 Thread via GitHub
peterxcli commented on code in PR #17873: URL: https://github.com/apache/kafka/pull/17873#discussion_r1859783457 ## clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java: ## @@ -1646,9 +1646,38 @@ private DescribeQuorumRequest createDescribeQuorumReque

Re: [PR] KAFKA-18081: Remove isKRaftTest from the kraft-only tests [kafka]

2024-11-26 Thread via GitHub
Yunyung commented on code in PR #17934: URL: https://github.com/apache/kafka/pull/17934#discussion_r1859769745 ## core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala: ## @@ -387,9 +387,7 @@ class ControllerMutationQuotaTest extends BaseRequestTest { priv

Re: [PR] KAFKA-17010: Remove `DescribeLogDirsResponse#ReplicaInfo` [kafka]

2024-11-26 Thread via GitHub
chiacyu commented on PR #17953: URL: https://github.com/apache/kafka/pull/17953#issuecomment-2502544926 Hi, @chia7712 Please take a look if you're available, thanks! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use

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

2024-11-26 Thread via GitHub
TaiJuWu commented on code in PR #17946: URL: https://github.com/apache/kafka/pull/17946#discussion_r1859537038 ## clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java: ## @@ -1223,6 +1223,11 @@ private void ensureValidRecordSize(int size) { */ @O

Re: [PR] KAFKA-14563: RemoveClient-Side AddPartitionsToTxn Requests [kafka]

2024-11-26 Thread via GitHub
artemlivshits commented on code in PR #17698: URL: https://github.com/apache/kafka/pull/17698#discussion_r1859632802 ## core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala: ## @@ -525,6 +558,7 @@ class TransactionCoordinator(txnConfig: TransactionConfi

Re: [PR] KAFKA-16640: Replace TestUtils#resource by scala.util.Using [kafka]

2024-11-26 Thread via GitHub
ijuma commented on PR #15881: URL: https://github.com/apache/kafka/pull/15881#issuecomment-2502412144 > It looks like this change is silently suppressing all errors in tests that were updated, since we don't check the resulting Try for failures. Ouch! A bunch of tests were broken as a

Re: [PR] KAFKA-14563: RemoveClient-Side AddPartitionsToTxn Requests [kafka]

2024-11-26 Thread via GitHub
CalvinConfluent commented on code in PR #17698: URL: https://github.com/apache/kafka/pull/17698#discussion_r1859536529 ## core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala: ## @@ -564,41 +627,44 @@ class TransactionCoordinator(txnConfig: TransactionC

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

2024-11-26 Thread via GitHub
TaiJuWu commented on code in PR #17946: URL: https://github.com/apache/kafka/pull/17946#discussion_r1859537038 ## clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java: ## @@ -1223,6 +1223,11 @@ private void ensureValidRecordSize(int size) { */ @O

Re: [PR] KAFKA-14563: RemoveClient-Side AddPartitionsToTxn Requests [kafka]

2024-11-26 Thread via GitHub
CalvinConfluent commented on code in PR #17698: URL: https://github.com/apache/kafka/pull/17698#discussion_r1859537198 ## core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala: ## @@ -564,41 +627,44 @@ class TransactionCoordinator(txnConfig: TransactionC

Re: [PR] KAFKA-14563: RemoveClient-Side AddPartitionsToTxn Requests [kafka]

2024-11-26 Thread via GitHub
CalvinConfluent commented on code in PR #17698: URL: https://github.com/apache/kafka/pull/17698#discussion_r1859530416 ## core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala: ## @@ -505,20 +579,68 @@ class TransactionCoordinatorTest { ve

Re: [PR] KAFKA-14563: RemoveClient-Side AddPartitionsToTxn Requests [kafka]

2024-11-26 Thread via GitHub
CalvinConfluent commented on code in PR #17698: URL: https://github.com/apache/kafka/pull/17698#discussion_r1859529641 ## clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java: ## @@ -3036,12 +3178,10 @@ public void testEpochUpdateAfterBu

[PR] KAFKA-17010: Remove `DescribeLogDirsResponse#ReplicaInfo` [kafka]

2024-11-26 Thread via GitHub
chiacyu opened a new pull request, #17953: URL: https://github.com/apache/kafka/pull/17953 The `ReplicaInfo` with its related class has been depreciated and would be removed in this pr, please check [KAFKA-17010](https://issues.apache.org/jira/browse/KAFKA-17010) for more details. thanks!

Re: [PR] KAFKA-12844: KIP-740 follow up: clean up TaskId [kafka]

2024-11-26 Thread via GitHub
mjsax commented on PR #17904: URL: https://github.com/apache/kafka/pull/17904#issuecomment-2502383408 Thanks for the PR. Merged to `trunk`. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the spe

Re: [PR] KAFKA-12844: KIP-740 follow up: clean up TaskId [kafka]

2024-11-26 Thread via GitHub
mjsax merged PR #17904: URL: https://github.com/apache/kafka/pull/17904 -- 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] MINOR: Remove duplicate valid value in document [kafka]

2024-11-26 Thread via GitHub
m1a2st commented on PR #17947: URL: https://github.com/apache/kafka/pull/17947#issuecomment-2502373251 Hello @chia7712, there are changed configs in my local ![CleanShot 2024-11-27 at 08 52 28@2x](https://github.com/user-attachments/assets/8e0d169e-56eb-47f5-8849-f798a173bdba) ![CleanS

[jira] [Created] (KAFKA-18100) `Using` block suppresses all errors

2024-11-26 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-18100: -- Summary: `Using` block suppresses all errors Key: KAFKA-18100 URL: https://issues.apache.org/jira/browse/KAFKA-18100 Project: Kafka Issue Type: Bug

Re: [PR] KAFKA-16640: Replace TestUtils#resource by scala.util.Using [kafka]

2024-11-26 Thread via GitHub
jolshan commented on PR #15881: URL: https://github.com/apache/kafka/pull/15881#issuecomment-2502363433 https://issues.apache.org/jira/browse/KAFKA-18100 -- 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

Re: [PR] KAFKA-16758: Extend Consumer#close with an option to leave the group or not [kafka]

2024-11-26 Thread via GitHub
TaiJuWu commented on code in PR #17614: URL: https://github.com/apache/kafka/pull/17614#discussion_r1859485006 ## clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java: ## @@ -267,13 +269,92 @@ public interface Consumer extends Closeable { void close();

Re: [PR] KAFKA-16640: Replace TestUtils#resource by scala.util.Using [kafka]

2024-11-26 Thread via GitHub
jolshan commented on PR #15881: URL: https://github.com/apache/kafka/pull/15881#issuecomment-2502357138 @chia7712 I will file a ticket for this. I just tested by including a assertEquals(1, 2) in a `Using` block and the test still passed. -- This is an automated message from the Apache Gi

Re: [PR] KAFKA-17299: Fix Kafka streams consumer hang issue [kafka]

2024-11-26 Thread via GitHub
mjsax commented on PR #17899: URL: https://github.com/apache/kafka/pull/17899#issuecomment-2502352580 Well, both fixes together are necessary... (but yes, wit the `<=` fix only, we could still build up an incorrect overcount over time...) -- This is an automated message from the Apache Gi

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

2024-11-26 Thread via GitHub
mjsax commented on PR #17882: URL: https://github.com/apache/kafka/pull/17882#issuecomment-2502349548 Looking into the test code, it seem the `s` just goes somewhere else? There is `shouldNotAllowNullStoreNameOnProcess` and `shouldNotAllowNullStoreName[s]OnProcess`, one taking `(String) nul

[jira] [Assigned] (KAFKA-18088) Add consumer metric about paused partitions

2024-11-26 Thread PoAn Yang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18088?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] PoAn Yang reassigned KAFKA-18088: - Assignee: PoAn Yang > Add consumer metric about paused partitions > ---

[jira] [Commented] (KAFKA-18088) Add consumer metric about paused partitions

2024-11-26 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18088?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901343#comment-17901343 ] Matthias J. Sax commented on KAFKA-18088: - Feel free to pick it up. > Add consu

[PR] Kafka-16540 set up the min ISR configs if ELR is enabled. [kafka]

2024-11-26 Thread via GitHub
CalvinConfluent opened a new pull request, #17952: URL: https://github.com/apache/kafka/pull/17952 (no comment) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscri

Re: [PR] KAFKA-14563: RemoveClient-Side AddPartitionsToTxn Requests [kafka]

2024-11-26 Thread via GitHub
jolshan commented on code in PR #17698: URL: https://github.com/apache/kafka/pull/17698#discussion_r1859417510 ## core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala: ## @@ -564,41 +627,44 @@ class TransactionCoordinator(txnConfig: TransactionConfig,

Re: [PR] KAFKA-14563: RemoveClient-Side AddPartitionsToTxn Requests [kafka]

2024-11-26 Thread via GitHub
jolshan commented on code in PR #17698: URL: https://github.com/apache/kafka/pull/17698#discussion_r1859415059 ## core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala: ## @@ -564,41 +627,44 @@ class TransactionCoordinator(txnConfig: TransactionConfig,

Re: [PR] KAFKA-14563: RemoveClient-Side AddPartitionsToTxn Requests [kafka]

2024-11-26 Thread via GitHub
jolshan commented on code in PR #17698: URL: https://github.com/apache/kafka/pull/17698#discussion_r1859415059 ## core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala: ## @@ -564,41 +627,44 @@ class TransactionCoordinator(txnConfig: TransactionConfig,

Re: [PR] KAFKA-14563: RemoveClient-Side AddPartitionsToTxn Requests [kafka]

2024-11-26 Thread via GitHub
jolshan commented on code in PR #17698: URL: https://github.com/apache/kafka/pull/17698#discussion_r1859401198 ## core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala: ## @@ -505,20 +579,68 @@ class TransactionCoordinatorTest { verify(tra

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

2024-11-26 Thread via GitHub
kirktrue commented on code in PR #16899: URL: https://github.com/apache/kafka/pull/16899#discussion_r1859396777 ## clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java: ## @@ -713,9 +734,19 @@ private void maybeOverrideEnableAutoCommit(Map configs) {

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

2024-11-26 Thread via GitHub
kirktrue commented on code in PR #17199: URL: https://github.com/apache/kafka/pull/17199#discussion_r1859369437 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ## @@ -507,7 +511,8 @@ public void onGroupAssignmentUpdated(Set partitio

Re: [PR] KAFKA-14563: RemoveClient-Side AddPartitionsToTxn Requests [kafka]

2024-11-26 Thread via GitHub
jolshan commented on code in PR #17698: URL: https://github.com/apache/kafka/pull/17698#discussion_r1859375050 ## clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java: ## @@ -3036,12 +3178,10 @@ public void testEpochUpdateAfterBumpFromEn

Re: [PR] KAFKA-14563: RemoveClient-Side AddPartitionsToTxn Requests [kafka]

2024-11-26 Thread via GitHub
jolshan commented on code in PR #17698: URL: https://github.com/apache/kafka/pull/17698#discussion_r1859379951 ## clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java: ## @@ -125,7 +126,8 @@ public void testBuildWithCurrentMessageFormat() {

Re: [PR] KAFKA-14563: RemoveClient-Side AddPartitionsToTxn Requests [kafka]

2024-11-26 Thread via GitHub
jolshan commented on code in PR #17698: URL: https://github.com/apache/kafka/pull/17698#discussion_r1859362179 ## clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java: ## @@ -922,6 +946,103 @@ public void testTransactionManagerEnablesV2()

[jira] [Commented] (KAFKA-18034) CommitRequestManager should fail pending requests on fatal coordinator errors

2024-11-26 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18034?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901325#comment-17901325 ] Kirk True commented on KAFKA-18034: --- Thanks [~m1a2st]! > CommitRequestManager should

Re: [PR] KAFKA-14563: RemoveClient-Side AddPartitionsToTxn Requests [kafka]

2024-11-26 Thread via GitHub
jolshan commented on code in PR #17698: URL: https://github.com/apache/kafka/pull/17698#discussion_r1859352623 ## clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java: ## @@ -172,7 +172,11 @@ private void initializeTransactionManager(Opti

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

2024-11-26 Thread via GitHub
kirktrue commented on code in PR #17440: URL: https://github.com/apache/kafka/pull/17440#discussion_r1859304634 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java: ## @@ -168,6 +171,8 @@ private void processApplicationEvents() {

Re: [PR] KAFKA-18062: use feature version to enable ELR [kafka]

2024-11-26 Thread via GitHub
cmccabe merged PR #17867: URL: https://github.com/apache/kafka/pull/17867 -- 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-17997: Remove deprecated config log.message.timestamp.difference.max.ms [kafka]

2024-11-26 Thread via GitHub
HyunSangHan commented on PR #17928: URL: https://github.com/apache/kafka/pull/17928#issuecomment-2502021527 @chia7712 Thank you for feedback! I removed it. 0de0768 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use th

[PR] KAFKA-15561 [3/N]: Client support for SubscriptionPattern in HB [kafka]

2024-11-26 Thread via GitHub
lianetm opened a new pull request, #17951: URL: https://github.com/apache/kafka/pull/17951 Include regex from SubscriptionPattern in HB and handle related errors. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use th

[jira] [Created] (KAFKA-18099) The lkc prefix should be removed from the topic name when generating sync response for classic members in consumer group

2024-11-26 Thread Dongnuo Lyu (Jira)
Dongnuo Lyu created KAFKA-18099: --- Summary: The lkc prefix should be removed from the topic name when generating sync response for classic members in consumer group Key: KAFKA-18099 URL: https://issues.apache.org/jir

[jira] [Commented] (KAFKA-18084) Null and leaked AcquisitionLockTimerTask causes hanging AcknowledgeRequest and corrupted state of batch

2024-11-26 Thread Apoorv Mittal (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901301#comment-17901301 ] Apoorv Mittal commented on KAFKA-18084: --- [~chia7712] Thanks for flagging it and an

Re: [PR] KAFKA-16640: Replace TestUtils#resource by scala.util.Using [kafka]

2024-11-26 Thread via GitHub
xvrl commented on PR #15881: URL: https://github.com/apache/kafka/pull/15881#issuecomment-2501874334 it looks like this change is silently suppressing all errors in tests that were updated, since we don't check the resulting Try for failures. -- This is an automated message from the Apach

[PR] KAFKA-7025: Some Android compatibility for kafka-client [kafka]

2024-11-26 Thread via GitHub
gtbX opened a new pull request, #17950: URL: https://github.com/apache/kafka/pull/17950 Fixes and workarounds for some of the peculiarities of the Android runtime. This is by no means complete Android support, but gets a little bit closer. * fix for `Socket.getInetAddress()` returning nul

Re: [PR] KAFKA-17757: Remove Utils.mkEntry [kafka]

2024-11-26 Thread via GitHub
mingyen066 commented on PR #17488: URL: https://github.com/apache/kafka/pull/17488#issuecomment-2501787816 Hi @chia7712, sorry for the late reply. I've migrated all the mkEntry usages to Map.entry, except for those used to create entries with null values. -- This is an automated message f

Re: [PR] KAFKA-17757: Remove Utils.mkEntry [kafka]

2024-11-26 Thread via GitHub
mingyen066 commented on code in PR #17488: URL: https://github.com/apache/kafka/pull/17488#discussion_r1859137211 ## connect/runtime/src/test/java/org/apache/kafka/connect/storage/ConnectorOffsetBackingStoreTest.java: ## @@ -94,8 +93,8 @@ public void testFlushFailureWhenWriteTo

Re: [PR] KAFKA-14563: RemoveClient-Side AddPartitionsToTxn Requests [kafka]

2024-11-26 Thread via GitHub
jolshan commented on code in PR #17698: URL: https://github.com/apache/kafka/pull/17698#discussion_r1859110614 ## clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java: ## @@ -1672,6 +1682,11 @@ private TxnOffsetCommitHandler(TransactionalReq

Re: [PR] KAFKA-18039: Test all versions of quorum responses in RequestResponseTest [kafka]

2024-11-26 Thread via GitHub
ahuang98 commented on code in PR #17873: URL: https://github.com/apache/kafka/pull/17873#discussion_r1859102483 ## clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java: ## @@ -1646,9 +1646,38 @@ private DescribeQuorumRequest createDescribeQuorumReques

Re: [PR] KAFKA-17915: Convert Kafka Client system tests to use KRaft [kafka]

2024-11-26 Thread via GitHub
kirktrue commented on PR #17669: URL: https://github.com/apache/kafka/pull/17669#issuecomment-2501738430 > @kirktrue Could you please run the related E2E tests on your local setup? @chia7712—I've attached links to the test runs in the main description. -- This is an automated messag

[jira] [Commented] (KAFKA-18084) Null and leaked AcquisitionLockTimerTask causes hanging AcknowledgeRequest and corrupted state of batch

2024-11-26 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901289#comment-17901289 ] Chia-Ping Tsai commented on KAFKA-18084: {quote} I wonder whether there's a simi

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

2024-11-26 Thread via GitHub
chia7712 commented on PR #17905: URL: https://github.com/apache/kafka/pull/17905#issuecomment-2501701718 or the test can be disabled temporarily and I will file a jira to trace it -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHu

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

2024-11-26 Thread via GitHub
chia7712 commented on PR #17905: URL: https://github.com/apache/kafka/pull/17905#issuecomment-2501694869 @cmccabe Could you please check the failed tests? `DynamicBrokerReconfigurationTest#testUncleanLeaderElectionEnable` -- This is an automated message from the Apache Git Service. To res

[PR] MINOR: remove zk from several tests [kafka]

2024-11-26 Thread via GitHub
cmccabe opened a new pull request, #17949: URL: https://github.com/apache/kafka/pull/17949 Remove ZK from - core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala - core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala - core/src/test/scala/unit/kafka/server/

Re: [PR] MINOR: fix warnings in Kafka Streams state store tests [kafka]

2024-11-26 Thread via GitHub
chia7712 merged PR #17855: URL: https://github.com/apache/kafka/pull/17855 -- 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-18068: Fixing typo in ProducerConfig [kafka]

2024-11-26 Thread via GitHub
AndrewJSchofield commented on PR #17908: URL: https://github.com/apache/kafka/pull/17908#issuecomment-2501575544 @chia7712 is correct. It would need a KIP too, which would be a trivial KIP so not much effort. -- This is an automated message from the Apache Git Service. To respond to the m

Re: [PR] KAFKA-18068: Fixing typo in ProducerConfig [kafka]

2024-11-26 Thread via GitHub
AndrewJSchofield commented on code in PR #17908: URL: https://github.com/apache/kafka/pull/17908#discussion_r1858989261 ## clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java: ## @@ -97,17 +97,17 @@ public class ProducerConfig extends AbstractConfig {

[jira] [Resolved] (KAFKA-18028) the effective kraft version of `--no-initial-controllers` should be 1 rather than 0

2024-11-26 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18028?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-18028. Fix Version/s: 4.0.0 Resolution: Fixed > the effective kraft version of `--no-initi

Re: [PR] KAFKA-18028: the effective kraft version of --no-initial-controllers should be 1 rather than 0 [kafka]

2024-11-26 Thread via GitHub
chia7712 merged PR #17836: URL: https://github.com/apache/kafka/pull/17836 -- 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] [Commented] (KAFKA-18098) add kraft support to testReplicaPlacementAllServers and testReplicaPlacementPartialServers

2024-11-26 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901271#comment-17901271 ] Chia-Ping Tsai commented on KAFKA-18098: [~peterxcli] You are the one working on

[jira] [Assigned] (KAFKA-18098) add kraft support to testReplicaPlacementAllServers and testReplicaPlacementPartialServers

2024-11-26 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18098?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai reassigned KAFKA-18098: -- Assignee: Peter Lee (was: Chia-Ping Tsai) > add kraft support to testReplicaPlacemen

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

2024-11-26 Thread via GitHub
AndrewJSchofield commented on code in PR #17946: URL: https://github.com/apache/kafka/pull/17946#discussion_r1858977786 ## clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java: ## @@ -1223,6 +1223,11 @@ private void ensureValidRecordSize(int size) { *

[jira] [Commented] (KAFKA-18098) add kraft support to testReplicaPlacementAllServers and testReplicaPlacementPartialServers

2024-11-26 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901270#comment-17901270 ] Chia-Ping Tsai commented on KAFKA-18098: [~peterxcli] Sorry, we're currently wor

[jira] [Commented] (KAFKA-18098) add kraft support to testReplicaPlacementAllServers and testReplicaPlacementPartialServers

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

[jira] [Created] (KAFKA-18098) add kraft support to testReplicaPlacementAllServers and testReplicaPlacementPartialServers

2024-11-26 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-18098: -- Summary: add kraft support to testReplicaPlacementAllServers and testReplicaPlacementPartialServers Key: KAFKA-18098 URL: https://issues.apache.org/jira/browse/KAFKA-18098

[jira] [Commented] (KAFKA-18089) RemoteIndexCacheTest fails with caffeine > 3.1.1

2024-11-26 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18089?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17901266#comment-17901266 ] Chia-Ping Tsai commented on KAFKA-18089: the explanation of related change of 3.

[jira] [Resolved] (KAFKA-18049) Upgrade the caffeine version to 3.1.1

2024-11-26 Thread Mickael Maison (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18049?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mickael Maison resolved KAFKA-18049. Fix Version/s: 4.0.0 Resolution: Fixed > Upgrade the caffeine version to 3.1.1 > --

Re: [PR] KAFKA-18049 Upgrade the caffeine version to 3.1.1 [kafka]

2024-11-26 Thread via GitHub
mimaison merged PR #17879: URL: https://github.com/apache/kafka/pull/17879 -- 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-17593; [8/N] Resolve regular expressions [kafka]

2024-11-26 Thread via GitHub
dajac merged PR #17864: URL: https://github.com/apache/kafka/pull/17864 -- 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-18081: Remove isKRaftTest from the kraft-only tests [kafka]

2024-11-26 Thread via GitHub
chia7712 commented on code in PR #17934: URL: https://github.com/apache/kafka/pull/17934#discussion_r1858862670 ## core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala: ## @@ -387,9 +387,7 @@ class ControllerMutationQuotaTest extends BaseRequestTest { pri

Re: [PR] MINOR: Remove duplicate valid value in document [kafka]

2024-11-26 Thread via GitHub
chia7712 commented on PR #17947: URL: https://github.com/apache/kafka/pull/17947#issuecomment-2501310869 @m1a2st Could you please generate the output to display the change? -- 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-17979: Change [pytest] to [tool:pytest] in setup.cfg file [kafka]

2024-11-26 Thread via GitHub
chia7712 commented on code in PR #17740: URL: https://github.com/apache/kafka/pull/17740#discussion_r1858850116 ## tests/setup.py: ## @@ -15,27 +15,24 @@ import re import sys -from setuptools import find_packages, setup -from setuptools.command.test import test as TestComman

[jira] [Resolved] (KAFKA-18083) ClusterInstance custom controllerListener not work

2024-11-26 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18083?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-18083. Fix Version/s: 4.0.0 Resolution: Fixed > ClusterInstance custom controllerListener

Re: [PR] KAFKA-16437 - Upgrade to Jakarta and Jetty 12 (KIP-1032) [kafka]

2024-11-26 Thread via GitHub
cshannon commented on PR #16754: URL: https://github.com/apache/kafka/pull/16754#issuecomment-2501289599 @chia7712 - I merged the latest trunk and fixed the conflicts so this PR is now up to date -- This is an automated message from the Apache Git Service. To respond to the message, pleas

Re: [PR] KAFKA-18092: Mark testBumpTransactionalEpochWithTV2Enabled as flaky [kafka]

2024-11-26 Thread via GitHub
lianetm merged PR #17945: URL: https://github.com/apache/kafka/pull/17945 -- 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-18097: Upgrade readme to include min JDK changes [kafka]

2024-11-26 Thread via GitHub
chia7712 commented on code in PR #17948: URL: https://github.com/apache/kafka/pull/17948#discussion_r1858797408 ## README.md: ## @@ -4,10 +4,10 @@ See our [web site](https://kafka.apache.org) for details on the project. You need to have [Java](http://www.oracle.com/technetw

Re: [PR] KAFKA-17299: Fix Kafka streams consumer hang issue [kafka]

2024-11-26 Thread via GitHub
laxman-ch commented on PR #17899: URL: https://github.com/apache/kafka/pull/17899#issuecomment-2501242346 Thanks @mjsax. Your followup PR fixes the root cause (accounting problem). -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitH

[PR] KAFKA-18097: Upgrade readme to include min JDK changes [kafka]

2024-11-26 Thread via GitHub
frankvicky opened a new pull request, #17948: URL: https://github.com/apache/kafka/pull/17948 JIRA: KAFKA-18097 Since we have applied the KIP-1013, we should also update the README to reflect the change. ### Committer Checklist (excluded from commit message) - [ ] Verify des

[jira] [Resolved] (KAFKA-17569) Rewrite TestLinearWriteSpeed by JMH

2024-11-26 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17569?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-17569. Fix Version/s: 4.0.0 Resolution: Fixed > Rewrite TestLinearWriteSpeed by JMH >

[jira] [Resolved] (KAFKA-17566) What is the next about other.kafka

2024-11-26 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17566?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-17566. Fix Version/s: 4.0.0 Resolution: Fixed > What is the next about other.kafka > -

Re: [PR] KAFKA-17569: Rewrite TestLinearWriteSpeed by Java [kafka]

2024-11-26 Thread via GitHub
chia7712 merged PR #17736: URL: https://github.com/apache/kafka/pull/17736 -- 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] Integrate Streams membership manager with stream thread [kafka]

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

  1   2   >