Re: [PR] KAFKA-17400: Added share fetch purgatory for delaying share fetch requests which cannot be completed because of reaching record lock partition limit [kafka]

2024-08-30 Thread via GitHub
adixitconfluent commented on code in PR #16969: URL: https://github.com/apache/kafka/pull/16969#discussion_r1739617917 ## server/src/main/java/org/apache/kafka/server/config/ShareGroupConfig.java: ## @@ -63,14 +63,19 @@ public class ShareGroupConfig { public static final in

Re: [PR] KAFKA-17400: Added share fetch purgatory for delaying share fetch requests which cannot be completed because of reaching record lock partition limit [kafka]

2024-08-30 Thread via GitHub
adixitconfluent commented on code in PR #16969: URL: https://github.com/apache/kafka/pull/16969#discussion_r1739613663 ## core/src/main/java/kafka/server/share/DelayedShareFetch.java: ## @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more

Re: [PR] KAFKA-17400: Added share fetch purgatory for delaying share fetch requests which cannot be completed because of reaching record lock partition limit [kafka]

2024-08-30 Thread via GitHub
adixitconfluent commented on code in PR #16969: URL: https://github.com/apache/kafka/pull/16969#discussion_r1739613663 ## core/src/main/java/kafka/server/share/DelayedShareFetch.java: ## @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more

Re: [PR] MINOR: add helper function for clusterInstance [kafka]

2024-08-30 Thread via GitHub
TaiJuWu commented on code in PR #16852: URL: https://github.com/apache/kafka/pull/16852#discussion_r1739610420 ## core/src/test/java/kafka/test/ClusterTestExtensionsTest.java: ## @@ -236,6 +236,20 @@ public void testNotSupportedNewGroupProtocols(ClusterInstance clusterInstance)

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

2024-08-30 Thread via GitHub
TaiJuWu closed pull request #16703: KAFKA-17209: Revisit testCurrentLag for asyncConsumer URL: https://github.com/apache/kafka/pull/16703 -- 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

[jira] [Commented] (KAFKA-16792) Enable consumer unit tests that fail to fetch offsets only for new consumer with poll(0)

2024-08-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16792?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17878298#comment-17878298 ] Chia-Ping Tsai commented on KAFKA-16792: {quote} With that we would guarantee t

Re: [PR] KAFKA-17137[part-5]: Ensure Admin APIs are properly tested [kafka]

2024-08-30 Thread via GitHub
TaiJuWu commented on code in PR #16905: URL: https://github.com/apache/kafka/pull/16905#discussion_r1739606119 ## core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala: ## @@ -337,6 +338,53 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationT

Re: [PR] KAFKA-17137[part-5]: Ensure Admin APIs are properly tested [kafka]

2024-08-30 Thread via GitHub
TaiJuWu commented on code in PR #16905: URL: https://github.com/apache/kafka/pull/16905#discussion_r1739606015 ## core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala: ## @@ -337,6 +338,53 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationT

[jira] [Resolved] (KAFKA-17449) Move Quota classes to server-common module

2024-08-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17449?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-17449. Fix Version/s: 4.0.0 Resolution: Fixed > Move Quota classes to server-common module

Re: [PR] KAFKA-17449: Move Quota classes to server-common module [kafka]

2024-08-30 Thread via GitHub
chia7712 merged PR #17060: URL: https://github.com/apache/kafka/pull/17060 -- 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-17454) Fix failed transactions_mixed_versions_test.py when running with 3.2

2024-08-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17454?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17878293#comment-17878293 ] Chia-Ping Tsai commented on KAFKA-17454: [~frankvicky] thanks for taking over th

[jira] [Assigned] (KAFKA-17454) Fix failed transactions_mixed_versions_test.py when running with 3.2

2024-08-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17454?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai reassigned KAFKA-17454: -- Assignee: TengYao Chi (was: Chia-Ping Tsai) > Fix failed transactions_mixed_versions

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

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

Re: [PR] KAFKA-17137: Feat admin client it acl configs [kafka]

2024-08-30 Thread via GitHub
chia7712 merged PR #16648: URL: https://github.com/apache/kafka/pull/16648 -- 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] [Assigned] (KAFKA-17451) Remove deprecated Consumer#committed

