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

2017-08-31 Thread Roger Hoover
Makes sense in terms of priorities. Thanks, Apurva. On Thu, Aug 31, 2017 at 11:15 AM, Apurva Mehta wrote: > Thanks for the message, Roger. > > I think having 'acks=all' imply 'acks=minIsr' will probably result in some > improvement in the latency. However, I would note two

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

2017-08-31 Thread Apurva Mehta
Thanks for the message, Roger. I think having 'acks=all' imply 'acks=minIsr' will probably result in some improvement in the latency. However, I would note two things: 1. The numbers on the wiki are latency at max throughput, which should not be representative of actual latency degradation. We

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

2017-08-31 Thread Roger Hoover
Sorry, my math was sloppy. It's not twice as many requests taking longer. If the probability of replication latency longer than X is Px for both replicas then, acks=all will have probability of Px(2-Px) of replication lag longer than X while acks=minIsr will be Px On Wed, Aug 30, 2017 at 5:18

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

2017-08-30 Thread Roger Hoover
Sorry if this is a bit out of left field but can't help wondering... One way to improve producer performance while still having good guarantees would be to allow a setting between acks=1 and acks=all. We could introduce "acks=minIsr". This is already the guarantee you get when the ISR set

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

2017-08-30 Thread Apurva Mehta
Hi Ted, int16 is sufficient. I forgot to specify initially. I have updated the KIP. Thanks for pointing it out! Apurva On Wed, Aug 30, 2017 at 4:43 PM, Ted Yu wrote: > For ProduceRequest v4, would int32 or int16 be enough for idempotenceLevel > ? > > Cheers > > On Wed, Aug

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

2017-08-30 Thread Ted Yu
For ProduceRequest v4, would int32 or int16 be enough for idempotenceLevel ? Cheers On Wed, Aug 30, 2017 at 3:47 PM, Apurva Mehta wrote: > Thanks Ismael and Jason, I filed a separate KIP to solve the problems > identified through this discussion. I also incorporated

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

2017-08-30 Thread Apurva Mehta
Thanks Ismael and Jason, I filed a separate KIP to solve the problems identified through this discussion. I also incorporated Jason's comments in that document: https://cwiki.apache.org/confluence/display/KAFKA/KIP-192+%3A+Provide+cleaner+semantics+when+idempotence+is+enabled Please have a look,

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

2017-08-29 Thread Ismael Juma
Thanks for the proposals. I think they make sense and I also agree with Jason's suggestions. Also, it would be good to include the updated ProduceRequest/Response schema in the KIP. Ismael On Tue, Aug 22, 2017 at 11:42 PM, Jason Gustafson wrote: > Thanks Apurva, > > On

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

2017-08-22 Thread Jason Gustafson
Thanks Apurva, On compatibility: I think the proposal makes sense. It's a pity that we can't support idempotence for 0.11.0.0 brokers in the "safe" mode even if it is supported by the broker. I can already imagine users complaining about this, but I guess it's the consequence of missing the

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

2017-08-18 Thread Apurva Mehta
Thanks Jason and Ismael. The message format problem is an acute one: if we enable idempotence by default, the UnsupportedVersionException when writing to topics with the older message format would mean that our prescribed upgrade steps would not work. I have detailed the problems and the

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

2017-08-18 Thread Jason Gustafson
> > so this change will break client backward compatibility while connecting > to 0.10.X brokers. > users need to change producer default settings while connecting older > brokers. At the moment, I think the answer is yes. The old broker will not support the InitProducerId request, so the

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

2017-08-18 Thread Jason Gustafson
> > 1. We should only delay it if there's a concrete benefit (e.g. we agree > that we need to improve OutOfOrderSequence and we can't do it in time). I > doubt that we will get much additional testing from users if we keep it off > by default for another release cycle (i.e. 4 months). Not sure

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

2017-08-18 Thread Manikumar
> > 3. The message format requirement is a good point. This should be mentioned > in the compatibility section. Users who are still using the old message > format will get an error after the upgrade, right? > so this change will break client backward compatibility while connecting to 0.10.X

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

