[jira] [Created] (KAFKA-16153) kraft_upgrade_test system test is broken

2024-01-17 Thread Mickael Maison (Jira)
Mickael Maison created KAFKA-16153: -- Summary: kraft_upgrade_test system test is broken Key: KAFKA-16153 URL: https://issues.apache.org/jira/browse/KAFKA-16153 Project: Kafka Issue Type: Bug

Re: [PR] KAFKA-15853: Move ClientQuotaManagerConfig outside of core [kafka]

2024-01-17 Thread via GitHub
mimaison commented on code in PR #15159: URL: https://github.com/apache/kafka/pull/15159#discussion_r1454953584 ## checkstyle/import-control-core.xml: ## @@ -94,6 +94,7 @@ + Review Comment: I think we can remove that line too -- This is an automated m

Re: [PR] MINOR Removed unused CommittedOffsetsFile class. [kafka]

2024-01-17 Thread via GitHub
satishd closed pull request #15209: MINOR Removed unused CommittedOffsetsFile class. URL: https://github.com/apache/kafka/pull/15209 -- 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 comm

Re: [PR] KAFKA-15942: Implement ConsumerInterceptors in the async consumer [kafka]

2024-01-17 Thread via GitHub
AndrewJSchofield commented on code in PR #15000: URL: https://github.com/apache/kafka/pull/15000#discussion_r1455084554 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java: ## @@ -367,6 +372,42 @@ public void testAutocommitEnsureOnl

[jira] [Commented] (KAFKA-4759) Add support for subnet masks in SimpleACLAuthorizer

