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

2021-03-01 Thread Ori Popowski (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8154?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17292717#comment-17292717 ] Ori Popowski commented on KAFKA-8154: - [~gordonmessmer] I don't see any Alena or a pa

[jira] [Commented] (KAFKA-9752) Consumer rebalance can be stuck after new member timeout with old JoinGroup version

2021-03-01 Thread Peter Sinoros-Szabo (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9752?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17292724#comment-17292724 ] Peter Sinoros-Szabo commented on KAFKA-9752: Do you think [~hachikuji] that t

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

2021-03-01 Thread GitBox
dajac commented on pull request #10141: URL: https://github.com/apache/kafka/pull/10141#issuecomment-787783411 > @dajac Nice improvement. Some minor comments are left. Please take a look. > > 1. https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients

[GitHub] [kafka] rajinisivaram merged pull request #10217: KAFKA-12254: Ensure MM2 creates topics with source topic configs

2021-03-01 Thread GitBox
rajinisivaram merged pull request #10217: URL: https://github.com/apache/kafka/pull/10217 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

[jira] [Created] (KAFKA-12390) error when storing group assignment during SyncGroup

2021-03-01 Thread Andrey (Jira)
Andrey created KAFKA-12390: -- Summary: error when storing group assignment during SyncGroup Key: KAFKA-12390 URL: https://issues.apache.org/jira/browse/KAFKA-12390 Project: Kafka Issue Type: Bug

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

2021-03-01 Thread GitBox
dajac commented on pull request #10141: URL: https://github.com/apache/kafka/pull/10141#issuecomment-787818715 @chia7712 I just pushed an update. Please take a look and let me know what you think about it. This is an automat

[jira] [Created] (KAFKA-12389) Upgrade of netty-codec due to CVE-2021-21290

2021-03-01 Thread Dominique Mongelli (Jira)
Dominique Mongelli created KAFKA-12389: -- Summary: Upgrade of netty-codec due to CVE-2021-21290 Key: KAFKA-12389 URL: https://issues.apache.org/jira/browse/KAFKA-12389 Project: Kafka Issu

[jira] [Created] (KAFKA-12391) Add an option to store arbitrary metadata to a SourceRecord

2021-03-01 Thread Luca Burgazzoli (Jira)
Luca Burgazzoli created KAFKA-12391: --- Summary: Add an option to store arbitrary metadata to a SourceRecord Key: KAFKA-12391 URL: https://issues.apache.org/jira/browse/KAFKA-12391 Project: Kafka

[jira] [Updated] (KAFKA-12391) Add an option to store arbitrary metadata to a SourceRecord

2021-03-01 Thread Luca Burgazzoli (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12391?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Luca Burgazzoli updated KAFKA-12391: Description: When writing Source Connectors for Kafka, it may be required to perform some

[GitHub] [kafka] cadonna commented on a change in pull request #10215: KAFKA-12375: don't reuse thread.id until a thread has fully shut down

2021-03-01 Thread GitBox
cadonna commented on a change in pull request #10215: URL: https://github.com/apache/kafka/pull/10215#discussion_r584613463 ## File path: checkstyle/suppressions.xml ## @@ -161,7 +161,7 @@ files="StreamThread.java"/> + files="(KafkaStreams|KS

[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-01 Thread GitBox
chia7712 commented on a change in pull request #10141: URL: https://github.com/apache/kafka/pull/10141#discussion_r584616679 ## File path: core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala ## @@ -812,6 +812,26 @@ object ReassignPartitionsCommand extends Logging {

[GitHub] [kafka] cadonna commented on a change in pull request #10215: KAFKA-12375: don't reuse thread.id until a thread has fully shut down

2021-03-01 Thread GitBox
cadonna commented on a change in pull request #10215: URL: https://github.com/apache/kafka/pull/10215#discussion_r584621434 ## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ## @@ -463,9 +464,8 @@ private void replaceStreamThread(final Throwable thr

[GitHub] [kafka] chia7712 commented on a change in pull request #10184: MINOR: enable topic deletion in the KIP-500 controller

2021-03-01 Thread GitBox
chia7712 commented on a change in pull request #10184: URL: https://github.com/apache/kafka/pull/10184#discussion_r584139776 ## File path: core/src/main/scala/kafka/server/ControllerApis.scala ## @@ -154,6 +163,162 @@ class ControllerApis(val requestChannel: RequestChannel,

[GitHub] [kafka] chia7712 commented on a change in pull request #10184: MINOR: enable topic deletion in the KIP-500 controller

2021-03-01 Thread GitBox
chia7712 commented on a change in pull request #10184: URL: https://github.com/apache/kafka/pull/10184#discussion_r584628216 ## File path: core/src/main/scala/kafka/server/ControllerApis.scala ## @@ -280,25 +281,34 @@ class ControllerApis(val requestChannel: RequestChannel,

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

2021-03-01 Thread GitBox
dajac commented on pull request #10141: URL: https://github.com/apache/kafka/pull/10141#issuecomment-787870948 @chia7712 Thanks. I just pushed a commit to address your comment. Could you take another look? This is an automat

[GitHub] [kafka] tombentley commented on a change in pull request #10206: KAFKA-12369; Implement `ListTransactions` API

2021-03-01 Thread GitBox
tombentley commented on a change in pull request #10206: URL: https://github.com/apache/kafka/pull/10206#discussion_r584641483 ## File path: core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala ## @@ -223,6 +224,46 @@ class TransactionStateManager(br

[GitHub] [kafka] tombentley commented on a change in pull request #10193: MINOR: correct the error message of validating uint32

2021-03-01 Thread GitBox
tombentley commented on a change in pull request #10193: URL: https://github.com/apache/kafka/pull/10193#discussion_r584646665 ## File path: clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java ## @@ -320,7 +320,7 @@ public Long validate(Object item) {

[jira] [Updated] (KAFKA-10886) Kafka crashed in windows environment2

2021-03-01 Thread Wenbing Shen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10886?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Wenbing Shen updated KAFKA-10886: - Attachment: (was: image-2021-03-01-14-29-55-418.png) > Kafka crashed in windows environment2

[GitHub] [kafka] tombentley commented on a change in pull request #10193: MINOR: correct the error message of validating uint32

2021-03-01 Thread GitBox
tombentley commented on a change in pull request #10193: URL: https://github.com/apache/kafka/pull/10193#discussion_r584648224 ## File path: clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java ## @@ -320,7 +320,7 @@ public Long validate(Object item) {

[GitHub] [kafka] tombentley commented on pull request #10202: MINOR: Deprecate the batch-size option in console producer

2021-03-01 Thread GitBox
tombentley commented on pull request #10202: URL: https://github.com/apache/kafka/pull/10202#issuecomment-787888644 Is there a JIRA/KIP where this deprecation was discussed? This is an automated message from the Apache Git Se

[GitHub] [kafka] kamalcph commented on pull request #10202: MINOR: Deprecate the batch-size option in console producer

2021-03-01 Thread GitBox
kamalcph commented on pull request #10202: URL: https://github.com/apache/kafka/pull/10202#issuecomment-787891934 > Is there a JIRA/KIP where this deprecation was discussed? No, I don't find any. The batch size config is unused. --

[GitHub] [kafka] kamalcph edited a comment on pull request #10202: MINOR: Deprecate the batch-size option in console producer

2021-03-01 Thread GitBox
kamalcph edited a comment on pull request #10202: URL: https://github.com/apache/kafka/pull/10202#issuecomment-787891934 > Is there a JIRA/KIP where this deprecation was discussed? No, I don't find any and it wasn't discussed. The config is unused so I propose to deprecate it.

[jira] [Commented] (KAFKA-8946) Single byte header issues WARN logging

2021-03-01 Thread Tom Bentley (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8946?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17292852#comment-17292852 ] Tom Bentley commented on KAFKA-8946: I think this is a bug in org.apache.kafka.connec

[jira] [Created] (KAFKA-12392) Deprecate batch-size config from console producer

2021-03-01 Thread Kamal Chandraprakash (Jira)
Kamal Chandraprakash created KAFKA-12392: Summary: Deprecate batch-size config from console producer Key: KAFKA-12392 URL: https://issues.apache.org/jira/browse/KAFKA-12392 Project: Kafka

[jira] [Updated] (KAFKA-12392) Deprecate batch-size config from console producer

2021-03-01 Thread Kamal Chandraprakash (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12392?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kamal Chandraprakash updated KAFKA-12392: - Affects Version/s: (was: 3.0.0) 2.7.0 > Deprecate bat

[GitHub] [kafka] viktorsomogyi opened a new pull request #10233: KAFKA-9413: Auditing in Kafka

2021-03-01 Thread GitBox
viktorsomogyi opened a new pull request #10233: URL: https://github.com/apache/kafka/pull/10233 *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 t

[GitHub] [kafka] kamalcph commented on pull request #10202: KAFKA-12392: Deprecate the batch-size option in console producer

2021-03-01 Thread GitBox
kamalcph commented on pull request #10202: URL: https://github.com/apache/kafka/pull/10202#issuecomment-787919773 Filed [KAFKA-12392](https://issues.apache.org/jira/browse/KAFKA-12392) ticket and opened [KIP-717](https://cwiki.apache.org/confluence/x/DB1RCg) for this. ---

[GitHub] [kafka] kamalcph edited a comment on pull request #10202: KAFKA-12392: Deprecate the batch-size option in console producer

2021-03-01 Thread GitBox
kamalcph edited a comment on pull request #10202: URL: https://github.com/apache/kafka/pull/10202#issuecomment-787919773 Filed [KAFKA-12392](https://issues.apache.org/jira/browse/KAFKA-12392) ticket and opened [KIP-717](https://cwiki.apache.org/confluence/x/DB1RCg) for discussion. --

[GitHub] [kafka] dajac opened a new pull request #10234: MINOR; Clean up LeaderAndIsrResponse construction in `ReplicaManager#becomeLeaderOrFollower`

2021-03-01 Thread GitBox
dajac opened a new pull request #10234: URL: https://github.com/apache/kafka/pull/10234 ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ] Verify test coverage and CI build status - [ ] Verify documentation (including upgrade

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

2021-03-01 Thread GitBox
dajac commented on pull request #10141: URL: https://github.com/apache/kafka/pull/10141#issuecomment-787968243 Failed test is not related. Merging to trunk. This is an automated message from the Apache Git Service. To respond

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

2021-03-01 Thread GitBox
dajac merged pull request #10141: URL: https://github.com/apache/kafka/pull/10141 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 th

[jira] [Resolved] (KAFKA-12329) kafka-reassign-partitions command should give a better error message when a topic does not exist

2021-03-01 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12329?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-12329. - Fix Version/s: 3.0.0 Resolution: Fixed > kafka-reassign-partitions command should give a

[GitHub] [kafka] dongjinleekr opened a new pull request #10235: KAFKA-12389: Upgrade of netty-codec due to CVE-2021-21290

2021-03-01 Thread GitBox
dongjinleekr opened a new pull request #10235: URL: https://github.com/apache/kafka/pull/10235 This security vulnerability is related to netty itself and fixed in 4.1.59.Final. So, upgrade the netty version from 4.1.51.Final to 4.1.59.Final. ### Committer Checklist (excluded from com

[jira] [Assigned] (KAFKA-12389) Upgrade of netty-codec due to CVE-2021-21290

2021-03-01 Thread Dongjin Lee (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12389?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dongjin Lee reassigned KAFKA-12389: --- Assignee: Dongjin Lee > Upgrade of netty-codec due to CVE-2021-21290 >

[GitHub] [kafka] dongjinleekr commented on pull request #10235: KAFKA-12389: Upgrade of netty-codec due to CVE-2021-21290

2021-03-01 Thread GitBox
dongjinleekr commented on pull request #10235: URL: https://github.com/apache/kafka/pull/10235#issuecomment-787990779 @omkreddy This commit should be cherry-picked onto 2.8, 2.7 and 2.6 branch like [KAFKA-12324: Upgrade jetty to fix CVE-2020-27218](https://github.com/apache/kafka/pull/1017

[GitHub] [kafka] dajac opened a new pull request #10236: MINOR; Small refactor in `GroupMetadata`

2021-03-01 Thread GitBox
dajac opened a new pull request #10236: URL: https://github.com/apache/kafka/pull/10236 Follow-up of https://github.com/apache/kafka/pull/9958#discussion_r582662669. ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ] Verify tes

[jira] [Created] (KAFKA-12393) Document multi-tenancy considerations

2021-03-01 Thread Michael G. Noll (Jira)
Michael G. Noll created KAFKA-12393: --- Summary: Document multi-tenancy considerations Key: KAFKA-12393 URL: https://issues.apache.org/jira/browse/KAFKA-12393 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-12393) Document multi-tenancy considerations

2021-03-01 Thread ASF GitHub Bot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12393?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17292946#comment-17292946 ] ASF GitHub Bot commented on KAFKA-12393: miguno opened a new pull request #334:

[jira] [Commented] (KAFKA-12393) Document multi-tenancy considerations

2021-03-01 Thread Michael G. Noll (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12393?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17292947#comment-17292947 ] Michael G. Noll commented on KAFKA-12393: - PR up at https://github.com/apache/ka

[GitHub] [kafka] C0urante commented on a change in pull request #9541: KAFKA-10675: Add schema name to ConnectSchema.validateValue() error message

2021-03-01 Thread GitBox
C0urante commented on a change in pull request #9541: URL: https://github.com/apache/kafka/pull/9541#discussion_r584810742 ## File path: connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java ## @@ -266,10 +261,19 @@ public static void validateValue(String

[GitHub] [kafka] wcarlson5 commented on a change in pull request #10215: KAFKA-12375: don't reuse thread.id until a thread has fully shut down

2021-03-01 Thread GitBox
wcarlson5 commented on a change in pull request #10215: URL: https://github.com/apache/kafka/pull/10215#discussion_r584812254 ## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ## @@ -463,9 +464,8 @@ private void replaceStreamThread(final Throwable t

[GitHub] [kafka] rhauch merged pull request #10016: KAFKA-10340: Proactively close producer when cancelling source tasks

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

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

2021-03-01 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.7.1 > Source connectors should report error when trying to produce records to

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

2021-03-01 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: 3.0.0 > Source connectors should report error when trying to produce records to

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

2021-03-01 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10340?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17292993#comment-17292993 ] Randall Hauch commented on KAFKA-10340: --- [~xakassi] wrote: {quote} And also I can

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

2021-03-01 Thread Chris Egerton (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10340?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chris Egerton updated KAFKA-10340: -- Affects Version/s: 2.8.0 2.5.1 2.7.0

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

2021-03-01 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10340?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17292993#comment-17292993 ] Randall Hauch edited comment on KAFKA-10340 at 3/1/21, 4:16 PM: --

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

2021-03-01 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.7.1) > Source connectors should report error when trying to produce

[jira] [Commented] (KAFKA-12253) Add tests that cover all of all of the cases for ReplicatedLog::validateOffsetAndEpoch

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

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

2021-03-01 Thread GitBox
ijuma commented on a change in pull request #9758: URL: https://github.com/apache/kafka/pull/9758#discussion_r584812552 ## File path: clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java ## @@ -791,15 +791,18 @@ public void produceRequestGetErrorResp

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

2021-03-01 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.7.1 > Source connectors should report error when trying to produce records to

[GitHub] [kafka] ijuma commented on pull request #10203: MINOR: Prepare for Gradle 7.0

2021-03-01 Thread GitBox
ijuma commented on pull request #10203: URL: https://github.com/apache/kafka/pull/10203#issuecomment-788083914 @chia7712 Are you ok after the latest changes? This is an automated message from the Apache Git Service. To respon

[GitHub] [kafka] ijuma commented on pull request #10231: MINOR: Remove stack trace of the lock exception in a debug log4j

2021-03-01 Thread GitBox
ijuma commented on pull request #10231: URL: https://github.com/apache/kafka/pull/10231#issuecomment-788086223 It would be good to include a bit more context in the PR description regarding the motivation. I assume it's noisy and doesn't add useful information. --

[GitHub] [kafka] rondagostino opened a new pull request #10237: MINOR: fix failing system test delegation_token_test

2021-03-01 Thread GitBox
rondagostino opened a new pull request #10237: URL: https://github.com/apache/kafka/pull/10237 The system test in `delegation_token_test.py` broke due to https://github.com/apache/kafka/pull/10199/. That patch changed the logic of `SecurityConfig.enabled_sasl_mechanisms()` to only add

[jira] [Commented] (KAFKA-12393) Document multi-tenancy considerations

2021-03-01 Thread ASF GitHub Bot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12393?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17293011#comment-17293011 ] ASF GitHub Bot commented on KAFKA-12393: miguno commented on pull request #334:

[GitHub] [kafka] rhauch opened a new pull request #10238: KAFKA-10340: Proactively close producer when cancelling source tasks

2021-03-01 Thread GitBox
rhauch opened a new pull request #10238: URL: https://github.com/apache/kafka/pull/10238 **_NOTE: This is targeted to the `2.8` branch (currently frozen), but includes the same commit cherry-picked from `trunk` and originating in #10016 for (KAFKA-10340)[https://issues.apache.org/jira/brow

[GitHub] [kafka] rondagostino commented on a change in pull request #10227: KAFKA-12382: add a README for KIP-500

2021-03-01 Thread GitBox
rondagostino commented on a change in pull request #10227: URL: https://github.com/apache/kafka/pull/10227#discussion_r584875012 ## File path: KIP-500.md ## @@ -0,0 +1,131 @@ +KIP-500 Early Access Release + + +# Introduction +It is now possible to ru

[jira] [Updated] (KAFKA-12254) MirrorMaker 2.0 creates destination topic with default configs

2021-03-01 Thread Dhruvil Shah (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12254?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dhruvil Shah updated KAFKA-12254: - Priority: Blocker (was: Major) > MirrorMaker 2.0 creates destination topic with default configs

[jira] [Updated] (KAFKA-12254) MirrorMaker 2.0 creates destination topic with default configs

2021-03-01 Thread Dhruvil Shah (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12254?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dhruvil Shah updated KAFKA-12254: - Affects Version/s: 2.8.0 > MirrorMaker 2.0 creates destination topic with default configs >

[jira] [Updated] (KAFKA-8929) MM2 system tests

2021-03-01 Thread Dhruvil Shah (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8929?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dhruvil Shah updated KAFKA-8929: Priority: Major (was: Minor) > MM2 system tests > > > Key: KAFKA-

[jira] [Updated] (KAFKA-12254) MirrorMaker 2.0 creates destination topic with default configs

2021-03-01 Thread Dhruvil Shah (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12254?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dhruvil Shah updated KAFKA-12254: - Affects Version/s: 2.4.0 2.5.0 2.4.1

[jira] [Commented] (KAFKA-10582) Mirror Maker 2 not replicating new topics until restart

2021-03-01 Thread Leonid Ilyevsky (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10582?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17293034#comment-17293034 ] Leonid Ilyevsky commented on KAFKA-10582: - Actually, it does work without cleani

[GitHub] [kafka] dajac merged pull request #10236: MINOR; Small refactor in `GroupMetadata`

2021-03-01 Thread GitBox
dajac merged pull request #10236: URL: https://github.com/apache/kafka/pull/10236 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 th

[GitHub] [kafka] tombentley commented on a change in pull request #10227: KAFKA-12382: add a README for KIP-500

2021-03-01 Thread GitBox
tombentley commented on a change in pull request #10227: URL: https://github.com/apache/kafka/pull/10227#discussion_r584906630 ## File path: KIP-500.md ## @@ -0,0 +1,131 @@ +KIP-500 Early Access Release + + +# Introduction +It is now possible to run

[GitHub] [kafka] junrao commented on a change in pull request #10216: KAFKA-12177: apply log start offset retention before time and size based retention

2021-03-01 Thread GitBox
junrao commented on a change in pull request #10216: URL: https://github.com/apache/kafka/pull/10216#discussion_r584915517 ## File path: core/src/test/scala/unit/kafka/log/LogTest.scala ## @@ -1515,6 +1515,27 @@ class LogTest { "expect a single producer state snapshot re

[jira] [Commented] (KAFKA-12377) Flaky Test SaslAuthenticatorTest#testSslClientAuthRequiredForSaslSslListener

2021-03-01 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12377?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17293057#comment-17293057 ] Matthias J. Sax commented on KAFKA-12377: - Failed again with same error message.

[jira] [Commented] (KAFKA-12253) Add tests that cover all of all of the cases for ReplicatedLog::validateOffsetAndEpoch

2021-03-01 Thread Rohit Deshpande (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12253?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17293060#comment-17293060 ] Rohit Deshpande commented on KAFKA-12253: - Thank you. Will go through those test

[jira] [Assigned] (KAFKA-12253) Add tests that cover all of all of the cases for ReplicatedLog::validateOffsetAndEpoch

2021-03-01 Thread Rohit Deshpande (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12253?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rohit Deshpande reassigned KAFKA-12253: --- Assignee: Rohit Deshpande > Add tests that cover all of all of the cases for > Rep

[GitHub] [kafka] abbccdda commented on a change in pull request #10156: KAFKA-10345: File watch store reloading

2021-03-01 Thread GitBox
abbccdda commented on a change in pull request #10156: URL: https://github.com/apache/kafka/pull/10156#discussion_r584949583 ## File path: clients/src/main/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactory.java ## @@ -106,10 +138,7 @@ public boolean shouldBeReb

[GitHub] [kafka] abbccdda commented on a change in pull request #10156: KAFKA-10345: File watch store reloading

2021-03-01 Thread GitBox
abbccdda commented on a change in pull request #10156: URL: https://github.com/apache/kafka/pull/10156#discussion_r584950105 ## File path: clients/src/main/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactory.java ## @@ -211,15 +390,15 @@ private SSLEngine createS

[jira] [Commented] (KAFKA-10383) KTable Join on Foreign key is opinionated

2021-03-01 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10383?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17293091#comment-17293091 ] Matthias J. Sax commented on KAFKA-10383: - [~marcolotz] Are you all set in the m

[GitHub] [kafka] bbejeck commented on pull request #10150: KAFKA-3745: Add access to read-only key in value joiner

2021-03-01 Thread GitBox
bbejeck commented on pull request #10150: URL: https://github.com/apache/kafka/pull/10150#issuecomment-788178028 ping @mjsax @cadonna @vvcephei for reviews This is an automated message from the Apache Git Service. To respond

[jira] [Commented] (KAFKA-12283) Flaky Test RebalanceSourceConnectorsIntegrationTest#testMultipleWorkersRejoining

2021-03-01 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12283?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17293107#comment-17293107 ] Matthias J. Sax commented on KAFKA-12283: - [https://github.com/apache/kafka/pull

[GitHub] [kafka] mjsax merged pull request #10229: MINOR: Account for extra whitespaces in WordCountDemo

2021-03-01 Thread GitBox
mjsax merged pull request #10229: URL: https://github.com/apache/kafka/pull/10229 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 th

[GitHub] [kafka] mjsax commented on pull request #10229: MINOR: Account for extra whitespaces in WordCountDemo

2021-03-01 Thread GitBox
mjsax commented on pull request #10229: URL: https://github.com/apache/kafka/pull/10229#issuecomment-788192044 Thanks @dengziming! Merged to `trunk`. This is an automated message from the Apache Git Service. To respond to the

[GitHub] [kafka] bbejeck commented on pull request #10150: KAFKA-3745: Add access to read-only key in value joiner

2021-03-01 Thread GitBox
bbejeck commented on pull request #10150: URL: https://github.com/apache/kafka/pull/10150#issuecomment-788208213 including @lct45 and @wcarlson5 in the review request as well This is an automated message from the Apache Git

[GitHub] [kafka] guozhangwang commented on pull request #10231: MINOR: Remove stack trace of the lock exception in a debug log4j

2021-03-01 Thread GitBox
guozhangwang commented on pull request #10231: URL: https://github.com/apache/kafka/pull/10231#issuecomment-788209240 > It would be good to include a bit more context in the PR description regarding the motivation. I assume it's noisy and doesn't add useful information. SG, will add

[GitHub] [kafka] guozhangwang merged pull request #10231: MINOR: Remove stack trace of the lock exception in a debug log4j

2021-03-01 Thread GitBox
guozhangwang merged pull request #10231: URL: https://github.com/apache/kafka/pull/10231 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] rondagostino commented on pull request #10225: MINOR: fix security_test for ZK case due to error change

2021-03-01 Thread GitBox
rondagostino commented on pull request #10225: URL: https://github.com/apache/kafka/pull/10225#issuecomment-788225857 https://issues.apache.org/jira/browse/KAFKA-12381 This is an automated message from the Apache Git Service.

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

2021-03-01 Thread GitBox
rondagostino commented on pull request #10225: URL: https://github.com/apache/kafka/pull/10225#issuecomment-788226851 This PR "fixes" the issue by changing the system test to check for the new error. We can close this PR in favor of another one if @abbccdda decides the behavior change is

[jira] [Created] (KAFKA-12394) Consider topic id existence and authorization errors

2021-03-01 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-12394: --- Summary: Consider topic id existence and authorization errors Key: KAFKA-12394 URL: https://issues.apache.org/jira/browse/KAFKA-12394 Project: Kafka Is

[GitHub] [kafka] hachikuji commented on a change in pull request #10223: MINOR: Do not expose topic name in `DeleteTopic` response if no describe permission

2021-03-01 Thread GitBox
hachikuji commented on a change in pull request #10223: URL: https://github.com/apache/kafka/pull/10223#discussion_r585014029 ## File path: core/src/main/scala/kafka/server/KafkaApis.scala ## @@ -1884,20 +1884,24 @@ class KafkaApis(val requestChannel: RequestChannel, val

[GitHub] [kafka] guozhangwang merged pull request #10205: KAFKA-12323 Follow-up: Refactor the unit test a bit

2021-03-01 Thread GitBox
guozhangwang merged pull request #10205: URL: https://github.com/apache/kafka/pull/10205 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] guozhangwang commented on pull request #10205: KAFKA-12323 Follow-up: Refactor the unit test a bit

2021-03-01 Thread GitBox
guozhangwang commented on pull request #10205: URL: https://github.com/apache/kafka/pull/10205#issuecomment-788238202 Cherry-picked to 2.8 and 2.7, cc RM @vvcephei This is an automated message from the Apache Git Service. To

[GitHub] [kafka] hachikuji commented on a change in pull request #10223: MINOR: Do not expose topic name in `DeleteTopic` response if no describe permission

2021-03-01 Thread GitBox
hachikuji commented on a change in pull request #10223: URL: https://github.com/apache/kafka/pull/10223#discussion_r585017852 ## File path: core/src/main/scala/kafka/server/KafkaApis.scala ## @@ -1874,7 +1874,7 @@ class KafkaApis(val requestChannel: RequestChannel, if

[GitHub] [kafka] hachikuji commented on pull request #10223: MINOR: Do not expose topic name in `DeleteTopic` response if no describe permission

2021-03-01 Thread GitBox
hachikuji commented on pull request #10223: URL: https://github.com/apache/kafka/pull/10223#issuecomment-788239224 @jolshan @chia7712 I don't think `KafkaApis` has been rewritten yet. My suggestion is probably get this checked in as is so that we can definitely get it into 2.8. Then we sho

[GitHub] [kafka] hachikuji edited a comment on pull request #10223: MINOR: Do not expose topic name in `DeleteTopic` response if no describe permission

2021-03-01 Thread GitBox
hachikuji edited a comment on pull request #10223: URL: https://github.com/apache/kafka/pull/10223#issuecomment-788239224 @jolshan @chia7712 I don't think `KafkaApis` has been rewritten yet. My suggestion is probably get this checked in as is so that we can definitely get it into 2.8. Then

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

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

[jira] [Commented] (KAFKA-12394) Consider topic id existence and authorization errors

2021-03-01 Thread Ismael Juma (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12394?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17293155#comment-17293155 ] Ismael Juma commented on KAFKA-12394: - {quote}There is an argument to be made for TO

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

2021-03-01 Thread GitBox
gharris1727 commented on pull request #8259: URL: https://github.com/apache/kafka/pull/8259#issuecomment-788250316 Now that the bug is fixed, some comments are explicitly incorrect. For example: ``` // 1. Lock the PluginClassLoader (via PluginClassLoader::loadClass)

[GitHub] [kafka] hachikuji commented on a change in pull request #10223: MINOR: Do not expose topic name in `DeleteTopic` response if no describe permission

2021-03-01 Thread GitBox
hachikuji commented on a change in pull request #10223: URL: https://github.com/apache/kafka/pull/10223#discussion_r585027051 ## File path: core/src/test/scala/unit/kafka/server/KafkaApisTest.scala ## @@ -3479,6 +3481,161 @@ class KafkaApisTest { assertEquals(List(mkTopicD

[jira] [Updated] (KAFKA-12381) Incompatible change in verifiable_producer.log in 2.8

2021-03-01 Thread Boyang Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12381?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Boyang Chen updated KAFKA-12381: Component/s: core > Incompatible change in verifiable_producer.log in 2.8 > --

[jira] [Updated] (KAFKA-12381) Incompatible change in verifiable_producer.log in 2.8

2021-03-01 Thread Boyang Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12381?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Boyang Chen updated KAFKA-12381: Labels: kip-500 (was: ) > Incompatible change in verifiable_producer.log in 2.8 > ---

[jira] [Commented] (KAFKA-12381) Incompatible change in verifiable_producer.log in 2.8

2021-03-01 Thread Boyang Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12381?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17293161#comment-17293161 ] Boyang Chen commented on KAFKA-12381: - [~cmccabe] Could you provide more specific de

[GitHub] [kafka] jolshan commented on pull request #10223: MINOR: Do not expose topic name in `DeleteTopic` response if no describe permission

2021-03-01 Thread GitBox
jolshan commented on pull request #10223: URL: https://github.com/apache/kafka/pull/10223#issuecomment-788251941 @hachikuji Ah sorry. I was misreading the file name. Makes sense. This is an automated message from the Apache G

[jira] [Commented] (KAFKA-10383) KTable Join on Foreign key is opinionated

2021-03-01 Thread Marco Lotz (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10383?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17293166#comment-17293166 ] Marco Lotz commented on KAFKA-10383: [~mjsax] the KIP is available [here|https://cw

[jira] [Created] (KAFKA-12395) Drop topic mapKey in DeleteTopics response

2021-03-01 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-12395: --- Summary: Drop topic mapKey in DeleteTopics response Key: KAFKA-12395 URL: https://issues.apache.org/jira/browse/KAFKA-12395 Project: Kafka Issue Type:

[GitHub] [kafka] hachikuji commented on a change in pull request #10223: MINOR: Do not expose topic name in `DeleteTopic` response if no describe permission

2021-03-01 Thread GitBox
hachikuji commented on a change in pull request #10223: URL: https://github.com/apache/kafka/pull/10223#discussion_r585036724 ## File path: core/src/main/scala/kafka/server/KafkaApis.scala ## @@ -1884,20 +1884,24 @@ class KafkaApis(val requestChannel: RequestChannel, val

[GitHub] [kafka] vanhoale opened a new pull request #10239: KAFKA-12372 - Enhance TimestampCoverter to handle multiple timestamp or date fields

2021-03-01 Thread GitBox
vanhoale opened a new pull request #10239: URL: https://github.com/apache/kafka/pull/10239 [JIRA](https://issues.apache.org/jira/browse/KAFKA-12372) Our team is having an issue of handling multiple timestamp fields in a kafka message, so for now if we use the converter then we have t

[GitHub] [kafka] MarcoLotz commented on pull request #10042: KAFKA-9527: fix NPE when using time-based argument for Stream Resetter

2021-03-01 Thread GitBox
MarcoLotz commented on pull request #10042: URL: https://github.com/apache/kafka/pull/10042#issuecomment-788295749 @mjsax can you please have a quick look on the updated tests? This is an automated message from the Apache Git

  1   2   >