[jira] [Comment Edited] (KAFKA-14934) KafkaClusterTestKit makes FaultHandler accessible

2023-05-26 Thread Owen C.H. Leung (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14934?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17726127#comment-17726127 ] Owen C.H. Leung edited comment on KAFKA-14934 at 5/27/23 2:49 AM: -- Hi

[GitHub] [kafka] vamossagar12 commented on a diff in pull request #13504: KAFKA-14750: Check if topic exists in WorkerSinkTask when committing offsets.

2023-05-26 Thread via GitHub
vamossagar12 commented on code in PR #13504: URL: https://github.com/apache/kafka/pull/13504#discussion_r1207577607 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java: ## @@ -700,9 +714,28 @@ private class HandleRebalance implements

[GitHub] [kafka] jolshan commented on pull request #13768: Suggest for performance fix: KAFKA-9693 Kafka latency spikes caused by log segment flush on roll

2023-05-26 Thread via GitHub
jolshan commented on PR #13768: URL: https://github.com/apache/kafka/pull/13768#issuecomment-1565102014 Thanks for the PR! This looks promising. As Ismael said, let's share in trunk first. -- This is an automated message from the Apache Git Service. To respond to the message, please log

[GitHub] [kafka] gharris1727 opened a new pull request, #13771: MINOR: Refactor DelegatingClassLoader to emit immutable PluginScanResult

2023-05-26 Thread via GitHub
gharris1727 opened a new pull request, #13771: URL: https://github.com/apache/kafka/pull/13771 The DelegatingClassLoader has a large number of fields and methods for keeping track of known PluginDesc objects. It has this in common with the PluginScanResult data object, which has a similar

[jira] [Created] (KAFKA-15031) Add plugin.discovery worker configuration

2023-05-26 Thread Greg Harris (Jira)
Greg Harris created KAFKA-15031: --- Summary: Add plugin.discovery worker configuration Key: KAFKA-15031 URL: https://issues.apache.org/jira/browse/KAFKA-15031 Project: Kafka Issue Type: New

[jira] [Created] (KAFKA-15030) Add connect-plugin-path command line tool

2023-05-26 Thread Greg Harris (Jira)
Greg Harris created KAFKA-15030: --- Summary: Add connect-plugin-path command line tool Key: KAFKA-15030 URL: https://issues.apache.org/jira/browse/KAFKA-15030 Project: Kafka Issue Type: New

[GitHub] [kafka] jolshan commented on a diff in pull request #13608: KAFKA-14884: Include check transaction is still ongoing right before append

2023-05-26 Thread via GitHub
jolshan commented on code in PR #13608: URL: https://github.com/apache/kafka/pull/13608#discussion_r1207484112 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -683,6 +702,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, validateAndAssignOffsets = false,

[GitHub] [kafka] jolshan commented on a diff in pull request #13608: KAFKA-14884: Include check transaction is still ongoing right before append

2023-05-26 Thread via GitHub
jolshan commented on code in PR #13608: URL: https://github.com/apache/kafka/pull/13608#discussion_r1207484112 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -683,6 +702,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, validateAndAssignOffsets = false,

[jira] [Assigned] (KAFKA-14991) Improving Producer's record timestamp validation

2023-05-26 Thread Mehari Beyene (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14991?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mehari Beyene reassigned KAFKA-14991: - Assignee: Mehari Beyene > Improving Producer's record timestamp validation >

[GitHub] [kafka] jolshan commented on a diff in pull request #13608: KAFKA-14884: Include check transaction is still ongoing right before append

2023-05-26 Thread via GitHub
jolshan commented on code in PR #13608: URL: https://github.com/apache/kafka/pull/13608#discussion_r1207324170 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -683,6 +702,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, validateAndAssignOffsets = false,

[GitHub] [kafka] mumrah commented on a diff in pull request #13767: KAFKA-15004: Fix configuration dual-write during migration

2023-05-26 Thread via GitHub
mumrah commented on code in PR #13767: URL: https://github.com/apache/kafka/pull/13767#discussion_r1207482435 ## core/src/main/scala/kafka/zk/ZkMigrationClient.scala: ## @@ -145,44 +144,47 @@ class ZkMigrationClient( topicClient.iterateTopics(

[GitHub] [kafka] jolshan opened a new pull request, #13770: MINOR: Add config to producerStateManager config

2023-05-26 Thread via GitHub
jolshan opened a new pull request, #13770: URL: https://github.com/apache/kafka/pull/13770 Originally part of https://github.com/apache/kafka/pull/13608/files. Since there are so many files changed, I decided to just pull this out into its own PR. I have moved this config into

[GitHub] [kafka] jolshan opened a new pull request, #13769: MINOR: Covering all epoch cases in add partitions to txn manager

2023-05-26 Thread via GitHub
jolshan opened a new pull request, #13769: URL: https://github.com/apache/kafka/pull/13769 Originally part of https://github.com/apache/kafka/pull/13608, Artem made a good point that this change was unrelated, so I'm making a minor PR to cover it. Cleaning up the

[GitHub] [kafka] cmccabe commented on a diff in pull request #13759: KAFKA-15019: Improve handling of overload situations in the kcontroller

2023-05-26 Thread via GitHub
cmccabe commented on code in PR #13759: URL: https://github.com/apache/kafka/pull/13759#discussion_r1207395237 ## metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java: ## @@ -223,6 +283,21 @@ public BrokerHeartbeatState next() { } } +

[GitHub] [kafka] cmccabe commented on a diff in pull request #13759: KAFKA-15019: Improve handling of overload situations in the kcontroller

2023-05-26 Thread via GitHub
cmccabe commented on code in PR #13759: URL: https://github.com/apache/kafka/pull/13759#discussion_r1207382044 ## server-common/src/main/java/org/apache/kafka/server/metrics/WindowedEventCounter.java: ## @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF)

[GitHub] [kafka] jolshan commented on a diff in pull request #13608: KAFKA-14884: Include check transaction is still ongoing right before append

2023-05-26 Thread via GitHub
jolshan commented on code in PR #13608: URL: https://github.com/apache/kafka/pull/13608#discussion_r1207368803 ## core/src/main/scala/kafka/server/AddPartitionsToTxnManager.scala: ## @@ -45,35 +45,48 @@ class AddPartitionsToTxnManager(config: KafkaConfig, client:

[GitHub] [kafka] jolshan commented on a diff in pull request #13608: KAFKA-14884: Include check transaction is still ongoing right before append

2023-05-26 Thread via GitHub
jolshan commented on code in PR #13608: URL: https://github.com/apache/kafka/pull/13608#discussion_r1207333185 ## core/src/main/scala/kafka/server/ReplicaManager.scala: ## @@ -671,6 +671,7 @@ class ReplicaManager(val config: KafkaConfig, val sTime = time.milliseconds

[GitHub] [kafka] jolshan commented on a diff in pull request #13608: KAFKA-14884: Include check transaction is still ongoing right before append

2023-05-26 Thread via GitHub
jolshan commented on code in PR #13608: URL: https://github.com/apache/kafka/pull/13608#discussion_r1207330097 ## core/src/main/scala/kafka/server/AddPartitionsToTxnManager.scala: ## @@ -45,35 +45,48 @@ class AddPartitionsToTxnManager(config: KafkaConfig, client:

[GitHub] [kafka] jolshan commented on a diff in pull request #13608: KAFKA-14884: Include check transaction is still ongoing right before append

2023-05-26 Thread via GitHub
jolshan commented on code in PR #13608: URL: https://github.com/apache/kafka/pull/13608#discussion_r1207324170 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -683,6 +702,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, validateAndAssignOffsets = false,

[GitHub] [kafka] cmccabe closed pull request #13766: KAFKA-14996: Limit partition count in Create Topic and Create Partitions

2023-05-26 Thread via GitHub
cmccabe closed pull request #13766: KAFKA-14996: Limit partition count in Create Topic and Create Partitions URL: https://github.com/apache/kafka/pull/13766 -- 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

[GitHub] [kafka] cmccabe commented on pull request #13766: KAFKA-14996: Limit partition count in Create Topic and Create Partitions

2023-05-26 Thread via GitHub
cmccabe commented on PR #13766: URL: https://github.com/apache/kafka/pull/13766#issuecomment-1564905334 Thanks @edoardocomar. Closing as duplicate of #13742 . If you're interested in doing more here, check out the discussion on the other PR about a possible KIP we could have (to add

[GitHub] [kafka] artemlivshits commented on a diff in pull request #13608: KAFKA-14884: Include check transaction is still ongoing right before append

2023-05-26 Thread via GitHub
artemlivshits commented on code in PR #13608: URL: https://github.com/apache/kafka/pull/13608#discussion_r1206199912 ## core/src/main/scala/kafka/log/UnifiedLog.scala: ## @@ -579,10 +579,28 @@ class UnifiedLog(@volatile var logStartOffset: Long, result } + def

[GitHub] [kafka] ijuma commented on pull request #13768: Suggest for performance fix: KAFKA-9693 Kafka latency spikes caused by log segment flush on roll

2023-05-26 Thread via GitHub
ijuma commented on PR #13768: URL: https://github.com/apache/kafka/pull/13768#issuecomment-1564903842 We typically make changes to master first. Would you be willing to submit a PR for that instead? -- This is an automated message from the Apache Git Service. To respond to the message,

[GitHub] [kafka] cmccabe commented on pull request #13742: KAFKA-14996: Handle overly large user operations on the kcontroller

2023-05-26 Thread via GitHub
cmccabe commented on PR #13742: URL: https://github.com/apache/kafka/pull/13742#issuecomment-1564904075 Thanks for all the reviews, and thanks @mumrah for the LGTM. Since this is a 3.5 blocker I am getting it in today so that it will be in the next RC. As I said before, this doesn't

[GitHub] [kafka] cmccabe merged pull request #13742: KAFKA-14996: Handle overly large user operations on the kcontroller

2023-05-26 Thread via GitHub
cmccabe merged PR #13742: URL: https://github.com/apache/kafka/pull/13742 -- 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] [Comment Edited] (KAFKA-9693) Kafka latency spikes caused by log segment flush on roll

2023-05-26 Thread Ruslan Scherbakov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17726728#comment-17726728 ] Ruslan Scherbakov edited comment on KAFKA-9693 at 5/26/23 8:05 PM: ---

[jira] [Comment Edited] (KAFKA-9693) Kafka latency spikes caused by log segment flush on roll

2023-05-26 Thread Ruslan Scherbakov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17726728#comment-17726728 ] Ruslan Scherbakov edited comment on KAFKA-9693 at 5/26/23 8:00 PM: ---

[jira] [Commented] (KAFKA-9693) Kafka latency spikes caused by log segment flush on roll

2023-05-26 Thread Ruslan Scherbakov (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17726728#comment-17726728 ] Ruslan Scherbakov commented on KAFKA-9693: -- Related pull request:

[GitHub] [kafka] novosibman opened a new pull request, #13768: Suggest for performance fix: KAFKA-9693 Kafka latency spikes caused by log segment flush on roll

2023-05-26 Thread via GitHub
novosibman opened a new pull request, #13768: URL: https://github.com/apache/kafka/pull/13768 Related issue https://issues.apache.org/jira/browse/KAFKA-9693 The issue with repeating latency spikes during Kafka log segments rolling still reproduced on the latest versions including

[GitHub] [kafka] cmccabe commented on a diff in pull request #13767: KAFKA-15004: Fix configuration dual-write during migration

2023-05-26 Thread via GitHub
cmccabe commented on code in PR #13767: URL: https://github.com/apache/kafka/pull/13767#discussion_r1207234024 ## metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationZkWriter.java: ## @@ -194,23 +195,44 @@ void handleTopicsDelta(Function

[GitHub] [kafka] cmccabe commented on a diff in pull request #13767: KAFKA-15004: Fix configuration dual-write during migration

2023-05-26 Thread via GitHub
cmccabe commented on code in PR #13767: URL: https://github.com/apache/kafka/pull/13767#discussion_r1207225622 ## core/src/main/scala/kafka/zk/ZkMigrationClient.scala: ## @@ -145,44 +144,47 @@ class ZkMigrationClient( topicClient.iterateTopics(

[GitHub] [kafka] mumrah commented on pull request #13757: [WIP] Fix snapshot load during dual write.

2023-05-26 Thread via GitHub
mumrah commented on PR #13757: URL: https://github.com/apache/kafka/pull/13757#issuecomment-1564750482 The jira for this is KAFKA-15017 btw -- 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

[GitHub] [kafka] mumrah commented on pull request #13766: KAFKA-14996: Limit partition count in Create Topic and Create Partitions

2023-05-26 Thread via GitHub
mumrah commented on PR #13766: URL: https://github.com/apache/kafka/pull/13766#issuecomment-1564748396 @edoardocomar, thanks for taking a look at this. This issue is a little bit tricky since PartitionRecords are not the only thing inside the topic creation batch. There's also a

[GitHub] [kafka] cmccabe commented on pull request #13742: KAFKA-14996: Handle overly large user operations on the kcontroller

2023-05-26 Thread via GitHub
cmccabe commented on PR #13742: URL: https://github.com/apache/kafka/pull/13742#issuecomment-1564712814 Hi all, Thanks for the reviews and comments. > @divijvaidya : But I was wondering if an additional guard could be to have a default TopicCreationPolicy with a MaxValue of X

[GitHub] [kafka] cmccabe commented on a diff in pull request #13742: KAFKA-14996: Handle overly large user operations on the kcontroller

2023-05-26 Thread via GitHub
cmccabe commented on code in PR #13742: URL: https://github.com/apache/kafka/pull/13742#discussion_r1207123242 ## metadata/src/main/java/org/apache/kafka/controller/QuorumController.java: ## @@ -457,9 +466,13 @@ private Throwable handleEventException(String name, long

[GitHub] [kafka] cmccabe commented on a diff in pull request #13742: KAFKA-14996: Handle overly large user operations on the kcontroller

2023-05-26 Thread via GitHub
cmccabe commented on code in PR #13742: URL: https://github.com/apache/kafka/pull/13742#discussion_r1207117766 ## metadata/src/main/java/org/apache/kafka/controller/QuorumController.java: ## @@ -160,6 +161,14 @@ * the controller can fully initialize. */ public final class

[GitHub] [kafka] cmccabe commented on a diff in pull request #13742: KAFKA-14996: Handle overly large user operations on the kcontroller

2023-05-26 Thread via GitHub
cmccabe commented on code in PR #13742: URL: https://github.com/apache/kafka/pull/13742#discussion_r1207117766 ## metadata/src/main/java/org/apache/kafka/controller/QuorumController.java: ## @@ -160,6 +161,14 @@ * the controller can fully initialize. */ public final class

[GitHub] [kafka] mumrah commented on a diff in pull request #13767: KAFKA-15004: Fix configuration dual-write during migration

2023-05-26 Thread via GitHub
mumrah commented on code in PR #13767: URL: https://github.com/apache/kafka/pull/13767#discussion_r1207096649 ## core/src/main/scala/kafka/zk/ZkMigrationClient.scala: ## @@ -145,44 +144,47 @@ class ZkMigrationClient( topicClient.iterateTopics(

[GitHub] [kafka] mumrah opened a new pull request, #13767: KAFKA-15004: Fix configuration dual-write during migration

2023-05-26 Thread via GitHub
mumrah opened a new pull request, #13767: URL: https://github.com/apache/kafka/pull/13767 This PR builds on top of #13736. Fixes the following: * Topic configs are not sycned while handling snapshot. * New broker/topic configs in KRaft that did not exist in ZK will not be

[jira] [Commented] (KAFKA-14996) The KRaft controller should properly handle overly large user operations

2023-05-26 Thread Edoardo Comar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14996?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17726688#comment-17726688 ] Edoardo Comar commented on KAFKA-14996: --- Opened a PR to allow responding gracefully with

[GitHub] [kafka] erikvanoosten commented on pull request #13678: KAFKA-10337: await async commits in commitSync even if no offsets given

2023-05-26 Thread via GitHub
erikvanoosten commented on PR #13678: URL: https://github.com/apache/kafka/pull/13678#issuecomment-1564684675 > @philipnee I will try to review it this week. Thanks! Hi @dajac, did you already get the chance to look at this PR? -- This is an automated message from the Apache Git

[GitHub] [kafka] edoardocomar opened a new pull request, #13766: KAFKA-14996: Limit partition count in Create Topic and Create Partitions

2023-05-26 Thread via GitHub
edoardocomar opened a new pull request, #13766: URL: https://github.com/apache/kafka/pull/13766 Limit partition count in Create Topic and Create Partitions to avoid exceeding the QuorumController's MAX_RECORDS_PER_BATCH, and respond gracefully to the client. -- This is an

[GitHub] [kafka] yashmayya commented on pull request #13465: KAFKA-14368: Connect offset write REST API

2023-05-26 Thread via GitHub
yashmayya commented on PR #13465: URL: https://github.com/apache/kafka/pull/13465#issuecomment-1564677170 Thanks Chris! -- 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

[GitHub] [kafka] splett2 commented on a diff in pull request #13765: KAFKA-15021; Skip leader epoch bump on ISR shrink

2023-05-26 Thread via GitHub
splett2 commented on code in PR #13765: URL: https://github.com/apache/kafka/pull/13765#discussion_r1207073563 ## core/src/main/scala/kafka/cluster/Partition.scala: ## @@ -1087,12 +1087,14 @@ class Partition(val topicPartition: TopicPartition, // avoid unnecessary

[GitHub] [kafka] C0urante merged pull request #13334: MINOR: Move plugin path parsing from DelegatingClassLoader to PluginUtils

2023-05-26 Thread via GitHub
C0urante merged PR #13334: URL: https://github.com/apache/kafka/pull/13334 -- 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:

[GitHub] [kafka] hachikuji commented on a diff in pull request #13267: KAFKA-14694: RPCProducerIdManager should not wait on new block

2023-05-26 Thread via GitHub
hachikuji commented on code in PR #13267: URL: https://github.com/apache/kafka/pull/13267#discussion_r1207028405 ## core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala: ## @@ -113,38 +142,113 @@ class ProducerIdManagerTest { }

[GitHub] [kafka] C0urante commented on a diff in pull request #13334: MINOR: Move plugin path parsing from DelegatingClassLoader to PluginUtils

2023-05-26 Thread via GitHub
C0urante commented on code in PR #13334: URL: https://github.com/apache/kafka/pull/13334#discussion_r1207028289 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginUtils.java: ## @@ -188,11 +191,34 @@ public static boolean isClassFile(Path path) {

[GitHub] [kafka] C0urante merged pull request #13356: KAFKA-14789: Prevent mis-attributing classpath plugins, allow discovery of classpath RestExtension and ConfigProvider

2023-05-26 Thread via GitHub
C0urante merged PR #13356: URL: https://github.com/apache/kafka/pull/13356 -- 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:

[GitHub] [kafka] rondagostino commented on a diff in pull request #13759: KAFKA-15019: Improve handling of overload situations in the kcontroller

2023-05-26 Thread via GitHub
rondagostino commented on code in PR #13759: URL: https://github.com/apache/kafka/pull/13759#discussion_r1207013139 ## metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java: ## @@ -223,6 +283,21 @@ public BrokerHeartbeatState next() { } }

[GitHub] [kafka] rondagostino commented on a diff in pull request #13759: KAFKA-15019: Improve handling of overload situations in the kcontroller

2023-05-26 Thread via GitHub
rondagostino commented on code in PR #13759: URL: https://github.com/apache/kafka/pull/13759#discussion_r1207013139 ## metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java: ## @@ -223,6 +283,21 @@ public BrokerHeartbeatState next() { } }

[GitHub] [kafka] rondagostino commented on a diff in pull request #13759: KAFKA-15019: Improve handling of overload situations in the kcontroller

2023-05-26 Thread via GitHub
rondagostino commented on code in PR #13759: URL: https://github.com/apache/kafka/pull/13759#discussion_r1207013139 ## metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java: ## @@ -223,6 +283,21 @@ public BrokerHeartbeatState next() { } }

[GitHub] [kafka] C0urante merged pull request #13465: KAFKA-14368: Connect offset write REST API

2023-05-26 Thread via GitHub
C0urante merged PR #13465: URL: https://github.com/apache/kafka/pull/13465 -- 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:

[GitHub] [kafka] rondagostino commented on a diff in pull request #13759: KAFKA-15019: Improve handling of overload situations in the kcontroller

2023-05-26 Thread via GitHub
rondagostino commented on code in PR #13759: URL: https://github.com/apache/kafka/pull/13759#discussion_r1207013139 ## metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java: ## @@ -223,6 +283,21 @@ public BrokerHeartbeatState next() { } }

[GitHub] [kafka] jolshan commented on a diff in pull request #13493: KAFKA-14852: Propagate Topic Ids to the Group Coordinator for Offset Fetch

2023-05-26 Thread via GitHub
jolshan commented on code in PR #13493: URL: https://github.com/apache/kafka/pull/13493#discussion_r1206991344 ## core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala: ## @@ -492,42 +492,59 @@ class GroupMetadataManager(brokerId: Int, * The most important

[GitHub] [kafka] C0urante commented on a diff in pull request #13504: KAFKA-14750: Check if topic exists in WorkerSinkTask when committing offsets.

2023-05-26 Thread via GitHub
C0urante commented on code in PR #13504: URL: https://github.com/apache/kafka/pull/13504#discussion_r1206885931 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java: ## @@ -695,9 +705,28 @@ private class HandleRebalance implements

[GitHub] [kafka] ijuma commented on a diff in pull request #13742: KAFKA-14996: Handle overly large user operations on the kcontroller

2023-05-26 Thread via GitHub
ijuma commented on code in PR #13742: URL: https://github.com/apache/kafka/pull/13742#discussion_r1206985698 ## metadata/src/main/java/org/apache/kafka/controller/QuorumController.java: ## @@ -457,9 +466,13 @@ private Throwable handleEventException(String name, long

[GitHub] [kafka] ijuma commented on a diff in pull request #13742: KAFKA-14996: Handle overly large user operations on the kcontroller

2023-05-26 Thread via GitHub
ijuma commented on code in PR #13742: URL: https://github.com/apache/kafka/pull/13742#discussion_r1206985698 ## metadata/src/main/java/org/apache/kafka/controller/QuorumController.java: ## @@ -457,9 +466,13 @@ private Throwable handleEventException(String name, long

[GitHub] [kafka] machi1990 commented on pull request #13611: MINOR: remove unused variable from QuorumMetaLogListener#handleCommit method

2023-05-26 Thread via GitHub
machi1990 commented on PR #13611: URL: https://github.com/apache/kafka/pull/13611#issuecomment-1564562097 Thanks @jsancio for the 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

[GitHub] [kafka] jsancio merged pull request #13611: MINOR: remove unused variable from QuorumMetaLogListener#handleCommit method

2023-05-26 Thread via GitHub
jsancio merged PR #13611: URL: https://github.com/apache/kafka/pull/13611 -- 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:

[GitHub] [kafka] mumrah commented on pull request #13742: KAFKA-14996: Handle overly large user operations on the kcontroller

2023-05-26 Thread via GitHub
mumrah commented on PR #13742: URL: https://github.com/apache/kafka/pull/13742#issuecomment-1564483362 @divijvaidya Colin can correct me if I'm mistaken, but I believe this patch is mainly about closing an existing edge case until we implement KIP-868 (metadata transactions). Once we have

[GitHub] [kafka] vvcephei merged pull request #13455: KAFKA-14841 Handle callbacks to ConsumerRebalanceListener in MockConsumer

2023-05-26 Thread via GitHub
vvcephei merged PR #13455: URL: https://github.com/apache/kafka/pull/13455 -- 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:

[GitHub] [kafka] machi1990 commented on pull request #13664: KAFKA-14961: harden DefaultBackgroundThreadTest.testStartupAndTearDown test

2023-05-26 Thread via GitHub
machi1990 commented on PR #13664: URL: https://github.com/apache/kafka/pull/13664#issuecomment-1564402596 Thanks @philipnee @vvcephei for the review and merge! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL

[GitHub] [kafka] vvcephei merged pull request #13664: KAFKA-14961: harden DefaultBackgroundThreadTest.testStartupAndTearDown test

2023-05-26 Thread via GitHub
vvcephei merged PR #13664: URL: https://github.com/apache/kafka/pull/13664 -- 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:

[GitHub] [kafka] Hangleton commented on a diff in pull request #13493: KAFKA-14852: Propagate Topic Ids to the Group Coordinator for Offset Fetch

2023-05-26 Thread via GitHub
Hangleton commented on code in PR #13493: URL: https://github.com/apache/kafka/pull/13493#discussion_r1206778975 ## core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala: ## @@ -492,42 +492,59 @@ class GroupMetadataManager(brokerId: Int, * The most

[GitHub] [kafka] dajac commented on pull request #13764: KAFKA-14691; [1/N] Add new fields to OffsetFetchRequest and OffsetFetchResponse

2023-05-26 Thread via GitHub
dajac commented on PR #13764: URL: https://github.com/apache/kafka/pull/13764#issuecomment-1564350280 I have a few comments/questions: * I am not really comfortable with merging this without the server side implementation. @clolov Is there a strong reason to not do them together? * I

[GitHub] [kafka] dajac commented on a diff in pull request #13493: KAFKA-14852: Propagate Topic Ids to the Group Coordinator for Offset Fetch

2023-05-26 Thread via GitHub
dajac commented on code in PR #13493: URL: https://github.com/apache/kafka/pull/13493#discussion_r1206732981 ## core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala: ## @@ -492,42 +492,59 @@ class GroupMetadataManager(brokerId: Int, * The most important

[GitHub] [kafka] Hangleton commented on a diff in pull request #13493: KAFKA-14852: Propagate Topic Ids to the Group Coordinator for Offset Fetch

2023-05-26 Thread via GitHub
Hangleton commented on code in PR #13493: URL: https://github.com/apache/kafka/pull/13493#discussion_r1206702136 ## core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala: ## @@ -492,42 +492,59 @@ class GroupMetadataManager(brokerId: Int, * The most

[GitHub] [kafka] Hangleton commented on pull request #13558: KAFKA-14845: Fix broker registration with Zookeeper when the previous ephemeral znode was not properly recorded by the broker

2023-05-26 Thread via GitHub
Hangleton commented on PR #13558: URL: https://github.com/apache/kafka/pull/13558#issuecomment-1564157619 Hi, Igor, thanks for the review. I added the changes you reminded me about above. I am going through an additional test runs for this integration test to ensure there is no

[GitHub] [kafka] mimaison commented on pull request #13748: [BUGFIX] Bugfixed in KAFKA-8713, but it doesn't work properly.

2023-05-26 Thread via GitHub
mimaison commented on PR #13748: URL: https://github.com/apache/kafka/pull/13748#issuecomment-1564113409 Good catch! Yes it would be good to have this in 3.5. @krespo can you update your PR with the suggestions from @gharris1727 ? Thanks -- This is an automated message from the

[jira] [Commented] (KAFKA-14953) Add metrics for tiered storage

2023-05-26 Thread Luke Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14953?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17726550#comment-17726550 ] Luke Chen commented on KAFKA-14953: --- Sounds good! Thank you! > Add metrics for tiered storage >

[GitHub] [kafka] dajac commented on a diff in pull request #13639: KAFKA-14462; [12/N] Add GroupMetadataManager and ConsumerGroup

2023-05-26 Thread via GitHub
dajac commented on code in PR #13639: URL: https://github.com/apache/kafka/pull/13639#discussion_r1206322259 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java: ## @@ -0,0 +1,876 @@ +/* + * Licensed to the Apache Software Foundation

[GitHub] [kafka] dajac commented on a diff in pull request #13639: KAFKA-14462; [12/N] Add GroupMetadataManager and ConsumerGroup

2023-05-26 Thread via GitHub
dajac commented on code in PR #13639: URL: https://github.com/apache/kafka/pull/13639#discussion_r1206317753 ## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java: ## @@ -0,0 +1,500 @@ +/* + * Licensed to the Apache Software