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 <http://request.timeout.ms>* to bound individual requests during close (the KIP does not address timeouts in any other code path) to ensure that *close()* always completes within a bounded time even when timeout is not specified. This is similar to the producer where requests are aborted after *request.timeout.ms <http://request.timeout.ms>. *The PR contains unit and integration tests for all the close scenarios I could think of (but there could be more). On Mon, Dec 19, 2016 at 10:32 PM, Guozhang Wang <wangg...@gmail.com> wrote: > +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 itself I think the > trickiness lies in the prefetching calls as well as commit / HB requests > cleanup with the timeout, and I agree with Ewen that it's better to be > merged in the early release cycle than a last minute merge. > > > > Guozhang > > On Mon, Dec 19, 2016 at 4:18 AM, Rajini Sivaram <rajinisiva...@gmail.com> > wrote: > > > 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 failures for > > testing the timeout, but they cannot completely eliminate the risk of a > > hang. Over time, hopefully system tests will help catch most issues. > > > > > > On Sat, Dec 17, 2016 at 1:15 AM, Jay Kreps <j...@confluent.io> wrote: > > > > > 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 > > > okay, but might be good to include in the release notes. > > > > > > -jay > > > > > > On Thu, Dec 15, 2016 at 5:32 AM, Rajini Sivaram <rsiva...@pivotal.io> > > > wrote: > > > > > > 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... > > > > > > > > > > > > > > > > > > Regards, > > > > > > > > > > > > > > > > > > Rajini > > > > > > > > > > > -- > > Regards, > > > > Rajini > > > > > > -- > -- Guozhang >