Re: [PR] KAFKA-5863: Avoid NPE when RestClient calls expecting no-content receive content. [kafka]

2024-01-16 Thread via GitHub
gharris1727 commented on PR #13294: URL: https://github.com/apache/kafka/pull/13294#issuecomment-1895192159 > Wouldn't any unexpected response here simply be indicative of a bug in the Connect runtime since the REST client is only being used internally to forward requests within a Connect

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

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

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

2024-01-16 Thread via GitHub
dajac commented on code in PR #15176: URL: https://github.com/apache/kafka/pull/15176#discussion_r1454771868 ## core/src/main/scala/kafka/server/ReplicaManager.scala: ## @@ -1090,38 +1090,29 @@ class ReplicaManager(val config: KafkaConfig, * @param producerId the

Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub
Phuc-Hong-Tran commented on code in PR #15188: URL: https://github.com/apache/kafka/pull/15188#discussion_r1454760385 ## clients/src/main/java/org/apache/kafka/clients/consumer/SubscriptionPattern.java: ## @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation

Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub
Phuc-Hong-Tran commented on code in PR #15188: URL: https://github.com/apache/kafka/pull/15188#discussion_r1454761116 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java: ## @@ -86,7 +86,6 @@ import static

Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub
Phuc-Hong-Tran commented on code in PR #15188: URL: https://github.com/apache/kafka/pull/15188#discussion_r1454760385 ## clients/src/main/java/org/apache/kafka/clients/consumer/SubscriptionPattern.java: ## @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation

[jira] [Commented] (KAFKA-16022) AsyncKafkaConsumer sometimes complains “No current assignment for partition {}”

