[GitHub] [kafka] ableegoldman commented on a change in pull request #11424: KAFKA-13152: Replace "buffered.records.per.partition" with "input.buffer.max.bytes"

2022-02-04 Thread GitBox
ableegoldman commented on a change in pull request #11424: URL: https://github.com/apache/kafka/pull/11424#discussion_r799253840 ## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/TopologyConfig.java ## @@ -116,14 +123,30 @@ public T

[GitHub] [kafka] ableegoldman commented on a change in pull request #11424: KAFKA-13152: Replace "buffered.records.per.partition" with "input.buffer.max.bytes"

2022-02-04 Thread GitBox
ableegoldman commented on a change in pull request #11424: URL: https://github.com/apache/kafka/pull/11424#discussion_r799261397 ## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ## @@ -1034,14 +1037,15 @@ private static Metrics getMetrics(final Str

[GitHub] [kafka] predatorray commented on pull request #10525: KAFKA-7572: Producer should not send requests with negative partition id

2022-02-04 Thread GitBox
predatorray commented on pull request #10525: URL: https://github.com/apache/kafka/pull/10525#issuecomment-1029767796 @guozhangwang Thanks for the reply and the review. The change was rebased. -- This is an automated message from the Apache Git Service. To respond to the message, please l

[GitHub] [kafka] satishd commented on a change in pull request #11390: [KAFKA-13369] Follower fetch protocol changes for tiered storage.

2022-02-04 Thread GitBox
satishd commented on a change in pull request #11390: URL: https://github.com/apache/kafka/pull/11390#discussion_r796465473 ## File path: clients/src/main/java/org/apache/kafka/common/record/RemoteLogInputStream.java ## @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software

[jira] [Comment Edited] (KAFKA-12635) Mirrormaker 2 offset sync is incorrect if the target partition is empty

