[jira] [Commented] (KAFKA-12308) ConfigDef.parseType deadlock

2021-02-24 Thread Konstantine Karantasis (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12308?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17290756#comment-17290756 ] Konstantine Karantasis commented on KAFKA-12308: [~tombentley] I actually think that the

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

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

[GitHub] [kafka] kkonstantine commented on pull request #8259: KAFKA-7421: Reproduce Plugin/Delegating ClassLoader deadlock

2021-02-24 Thread GitBox
kkonstantine commented on pull request #8259: URL: https://github.com/apache/kafka/pull/8259#issuecomment-785695890 I posted a change that hopefully fixes the issue and is also mentioned in https://issues.apache.org/jira/browse/KAFKA-7421 The method lock seems indeed to be there as an

[GitHub] [kafka] dpoldrugo commented on pull request #10059: KAFKA-8562: SaslChannelBuilder - avoid (reverse) DNS lookup while building underlying SslTransportLayer

2021-02-24 Thread GitBox
dpoldrugo commented on pull request #10059: URL: https://github.com/apache/kafka/pull/10059#issuecomment-785690250 Thanks @hachikuji and @omkreddy for you comments. Changed the tests as you proposed:

[jira] [Commented] (KAFKA-9203) kafka-client 2.3.1 fails to consume lz4 compressed topic

2021-02-24 Thread Jira
[ https://issues.apache.org/jira/browse/KAFKA-9203?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17290739#comment-17290739 ] Xavier Léauté commented on KAFKA-9203: -- I submitted a PR that will help detect if we have an older

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

2021-02-24 Thread GitBox
chia7712 commented on a change in pull request #10206: URL: https://github.com/apache/kafka/pull/10206#discussion_r582592473 ## File path: core/src/main/scala/kafka/server/KafkaApis.scala ## @@ -3303,6 +3304,28 @@ class KafkaApis(val requestChannel: RequestChannel, new

[GitHub] [kafka] chia7712 commented on a change in pull request #10162: DOCS: Update protocol doc for missing data type

2021-02-24 Thread GitBox
chia7712 commented on a change in pull request #10162: URL: https://github.com/apache/kafka/pull/10162#discussion_r582587709 ## File path: clients/src/main/resources/common/message/README.md ## @@ -75,16 +75,24 @@ There are several primitive field types available. *

[GitHub] [kafka] dengziming commented on a change in pull request #10021: KAFKA-12205: Delete snapshots less than the snapshot at the log start

2021-02-24 Thread GitBox
dengziming commented on a change in pull request #10021: URL: https://github.com/apache/kafka/pull/10021#discussion_r582587511 ## File path: core/src/main/scala/kafka/raft/KafkaMetadataLog.scala ## @@ -299,7 +290,16 @@ final class KafkaMetadataLog private ( // If

[GitHub] [kafka] dengziming commented on a change in pull request #10021: KAFKA-12205: Delete snapshots less than the snapshot at the log start

2021-02-24 Thread GitBox
dengziming commented on a change in pull request #10021: URL: https://github.com/apache/kafka/pull/10021#discussion_r582587332 ## File path: core/src/main/scala/kafka/raft/KafkaMetadataLog.scala ## @@ -341,7 +342,7 @@ object KafkaMetadataLog { } } -val

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

2021-02-24 Thread GitBox
chia7712 commented on a change in pull request #10206: URL: https://github.com/apache/kafka/pull/10206#discussion_r582586860 ## File path: clients/src/main/resources/common/message/ListTransactionsRequest.json ## @@ -0,0 +1,31 @@ +// Licensed to the Apache Software Foundation

[GitHub] [kafka] vamossagar12 commented on pull request #10052: KAFKA-12289: Adding test cases for prefix scan in InMemoryKeyValueStore

2021-02-24 Thread GitBox
vamossagar12 commented on pull request #10052: URL: https://github.com/apache/kafka/pull/10052#issuecomment-785663227 Thanks @cadonna , i have committed your suggestions. This is an automated message from the Apache Git

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

2021-02-24 Thread GitBox
chia7712 commented on a change in pull request #10206: URL: https://github.com/apache/kafka/pull/10206#discussion_r582583372 ## File path: core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala ## @@ -223,6 +224,46 @@ class

[GitHub] [kafka] hachikuji commented on pull request #10206: KAFKA-12369; Implement `ListTransactions` API

2021-02-24 Thread GitBox
hachikuji commented on pull request #10206: URL: https://github.com/apache/kafka/pull/10206#issuecomment-785640598 @chia7712 Thanks, I really appreciate your help reviewing these patches. I pushed a couple comments and left a few responses to your questions.

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

2021-02-24 Thread GitBox
hachikuji commented on a change in pull request #10206: URL: https://github.com/apache/kafka/pull/10206#discussion_r582568344 ## File path: core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala ## @@ -25,8 +25,50 @@ import

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

2021-02-24 Thread GitBox
hachikuji commented on a change in pull request #10206: URL: https://github.com/apache/kafka/pull/10206#discussion_r582567640 ## File path: clients/src/main/resources/common/message/ListTransactionsRequest.json ## @@ -0,0 +1,31 @@ +// Licensed to the Apache Software

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

2021-02-24 Thread GitBox
hachikuji commented on a change in pull request #10206: URL: https://github.com/apache/kafka/pull/10206#discussion_r582566067 ## File path: core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala ## @@ -223,6 +224,46 @@ class

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

2021-02-24 Thread GitBox
hachikuji commented on a change in pull request #10206: URL: https://github.com/apache/kafka/pull/10206#discussion_r582565318 ## File path: core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala ## @@ -223,6 +224,46 @@ class

[GitHub] [kafka] dengziming commented on a change in pull request #10162: DOCS: Update protocol doc for missing data type

2021-02-24 Thread GitBox
dengziming commented on a change in pull request #10162: URL: https://github.com/apache/kafka/pull/10162#discussion_r582564994 ## File path: clients/src/main/resources/common/message/README.md ## @@ -75,16 +75,24 @@ There are several primitive field types available. *

[jira] [Commented] (KAFKA-10847) Avoid spurious left/outer join results in stream-stream join

2021-02-24 Thread Guozhang Wang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10847?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17290685#comment-17290685 ] Guozhang Wang commented on KAFKA-10847: --- I agree that we should delete upon emitting expired

[GitHub] [kafka] ivanyu commented on a change in pull request #9990: KAFKA-12235: Fix ZkAdminManager.describeConfigs on 2+ config keys

2021-02-24 Thread GitBox
ivanyu commented on a change in pull request #9990: URL: https://github.com/apache/kafka/pull/9990#discussion_r582559897 ## File path: core/src/main/scala/kafka/server/ConfigHelper.scala ## @@ -47,11 +47,11 @@ class ConfigHelper(metadataCache: MetadataCache, config:

[GitHub] [kafka] ivanyu commented on a change in pull request #9990: KAFKA-12235: Fix ZkAdminManager.describeConfigs on 2+ config keys

2021-02-24 Thread GitBox
ivanyu commented on a change in pull request #9990: URL: https://github.com/apache/kafka/pull/9990#discussion_r582559897 ## File path: core/src/main/scala/kafka/server/ConfigHelper.scala ## @@ -47,11 +47,11 @@ class ConfigHelper(metadataCache: MetadataCache, config:

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

2021-02-24 Thread GitBox
hachikuji commented on a change in pull request #10206: URL: https://github.com/apache/kafka/pull/10206#discussion_r582558759 ## File path: clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsRequest.java ## @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache

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

2021-02-24 Thread GitBox
chia7712 commented on a change in pull request #10206: URL: https://github.com/apache/kafka/pull/10206#discussion_r582558391 ## File path: clients/src/main/resources/common/message/ListTransactionsRequest.json ## @@ -0,0 +1,31 @@ +// Licensed to the Apache Software Foundation

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

2021-02-24 Thread GitBox
chia7712 commented on a change in pull request #10206: URL: https://github.com/apache/kafka/pull/10206#discussion_r582556446 ## File path: core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala ## @@ -223,6 +224,46 @@ class

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

2021-02-24 Thread GitBox
chia7712 commented on a change in pull request #10206: URL: https://github.com/apache/kafka/pull/10206#discussion_r582549509 ## File path: core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala ## @@ -223,6 +224,46 @@ class

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

2021-02-24 Thread GitBox
chia7712 commented on a change in pull request #10206: URL: https://github.com/apache/kafka/pull/10206#discussion_r582547244 ## File path: core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala ## @@ -25,8 +25,50 @@ import

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

2021-02-24 Thread GitBox
chia7712 commented on a change in pull request #10206: URL: https://github.com/apache/kafka/pull/10206#discussion_r582547180 ## File path: core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala ## @@ -25,8 +25,50 @@ import

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

2021-02-24 Thread GitBox
chia7712 commented on a change in pull request #10206: URL: https://github.com/apache/kafka/pull/10206#discussion_r582545612 ## File path: clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsRequest.java ## @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache

[GitHub] [kafka] omkreddy edited a comment on pull request #10059: KAFKA-8562: SaslChannelBuilder - avoid (reverse) DNS lookup while building underlying SslTransportLayer

2021-02-24 Thread GitBox
omkreddy edited a comment on pull request #10059: URL: https://github.com/apache/kafka/pull/10059#issuecomment-785604829 @dpoldrugo Thanks for the PR. LGTM. As @hachikuji mentioned, we should update `SaslAuthenticatorFailureNoDelayTest`, `SaslAuthenticatorTest` tests. I think, we may

[GitHub] [kafka] omkreddy commented on pull request #10059: KAFKA-8562: SaslChannelBuilder - avoid (reverse) DNS lookup while building underlying SslTransportLayer

2021-02-24 Thread GitBox
omkreddy commented on pull request #10059: URL: https://github.com/apache/kafka/pull/10059#issuecomment-785604829 @dpoldrugo Thanks for the PR. LGTM. As @hachikuji mentioned, we should update `SaslAuthenticatorFailureNoDelayTest`, `SaslAuthenticatorTest` tests. I think, we may not

[GitHub] [kafka] chia7712 commented on a change in pull request #10162: DOCS: Update protocol doc for missing data type

2021-02-24 Thread GitBox
chia7712 commented on a change in pull request #10162: URL: https://github.com/apache/kafka/pull/10162#discussion_r582529627 ## File path: clients/src/main/resources/common/message/README.md ## @@ -75,16 +75,24 @@ There are several primitive field types available. *

[GitHub] [kafka] twmb commented on pull request #10183: KAFKA-12267; Implement `DescribeTransactions` API

2021-02-24 Thread GitBox
twmb commented on pull request #10183: URL: https://github.com/apache/kafka/pull/10183#issuecomment-785551804 Also is there a reason not to include the `LastUpdateTimestamp`? It seems pretty valuable to know how long ago the transaction state was modified. I'm not sure how valuable

[GitHub] [kafka] twmb edited a comment on pull request #10183: KAFKA-12267; Implement `DescribeTransactions` API

2021-02-24 Thread GitBox
twmb edited a comment on pull request #10183: URL: https://github.com/apache/kafka/pull/10183#issuecomment-785548579 Can the KIP be updated to change TransactionState from an int8 to a string (as implemented in this PR)? Alternatively, is there value in using a string vs. an int8?

[GitHub] [kafka] hachikuji commented on a change in pull request #10021: KAFKA-12205: Delete snapshots less than the snapshot at the log start

2021-02-24 Thread GitBox
hachikuji commented on a change in pull request #10021: URL: https://github.com/apache/kafka/pull/10021#discussion_r582491373 ## File path: raft/src/main/java/org/apache/kafka/snapshot/Snapshots.java ## @@ -92,4 +100,22 @@ public static Path createTempFile(Path logDir,

[GitHub] [kafka] hachikuji commented on a change in pull request #10021: KAFKA-12205: Delete snapshots less than the snapshot at the log start

2021-02-24 Thread GitBox
hachikuji commented on a change in pull request #10021: URL: https://github.com/apache/kafka/pull/10021#discussion_r582471506 ## File path: core/src/main/scala/kafka/raft/KafkaMetadataLog.scala ## @@ -299,7 +302,16 @@ final class KafkaMetadataLog private ( // If

[GitHub] [kafka] twmb commented on pull request #10183: KAFKA-12267; Implement `DescribeTransactions` API

2021-02-24 Thread GitBox
twmb commented on pull request #10183: URL: https://github.com/apache/kafka/pull/10183#issuecomment-785548579 Can the KIP be updated to change TransactionState from an int8 to a string (as implemented in this PR)? This is

[GitHub] [kafka] dengziming commented on a change in pull request #10021: KAFKA-12205: Delete snapshots less than the snapshot at the log start

2021-02-24 Thread GitBox
dengziming commented on a change in pull request #10021: URL: https://github.com/apache/kafka/pull/10021#discussion_r582479381 ## File path: raft/src/main/java/org/apache/kafka/snapshot/Snapshots.java ## @@ -92,4 +100,22 @@ public static Path createTempFile(Path logDir,

[GitHub] [kafka] dengziming commented on a change in pull request #10021: KAFKA-12205: Delete snapshots less than the snapshot at the log start

2021-02-24 Thread GitBox
dengziming commented on a change in pull request #10021: URL: https://github.com/apache/kafka/pull/10021#discussion_r582477733 ## File path: raft/src/main/java/org/apache/kafka/snapshot/Snapshots.java ## @@ -92,4 +100,22 @@ public static Path createTempFile(Path logDir,

[GitHub] [kafka] hachikuji opened a new pull request #10206: KAFKA-12369; Implement `ListTransactions` API

2021-02-24 Thread GitBox
hachikuji opened a new pull request #10206: URL: https://github.com/apache/kafka/pull/10206 This patch implements the `ListTransactions` API as documented in KIP-664: https://cwiki.apache.org/confluence/display/KAFKA/KIP-664%3A+Provide+tooling+to+detect+and+abort+hanging+transactions.

[jira] [Commented] (KAFKA-12371) MirrorMaker 2.0 documentation is incorrect

2021-02-24 Thread Luke Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17290598#comment-17290598 ] Luke Chen commented on KAFKA-12371: --- [~Scott-kirk], thanks for reporting the issue. I've fixed this

[GitHub] [kafka] dengziming commented on a change in pull request #10162: DOCS: Update protocol doc for missing data type

2021-02-24 Thread GitBox
dengziming commented on a change in pull request #10162: URL: https://github.com/apache/kafka/pull/10162#discussion_r582432795 ## File path: clients/src/main/resources/common/message/README.md ## @@ -75,16 +75,24 @@ There are several primitive field types available. *

[GitHub] [kafka] mjsax merged pull request #10186: MINOR: bump release version to 3.0.0-SNAPSHOT

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

2021-02-24 Thread GitBox
guozhangwang commented on a change in pull request #10205: URL: https://github.com/apache/kafka/pull/10205#discussion_r582428237 ## File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java ## @@ -1936,7 +1934,7 @@ public void

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

2021-02-24 Thread GitBox
guozhangwang commented on a change in pull request #10205: URL: https://github.com/apache/kafka/pull/10205#discussion_r582426740 ## File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java ## @@ -1844,19 +1844,17 @@ public void

[GitHub] [kafka] highluck commented on a change in pull request #9851: KAFKA-10769 Remove JoinGroupRequest#containsValidPattern as it is dup…

2021-02-24 Thread GitBox
highluck commented on a change in pull request #9851: URL: https://github.com/apache/kafka/pull/9851#discussion_r582425104 ## File path: clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java ## @@ -59,38 +59,12 @@ public String toString() { public

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

2021-02-24 Thread GitBox
jolshan commented on a change in pull request #9590: URL: https://github.com/apache/kafka/pull/9590#discussion_r582415502 ## File path: core/src/main/scala/kafka/log/LogCleaner.scala ## @@ -701,11 +719,17 @@ private[log] class Cleaner(val id: Int, // if any messages are

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

2021-02-24 Thread GitBox
jolshan commented on a change in pull request #9590: URL: https://github.com/apache/kafka/pull/9590#discussion_r582413737 ## File path: core/src/main/scala/kafka/log/LogCleaner.scala ## @@ -701,11 +719,17 @@ private[log] class Cleaner(val id: Int, // if any messages are

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

2021-02-24 Thread GitBox
jolshan commented on a change in pull request #9590: URL: https://github.com/apache/kafka/pull/9590#discussion_r582413737 ## File path: core/src/main/scala/kafka/log/LogCleaner.scala ## @@ -701,11 +719,17 @@ private[log] class Cleaner(val id: Int, // if any messages are

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

2021-02-24 Thread GitBox
jolshan commented on a change in pull request #9590: URL: https://github.com/apache/kafka/pull/9590#discussion_r582413737 ## File path: core/src/main/scala/kafka/log/LogCleaner.scala ## @@ -701,11 +719,17 @@ private[log] class Cleaner(val id: Int, // if any messages are

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

2021-02-24 Thread GitBox
jolshan commented on a change in pull request #9590: URL: https://github.com/apache/kafka/pull/9590#discussion_r582413737 ## File path: core/src/main/scala/kafka/log/LogCleaner.scala ## @@ -701,11 +719,17 @@ private[log] class Cleaner(val id: Int, // if any messages are

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

2021-02-24 Thread GitBox
jolshan commented on a change in pull request #9590: URL: https://github.com/apache/kafka/pull/9590#discussion_r582407274 ## File path: core/src/main/scala/kafka/log/LogCleaner.scala ## @@ -599,21 +606,30 @@ private[log] class Cleaner(val id: Int, }

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

2021-02-24 Thread GitBox
jolshan commented on a change in pull request #9590: URL: https://github.com/apache/kafka/pull/9590#discussion_r582406089 ## File path: core/src/main/scala/kafka/log/LogCleaner.scala ## @@ -599,21 +606,30 @@ private[log] class Cleaner(val id: Int, }

[jira] [Resolved] (KAFKA-12211) NoSuchFileException will be thrown if hasPersistentStores is false when creating stateDir

2021-02-24 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12211?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman resolved KAFKA-12211. Resolution: Fixed > NoSuchFileException will be thrown if

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

2021-02-24 Thread GitBox
jolshan commented on a change in pull request #9590: URL: https://github.com/apache/kafka/pull/9590#discussion_r582402100 ## File path: core/src/main/scala/kafka/log/LogCleaner.scala ## @@ -582,13 +586,16 @@ private[log] class Cleaner(val id: Int,

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

2021-02-24 Thread GitBox
jolshan commented on a change in pull request #9590: URL: https://github.com/apache/kafka/pull/9590#discussion_r582402100 ## File path: core/src/main/scala/kafka/log/LogCleaner.scala ## @@ -582,13 +586,16 @@ private[log] class Cleaner(val id: Int,

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

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

[jira] [Closed] (KAFKA-10810) Add a replace thread option to the streams uncaught exception handler

2021-02-24 Thread Walker Carlson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10810?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Walker Carlson closed KAFKA-10810. -- > Add a replace thread option to the streams uncaught exception handler >

[jira] [Closed] (KAFKA-9331) Add option to terminate application when StreamThread(s) die

2021-02-24 Thread Walker Carlson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9331?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Walker Carlson closed KAFKA-9331. - > Add option to terminate application when StreamThread(s) die >

[jira] [Closed] (KAFKA-4748) Need a way to shutdown all workers in a Streams application at the same time

2021-02-24 Thread Walker Carlson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-4748?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Walker Carlson closed KAFKA-4748. - > Need a way to shutdown all workers in a Streams application at the same time >

[jira] [Closed] (KAFKA-6943) Have option to shutdown KS cleanly if any threads crashes, or if all threads crash

2021-02-24 Thread Walker Carlson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-6943?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Walker Carlson closed KAFKA-6943. - > Have option to shutdown KS cleanly if any threads crashes, or if all threads > crash >

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

2021-02-24 Thread GitBox
jolshan commented on a change in pull request #9590: URL: https://github.com/apache/kafka/pull/9590#discussion_r582363641 ## File path: core/src/main/scala/kafka/log/LogCleaner.scala ## @@ -1094,6 +1119,8 @@ private[log] class CleanedTransactionMetadata { // Output

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

2021-02-24 Thread GitBox
hachikuji commented on a change in pull request #9590: URL: https://github.com/apache/kafka/pull/9590#discussion_r582343277 ## File path: clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java ## @@ -198,7 +198,7 @@ private static FilterResult

[jira] [Commented] (KAFKA-10847) Avoid spurious left/outer join results in stream-stream join

2021-02-24 Thread Jira
[ https://issues.apache.org/jira/browse/KAFKA-10847?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17290502#comment-17290502 ] Sergio Peña commented on KAFKA-10847: - [~mjsax] Just did a quick test when deleting the emitted the

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

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

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

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

[jira] [Created] (KAFKA-12374) Add missing config sasl.mechanism.controller.protocol

2021-02-24 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-12374: - Summary: Add missing config sasl.mechanism.controller.protocol Key: KAFKA-12374 URL: https://issues.apache.org/jira/browse/KAFKA-12374 Project: Kafka

[GitHub] [kafka] mjsax commented on a change in pull request #10205: KAFKA-12323 Follow-up: Refactor the unit test a bit

2021-02-24 Thread GitBox
mjsax commented on a change in pull request #10205: URL: https://github.com/apache/kafka/pull/10205#discussion_r582349190 ## File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java ## @@ -1844,19 +1844,17 @@ public void

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

2021-02-24 Thread GitBox
hachikuji commented on a change in pull request #10112: URL: https://github.com/apache/kafka/pull/10112#discussion_r582259574 ## File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java ## @@ -98,7 +98,8 @@ private IdentityHashMap,

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

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

[jira] [Resolved] (KAFKA-12267) Implement DescribeTransactions API

2021-02-24 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12267?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-12267. - Resolution: Fixed > Implement DescribeTransactions API >

[GitHub] [kafka] hachikuji merged pull request #10183: KAFKA-12267; Implement `DescribeTransactions` API

2021-02-24 Thread GitBox
hachikuji merged pull request #10183: URL: https://github.com/apache/kafka/pull/10183 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] cmccabe commented on pull request #9990: KAFKA-12235: Fix ZkAdminManager.describeConfigs on 2+ config keys

2021-02-24 Thread GitBox
cmccabe commented on pull request #9990: URL: https://github.com/apache/kafka/pull/9990#issuecomment-785357271 ping @ivanyu This is an automated message from the Apache Git Service. To respond to the message, please log on

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

2021-02-24 Thread GitBox
cmccabe commented on pull request #10184: URL: https://github.com/apache/kafka/pull/10184#issuecomment-785354588 rebased on trunk This is an automated message from the Apache Git Service. To respond to the message, please

[jira] [Updated] (KAFKA-12373) Improve KafkaRaftClient handling of graceful shutdown

2021-02-24 Thread Jose Armando Garcia Sancio (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12373?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jose Armando Garcia Sancio updated KAFKA-12373: --- Description: The current implementation simply closes the metrics

[GitHub] [kafka] guozhangwang commented on pull request #10204: MINOR: Fix the generation extraction util

2021-02-24 Thread GitBox
guozhangwang commented on pull request #10204: URL: https://github.com/apache/kafka/pull/10204#issuecomment-785350477 > Sorry that I didn't notice this issue :( No worries!! :) This is an automated message from the

[GitHub] [kafka] guozhangwang merged pull request #10204: MINOR: Fix the generation extraction util

2021-02-24 Thread GitBox
guozhangwang merged pull request #10204: URL: https://github.com/apache/kafka/pull/10204 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-12373) Improve KafkaRaftClient handling of graceful shutdown

2021-02-24 Thread Jose Armando Garcia Sancio (Jira)
Jose Armando Garcia Sancio created KAFKA-12373: -- Summary: Improve KafkaRaftClient handling of graceful shutdown Key: KAFKA-12373 URL: https://issues.apache.org/jira/browse/KAFKA-12373

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

2021-02-24 Thread GitBox
ijuma commented on a change in pull request #10184: URL: https://github.com/apache/kafka/pull/10184#discussion_r582266116 ## File path: core/src/main/scala/kafka/server/ControllerApis.scala ## @@ -195,6 +198,61 @@ class ControllerApis(val requestChannel: RequestChannel,

[jira] [Commented] (KAFKA-12158) Consider better return type of RaftClient.scheduleAppend

2021-02-24 Thread Jose Armando Garcia Sancio (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12158?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17290233#comment-17290233 ] Jose Armando Garcia Sancio commented on KAFKA-12158: Yes [~sagarrao] . Feel free to

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

2021-02-24 Thread GitBox
cmccabe commented on a change in pull request #10184: URL: https://github.com/apache/kafka/pull/10184#discussion_r582259112 ## File path: core/src/main/scala/kafka/server/ControllerApis.scala ## @@ -195,6 +198,61 @@ class ControllerApis(val requestChannel: RequestChannel,

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

2021-02-24 Thread GitBox
cmccabe commented on a change in pull request #10184: URL: https://github.com/apache/kafka/pull/10184#discussion_r582257154 ## File path: metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java ## @@ -541,6 +559,41 @@ static void

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

2021-02-24 Thread GitBox
cmccabe commented on a change in pull request #10184: URL: https://github.com/apache/kafka/pull/10184#discussion_r582257154 ## File path: metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java ## @@ -541,6 +559,41 @@ static void

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

2021-02-24 Thread GitBox
cmccabe commented on a change in pull request #10184: URL: https://github.com/apache/kafka/pull/10184#discussion_r582256432 ## File path: core/src/main/scala/kafka/server/ControllerApis.scala ## @@ -195,6 +198,61 @@ class ControllerApis(val requestChannel: RequestChannel,

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

2021-02-24 Thread GitBox
cmccabe commented on a change in pull request #10184: URL: https://github.com/apache/kafka/pull/10184#discussion_r582256432 ## File path: core/src/main/scala/kafka/server/ControllerApis.scala ## @@ -195,6 +198,61 @@ class ControllerApis(val requestChannel: RequestChannel,

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

2021-02-24 Thread GitBox
cmccabe commented on a change in pull request #10184: URL: https://github.com/apache/kafka/pull/10184#discussion_r582255681 ## File path: core/src/main/scala/kafka/server/ControllerApis.scala ## @@ -195,6 +198,61 @@ class ControllerApis(val requestChannel: RequestChannel,

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

2021-02-24 Thread GitBox
cmccabe commented on a change in pull request #10184: URL: https://github.com/apache/kafka/pull/10184#discussion_r582255514 ## File path: core/src/main/scala/kafka/server/ControllerApis.scala ## @@ -195,6 +198,61 @@ class ControllerApis(val requestChannel: RequestChannel,

[GitHub] [kafka] hachikuji commented on pull request #10059: KAFKA-8562: SaslChannelBuilder - avoid (reverse) DNS lookup while building underlying SslTransportLayer

2021-02-24 Thread GitBox
hachikuji commented on pull request #10059: URL: https://github.com/apache/kafka/pull/10059#issuecomment-785326212 I think the test failures are caused by this:

[GitHub] [kafka] vvcephei commented on pull request #10051: Adding documentation for KIP-614

2021-02-24 Thread GitBox
vvcephei commented on pull request #10051: URL: https://github.com/apache/kafka/pull/10051#issuecomment-785322634 Merged to 2.8 This is an automated message from the Apache Git Service. To respond to the message, please log

[GitHub] [kafka] vvcephei merged pull request #10051: Adding documentation for KIP-614

2021-02-24 Thread GitBox
vvcephei merged pull request #10051: URL: https://github.com/apache/kafka/pull/10051 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] vvcephei commented on pull request #10051: Adding documentation for KIP-614

2021-02-24 Thread GitBox
vvcephei commented on pull request #10051: URL: https://github.com/apache/kafka/pull/10051#issuecomment-785321000 Fixed the merge conflict. This is an automated message from the Apache Git Service. To respond to the message,

[jira] [Updated] (KAFKA-12372) Enhance TimestampCoverter to handle multiple timestamp or date fields

2021-02-24 Thread Hoa Le (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12372?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Hoa Le updated KAFKA-12372: --- Summary: Enhance TimestampCoverter to handle multiple timestamp or date fields (was: Enhance

[jira] [Created] (KAFKA-12372) Enhance TimestampCoverter to handle fields schema name to convert to Timestamp or Date

2021-02-24 Thread Hoa Le (Jira)
Hoa Le created KAFKA-12372: -- Summary: Enhance TimestampCoverter to handle fields schema name to convert to Timestamp or Date Key: KAFKA-12372 URL: https://issues.apache.org/jira/browse/KAFKA-12372 Project:

[GitHub] [kafka] hachikuji commented on pull request #10059: KAFKA-8562: SaslChannelBuilder - avoid (reverse) DNS lookup while building underlying SslTransportLayer

2021-02-24 Thread GitBox
hachikuji commented on pull request #10059: URL: https://github.com/apache/kafka/pull/10059#issuecomment-785315990 Hmm.. I may have been too hasty with my approval. We need to investigate the test failures. I've triggered another build.

[GitHub] [kafka] rondagostino commented on pull request #10199: MINOR: Fix security_test system test for Raft case

2021-02-24 Thread GitBox
rondagostino commented on pull request #10199: URL: https://github.com/apache/kafka/pull/10199#issuecomment-785311895 @mumrah I separated out the communication-to-quorum failure cases into a separate test -- so now we confirm that hostname verification failure to both ZK and Raft

[GitHub] [kafka] hachikuji commented on a change in pull request #10183: KAFKA-12267; Implement `DescribeTransactions` API

2021-02-24 Thread GitBox
hachikuji commented on a change in pull request #10183: URL: https://github.com/apache/kafka/pull/10183#discussion_r582219028 ## File path: core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala ## @@ -255,6 +256,47 @@ class

[GitHub] [kafka] hachikuji commented on a change in pull request #10183: KAFKA-12267; Implement `DescribeTransactions` API

2021-02-24 Thread GitBox
hachikuji commented on a change in pull request #10183: URL: https://github.com/apache/kafka/pull/10183#discussion_r582213713 ## File path: core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala ## @@ -255,6 +256,47 @@ class

[GitHub] [kafka] mjsax commented on a change in pull request #10170: KAFKA-12323: Set timestamp in record context when punctuate

2021-02-24 Thread GitBox
mjsax commented on a change in pull request #10170: URL: https://github.com/apache/kafka/pull/10170#discussion_r582202700 ## File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java ## @@ -1874,6 +1873,85 @@ public void close() {}

[GitHub] [kafka] mjsax commented on a change in pull request #10170: KAFKA-12323: Set timestamp in record context when punctuate

2021-02-24 Thread GitBox
mjsax commented on a change in pull request #10170: URL: https://github.com/apache/kafka/pull/10170#discussion_r582202700 ## File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java ## @@ -1874,6 +1873,85 @@ public void close() {}

[jira] [Commented] (KAFKA-10847) Avoid spurious left/outer join results in stream-stream join

2021-02-24 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10847?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17290176#comment-17290176 ] Matthias J. Sax commented on KAFKA-10847: - Thanks for the update! {quote}I still need to know

[jira] [Commented] (KAFKA-12360) Improve documentation of max.task.idle.ms (kafka-streams)

2021-02-24 Thread ASF GitHub Bot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12360?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17290164#comment-17290164 ] ASF GitHub Bot commented on KAFKA-12360: mjsax commented on pull request #333: URL:

  1   2   >