Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-12 Thread Ismael Juma
Hi all, I will send a more detail email later, some quick comments: 1. It's unlikely that defaults will suit everyone. I think the question is: what is the most likely configuration for a typical Kafka user _today_? Kafka's usage is growing well beyond its original use cases and correctness is of

Re: [VOTE] KIP-180: Adding a new metric for brokers specifying the number of consumer group rebalances in progress

2017-08-12 Thread Mickael Maison
+1, thanks for the KIP On Fri, Aug 11, 2017 at 11:59 PM, Apurva Mehta wrote: > +1, > > Thanks, > Apurva > > On Fri, Aug 11, 2017 at 3:02 PM, Bill Bejeck wrote: > >> +1 >> >> Thanks, >> Bill >> >> On Fri, Aug 11, 2017 at 6:00 PM, Colin McCabe wrote: >> >> > Hi all, >> > >> > I think it's a good

Re: [VOTE] KIP-180: Adding a new metric for brokers specifying the number of consumer group rebalances in progress

2017-08-12 Thread Gwen Shapira
+1 Thanks! This would be most useful. On Sat, Aug 12, 2017 at 3:09 AM Mickael Maison wrote: > +1, thanks for the KIP > > On Fri, Aug 11, 2017 at 11:59 PM, Apurva Mehta > wrote: > > +1, > > > > Thanks, > > Apurva > > > > On Fri, Aug 11, 2017 at 3:02 PM, Bill Bejeck wrote: > > > >> +1 > >> > >>

Re: [VOTE] KIP-180: Adding a new metric for brokers specifying the number of consumer group rebalances in progress

2017-08-12 Thread Ismael Juma
Thanks for the KIP, +1 (binding). Seems like the KIP title has to be updated still. Ismael On Fri, Aug 11, 2017 at 11:00 PM, Colin McCabe wrote: > Hi all, > > I think it's a good time to vote on KIP-180. It adds a helpful new > metric that shows consumer group states. > > The full proposal, an

[jira] [Created] (KAFKA-5732) Kafka 0.11 Consumer.Poll() hangs for consumer.subscribe()

2017-08-12 Thread Ramkumar (JIRA)
Ramkumar created KAFKA-5732: --- Summary: Kafka 0.11 Consumer.Poll() hangs for consumer.subscribe() Key: KAFKA-5732 URL: https://issues.apache.org/jira/browse/KAFKA-5732 Project: Kafka Issue Type: Bu

Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-12 Thread Apurva Mehta
I think the question of the default broker level configs is a good one. I don't think we need to touch the min.isr config or the replication factor to satisfy 'exactly-once' going by the definition laid out earlier. On the broker side, I think the only thing we should change is to disable unclean l

Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-12 Thread Ismael Juma
Apurva, unclean leader election is disabled by default since 0.11.0.0. Ismael On 12 Aug 2017 8:06 pm, "Apurva Mehta" wrote: > I think the question of the default broker level configs is a good one. I > don't think we need to touch the min.isr config or the replication factor > to satisfy 'exact

Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-12 Thread Jay Kreps
+1 I think there is some pain with changing any default, but this is well worth it. The reality is that the profile of Kafka usage has changed significantly since replication was added to Kafka and these defaults were chosen. At that time usage was primarily high volume event and log data and the

Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-12 Thread Jay Kreps
Becket, I think this proposal actually does a great deal to address the configuration complexity. It is true that there are a number of knobs, but the result of this change is that 99% of people don't need to think about them (and the mechanism we have to communicate that is to reduce the importan

Re: [DISCUSS] KIP-171: Extend Consumer Group Reset Offset for Stream Application

2017-08-12 Thread Jorge Esteban Quilcate Otoya
I have updated the KIP: - Change execution parameters, using `--dry-run` - Reference KAFKA-4327 - And advise about changes on `StreamResetter` Also includes that it will cover a change on `ConsumerGroupCommand` to align execution options. El dom., 16 jul. 2017 a las 5:37, Matthias J. Sax () escr

Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-12 Thread Becket Qin
Hi Jay and Apurva, Thanks for the reply. I agree that it is a good time to reconsider all the configurations we want. I also would like to ship Kafka with a stronger guarantee if possible. The concerns I have were mainly the following: 1. For users who used to be relying on the default settings,