Re: [DISCUSS] KIP-596 Safely abort Producer transactions during application shutdown

2020-04-16 Thread 张祥
Guozhang, thanks for the valuable suggestion. A new part called "suggested coding pattern" has been added and I copy the core code here: try { producer.beginTransaction(); for (int i = 0; i < 100; i++) producer.send(new ProducerRecord<>("my-topic", Integer.toString(i),

Re: [KAFKA-9861] Process Simplification - Community Validation of Kafka Release Candidates

2020-04-16 Thread Gwen Shapira
Hi Israel, I don't think a KIP is required. We evolved our release process few times without KIPs, IIRC. Subtasks in JIRA seem like a good place to track this right now. Regarding the specific tasks: - Most of the sample scripts and environment setup can/should be completely decoupled from

Build failed in Jenkins: kafka-trunk-jdk11 #1361

2020-04-16 Thread Apache Jenkins Server
See Changes: [wangguoz] HOTFIX: fix active task process ratio metric recording -- [...truncated 6.09 MB...] org.apache.kafka.streams.test.ConsumerRecordFactoryTest >

Jenkins build is back to normal : kafka-trunk-jdk8 #4442

2020-04-16 Thread Apache Jenkins Server
See

Re: [Vote] KIP-588: Allow producers to recover gracefully from transaction timeouts

2020-04-16 Thread Boyang Chen
Thanks Jason and Guozhang for the thoughts. On Thu, Apr 16, 2020 at 6:09 PM Guozhang Wang wrote: > For 2/3 above, originally I was not thinking that we will have a different > exception for INVALID_PRODUCER_EPOCH and hence was thinking that in order > to leverage KIP-360 for it, we'd have to

Re: [DISCUSS] KIP-594: Expose output topic names from TopologyTestDriver

2020-04-16 Thread John Roesler
Thanks Matthias, It sounds like your proposal would be instead to add methods to TopologyDescription: getOuputTopicNames(), getInternalTopicNames(), and getInputTopicNames(). This sounds good to me. For one thing, it nicely resolved the ambiguity with the method name. What do you think about

Re: [VOTE] KIP-584: Versioning scheme for features

2020-04-16 Thread Boyang Chen
Thanks Kowshik for driving this effort, I'm +1 (non-binding). On Thu, Apr 16, 2020 at 11:14 AM Kowshik Prakasam wrote: > Hi all, > > I'd like to start a vote for KIP-584. The link to the KIP can be found > here: > >

Re: [Vote] KIP-588: Allow producers to recover gracefully from transaction timeouts

2020-04-16 Thread Guozhang Wang
For 2/3 above, originally I was not thinking that we will have a different exception for INVALID_PRODUCER_EPOCH and hence was thinking that in order to leverage KIP-360 for it, we'd have to let the broker to return UNKNOWN_PRODUCER_ID. I.e. we'd change the logic of partition leader as well to

Re: [DISCUSS] KIP-596 Safely abort Producer transactions during application shutdown

2020-04-16 Thread Guozhang Wang
Xiang, thanks for the written KIP. I just have one meta comment and otherwise it looks good to me: could you also add a section about suggested coding patterns (especially how try - catch should be implemented) as we discussed on the JIRA to the wiki page as well? And please also note that

Re: [Vote] KIP-588: Allow producers to recover gracefully from transaction timeouts

2020-04-16 Thread Jason Gustafson
Hi Boyang, A few minor questions below: 1. You mention UNKNOWN_PRODUCER_ID in 2.a under Resilience Improvements. I assume that should be INVALID_PRODUCER_EPOCH? I am not sure this case makes sense for 2.5 clients which would view this error as fatal regardless of whatever the broker does. Not

Re: [DISCUSS] KIP-594: Expose output topic names from TopologyTestDriver

2020-04-16 Thread Matthias J. Sax
John, the "dynamic routing" case is certainly interesting. However, your argument about "simulating brokers" and verify which topics are created does not really hold up IMHO, because the names of all internal topics are contained in the TopologyDescription, too. To be fair, the

[DISCUSS] KIP-595: A Raft Protocol for the Metadata Quorum

2020-04-16 Thread Jason Gustafson
Hi All, I'd like to start a discussion on KIP-595: https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum. This proposal specifies a Raft protocol to ultimately replace Zookeeper as documented in KIP-500. Please take a look and share your thoughts. A

Build failed in Jenkins: kafka-trunk-jdk11 #1360

2020-04-16 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9796; Ensure broker shutdown is not stuck when Acceptor is waiting -- [...truncated 6.09 MB...]

[jira] [Created] (KAFKA-9883) Connect request to restart task can result in IllegalArgumentError: "uriTemplate" parameter is null

2020-04-16 Thread Randall Hauch (Jira)
Randall Hauch created KAFKA-9883: Summary: Connect request to restart task can result in IllegalArgumentError: "uriTemplate" parameter is null Key: KAFKA-9883 URL: https://issues.apache.org/jira/browse/KAFKA-9883

Build failed in Jenkins: kafka-trunk-jdk8 #4441

2020-04-16 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9796; Ensure broker shutdown is not stuck when Acceptor is waiting -- [...truncated 6.05 MB...]

Re: [Vote] KIP-588: Allow producers to recover gracefully from transaction timeouts

2020-04-16 Thread Guozhang Wang
+1 (binding), thanks! On Tue, Apr 14, 2020 at 4:36 PM Boyang Chen wrote: > Hey all, > > I would like to start the vote for KIP-588: > https://cwiki.apache.org/confluence/display/KAFKA/KIP-588 > %3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts > > Feel free to continue posting

Build failed in Jenkins: kafka-2.5-jdk8 #97

2020-04-16 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-9854 Re-authenticating causes mismatched parse of response (#8471) -- [...truncated 5.89 MB...]

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-16 Thread Boyang Chen
Thanks Jose for the questions! On Thu, Apr 16, 2020 at 12:33 PM Jose Garcia Sancio wrote: > Hi Boyang, > > Thanks for the KIP. The KIP looks good. I have a few questions and > comments. > > > As part of the KIP-500 > < >

Re: [DISCUSS] KIP-589 Add API to Update Replica State in Controller

2020-04-16 Thread Jose Garcia Sancio
Hi David, Thanks for the KIP. > ReplicaStateEventResponse => ErrorCode [Topic [PartitionId]] >ErrorCode => Int32 >Topic => String >PartitionId => Int32 > ... > Partition-level errors: Based on my understanding of the response, it doesn't look like the controller has a way of

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-16 Thread Jose Garcia Sancio
Hi Boyang, Thanks for the KIP. The KIP looks good. I have a few questions and comments. > As part of the KIP-500 initiative, we need to build a bridge release version of Kafka

Jenkins build is back to normal : kafka-trunk-jdk11 #1359

2020-04-16 Thread Apache Jenkins Server
See

Build failed in Jenkins: kafka-trunk-jdk8 #4440

2020-04-16 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9854 Re-authenticating causes mismatched parse of response (#8471) [github] Add log message in release.py (#8461) [github] MINOR: Use streaming iterator with decompression buffer

[VOTE] KIP-584: Versioning scheme for features

2020-04-16 Thread Kowshik Prakasam
Hi all, I'd like to start a vote for KIP-584. The link to the KIP can be found here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-584%3A+Versioning+scheme+for+features . Thanks! Cheers, Kowshik

Re: [DISCUSS] KIP-597: MirrorMaker2 internal topics Formatters

2020-04-16 Thread Ryanne Dolan
Thanks Mickael, this will be very useful. Ryanne On Thu, Apr 16, 2020 at 11:44 AM Mickael Maison wrote: > Hi, > > I have submitted a small KIP to provide Formatters for the > MirrorMaker2 internal topics. > > >

[DISCUSS] KIP-597: MirrorMaker2 internal topics Formatters

2020-04-16 Thread Mickael Maison
Hi, I have submitted a small KIP to provide Formatters for the MirrorMaker2 internal topics. https://cwiki.apache.org/confluence/display/KAFKA/KIP-597%3A+MirrorMaker2+internal+topics+Formatters Looking forward to your feedback. Thanks

Re: [VOTE] KIP-409: Allow creating under-replicated topics and partitions

2020-04-16 Thread Mickael Maison
Bumping this thread. We are still missing binding votes. Please take a look at the KIP https://cwiki.apache.org/confluence/display/KAFKA/KIP-409%3A+Allow+creating+under-replicated+topics+and+partitions and let me know if you have any feedback. Thanks On Fri, Mar 6, 2020 at 10:03 AM Mickael

Re: [ANNOUNCE] Apache Kafka 2.5.0

2020-04-16 Thread David Arthur
I've just published a blog post highlighting many of the improvements that landed with 2.5.0. https://blogs.apache.org/kafka/entry/what-s-new-in-apache2 -David On Wed, Apr 15, 2020 at 4:15 PM David Arthur wrote: > The Apache Kafka community is pleased to announce the release for Apache >

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-16 Thread Boyang Chen
Hey Ismael, my understanding is that we are inevitably making the controller single point, even when the metadata quorum work is done. Redirecting won't make things easier as the controller will still be the role to perform alterations. Just for the sake of argument, DescribeTopic is not

[jira] [Resolved] (KAFKA-9796) Broker shutdown could be stuck forever under certain conditions

2020-04-16 Thread Rajini Sivaram (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9796?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-9796. --- Fix Version/s: 2.6.0 Reviewer: Rajini Sivaram Resolution: Fixed > Broker

[jira] [Created] (KAFKA-9882) Add Block getAssignments()

2020-04-16 Thread Jesse Anderson (Jira)
Jesse Anderson created KAFKA-9882: - Summary: Add Block getAssignments() Key: KAFKA-9882 URL: https://issues.apache.org/jira/browse/KAFKA-9882 Project: Kafka Issue Type: New Feature

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-16 Thread Ismael Juma
I don't think these requests are necessarily infrequent under multi tenant environments though. I've seen Controller availability being an issue for describe topics for example (before it was changed to go to any broker). Would it be better to redirect once the controller quorum is there? Note

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-16 Thread Boyang Chen
Thanks David for the review! Great suggestion, addressed in the KIP. On Thu, Apr 16, 2020 at 7:40 AM David Jacot wrote: > Hi Boyang, > > Thanks for the KIP. Overall, it looks good to me. I really like the > envelope RPC! > > One minor comment regarding the `old-client-connections-count` metric.

Build failed in Jenkins: kafka-2.4-jdk8 #188

2020-04-16 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-9838; Add log concurrency test and fix minor race condition -- [...truncated 7.04 MB...]

Kafka: documentation - reliable reference?

2020-04-16 Thread Walenwein. Jakob
Hello Apache Kafka team, comparing the 2.4.1 code state of KafkaProducer with documentation, i noticed the following difference: The "send(record, callback)" method catches internally the apiException's and set it into Future-Object. The callback object handles this exceptions afterwards. But

Re: [ANNOUNCE] Apache Kafka 2.5.0

2020-04-16 Thread Guozhang Wang
Thanks David for driving the release! Congrats all. Guozhang On Thu, Apr 16, 2020 at 1:50 AM Mickael Maison wrote: > Thanks David for running this release! > > On Thu, Apr 16, 2020 at 5:41 AM Navinder Brar > wrote: > > > > Thanks for running the release David. Congratulations to everyone >

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-16 Thread David Jacot
Hi Boyang, Thanks for the KIP. Overall, it looks good to me. I really like the envelope RPC! One minor comment regarding the `old-client-connections-count` metric. Is it really necessary? The number of connected clients whose version is not known (prior to KIP-511) is already reported but with

[jira] [Created] (KAFKA-9881) Verify whether RocksDBMetrics Get Measurements from RocksDB in a Unit Test instead of a Intergration Test

2020-04-16 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-9881: Summary: Verify whether RocksDBMetrics Get Measurements from RocksDB in a Unit Test instead of a Intergration Test Key: KAFKA-9881 URL:

Build failed in Jenkins: kafka-2.5-jdk8 #96

2020-04-16 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-9838; Add log concurrency test and fix minor race condition -- [...truncated 5.89 MB...] org.apache.kafka.streams.MockProcessorContextTest >

[jira] [Resolved] (KAFKA-9854) Re-authenticating causes mismatched parse of response

2020-04-16 Thread Rajini Sivaram (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9854?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-9854. --- Fix Version/s: 2.5.1 2.6.0 Reviewer: Rajini Sivaram

[jira] [Created] (KAFKA-9880) Error while range compacting during bulk loading of FIFO compacted RocksDB Store

2020-04-16 Thread Nicolas Carlot (Jira)
Nicolas Carlot created KAFKA-9880: - Summary: Error while range compacting during bulk loading of FIFO compacted RocksDB Store Key: KAFKA-9880 URL: https://issues.apache.org/jira/browse/KAFKA-9880

[jira] [Created] (KAFKA-9879) How kafka deletes tombstone messages?

2020-04-16 Thread VIkram (Jira)
VIkram created KAFKA-9879: - Summary: How kafka deletes tombstone messages? Key: KAFKA-9879 URL: https://issues.apache.org/jira/browse/KAFKA-9879 Project: Kafka Issue Type: Bug Reporter:

Re: [DISCUSS] KIP-578: Add configuration to limit number of partitions

2020-04-16 Thread Gokul Ramanan Subramanian
Hi. For the sake of expediting the discussion, I have created a prototype PR: https://github.com/apache/kafka/pull/8499. Eventually, (if and) when the KIP is accepted, I'll modify this to add the full implementation and tests etc. in there. Would appreciate if a Kafka committer could share their

Re: [DISCUSS] KIP-578: Add configuration to limit number of partitions

2020-04-16 Thread Gokul Ramanan Subramanian
Thanks for your comments Alex. The KIP proposes using two configurations max.partitions and max.broker.partitions. It does not enforce their use. The default values are pretty large (INT MAX), therefore should be non-intrusive. In multi-tenant environments and in partition assignment and

[jira] [Resolved] (KAFKA-9117) Add active-process-ratio Metric

2020-04-16 Thread Bruno Cadonna (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9117?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bruno Cadonna resolved KAFKA-9117. -- Fix Version/s: (was: 2.6.0) Resolution: Duplicate > Add active-process-ratio Metric

Restart without shutdown log

2020-04-16 Thread 张祥
Hi, I am using Ambari to manage Kafka, info as listed below: Ambari version: 2.7.4.0 Kafka version: 2.0.0 The problem I ran into is that one broker restarts without shutdown log, which makes it difficult to track down the reason. The related logs are as follows, in which I cannot find "shut

Re: [ANNOUNCE] Apache Kafka 2.5.0

2020-04-16 Thread Mickael Maison
Thanks David for running this release! On Thu, Apr 16, 2020 at 5:41 AM Navinder Brar wrote: > > Thanks for running the release David. Congratulations to everyone involved. > > -Navinder Pal Singh Brar > > > On Thursday, 16 April, 2020, 07:26:25 am IST, Konstantine Karantasis > wrote: > >

endless authorizer log

2020-04-16 Thread 张祥
I am using Ambari to manage Kafka, info listed below: Ambari version: 2.7.4.0 Kafka version: 2.0.0 broker number: 10 On every broker, authorizer logger keeps outputting following logs: [2020-04-14 07:56:40,214] INFO Principal = User:xxx is Denied Operation = Describe from host = 10.90.1.213 on

Re: [DISCUSS] KIP-596 Safely abort Producer transactions during application shutdown

2020-04-16 Thread 张祥
Thanks for the structure change Boyang. And I agree with you on the weak proposal part, I have adjusted it according to your suggestion. Thanks again! Boyang Chen 于2020年4月16日周四 下午2:39写道: > Thanks for the KIP Xiang! > > I think the motivation looks good, and I just did a slight structure change

[jira] [Resolved] (KAFKA-9824) Consumer loses partition offset and resets post 2.4.1 version upgrade

2020-04-16 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9824?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-9824. Resolution: Duplicate Closing this as a duplicate of KAFKA-9543. Let's continue discussion

Re: [DISCUSS] KIP-596 Safely abort Producer transactions during application shutdown

2020-04-16 Thread Boyang Chen
Thanks for the KIP Xiang! I think the motivation looks good, and I just did a slight structure change to separate "Proposed Changes" and "Public Interfaces", hope you don't mind. However, "we can determine whether the producer client is already in error state in abortTransaction" sounds a bit

[jira] [Created] (KAFKA-9878) Block EndTxn call until the txn markers are committed

2020-04-16 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-9878: -- Summary: Block EndTxn call until the txn markers are committed Key: KAFKA-9878 URL: https://issues.apache.org/jira/browse/KAFKA-9878 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-9877) ERROR Shutdown broker because all log dirs in /tmp/kafka-logs have failed (kafka.log.LogManager)

2020-04-16 Thread Hawking Du (Jira)
Hawking Du created KAFKA-9877: - Summary: ERROR Shutdown broker because all log dirs in /tmp/kafka-logs have failed (kafka.log.LogManager) Key: KAFKA-9877 URL: https://issues.apache.org/jira/browse/KAFKA-9877