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

2017-01-05 Thread Rajini Sivaram
Thank you, Jason. I will start the vote. On Thu, Jan 5, 2017 at 5:52 PM, Jason Gustafson wrote: > Yeah, if you start a vote soon, I think it has a chance to get into 0.10.2. > I guess it's up to Ewen, but I'm happy to help review. > > -Jason > > On Wed, Jan 4, 2017 at 11:42

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

2017-01-05 Thread Jason Gustafson
Yeah, if you start a vote soon, I think it has a chance to get into 0.10.2. I guess it's up to Ewen, but I'm happy to help review. -Jason On Wed, Jan 4, 2017 at 11:42 AM, Rajini Sivaram wrote: > Hi Jason, > > Yes, we do potentially timeout even before sending pending

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

2017-01-04 Thread Rajini Sivaram
Hi Jason, Yes, we do potentially timeout even before sending pending commits after the request timeout (default is > 5 minutes, so this should only happen when there are real issues or when brokers are shutdown). I have updated the KIP to use a default timeout of 30 seconds for the existing

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

2017-01-04 Thread Jason Gustafson
Hi Rajini, Thanks for the clarification. I looked again at the patch and I see what you're saying now. I was confused because I assumed the request timeout was being enforced on the requests themselves, but it is more that the request timeout bounds the attempt to send them in addition to the

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

2017-01-04 Thread Rajini Sivaram
Hi Jason, Thank you for the review. During close(), if there is a rebalance and the coordinator has to be rediscovered, close terminates without trying to find the coordinator. The poll() loop within close terminates if the coordinator is not known (as it does now) or if the timeout expires. At

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

2017-01-03 Thread Jason Gustafson
Hey Rajini, Thanks for the KIP. I had a quick look at the patch and the impact doesn't seem too bad. Just wanted to clarify one point. This is from the KIP: The existing close() method without a timeout will attempt to close the > consumer gracefully with a timeout of Long.MAX_VALUE. Since

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

2016-12-21 Thread Rajini Sivaram
I have added some more detail to the "Proposed Changes" section. Also created a preliminary PR for the JIRA ( https://github.com/apache/kafka/pull/2285). I am using *request.timeout.ms * to bound individual requests during close (the KIP does not address timeouts in any

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

2016-12-19 Thread Guozhang Wang
+1 on this idea as well. Streams has also added a similar feature itself partly because consumer does not support it directly (other part of the reason is that like brokers, streams also have some exception handling logic which could lead to deadlock with careless System.exit). For consumer

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

2016-12-19 Thread Rajini Sivaram
Thank you for the reviews. @Becket @Ewen, Agree that making all blocking calls have a timeout will be trickier and hence the scope of this KIP is limited to close(). @Jay Yes, this should definitely go into release notes, will make sure it is added. I will add some integration tests with broker

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

2016-12-16 Thread Jay Kreps
I think this is great. Sounds like one implication is that existing code that called close() and hit the timeout would now hang indefinitely. We saw this kind of thing a lot in automated testing scenarios where people don't correctly sequence their shutdown of client and server. I think this is

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

2016-12-15 Thread Becket Qin
+1 on the idea. We have a ticket about making all the blocking call have a timeout in KafkaConsumer. The implementation could be a little tricky as Ewen mentioned. But for close it is probably a simpler case because in the worst case the consumer will just stop polling and heartbeating and

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

2016-12-15 Thread Rajini Sivaram
Ewen, Thank you, I will try to prototype a solution early next week to get a better understanding of how invasive the changes are. On Thu, Dec 15, 2016 at 9:35 PM, Ewen Cheslack-Postava wrote: > Rajini, > > Thanks for this KIP, I'd definitely like to see this. Connect has

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

2016-12-15 Thread Ewen Cheslack-Postava
Rajini, Thanks for this KIP, I'd definitely like to see this. Connect has had a long-standing TODO around stopping sink tasks where we can't properly manage the rebalance process (which involves stopping consumers) because we lack a timeout here. Not a huge problem in practice, but would be nice

[DISCUSS] KIP-102 - Add close with timeout for consumers

2016-12-15 Thread Rajini Sivaram
Hi all, I have just created KIP-102 to add a new close method for consumers with a timeout parameter, making Consumer consistent with Producer: https://cwiki.apache.org/confluence/display/KAFKA/KIP-102+-+Add+close+with+timeout+for+consumers Comments and suggestions are welcome. Thank you...