Re: [VOTE] 2.1.0 RC0

2018-11-05 Thread Satish Duggana
Hi Dong, Is there a RC1 planned with configs documentation fixes and https://github.com/apache/kafka/pull/5857 ? Thanks, Satish. On Thu, Nov 1, 2018 at 4:05 PM Jakub Scholz wrote: > > +1 (non-binding) ... I used the staged binaries and checked it with > different clients. > > On Wed, Oct 24,

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

2018-11-05 Thread Boyang Chen
Hey Mike, thanks for the feedback, the two question are very thoughtful! > 1) I am a little confused about the distinction for the leader. If the > consumer node that was assigned leader does a bounce (goes down and quickly > comes up) to update application code, will a rebalance be

Re: [DISCUSS] KIP-351: Add --under-min-isr option to describe topics command

2018-11-05 Thread Kevin Lu
Hi Stanislav, Thanks for the response. I will give it a few more days to let others look at the KIP before putting it to a vote then. Regards, Kevin On Mon, Nov 5, 2018 at 7:32 AM Stanislav Kozlovski wrote: > Hey Kevin, > > Thanks for the KIP! This sounds like a very useful feature to have

[DISCUSS] KIP-388 Add observer interface to record request and response

2018-11-05 Thread Lincong Li
Hi everyone. Here is my KIP. Any feedback is appreciated. Thanks, Lincong Li

[jira] [Created] (KAFKA-7596) Add observer interface to record request and response

2018-11-05 Thread Lincong Li (JIRA)
Lincong Li created KAFKA-7596: - Summary: Add observer interface to record request and response Key: KAFKA-7596 URL: https://issues.apache.org/jira/browse/KAFKA-7596 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-7595) Kafka Streams: KTrable to KTable join introduces duplicates in downstream KTable

2018-11-05 Thread Vik Gamov (JIRA)
Vik Gamov created KAFKA-7595: Summary: Kafka Streams: KTrable to KTable join introduces duplicates in downstream KTable Key: KAFKA-7595 URL: https://issues.apache.org/jira/browse/KAFKA-7595 Project:

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

2018-11-05 Thread Apache Jenkins Server
See Changes: [colin] MINOR: KStreams SuppressionDurabilityIntegrationTest should set -- [...truncated 2.73 MB...] org.apache.kafka.streams.test.OutputVerifierTest >

Re: [kafka-clients] [VOTE] 2.0.1 RC0

2018-11-05 Thread Jun Rao
Hi, Mani, Thanks for running the release. Verified quickstart on 2.12 binary. +1 Jun On Thu, Oct 25, 2018 at 7:28 PM, Manikumar wrote: > Hello Kafka users, developers and client-developers, > > This is the first candidate for release of Apache Kafka 2.0.1. > > This is a bug fix release

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

2018-11-05 Thread Apache Jenkins Server
See Changes: [colin] MINOR: KStreams SuppressionDurabilityIntegrationTest should set -- [...truncated 2.33 MB...] org.apache.kafka.streams.test.OutputVerifierTest >

[DISCUSS] KIP-387: Fair Message Consumption Across Partitions in KafkaConsumer

2018-11-05 Thread ChienHsing Wu
Hi I just put together the KIP page as requested. This email is to start the discussion thread. KIP: KIP-387: Fair Message Consumption Across Partitions in KafkaConsumer

[DISCUSS] KIP-304: Connect runtime mode improvements for container platforms

2018-11-05 Thread Cyrus Vafadari
I see there are no "Rejected Alternatives" in the KIP -- What is the reason you can't run a distributed herder as a single node? Is the goal here to run multiple standalone workers with the same group.id?

[jira] [Created] (KAFKA-7594) Make the time out for connect service to be dynamic

2018-11-05 Thread Magesh kumar Nandakumar (JIRA)
Magesh kumar Nandakumar created KAFKA-7594: -- Summary: Make the time out for connect service to be dynamic Key: KAFKA-7594 URL: https://issues.apache.org/jira/browse/KAFKA-7594 Project: Kafka

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