2024-08-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17451?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai reassigned KAFKA-17451: -- Assignee: Ming-Yen Chung (was: Chia-Ping Tsai) > Remove deprecated Consumer#committe

[jira] [Commented] (KAFKA-17454) Fix failed transactions_mixed_versions_test.py when running with 3.2

2024-08-30 Thread TengYao Chi (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17454?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17878292#comment-17878292 ] TengYao Chi commented on KAFKA-17454: - Hi [~chia7712]  I would like to give it a try

[jira] [Commented] (KAFKA-17454) Fix failed transactions_mixed_versions_test.py when running with 3.2

2024-08-30 Thread TengYao Chi (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17454?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17878291#comment-17878291 ] TengYao Chi commented on KAFKA-17454: - Hi [~jolshan] and [~chia7712]  I could reprod

Re: [PR] MINOR: add concurrent test for consumer.poll [kafka]

2024-08-30 Thread via GitHub
TaiJuWu commented on code in PR #17047: URL: https://github.com/apache/kafka/pull/17047#discussion_r1739597756 ## clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java: ## @@ -3382,7 +3382,43 @@ public void testCommittedThrowsTimeoutExceptionForNoRespon

[jira] [Commented] (KAFKA-17454) Fix failed transactions_mixed_versions_test.py when running with 3.2

2024-08-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17454?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17878290#comment-17878290 ] Chia-Ping Tsai commented on KAFKA-17454: [~jolshan] thanks for response {*}{*}{*

[jira] [Resolved] (KAFKA-17433) Add a CI job to run a single test N times

2024-08-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17433?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-17433. Fix Version/s: 4.0.0 Resolution: Fixed > Add a CI job to run a single test N times

Re: [PR] KAFKA-17433 Add a deflake Github action [kafka]

2024-08-30 Thread via GitHub
chia7712 merged PR #17019: URL: https://github.com/apache/kafka/pull/17019 -- 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-16448 Add timestamp to error handler context [kafka]

2024-08-30 Thread via GitHub
mjsax commented on code in PR #17054: URL: https://github.com/apache/kafka/pull/17054#discussion_r1739583131 ## streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java: ## @@ -102,4 +106,23 @@ public interface ErrorHandlerContext { * @return the task

Re: [PR] KAFKA-16379: Coordinator event queue, processing, flush, purgatory time histograms [kafka]

2024-08-30 Thread via GitHub
jeffkbkim commented on code in PR #16949: URL: https://github.com/apache/kafka/pull/16949#discussion_r1739562405 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/HdrHistogram.java: ## @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundati

Re: [PR] KAFKA-17137: Feat admin client it acl configs [kafka]

2024-08-30 Thread via GitHub
unknowntpo commented on code in PR #16648: URL: https://github.com/apache/kafka/pull/16648#discussion_r1739520357 ## core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala: ## @@ -96,6 +96,50 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegra

Re: [PR] KAFKA-17109: Reduce log message load for failed locking [kafka]

2024-08-30 Thread via GitHub
danicafine commented on PR #16705: URL: https://github.com/apache/kafka/pull/16705#issuecomment-2322520138 @cadonna given that the state updater feature is relatively new, I can see why we'd want to keep it at info-level. Happy to do so if @eduwercamacaro has no objections. -- This is an

Re: [PR] KAFKA-17277: [1/2] Add version mapping command to the storage tool and feature command tool [kafka]

2024-08-30 Thread via GitHub
jolshan commented on code in PR #16973: URL: https://github.com/apache/kafka/pull/16973#discussion_r1739491401 ## core/src/test/scala/unit/kafka/tools/StorageToolTest.scala: ## @@ -433,5 +433,95 @@ Found problem: contains("Formatting dynamic metadata voter directory %s".

Re: [PR] KAFKA-17442: Handled persister errors with write async calls (KIP-932) [kafka]

2024-08-30 Thread via GitHub
apoorvmittal10 commented on code in PR #16956: URL: https://github.com/apache/kafka/pull/16956#discussion_r1739487499 ## core/src/test/java/kafka/server/share/SharePartitionTest.java: ## @@ -4191,11 +4337,11 @@ public void testAcknowledgeSubsetWithAnotherMember() {

Re: [PR] Kafka 8850: Updated documentation to clarify fetch.min.bytes behaviour. [kafka]

2024-08-30 Thread via GitHub
lianetm commented on PR #16749: URL: https://github.com/apache/kafka/pull/16749#issuecomment-2322419699 Actually, I would expect the behaviour that the documentation currently had: > If insufficient data is available the request will wait for that much data to accumulate and it

[jira] [Commented] (KAFKA-17454) Fix failed transactions_mixed_versions_test.py when running with 3.2

2024-08-30 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17454?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17878261#comment-17878261 ] Justine Olshan commented on KAFKA-17454: Did we see this fail? I already fixed f

Re: [PR] KAFKA-17439: Make polling for new records an explicit action/event in the new consumer [kafka]

2024-08-30 Thread via GitHub
AndrewJSchofield commented on code in PR #17035: URL: https://github.com/apache/kafka/pull/17035#discussion_r1739422737 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java: ## @@ -159,6 +159,9 @@ private void process(final P

Re: [PR] KAFKA-17411: Create local state Standbys on start [kafka]

2024-08-30 Thread via GitHub
mjsax commented on code in PR #16922: URL: https://github.com/apache/kafka/pull/16922#discussion_r1739425714 ## streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java: ## @@ -519,6 +563,33 @@ private void handleTasksPendingInitialization() {

Re: [PR] KAFKA-17447: Changed fetch queue processing to reduce the no. of locking and unlocking activity [kafka]

2024-08-30 Thread via GitHub
AndrewJSchofield commented on code in PR #17055: URL: https://github.com/apache/kafka/pull/17055#discussion_r1739412240 ## core/src/main/java/kafka/server/share/SharePartitionManager.java: ## @@ -683,19 +683,22 @@ CompletableFuture> processFetchResponse( } // Visibl

[jira] [Commented] (KAFKA-17455) `TaskCorruptedException` After Client Quota Throttling

2024-08-30 Thread Colt McNealy (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17455?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17878249#comment-17878249 ] Colt McNealy commented on KAFKA-17455: -- This is probably a problem in the Producer

Re: [PR] KAFKA-12823 Remove Deprecated method KStream#through [kafka]

2024-08-30 Thread via GitHub
ardada2468 commented on code in PR #16761: URL: https://github.com/apache/kafka/pull/16761#discussion_r1739415826 ## tools/src/main/java/org/apache/kafka/tools/StreamsResetter.java: ## @@ -584,8 +584,8 @@ public StreamsResetterOptions(String[] args) { .ofType(St

Re: [PR] KAFKA-16379: Coordinator event queue, processing, flush, purgatory time histograms [kafka]

2024-08-30 Thread via GitHub
junrao commented on code in PR #16949: URL: https://github.com/apache/kafka/pull/16949#discussion_r1739412072 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/HdrHistogram.java: ## @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation

[jira] [Created] (KAFKA-17455) `TaskCorruptedException` After Client Quota Throttling

2024-08-30 Thread Colt McNealy (Jira)
Colt McNealy created KAFKA-17455: Summary: `TaskCorruptedException` After Client Quota Throttling Key: KAFKA-17455 URL: https://issues.apache.org/jira/browse/KAFKA-17455 Project: Kafka Issue

Re: [PR] KAFKA-17442: Handled persister errors with write async calls (KIP-932) [kafka]

2024-08-30 Thread via GitHub
junrao commented on code in PR #16956: URL: https://github.com/apache/kafka/pull/16956#discussion_r1739394900 ## core/src/main/java/kafka/server/share/SharePartition.java: ## @@ -1599,15 +1655,17 @@ && checkForStartOffsetWithinBatch(inFlightBatch.firstOffset(), inFlightBatch.la

Re: [PR] Kafka 8850: Updated documentation to clarify fetch.min.bytes behaviour. [kafka]

2024-08-30 Thread via GitHub
mjsax commented on PR #16749: URL: https://github.com/apache/kafka/pull/16749#issuecomment-2322298335 \cc @lianetm for 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 specific comment.

Re: [PR] KAFKA-17400: Added share fetch purgatory for delaying share fetch requests which cannot be completed because of reaching record lock partition limit [kafka]

2024-08-30 Thread via GitHub
junrao commented on code in PR #16969: URL: https://github.com/apache/kafka/pull/16969#discussion_r1739387577 ## core/src/main/java/kafka/server/share/SharePartitionManager.java: ## @@ -498,10 +519,17 @@ List cachedTopicIdPartitionsInShareSession(String groupId, Uui re

Re: [PR] KAFKA-17442: Handled persister errors with write async calls (KIP-932) [kafka]

2024-08-30 Thread via GitHub
junrao commented on code in PR #16956: URL: https://github.com/apache/kafka/pull/16956#discussion_r1739379496 ## core/src/test/java/kafka/server/share/SharePartitionTest.java: ## @@ -4191,11 +4337,11 @@ public void testAcknowledgeSubsetWithAnotherMember() { Coll

[PR] Testing 2 [kafka]

2024-08-30 Thread via GitHub
mumrah opened a new pull request, #17062: URL: https://github.com/apache/kafka/pull/17062 (no comment) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mai

Re: [PR] MINOR Handle test re-runs in junit.py [kafka]

2024-08-30 Thread via GitHub
mumrah commented on code in PR #17034: URL: https://github.com/apache/kafka/pull/17034#discussion_r1739299366 ## .github/scripts/junit.py: ## @@ -148,29 +168,73 @@ def pretty_time_duration(seconds: float) -> str: total_failures += suite.failures

[jira] [Updated] (KAFKA-17454) Fix failed transactions_mixed_versions_test.py when running with 3.2

2024-08-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17454?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai updated KAFKA-17454: --- Fix Version/s: 3.9.0 > Fix failed transactions_mixed_versions_test.py when running with 3.2

[jira] [Updated] (KAFKA-17454) Fix failed transactions_mixed_versions_test.py when running with 3.2

2024-08-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17454?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai updated KAFKA-17454: --- Fix Version/s: (was: 3.9.0) > Fix failed transactions_mixed_versions_test.py when runnin

Re: [PR] Update README.md [kafka-merge-queue-sandbox]

2024-08-30 Thread via GitHub
mumrah merged PR #43: URL: https://github.com/apache/kafka-merge-queue-sandbox/pull/43 -- 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...@

[jira] [Updated] (KAFKA-17454) Fix failed transactions_mixed_versions_test.py when running with 3.2

2024-08-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17454?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai updated KAFKA-17454: --- Priority: Blocker (was: Major) > Fix failed transactions_mixed_versions_test.py when runnin

[jira] [Commented] (KAFKA-17454) Fix failed transactions_mixed_versions_test.py when running with 3.2

2024-08-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17454?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17878223#comment-17878223 ] Chia-Ping Tsai commented on KAFKA-17454: noted: the error happens only if the si

[jira] [Updated] (KAFKA-17454) Fix failed transactions_mixed_versions_test.py when running with 3.2

2024-08-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17454?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai updated KAFKA-17454: --- Fix Version/s: 4.0.0 3.9.0 > Fix failed transactions_mixed_versions_test.

[PR] Update README.md [kafka-merge-queue-sandbox]

2024-08-30 Thread via GitHub
mumrah opened a new pull request, #43: URL: https://github.com/apache/kafka-merge-queue-sandbox/pull/43 (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 unsu

Re: [PR] ISSUE-1014 Should fail in the queue [kafka-merge-queue-sandbox]

2024-08-30 Thread via GitHub
mumrah closed pull request #21: ISSUE-1014 Should fail in the queue URL: https://github.com/apache/kafka-merge-queue-sandbox/pull/21 -- 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 comme

[jira] [Created] (KAFKA-17454) Fix failed transactions_mixed_versions_test.py when running with 3.2

2024-08-30 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-17454: -- Summary: Fix failed transactions_mixed_versions_test.py when running with 3.2 Key: KAFKA-17454 URL: https://issues.apache.org/jira/browse/KAFKA-17454 Project: Kaf

Re: [PR] KAFKA-17400: Added share fetch purgatory for delaying share fetch requests which cannot be completed because of reaching record lock partition limit [kafka]

2024-08-30 Thread via GitHub
junrao commented on code in PR #16969: URL: https://github.com/apache/kafka/pull/16969#discussion_r1739168653 ## core/src/main/java/kafka/server/share/DelayedShareFetch.java: ## @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * cont

[jira] [Comment Edited] (KAFKA-16758) Extend Consumer#close with option to leave the group or not

2024-08-30 Thread Lucas Brutschy (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16758?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17878220#comment-17878220 ] Lucas Brutschy edited comment on KAFKA-16758 at 8/30/24 6:19 PM: -

[jira] [Commented] (KAFKA-16758) Extend Consumer#close with option to leave the group or not

2024-08-30 Thread Lucas Brutschy (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16758?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17878220#comment-17878220 ] Lucas Brutschy commented on KAFKA-16758: Hey lianet, are you still planning to d

[jira] [Commented] (KAFKA-16792) Enable consumer unit tests that fail to fetch offsets only for new consumer with poll(0)

2024-08-30 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16792?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17878191#comment-17878191 ] Lianet Magrans commented on KAFKA-16792: Thinking more about this, I wonder if w

Re: [PR] KAFKA-16792: Enable consumer unit tests that fail to fetch offsets only for new consumer with poll(0) [kafka]

2024-08-30 Thread via GitHub
lianetm commented on code in PR #16982: URL: https://github.com/apache/kafka/pull/16982#discussion_r1739141821 ## clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java: ## @@ -1053,12 +1053,20 @@ public void testFetchStableOffsetThrowInCommitted(GroupPr

Re: [PR] KAFKA-17331: Throw unsupported version exception if the server does NOT support EarliestLocalSpec and LatestTieredSpec [kafka]

2024-08-30 Thread via GitHub
junrao commented on code in PR #16873: URL: https://github.com/apache/kafka/pull/16873#discussion_r1739140459 ## core/src/main/scala/kafka/server/KafkaApis.scala: ## @@ -1104,35 +1104,43 @@ class KafkaApis(val requestChannel: RequestChannel, val responseTopics = authorize

Re: [PR] KAFKA-17331: Throw unsupported version exception if the server does NOT support EarliestLocalSpec and LatestTieredSpec [kafka]

2024-08-30 Thread via GitHub
chia7712 commented on PR #16873: URL: https://github.com/apache/kafka/pull/16873#issuecomment-2321894140 I will merge this PR to trunk and 3.9 tomorrow if no objection -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use th

Re: [PR] KAFKA-17331: Throw unsupported version exception if the server does NOT support EarliestLocalSpec and LatestTieredSpec [kafka]

2024-08-30 Thread via GitHub
chia7712 commented on PR #16873: URL: https://github.com/apache/kafka/pull/16873#issuecomment-2321891055 The failed tests are flaky. the related jira are shown below. https://issues.apache.org/jira/browse/KAFKA-17265 https://issues.apache.org/jira/browse/KAFKA-16174 https://issues.ap

[jira] [Updated] (KAFKA-17453) Fix flaky PlaintextConsumerFetchTest#testFetchOutOfRangeOffsetResetConfigLatest

2024-08-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17453?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai updated KAFKA-17453: --- Description: {code:java} Errorjava.util.NoSuchElementExceptionStacktracejava.util.NoSuchElem

[jira] [Created] (KAFKA-17453) Fix flaky PlaintextConsumerFetchTest#testFetchOutOfRangeOffsetResetConfigLatest

2024-08-30 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-17453: -- Summary: Fix flaky PlaintextConsumerFetchTest#testFetchOutOfRangeOffsetResetConfigLatest Key: KAFKA-17453 URL: https://issues.apache.org/jira/browse/KAFKA-17453 P

[jira] [Created] (KAFKA-17452) Fix flaky QuorumControllerTest#testUncleanShutdownBroker

2024-08-30 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-17452: -- Summary: Fix flaky QuorumControllerTest#testUncleanShutdownBroker Key: KAFKA-17452 URL: https://issues.apache.org/jira/browse/KAFKA-17452 Project: Kafka

Re: [PR] KAFKA-16863 : Deprecate default exception handlers [kafka]

2024-08-30 Thread via GitHub
mjsax commented on PR #17005: URL: https://github.com/apache/kafka/pull/17005#issuecomment-2321839005 Yeah. Looks related. Can you take a look and push a fix? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL abo

[jira] [Commented] (KAFKA-17451) Remove deprecated Consumer#committed

2024-08-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17878180#comment-17878180 ] Chia-Ping Tsai commented on KAFKA-17451: [~javakillah] oh, sorry that my team me

Re: [PR] KAFKA-17137: Feat admin client it acl configs [kafka]

2024-08-30 Thread via GitHub
chia7712 commented on code in PR #16648: URL: https://github.com/apache/kafka/pull/16648#discussion_r1739046762 ## core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala: ## @@ -96,6 +96,50 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrati

[jira] [Commented] (KAFKA-17451) Remove deprecated Consumer#committed

2024-08-30 Thread Dmitry Werner (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17878173#comment-17878173 ] Dmitry Werner commented on KAFKA-17451: --- [~chia7712] Hello, can I take it? > Remo

Re: [PR] KAFKA-17449: Move Quota classes to server-common module [kafka]

2024-08-30 Thread via GitHub
mimaison commented on PR #17060: URL: https://github.com/apache/kafka/pull/17060#issuecomment-2321731110 Yup, just rebased. Let's wait for the CI to run again -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL abo

Re: [PR] Back-port KAFKA-16230 to 3.7 branch [kafka]

2024-08-30 Thread via GitHub
lianetm commented on PR #16951: URL: https://github.com/apache/kafka/pull/16951#issuecomment-2321721217 Hey @kirktrue , took a first look and overall it looks good. Is there a run of the system tests with this change? (agree that failures in PlainTextConsumer are unrelated to this PR). Than

Re: [PR] KAFKA-17449: Move Quota classes to server-common module [kafka]

2024-08-30 Thread via GitHub
chia7712 commented on PR #17060: URL: https://github.com/apache/kafka/pull/17060#issuecomment-2321633915 @mimaison Could you please fix the conflicts :) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to

[jira] [Created] (KAFKA-17451) Remove deprecated Consumer#committed

2024-08-30 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-17451: -- Summary: Remove deprecated Consumer#committed Key: KAFKA-17451 URL: https://issues.apache.org/jira/browse/KAFKA-17451 Project: Kafka Issue Type: Sub-task

[jira] [Updated] (KAFKA-17428) Remote segments stay in DELETE_SEGMENT_STARTED state after RLMCopyTask fails to upload

2024-08-30 Thread Federico Valeri (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17428?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Federico Valeri updated KAFKA-17428: Summary: Remote segments stay in DELETE_SEGMENT_STARTED state after RLMCopyTask fails to u

[jira] [Updated] (KAFKA-17428) Remote segments stay in COPY_SEGMENT_STARTED state after RLMCopyTask fails to upload

2024-08-30 Thread Federico Valeri (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17428?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Federico Valeri updated KAFKA-17428: Summary: Remote segments stay in COPY_SEGMENT_STARTED state after RLMCopyTask fails to upl

[jira] [Updated] (KAFKA-17428) remote segments deleted in RLMCopyTask stays DELETE_SEGMENT_STARTED state

2024-08-30 Thread Federico Valeri (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17428?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Federico Valeri updated KAFKA-17428: Summary: remote segments deleted in RLMCopyTask stays DELETE_SEGMENT_STARTED state (was:

[jira] [Resolved] (KAFKA-15909) Throw error when consumer configured with empty/whitespace-only group.id for LegacyKafkaConsumer

2024-08-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15909?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-15909. Resolution: Fixed > Throw error when consumer configured with empty/whitespace-only group.

Re: [PR] KAFKA-15909: Throw error when consumer configured with empty/whitespace-only group.id for LegacyKafkaConsumer [kafka]

2024-08-30 Thread via GitHub
chia7712 merged PR #16933: URL: https://github.com/apache/kafka/pull/16933 -- 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-17449: Move Quota classes to server-common module [kafka]

2024-08-30 Thread via GitHub
mimaison commented on code in PR #17060: URL: https://github.com/apache/kafka/pull/17060#discussion_r1738925546 ## server-common/src/main/java/org/apache/kafka/server/quota/QuotaUtils.java: ## @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or

Re: [PR] KAFKA-17374: add bootstrap.controller to kafka-reassign-partitions.sh [kafka]

2024-08-30 Thread via GitHub
chia7712 commented on PR #16964: URL: https://github.com/apache/kafka/pull/16964#issuecomment-2321570487 @m1a2st Could you please rebase code to include #16644? -- 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] MINOR: add concurrent test for consumer.poll [kafka]

2024-08-30 Thread via GitHub
chia7712 commented on code in PR #17047: URL: https://github.com/apache/kafka/pull/17047#discussion_r1738898118 ## clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java: ## @@ -3382,7 +3382,43 @@ public void testCommittedThrowsTimeoutExceptionForNoRespo

Re: [PR] KAFKA-17449: Move Quota classes to server-common module [kafka]

2024-08-30 Thread via GitHub
chia7712 commented on code in PR #17060: URL: https://github.com/apache/kafka/pull/17060#discussion_r1738763455 ## server-common/src/main/java/org/apache/kafka/server/quota/QuotaUtils.java: ## @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or

Re: [PR] KAFKA-17137[part-5]: Ensure Admin APIs are properly tested [kafka]

2024-08-30 Thread via GitHub
m1a2st commented on code in PR #16905: URL: https://github.com/apache/kafka/pull/16905#discussion_r1738862139 ## core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala: ## @@ -337,6 +338,53 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTe

Re: [PR] KAFKA-16863 : Deprecate default exception handlers [kafka]

2024-08-30 Thread via GitHub
muralibasani commented on PR #17005: URL: https://github.com/apache/kafka/pull/17005#issuecomment-2321453957 there are failing tests -- 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

Re: [PR] KAFKA-16863 : Deprecate default exception handlers [kafka]

2024-08-30 Thread via GitHub
muralibasani commented on PR #17005: URL: https://github.com/apache/kafka/pull/17005#issuecomment-2321425276 > Committed the fix. > > LGTM now, assuming build passes... Great, thanks. -- This is an automated message from the Apache Git Service. To respond to the message, plea

Re: [PR] KAFKA-17450: Reduced the handlers for handling ShareAcknowledgeResponse. [kafka]

2024-08-30 Thread via GitHub
AndrewJSchofield commented on code in PR #17061: URL: https://github.com/apache/kafka/pull/17061#discussion_r1738762496 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManager.java: ## @@ -613,74 +607,100 @@ private void handleShareAcknowl

[jira] [Resolved] (KAFKA-17386) Remove broker-list, threads, num-fetch-threads in ConsumerPerformance

2024-08-30 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17386?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-17386. Fix Version/s: 4.0.0 Resolution: Fixed > Remove broker-list, threads, num-fetch-thr

Re: [PR] KAFKA-17386: Remove broker-list, threads, num-fetch-threads in ConsumerPerformance [kafka]

2024-08-30 Thread via GitHub
chia7712 merged PR #16983: URL: https://github.com/apache/kafka/pull/16983 -- 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] [Updated] (KAFKA-17449) Move Quota classes to server-common module

2024-08-30 Thread Mickael Maison (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17449?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mickael Maison updated KAFKA-17449: --- Description: RLMQuotaManager which will utlimately move to storage depends on the QuotaType

[jira] [Updated] (KAFKA-17449) Move Quota classes to server-common module

2024-08-30 Thread Mickael Maison (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17449?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mickael Maison updated KAFKA-17449: --- Summary: Move Quota classes to server-common module (was: Move Quota classes to server modu

Re: [PR] KAFKA-17347: Add missing client-metrics option to kafka-configs.sh [kafka]

2024-08-30 Thread via GitHub
AndrewJSchofield commented on PR #17046: URL: https://github.com/apache/kafka/pull/17046#issuecomment-2321352952 Only 3 unit test failures, unrelated to this 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

[PR] KAFKA-17450: Reduced the handlers for handling ShareAcknowledgeResponse. [kafka]

2024-08-30 Thread via GitHub
ShivsundarR opened a new pull request, #17061: URL: https://github.com/apache/kafka/pull/17061 *What* Currently there are 4 handler functions present for handling ShareAcknowledge responses. ShareConsumeRequestManager had an interface and the respective handlers would implement it. Inste

[PR] KAFKA-17449: Move Quota classes to server module [kafka]

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

Re: [PR] MINOR: Set `group.coordinator.rebalance.protocols` to `classic,consumer` by default [kafka]

2024-08-30 Thread via GitHub
chia7712 commented on code in PR #17057: URL: https://github.com/apache/kafka/pull/17057#discussion_r1738713919 ## core/src/main/scala/kafka/server/KafkaConfig.scala: ## @@ -576,19 +576,13 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) throw new

Re: [PR] MINOR: add ReconfigurableQuorumIntegrationTest [kafka]

2024-08-30 Thread via GitHub
chia7712 commented on code in PR #16991: URL: https://github.com/apache/kafka/pull/16991#discussion_r1738650234 ## metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java: ## @@ -401,7 +409,7 @@ void doFormat(BootstrapMetadata bootstrapMetadata) throws Exception

[jira] [Commented] (KAFKA-17448) New consumer seek should update positions in background thread

2024-08-30 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17878124#comment-17878124 ] Lianet Magrans commented on KAFKA-17448: Thanks! Let me know if you have questio

[jira] [Assigned] (KAFKA-17448) New consumer seek should update positions in background thread

2024-08-30 Thread PoAn Yang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17448?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] PoAn Yang reassigned KAFKA-17448: - Assignee: PoAn Yang > New consumer seek should update positions in background thread >

[jira] [Commented] (KAFKA-17448) New consumer seek should update positions in background thread

2024-08-30 Thread PoAn Yang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17878122#comment-17878122 ] PoAn Yang commented on KAFKA-17448: --- Hi [~lianetm], thank you. I can handle it. 👍 > N

[jira] [Created] (KAFKA-17450) Optimise the handler methods in ShareConsumeRequestManager.

2024-08-30 Thread Shivsundar R (Jira)
Shivsundar R created KAFKA-17450: Summary: Optimise the handler methods in ShareConsumeRequestManager. Key: KAFKA-17450 URL: https://issues.apache.org/jira/browse/KAFKA-17450 Project: Kafka

[jira] [Assigned] (KAFKA-17450) Optimise the handler methods in ShareConsumeRequestManager.

2024-08-30 Thread Shivsundar R (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-17450?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shivsundar R reassigned KAFKA-17450: Assignee: Shivsundar R > Optimise the handler methods in ShareConsumeRequestManager. > --

Re: [PR] KAFKA-16379: Coordinator event queue, processing, flush, purgatory time histograms [kafka]

2024-08-30 Thread via GitHub
jeffkbkim commented on PR #16949: URL: https://github.com/apache/kafka/pull/16949#issuecomment-2321260689 The HdrHistogram wrapper implementation (HdrHistogram, KafkaMetricHistogram) was authored by @dimitarndimitrov -- This is an automated message from the Apache Git Service. To respon

Re: [PR] Kafka 12822 2 [kafka]

2024-08-30 Thread via GitHub
pegasas commented on PR #17059: URL: https://github.com/apache/kafka/pull/17059#issuecomment-2321259947 ![image](https://github.com/user-attachments/assets/37b1c76a-11c6-4d19-bd77-3c55f00904c2) CI passed. -- This is an automated message from the Apache Git Service. To respond to th

[jira] [Created] (KAFKA-17449) Move Quota classes to server module

2024-08-30 Thread Mickael Maison (Jira)
Mickael Maison created KAFKA-17449: -- Summary: Move Quota classes to server module Key: KAFKA-17449 URL: https://issues.apache.org/jira/browse/KAFKA-17449 Project: Kafka Issue Type: Sub-task

  1   2   >