2017-08-18 Thread Ismael Juma
Hi Apurva and Jason, A few thoughts: 1. We should only delay it if there's a concrete benefit (e.g. we agree that we need to improve OutOfOrderSequence and we can't do it in time). I doubt that we will get much additional testing from users if we keep it off by default for another release cycle

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

2017-08-18 Thread Manikumar
I agree with Jason/Apruva comments. we can wait some more time and enable it in 1.1.0 release. In general, we are in agreement with the change. On Fri, Aug 18, 2017 at 10:32 AM, Apurva Mehta wrote: > Hi Jason, > > You make some good points. > > First, I agree that we could

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

2017-08-17 Thread Apurva Mehta
Hi Jason, You make some good points. First, I agree that we could have false positives which result in an OutOfOrderSequence exception for topics with low retention where messages are removed intentionally. I 100% agree that this should be tightened up. One solution is to return the log start

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

2017-08-17 Thread Jason Gustafson
Some additional points for discussion: 1. The idempotent producer is still a new feature and I doubt it has gotten much use yet (in particular since it depends on a message format upgrade). Do we feel it has reached a level of stability where we are comfortable making it the default? 2. I'm

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

2017-08-16 Thread Apurva Mehta
Thanks for the followup Becket. It sounds we are on agreement on the scope of this KIP, and the discussion has definitely clarified a lot of the subtle points. Apurva On Tue, Aug 15, 2017 at 10:49 PM, Becket Qin wrote: > Hi Apurva, > > Thanks for the clarification of the

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

2017-08-15 Thread Becket Qin
Hi Apurva, Thanks for the clarification of the definition. The definitions are clear and helpful. It seems the scope of this KIP is just about the producer side configuration change, but not attempting to achieve the exactly once semantic with all default settings out of the box. The broker

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

2017-08-15 Thread Guozhang Wang
Hi Jay, I chatted with Apurva offline, and we think the key of the discussion is that, as summarized in the updated KIP wiki, whether we should consider replication as a necessary condition of at-least-once, and of course also exactly-once. Originally I think replication is not a necessary

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

2017-08-15 Thread Jay Kreps
Hey Guozhang, I think the argument is that with acks=1 the message could be lost and hence you aren't guaranteeing exactly once delivery. -Jay On Mon, Aug 14, 2017 at 1:36 PM, Guozhang Wang wrote: > Just want to clarify that regarding 1), I'm fine with changing it to `all`

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

2017-08-14 Thread Apurva Mehta
Hi Becket, Thanks for your comments. I added a 'Background' section to the KIP which defines the terms we are using. It also has a note on replication and delivery guarantees. Please have a look:

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

2017-08-14 Thread Becket Qin
Hi Apurva, Regarding acks. I think acks=all is not an isolated independent configuration for durability. The reason we want to enable acks=all is to tolerate broker failures. But If that is the case, not setting min.isr to >=2 seems defeating that purpose. If we set min.isr=2, setting replication

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

2017-08-14 Thread Guozhang Wang
Just want to clarify that regarding 1), I'm fine with changing it to `all` but just wanted to argue it is not necessarily correlate with the exactly-once semantics, but rather on persistence v.s. availability trade-offs, so I'd like to discuss them separately. Regarding 2), one minor concern I

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

2017-08-14 Thread Apurva Mehta
Hello, I just want to summarize where we are in this discussion There are two major points of contention: should we have acks=1 or acsk=all by default? and how to cap max.in.flight.requests.per.connection? 1) acks=1 vs acks=all1 Here are the tradeoffs of each: If you have

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

2017-08-13 Thread Guozhang Wang
Hi all, Here are my two cents: 1. I'm inclined to agree with Ewen that "acks" config is semi-orthogonal to the exactly-once semantics. For example, if some topics only have replication factor 1, then `acks = all` does not improve on any guarantees. More generally speaking, setting idempotency +

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

2017-08-13 Thread Becket Qin
A, never mind, my last calculation actually forget to take the number of partitions into account. So it does seem a problem if we keep info of last N appended batches on the broker. On Sat, Aug 12, 2017 at 9:50 PM, Becket Qin wrote: > Hi Jay and Apurva, > > Thanks for the

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

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

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

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

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

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

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

