[ 
https://issues.apache.org/jira/browse/KAFKA-3197?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15132698#comment-15132698
 ] 

Jiangjie Qin commented on KAFKA-3197:
-------------------------------------

[~fpj] [~ijuma] That was my first thinking as well. After a second thought it 
might be a little bit complicated for the current implementation.

This approach needs the following works:
1. Detect leader movement on each metadata refresh.
2. If leader moves, that does not necessarily mean the request to old leader 
failed. We can always send the unacknowledged message to new leader, but that 
probably introduce duplicate almost every time leader moves.
3. Currently after batches leave the record accumulator, we only track them in 
requests. If leader migrates, now we need to peek into every in flight request, 
take out the batches to the partition whose leader moved, and re-enqueue them 
in the to record accumulator. This is even more intrusive because we store the 
batches in the ProduceResponseHandler which we don't even track today. 

Compared with current approach, the benefit of doing that seems we potentially 
don't need to wait for request timeout if a broker is actually down. However, 
given the metadata refresh itself is usually triggered by request timeout, this 
benefit becomes marginal. 

So while the idea of resend unacknowledged message to both old and new leader 
is natural and makes sense, it seems much more complicated and error prone 
based on our current implementation and does not buy us much.

> Producer can send message out of order even when in flight request is set to 
> 1.
> -------------------------------------------------------------------------------
>
>                 Key: KAFKA-3197
>                 URL: https://issues.apache.org/jira/browse/KAFKA-3197
>             Project: Kafka
>          Issue Type: Bug
>          Components: clients, producer 
>    Affects Versions: 0.9.0.0
>            Reporter: Jiangjie Qin
>            Assignee: Jiangjie Qin
>             Fix For: 0.9.0.1
>
>
> The issue we saw is following:
> 1. Producer send message 0 to topic-partition-0 on broker A. The in-flight 
> request to broker A is 1.
> 2. The request is somehow lost
> 3. Producer refreshed its topic metadata and found leader of 
> topic-partition-0 migrated from broker A to broker B.
> 4. Because there is no in-flight request to broker B. All the subsequent 
> messages to topic-partition-0 in the record accumulator are sent to broker B.
> 5. Later on when the request in step (1) times out, message 0 will be retried 
> and sent to broker B. At this point, all the later messages has already been 
> sent, so we have re-order.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to