[
https://issues.apache.org/jira/browse/KAFKA-3197?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15130761#comment-15130761
]
Jiangjie Qin commented on KAFKA-3197:
-------------------------------------
[~enothereska] The documentation of max.in.flight.request.per.connection did
not say it explicitly, but I think followings are the guarantees we currently
claim (or think) we are providing with different
in.flight.request.per.connection and retries.
1. retries = 0, regardless of in.flight.request.per.connection
Producer itself does not introduce reordering in this case (all messages are
only sent once), but message send will very likely fail immediately when event
such as leader migration occurs. Users probably only have three choices when
message failure occurs: a) let it go so the message is dropped; b) close
producer if user do not tolerate message loss or re-ordering; c) resend the
message and have re-ordering (this re-ordering is introduced by user)
2. in.flight.request.per.connection >1 and retries > 0 (some reasonable number)
No worry about frequent message send failure, but re-order could happen when
there is retry.
3. in.flight.request.per.connection = 1 and retries > 0
No re-ordering and no frequent failure.
The bug here breaks the 3rd guarantee which we thought we are providing.
> 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)