Re: [DISCUSS] KIP-207: Offsets returned by ListOffsetsResponse should be monotonically increasing even during a partition leader change

2017-10-06 Thread Apurva Mehta
Thanks for the KIP Colin. That looks like a reasonable proposal. On Thu, Oct 5, 2017 at 11:23 AM, Colin McCabe wrote: > Hi all, > > I created a KIP for discussion about fixing a corner case in > ListOffsetsResponse. Check it out at: > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 207

Re: [VOTE] KIP-207:The Offsets which ListOffsetsResponse returns should monotonically increase even during a partition leader change

2017-10-17 Thread Apurva Mehta
+1 (non-binding) On Tue, Oct 17, 2017 at 11:11 AM, Colin McCabe wrote: > Hi all, > > I'd like to start the voting process for KIP-207:The Offsets which > ListOffsetsResponse returns should monotonically increase even during a > partition leader change. > > See > https://cwiki.apache.org/conflue

Re: [DISCUSS] Kafka 2.0.0 in June 2018

2017-11-09 Thread Apurva Mehta
I think this is a good idea and your proposed changes look good. I also think that this might be a good time to adopt KIP-185 ( https://cwiki.apache.org/confluence/display/KAFKA/KIP-185%3A+Make+exactly+once+in+order+delivery+per+partition+the+default+producer+setting), and make the idempotent prod

Re: [VOTE] 0.11.0.0 RC1

2017-06-21 Thread Apurva Mehta
Hi Tom, I actually made modifications to the produce performance tool to do real transactions earlier this week as part of our benchmarking (results published here: bit.ly/kafka-eos-perf). I just submitted that patch here: https://github.com/apache/kafka/pull/3400/files I think my version is more

Re: Consumer throughput drop

2017-07-20 Thread Apurva Mehta
Hi Ovidu, The see-saw behavior is inevitable with linux when you have concurrent reads and writes. However, tuning the following two settings may help achieve more stable performance (from Jay's link): > *dirty_ratio*Defines a percentage value. Writeout of dirty data begins > (via *pdflush*) whe

Re: [DISCUSS] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-08-04 Thread Apurva Mehta
If we are going to have a separate configuration for expiry, I prefer my proposal of max.message.delivery.wait.ms and its semantics. However, one thing which has not come out of the JIRA discussion is the actual use cases for batch expiry. Also, the KIP document states the following: *The per mes

Re: [DISCUSS] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-08-07 Thread Apurva Mehta
Responses inline: On Mon, Aug 7, 2017 at 9:37 AM, Sumant Tambe wrote: > > > > > However, one thing which has not come out of the JIRA discussion is the > > actual use cases for batch expiry. > > There are two usecases I can think of for batch expiry mechanism > irrespective of how we try to boun

Re: [DISCUSS] KIP-180: Add a broker metric specifying the number of consumer group rebalances in progress

2017-08-07 Thread Apurva Mehta
Hi Colin, The KIP looks good to me. In your latest proposal, the change of state would be captured as followed in the metrics for groups using Kafka for membership management: PreparingRebalance -> CompletingRebalance -> Stable -> Dead? If a group is just being used to store offsets, then it is

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

2017-08-08 Thread Apurva Mehta
Hi, I've put together a new KIP which proposes to ship Kafka with its strongest delivery guarantees by default. We currently ship with at most once semantics and don't provide any ordering guarantees per partition. The proposal is is to provide exactly once in order delivery per partition by defa

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

2017-08-09 Thread Apurva Mehta
default behavior. As mentioned above, this means that each acknowledged send should result in exactly one copy of the message in the log. With acks=1, we can only ever have at-most once delivery, ie. an acknowledged send could result in 0 copies of the message in the log. Please let me know if I

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-186: Increase offsets retention default to 7 days

2017-08-09 Thread Apurva Mehta
Thanks for the KIP. +1 from me. On Tue, Aug 8, 2017 at 5:24 PM, Ewen Cheslack-Postava wrote: > Hi all, > > I posted a simple new KIP for a problem we see with a lot of users: > KIP-186: Increase offsets retention default to 7 days > > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 186%

Re: [DISCUSS] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-08-09 Thread Apurva Mehta
> > There seems to be no relationship with cluster metadata availability or > > staleness. Expiry is just based on the time since the batch has been > ready. > > Please correct me if I am wrong. > > > > I was not very specific about where we do expiration. I glossed over some > details because (aga

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

2017-08-09 Thread Apurva Mehta
t.requests. It seems > that on some long RTT link, sending more requests in the pipeline would be > the only way to keep the latency to be close to RTT. > > Thanks, > > Jiangjie (Becket) Qin > > > On Wed, Aug 9, 2017 at 11:28 AM, Apurva Mehta wrote: > > > Thanks

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

2017-08-10 Thread Apurva Mehta
e unless there is strong benefit of doing so (e.g. exactly-once). > > Thanks, > Dong > > > > > On Wed, Aug 9, 2017 at 10:43 PM, Apurva Mehta wrote: > > > Thanks for your email Becket. > > > > Your observations around using acks=1 and acks=-1 are correct

Re: Using Java Kafka Producer API for nearly Real Time Applications

2017-08-11 Thread Apurva Mehta
What precise use case do you have in mind? If you don't have cluster metadata, you can't send the requests anyway. If you want to bound your memory and run out of it, that means that you are not able to send data for some reason. The best you can do in both cases is to drop old messages from the p

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

2017-08-11 Thread Apurva Mehta
end, we need to have max.in.flight.requests ~ 20 in order to fully utilize > the network bandwidth. When the requests are smaller, we will need to > pipeline more requests. > > Thanks, > > Jiangjie (Becket) Qin > > > > > On Thu, Aug 10, 2017 at 10:43 PM, Apurva Meht

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

2017-08-11 Thread Apurva Mehta
+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 time to vote on KIP-180. It adds a helpful new > > metric that shows consumer group states. > > > >

Re: Using Java Kafka Producer API for nearly Real Time Applications

2017-08-11 Thread Apurva Mehta
able memory in buffer then report > to me immidiately that there is not enough memory. > > > 2017-08-11 21:04 GMT+03:00 Apurva Mehta : > > > What precise use case do you have in mind? If you don't have cluster > > metadata, you can't send the requests anyway.

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-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 replication-factor=N,

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

2017-08-14 Thread Apurva Mehta
t; likely break the broker code. If we do enforce this we'd better enforce > it > > at the broker side. Also, I'm wondering if we have considered the > approach > > for brokers to read the logs in order to get the starting offset when it > > does not about it in it

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

2017-08-16 Thread Apurva Mehta
e broker but relying at the client > side's > > > > sanity. So other implementations of the client that may not obey this > > may > > > > likely break the broker code. If we do enforce this we'd better > enforce > > > it > > > >

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

2017-08-17 Thread Apurva Mehta
ght requests. Is that correct? > > Overall I'm definitely supportive of making idempotence the default > eventually, but I think it might be a tad premature now. > > Thanks, > Jason > > On Wed, Aug 16, 2017 at 8:58 PM, Apurva Mehta wrote: > > > Thanks f

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 solutions

Re: [DISCUSS] KIP-190 Asynchronous Java Kafka Producer

2017-08-21 Thread Apurva Mehta
Hi Pavel, Thanks for the KIP. You need to be added to the contributors list before you can modify the Kafka wiki space. One of the committers will add you and let you know when it has happened. After that, you can create the KIP in the right space. Thanks, Apurva On Fri, Aug 18, 2017 at 12:55 PM

Re: [DISCUSS] KIP-190 Asynchronous Java Kafka Producer

2017-08-21 Thread Apurva Mehta
Hi Pavel, I believe that you have been added as a contributor on the Wiki. You can move your KIP to the correct space now. Note that you will need to reassign the KIP number since 190 is already taken. Thanks, Apurva On Mon, Aug 21, 2017 at 10:52 AM, Apurva Mehta wrote: > Hi Pavel, > &g

[DISCUSS] KIP-192 - Provide cleaner semantics when idempotence is enabled

2017-08-29 Thread Apurva Mehta
Hi, In the discussion of KIP-185 (enable idempotence by default), we discovered some shortcomings of the existing idempotent producer implementation. Fixing these issues requires changes to the ProduceRequest and ProduceResponse protocols as well as changes to the values of the 'enable.idempotence

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

2017-08-30 Thread Apurva Mehta
t; to > > include that here so that the OutOfOrderSequence error is unambiguous. > > > > Finally, do you plan to roll these proposals into the current KIP or do > > them separately? Probably makes sense to combine them since they both > > require a bump to the ProduceRequest. >

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

2017-08-30 Thread Apurva Mehta
0, 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 Jason's comments > in > > that document: > > https://cwiki.apache.org/confluence/display/

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

2017-08-31 Thread Apurva Mehta
ime of each request will be min(lag of 2 > > followers) > > > > Whatever your latency distribution is for replication, for any given > > remote time (say 100 ms), twice as many requests take longer than that > time > > with acks=all vs acks=minIsr. > > > >

Re: [DISCUSS] KIP-192 - Provide cleaner semantics when idempotence is enabled

2017-09-06 Thread Apurva Mehta
ernatively, maybe we really should be returning the message > format version of each topic in the TopicMetadata response. A nice bonus of > doing so is that it gives the producer the ability to craft the right > format version ahead of time and avoid the need for conversion on the > brok

[VOTE] KIP-192 : Provide cleaner semantics when idempotence is enabled

2017-09-07 Thread Apurva Mehta
Hi, I'd like to start a vote for KIP-192: https://cwiki.apache.org/confluence/display/KAFKA/KIP-192+%3A+Provide+cleaner+semantics+when+idempotence+is+enabled Thanks, Apurva

Re: [DISCUSS] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-07 Thread Apurva Mehta
I agree with what Ismael said. Having both retries and delivery.timeout.ms is confusing, and thus the goal is to not have a retries option at all once idempotence is fully battle tested and has become the entrenched default. Until that time, it makes sense to expire batch earlier than delivery.tim

Re: [VOTE] KIP-192 : Provide cleaner semantics when idempotence is enabled

2017-09-07 Thread Apurva Mehta
On Thu, Sep 7, 2017 at 2:25 PM, Jason Gustafson > wrote: > > > +1. Thanks for the KIP. One nit: we could use int8 to represent the > message > > format version. That is how it is represented in the messages themselves. > > > > -Jason > > > > On Th

Re: [VOTE] KIP-192 : Provide cleaner semantics when idempotence is enabled

2017-09-07 Thread Apurva Mehta
easons are: 1. The implementation > effort to add it is minimal since it's a topic config like message format > version, 2. It's clearly beneficial for the producer to have that > information, 3. It's compact (just a number), 4. It's nice to avoid another > protocol bum

Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-07 Thread Apurva Mehta
Thanks for the KIP Sumant, +1 from me. That is the most exhaustive 'Rejected Alternatives' section that I have seen :) One minor point: In the compatibility section, your note on 'max.in.flight.requests.per.connection == 5' resulting in out of order delivery is true irrespective of these changes.

Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-07 Thread Apurva Mehta
lue of > max.in.flight.request.per.connection==5, reordering becomes a possibility > by default. To prevent reordering, set > max.in.flight.request.per.connection==1. > > It does not hurt to mention it as it's a default behavior change? > > On 7 September 2017 at 22:15, A

Re: [VOTE] KIP-192 : Provide cleaner semantics when idempotence is enabled

2017-09-11 Thread Apurva Mehta
nest way forward and I would like to make this small change to the KIP. Does anybody have any objections? Thanks, Apurva On Thu, Sep 7, 2017 at 9:44 PM, Apurva Mehta wrote: > Thanks for the comments Ismael. > > I have gone ahead and incorporated all your suggestions in the KIP &g

Re: [VOTE] KIP-192 : Provide cleaner semantics when idempotence is enabled

2017-09-11 Thread Apurva Mehta
entries are the entries that hasn't been confirmed by the > producer. > 2. no magic number of 5 max.in.flight.requests.per.connection > 3. bounded memory footprint on the cached sequence/timestamp/offset > entries. > > Hope it's not too late to have the changes if that make

Re: [VOTE] KIP-192 : Provide cleaner semantics when idempotence is enabled

2017-09-11 Thread Apurva Mehta
5 entries (142 > bytes) in cache. So it is probably still OK and it is more urgent to fix > the upgrade path. > > Thanks, > > Jiangjie (Becket) Qin > > > > > > On Mon, Sep 11, 2017 at 4:13 PM, Apurva Mehta wrote: > > > Hi Becket, > > > > Rega

Re: [VOTE] KIP-192 : Provide cleaner semantics when idempotence is enabled

2017-09-13 Thread Apurva Mehta
The KIP has passed with three binding +1 votes (Guozhang, Ismael, Jason) and no -1 or +0 votes. Thanks to everyone for the feedback. Apurva On Mon, Sep 11, 2017 at 8:31 PM, Apurva Mehta wrote: > Hi Becket, > > You are right: the calculations are per partition produced to by each >

Re: system test builder

2017-09-15 Thread Apurva Mehta
Hi Ted, Unfortunately the jenkins.confluent.io address is no longer publicly accessible. Thanks, Apurva On Thu, Sep 14, 2017 at 7:35 PM, Ted Yu wrote: > Hi, > When I put the following in the address bar of Chrome: > > https://jenkins.confluent.io/job/system-test-kafka-branch-builder > > I was

Re: Resources about Apache Kafka

2016-11-20 Thread Apurva Mehta
Not sure if this answers your question, but the entry point for handling the FetchRequest is here: https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/KafkaApis.scala#L436 That file has the entry points for the handlers of other requests as well. Thanks, Apurva On Sun, No

Re: How to commit bug fix to kafka mesos framework

2016-11-28 Thread Apurva Mehta
If the bug is in Kafka, here is the process for submitting a fix: http://kafka.apache.org/contributing If the bug is in the mesos-kafka framework, I think you should look up that project and find out how to commit a bugfix there. I think it should not be more complicated than submitting a PR. Tha

Re: [DISCUSS] KIP-96 - Add per partition metrics for in-sync and assigned replica count

2016-11-30 Thread Apurva Mehta
+1 (non-binding) On Wed, Nov 30, 2016 at 1:20 PM, Xavier Léauté wrote: > FYI, Based on internal feedback I renamed AssignedReplicasCount to simply > be called ReplicasCount. > > On Tue, Nov 29, 2016 at 7:56 PM Neha Narkhede wrote: > > > This seems useful, +1 > > > > On Tue, Nov 29, 2016 at 5:39

Re: [VOTE] KIP-96 - Add per partition metrics for in-sync and replica count

2016-11-30 Thread Apurva Mehta
+1 (non-binding) On Wed, Nov 30, 2016 at 2:00 PM, Jason Gustafson wrote: > +1. Thanks for the KIP! > > On Wed, Nov 30, 2016 at 1:47 PM, Gwen Shapira wrote: > > > +1 (binding) > > > > On Wed, Nov 30, 2016 at 1:34 PM, Xavier Léauté > > wrote: > > > Based on the feedback KIP-96 seems pretty uncon

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-11-30 Thread Apurva Mehta
Thanks for your comments, let me deal with your second point regarding merging the __consumer-offsets and transactions topic. Needless to say, we considered doing this, but chose to keep them separate for the following reasons: 1. Your assumption that group.id and transaction.app.id can be the

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-11-30 Thread Apurva Mehta
. > > What about if the app-id/pid pair already exists and we increment the > epoch? Should we append (app-id, pid, epoch++) to the transaction log? I > think we should, but step 2 doesn't mention this. > > On Wed, Nov 30, 2016 at 5:35 PM, Apurva Mehta wrote: > > &

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-11-30 Thread Apurva Mehta
gt; Just wanted to make it explicit because: > 1. The "append an AppID message..." chunk was ambiguous on whether it > applied to the "if exists" or "if not exists" condition > 2. I think the google doc is pretty explicit on appending to the log > everywhere el

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-01 Thread Apurva Mehta
Hi Ismael, That is a good suggestion. We did not plan to move the design to a wiki, but I think it is valuable to move at least the message format and RPC changes to the wiki. We shall do so once the design is close to final so that we do not have to edit multiple places as we iterate. Thanks, Ap

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-01 Thread Apurva Mehta
on anyway. So the result is a bit more work for the user, though > of > > course it would be transparent to for Kafka Streams users. > > > > One final note. I've described above how to get the strongest guarantees > > that this work is capable of providing in an auto-sca

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-01 Thread Apurva Mehta
Hi Jay, Thanks for your comments. Answers to some of your points are below: 2. There have been long debates about the necessity of the initTransactions method. Let's consider the options for doing without the initTransactions method: - If we do it on demand, we have to consider that the InitP

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-06 Thread Apurva Mehta
Hi Ben, Those are both great questions. I will tackle the second one now, and address the first one a bit later. AppIds are prerequisite for using transactions, and must be consistent across across application sessions. They are the mechanism by which transaction recovery can occur across session

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-06 Thread Apurva Mehta
Hi Ben, Now, on to your first question of how deal with consumer rebalances. The short answer is that the application needs to ensure that the the assignment of input partitions to appId is consistent across rebalances. For Kafka streams, they already ensure that the mapping of input partitions t

Re: [VOTE] KIP-88: OffsetFetch Protocol Update

2016-12-07 Thread Apurva Mehta
+1 (non-binding) On Wed, Dec 7, 2016 at 10:05 AM, Jason Gustafson wrote: > +1 Thanks for the KIP! > > On Wed, Dec 7, 2016 at 2:53 AM, Ismael Juma wrote: > > > Thanks for the KIP, Vahid. +1 (binding) > > > > On Mon, Dec 5, 2016 at 6:16 PM, Vahid S Hashemian < > > vahidhashem...@us.ibm.com > > >

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-12 Thread Apurva Mehta
@Becket and @Rajini, Thanks for those comments. You raise some very astute points. I will address a subset of them here. One common thread across your emails has to do with the notion of a 'batch' of messages from the consumer's point of view. In particular, Rajini's points 12 and 16, and Becket'

Re: kafka_2.11-0.9.0.1 crash with java coredump

2016-12-14 Thread Apurva Mehta
I would suggest creating a JIRA and describing in detail what was going on in the cluster when this happened, and posting the associated broker / state change / controller logs. Thanks, Apurva On Wed, Dec 14, 2016 at 3:28 AM, Mazhar Shaikh wrote: > Hi All, > > I am using kafka_2.11-0.9.0.1 with

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-14 Thread Apurva Mehta
t; > ...which suggests that the distinction is useful for performance, but not > necessary for correctness, which makes good sense to me. (Would a 128-bid > ID be a reasonable compromise? That's enough room for a UUID, or a > reasonable hash of an arbitrary string, and has only a ma

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-14 Thread Apurva Mehta
with a > fixed > > > size ID. > > > > ...which suggests that the distinction is useful for performance, but not > > necessary for correctness, which makes good sense to me. (Would a 128-bid > > ID be a reasonable compromise? That's enough room for a UUID, o

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-16 Thread Apurva Mehta
buffering, generally > > cleaner overall? > > > > there would need to be adjustments made to watermark and follower fetch > > logic but some of us here have discussed this over lunch and we think its > > doable. > > > > > > On Thu, Dec 15, 2016 at

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-20 Thread Apurva Mehta
; > > > > > > > significantly improve consumer side efficiency in normal > > cases. > > > > It > > > > > > > again > > > > > > > > > depends on the use case, i.e. wheth

Re: [VOTE] KIP-90 Remove zkClient dependency from Streams

2016-12-20 Thread Apurva Mehta
+1 On Tue, Dec 20, 2016 at 1:01 PM, Hojjat Jafarpour wrote: > Hi all, > > Seems that there is no opposition to this KIP. This email is to start the > voting for this KIP. > Once again the KIP is for removing zkClient dependency from Streams. Please > check out the KIP page: > > https://cwiki.apa

Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

2016-12-20 Thread Apurva Mehta
Hi Ben, Thanks for the KIP. It is very well written and explains the problem and solution very nicely. I have one --very minor-- question. In the 'steps' section, you write: > 4.6 The follower starts fetching from the leader from its log end offset. The use of 'its' is a bit ambiguous here. I pr

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-21 Thread Apurva Mehta
Hi Joel, The alternatives are embedded in the 'discussion' sections which are spread throughout the google doc. Admittedly, we have not covered high level alternatives like those which have been brought up in this thread. In particular, having a separate log for transactional mesages and also hav

Re: [DISCUSS] 0.10.3.0/0.11.0.0 release planning

2017-02-28 Thread Apurva Mehta
+1 (non-binding) for 0.11.0 I do agree with Ismael's point that exactly-once should go through one release of stabilization before bumping the version to 1.0. Thanks, Apurva On Mon, Feb 27, 2017 at 7:47 PM, Ismael Juma wrote: > Hi all, > > With 0.10.2.0 out of the way, I would like to voluntee

Re: [VOTE] KIP-119: Drop Support for Scala 2.10 in Kafka 0.11

2017-03-01 Thread Apurva Mehta
+1 (non-binding) On Wed, Mar 1, 2017 at 4:41 PM, Ewen Cheslack-Postava wrote: > +1 (binding) > > -Ewen > > On Wed, Mar 1, 2017 at 4:56 AM, Jozef.koval > wrote: > > > +1 (non-binding) > > > > Jozef > > > > P.S. I volunteer to help with this KIP. > > > > > > Sent from [ProtonMail](https://protonm

Re: [VOTE] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-29 Thread Apurva Mehta
+1 (non-binding) On Wed, Mar 29, 2017 at 9:17 AM, Jay Kreps wrote: > +1 > > -Jay > > On Mon, Mar 20, 2017 at 11:27 AM, Matthias J. Sax > wrote: > > > Hi, > > > > I would like to start the vote for KIP-129. Of course, feel free to > > provide some more feedback on the DISCUSS thread. > > > > Tha

Re: [VOTE] Vote for KIP-101 - Leader Epochs

2017-01-04 Thread Apurva Mehta
Looks good to me! +1 (non-binding) On Wed, Jan 4, 2017 at 9:24 AM, Ben Stopford wrote: > Hi All > > We’re having some problems with this thread being subsumed by the > [Discuss] thread. Hopefully this one will appear distinct. If you see more > than one, please use this one. > > KIP-101 should

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-05 Thread Apurva Mehta
Hello all, I updated the KIP document with the changes discussed here. In particular: 1. The consumer config was renamed from 'fetch.mode' to 'isolation.level'. The values were changed appropriately as well. 2. Clarified the transactional guarantees for the consumer. 3. Added a subse

Re: [VOTE] KIP-102 - Add close with timeout for consumers

2017-01-06 Thread Apurva Mehta
+1 (non-binding). On Fri, Jan 6, 2017 at 9:24 AM, Jason Gustafson wrote: > Thanks for the KIP. +1 > > On Fri, Jan 6, 2017 at 2:26 AM, Ismael Juma wrote: > > > Thanks for the KIP, +1 (binding). > > > > As I said in the discussion thread, I'm not too sure about the hardcoded > 30 > > seconds time

Re: [VOTE] KIP-108: Create Topic Policy

2017-01-09 Thread Apurva Mehta
+1, that1 On Mon, Jan 9, 2017 at 2:47 PM, Neha Narkhede wrote: > +1 - thanks Ismael! > > On Mon, Jan 9, 2017 at 2:30 PM Gwen Shapira wrote: > > > +1 - thanks for the proposal, I think it will be super useful for admins. > > > > On Sun, Jan 8, 2017 at 6:50 AM, Ismael Juma wrote: > > > Hi all, >

Re: [VOTE] KIP-108: Create Topic Policy

2017-01-09 Thread Apurva Mehta
(hit send too soon) +1 (non-binding).. that is a very well written KIP! On Mon, Jan 9, 2017 at 3:28 PM, Apurva Mehta wrote: > +1, that1 > > On Mon, Jan 9, 2017 at 2:47 PM, Neha Narkhede wrote: > >> +1 - thanks Ismael! >> >> On Mon, Jan 9, 2017 at 2:30 PM G

Re: [VOTE] KIP-106 - Default unclean.leader.election.enabled True => False

2017-01-11 Thread Apurva Mehta
+1 (non-binding) On Wed, Jan 11, 2017 at 11:45 AM, Gwen Shapira wrote: > +1 > > On Wed, Jan 11, 2017 at 10:56 AM, Ben Stopford wrote: > > Looks like there was a good consensus on the discuss thread for KIP-106 > so > > lets move to a vote. > > > > Please chime in if you would like to change the

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-17 Thread Apurva Mehta
ow, the > broker > > > > > simply advances the LSO to the NSO. If there is at least one > aborted > > > > > message, the broker first replaces the current log segment with new > > log > > > > > segments excluding the aborted messages and then advances the LSO

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-17 Thread Apurva Mehta
with > 3 > > > new segments: (1) a new "shadow" log segment that simply references the > > > portion of the current log segment from the beginning to the LSO, (2) a > > log > > > segment created by copying only committed messages between the LSO and > &

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-17 Thread Apurva Mehta
e >> > > broker. It may be possible to do this efficiently if we assume aborted >> > > transactions are rare. The following is a draft proposal. For each >> > > partition, the broker maintains the last stable offset (LSO) as >> described >> >

Re: [VOTE] KIP-74: Add FetchResponse size limit in bytes

2017-01-20 Thread Apurva Mehta
+1 On Fri, Jan 20, 2017 at 5:19 PM, Jason Gustafson wrote: > +1 > > On Fri, Jan 20, 2017 at 4:51 PM, Ismael Juma wrote: > > > Good catch, Colin. +1 to editing the wiki to match the desired behaviour > > and what was implemented in 0.10.1. > > > > Ismael > > > > On Sat, Jan 21, 2017 at 12:19 AM,

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-25 Thread Apurva Mehta
e portion of the > > > current log segment from the NSO (open ended). Note that only (2) > > involves > > > real data copying. If aborted transactions are rare, this overhead will > > be > > > insignificant. Assuming that applications typically don't abo

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-25 Thread Apurva Mehta
On Tue, Jan 17, 2017 at 6:17 PM, Apurva Mehta wrote: > Hi Jun, > > Some answers in line. > > > 109. Could you describe when Producer.send() will receive an Unrecognized > > MessageException? > > > This exception will be thrown if the producer sends a sequence n

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-26 Thread Apurva Mehta
ion the prevention of duplicate messages in case of a partition > leader migration? Because if a broker fails or the publisher fails, KIP-98 > does not seem to change the risk of dupes for non-transactional producers. > > Btw: Good job! Both in terms of Kafka in general, and KIP-9

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-27 Thread Apurva Mehta
Hi again Jun, I have update the document to address your comments below, but including the responses inline to make it easier for everyone to stay on top of the conversation. > 106. Compacted topics. > 106.1. When all messages in a transaction are removed, we could remove the > commit/abort mar

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-30 Thread Apurva Mehta
> 102. When there is a correlated hard failure (e.g., power outage), it's >> possible that an existing commit/abort marker is lost in all replicas. >> This may not be fixed by the transaction coordinator automatically and >> the >> consumer may get stuck on that incomplete transaction forev

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-30 Thread Apurva Mehta
> > Eugen, moving your email to the main thread so that it doesn't get split. >> >> The `transaction.app.id` is a prerequisite for using transactional APIs. >> And only messages wrapped inside transactions will enjoy idempotent >> guarantees across sessions, and that too only when they employ a >>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-30 Thread Apurva Mehta
> Bumping one suggestion from Apurva above. The name "AppID" has caused some > confusion. We're considering the following renaming: > > 1. AppID -> ProducerId (transaction.app.id -> producer.id) > 2. PID -> IPID (internal producer ID) > How about AppId -> TransactionalId (transaction.app.id -> tr

Re: [VOTE] KIP-107 Add purgeDataBefore() API in AdminClient

2017-01-31 Thread Apurva Mehta
Hi Dong, It looks like this vote passed. Can you close this thread and update the KIP table? Thanks, Apurva On Tue, Jan 24, 2017 at 1:30 PM, Jun Rao wrote: > Hi, Dong, > > The changes sound good to me. Also, thanks for the explanation of returning > a future from purgeDataFrom(). We can keep i

Re: [VOTE] KIP-107 Add purgeDataBefore() API in AdminClient

2017-01-31 Thread Apurva Mehta
> > > > Thanks, > > Dong > > > > On Tue, Jan 31, 2017 at 5:45 PM, Apurva Mehta > wrote: > > > >> Hi Dong, > >> > >> It looks like this vote passed. Can you close this thread and update the > >> KIP table? > >> >

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-02 Thread Apurva Mehta
gt;>>> the magic byte cannot be changed in order to provide a path for >>> upgrades >>>>> following a similar approach as was used in KIP-32". This makes >> things >>>>> easier, but it seems to me that the only strict requirement is that >> th

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-02 Thread Apurva Mehta
The wiki has been updated with a section on authorization, as well a summary of the message format changes. On Thu, Feb 2, 2017 at 9:38 AM, Jason Gustafson wrote: > Thanks Tom, we'll update the wiki to reflect all the movement on the design > document. Did you have a specific concern with the ne

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-06 Thread Apurva Mehta
Hi Tom, I updated the KIP with a note our plans for performance testing: https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging#KIP-98-ExactlyOnceDeliveryandTransactionalMessaging-Performance Thanks for pointing that out. Regards, Apurva

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-06 Thread Apurva Mehta
e through a coordinator is copied from how consumer offsets are > managed). We've moved as much work from the clients to the broker as > possible, and we have resisted at every turn complicating the client APIs > even where it may have simplified some internals. > > -Jason > &g

Re: [VOTE] KIP-118: Drop Support for Java 7 in Kafka 0.11

2017-02-09 Thread Apurva Mehta
+1 (non-binding) On Thu, Feb 9, 2017 at 10:33 AM, Colin McCabe wrote: > +1 (non-binding) > > > > On Thu, Feb 9, 2017, at 07:31, Ismael Juma wrote: > > Hi everyone, > > > > Since everyone in the discuss thread was in favour (10 people responded), > > I > > would like to initiate the voting proces

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-10 Thread Apurva Mehta
ack? > >> > >> I don't think anyone needs non-idempotent producer, just want to > clarify :) > >> > >> On Tue, Feb 7, 2017 at 3:10 PM, Jason Gustafson > >> wrote: > >> > A couple follow-ups from Ismael's previous comments. > >

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-13 Thread Apurva Mehta
Hi Jun, Thanks for the comments. 200.1, 200.2, 200.3 were all oversights which have been fixed. 201.1: This has been added to the BeginTxnRequest now. If the transaction.timeout.ms value is greater than max.transaction.timeout.ms, then the BeginTxnRequest will fail with a `InvalidTransactionTime

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-15 Thread Apurva Mehta
Hi Jun, Answers inline: 210. Pid snapshots: Is the number of pid snapshot configurable or hardcoded > with 2? When do we decide to roll a new snapshot? Based on time, byte, or > offset? Is that configurable too? > These are good questions. We haven't fleshed out the policy by which the snapshots

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-16 Thread Apurva Mehta
Hi Jun, Thanks for the reply. Comments inline. On Thu, Feb 16, 2017 at 2:29 PM, Jun Rao wrote: > Hi, Apurva, > > Thanks for the reply. A couple of comment below. > > On Wed, Feb 15, 2017 at 9:45 PM, Apurva Mehta wrote: > > > Hi Jun, > > > > Answers inline

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-21 Thread Apurva Mehta
th a message set? > > > > > > > > On Fri, Feb 17, 2017 at 3:28 AM, Michael Pearce < > michael.pea...@ig.com > > > > > > > wrote: > > > > > > > > > +0 > > > > > >

Re: [DISCUSS] KIP-126 - Allow KafkaProducer to batch based on uncompressed size

2017-02-21 Thread Apurva Mehta
Hi Becket, Thanks for the kip. I think one of the risks here is that when compression estimation is disabled, you could have much smaller batches than expected, and throughput could be hurt. It would be worth adding this to the documentation of this setting. Also, one of the rejected alternatives

Re: [ANNOUNCE] New committer: Rajini Sivaram

2017-04-24 Thread Apurva Mehta
Congratulations Rajini! On Mon, Apr 24, 2017 at 2:06 PM, Gwen Shapira wrote: > The PMC for Apache Kafka has invited Rajini Sivaram as a committer and we > are pleased to announce that she has accepted! > > Rajini contributed 83 patches, 8 KIPs (all security and quota > improvements) and a signif

[jira] [Resolved] (KAFKA-5865) Expiring batches with idempotence enabled could cause data loss.

2017-09-25 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5865?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apurva Mehta resolved KAFKA-5865. - Resolution: Fixed This is fixed in 1.0.0 by the changes in https://github.com/apache/kafka/pull

  1   2   3   4   5   >