+1, let's just start a new thread for this. On Tue, Mar 24, 2015 at 12:23 PM, Joel Koshy <jjkosh...@gmail.com> wrote:
> Actually, since there are already votes on this and the KIP has > changed a bit we should cancel this and start a new thread. > > On Tue, Mar 24, 2015 at 12:19 PM, Jiangjie Qin > <j...@linkedin.com.invalid> wrote: > > Push up the thread for voting after discussion on the KIP hangout. > > > > On 3/19/15, 9:03 PM, "Jiangjie Qin" <j...@linkedin.com> wrote: > > > >>We had some additional discussions on the discussion thread. Pushing up > >>this thread to resume voting. > >> > >>On 3/11/15, 8:47 PM, "Jay Kreps" <jay.kr...@gmail.com> wrote: > >> > >>>Yeah guys, I'd like to second that. I'd really really love to get the > >>>quality of these to the point where we could broadly solicit user input > >>>and > >>>use them as a permanent document of the alternatives and rationale. > >>> > >>>I know it is a little painful to have process, but I think we all saw > >>>what > >>>happened to the previous clients as public interfaces so I really really > >>>really want us to just be incredibly thoughtful and disciplined as we > >>>make > >>>changes. I think we all want to avoid another "client rewrite". > >>> > >>>To second Joe's question in a more specific way, I think an alternative > I > >>>don't see considered to give close() a bounded time is just to enforce > >>>the > >>>request time on the client side, which will cause all requests to be > >>>failed > >>>after the request timeout expires. This was the same behavior as for > >>>flush. > >>>In the case where the user just wants to ensure close doesn't block > >>>forever > >>>I think that may be sufficient? > >>> > >>>So one alternative might be to just do that request timeout feature and > >>>add > >>>a new producer config that is something like > >>> abort.on.failure=false > >>>which causes the producer to hard exit if it can't send a request. Which > >>>I > >>>think is closer to what you want, with this just being a way to > implement > >>>that behavior. > >>> > >>>I'm not sure if this is better or worse, but we should be sure before we > >>>make the change. > >>> > >>>I also have a concern about > >>> producer.close(0, TimeUnit.MILLISECONDS) > >>>not meaning close with a timeout of 0 ms. > >>> > >>>I realize this exists in other java apis, but it is so confusing it even > >>>confused us into having that recent producer bug because of course all > >>>the > >>>other numbers mean "wait that long". > >>> > >>>I'd propose > >>> close()--block until all completed > >>> close(0, TimeUnit.MILLISECONDS)--block for 0 ms > >>> close(5, TimeUnit.MILLISECONDS)--block for 5 ms > >>> close(-1, TimeUnit.MILLISECONDS)--error because blocking for negative > >>>ms > >>>would mean completing in the past :-) > >>> > >>>-Jay > >>> > >>>On Wed, Mar 11, 2015 at 8:31 PM, Joe Stein <joe.st...@stealth.ly> > wrote: > >>> > >>>> Could the KIP confluence please have updated the discussion thread > >>>>link, > >>>> thanks... could you also remove the template boilerplate at the top > >>>>"*This > >>>> page is meant as a template ..*" so we can capture it for the release > >>>> cleanly. > >>>> > >>>> Also I don't really/fully understand how this is different than > >>>> flush(time); close() and why close has its own timeout also? > >>>> > >>>> Lastly, what is the forceClose flag? This isn't documented in the > >>>>public > >>>> interface so it isn't clear how to completely use the feature just by > >>>> reading the KIP. > >>>> > >>>> ~ Joe Stein > >>>> - - - - - - - - - - - - - - - - - > >>>> > >>>> http://www.stealth.ly > >>>> - - - - - - - - - - - - - - - - - > >>>> > >>>> On Wed, Mar 11, 2015 at 11:24 PM, Guozhang Wang <wangg...@gmail.com> > >>>> wrote: > >>>> > >>>> > +1 (binding) > >>>> > > >>>> > On Wed, Mar 11, 2015 at 8:10 PM, Jiangjie Qin > >>>><j...@linkedin.com.invalid > >>>> > > >>>> > wrote: > >>>> > > >>>> > > > >>>> > > > >>>> > > >>>> > >>>> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-15+-+Add+a+close+m > >>>>e > >>>>thod+with+a+timeout+in+the+producer > >>>> > > > >>>> > > > >>>> > > >>>> > > >>>> > -- > >>>> > -- Guozhang > >>>> > > >>>> > >> > > > -- -- Guozhang