2024-01-17 Thread Szymon Scharmach (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-4759?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17807680#comment-17807680 ] Szymon Scharmach commented on KAFKA-4759: - Updated PR is ready: [https://github.c

[PR] KAFKA-16095: Update list group state type filter to include the states for the new consumer group type [kafka]

2024-01-17 Thread via GitHub
DL1231 opened a new pull request, #15211: URL: https://github.com/apache/kafka/pull/15211 While using —list —state the current accepted values correspond to the classic group type states. This PR include support for the new group type states. ### Committer Checklist (excluded from

Re: [PR] KAFKA-16095: Update list group state type filter to include the states for the new consumer group type [kafka]

2024-01-17 Thread via GitHub
DL1231 commented on PR #15211: URL: https://github.com/apache/kafka/pull/15211#issuecomment-1895572781 @rreddy-22 PTAL. -- 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 u

[jira] [Assigned] (KAFKA-16147) Partition is assigned to two members at the same time

2024-01-17 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16147?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot reassigned KAFKA-16147: --- Assignee: David Jacot > Partition is assigned to two members at the same time > ---

[PR] KAFKA-16147; Partition is assigned to two members at the same time [kafka]

2024-01-17 Thread via GitHub
dajac opened a new pull request, #15212: URL: https://github.com/apache/kafka/pull/15212 We had a case where a partition got assigned to two members and we found a bug in the partition epochs bookkeeping. Basically, when a member has a partition pending revocation re-assigned to him before

Re: [PR] KAFKA-14511: extend AlterIncrementalConfigs API to support group config [kafka]

2024-01-17 Thread via GitHub
DL1231 commented on PR #15067: URL: https://github.com/apache/kafka/pull/15067#issuecomment-1895664514 @dajac, PTAL, thanks in advance. -- 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 specifi

[jira] [Created] (KAFKA-16154) Make broker changes to return an offset for LATEST_TIERED_TIMESTAMP

2024-01-17 Thread Christo Lolov (Jira)
Christo Lolov created KAFKA-16154: - Summary: Make broker changes to return an offset for LATEST_TIERED_TIMESTAMP Key: KAFKA-16154 URL: https://issues.apache.org/jira/browse/KAFKA-16154 Project: Kafka

[PR] KAFKA-16154: Broker returns offset for LATEST_TIERED_TIMESTAMP [kafka]

2024-01-17 Thread via GitHub
clolov opened a new pull request, #15213: URL: https://github.com/apache/kafka/pull/15213 ### Summary This is the first part of the implementation of [KIP-1005](https://cwiki.apache.org/confluence/display/KAFKA/KIP-1005%3A+Expose+EarliestLocalOffset+and+TieredOffset) The purpos

Re: [PR] KAFKA-16154: Broker returns offset for LATEST_TIERED_TIMESTAMP [kafka]

2024-01-17 Thread via GitHub
clolov commented on code in PR #15213: URL: https://github.com/apache/kafka/pull/15213#discussion_r1455525955 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -1300,18 +1303,27 @@ class UnifiedLog(@volatile var logStartOffset: Long, } else if (targetTimestamp ==

[jira] [Created] (KAFKA-16155) Investigate testAutoCommitIntercept

2024-01-17 Thread Lucas Brutschy (Jira)
Lucas Brutschy created KAFKA-16155: -- Summary: Investigate testAutoCommitIntercept Key: KAFKA-16155 URL: https://issues.apache.org/jira/browse/KAFKA-16155 Project: Kafka Issue Type: Bug

[jira] [Updated] (KAFKA-16155) Investigate testAutoCommitIntercept

2024-01-17 Thread Lucas Brutschy (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16155?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lucas Brutschy updated KAFKA-16155: --- Description: Even with KAFKA-15942, the test PlaintextConsumerTest.testAutoCommitIntercept

Re: [PR] KAFKA-15853: Move ClientQuotaManagerConfig outside of core [kafka]

2024-01-17 Thread via GitHub
OmniaGM commented on code in PR #15159: URL: https://github.com/apache/kafka/pull/15159#discussion_r1455645909 ## checkstyle/import-control-core.xml: ## @@ -94,6 +94,7 @@ + Review Comment: remove it -- This is an automated message from the Apache Git

Re: [PR] KAFKA-15942: Implement ConsumerInterceptors in the async consumer [kafka]

2024-01-17 Thread via GitHub
lucasbru commented on code in PR #15000: URL: https://github.com/apache/kafka/pull/15000#discussion_r1455660640 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java: ## @@ -367,6 +372,42 @@ public void testAutocommitEnsureOnlyOneInfl

Re: [PR] KAFKA-15942: Implement ConsumerInterceptors in the async consumer [kafka]

2024-01-17 Thread via GitHub
lucasbru commented on code in PR #15000: URL: https://github.com/apache/kafka/pull/15000#discussion_r1455670090 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java: ## @@ -93,9 +94,11 @@ public CommitRequestManager( final S

Re: [PR] KAFKA-15942: Implement ConsumerInterceptors in the async consumer [kafka]

2024-01-17 Thread via GitHub
lucasbru commented on code in PR #15000: URL: https://github.com/apache/kafka/pull/15000#discussion_r1455660640 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java: ## @@ -367,6 +372,42 @@ public void testAutocommitEnsureOnlyOneInfl

Re: [PR] KAFKA-15942: Implement ConsumerInterceptors in the async consumer [kafka]

2024-01-17 Thread via GitHub
lucasbru commented on code in PR #15000: URL: https://github.com/apache/kafka/pull/15000#discussion_r1455660640 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java: ## @@ -367,6 +372,42 @@ public void testAutocommitEnsureOnlyOneInfl

Re: [PR] KAFKA-16131: Only update directoryIds if the metadata version supports DirectoryAssignment [kafka]

2024-01-17 Thread via GitHub
pprovenzano commented on code in PR #15197: URL: https://github.com/apache/kafka/pull/15197#discussion_r1455684322 ## metadata/src/main/java/org/apache/kafka/image/FeaturesImage.java: ## @@ -46,6 +46,18 @@ public final class FeaturesImage { ZkMigrationState.NONE );

Re: [PR] KAFKA-16097: Disable state updater in trunk (#15146) [kafka]

2024-01-17 Thread via GitHub
lucasbru merged PR #15204: URL: https://github.com/apache/kafka/pull/15204 -- 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] [Created] (KAFKA-16156) System test failing on endOffsets with negative timestamps

2024-01-17 Thread Lianet Magrans (Jira)
Lianet Magrans created KAFKA-16156: -- Summary: System test failing on endOffsets with negative timestamps Key: KAFKA-16156 URL: https://issues.apache.org/jira/browse/KAFKA-16156 Project: Kafka

[jira] [Updated] (KAFKA-16156) System test failing on endOffsets with negative timestamps

2024-01-17 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16156?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans updated KAFKA-16156: --- Description: TransactionalMessageCopier run with 3.7 new consumer fails with "Invalid negat

[jira] [Updated] (KAFKA-16156) System test failing for new consumer on endOffsets with negative timestamps

2024-01-17 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16156?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans updated KAFKA-16156: --- Summary: System test failing for new consumer on endOffsets with negative timestamps (was:

Re: [PR] KAFKA-15942: Implement ConsumerInterceptors in the async consumer [kafka]

2024-01-17 Thread via GitHub
lucasbru commented on code in PR #15000: URL: https://github.com/apache/kafka/pull/15000#discussion_r1455787900 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java: ## @@ -844,6 +849,54 @@ public void testWakeupCommitted() {

Re: [PR] KAFKA-15942: Implement ConsumerInterceptors in the async consumer [kafka]

2024-01-17 Thread via GitHub
lucasbru commented on code in PR #15000: URL: https://github.com/apache/kafka/pull/15000#discussion_r1455790243 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ## @@ -1902,65 +1912,14 @@ private void maybeThrowFencedInstanceException

[jira] [Commented] (KAFKA-16134) kafka.api.PlaintextConsumerTest.testPerPartitionLagMetricsCleanUpWithSubscribe(String, String).quorum=kraft+kip848.groupProtocol=consumer is flaky

2024-01-17 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17807767#comment-17807767 ] Lianet Magrans commented on KAFKA-16134: Might be related > kafka.api.Plaintext

[jira] [Comment Edited] (KAFKA-16134) kafka.api.PlaintextConsumerTest.testPerPartitionLagMetricsCleanUpWithSubscribe(String, String).quorum=kraft+kip848.groupProtocol=consumer is flaky

2024-01-17 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17807767#comment-17807767 ] Lianet Magrans edited comment on KAFKA-16134 at 1/17/24 3:21 PM: -

[jira] [Commented] (KAFKA-16150) Fix PlaintextConsumerTest.testPerPartitionLagMetricsCleanUpWithSubscribe

2024-01-17 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17807772#comment-17807772 ] Lianet Magrans commented on KAFKA-16150: [~kirktrue] this seems the same as htt

[jira] [Commented] (KAFKA-16151) Fix PlaintextConsumerTest.testPerPartitionLeadMetricsCleanUpWithSubscribe

2024-01-17 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16151?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17807773#comment-17807773 ] Lianet Magrans commented on KAFKA-16151: [~kirktrue] isn't this the same as htt

[PR] KAFKA-16137: Add missing RPC field descriptions [kafka]

2024-01-17 Thread via GitHub
AndrewJSchofield opened a new pull request, #15214: URL: https://github.com/apache/kafka/pull/15214 The `ListClientMetricsResourcesResponse` definition is missing several `"about"` descriptions. The main effect of this is that the Kafka protocol documentation misses the descriptions of thes

[jira] [Updated] (KAFKA-15538) Client support for java regex based subscription

2024-01-17 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15538?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lianet Magrans updated KAFKA-15538: --- Description: When using subscribe with a java regex (Pattern), we need to resolve it on the

[jira] [Commented] (KAFKA-15538) Client support for java regex based subscription

2024-01-17 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15538?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17807783#comment-17807783 ] Lianet Magrans commented on KAFKA-15538: Hey [~phuctran], you're right that the

Re: [PR] KAFKA-16104: Enable additional PlaintextConsumerTest tests for new consumer [kafka]

2024-01-17 Thread via GitHub
kirktrue closed pull request #15206: KAFKA-16104: Enable additional PlaintextConsumerTest tests for new consumer URL: https://github.com/apache/kafka/pull/15206 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL abov

Re: [PR] KAFKA-16104: Enable additional PlaintextConsumerTest tests for new consumer [kafka]

2024-01-17 Thread via GitHub
kirktrue commented on PR #15206: URL: https://github.com/apache/kafka/pull/15206#issuecomment-1896159822 Closing and reopening to kick off another test run -- 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

[jira] [Created] (KAFKA-16157) Topic recreation with offline disk doesn't update leadership/shrink ISR correctly

2024-01-17 Thread Gaurav Narula (Jira)
Gaurav Narula created KAFKA-16157: - Summary: Topic recreation with offline disk doesn't update leadership/shrink ISR correctly Key: KAFKA-16157 URL: https://issues.apache.org/jira/browse/KAFKA-16157 P

[jira] [Updated] (KAFKA-16157) Topic recreation with offline disk doesn't update leadership/shrink ISR correctly

2024-01-17 Thread Gaurav Narula (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16157?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gaurav Narula updated KAFKA-16157: -- Attachment: broker.log broker.log.1 broker.log.2

Re: [PR] KAFKA-14589 ConsumerGroupCommand options and case classes rewritten [kafka]

2024-01-17 Thread via GitHub
mimaison commented on code in PR #14856: URL: https://github.com/apache/kafka/pull/14856#discussion_r1456011084 ## tools/src/main/java/org/apache/kafka/tools/consumergroup/PartitionAssignmentState.java: ## @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF)

[jira] [Commented] (KAFKA-16157) Topic recreation with offline disk doesn't update leadership/shrink ISR correctly

2024-01-17 Thread Mickael Maison (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16157?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17807814#comment-17807814 ] Mickael Maison commented on KAFKA-16157: Did you hit this issue while testing 3.

Re: [PR] KAFKA-15942: Implement ConsumerInterceptors in the async consumer [kafka]

2024-01-17 Thread via GitHub
philipnee commented on code in PR #15000: URL: https://github.com/apache/kafka/pull/15000#discussion_r1456105594 ## core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala: ## @@ -1336,8 +1336,9 @@ class PlaintextConsumerTest extends BaseConsumerTest { MockPro

[jira] [Updated] (KAFKA-16157) Topic recreation with offline disk doesn't update leadership/shrink ISR correctly

2024-01-17 Thread Proven Provenzano (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16157?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Proven Provenzano updated KAFKA-16157: -- Priority: Blocker (was: Major) > Topic recreation with offline disk doesn't update le

[jira] [Updated] (KAFKA-16157) Topic recreation with offline disk doesn't update leadership/shrink ISR correctly

2024-01-17 Thread Gaurav Narula (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16157?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gaurav Narula updated KAFKA-16157: -- Description: In a cluster with 4 brokers, `broker-1..broker-4` with 2 disks `d1` and `d2` in

Re: [PR] KAFKA-16113: Add committed and commit sensor to record metrics [kafka]

2024-01-17 Thread via GitHub
philipnee commented on PR #15210: URL: https://github.com/apache/kafka/pull/15210#issuecomment-1896275067 @lucasbru - Would you have time to review this? Seems like the failed tests aren't necessary related. -- This is an automated message from the Apache Git Service. To respond to the me

[PR] KAFKA-16107: Stop fetching while onPartitionsAssign completes [kafka]

2024-01-17 Thread via GitHub
lianetm opened a new pull request, #15215: URL: https://github.com/apache/kafka/pull/15215 This ensures that no records are fetched in the background thread while the onPartitionsAssigned callback completes running in the Application thread. This is achieved by pausing the partitions before

[jira] [Updated] (KAFKA-16157) Topic recreation with offline disk doesn't update leadership/shrink ISR correctly

2024-01-17 Thread Mickael Maison (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16157?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mickael Maison updated KAFKA-16157: --- Affects Version/s: 3.7.0 (was: 3.7.1) > Topic recreation with off

Re: [PR] KAFKA-14505; [6/N] Avoid recheduling callback in request thread [kafka]

2024-01-17 Thread via GitHub
jolshan commented on code in PR #15176: URL: https://github.com/apache/kafka/pull/15176#discussion_r1456228457 ## core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala: ## @@ -935,8 +935,9 @@ private[group] class GroupCoordinator( producerId, p

Re: [PR] KAFKA-14589 ConsumerGroupCommand options and case classes rewritten [kafka]

2024-01-17 Thread via GitHub
jolshan commented on code in PR #14856: URL: https://github.com/apache/kafka/pull/14856#discussion_r1456242441 ## tools/src/main/java/org/apache/kafka/tools/consumergroup/ConsumerGroupCommandOptions.java: ## @@ -0,0 +1,263 @@ +/* Review Comment: Is the tools/consumergroup a

Re: [PR] KAFKA-14589 ConsumerGroupCommand options and case classes rewritten [kafka]

2024-01-17 Thread via GitHub
jolshan commented on PR #14856: URL: https://github.com/apache/kafka/pull/14856#issuecomment-1896365557 Did we want to delete the old files in this PR or a follow up? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the

Re: [PR] KAFKA-15878: KIP-768 - Extend support for opaque (i.e. non-JWT) tokens in SASL/OAUTHBEARER [kafka]

2024-01-17 Thread via GitHub
kirktrue commented on PR #14818: URL: https://github.com/apache/kafka/pull/14818#issuecomment-1896409297 @jcme—Can you trigger a rebuild of the CI job? It looks like the last run didn't work. -- This is an automated message from the Apache Git Service. To respond to the message, please lo

[PR] KAFKA-16115: Adding missing heartbeat metrics [kafka]

2024-01-17 Thread via GitHub
philipnee opened a new pull request, #15216: URL: https://github.com/apache/kafka/pull/15216 (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-

Re: [PR] KAFKA-16147; Partition is assigned to two members at the same time [kafka]

2024-01-17 Thread via GitHub
jolshan commented on code in PR #15212: URL: https://github.com/apache/kafka/pull/15212#discussion_r1456314640 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java: ## @@ -861,19 +861,9 @@ private void maybeUpdatePartitionEpoch(

Re: [PR] KAFKA-16131: Only update directoryIds if the metadata version supports DirectoryAssignment [kafka]

2024-01-17 Thread via GitHub
pprovenzano commented on PR #15197: URL: https://github.com/apache/kafka/pull/15197#issuecomment-1896448532 Test failures are not related. -- 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

Re: [PR] KAFKA-15811: Enhance request context with client socket port information (KIP-714) [kafka]

2024-01-17 Thread via GitHub
junrao commented on code in PR #15190: URL: https://github.com/apache/kafka/pull/15190#discussion_r1456303419 ## core/src/main/scala/kafka/network/SocketServer.scala: ## @@ -1140,9 +1141,9 @@ private[kafka] class Processor( expiredConnectionsKilledCount.record(n

Re: [PR] KAFKA-15804: Close SocketServer channels when calling shutdown before enableRequestProcessing [kafka]

2024-01-17 Thread via GitHub
gharris1727 commented on PR #14729: URL: https://github.com/apache/kafka/pull/14729#issuecomment-1896468099 @chia7712 @showuon @mimaison Are any of you able to review this resource leak fix? Thanks! -- This is an automated message from the Apache Git Service. To respond to the message, pl

Re: [PR] KAFKA-15826: Close consumer when sink task is cancelled [kafka]

2024-01-17 Thread via GitHub
gharris1727 closed pull request #14762: KAFKA-15826: Close consumer when sink task is cancelled URL: https://github.com/apache/kafka/pull/14762 -- 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 sp

Re: [PR] KAFKA-15826: Close consumer when sink task is cancelled [kafka]

2024-01-17 Thread via GitHub
gharris1727 commented on PR #14762: URL: https://github.com/apache/kafka/pull/14762#issuecomment-1896474948 I think due to the locking concerns I raised earlier, and that we can resolve this resource leak in our tests, this PR is not viable to merge. We can revisit this in the future if the

[jira] [Commented] (KAFKA-16156) System test failing for new consumer on endOffsets with negative timestamps

2024-01-17 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16156?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17807858#comment-17807858 ] Justine Olshan commented on KAFKA-16156: The transactional copier is pretty old.

[jira] [Updated] (KAFKA-16116) AsyncKafkaConsumer: Add missing rebalance metrics

2024-01-17 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16116?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16116: -- Parent: (was: KAFKA-14246) Issue Type: Improvement (was: Sub-task) > AsyncKafkaConsumer:

[jira] [Commented] (KAFKA-15878) KIP-768: Extend support for opaque (i.e. non-JWT) tokens in SASL/OAUTHBEARER

2024-01-17 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15878?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17807866#comment-17807866 ] Kirk True commented on KAFKA-15878: --- [~philomathanuj]—for this to work, I assume the u

[jira] [Resolved] (KAFKA-16150) Fix PlaintextConsumerTest.testPerPartitionLagMetricsCleanUpWithSubscribe

2024-01-17 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True resolved KAFKA-16150. --- Resolution: Duplicate > Fix PlaintextConsumerTest.testPerPartitionLagMetricsCleanUpWithSubscribe > -

[jira] [Resolved] (KAFKA-16151) Fix PlaintextConsumerTest.testPerPartitionLeadMetricsCleanUpWithSubscribe

2024-01-17 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16151?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True resolved KAFKA-16151. --- Resolution: Duplicate > Fix PlaintextConsumerTest.testPerPartitionLeadMetricsCleanUpWithSubscribe >

Re: [PR] Metadata schema checker [kafka]

2024-01-17 Thread via GitHub
cmccabe commented on code in PR #14389: URL: https://github.com/apache/kafka/pull/14389#discussion_r1456440526 ## tools/src/main/java/org/apache/kafka/tools/SchemaChecker/MetadataSchemaChecker.java: ## @@ -0,0 +1,347 @@ +/* + * Licensed to the Apache Software Foundation (ASF) un

Re: [PR] Metadata schema checker [kafka]

2024-01-17 Thread via GitHub
cmccabe commented on code in PR #14389: URL: https://github.com/apache/kafka/pull/14389#discussion_r1456442184 ## tools/src/main/java/org/apache/kafka/tools/SchemaChecker/MetadataSchemaChecker.java: ## @@ -0,0 +1,359 @@ +/* + * Licensed to the Apache Software Foundation (ASF) un

Re: [PR] Metadata schema checker [kafka]

2024-01-17 Thread via GitHub
cmccabe commented on code in PR #14389: URL: https://github.com/apache/kafka/pull/14389#discussion_r1456441308 ## tools/src/main/java/org/apache/kafka/tools/SchemaChecker/MetadataSchemaChecker.java: ## @@ -0,0 +1,359 @@ +/* + * Licensed to the Apache Software Foundation (ASF) un

Re: [PR] Metadata schema checker [kafka]

2024-01-17 Thread via GitHub
cmccabe commented on code in PR #14389: URL: https://github.com/apache/kafka/pull/14389#discussion_r1456447370 ## tools/src/main/java/org/apache/kafka/tools/SchemaChecker/MetadataSchemaChecker.java: ## @@ -0,0 +1,359 @@ +/* + * Licensed to the Apache Software Foundation (ASF) un

Re: [PR] Metadata schema checker [kafka]

2024-01-17 Thread via GitHub
cmccabe commented on code in PR #14389: URL: https://github.com/apache/kafka/pull/14389#discussion_r1456448643 ## tools/src/main/java/org/apache/kafka/tools/SchemaChecker/MetadataSchemaChecker.java: ## @@ -0,0 +1,359 @@ +/* + * Licensed to the Apache Software Foundation (ASF) un

Re: [PR] Metadata schema checker [kafka]

2024-01-17 Thread via GitHub
cmccabe commented on code in PR #14389: URL: https://github.com/apache/kafka/pull/14389#discussion_r1456448973 ## tools/src/main/java/org/apache/kafka/tools/SchemaChecker/MetadataSchemaChecker.java: ## @@ -0,0 +1,347 @@ +/* + * Licensed to the Apache Software Foundation (ASF) un

Re: [PR] Metadata schema checker [kafka]

2024-01-17 Thread via GitHub
cmccabe commented on code in PR #14389: URL: https://github.com/apache/kafka/pull/14389#discussion_r1456451709 ## tools/src/main/java/org/apache/kafka/tools/SchemaChecker/MetadataSchemaChecker.java: ## @@ -0,0 +1,359 @@ +/* + * Licensed to the Apache Software Foundation (ASF) un

[jira] [Assigned] (KAFKA-16141) StreamsStandbyTask##test_standby_tasks_rebalanceArguments:{ “metadata_quorum”: “ISOLATED_KRAFT”, “use_new_coordinator”: false} fails consistently in 3.7

2024-01-17 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16141?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-16141: --- Assignee: Matthias J. Sax (was: Almog Gavra) > StreamsStandbyTask##test_standby_ta

[jira] [Commented] (KAFKA-16141) StreamsStandbyTask##test_standby_tasks_rebalanceArguments:{ “metadata_quorum”: “ISOLATED_KRAFT”, “use_new_coordinator”: false} fails consistently in 3.7

2024-01-17 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17807890#comment-17807890 ] Matthias J. Sax commented on KAFKA-16141: - After discussion with Almog (and test

[PR] KAFKA-16141: Fix StreamsStandbyTask system test [kafka]

2024-01-17 Thread via GitHub
mjsax opened a new pull request, #15217: URL: https://github.com/apache/kafka/pull/15217 KAFKA-15629 added `TimestampedByteStore` interface to `KeyValueToTimestampedKeyValueByteStoreAdapter` which break the restore code path and thus some system tests. This PR reverts this change for

Re: [PR] KAFKA-16141: Fix StreamsStandbyTask system test [kafka]

2024-01-17 Thread via GitHub
mjsax commented on PR #15217: URL: https://github.com/apache/kafka/pull/15217#issuecomment-1896696469 \cc @agavra @stanislavkozlovski -- 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

Re: [PR] KAFKA-16141: Fix StreamsStandbyTask system test [kafka]

2024-01-17 Thread via GitHub
mjsax commented on code in PR #15217: URL: https://github.com/apache/kafka/pull/15217#discussion_r1456485535 ## streams/src/main/java/org/apache/kafka/streams/state/internals/StoreQueryUtils.java: ## @@ -422,6 +423,16 @@ public static Function getDeserializeValue(final StateSe

[jira] [Created] (KAFKA-16158) Cleanup usage of `TimestampedBytesStore` interface

2024-01-17 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16158: --- Summary: Cleanup usage of `TimestampedBytesStore` interface Key: KAFKA-16158 URL: https://issues.apache.org/jira/browse/KAFKA-16158 Project: Kafka Issu

[jira] [Comment Edited] (KAFKA-16141) StreamsStandbyTask##test_standby_tasks_rebalanceArguments:{ “metadata_quorum”: “ISOLATED_KRAFT”, “use_new_coordinator”: false} fails consistently in 3.7

2024-01-17 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17807890#comment-17807890 ] Matthias J. Sax edited comment on KAFKA-16141 at 1/17/24 9:15 PM:

Re: [PR] KAFKA-16141: Fix StreamsStandbyTask system test [kafka]

2024-01-17 Thread via GitHub
mjsax commented on PR #15217: URL: https://github.com/apache/kafka/pull/15217#issuecomment-1896752852 Triggered a system test run: https://jenkins.confluent.io/job/system-test-kafka-branch-builder/6035/ -- This is an automated message from the Apache Git Service. To respond to the message

[PR] KAFKA-16107: Stop fetching while onPartitionsAssign completes [kafka]

2024-01-17 Thread via GitHub
anurag-harness opened a new pull request, #15218: URL: https://github.com/apache/kafka/pull/15218 *More detailed description of your change, if necessary. The PR title and PR message become the squashed commit message, so use a separate comment to ping reviewers.* *Summary of

Re: [PR] KAFKA-16147; Partition is assigned to two members at the same time [kafka]

2024-01-17 Thread via GitHub
jeffkbkim commented on PR #15212: URL: https://github.com/apache/kafka/pull/15212#issuecomment-1896786496 > when a member has a partition pending revocation re-assigned to him before the revocation is completed, the partition epoch is lost to confirm, the `previousMemberEpoch=11` shou

[jira] [Updated] (KAFKA-16141) StreamsStandbyTask##test_standby_tasks_rebalanceArguments:{ “metadata_quorum”: “ISOLATED_KRAFT”, “use_new_coordinator”: false} fails consistently in 3.7

2024-01-17 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16141?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-16141: Issue Type: Bug (was: Test) > StreamsStandbyTask##test_standby_tasks_rebalanceArguments:{

Re: [PR] KAFKA-16139: Fix StreamsUpgradeTest [kafka]

2024-01-17 Thread via GitHub
mjsax merged PR #15207: URL: https://github.com/apache/kafka/pull/15207 -- 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-16139: Fix StreamsUpgradeTest [kafka]

2024-01-17 Thread via GitHub
mjsax commented on PR #15207: URL: https://github.com/apache/kafka/pull/15207#issuecomment-1896826978 Merged to `trunk` and cherry-picked to `3.7` branch. -- 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 t

[jira] [Resolved] (KAFKA-16139) StreamsUpgradeTest fails consistently in 3.7.0

2024-01-17 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16139?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-16139. - Fix Version/s: 3.7.0 3.6.1 Resolution: Fixed > StreamsUpgradeT

Re: [PR] KAFKA-14589 ConsumerGroupCommand options and case classes rewritten [kafka]

2024-01-17 Thread via GitHub
nizhikov commented on code in PR #14856: URL: https://github.com/apache/kafka/pull/14856#discussion_r1456518422 ## tools/src/main/java/org/apache/kafka/tools/consumergroup/PartitionAssignmentState.java: ## @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF)

Re: [PR] KAFKA-14589 ConsumerGroupCommand options and case classes rewritten [kafka]

2024-01-17 Thread via GitHub
nizhikov commented on code in PR #14856: URL: https://github.com/apache/kafka/pull/14856#discussion_r1456521165 ## tools/src/main/java/org/apache/kafka/tools/consumergroup/ConsumerGroupCommandOptions.java: ## @@ -0,0 +1,263 @@ +/* + * Licensed to the Apache Software Foundation (

Re: [PR] KAFKA-15807: Added support for compression of metrics (KIP-714) [kafka]

2024-01-17 Thread via GitHub
mjsax commented on code in PR #15148: URL: https://github.com/apache/kafka/pull/15148#discussion_r1456522509 ## clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java: ## @@ -715,15 +716,22 @@ private Optional> createPushRequest(ClientTel

Re: [PR] KAFKA-14589 ConsumerGroupCommand options and case classes rewritten [kafka]

2024-01-17 Thread via GitHub
nizhikov commented on code in PR #14856: URL: https://github.com/apache/kafka/pull/14856#discussion_r1456523688 ## tools/src/main/java/org/apache/kafka/tools/consumergroup/ConsumerGroupCommandOptions.java: ## @@ -0,0 +1,263 @@ +/* + * Licensed to the Apache Software Foundation (

Re: [PR] KAFKA-15807: Added support for compression of metrics (KIP-714) [kafka]

2024-01-17 Thread via GitHub
mjsax merged PR #15148: URL: https://github.com/apache/kafka/pull/15148 -- 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: Uniformize error handling/transformation in GroupCoordinatorService [kafka]

2024-01-17 Thread via GitHub
jeffkbkim commented on code in PR #15196: URL: https://github.com/apache/kafka/pull/15196#discussion_r1456526190 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java: ## @@ -1098,30 +1054,36 @@ private static boolean isGroupIdNotEmpt

Re: [PR] KAFKA-14589 ConsumerGroupCommand options and case classes rewritten [kafka]

2024-01-17 Thread via GitHub
nizhikov commented on PR #14856: URL: https://github.com/apache/kafka/pull/14856#issuecomment-1896962207 @jolshan > Did we want to delete the old files in this PR or a follow up? For the previous command we remove old files only when command was merged 1. Reassign case

Re: [PR] KAFKA-14589 ConsumerGroupCommand options and case classes rewritten [kafka]

2024-01-17 Thread via GitHub
nizhikov commented on code in PR #14856: URL: https://github.com/apache/kafka/pull/14856#discussion_r1456530381 ## tools/src/main/java/org/apache/kafka/tools/consumergroup/ConsumerGroupCommandOptions.java: ## @@ -0,0 +1,263 @@ +/* Review Comment: > Is the tools/consumergroup

Re: [PR] KAFKA-14589 ConsumerGroupCommand options and case classes rewritten [kafka]

2024-01-17 Thread via GitHub
jolshan commented on code in PR #14856: URL: https://github.com/apache/kafka/pull/14856#discussion_r1456536862 ## tools/src/main/java/org/apache/kafka/tools/consumergroup/ConsumerGroupCommandOptions.java: ## @@ -0,0 +1,263 @@ +/* Review Comment: I wasn't sure if we wanted to

Re: [PR] KAFKA-16131: Only update directoryIds if the metadata version supports DirectoryAssignment [kafka]

2024-01-17 Thread via GitHub
cmccabe merged PR #15197: URL: https://github.com/apache/kafka/pull/15197 -- 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] [Created] (KAFKA-16159) Prune excessive logging from Telemetry Reporter

2024-01-17 Thread Philip Nee (Jira)
Philip Nee created KAFKA-16159: -- Summary: Prune excessive logging from Telemetry Reporter Key: KAFKA-16159 URL: https://issues.apache.org/jira/browse/KAFKA-16159 Project: Kafka Issue Type: Task

Re: [PR] KAFKA-14589 ConsumerGroupCommand options and case classes rewritten [kafka]

2024-01-17 Thread via GitHub
nizhikov commented on code in PR #14856: URL: https://github.com/apache/kafka/pull/14856#discussion_r1456546534 ## tools/src/main/java/org/apache/kafka/tools/consumergroup/ConsumerGroupCommandOptions.java: ## @@ -0,0 +1,263 @@ +/* Review Comment: > I think when we have had t

[jira] [Updated] (KAFKA-16159) Prune excessive logging from Telemetry Reporter

2024-01-17 Thread Philip Nee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16159?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Nee updated KAFKA-16159: --- Labels: logging (was: ) > Prune excessive logging from Telemetry Reporter > ---

[jira] [Updated] (KAFKA-16159) Prune excessive logging from Telemetry Reporter

2024-01-17 Thread Philip Nee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16159?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Nee updated KAFKA-16159: --- Labels: consumer logging (was: logging) > Prune excessive logging from Telemetry Reporter > ---

[jira] [Updated] (KAFKA-16159) Prune excessive logging from Telemetry Reporter

2024-01-17 Thread Philip Nee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16159?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Nee updated KAFKA-16159: --- Fix Version/s: 3.8.0 > Prune excessive logging from Telemetry Reporter > ---

[jira] [Resolved] (KAFKA-16131) Repeated UnsupportedVersionException logged when running Kafka 3.7.0-RC2 KRaft cluster with metadata version 3.6

2024-01-17 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16131?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-16131. -- Resolution: Fixed > Repeated UnsupportedVersionException logged when running Kafka 3.7.0-RC2

[jira] [Updated] (KAFKA-16159) Prune excessive logging from Telemetry Reporter

2024-01-17 Thread Philip Nee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16159?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Nee updated KAFKA-16159: --- Description: While running system tests locally, I've noticed excessive logging of the Telemtry Rep

  1   2   >