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

Apurva Mehta commented on KAFKA-5870:
-------------------------------------

The only downside is that when previously sent batches expire, _and_ inflight 
batches get a retriable error, then the latter will get a spurious 
{{OutOfOrderSequenceException}}.

I think we should fix it, but the 'proper' fix would require yet more state in 
the producer: reset is requested, but should only kick in once there are no 
more inflights, and there are no more batches with the old producer id in the 
queue (ie. those which were previously drained and are now being retried). And 
how long should we wait to drain the old batches in the queue and block 
everything else (which may cause more batches to expire, etc.). 

There are a few things think through. So I am hesitant to add that complication 
to 1.0.0, since we already have a major complication with max.in.flight > 1.

Perhaps a simple solution is to send a different exception when we give up on 
the inflight batch, rather than an {{OutOfOrderSequenceException}}, to reduce 
confusion. Maybe an {{OperationNotAttemptedException}} ?

> Idempotent producer: a producerId reset causes undesirable behavior for 
> inflight batches to other partitions
> ------------------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-5870
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5870
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.11.0.0
>            Reporter: Apurva Mehta
>            Assignee: Apurva Mehta
>              Labels: exactly-once
>             Fix For: 1.1.0
>
>
> Currently, if we have to reset the producer id for any reason (for instance 
> if batches to a partition get expired, if we get an 
> {{OutOfOrderSequenceException}}, etc) we could cause batches to other 
> --healthy-- partitions to fail with a spurious 
> {{OutOfOrderSequenceException}}.
> This is detailed in this PR discussion: 
> https://github.com/apache/kafka/pull/3743#discussion_r137907630
> Ideally, we would want all inflight batches to be handled to completion 
> rather than potentially failing them prematurely. Further, since we want to 
> tighten up the semantics of the {{OutOfOrderSequenceException}}, at the very 
> least we should raise another exception in this case, because there is no 
> data loss on the broker when the client gives up. 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to