Re: [VOTE] KIP-557: Add emit on change support for Kafka Streams

2020-03-02 Thread Guozhang Wang
Hello Richard, Thanks for the KIP. I once reviewed it and was concerned about its effects on stream time advancing. After reading the updated KIP I think it has answered a lot of them already. I have a couple minor comments still, otherwise I'm +1: 1) I want to clarify that for operations

Re: 回复:回复:[Vote] KIP-571: Add option to force remove members in StreamsResetter

2020-03-02 Thread Guozhang Wang
Hello Feyman, thanks for the proposal! I read through the doc and overall it looks good to me. One minor thing I'd still like to point out is that, the "removeMembersFromConsumerGroup" only sends a leave-group request to the coordinator to let it remove the member, however, if the member is

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

2020-03-02 Thread Apache Jenkins Server
See Changes: [github] MINOR: Update year in NOTICE (#8207) [github] KAFKA-8995: delete all topics before recreating (#8208) -- [...truncated 2.90 MB...]

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

2020-03-02 Thread Apache Jenkins Server
See Changes: [manikumar] Kafka 9626: Improve ACLAuthorizer.acls() performance -- [...truncated 1.83 MB...] kafka.zookeeper.ZooKeeperClientTest >

Subject: [VOTE] 2.4.1 RC0

2020-03-02 Thread Bill Bejeck
Hello Kafka users, developers and client-developers, This is the first candidate for release of Apache Kafka 2.4.1. This is a bug fix release and it includes fixes and improvements from 38 JIRAs, including a few critical bugs. Release notes for the 2.4.1 release:

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

2020-03-02 Thread Apache Jenkins Server
See Changes: [github] MINOR: Port streams broker compatibility fix (#8203) [mumrah] Fix NOTICE year -- [...truncated 5.86 MB...]

Re: [DISCUSS] KIP-569: DescribeConfigsResponse - Update the schema to include datatype of the field

2020-03-02 Thread Shailesh Panwar
I have updated the KIP to incorporate the feedback received so far. Changes include 1. By default, the new fields would not be included in the response. Client can include them via DescribeConfigsOptions 2. Updated the Compatibility section with backward and forward compatibility behaviour.

Re: [VOTE] KIP-557: Add emit on change support for Kafka Streams

2020-03-02 Thread Richard Yu
Hi all, Thanks for the votes so far! @Matthias or @Guozhang Wang it would be great to also get your input on this KIP. It looks to be pretty close to completion, so the finishing touches are all we need. :) Best, Richard On Mon, Mar 2, 2020 at 11:45 AM Ghassan Yammine <

Re: [VOTE] 2.5.0 RC0

2020-03-02 Thread David Arthur
Just a quick update. We are working through some system test issues and hope to have a passing build today or tomorrow. Once we have a blue build, I'll work on RC1 and send out a new vote thread. In the mean time, please feel free to continue testing RC0 and looking for blocker bugs. Thanks!

[jira] [Resolved] (KAFKA-9626) Benchmark and optimize AclAuthorizer.acls()

2020-03-02 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9626?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-9626. -- Resolution: Fixed Issue resolved by pull request 8199 [https://github.com/apache/kafka/pull/8199] >

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

2020-03-02 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-557: Add emit on change support for Kafka Streams

2020-03-02 Thread Ghassan Yammine
Hello all, +1 (non-binding) Thanks, Ghassan On 3/2/20, 12:43 PM, "Bruno Cadonna" wrote: EXTERNAL: This email originated from outside of Bazaarvoice. Do not click any links or open any attachments unless you trust the sender and know the content is safe. Hi Richard,

Re: [VOTE] KIP-570: Add leader epoch in StopReplicaRequest

2020-03-02 Thread Gwen Shapira
+1 On Mon, Feb 24, 2020, 2:16 AM David Jacot wrote: > Hi all, > > I would like to start a vote on KIP-570: Add leader epoch in > StopReplicaRequest > > The KIP is here: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-570%3A+Add+leader+epoch+in+StopReplicaRequest > > Thanks, > David >

Re: [VOTE] KIP-570: Add leader epoch in StopReplicaRequest

2020-03-02 Thread Jason Gustafson
+1 On Mon, Feb 24, 2020 at 2:16 AM David Jacot wrote: > Hi all, > > I would like to start a vote on KIP-570: Add leader epoch in > StopReplicaRequest > > The KIP is here: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-570%3A+Add+leader+epoch+in+StopReplicaRequest > > Thanks, > David

[VOTE] KIP-573: Enable TLSv1.3 by default

2020-03-02 Thread Nikolay Izhikov
Hello. I would like to start vote for KIP-573: Enable TLSv1.3 by default KIP - https://cwiki.apache.org/confluence/display/KAFKA/KIP-573%3A+Enable+TLSv1.3+by+default Discussion thread -

Re: [VOTE] KIP-557: Add emit on change support for Kafka Streams

2020-03-02 Thread Bruno Cadonna
Hi Richard, +1 (non-binding) Best, Bruno On Mon, Mar 2, 2020 at 4:33 PM John Roesler wrote: > > Hi Richard, > > Thanks for the KIP! > > I'm +1 (binding) > > -john > > On Thu, Feb 27, 2020, at 14:40, Richard Yu wrote: > > Hi all, > > > > I am proposing a new optimization to Kafka Streams which

[jira] [Created] (KAFKA-9637) Create separate state directory to manage locks

2020-03-02 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-9637: -- Summary: Create separate state directory to manage locks Key: KAFKA-9637 URL: https://issues.apache.org/jira/browse/KAFKA-9637 Project: Kafka Issue Type:

Add a customized logo for Kafka Streams

2020-03-02 Thread Boyang Chen
Hey Apache Kafka committers and community folks, over the years Kafka Streams has been widely adopted and tons of blog posts and tech talks have been trying to introduce it to people with need of stream processing. As it is part of Apache Kafka project, there is always an awkward situation where

[jira] [Resolved] (KAFKA-6819) Refactor build-in StreamsMetrics internal implementations

2020-03-02 Thread Guozhang Wang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-6819?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-6819. -- Fix Version/s: 2.5.0 Resolution: Fixed > Refactor build-in StreamsMetrics internal

Re: [VOTE] KIP-518: Allow listing consumer groups per state

2020-03-02 Thread Gwen Shapira
+1 (binding) Gwen Shapira Engineering Manager | Confluent 650.450.2760 | @gwenshap Follow us: Twitter | blog On Mon, Mar 02, 2020 at 8:32 AM, David Jacot < dja...@confluent.io > wrote: > > > > +1 (non-binding). Thanks for the KIP! > > > > David > > > > On Thu, Feb 6, 2020 at 10:45 PM

[jira] [Resolved] (KAFKA-8345) Create an Administrative API for Replica Reassignment

2020-03-02 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8345?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-8345. - Fix Version/s: 2.4.0 Resolution: Fixed > Create an Administrative API for Replica

Re: [VOTE] KIP-518: Allow listing consumer groups per state

2020-03-02 Thread David Jacot
+1 (non-binding). Thanks for the KIP! David On Thu, Feb 6, 2020 at 10:45 PM Colin McCabe wrote: > Hi Mickael, > > Thanks for the KIP. I left a comment on the DISCUSS thread as well. > > best, > Colin > > > On Thu, Feb 6, 2020, at 08:58, Mickael Maison wrote: > > Hi Manikumar, > > > > I

Re: [DISCUSS] KIP-518: Allow listing consumer groups per state

2020-03-02 Thread David Jacot
Hi Mickael, My apologies for the delay. Overall, the KIP looks good to me. One small suggestion is to update the docstring of the States field in the request to clearly state that all groups are returned if omitted or empty. People rely on the JSON description so it would be good to provide the

[jira] [Created] (KAFKA-9636) Simple join of two KTables fails

2020-03-02 Thread Paul Snively (Jira)
Paul Snively created KAFKA-9636: --- Summary: Simple join of two KTables fails Key: KAFKA-9636 URL: https://issues.apache.org/jira/browse/KAFKA-9636 Project: Kafka Issue Type: Bug

[jira] [Resolved] (KAFKA-9619) Receiving duplicates when application is configured for exactly once

2020-03-02 Thread Cristian Manoliu (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9619?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Cristian Manoliu resolved KAFKA-9619. - Resolution: Invalid > Receiving duplicates when application is configured for exactly

[jira] [Created] (KAFKA-9635) Should ConfigProvider.subscribe be decrecated?

2020-03-02 Thread Tom Bentley (Jira)
Tom Bentley created KAFKA-9635: -- Summary: Should ConfigProvider.subscribe be decrecated? Key: KAFKA-9635 URL: https://issues.apache.org/jira/browse/KAFKA-9635 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-9634) ConfigProvider does not document thread safety

2020-03-02 Thread Tom Bentley (Jira)
Tom Bentley created KAFKA-9634: -- Summary: ConfigProvider does not document thread safety Key: KAFKA-9634 URL: https://issues.apache.org/jira/browse/KAFKA-9634 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-9633) ConfigProvider.close() not called

2020-03-02 Thread Tom Bentley (Jira)
Tom Bentley created KAFKA-9633: -- Summary: ConfigProvider.close() not called Key: KAFKA-9633 URL: https://issues.apache.org/jira/browse/KAFKA-9633 Project: Kafka Issue Type: Bug

[jira] [Resolved] (KAFKA-8904) Reduce metadata lookups when producing to a large number of topics

2020-03-02 Thread Brian Byrne (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8904?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brian Byrne resolved KAFKA-8904. Fix Version/s: 2.5.0 Reviewer: Rajini Sivaram Resolution: Fixed > Reduce metadata

Re: [VOTE] KIP-557: Add emit on change support for Kafka Streams

2020-03-02 Thread John Roesler
Hi Richard, Thanks for the KIP! I'm +1 (binding) -john On Thu, Feb 27, 2020, at 14:40, Richard Yu wrote: > Hi all, > > I am proposing a new optimization to Kafka Streams which would greatly > reduce the number of idempotent updates (or no-ops) in the Kafka Streams > DAG. > A number of users

Discussion on new feature to Kafka Streams API

2020-03-02 Thread Muhammad Sufyian
Overload the functionality of Aggregate in KafkaStreams API such that , the lookup is allowed on a particular record of the value , in pair . Also not only aggregate based on the last aggregate + present record , but based on all the historical data , which means multiple values given a single

[jira] [Created] (KAFKA-9632) Transient test failure: PartitionLockTest.testAppendReplicaFetchWithUpdateIsr

2020-03-02 Thread Rajini Sivaram (Jira)
Rajini Sivaram created KAFKA-9632: - Summary: Transient test failure: PartitionLockTest.testAppendReplicaFetchWithUpdateIsr Key: KAFKA-9632 URL: https://issues.apache.org/jira/browse/KAFKA-9632

回复:回复:回复:[Vote] KIP-571: Add option to force remove members in StreamsResetter

2020-03-02 Thread feyman2009
Hi, John Sorry, I have mistaken the KIP approval standard, anyway, I will start the PR soon and waiting for more binding approvals. Thanks! Feyman -- 发件人:John Roesler 发送时间:2020年3月2日(星期一) 22:00 收件人:dev 主 题:Re: 回复:回复:[Vote]

Re: 回复:回复:[Vote] KIP-571: Add option to force remove members in StreamsResetter

2020-03-02 Thread John Roesler
Hi Feyman, Sorry, but we actually need 3 binding votes for the KIP to pass. Please feel free to keep bumping the thread until some more committers can take a look. By the way, you can totally start a PR, but we can’t merge it until the KIP passes the vote. Thanks! John On Mon, Mar 2, 2020,

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

2020-03-02 Thread Apache Jenkins Server
See Changes: [manikumar] KAFKA-9327: Document GroupMetadata metrics -- [...truncated 5.86 MB...] org.apache.kafka.streams.internals.WindowStoreFacadeTest >