Rajini,

For this to have the desired effect, we'd probably need to lower the
default request.timeout.ms for the consumer and fix the underlying reason
why it is a little over 5 minutes at the moment.

Ismael

On Mon, May 22, 2017 at 4:15 PM, Rajini Sivaram <rajinisiva...@gmail.com>
wrote:

> Hi David,
>
> Sorry, what I meant was: Can you reuse the existing configuration option
> request.timeout,ms , instead of adding a new config and add the behaviour
> that you have proposed in the KIP for the connection phase using this
> timeout? I think the timeout for connection is useful. I am not sure we
> need another configuration option to implement it.
>
> Regards,
>
> Rajini
>
>
> On Mon, May 22, 2017 at 11:06 AM, 东方甲乙 <254479...@qq.com> wrote:
>
> > Hi Rajini.
> >
> > When kafka node' machine is shutdown or network is closed, the connecting
> > phase could not use the request.timeout.ms, because the client haven't
> > send a req yet.   And no response for the nio, the selector will not
> close
> > the connect, so it will not choose other good node to get the metadata.
> >
> >
> > Thanks
> > David
> >
> > ------------------ 原始邮件 ------------------
> > *发件人:* "Rajini Sivaram" <rajinisiva...@gmail.com>;
> > *发送时间:* 2017年5月22日(星期一) 20:17
> > *收件人:* "dev" <dev@kafka.apache.org>;
> > *主题:* Re: [DISCUSS] KIP-148: Add a connect timeout for client
> >
> >
> > Hi David,
> >
> > Is there a reason why you wouldn't want to use request.timeout.ms as the
> > timeout parameter for connections? Then you would use the same timeout
> for
> > connected and connecting phases when shutdown is unclean. You could still
> > use the timeout to ensure that next metadata request is sent to another
> > node.
> >
> > Regards,
> >
> > Rajini
> >
> > On Sun, May 21, 2017 at 9:51 AM, 东方甲乙 <254479...@qq.com> wrote:
> >
> > > Hi Guozhang,
> > >
> > >
> > > Thanks for the clarify. For the clarify 2, I think the key thing is not
> > > users control how much time in maximum to wait for inside code, but is
> > the
> > > network client can be aware of the connecting can't be finished and
> try a
> > > good node. In the producer.sender even the selector.poll can timeout,
> but
> > > the next time is also not close the previous connecting and try another
> > > good node.
> > >
> > >
> > > In out test env, QA shutdown one of the leader node, the producer send
> > the
> > > request will timeout and close the node's connection then request the
> > > metadata.  But sometimes the request node is also the shutdown node.
> > When
> > > connecting the shutting down node to get the metadata, it is in the
> > > connecting phase, network client mark the connecting node's state to
> > > CONNECTING, but if the node is shutdown,  the socket can't be aware of
> > the
> > > connecting is broken. Though the selector.poll has timeout parameter,
> but
> > > it will not close the connection, so the next
> > > time in the "networkclient.maybeUpdate" it will check if
> > > isAnyNodeConnecting, then will not connect to any good node the get the
> > > metadata.  It need about several minutes to
> > > aware the connecting is timeout and try other node.
> > >
> > >
> > > So I want to add a connect.timeout parameter,  the selector can find
> the
> > > connecting is timeout and close the connection.  It seems the currently
> > the
> > > timeout value passed in `selector.poll()`
> > > seems can not do this.
> > >
> > >
> > > Thanks,
> > > David
> > >
> > >
> > >
> > >
> > >
> > >
> > > ------------------ 原始邮件 ------------------
> > > 发件人: "Guozhang Wang";<wangg...@gmail.com>;
> > > 发送时间: 2017年5月16日(星期二) 凌晨1:51
> > > 收件人: "dev@kafka.apache.org"<dev@kafka.apache.org>;
> > >
> > > 主题: Re: [DISCUSS] KIP-148: Add a connect timeout for client
> > >
> > >
> > >
> > > Hi David,
> > >
> > > I may be a bit confused before, just clarifying a few things:
> > >
> > > 1. As you mentioned, a client will always try to first establish the
> > > connection with a broker node before it tries to send any request to
> it.
> > > And after connection is established, it will either continuously send
> > many
> > > requests (e.g. produce) for just a single request (e.g. metadata) to
> the
> > > broker, so these two phases are indeed different.
> > >
> > > 2. In the connected phase, connections.max.idle.ms is used to
> > > auto-disconnect the socket if no requests has been sent / received
> during
> > > that period of time; in the connecting phase, we always try to create
> the
> > > socket via "socketChannel.connect" in a non-blocking call, and then
> > checks
> > > if the connection has been established, but all the callers of this
> > > function (in either producer or consumer) has a timeout parameter as in
> > > `selector.poll()`, and the timeout parameter is set either by
> > calculations
> > > based on metadata.expiration.time and backoff for producer#sender, or
> by
> > > directly passed values from consumer#poll(timeout), so although there
> is
> > no
> > > directly config controlling that, users can still control how much time
> > in
> > > maximum to wait for inside code.
> > >
> > > I originally thought your scenarios is more on the connected phase, but
> > now
> > > I feel you are talking about the connecting phase. For that case, I
> still
> > > feel currently the timeout value passed in `selector.poll()` which is
> > > controllable from user code should be sufficient?
> > >
> > >
> > > Guozhang
> > >
> > >
> > >
> > >
> > > On Sun, May 14, 2017 at 2:37 AM, 东方甲乙 <254479...@qq.com> wrote:
> > >
> > > > Hi Guozhang,
> > > >
> > > >
> > > > Sorry for the delay, thanks for the question.  It seems two different
> > > > parameters to me:
> > > > connect.timeout.ms: only work for the connecting phrase, after
> > connected
> > > > phrase this parameter is not used.
> > > > connections.max.idle.ms: currently not work in the connecting phrase
> > > > (only select return readyKeys >0) will add to the expired manager,
> > after
> > > > connected will check if the connection is still alive in some time.
> > > >
> > > >
> > > > Even if we change the connections.max.idle.ms to work including the
> > > > connecting phrase, we can not set this parameter to a small value,
> such
> > > as
> > > > 5 seconds. Because the client is maybe busy sending message to other
> > > node,
> > > > it will be disconnected in 5 seconds, so the default value of
> > > > connections.max.idle.ms is setting to a larger time. We should have
> > two
> > > > parameters to control the connecting phrase behavior and the
> connected
> > > > phrase behavior, do you think so?
> > > >
> > > >
> > > > Thanks,
> > > >
> > > >
> > > > David
> > > >
> > > >
> > > >
> > > >
> > > > ------------------ 原始邮件 ------------------
> > > > 发件人: "Guozhang Wang";<wangg...@gmail.com>;
> > > > 发送时间: 2017年5月6日(星期六) 上午7:52
> > > > 收件人: "dev@kafka.apache.org"<dev@kafka.apache.org>;
> > > >
> > > > 主题: Re: [DISCUSS] KIP-148: Add a connect timeout for client
> > > >
> > > >
> > > >
> > > > Hello David,
> > > >
> > > > Thanks for the KIP. For the described issue, I'm wondering if it can
> be
> > > > resolved by tuning the CONNECTIONS_MAX_IDLE_MS_CONFIG (
> > > > connections.max.idle.ms) on the client side? Default is 9 minutes.
> > > >
> > > >
> > > > Guozhang
> > > >
> > > > On Tue, May 2, 2017 at 8:22 AM, 东方甲乙 <254479...@qq.com> wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > Currently in our test environment, we found that after one of the
> > > broker
> > > > > node crash (reboot or os crash), the client may still be connecting
> > to
> > > > the
> > > > > crash node to send metadata request or other request, and it needs
> > > > several
> > > > > minutes to be aware that the connection is timeout then try another
> > > node
> > > > to
> > > > > connect to send the request. Then the client may still not be aware
> > of
> > > > the
> > > > > metadata change after several minutes.
> > > > >
> > > > >
> > > > > So I want to add a connect timeout on the  client,  please take a
> > look
> > > > at:
> > > > >
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > 148%3A+Add+a+connect+timeout+for+client
> > > > >
> > > > > Regards,
> > > > >
> > > > > David
> > > >
> > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
> >
>

Reply via email to