2017-08-11 Thread Becket Qin
BTW, I feel that the configurations we have around those guarantees have become too complicated for the users. Not sure if this is considered before but Maybe we can have some helper functions provided to the users. For example: Properties TopicConfig.forSemantc(Semantic semantic); Properties

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

2017-08-11 Thread Becket Qin
Hi Apurva, Thanks for the reply. When I was thinking of exactly once I am thinking of "exactly once with availability", Users probably wouldn't want to sacrifice availability for exactly once. To achieve exactly once with same availability and acks=all, users actually need to pay more cost. To

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

2017-08-11 Thread Apurva Mehta
Thanks for your email Becket. I would be interested in hearing others opinions on which should be a better default between acks=1 and acks=all. One important point on which I disagree is your statement that 'users need to do a lot of work to get exactly-once with acks=all'. This is debatable. If

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

2017-08-11 Thread Becket Qin
Hi Apurva, I agree that most changes we are talking about here are for default values of the configurations and users can always override them. So I think the question to ask is more about the out of the box experience. If the change makes strict improvement compared with the current settings,

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

2017-08-10 Thread Apurva Mehta
Hi Dong, Thanks for your comments. Yes, with retries=MAX_INT, producer.flush() may block. I think there are two solutions: a good one would be to adopt some form of KIP-91 to bound the time a message can remain unacknowledged. Alternately, we could set the default retries to 10 or something. I

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

2017-08-10 Thread Dong Lin
Hey Apurva, Thanks for the KIP. I have read through the KIP and the prior discussion in this thread. I have three concerns that are related to Becket's comments: - Is it true that, as Becket has mentioned, producer.flush() may block infinitely if retries=MAX_INT? This seems like a possible

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

2017-08-09 Thread Apurva Mehta
Thanks for your email Becket. Your observations around using acks=1 and acks=-1 are correct. Do note that getting an OutOfOrderSequence means that acknowledged data has been lost. This could be due to a weaker acks setting like acks=1 or due to a topic which is not configured to handle broker

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

2017-08-09 Thread Becket Qin
Thanks for the KIP, Apurva. It is a good time to review the configurations to see if we can improve the user experience. We also might need to think from users standpoint about the out of the box experience. 01. Generally speaking, I think it makes sense to make idempotence=true so we can enable

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

2017-08-09 Thread Apurva Mehta
Thanks for the comments Ismael and Jason. Regarding the OutOfOrderSequenceException, it is more likely when you enable idempotence and have acks=1, simply because you have a greater probability of losing acknowledged data with acks=1, and the error code indicates that. The particular scenario is

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

2017-08-09 Thread Apurva Mehta
Thanks for the comments, Ewen. Responses inline. > 1. Re: the mention of exactly once, this is within a producer session, > right? And so really only idempotent. Applications still need to take extra > steps for exactly once if they, e.g., are producing data from some other > log like a DB txn

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

2017-08-09 Thread Ismael Juma
Hi Jason, Thanks for the correction. See inline. On Wed, Aug 9, 2017 at 5:13 PM, Jason Gustafson wrote: > Minor correction: the OutOfOrderSequenceException is not fatal for the > idempotent producer and it is not necessarily tied to the acks setting > (though it is more

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

2017-08-09 Thread Jason Gustafson
Minor correction: the OutOfOrderSequenceException is not fatal for the idempotent producer and it is not necessarily tied to the acks setting (though it is more likely to be thrown with acks=1). It is used to signal the user that there was a gap in the delivery of messages. You can hit this if

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

2017-08-09 Thread Ismael Juma
Thanks for the KIP, Apurva. In general, I think it's a good idea to strengthen the guarantees we provide by default. And people who are willing to trade correctness for performance can then change the configs to suit them. I will comment on the KIP specifics in more detail later, but one

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

2017-08-09 Thread Ewen Cheslack-Postava
Apurva, For the benchmarking, I have a couple of questions: 1. Re: the mention of exactly once, this is within a producer session, right? And so really only idempotent. Applications still need to take extra steps for exactly once if they, e.g., are producing data from some other log like a DB