2022-02-04 Thread Federico Valeri (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12635?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17486576#comment-17486576 ] Federico Valeri edited comment on KAFKA-12635 at 2/4/22, 8:51 AM:

[jira] (KAFKA-12635) Mirrormaker 2 offset sync is incorrect if the target partition is empty

2022-02-04 Thread Federico Valeri (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12635 ] Federico Valeri deleted comment on KAFKA-12635: - was (Author: fvaleri): I was able to reproduce the issue on Kafka 2.7.2 and 2.8.1, but not on 3.1.0. State of the source cluster after pr

[jira] [Commented] (KAFKA-13626) NullPointerException in Selector.pollSelectionKeys: channel is null

2022-02-04 Thread Jira
[ https://issues.apache.org/jira/browse/KAFKA-13626?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17486916#comment-17486916 ] Daniel Häuser commented on KAFKA-13626: --- [~Kvicii]  I don't know exactly what hap

[GitHub] [kafka] ijuma commented on a change in pull request #11722: KAFKA-13630: reduce amount of time that producer network thread holds batch queue lock

2022-02-04 Thread GitBox
ijuma commented on a change in pull request #11722: URL: https://github.com/apache/kafka/pull/11722#discussion_r799428683 ## File path: clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java ## @@ -578,41 +587,45 @@ private boolean shouldStop

[GitHub] [kafka] ijuma commented on a change in pull request #11721: KAFKA-13629: use faster algorithm for ByteUtils sizeOfXxx algorithm

2022-02-04 Thread GitBox
ijuma commented on a change in pull request #11721: URL: https://github.com/apache/kafka/pull/11721#discussion_r799456068 ## File path: clients/src/test/java/org/apache/kafka/common/utils/ByteUtilsTest.java ## @@ -239,6 +239,66 @@ public void testDouble() throws IOException {

[GitHub] [kafka] ijuma commented on a change in pull request #11722: KAFKA-13630: reduce amount of time that producer network thread holds batch queue lock

2022-02-04 Thread GitBox
ijuma commented on a change in pull request #11722: URL: https://github.com/apache/kafka/pull/11722#discussion_r799462063 ## File path: clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java ## @@ -448,40 +448,48 @@ public ReadyCheckResult rea

[jira] [Commented] (KAFKA-13600) Rebalances while streams is in degraded state can cause stores to be reassigned and restore from scratch

2022-02-04 Thread Bruno Cadonna (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13600?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17487148#comment-17487148 ] Bruno Cadonna commented on KAFKA-13600: --- I am fine with discussing the improvement

[jira] [Commented] (KAFKA-13638) Slow KTable update when forwarding multiple values from transformer

2022-02-04 Thread Bruno Cadonna (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13638?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17487184#comment-17487184 ] Bruno Cadonna commented on KAFKA-13638: --- [~Lejon] How did you measure time with yo

[jira] [Commented] (KAFKA-13638) Slow KTable update when forwarding multiple values from transformer

2022-02-04 Thread Ulrik (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13638?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17487211#comment-17487211 ] Ulrik commented on KAFKA-13638: --- [~cadonna] i ran the test from within Intellij and just t

[GitHub] [kafka] hachikuji commented on a change in pull request #11691: KAFKA-13598: enable idempotence producer by default and validate the configs

2022-02-04 Thread GitBox
hachikuji commented on a change in pull request #11691: URL: https://github.com/apache/kafka/pull/11691#discussion_r799691615 ## File path: clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java ## @@ -514,12 +512,11 @@ public ProducerConfig(Map props) {

[GitHub] [kafka] ijuma commented on a change in pull request #11691: KAFKA-13598: enable idempotence producer by default and validate the configs

2022-02-04 Thread GitBox
ijuma commented on a change in pull request #11691: URL: https://github.com/apache/kafka/pull/11691#discussion_r799697679 ## File path: docs/upgrade.html ## @@ -19,6 +19,13 @@

[GitHub] [kafka] cmccabe commented on pull request #11659: KAFKA-13503: Validate broker configs for KRaft

2022-02-04 Thread GitBox
cmccabe commented on pull request #11659: URL: https://github.com/apache/kafka/pull/11659#issuecomment-1030241564 Thanks for the PR, @dengziming. The broker validation needs to be done on the broker side, since the broker has access to information that the controller does not. For ex

[jira] [Resolved] (KAFKA-13503) Validate broker configs for KRaft

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13503?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-13503. -- Fix Version/s: 3.2.0 Reviewer: Jose Armando Garcia Sancio Assignee: Colin McC

[jira] [Updated] (KAFKA-13552) Unable to dynamically change broker log levels on KRaft

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13552?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe updated KAFKA-13552: - Fix Version/s: 3.2.0 > Unable to dynamically change broker log levels on KRaft > ---

[jira] [Commented] (KAFKA-13503) Validate broker configs for KRaft

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13503?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17487221#comment-17487221 ] Colin McCabe commented on KAFKA-13503: -- The PR for KAFKA-13552 added broker-side va

[GitHub] [kafka] guozhangwang commented on a change in pull request #11424: KAFKA-13152: Replace "buffered.records.per.partition" with "input.buffer.max.bytes"

2022-02-04 Thread GitBox
guozhangwang commented on a change in pull request #11424: URL: https://github.com/apache/kafka/pull/11424#discussion_r799712506 ## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ## @@ -1034,14 +1037,15 @@ private static Metrics getMetrics(final Str

[jira] [Resolved] (KAFKA-9154) ProducerId generation should be managed by the Controller

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9154?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-9154. - Fix Version/s: 3.1.0 Resolution: Fixed We implemented KIP-730 in Kafka 3.1. Closing this.

[jira] [Commented] (KAFKA-9837) New RPC for notifying controller of failed replica

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17487226#comment-17487226 ] Colin McCabe commented on KAFKA-9837: - I'm sorry that there hasn't been any activity

[jira] [Commented] (KAFKA-10724) Command to run single quorum in raft is missing "--config" parameters.

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10724?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17487230#comment-17487230 ] Colin McCabe commented on KAFKA-10724: -- The command has been renamed to {{raft/bin/

[jira] [Assigned] (KAFKA-10724) Command to run single quorum in raft is missing "--config" parameters.

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10724?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe reassigned KAFKA-10724: Assignee: Jason Gustafson > Command to run single quorum in raft is missing "--config" pa

[jira] [Resolved] (KAFKA-10724) Command to run single quorum in raft is missing "--config" parameters.

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10724?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-10724. -- Resolution: Fixed > Command to run single quorum in raft is missing "--config" parameters. > -

[jira] [Resolved] (KAFKA-12209) Add the timeline data structures for the KIP-631 controller

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12209?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12209. -- Fix Version/s: 2.8.0 Assignee: Colin McCabe Resolution: Fixed > Add the timeli

[jira] [Resolved] (KAFKA-12214) Generated code does not include UUID or struct fields in its toString output

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12214?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12214. -- Resolution: Fixed > Generated code does not include UUID or struct fields in its toString outp

[jira] [Resolved] (KAFKA-12271) Expose consistent Raft metadata to components

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12271?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12271. -- Fix Version/s: 3.0.0 Resolution: Fixed > Expose consistent Raft metadata to components

[jira] [Commented] (KAFKA-12421) Improve controller's atomic grouping

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17487233#comment-17487233 ] Colin McCabe commented on KAFKA-12421: -- We did introduce a distinction between atom

[jira] [Resolved] (KAFKA-12421) Improve controller's atomic grouping

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12421?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12421. -- Assignee: Jose Armando Garcia Sancio (was: HaiyuanZhao) Resolution: Fixed > Improve con

[jira] [Assigned] (KAFKA-12502) Quorum controller should return topic configs in CreateTopic response

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12502?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe reassigned KAFKA-12502: Assignee: Colin McCabe (was: Ryan Dielhenn) > Quorum controller should return topic conf

[jira] [Created] (KAFKA-13646) Implement KIP-801: KRaft authorizer

2022-02-04 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13646: Summary: Implement KIP-801: KRaft authorizer Key: KAFKA-13646 URL: https://issues.apache.org/jira/browse/KAFKA-13646 Project: Kafka Issue Type: Improvement

[jira] [Resolved] (KAFKA-13193) Replica manager doesn't update partition state when transitioning from leader to follower with unknown leader

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-13193. -- Resolution: Fixed > Replica manager doesn't update partition state when transitioning from lea

[GitHub] [kafka] jasonk000 commented on pull request #11721: KAFKA-13629: use faster algorithm for ByteUtils sizeOfXxx algorithm

2022-02-04 Thread GitBox
jasonk000 commented on pull request #11721: URL: https://github.com/apache/kafka/pull/11721#issuecomment-1030278393 Thanks @ijuma , I believe I've sorted these out in [e891ebf](https://github.com/apache/kafka/pull/11721/commits/e891ebfd46ed97ab1c6face21f3d7f6565734a77). -- This is an aut

[GitHub] [kafka] jasonk000 commented on a change in pull request #11721: KAFKA-13629: use faster algorithm for ByteUtils sizeOfXxx algorithm

2022-02-04 Thread GitBox
jasonk000 commented on a change in pull request #11721: URL: https://github.com/apache/kafka/pull/11721#discussion_r799747533 ## File path: clients/src/test/java/org/apache/kafka/common/utils/ByteUtilsTest.java ## @@ -239,6 +239,66 @@ public void testDouble() throws IOExceptio

[GitHub] [kafka] jasonk000 commented on a change in pull request #11721: KAFKA-13629: use faster algorithm for ByteUtils sizeOfXxx algorithm

2022-02-04 Thread GitBox
jasonk000 commented on a change in pull request #11721: URL: https://github.com/apache/kafka/pull/11721#discussion_r799748066 ## File path: clients/src/test/java/org/apache/kafka/common/utils/ByteUtilsTest.java ## @@ -239,6 +239,66 @@ public void testDouble() throws IOExceptio

[GitHub] [kafka] dajac commented on pull request #11543: Update release.py

2022-02-04 Thread GitBox
dajac commented on pull request #11543: URL: https://github.com/apache/kafka/pull/11543#issuecomment-1030295682 @shharrnam Thanks for the PR. Could you explain why we would need to add this new line? That does not seem necessary to me. -- This is an automated message from the Apache Git

[GitHub] [kafka] dajac closed pull request #11594: How to define the quantity of consumption groups

2022-02-04 Thread GitBox
dajac closed pull request #11594: URL: https://github.com/apache/kafka/pull/11594 -- 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...

[GitHub] [kafka] dajac commented on pull request #11594: How to define the quantity of consumption groups

2022-02-04 Thread GitBox
dajac commented on pull request #11594: URL: https://github.com/apache/kafka/pull/11594#issuecomment-1030299168 @ayu-programer It depends on your use case. You could consume all the topics from the same group if the application needs to process all of them together. Otherwise, you can defi

[GitHub] [kafka] dajac commented on pull request #11353: KAFKA-13322: Reducing amount of garbage that gets generated during a poll operation

2022-02-04 Thread GitBox
dajac commented on pull request #11353: URL: https://github.com/apache/kafka/pull/11353#issuecomment-1030306522 @mprusakov Are you still interested by doing this? I can help reviewing the PR once the existing comments are addressed. Thanks. -- This is an automated message from the Apache

[GitHub] [kafka] dajac commented on pull request #10562: MINOR: Update tests to include the 2.8.0 release

2022-02-04 Thread GitBox
dajac commented on pull request #10562: URL: https://github.com/apache/kafka/pull/10562#issuecomment-1030315113 Closing as this was done by another 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

[GitHub] [kafka] dajac closed pull request #10562: MINOR: Update tests to include the 2.8.0 release

2022-02-04 Thread GitBox
dajac closed pull request #10562: URL: https://github.com/apache/kafka/pull/10562 -- 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...

[GitHub] [kafka] dajac commented on pull request #9963: MINOR: Extract ApiVersions logic from the `SocketServer` to the `KafkaApis`

2022-02-04 Thread GitBox
dajac commented on pull request #9963: URL: https://github.com/apache/kafka/pull/9963#issuecomment-1030320056 Closing. -- 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

[GitHub] [kafka] dajac closed pull request #9963: MINOR: Extract ApiVersions logic from the `SocketServer` to the `KafkaApis`

2022-02-04 Thread GitBox
dajac closed pull request #9963: URL: https://github.com/apache/kafka/pull/9963 -- 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...@k

[GitHub] [kafka] dajac closed pull request #9402: KAFKA-10588 update console consumer arguments for KIP-629

2022-02-04 Thread GitBox
dajac closed pull request #9402: URL: https://github.com/apache/kafka/pull/9402 -- 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...@k

[GitHub] [kafka] dajac commented on pull request #9402: KAFKA-10588 update console consumer arguments for KIP-629

2022-02-04 Thread GitBox
dajac commented on pull request #9402: URL: https://github.com/apache/kafka/pull/9402#issuecomment-1030324045 This was done by https://github.com/apache/kafka/pull/11008. Closing it. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to G

[GitHub] [kafka] dajac commented on pull request #9289: Dev/pasriva/json acl

2022-02-04 Thread GitBox
dajac commented on pull request #9289: URL: https://github.com/apache/kafka/pull/9289#issuecomment-1030325227 I am sure what this PR is doing. It seems to be a mistake. Closing it. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to Git

[GitHub] [kafka] dajac closed pull request #9289: Dev/pasriva/json acl

2022-02-04 Thread GitBox
dajac closed pull request #9289: URL: https://github.com/apache/kafka/pull/9289 -- 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...@k

[GitHub] [kafka] dajac closed pull request #9085: MINOR: Support java.util.Optional in the auto-generated protocol

2022-02-04 Thread GitBox
dajac closed pull request #9085: URL: https://github.com/apache/kafka/pull/9085 -- 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...@k

[GitHub] [kafka] dajac commented on pull request #9085: MINOR: Support java.util.Optional in the auto-generated protocol

2022-02-04 Thread GitBox
dajac commented on pull request #9085: URL: https://github.com/apache/kafka/pull/9085#issuecomment-1030326392 PR is outdated. Closing it. -- 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

[GitHub] [kafka] dajac closed pull request #7456: KAFKA-8997; Make Errors a first class type in the auto-generated protocol.

2022-02-04 Thread GitBox
dajac closed pull request #7456: URL: https://github.com/apache/kafka/pull/7456 -- 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...@k

[GitHub] [kafka] jasonk000 commented on a change in pull request #11722: KAFKA-13630: reduce amount of time that producer network thread holds batch queue lock

2022-02-04 Thread GitBox
jasonk000 commented on a change in pull request #11722: URL: https://github.com/apache/kafka/pull/11722#discussion_r799796771 ## File path: clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java ## @@ -448,40 +448,48 @@ public ReadyCheckResult

[GitHub] [kafka] mprusakov closed pull request #11353: KAFKA-13322: Reducing amount of garbage that gets generated during a poll operation

2022-02-04 Thread GitBox
mprusakov closed pull request #11353: URL: https://github.com/apache/kafka/pull/11353 -- 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-unsubsc

[GitHub] [kafka] mprusakov commented on pull request #11353: KAFKA-13322: Reducing amount of garbage that gets generated during a poll operation

2022-02-04 Thread GitBox
mprusakov commented on pull request #11353: URL: https://github.com/apache/kafka/pull/11353#issuecomment-1030340572 Unfortunately i have realized that this fix is a drop in the ocean and further changes are much more intrusive. This change alone did not meet my client's requirements so I'v

[jira] [Resolved] (KAFKA-13322) Java client produces a large amount of garbage during a poll

2022-02-04 Thread Michail (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13322?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michail resolved KAFKA-13322. - Resolution: Won't Fix Having done the analysis the changes required to achieve this are far too intrusi

[GitHub] [kafka] hachikuji commented on a change in pull request #11667: MINOR; Enable Kraft in ApiVersionTest

2022-02-04 Thread GitBox
hachikuji commented on a change in pull request #11667: URL: https://github.com/apache/kafka/pull/11667#discussion_r799823851 ## File path: core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala ## @@ -17,41 +17,35 @@ package kafka.server -import kafka.test.{Cl

[GitHub] [kafka] hachikuji commented on pull request #11685: Update dependencies.gradle

2022-02-04 Thread GitBox
hachikuji commented on pull request #11685: URL: https://github.com/apache/kafka/pull/11685#issuecomment-1030368363 @shubhamsingh002 Can you clarify your intent here? The upgrade to log4j 2 is not compatible and we are not planning to release from 0.10 in any case as far as I know. -- T

[GitHub] [kafka] ijuma commented on a change in pull request #11722: KAFKA-13630: reduce amount of time that producer network thread holds batch queue lock

2022-02-04 Thread GitBox
ijuma commented on a change in pull request #11722: URL: https://github.com/apache/kafka/pull/11722#discussion_r799828526 ## File path: clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java ## @@ -448,40 +448,48 @@ public ReadyCheckResult rea

[GitHub] [kafka] hachikuji commented on pull request #11646: KAFKA-13566: producer exponential backoff implementation for KIP-580

2022-02-04 Thread GitBox
hachikuji commented on pull request #11646: URL: https://github.com/apache/kafka/pull/11646#issuecomment-1030373051 @showuon Thanks for the PR and apologies for the delay. It looks to me like this is covering both the consumer and producer? Would it be possible to separate into two PRs?

[GitHub] [kafka] hachikuji closed pull request #11685: Update dependencies.gradle

2022-02-04 Thread GitBox
hachikuji closed pull request #11685: URL: https://github.com/apache/kafka/pull/11685 -- 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-unsubsc

[GitHub] [kafka] hachikuji commented on pull request #11685: Update dependencies.gradle

2022-02-04 Thread GitBox
hachikuji commented on pull request #11685: URL: https://github.com/apache/kafka/pull/11685#issuecomment-1030374498 I am going to go ahead and close this. Please feel free to reopen after you have updated the description to explain what you are trying to do. -- This is an automated messa

[GitHub] [kafka] hachikuji commented on a change in pull request #11620: MINOR: check for raft threads in verifyNoUnexpectedThreads

2022-02-04 Thread GitBox
hachikuji commented on a change in pull request #11620: URL: https://github.com/apache/kafka/pull/11620#discussion_r799835234 ## File path: core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala ## @@ -340,6 +340,7 @@ abstract class QuorumTestHarness extends Loggi

[GitHub] [kafka] ijuma commented on a change in pull request #11721: KAFKA-13629: use faster algorithm for ByteUtils sizeOfXxx algorithm

2022-02-04 Thread GitBox
ijuma commented on a change in pull request #11721: URL: https://github.com/apache/kafka/pull/11721#discussion_r799833387 ## File path: clients/src/main/java/org/apache/kafka/common/utils/ByteUtils.java ## @@ -413,15 +424,20 @@ public static int sizeOfVarint(int value) {

[GitHub] [kafka] ijuma commented on a change in pull request #11721: KAFKA-13629: use faster algorithm for ByteUtils sizeOfXxx algorithm

2022-02-04 Thread GitBox
ijuma commented on a change in pull request #11721: URL: https://github.com/apache/kafka/pull/11721#discussion_r799833387 ## File path: clients/src/main/java/org/apache/kafka/common/utils/ByteUtils.java ## @@ -413,15 +424,20 @@ public static int sizeOfVarint(int value) {

[GitHub] [kafka] hachikuji commented on a change in pull request #11688: KAFKA-13435; Static membership protocol should let the leader skip assignment (KIP-814)

2022-02-04 Thread GitBox
hachikuji commented on a change in pull request #11688: URL: https://github.com/apache/kafka/pull/11688#discussion_r799847085 ## File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java ## @@ -696,8 +698,12 @@ public void handle(Joi

[GitHub] [kafka] lbradstreet commented on a change in pull request #11620: MINOR: check for raft threads in verifyNoUnexpectedThreads

2022-02-04 Thread GitBox
lbradstreet commented on a change in pull request #11620: URL: https://github.com/apache/kafka/pull/11620#discussion_r799864255 ## File path: core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala ## @@ -340,6 +340,7 @@ abstract class QuorumTestHarness extends Log

[GitHub] [kafka] hachikuji opened a new pull request #11734: MINOR: Do not use optional args in `ProducerStateManager`

2022-02-04 Thread GitBox
hachikuji opened a new pull request #11734: URL: https://github.com/apache/kafka/pull/11734 We allowed `maxProducerIdExpirationMs` and `time` to be optional in the `ProducerStateManager` constructor. We generally frown on optional arguments since it is too easy to overlook them. In this ca

[GitHub] [kafka] hachikuji merged pull request #11694: MINOR: deleteHorizonMs update to documentation and DumpLogSegments tool

2022-02-04 Thread GitBox
hachikuji merged pull request #11694: URL: https://github.com/apache/kafka/pull/11694 -- 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-unsubsc

[GitHub] [kafka] jasonk000 commented on a change in pull request #11722: KAFKA-13630: reduce amount of time that producer network thread holds batch queue lock

2022-02-04 Thread GitBox
jasonk000 commented on a change in pull request #11722: URL: https://github.com/apache/kafka/pull/11722#discussion_r799904384 ## File path: clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java ## @@ -448,40 +448,48 @@ public ReadyCheckResult

[GitHub] [kafka] jasonk000 commented on a change in pull request #11722: KAFKA-13630: reduce amount of time that producer network thread holds batch queue lock

2022-02-04 Thread GitBox
jasonk000 commented on a change in pull request #11722: URL: https://github.com/apache/kafka/pull/11722#discussion_r799905219 ## File path: clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java ## @@ -448,40 +448,48 @@ public ReadyCheckResult

[jira] [Updated] (KAFKA-12984) Cooperative sticky assignor can get stuck with invalid SubscriptionState input metadata

2022-02-04 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12984?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman updated KAFKA-12984: --- Fix Version/s: 2.5.2 2.6.2 2.7.1 > Coo

[jira] [Updated] (KAFKA-12984) Cooperative sticky assignor can get stuck with invalid SubscriptionState input metadata

2022-02-04 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12984?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman updated KAFKA-12984: --- Fix Version/s: 2.7.2 2.6.3 (was: 2.

[GitHub] [kafka] ableegoldman commented on a change in pull request #11712: WIP: Put failed tasks to end of processing list

2022-02-04 Thread GitBox
ableegoldman commented on a change in pull request #11712: URL: https://github.com/apache/kafka/pull/11712#discussion_r799919908 ## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java ## @@ -270,6 +278,23 @@ Task task(final TaskId taskId) {

[GitHub] [kafka] jasonk000 commented on pull request #11721: KAFKA-13629: use faster algorithm for ByteUtils sizeOfXxx algorithm

2022-02-04 Thread GitBox
jasonk000 commented on pull request #11721: URL: https://github.com/apache/kafka/pull/11721#issuecomment-1030489854 I've addressed comments, I'll wait and see what spotbugs says this time. Locally, it's all clear, on CI it shows up issues with code not related to this PR. -- This is an

[jira] [Resolved] (KAFKA-13346) Kafka Streams fails due to RocksDB Locks Not Available Exception

2022-02-04 Thread Guozhang Wang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13346?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-13346. --- Resolution: Not A Problem > Kafka Streams fails due to RocksDB Locks Not Available Exception

[jira] [Commented] (KAFKA-13346) Kafka Streams fails due to RocksDB Locks Not Available Exception

2022-02-04 Thread Guozhang Wang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13346?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17487405#comment-17487405 ] Guozhang Wang commented on KAFKA-13346: --- I'm closing this ticket for now since we

[GitHub] [kafka] Kvicii commented on pull request #11543: Update release.py

2022-02-04 Thread GitBox
Kvicii commented on pull request #11543: URL: https://github.com/apache/kafka/pull/11543#issuecomment-1030526684 @dajac This is unnecessary, I think this PR can be closed. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub a

[GitHub] [kafka] ijuma commented on pull request #11586: KAFKA-13516: Connection level metrics are not closed

2022-02-04 Thread GitBox
ijuma commented on pull request #11586: URL: https://github.com/apache/kafka/pull/11586#issuecomment-1030531283 Cc @apovzner @splett2 -- 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

[GitHub] [kafka] ijuma commented on pull request #11579: KAFKA-13518: Update gson dependency

2022-02-04 Thread GitBox
ijuma commented on pull request #11579: URL: https://github.com/apache/kafka/pull/11579#issuecomment-1030531381 Thanks for the PR. Seems like the new version has more false positives. Do you know if they intend to fix those? -- This is an automated message from the Apache Git Service. To

[GitHub] [kafka] ijuma closed pull request #11543: Update release.py

2022-02-04 Thread GitBox
ijuma closed pull request #11543: URL: https://github.com/apache/kafka/pull/11543 -- 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...

[GitHub] [kafka] ijuma commented on pull request #11469: MINOR: disable zookeeper.sasl.client to avoid false error

2022-02-04 Thread GitBox
ijuma commented on pull request #11469: URL: https://github.com/apache/kafka/pull/11469#issuecomment-1030532433 Any reason why this wasn't merged? -- 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

[GitHub] [kafka] ijuma closed pull request #11377: MINOR: Use try-with-resource to close the stream opened by Files.list()

2022-02-04 Thread GitBox
ijuma closed pull request #11377: URL: https://github.com/apache/kafka/pull/11377 -- 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...

[GitHub] [kafka] ijuma commented on pull request #11376: KAFKA-13342: LeaderAndIsrRequest should not be sent for topic queued for deletion

2022-02-04 Thread GitBox
ijuma commented on pull request #11376: URL: https://github.com/apache/kafka/pull/11376#issuecomment-1030532699 Are you planning to address the comments @lbradstreet ? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use t

[GitHub] [kafka] ijuma commented on pull request #11137: KAFKA-13133 Replace EasyMock and PowerMock with Mockito for AbstractHerderTest

2022-02-04 Thread GitBox
ijuma commented on pull request #11137: URL: https://github.com/apache/kafka/pull/11137#issuecomment-1030532879 @wycc are you planning to address the comments? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the U

[GitHub] [kafka] ijuma commented on pull request #11159: MINOR: Fix logging in ClusterControlManager

2022-02-04 Thread GitBox
ijuma commented on pull request #11159: URL: https://github.com/apache/kafka/pull/11159#issuecomment-1030533071 @jsancio are you going to merge this? -- 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

[GitHub] [kafka] ijuma commented on pull request #11161: MINOR: Remove node from API versions cache on NetworkClient.close(nodeId)

2022-02-04 Thread GitBox
ijuma commented on pull request #11161: URL: https://github.com/apache/kafka/pull/11161#issuecomment-1030533145 @rajinisivaram friendly ping. -- 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] ijuma commented on a change in pull request #11204: KAFKA-13188 - Release the memory back into MemoryPool

2022-02-04 Thread GitBox
ijuma commented on a change in pull request #11204: URL: https://github.com/apache/kafka/pull/11204#discussion_r78166 ## File path: clients/src/main/java/org/apache/kafka/clients/ClientResponseWithFinalize.java ## @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Fo

[GitHub] [kafka] ijuma commented on pull request #11229: KAFKA-12961; Verify group generation in `DelayedJoin`

2022-02-04 Thread GitBox
ijuma commented on pull request #11229: URL: https://github.com/apache/kafka/pull/11229#issuecomment-1030533521 @dajac is this still relevant? -- 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] ijuma commented on pull request #11238: MINOR: Fix force kill of KRaft colocated controllers in system tests

2022-02-04 Thread GitBox
ijuma commented on pull request #11238: URL: https://github.com/apache/kafka/pull/11238#issuecomment-1030533632 Cc @jsancio @cmccabe -- 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

[GitHub] [kafka] showuon closed pull request #11661: [WIP] MINOR: shutdown thread test

2022-02-04 Thread GitBox
showuon closed pull request #11661: URL: https://github.com/apache/kafka/pull/11661 -- 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.

[GitHub] [kafka] showuon commented on pull request #11646: [WIP] KAFKA-13566: producer exponential backoff implementation for KIP-580

2022-02-04 Thread GitBox
showuon commented on pull request #11646: URL: https://github.com/apache/kafka/pull/11646#issuecomment-1030567555 @hachikuji , thanks for the comment. Yes, actually I've separated producer and consumer into 2 PRs. (actually, there's 3rd one for adminClient). It's just there are still some

[GitHub] [kafka] ableegoldman commented on a change in pull request #11712: WIP: Put failed tasks to end of processing list

2022-02-04 Thread GitBox
ableegoldman commented on a change in pull request #11712: URL: https://github.com/apache/kafka/pull/11712#discussion_r800029186 ## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java ## @@ -270,6 +278,23 @@ Task task(final TaskId taskId) {

[GitHub] [kafka] showuon commented on a change in pull request #11691: KAFKA-13598: enable idempotence producer by default and validate the configs

2022-02-04 Thread GitBox
showuon commented on a change in pull request #11691: URL: https://github.com/apache/kafka/pull/11691#discussion_r800029493 ## File path: docs/upgrade.html ## @@ -19,6 +19,13 @@

[GitHub] [kafka] showuon commented on a change in pull request #11691: KAFKA-13598: enable idempotence producer by default and validate the configs

2022-02-04 Thread GitBox
showuon commented on a change in pull request #11691: URL: https://github.com/apache/kafka/pull/11691#discussion_r800030072 ## File path: clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java ## @@ -514,12 +512,11 @@ public ProducerConfig(Map props) {

[GitHub] [kafka] dajac commented on a change in pull request #11586: KAFKA-13516: Connection level metrics are not closed

2022-02-04 Thread GitBox
dajac commented on a change in pull request #11586: URL: https://github.com/apache/kafka/pull/11586#discussion_r800030837 ## File path: clients/src/main/java/org/apache/kafka/common/network/Selector.java ## @@ -1144,7 +1149,11 @@ public void close() { public SelectorMe

[GitHub] [kafka] runom opened a new pull request #11735: MINOR: Check the help and version options firstly

2022-02-04 Thread GitBox
runom opened a new pull request #11735: URL: https://github.com/apache/kafka/pull/11735 Currently, `bin/kafka-consumer-groups.sh --version` reqiures unnecessary `bootstrap-server` option. ``` % bin/kafka-consumer-groups.sh --version Missing required argument "[bootstrap-server]"