2024-01-16 Thread Phuc Hong Tran (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16022?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807585#comment-17807585 ] Phuc Hong Tran commented on KAFKA-16022: [~pnee], were you seeing this exception in the

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

2024-01-16 Thread via GitHub
cmccabe commented on PR #15197: URL: https://github.com/apache/kafka/pull/15197#issuecomment-1895021752 Thanks for the quick response on this one, @pprovenzano . LGTM once you address the one comment I made. -- This is an automated message from the Apache Git Service. To respond to the

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

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

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

2024-01-16 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

[jira] [Commented] (KAFKA-15851) broker under replicated due to error java.nio.BufferOverflowException

2024-01-16 Thread Yu Wang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15851?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807561#comment-17807561 ] Yu Wang commented on KAFKA-15851: - got similar stack trace in 2.5.1, recovered after restart.

Re: [PR] KAFKA-16146: Checkpoint log-start-offset for remote log enabled topics [kafka]

2024-01-16 Thread via GitHub
kamalcph commented on PR #15201: URL: https://github.com/apache/kafka/pull/15201#issuecomment-1894989811 Test failures are unrelated -- 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] MINOR Removed unused CommittedOffsetsFile class. [kafka]

2024-01-16 Thread via GitHub
kamalcph commented on PR #15209: URL: https://github.com/apache/kafka/pull/15209#issuecomment-1894987437 Should we also avoid creating the

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

2024-01-16 Thread via GitHub
philipnee commented on code in PR #15210: URL: https://github.com/apache/kafka/pull/15210#discussion_r1454575608 ## clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java: ## @@ -2169,10 +2169,8 @@ public void

Re: [PR] KAFKA-5863: Avoid NPE when RestClient calls expecting no-content receive content. [kafka]

2024-01-16 Thread via GitHub
yashmayya commented on PR #13294: URL: https://github.com/apache/kafka/pull/13294#issuecomment-1894930823 > because it's inherently untrusted content that shouldn't be put into the log by default I'm not sure I follow why this would be inherently untrusted content? Wouldn't any

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

2024-01-16 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807543#comment-17807543 ] Matthias J. Sax commented on KAFKA-16141: - Thanks. I am wondering if the right fix would be, to

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

2024-01-16 Thread via GitHub
philipnee opened a new pull request, #15210: URL: https://github.com/apache/kafka/pull/15210 In this PR, I'm adding sensor to the `CommitRequestManager` to record the necessary metrics, i.e.: ``` commit-latency-avg commit-latency-max commit-rate commit-total

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

2024-01-16 Thread via GitHub
satishd opened a new pull request, #15209: URL: https://github.com/apache/kafka/pull/15209 We will introduce the same when it is required for enhancing TBRLMM to consume from a specific offset when snapshots are implemented. ### Committer Checklist (excluded from commit

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

2024-01-16 Thread Almog Gavra (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807520#comment-17807520 ] Almog Gavra commented on KAFKA-16141: - OK, after doing some more digging I don't think it's related

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

2024-01-16 Thread Proven Provenzano (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16131?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Proven Provenzano updated KAFKA-16131: -- Affects Version/s: 3.7.0 > Repeated UnsupportedVersionException logged when running

Re: [PR] KAFKA-16029: Fix "Unable to find FetchSessionHandler for node X" bug [kafka]

2024-01-16 Thread via GitHub
kirktrue commented on PR #15186: URL: https://github.com/apache/kafka/pull/15186#issuecomment-1894795635 @ijuma / @stanislavkozlovski are either of you able to review? Thanks! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub

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

2024-01-16 Thread via GitHub
mjsax commented on PR #15207: URL: https://github.com/apache/kafka/pull/15207#issuecomment-1894793526 This PR must be 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 to go

Re: [PR] MINOR: code cleanup [kafka]

2024-01-16 Thread via GitHub
mjsax commented on PR #15208: URL: https://github.com/apache/kafka/pull/15208#issuecomment-1894793322 This PR must be cherry-picked to `3.7` and `3.6` branches. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL

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

2024-01-16 Thread via GitHub
mjsax commented on code in PR #15199: URL: https://github.com/apache/kafka/pull/15199#discussion_r1454318352 ## streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java: ## @@ -71,6 +71,7 @@ import static org.apache.kafka.common.config.ConfigDef.Range.between; import

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

2024-01-16 Thread via GitHub
mjsax commented on PR #15199: URL: https://github.com/apache/kafka/pull/15199#issuecomment-1894790719 Follow-up PR to add `3.6`: 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

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

2024-01-16 Thread via GitHub
mjsax opened a new pull request, #15207: URL: https://github.com/apache/kafka/pull/15207 Adds version 3.6 to the possible values for config upgrade_from. -- 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

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

2024-01-16 Thread via GitHub
kirktrue opened a new pull request, #15206: URL: https://github.com/apache/kafka/pull/15206 We reevaluated the integration tests that were disabled for the new consumer group protocol which _should_ be supported. The evaluation was to run the `PlaintextConsumerTest` suite ten times and see

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

2024-01-16 Thread via GitHub
mjsax commented on PR #15199: URL: https://github.com/apache/kafka/pull/15199#issuecomment-1894786932 Merged to `trunk` and cherry-picked to `3.7` and `3.6` branches. -- 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-16139: Fix StreamsUpgradeTest [kafka]

2024-01-16 Thread via GitHub
mjsax merged PR #15199: URL: https://github.com/apache/kafka/pull/15199 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

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

2024-01-16 Thread via GitHub
mjsax commented on PR #15199: URL: https://github.com/apache/kafka/pull/15199#issuecomment-1894782497 I just checked the test failures on https://github.com/apache/kafka/pull/15151 (cf last comment) and it failed because `3.6` is missing. Can you also do a follow up PR for 3.6

[jira] [Updated] (KAFKA-16104) Enable additional PlaintextConsumerTest tests for new consumer

2024-01-16 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16104: -- Description: It should be possible to enable: * testAutoCommitOnClose *

Re: [PR] KAFKA-14505; [7/7] Always materialize the most recent committed offset [kafka]

2024-01-16 Thread via GitHub
jolshan commented on code in PR #15183: URL: https://github.com/apache/kafka/pull/15183#discussion_r1454293498 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java: ## @@ -712,13 +712,14 @@ public void run() {

Re: [PR] KAFKA-14505; [7/7] Always materialize the most recent committed offset [kafka]

2024-01-16 Thread via GitHub
jolshan commented on code in PR #15183: URL: https://github.com/apache/kafka/pull/15183#discussion_r1454285529 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java: ## @@ -979,15 +981,30 @@ public void replayEndTransactionMarker(

[jira] [Updated] (KAFKA-16008) Fix PlaintextConsumerTest.testMaxPollIntervalMs

2024-01-16 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16008?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16008: -- Issue Type: Bug (was: Test) > Fix PlaintextConsumerTest.testMaxPollIntervalMs >

[jira] [Updated] (KAFKA-16009) Fix PlaintextConsumerTest.testMaxPollIntervalMsDelayInRevocation

2024-01-16 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16009?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16009: -- Issue Type: Bug (was: Test) > Fix PlaintextConsumerTest.testMaxPollIntervalMsDelayInRevocation >

Re: [PR] KAFKA-14505; [7/7] Always materialize the most recent committed offset [kafka]

2024-01-16 Thread via GitHub
jolshan commented on code in PR #15183: URL: https://github.com/apache/kafka/pull/15183#discussion_r1454285529 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java: ## @@ -979,15 +981,30 @@ public void replayEndTransactionMarker(

Re: [PR] KAFKA-14505; [7/7] Always materialize the most recent committed offset [kafka]

2024-01-16 Thread via GitHub
jolshan commented on code in PR #15183: URL: https://github.com/apache/kafka/pull/15183#discussion_r1454282345 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java: ## @@ -918,7 +920,7 @@ public void replay(

Re: [PR] KAFKA-14505; [7/7] Always materialize the most recent committed offset [kafka]

2024-01-16 Thread via GitHub
jolshan commented on code in PR #15183: URL: https://github.com/apache/kafka/pull/15183#discussion_r1454280167 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetAndMetadata.java: ## @@ -92,30 +117,34 @@ public boolean equals(Object o) {

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

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

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

2024-01-16 Thread via GitHub
jolshan commented on code in PR #15176: URL: https://github.com/apache/kafka/pull/15176#discussion_r1454268198 ## core/src/main/scala/kafka/server/ReplicaManager.scala: ## @@ -1090,38 +1090,29 @@ class ReplicaManager(val config: KafkaConfig, * @param producerId the

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

2024-01-16 Thread via GitHub
jolshan commented on code in PR #15176: URL: https://github.com/apache/kafka/pull/15176#discussion_r1454267538 ## core/src/main/scala/kafka/server/ReplicaManager.scala: ## @@ -1090,38 +1090,29 @@ class ReplicaManager(val config: KafkaConfig, * @param producerId the

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

2024-01-16 Thread via GitHub
mjsax commented on PR #15199: URL: https://github.com/apache/kafka/pull/15199#issuecomment-1894743876 Is this fix sufficient? Don't we not also need to add `3.6` as "upgrade_from" version? -- This is an automated message from the Apache Git Service. To respond to the message, please log

Re: [PR] KAFKA-16042: Add byte-rate metrics for topic and partition [kafka]

2024-01-16 Thread via GitHub
ex172000 commented on PR #15085: URL: https://github.com/apache/kafka/pull/15085#issuecomment-1894726845 FYI: We are making a similar effort here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-977%3A+Partition-Level+Throughput+Metrics -- This is an automated message from the

Re: [PR] KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup [kafka]

2024-01-16 Thread via GitHub
cmccabe commented on PR #15192: URL: https://github.com/apache/kafka/pull/15192#issuecomment-1894711882 committed, thanks! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment.

Re: [PR] KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup [kafka]

2024-01-16 Thread via GitHub
cmccabe closed pull request #15192: KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup URL: https://github.com/apache/kafka/pull/15192 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL

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

2024-01-16 Thread via GitHub
pprovenzano commented on PR #15197: URL: https://github.com/apache/kafka/pull/15197#issuecomment-1894699271 Test failures are unrelated to the PR. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to

Re: [PR] MINOR: Uniformize error handling/transformation in GroupCoordinatorService [kafka]

2024-01-16 Thread via GitHub
dongnuo123 commented on code in PR #15196: URL: https://github.com/apache/kafka/pull/15196#discussion_r1454202465 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java: ## @@ -524,37 +495,39 @@ public CompletableFuture listGroups(

[PR] MINOR: populate TopicName in ConsumerGroupDescribe [kafka]

2024-01-16 Thread via GitHub
dongnuo123 opened a new pull request, #15205: URL: https://github.com/apache/kafka/pull/15205 The patch populates the topic name of `ConsumerGroupDescribeResponseData.TopicPartitions` with the corresponding topic id in `ConsumerGroupDescribe`. ### Committer Checklist (excluded from

[jira] [Created] (KAFKA-16152) Fix PlaintextConsumerTest.testStaticConsumerDetectsNewPartitionCreatedAfterRestart

2024-01-16 Thread Kirk True (Jira)
Kirk True created KAFKA-16152: - Summary: Fix PlaintextConsumerTest.testStaticConsumerDetectsNewPartitionCreatedAfterRestart Key: KAFKA-16152 URL: https://issues.apache.org/jira/browse/KAFKA-16152

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

2024-01-16 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16151?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-16151: -- Summary: Fix PlaintextConsumerTest.testPerPartitionLeadMetricsCleanUpWithSubscribe (was: Fix

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

2024-01-16 Thread Kirk True (Jira)
Kirk True created KAFKA-16150: - Summary: Fix PlaintextConsumerTest.testPerPartitionLagMetricsCleanUpWithSubscribe Key: KAFKA-16150 URL: https://issues.apache.org/jira/browse/KAFKA-16150 Project: Kafka

[jira] [Created] (KAFKA-16151) Fix PlaintextConsumerTest.testPerPartitionLedMetricsCleanUpWithSubscribe

2024-01-16 Thread Kirk True (Jira)
Kirk True created KAFKA-16151: - Summary: Fix PlaintextConsumerTest.testPerPartitionLedMetricsCleanUpWithSubscribe Key: KAFKA-16151 URL: https://issues.apache.org/jira/browse/KAFKA-16151 Project: Kafka

[jira] [Created] (KAFKA-16149) Aggressively expire unused client connections

2024-01-16 Thread Kirk True (Jira)
Kirk True created KAFKA-16149: - Summary: Aggressively expire unused client connections Key: KAFKA-16149 URL: https://issues.apache.org/jira/browse/KAFKA-16149 Project: Kafka Issue Type:

Re: [PR] KAFKA-15585: Add DescribeTopics API server side support [kafka]

2024-01-16 Thread via GitHub
CalvinConfluent commented on code in PR #14612: URL: https://github.com/apache/kafka/pull/14612#discussion_r1454185664 ## core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala: ## @@ -141,17 +144,33 @@ class KRaftMetadataCache(val brokerId: Int) extends

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

2024-01-16 Thread Phuc Hong Tran (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15538?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807487#comment-17807487 ] Phuc Hong Tran commented on KAFKA-15538: [~lianetm] thanks for the comments. Just to clarify

Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub
Phuc-Hong-Tran commented on PR #15188: URL: https://github.com/apache/kafka/pull/15188#issuecomment-1894635327 Thanks @lianetm -- 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] [Updated] (KAFKA-16083) Exclude throttle time when expiring inflight requests on a connection

2024-01-16 Thread Adithya Chandra (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16083?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Adithya Chandra updated KAFKA-16083: Affects Version/s: (was: 3.7.0) > Exclude throttle time when expiring inflight

[jira] [Updated] (KAFKA-16083) Exclude throttle time when expiring inflight requests on a connection

2024-01-16 Thread Adithya Chandra (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16083?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Adithya Chandra updated KAFKA-16083: Affects Version/s: 3.7.0 > Exclude throttle time when expiring inflight requests on a

[jira] [Resolved] (KAFKA-16083) Exclude throttle time when expiring inflight requests on a connection

2024-01-16 Thread Adithya Chandra (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16083?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Adithya Chandra resolved KAFKA-16083. - Fix Version/s: 3.8.0 Reviewer: Stanislav Kozlovski Resolution: Fixed >

Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub
lianetm commented on PR #15188: URL: https://github.com/apache/kafka/pull/15188#issuecomment-1894569927 This is the task to closely follow https://issues.apache.org/jira/browse/KAFKA-14517, where the broker will support the new regex. -- This is an automated message from the Apache Git

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

2024-01-16 Thread Almog Gavra (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807434#comment-17807434 ] Almog Gavra commented on KAFKA-16141: - I confirmed it’s most likely related to my change - changing

[jira] [Created] (KAFKA-16148) Implement GroupMetadataManager#onUnloaded

2024-01-16 Thread Jeff Kim (Jira)
Jeff Kim created KAFKA-16148: Summary: Implement GroupMetadataManager#onUnloaded Key: KAFKA-16148 URL: https://issues.apache.org/jira/browse/KAFKA-16148 Project: Kafka Issue Type: Sub-task

Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub
Phuc-Hong-Tran commented on PR #15188: URL: https://github.com/apache/kafka/pull/15188#issuecomment-1894475663 @lianetm, thanks for the comments, I will make sure to address those points in my next PR. Regarding your point about passing the regex for HeartbeatRequestManager, I

Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub
lianetm commented on code in PR #15188: URL: https://github.com/apache/kafka/pull/15188#discussion_r1453968041 ## clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java: ## @@ -86,7 +86,6 @@ import static org.mockito.Mockito.when;

Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub
lianetm commented on code in PR #15188: URL: https://github.com/apache/kafka/pull/15188#discussion_r1453966464 ## clients/src/main/java/org/apache/kafka/clients/consumer/SubscriptionPattern.java: ## @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under

Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub
lianetm commented on code in PR #15188: URL: https://github.com/apache/kafka/pull/15188#discussion_r1453958313 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java: ## @@ -84,6 +85,9 @@ private enum SubscriptionType { /* the pattern

Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub
lianetm commented on code in PR #15188: URL: https://github.com/apache/kafka/pull/15188#discussion_r1453953690 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/LegacyKafkaConsumer.java: ## @@ -494,6 +495,16 @@ public void subscribe(Pattern pattern) {

Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub
lianetm commented on code in PR #15188: URL: https://github.com/apache/kafka/pull/15188#discussion_r1453945617 ## clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java: ## @@ -753,6 +753,10 @@ public void subscribe(Pattern pattern,

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

2024-01-16 Thread Lianet Magrans (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15538?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807323#comment-17807323 ] Lianet Magrans edited comment on KAFKA-15538 at 1/16/24 7:45 PM: - Hey

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

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

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

2024-01-16 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807411#comment-17807411 ] Matthias J. Sax commented on KAFKA-16141: - Assigned to [~agavra] and marked as blocker. Might be

[jira] [Commented] (KAFKA-15561) Client support for new SubscriptionPattern based subscription

2024-01-16 Thread Phuc Hong Tran (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15561?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807412#comment-17807412 ] Phuc Hong Tran commented on KAFKA-15561: [~lianetm], PTAL if you have time. Thanks > Client

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

2024-01-16 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: Priority: Blocker (was: Major) >

Re: [PR] KAFKA-15853: Move KafkaConfig.Defaults to server module [kafka]

2024-01-16 Thread via GitHub
OmniaGM commented on PR #15158: URL: https://github.com/apache/kafka/pull/15158#issuecomment-1894343924 > @OmniaGM It looks like there are a [few build failures](https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-pr/detail/PR-15158/6/pipeline/10): > > ``` >

Re: [PR] KAFKA-15853: Move KafkaConfig.Defaults to server module [kafka]

2024-01-16 Thread via GitHub
OmniaGM commented on code in PR #15158: URL: https://github.com/apache/kafka/pull/15158#discussion_r1453877298 ## transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TransactionLogConfig.java: ## @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache

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

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

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

2024-01-16 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15878?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-15878: -- Fix Version/s: 3.8.0 > KIP-768: Extend support for opaque (i.e. non-JWT) tokens in SASL/OAUTHBEARER >

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

2024-01-16 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: Almog Gavra >

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

2024-01-16 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15878?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kirk True updated KAFKA-15878: -- Labels: oauth (was: ) > KIP-768: Extend support for opaque (i.e. non-JWT) tokens in SASL/OAUTHBEARER

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

2024-01-16 Thread via GitHub
kirktrue commented on PR #14818: URL: https://github.com/apache/kafka/pull/14818#issuecomment-1894339862 @jcme—I wrote and implemented KIP-768, so I'll take a look at this. Also, are you able to assign the Jira to yourself? Thanks! -- This is an automated message from the Apache

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

2024-01-16 Thread via GitHub
philipnee commented on code in PR #15000: URL: https://github.com/apache/kafka/pull/15000#discussion_r1453871780 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java: ## @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software

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

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

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

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

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

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

Re: [PR] KAFKA-15585: Add DescribeTopics API server side support [kafka]

2024-01-16 Thread via GitHub
mumrah commented on code in PR #14612: URL: https://github.com/apache/kafka/pull/14612#discussion_r1453861304 ## core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala: ## @@ -140,6 +141,71 @@ class KRaftMetadataCache(val brokerId: Int) extends MetadataCache with

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

2024-01-16 Thread via GitHub
OmniaGM commented on code in PR #15159: URL: https://github.com/apache/kafka/pull/15159#discussion_r1453859341 ## checkstyle/import-control-core.xml: ## @@ -82,6 +82,7 @@ + Review Comment: Remove it ##

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

2024-01-16 Thread via GitHub
OmniaGM commented on code in PR #15159: URL: https://github.com/apache/kafka/pull/15159#discussion_r1453859778 ## server/src/main/java/org/apache/kafka/server/config/ClientQuotaManagerConfig.java: ## @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF)

Re: [PR] KAFKA-14822: Allow restricting File and Directory ConfigProviders to specific paths [kafka]

2024-01-16 Thread via GitHub
gharris1727 commented on code in PR #14995: URL: https://github.com/apache/kafka/pull/14995#discussion_r1453854211 ## clients/src/main/java/org/apache/kafka/common/config/provider/FileConfigProvider.java: ## @@ -40,7 +42,13 @@ public class FileConfigProvider implements

Re: [PR] KAFKA-14822: Allow restricting File and Directory ConfigProviders to specific paths [kafka]

2024-01-16 Thread via GitHub
gharris1727 commented on code in PR #14995: URL: https://github.com/apache/kafka/pull/14995#discussion_r1453851791 ## clients/src/main/java/org/apache/kafka/common/config/provider/DirectoryConfigProvider.java: ## @@ -44,8 +44,15 @@ public class DirectoryConfigProvider

Re: [PR] KAFKA-14822: Allow restricting File and Directory ConfigProviders to specific paths [kafka]

2024-01-16 Thread via GitHub
gharris1727 commented on code in PR #14995: URL: https://github.com/apache/kafka/pull/14995#discussion_r1453850678 ## clients/src/main/java/org/apache/kafka/common/config/provider/FileConfigProvider.java: ## @@ -40,7 +42,13 @@ public class FileConfigProvider implements

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

2024-01-16 Thread Emanuele Sabellico (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16147?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807403#comment-17807403 ] Emanuele Sabellico commented on KAFKA-16147: This is the events sequence: SUBSCRIPTION_1 is

Re: [PR] KAFKA-16029: Fix "Unable to find FetchSessionHandler for node X" bug [kafka]

2024-01-16 Thread via GitHub
philipnee commented on code in PR #15186: URL: https://github.com/apache/kafka/pull/15186#discussion_r1453840738 ## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java: ## @@ -376,25 +376,21 @@ protected Map prepareCloseFetchSessi

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

2024-01-16 Thread Emanuele Sabellico (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16147?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Emanuele Sabellico updated KAFKA-16147: --- Description: While running [test 0113 of

[jira] [Commented] (KAFKA-16105) Reassignment of tiered topics is failing due to RemoteStorageException

2024-01-16 Thread Kamal Chandraprakash (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16105?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807397#comment-17807397 ] Kamal Chandraprakash commented on KAFKA-16105: -- [~anatolypopov]  Could you write an

[jira] [Comment Edited] (KAFKA-16095) Update list group state type filter to include the states for the new consumer group type

2024-01-16 Thread Ritika Reddy (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16095?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807394#comment-17807394 ] Ritika Reddy edited comment on KAFKA-16095 at 1/16/24 6:29 PM: --- Yep that

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

2024-01-16 Thread Emanuele Sabellico (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16147?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Emanuele Sabellico updated KAFKA-16147: --- Attachment: librdkafka.log > Partition is assigned to two members at the same time

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

2024-01-16 Thread Emanuele Sabellico (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16147?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Emanuele Sabellico updated KAFKA-16147: --- Attachment: (was: double-assignment.log) > Partition is assigned to two members

[jira] [Commented] (KAFKA-16095) Update list group state type filter to include the states for the new consumer group type

2024-01-16 Thread Ritika Reddy (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16095?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17807394#comment-17807394 ] Ritika Reddy commented on KAFKA-16095: -- Yep that would be merged this week > Update list group

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

2024-01-16 Thread Emanuele Sabellico (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16147?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Emanuele Sabellico updated KAFKA-16147: --- Attachment: server.properties server1.properties

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

2024-01-16 Thread Emanuele Sabellico (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16147?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Emanuele Sabellico updated KAFKA-16147: --- Description: While running test[ 0113 of

  1   2   >