Quick follow-up on the discussion on KIP-19. For partitionsFor() I think
the question was whether to use max.enqueue.block.ms or request.timeout.ms
to control the timeout. The proposed plan was to use request.timeout.ms.
Alternately we could change the config max.enqueue.block.ms to max.block.ms
Thanks Jay. max.block.ms looks good. I will update the wiki page.
Jiangjie (Becket) Qin
On 6/2/15, 11:26 AM, Jay Kreps jay.kr...@gmail.com wrote:
Quick follow-up on the discussion on KIP-19. For partitionsFor() I think
the question was whether to use max.enqueue.block.ms or request.timeout.ms
Option 3 seems a lot better than previous options, especially from the
user's perspective. I think it gives reasonable balance between control and
fewer options, and the only implementation details it's exposing are that
there is a buffer and there is a network request. Making the request
timeout
Bump up this thread.
After several discussions in LinkedIn, we came up with three options. I
have updated the KIP-19 wiki page to summarize the three options and
stated our preference. We can discuss on them in tomorrow’s KIP hangout.
Please let us know what do you think.
Thanks,
Jiangjie
The fact that I understand the producer internals and am still struggling
to understand the implications of the different settings, how I would set
them, and how they potentially interact such that I could set invalid
combinations seems like a red flag to me... Being able to say I want
Hey Ewen,
I agree with you that we should avoid any unnecessary configurations
exposed to user. And the necessity is defined by use case.
I also agree that the configurations should be named from users’
perspective and comply with the intuition - for example, like what Joel
said, something like
The fact that I understand the producer internals and am still struggling
to understand the implications of the different settings, how I would set
them, and how they potentially interact such that I could set invalid
combinations seems like a red flag to me... Being able to say I want
produce
Hey Mayuresh,
I think our purpose is to find out the simplest user interface but still
support all the flexibilities user might need.
In that sense, we have to expose
1. linger.ms for batching purpose,
2. request timeout to support different RTT.
3. Some blocking timeout for send()
Etc,..
So
Hi Jay,
I updated what I think int KIP wiki. Just a short summary here. Because we
need timeout for:
1. Send()
2. Batches in accumulator
3. Requests in flight.
That means we need to have at least three configurations if we do not
reuse configurations.
I think we probably want to also separate
Hi Jay,
If I am understanding this correctly, we should treat the Batch Timeout
separately from the Request Timeout. Request time out can be used only for
inflight request (request that have been sent to kafka brokers and waiting
for response) and Batch Timeout can be used to expire the batches
Yeah I think linger.ms remains separate, setting that is a performance
optimization rather than failure handling thing. We should ideally sanity
check this, though, in my proposal, since if they set linger.ms
request.timeout then that won't work.
It's true that in my proposal that the actual
So what I understand is that, we would have 3 time outs :
1) replication timeout
2) request timeout
3) metadata timeout (existing)
The request timeout has to be greater than the replication timeout.
request timeout is for messages already sent to kafka and the producer is
waiting for them.
Here is the concern I had with reusing the metadata.fetch.timeout.ms:
Previously people were using this as a bound on the time send() would
block. It isn't a bound on the time we will wait on a metadata request,
just the time the send call will block if metadata is missing for the
topic. We told
So the alternative to consider would be to instead have
max.block.ms (or something)
request.timeout
replication.timeout
I think this better captures what the user cares about. Here is how it
would work.
*max.send.block.ms http://max.send.block.ms* is the bound on the maximum
time the
Hey Jay,
That is also a viable solution.
I think the main purpose is to let user know how long they can block,
which is important.
I have some question over the proposal, though. Will user still need to
send linger.ms? Will request timeout cover linger.ms as well?
My concern of letting request
IMO, having 4 different timeouts makes it confusing for the user and it
requires the client to understand the internals of kafka. We should have a
single timeout from the users perspective and handle other timeouts
internally like a batch timeout.
Mayuresh
On Tue, May 19, 2015 at 12:42 PM,
I think this looks good. What I think is missing is an overview of the
timeouts from the user's perspective.
My worry is that it is quite complicated to reason about the current set of
timeouts. Currently we have
timeout.ms
metadata.fetch.timeout.ms
The proposed settings I think are:
Hey Jay,
I think that is a very reasonable concern.
So the current behavior for those users are:
1. Send() will go through as long as metadata is available.
2. Send() will throw exception if the metadata of a partition is lost
after pre-initialization.
3. The messages in accumulator will not be
Hi Jiangjie,
So when you say :
It is probably better to fail the
messages in accumulator than keeping them in that case because I really
cannot think of any case where metadata of a partition can disappear and
come up again shortly,
This is true
* unless there is a metadata refresh that occurs
+1 on creating a new connection.
So from what I understand the request timeout should be greater than
the replication timeout in any case.
If the broker is slow or not responding and the request times out we will
treat it as we treat disconnections and update metadata try sending it to
new leader
I modified the WIKI page to incorporate the feedbacks from mailing list
and KIP hangout.
- Added the deprecation plan for TIMEOUT_CONFIG
- Added the actions to take after request timeout
I finally chose to create a new connection if requests timeout. The reason
is:
1. In most cases, if a broker
I think my confusion is coming from this:
So in this KIP, we only address (3). The only public interface change is a
new configuration of request timeout (and maybe change the configuration
name of TIMEOUT_CONFIG to REPLICATION_TIMEOUT_CONFIG).
There are 3 possible compatibility issues I see
+1 Becket. That would give enough time for clients to move. We should make
this change very clear.
Thanks,
Mayuresh
On Tue, May 12, 2015 at 1:45 PM, Jiangjie Qin j...@linkedin.com.invalid
wrote:
Hey Ewen,
Very good summary about the compatibility. What you proposed makes sense.
So
Yes, that is the plan.
On 5/5/15, 8:23 PM, Mayuresh Gharat gharatmayures...@gmail.com wrote:
Just a quick question, can we handle REQUEST TIMEOUT as disconnections and
do a fresh MetaDataRequest and retry instead of failing the request?
Thanks,
Mayuresh
On Mon, May 4, 2015 at 10:32 AM,
Jiangjie,
Yes, I think using metadata timeout to expire batches in the record
accumulator makes sense.
Thanks,
Jun
On Mon, May 4, 2015 at 10:32 AM, Jiangjie Qin j...@linkedin.com.invalid
wrote:
I incorporated Ewen and Guozhang’s comments in the KIP page. Want to speed
up on this KIP because
Just a quick question, can we handle REQUEST TIMEOUT as disconnections and
do a fresh MetaDataRequest and retry instead of failing the request?
Thanks,
Mayuresh
On Mon, May 4, 2015 at 10:32 AM, Jiangjie Qin j...@linkedin.com.invalid
wrote:
I incorporated Ewen and Guozhang’s comments in the
I incorporated Ewen and Guozhang’s comments in the KIP page. Want to speed
up on this KIP because currently we experience mirror-maker hung very
likely when a broker is down.
I also took a shot to solve KAFKA-1788 in KAFKA-2142. I used metadata
timeout to expire the batches which are sitting in
Jun,
I thought a little bit differently on this.
Intuitively, I am thinking that if a partition is offline, the metadata
for that partition should be considered not ready because we don’t know
which broker we should send the message to. So those sends need to be
blocked on metadata timeout.
Jiangjie,
Allowing messages to be accumulated in an offline partition could be useful
since the partition may become available before the request timeout or
linger time is reached. Now that we are planning to add a new timeout, it
would be useful to think through whether/how that applies to
Agreed we also need to change in the code of Sender.java to indicate that
it resembles REPLICATION_TIMEOUT and not the request Timeout.
Thanks,
Mayuresh
On Thu, Apr 16, 2015 at 1:08 PM, Jiangjie Qin j...@linkedin.com.invalid
wrote:
Hi Guozhang,
By implicit timeout for close() and flush(), I
Hi Harsha,
Took a quick look at the patch. I think it is still a little bit
different. KAFKA-1788 only handles the case where a batch sitting in
accumulator for too long. The KIP is trying to solve the issue where a
batch has already been drained from accumulator and sent to broker.
We might be
Hi Guozhang,
By implicit timeout for close() and flush(), I meant that currently we
don’t have a explicit timeout for close() or flush() when a broker is
down, so they can take pretty long up to TCP timeout which is hours as you
mentioned. With the client side request timeout, the waiting time
Checked the code again. It seems that the disconnected channel is not
detected by selector as expected.
Currently we are depending on the
o.a.k.common.network.Selector.disconnected set to see if we need to do
something for a disconnected channel.
However Selector.disconnected set is only updated
Thanks for the update Jiangjie,
I think it is actually NOT expected that hardware disconnection will be
detected by the selector, but rather will only be revealed upon TCP
timeout, which could be hours.
A couple of comments on the wiki:
1. For KafkaProducer.close() and KafkaProducer.flush() we
Jiangjie,
Great start. I have a couple of comments.
Under the motivation section, is it really true that the request will never
be completed? Presumably if the broker goes down the connection will be
severed, at worst by a TCP timeout, which should clean up the connection
and any outstanding
I suppose adding this timeout will help. In cases were the broker is not
completely down but stops responding to the produce request, tools like
Mirror Makers will hang since they are waiting for responses. Adding this
timeout enables it to fail the current request and retry with fresh
metadata.
On Tue, Apr 14, 2015 at 1:57 PM, Jiangjie Qin j...@linkedin.com.invalid
wrote:
Hi Ewen, thanks for the comments. Very good points! Please see replies
inline.
On 4/13/15, 11:19 PM, Ewen Cheslack-Postava e...@confluent.io wrote:
Jiangjie,
Great start. I have a couple of comments.
Under
Hi Ewen, thanks for the comments. Very good points! Please see replies
inline.
On 4/13/15, 11:19 PM, Ewen Cheslack-Postava e...@confluent.io wrote:
Jiangjie,
Great start. I have a couple of comments.
Under the motivation section, is it really true that the request will
never
be completed?
Hi,
I just created a KIP to add a request timeout to NetworkClient for new Kafka
clients.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-19+-+Add+a+request+timeout+to+NetworkClient
Comments and suggestions are welcome!
Thanks.
Jiangjie (Becket) Qin
39 matches
Mail list logo