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

Flavio Junqueira commented on KAFKA-3197:
-----------------------------------------

[~becket_qin] thanks for the clarification. 

bq. 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.

I agree that duplicates are inconvenient, but in this scenario we aren't 
promising no duplicates, so I'd rather treat the duplicates separately.

bq. Currently after batches leave the record accumulator, we only track them in 
requests.

The record accumulator point is a good one and I'm not super familiar with that 
part of the code, so I don't have any concrete suggestion right now, but I'll 
have a closer look. However, 

bq. 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.

True, from your description, it sounds like the change isn't trivial. But let 
me ask you this: don't we ever have to retransmit messages after a leader 
change? If we do, then the code path for retransmitting on a different 
connection must be there. I'm not super familiar with that part of the code, so 
I don't have any concrete suggestion right now, but I can have a look to see if 
I'm able to help out.



> 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