[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API

2021-03-08 Thread GitBox
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r590038851 ## File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersResult.java ## @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Soft

[GitHub] [kafka] hachikuji commented on pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API

2021-03-08 Thread GitBox
hachikuji commented on pull request #10275: URL: https://github.com/apache/kafka/pull/10275#issuecomment-793492376 @chia7712 Thanks for the comments. My initial goal is just the KIP-664 APIs since they cover most (maybe even all) of the complex admin workflows. After that, if we like the n

[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API

2021-03-08 Thread GitBox
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r590027837 ## File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java ## @@ -0,0 +1,464 @@ +/* + * Licensed to the Apache So

[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API

2021-03-08 Thread GitBox
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r590025229 ## File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java ## @@ -0,0 +1,464 @@ +/* + * Licensed to the Apache So

[jira] [Created] (KAFKA-12441) Remove deprecated methods StreamsBuilder#addGlobalStore

2021-03-08 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-12441: --- Summary: Remove deprecated methods StreamsBuilder#addGlobalStore Key: KAFKA-12441 URL: https://issues.apache.org/jira/browse/KAFKA-12441 Project: Kafka

[jira] [Commented] (KAFKA-7641) Add `consumer.group.max.size` to cap consumer metadata size on broker

2021-03-08 Thread Justin Jack (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7641?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297891#comment-17297891 ] Justin Jack commented on KAFKA-7641: So sorry Stanislav for the confusion on my wordi

[GitHub] [kafka] chia7712 opened a new pull request #10283: MINOR: KRPC supports to get true type from entity type

2021-03-08 Thread GitBox
chia7712 opened a new pull request #10283: URL: https://github.com/apache/kafka/pull/10283 Except for array type, the usage of `EntityType` is almost same to `type`. It makes our KRPC json file verbose. We can add a bit sugar to our KRPC that `EntityType` can be converted to true type in g

[GitHub] [kafka] junrao commented on a change in pull request #10218: KAFKA-12368: Added inmemory implementations for RemoteStorageManager and RemoteLogMetadataManager.

2021-03-08 Thread GitBox
junrao commented on a change in pull request #10218: URL: https://github.com/apache/kafka/pull/10218#discussion_r589867244 ## File path: clients/src/main/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java ## @@ -0,0 +1,170 @@ +/* + * Licensed

[GitHub] [kafka] chia7712 merged pull request #10262: MINOR: Add missing log argument

2021-03-08 Thread GitBox
chia7712 merged pull request #10262: URL: https://github.com/apache/kafka/pull/10262 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] chia7712 commented on pull request #9758: MINOR: remove FetchResponse.AbortedTransaction and redundant construc…

2021-03-08 Thread GitBox
chia7712 commented on pull request #9758: URL: https://github.com/apache/kafka/pull/9758#issuecomment-793348223 > I noticed that all of these JIRAs are listed as open, but they reference this PR. Are there further changes to be made? Asking because I'm working on similar code for #9944 an

[jira] [Assigned] (KAFKA-12440) ClusterId validation for Vote, BeginQourum and EndQuorum

2021-03-08 Thread dengziming (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12440?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] dengziming reassigned KAFKA-12440: -- Assignee: dengziming > ClusterId validation for Vote, BeginQourum and EndQuorum > ---

[jira] [Commented] (KAFKA-12313) Consider deprecating the default.windowed.serde.inner.class configs

2021-03-08 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12313?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297818#comment-17297818 ] A. Sophie Blee-Goldman commented on KAFKA-12313: By the way, even though

[jira] [Commented] (KAFKA-12440) ClusterId validation for Vote, BeginQourum and EndQuorum

2021-03-08 Thread dengziming (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12440?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297817#comment-17297817 ] dengziming commented on KAFKA-12440: Thank you [~jagsancio], I will try to do this s

[jira] [Commented] (KAFKA-12313) Consider deprecating the default.windowed.serde.inner.class configs

2021-03-08 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12313?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297816#comment-17297816 ] A. Sophie Blee-Goldman commented on KAFKA-12313: Sure, feel free to kick

[GitHub] [kafka] jsancio commented on pull request #10276: Kafka 12253: Add tests that cover all of all of the cases for ReplicatedLog-validateOffsetAndEpoch

2021-03-08 Thread GitBox
jsancio commented on pull request #10276: URL: https://github.com/apache/kafka/pull/10276#issuecomment-793276622 > @jsancio Thanks. I'll take a look this week. This is an automated message from the Apache Git Service.

[jira] [Comment Edited] (KAFKA-10817) Add clusterId validation to Fetch handling

2021-03-08 Thread Jose Armando Garcia Sancio (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10817?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297787#comment-17297787 ] Jose Armando Garcia Sancio edited comment on KAFKA-10817 at 3/9/21, 2:19 AM: -

[jira] [Commented] (KAFKA-10817) Add clusterId validation to Fetch handling

2021-03-08 Thread Jose Armando Garcia Sancio (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10817?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297787#comment-17297787 ] Jose Armando Garcia Sancio commented on KAFKA-10817: > ClusterId val

[jira] [Created] (KAFKA-12440) ClusterId validation for Vote, BeginQourum and EndQuorum

2021-03-08 Thread Jose Armando Garcia Sancio (Jira)
Jose Armando Garcia Sancio created KAFKA-12440: -- Summary: ClusterId validation for Vote, BeginQourum and EndQuorum Key: KAFKA-12440 URL: https://issues.apache.org/jira/browse/KAFKA-12440 P

[jira] [Commented] (KAFKA-10821) Send cluster id information with the FetchSnapshot request

2021-03-08 Thread Jose Armando Garcia Sancio (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10821?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297783#comment-17297783 ] Jose Armando Garcia Sancio commented on KAFKA-10821: [~rohitdeshaws]

[jira] [Assigned] (KAFKA-10821) Send cluster id information with the FetchSnapshot request

2021-03-08 Thread Jose Armando Garcia Sancio (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10821?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jose Armando Garcia Sancio reassigned KAFKA-10821: -- Assignee: Rohit Deshpande (was: Jose Armando Garcia Sancio)

[jira] [Updated] (KAFKA-12435) Several streams-test-utils classes missing from javadoc

2021-03-08 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12435?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-12435: Component/s: streams-test-utils docs > Several streams-test-utils classes

[jira] [Assigned] (KAFKA-7785) Remove PartitionGrouper interface and it's config and move DefaultPartitionGrouper to internal package

2021-03-08 Thread highluck (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7785?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] highluck reassigned KAFKA-7785: --- Assignee: highluck > Remove PartitionGrouper interface and it's config and move > DefaultPartitionG

[jira] [Commented] (KAFKA-7785) Remove PartitionGrouper interface and it's config and move DefaultPartitionGrouper to internal package

2021-03-08 Thread highluck (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7785?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297768#comment-17297768 ] highluck commented on KAFKA-7785: - Oh thank [~mjsax]   > Remove PartitionGrouper interf

[jira] [Commented] (KAFKA-10821) Send cluster id information with the FetchSnapshot request

2021-03-08 Thread Rohit Deshpande (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10821?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297762#comment-17297762 ] Rohit Deshpande commented on KAFKA-10821: - [~jagsancio] can I please work on thi

[jira] [Comment Edited] (KAFKA-12419) Remove Deprecated APIs of Kafka Streams in 3.0

2021-03-08 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12419?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17295451#comment-17295451 ] Matthias J. Sax edited comment on KAFKA-12419 at 3/8/21, 11:26 PM: ---

[jira] [Updated] (KAFKA-7606) Remove deprecated options from StreamsResetter

2021-03-08 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7606?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-7606: --- Description: We should remove deprecated flags from StreamsResetter: * "--zookeeper" flag (d

[jira] [Updated] (KAFKA-7606) Remove deprecated options from StreamsResetter

2021-03-08 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7606?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-7606: --- Summary: Remove deprecated options from StreamsResetter (was: Remove deprecated --zookeeper

[jira] [Commented] (KAFKA-10434) Remove deprecated methods on WindowStore

2021-03-08 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10434?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297750#comment-17297750 ] Matthias J. Sax commented on KAFKA-10434: - [~jeqo] With upcoming 3.0 release, wo

[jira] [Commented] (KAFKA-7785) Remove PartitionGrouper interface and it's config and move DefaultPartitionGrouper to internal package

2021-03-08 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7785?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297749#comment-17297749 ] Matthias J. Sax commented on KAFKA-7785: [~high.lee] – Still interested? Next rel

[GitHub] [kafka] mjsax commented on pull request #7902: KAFKA-7785 Default partition grouper to internal

2021-03-08 Thread GitBox
mjsax commented on pull request #7902: URL: https://github.com/apache/kafka/pull/7902#issuecomment-793163127 @highluck -- the next release will be 3.0. If you are interested, you could open a new PR for this ticket? This is

[jira] [Commented] (KAFKA-7606) Remove deprecated --zookeeper option from StreamsResetter

2021-03-08 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7606?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297748#comment-17297748 ] Matthias J. Sax commented on KAFKA-7606: [~mrsrinivas] – the next release with be

[GitHub] [kafka] jolshan opened a new pull request #10282: KAFKA-12426: Missing logic to create partition.metadata files in RaftReplicaManager

2021-03-08 Thread GitBox
jolshan opened a new pull request #10282: URL: https://github.com/apache/kafka/pull/10282 KIP-516 introduced partition.metadata file to persist the topic ID on the broker. It is created through handling the LeaderAndIsrRequest in ReplicaManager. (See https://github.com/apache/kafka/pull/10

[jira] [Commented] (KAFKA-12439) When in KIP-500 mode, we should be able to assign new partitions to nodes that are fenced

2021-03-08 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12439?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297742#comment-17297742 ] Colin McCabe commented on KAFKA-12439: -- See also: https://github.com/apache/kafka/p

[jira] [Created] (KAFKA-12439) When in KIP-500 mode, we should be able to assign new partitions to nodes that are fenced

2021-03-08 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12439: Summary: When in KIP-500 mode, we should be able to assign new partitions to nodes that are fenced Key: KAFKA-12439 URL: https://issues.apache.org/jira/browse/KAFKA-12439

[GitHub] [kafka] cmccabe commented on pull request #10249: MINOR: disable round_trip_fault_test system tests for Raft quorums

2021-03-08 Thread GitBox
cmccabe commented on pull request #10249: URL: https://github.com/apache/kafka/pull/10249#issuecomment-793151321 Thanks, @rondagostino . I created a JIRA to track fixing this, post-2.8.0. https://issues.apache.org/jira/browse/KAFKA-12439 --

[GitHub] [kafka] cmccabe opened a new pull request #10281: KAFKA-12432: Fix AdminClient timeout handling in the presence of bad brokers

2021-03-08 Thread GitBox
cmccabe opened a new pull request #10281: URL: https://github.com/apache/kafka/pull/10281 Previously, we relied solely on the broker to enforce request.timeout.ms. If the broker failed to enforce this timeout, the admin client would hang until the call's API timeout hit, which would

[jira] [Updated] (KAFKA-10619) Producer will enable EOS by default

2021-03-08 Thread Ismael Juma (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10619?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-10619: Fix Version/s: 3.0.0 > Producer will enable EOS by default > --- >

[jira] [Updated] (KAFKA-12438) MirrorMaker 2: Not able to make it work for kerberized clusters ( bothe source and target are kerberized )

2021-03-08 Thread Bharadwaj Bhimavarapu (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12438?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bharadwaj Bhimavarapu updated KAFKA-12438: -- Issue Type: Wish (was: Bug) > MirrorMaker 2: Not able to make it work for ker

[GitHub] [kafka] kowshik opened a new pull request #10280: KIP-405: Log layer refactor

2021-03-08 Thread GitBox
kowshik opened a new pull request #10280: URL: https://github.com/apache/kafka/pull/10280 **This PR is a work in progress. Please do not treat this as a finished product yet.** **TL;DR:** This PR implements the details of the Log layer refactor, as outlined in this document:

[jira] [Created] (KAFKA-12438) MirrorMaker 2: Not able to make it work for kerberized clusters ( bothe source and target are kerberized )

2021-03-08 Thread Bharadwaj Bhimavarapu (Jira)
Bharadwaj Bhimavarapu created KAFKA-12438: - Summary: MirrorMaker 2: Not able to make it work for kerberized clusters ( bothe source and target are kerberized ) Key: KAFKA-12438 URL: https://issues.apache.

[GitHub] [kafka] splett2 commented on pull request #10267: KAFKA-12427: Don't update connection idle time for muted connections

2021-03-08 Thread GitBox
splett2 commented on pull request #10267: URL: https://github.com/apache/kafka/pull/10267#issuecomment-793092807 @rajinisivaram thanks for the review, that makes sense to me. This is an automated message from the Apache G

[GitHub] [kafka] jolshan commented on pull request #9758: MINOR: remove FetchResponse.AbortedTransaction and redundant construc…

2021-03-08 Thread GitBox
jolshan commented on pull request #9758: URL: https://github.com/apache/kafka/pull/9758#issuecomment-793082596 @chia7712 I noticed that all of these JIRAs are listed as open, but they reference this PR. Are there further changes to be made? Asking because I'm working on similar code for h

[GitHub] [kafka] hachikuji merged pull request #10279: MINOR: Refer user to `kafka-storage.sh` if `meta.properties` is missing

2021-03-08 Thread GitBox
hachikuji merged pull request #10279: URL: https://github.com/apache/kafka/pull/10279 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 t

[GitHub] [kafka] jolshan edited a comment on pull request #9590: KAFKA-7556: KafkaConsumer.beginningOffsets does not return actual first offsets

2021-03-08 Thread GitBox
jolshan edited a comment on pull request #9590: URL: https://github.com/apache/kafka/pull/9590#issuecomment-793061818 @hachikuji updated based on the conversations here + those we had offline. Quick summary is that we will only have one swap file upon crash in the log cleaning case, so we

[GitHub] [kafka] jolshan commented on pull request #9590: KAFKA-7556: KafkaConsumer.beginningOffsets does not return actual first offsets

2021-03-08 Thread GitBox
jolshan commented on pull request #9590: URL: https://github.com/apache/kafka/pull/9590#issuecomment-793061818 @hachikuji updated based on the conversations here + those we had offline. Quick summary is that we will only have one swap file upon crash in the log cleaning case, so we will ju

[GitHub] [kafka] rondagostino commented on pull request #10225: MINOR: fix security_test for ZK case due to error change

2021-03-08 Thread GitBox
rondagostino commented on pull request #10225: URL: https://github.com/apache/kafka/pull/10225#issuecomment-793057316 Closed in favor of https://github.com/apache/kafka/pull/10240/ This is an automated message from the Apache

[GitHub] [kafka] rondagostino closed pull request #10225: MINOR: fix security_test for ZK case due to error change

2021-03-08 Thread GitBox
rondagostino closed pull request #10225: URL: https://github.com/apache/kafka/pull/10225 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 g

[GitHub] [kafka] rohitrmd commented on pull request #10276: Kafka 12253: Add tests that cover all of all of the cases for ReplicatedLog-validateOffsetAndEpoch

2021-03-08 Thread GitBox
rohitrmd commented on pull request #10276: URL: https://github.com/apache/kafka/pull/10276#issuecomment-793011934 @jsancio This is an automated message from the Apache Git Service. To respond to the message, please log on to

[jira] [Resolved] (KAFKA-12403) Broker handling of delete topic events

2021-03-08 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12403?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-12403. - Resolution: Fixed > Broker handling of delete topic events > ---

[GitHub] [kafka] hachikuji merged pull request #10252: KAFKA-12403; Ensure local state deleted on `RemoveTopicRecord` received

2021-03-08 Thread GitBox
hachikuji merged pull request #10252: URL: https://github.com/apache/kafka/pull/10252 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 t

[jira] [Commented] (KAFKA-12431) Fetch Request/Response without Topic information

2021-03-08 Thread Peter Sinoros-Szabo (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297657#comment-17297657 ] Peter Sinoros-Szabo commented on KAFKA-12431: - The change was probably intro

[GitHub] [kafka] hachikuji opened a new pull request #10279: MINOR: Refer user to `kafka-storage.sh` if `meta.properties` is missing

2021-03-08 Thread GitBox
hachikuji opened a new pull request #10279: URL: https://github.com/apache/kafka/pull/10279 The KIP-500 server requires users to run kafka-storage.sh to format log directories before the server will start. If the directory is not formatted, the error message complains about a missing `meta

[GitHub] [kafka] chia7712 commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API

2021-03-08 Thread GitBox
chia7712 commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r589631318 ## File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersResult.java ## @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Softw

[jira] [Updated] (KAFKA-10340) Source connectors should report error when trying to produce records to non-existent topics instead of hanging forever

2021-03-08 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10340?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch updated KAFKA-10340: -- Fix Version/s: (was: 2.6.2) > Source connectors should report error when trying to produce

[jira] [Updated] (KAFKA-10340) Source connectors should report error when trying to produce records to non-existent topics instead of hanging forever

2021-03-08 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10340?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch updated KAFKA-10340: -- Fix Version/s: 2.6.2 > Source connectors should report error when trying to produce records to

[GitHub] [kafka] ijuma commented on a change in pull request #10269: KAFKA-12410 KafkaAPis ought to group fetch data before generating fet…

2021-03-08 Thread GitBox
ijuma commented on a change in pull request #10269: URL: https://github.com/apache/kafka/pull/10269#discussion_r589635348 ## File path: core/src/main/scala/kafka/server/KafkaApis.scala ## @@ -3368,8 +3396,16 @@ object KafkaApis { private[server] def sizeOfThrottledPartitions

[GitHub] [kafka] chia7712 commented on a change in pull request #10269: KAFKA-12410 KafkaAPis ought to group fetch data before generating fet…

2021-03-08 Thread GitBox
chia7712 commented on a change in pull request #10269: URL: https://github.com/apache/kafka/pull/10269#discussion_r589633902 ## File path: core/src/main/scala/kafka/server/KafkaApis.scala ## @@ -3368,8 +3396,16 @@ object KafkaApis { private[server] def sizeOfThrottledPartiti

[jira] [Commented] (KAFKA-12431) Fetch Request/Response without Topic information

2021-03-08 Thread Jun Rao (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297596#comment-17297596 ] Jun Rao commented on KAFKA-12431: - Hmm, I am not aware of any changes for throttling tha

[GitHub] [kafka] ijuma commented on a change in pull request #10269: KAFKA-12410 KafkaAPis ought to group fetch data before generating fet…

2021-03-08 Thread GitBox
ijuma commented on a change in pull request #10269: URL: https://github.com/apache/kafka/pull/10269#discussion_r589631664 ## File path: core/src/main/scala/kafka/server/KafkaApis.scala ## @@ -3368,8 +3396,16 @@ object KafkaApis { private[server] def sizeOfThrottledPartitions

[jira] [Created] (KAFKA-12437) Ensure all metadata records are covered by BrokerMetadataListenerTest

2021-03-08 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-12437: --- Summary: Ensure all metadata records are covered by BrokerMetadataListenerTest Key: KAFKA-12437 URL: https://issues.apache.org/jira/browse/KAFKA-12437 Project:

[GitHub] [kafka] hachikuji commented on a change in pull request #10252: KAFKA-12403; Ensure local state deleted on `RemoveTopicRecord` received

2021-03-08 Thread GitBox
hachikuji commented on a change in pull request #10252: URL: https://github.com/apache/kafka/pull/10252#discussion_r589627144 ## File path: core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala ## @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Softw

[GitHub] [kafka] ryannedolan commented on pull request #10277: KAFKA-9914: Fix replication cycle detection

2021-03-08 Thread GitBox
ryannedolan commented on pull request #10277: URL: https://github.com/apache/kafka/pull/10277#issuecomment-792948125 Thanks for the explanation. Works for me 👍 This is an automated message from the Apache Git Service. To res

[jira] [Resolved] (KAFKA-9186) Kafka Connect floods logs with probably bogus error messages from DelegatingClassLoader

2021-03-08 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9186?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris resolved KAFKA-9186. Reviewer: Jason Gustafson Assignee: Chia-Ping Tsai Resolution: Fixed > Kafka Connect fl

[jira] [Commented] (KAFKA-9186) Kafka Connect floods logs with probably bogus error messages from DelegatingClassLoader

2021-03-08 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9186?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297583#comment-17297583 ] Greg Harris commented on KAFKA-9186: This was fixed in [https://github.com/apache/kaf

[jira] [Updated] (KAFKA-9186) Kafka Connect floods logs with probably bogus error messages from DelegatingClassLoader

2021-03-08 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9186?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris updated KAFKA-9186: --- Fix Version/s: 2.5.0 3.0.0 > Kafka Connect floods logs with probably bogus error m

[GitHub] [kafka] chia7712 commented on pull request #10269: KAFKA-12410 KafkaAPis ought to group fetch data before generating fet…

2021-03-08 Thread GitBox
chia7712 commented on pull request #10269: URL: https://github.com/apache/kafka/pull/10269#issuecomment-792943140 > Thanks for the PR. Can you check the perf impact of these changes? sure. will add benchmark results tomorrow. -

[GitHub] [kafka] chia7712 commented on a change in pull request #10252: KAFKA-12403; Ensure local state deleted on `RemoveTopicRecord` received

2021-03-08 Thread GitBox
chia7712 commented on a change in pull request #10252: URL: https://github.com/apache/kafka/pull/10252#discussion_r589621057 ## File path: core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala ## @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Softwa

[GitHub] [kafka] tvainika commented on pull request #10277: KAFKA-9914: Fix replication cycle detection

2021-03-08 Thread GitBox
tvainika commented on pull request #10277: URL: https://github.com/apache/kafka/pull/10277#issuecomment-792942212 > Seems harmless, but can you share the motivation behind this change? Did you come across some scenario where the existing logic didn't work? Yes, those are documented i

[jira] [Commented] (KAFKA-12431) Fetch Request/Response without Topic information

2021-03-08 Thread Ismael Juma (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297555#comment-17297555 ] Ismael Juma commented on KAFKA-12431: - Thanks for getting to the bottom of it. I thi

[GitHub] [kafka] C0urante commented on a change in pull request #10112: KAFKA-12226: Prevent source task offset failure when producer is overwhelmed

2021-03-08 Thread GitBox
C0urante commented on a change in pull request #10112: URL: https://github.com/apache/kafka/pull/10112#discussion_r589603947 ## File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java ## @@ -475,11 +476,15 @@ public boolean commitOffsets

[GitHub] [kafka] hachikuji commented on a change in pull request #10252: KAFKA-12403; Ensure local state deleted on `RemoveTopicRecord` received

2021-03-08 Thread GitBox
hachikuji commented on a change in pull request #10252: URL: https://github.com/apache/kafka/pull/10252#discussion_r589600950 ## File path: core/src/main/scala/kafka/server/metadata/MetadataPartitions.scala ## @@ -198,15 +242,18 @@ class MetadataPartitionsBuilder(val brokerId:

[GitHub] [kafka] hachikuji commented on a change in pull request #10252: KAFKA-12403; Ensure local state deleted on `RemoveTopicRecord` received

2021-03-08 Thread GitBox
hachikuji commented on a change in pull request #10252: URL: https://github.com/apache/kafka/pull/10252#discussion_r589597256 ## File path: core/src/main/scala/kafka/server/metadata/MetadataPartitions.scala ## @@ -171,25 +202,38 @@ class MetadataPartitionsBuilder(val brokerId:

[GitHub] [kafka] rhauch commented on a change in pull request #8259: KAFKA-7421: Ensure Connect's PluginClassLoader is truly parallel capable and resolve deadlock occurrences

2021-03-08 Thread GitBox
rhauch commented on a change in pull request #8259: URL: https://github.com/apache/kafka/pull/8259#discussion_r589553615 ## File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/SynchronizationTest.java ## @@ -0,0 +1,468 @@ +/* + * Licensed to the

[jira] [Commented] (KAFKA-12431) Fetch Request/Response without Topic information

2021-03-08 Thread Peter Sinoros-Szabo (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297520#comment-17297520 ] Peter Sinoros-Szabo commented on KAFKA-12431: - Yes, it is about 500ms until

[GitHub] [kafka] ijuma commented on a change in pull request #10272: MINOR: Various javadoc fixes

2021-03-08 Thread GitBox
ijuma commented on a change in pull request #10272: URL: https://github.com/apache/kafka/pull/10272#discussion_r589575603 ## File path: build.gradle ## @@ -92,12 +92,6 @@ allprojects { } } - tasks.withType(Javadoc) { -// disable the crazy super-strict doclint too

[GitHub] [kafka] chia7712 commented on a change in pull request #10273: MINOR: Include number of members in group coordinator messages

2021-03-08 Thread GitBox
chia7712 commented on a change in pull request #10273: URL: https://github.com/apache/kafka/pull/10273#discussion_r589570181 ## File path: core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala ## @@ -561,7 +561,8 @@ class GroupCoordinator(val brokerId: Int,

[GitHub] [kafka] vamossagar12 commented on a change in pull request #10278: KAFKA-10526: leader fsync deferral on write

2021-03-08 Thread GitBox
vamossagar12 commented on a change in pull request #10278: URL: https://github.com/apache/kafka/pull/10278#discussion_r589561309 ## File path: raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java ## @@ -1876,7 +1876,7 @@ private long maybeAppendBatches(

[jira] [Commented] (KAFKA-12431) Fetch Request/Response without Topic information

2021-03-08 Thread Ismael Juma (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297503#comment-17297503 ] Ismael Juma commented on KAFKA-12431: - [~sinitw] Do you see 500ms between fetch resp

[GitHub] [kafka] vamossagar12 opened a new pull request #10278: KAFKA-10526: leader fsync deferral on write

2021-03-08 Thread GitBox
vamossagar12 opened a new pull request #10278: URL: https://github.com/apache/kafka/pull/10278 *More detailed description of your change, if necessary. The PR title and PR message become the squashed commit message, so use a separate comment to ping reviewers.* *Summary of te

[jira] [Commented] (KAFKA-12431) Fetch Request/Response without Topic information

2021-03-08 Thread Peter Sinoros-Szabo (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297479#comment-17297479 ] Peter Sinoros-Szabo commented on KAFKA-12431: - We do not set it, so it is on

[jira] [Updated] (KAFKA-12431) Fetch Request/Response without Topic information

2021-03-08 Thread Peter Sinoros-Szabo (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12431?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Sinoros-Szabo updated KAFKA-12431: Attachment: kafka-highcpu-node1.svg > Fetch Request/Response without Topic informa

[jira] [Commented] (KAFKA-12431) Fetch Request/Response without Topic information

2021-03-08 Thread Ismael Juma (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297435#comment-17297435 ] Ismael Juma commented on KAFKA-12431: - Thanks for the report. What's the `fetch.max.

[jira] [Commented] (KAFKA-12431) Fetch Request/Response without Topic information

2021-03-08 Thread Peter Sinoros-Szabo (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297432#comment-17297432 ] Peter Sinoros-Szabo commented on KAFKA-12431: - I attached the flame graphs a

[jira] [Updated] (KAFKA-12431) Fetch Request/Response without Topic information

2021-03-08 Thread Peter Sinoros-Szabo (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12431?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Sinoros-Szabo updated KAFKA-12431: Attachment: kafka-highcpu-26.svg.zip > Fetch Request/Response without Topic inform

[jira] [Updated] (KAFKA-12431) Fetch Request/Response without Topic information

2021-03-08 Thread Peter Sinoros-Szabo (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12431?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Sinoros-Szabo updated KAFKA-12431: Attachment: (was: kafka-highcpu-26.svg) > Fetch Request/Response without Topic

[jira] [Updated] (KAFKA-12431) Fetch Request/Response without Topic information

2021-03-08 Thread Peter Sinoros-Szabo (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12431?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Sinoros-Szabo updated KAFKA-12431: Attachment: kafka-highcpu-24.svg.zip > Fetch Request/Response without Topic inform

[jira] [Updated] (KAFKA-12431) Fetch Request/Response without Topic information

2021-03-08 Thread Peter Sinoros-Szabo (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12431?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Sinoros-Szabo updated KAFKA-12431: Attachment: kafka-highcpu-26.svg > Fetch Request/Response without Topic informatio

[jira] [Updated] (KAFKA-12431) Fetch Request/Response without Topic information

2021-03-08 Thread Peter Sinoros-Szabo (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12431?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Sinoros-Szabo updated KAFKA-12431: Attachment: (was: kafka-highcpu-node1.svg) > Fetch Request/Response without To

[jira] [Commented] (KAFKA-12431) Fetch Request/Response without Topic information

2021-03-08 Thread Peter Sinoros-Szabo (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297428#comment-17297428 ] Peter Sinoros-Szabo commented on KAFKA-12431: - We did further research and f

[GitHub] [kafka] dajac commented on a change in pull request #10141: KAFKA-12329; kafka-reassign-partitions command should give a better error message when a topic does not exist

2021-03-08 Thread GitBox
dajac commented on a change in pull request #10141: URL: https://github.com/apache/kafka/pull/10141#discussion_r589468725 ## File path: core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala ## @@ -812,6 +812,26 @@ object ReassignPartitionsCommand extends Logging {

[GitHub] [kafka] chia7712 commented on a change in pull request #10141: KAFKA-12329; kafka-reassign-partitions command should give a better error message when a topic does not exist

2021-03-08 Thread GitBox
chia7712 commented on a change in pull request #10141: URL: https://github.com/apache/kafka/pull/10141#discussion_r589448109 ## File path: core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala ## @@ -812,6 +812,26 @@ object ReassignPartitionsCommand extends Logging {

[GitHub] [kafka] ryannedolan commented on pull request #10277: KAFKA-9914: Fix replication cycle detection

2021-03-08 Thread GitBox
ryannedolan commented on pull request #10277: URL: https://github.com/apache/kafka/pull/10277#issuecomment-792764436 Seems harmless, but can you share the motivation behind this change? Did you come across some scenario where the existing logic didn't work? ---

[GitHub] [kafka] ijuma commented on a change in pull request #10141: KAFKA-12329; kafka-reassign-partitions command should give a better error message when a topic does not exist

2021-03-08 Thread GitBox
ijuma commented on a change in pull request #10141: URL: https://github.com/apache/kafka/pull/10141#discussion_r589425927 ## File path: core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala ## @@ -812,6 +812,26 @@ object ReassignPartitionsCommand extends Logging {

[GitHub] [kafka] ijuma commented on a change in pull request #10141: KAFKA-12329; kafka-reassign-partitions command should give a better error message when a topic does not exist

2021-03-08 Thread GitBox
ijuma commented on a change in pull request #10141: URL: https://github.com/apache/kafka/pull/10141#discussion_r589424932 ## File path: core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala ## @@ -812,6 +812,26 @@ object ReassignPartitionsCommand extends Logging {

[GitHub] [kafka] ijuma commented on a change in pull request #10141: KAFKA-12329; kafka-reassign-partitions command should give a better error message when a topic does not exist

2021-03-08 Thread GitBox
ijuma commented on a change in pull request #10141: URL: https://github.com/apache/kafka/pull/10141#discussion_r589424932 ## File path: core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala ## @@ -812,6 +812,26 @@ object ReassignPartitionsCommand extends Logging {

[jira] [Commented] (KAFKA-8154) Buffer Overflow exceptions between brokers and with clients

2021-03-08 Thread Ori Popowski (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8154?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17297334#comment-17297334 ] Ori Popowski commented on KAFKA-8154: - For anyone running on Google Cloud Dataproc, s

[GitHub] [kafka] bmaidics commented on pull request #10192: MINOR: Add missing unit tests for Mirror Connect

2021-03-08 Thread GitBox
bmaidics commented on pull request #10192: URL: https://github.com/apache/kafka/pull/10192#issuecomment-792650247 Thanks, @mimaison for the review, I addressed all the comments This is an automated message from the Apache Git