2018-11-05 Thread Mike Freyberger
Boyang, Thanks for updating the KIP. It's shaping up well. Two things: 1) I am a little confused about the distinction for the leader. If the consumer node that was assigned leader does a bounce (goes down and quickly comes up) to update application code, will a rebalance be triggered? I do

[jira] [Created] (KAFKA-7593) Infinite restart loop when failed to store big config for task

2018-11-05 Thread Oleg Kuznetsov (JIRA)
Oleg Kuznetsov created KAFKA-7593: - Summary: Infinite restart loop when failed to store big config for task Key: KAFKA-7593 URL: https://issues.apache.org/jira/browse/KAFKA-7593 Project: Kafka

Re: [DISCUSS] KIP-379: Multiple Consumer Group Management

2018-11-05 Thread Alex D
Hello guys, Thank you for your suggestions! I've made a short resume of all suggestions proposed for further possible code corrections. Since not all opinions match, let's review once again and decide. #1. Support old csv format. Proposed by Jason. Yes: Jason, Vahid If backwards

Re: [DISCUSS] KIP-351: Add --under-min-isr option to describe topics command

2018-11-05 Thread Stanislav Kozlovski
Hey Kevin, Thanks for the KIP! This sounds like a very useful feature to have and makes excellent use of the new AdminClient introduced in KIP-377 in my opinion. I believe it's perfectly fine to start a vote thread on the fact that KIP-377 was accepted already. On Mon, Nov 5, 2018 at 3:03 PM

Re: [DISCUSS] KIP-351: Add --under-min-isr option to describe topics command

2018-11-05 Thread Kevin Lu
Bumping this to hopefully get a couple more opinions. Since KIP-377 has been accepted (it adds the --bootstrap-server option needed for this KIP), do we think it is okay to proceed to vote on this KIP without KIP-377 fully merged yet? Thanks. Regards, Kevin On Mon, Oct 22, 2018 at 6:32 PM

Re: [DISCUSS] KIP-386: Make Min metrics' default value consistent with Max metrics

2018-11-05 Thread Stanislav Kozlovski
Hey everybody, Just wanted to say that I plan on starting a voting thread tomorrow if there aren't any further comments. Thanks for the discussion so far! On Wed, Oct 24, 2018 at 11:27 AM Stanislav Kozlovski wrote: > Hi Kevin, > > Thanks for providing context. > > I've edited the KIP to use

Re: [VOTE] 2.0.1 RC0

2018-11-05 Thread Mickael Maison
+1 (non-binding) - Checked signatures - Ran through quickstart on kafka_2.12-2.0.1.tgz - Built from source with Java 8 On Mon, Nov 5, 2018 at 9:13 AM Satish Duggana wrote: > > +1 (non-binding) > > - Ran testAll/releaseTarGzAll on 2.0.1 > - Ran through quickstart of core/streams on builds

[jira] [Created] (KAFKA-7592) kafka consumer poll不能关闭,3节点关闭2节点broke和zk

2018-11-05 Thread lvliguo (JIRA)
lvliguo created KAFKA-7592: -- Summary: kafka consumer poll不能关闭,3节点关闭2节点broke和zk Key: KAFKA-7592 URL: https://issues.apache.org/jira/browse/KAFKA-7592 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-7591) Changelog retention period doesn't synchronise with window-store size

2018-11-05 Thread Jon Bates (JIRA)
Jon Bates created KAFKA-7591: Summary: Changelog retention period doesn't synchronise with window-store size Key: KAFKA-7591 URL: https://issues.apache.org/jira/browse/KAFKA-7591 Project: Kafka

Re: [VOTE] 2.0.1 RC0

2018-11-05 Thread Satish Duggana
+1 (non-binding) - Ran testAll/releaseTarGzAll on 2.0.1 - Ran through quickstart of core/streams on builds generated from tag - Ran few internal apps targeting to topics on 3 node cluster. Thanks, Satish. On Sat, Nov 3, 2018 at 2:32 AM Ewen Cheslack-Postava wrote: > > +1 > > -Ewen > > On Thu,