In general, the producer would retry internally.

If it sends batch X, and didn't get an ack back for it, and gets a sequence error for X+1 that it sent after X, the producer would resend X again (as it knows X was never received by the broker) and afterwards X+1.

Only if the producer did get an ack for X (and thus purged the batch from it's internal buffer), it would raise the error to the application as batch X cannot be resend, because the broker did ack, but somehow seems to have lost the data. (Reason could be some miss-configuration or unclean leader election as examples.)

For the "risks reordering of sent records" the issue is as follows: after the exception, the producer would not drop buffered records, but would bump it's epoch including a sequence number reset. Thus, if you call `send()` for the record of X now to they would be put _after_ the already buffered record from X-1 (X-1 can still be retried with the new epoch and new sequence number Y, and the original batch X could become Y+1).

Only if you close() the producer and put all data back via send() in the right order, ordering can be preserved. But it's now the apps responsibility to call send() in the right order a second time.


-Matthias


On 6/7/22 3:37 PM, Gabriel Giussi wrote:
Thanks for the answer Matthias.
I still have doubts about the meaning of "risks reordering of sent record".
If I understood correctly the example you gave is something like this
1. Producer sends batch with sequence number X
2. That request gets lost in the network
3. Producer sends batch with sequence number X+1
4. Broker receives batch with sequence number X+1 and returns an error and
the Producer throws a OutOfOrderSequenceException

In that situation we could keep retrying sending batch with sequence number
X+1 but we will keep getting a OutOfOrderSequenceException, or we ideally
also resend a batch with sequence number X, and after being accepted send
the one with X+1.
If what I'm saying is correct then I can't see how this can reorder the
messages, I mean if both batches include a message being written to topic
A, could messages from batch with sn X+1 end up being persisted with an
offset lesser than the ones from the batch with sn X?
Does this question make sense?

El mar, 7 jun 2022 a las 16:13, Matthias J. Sax (<mj...@apache.org>)
escribió:

Yes, the broker de-dupes using the sequence number.

But for example, if a sequence number is skipped, you could get this
exception: the current batch of messages cannot be appended to the log,
as one batch is missing, and the producer would need to re-send the
previous/missing batch with lower sequence number before it can move to
the "next" (ie current) batch.

Does this make sense?


-Matthias

On 5/27/22 10:43 AM, Gabriel Giussi wrote:
The docs say
"This exception indicates that the broker received an unexpected sequence
number from the producer, which means that data may have been lost. If
the
producer is configured for idempotence only (i.e. if enable.idempotence
is
set and no transactional.id is configured), it is possible to continue
sending with the same producer instance, but doing so risks reordering of
sent record"

Isn't the broker using the monotonically increasing sequence number to
dedup messages? So how can it break message ordering without breaking
idempotency?
I can't see an example scenario where this could happen, I guess
the OutOfOrderSequenceException can only happen
with max.in.flight.requests.per.connection > 1, but even in that case why
are not going to keep getting an OutOfOrderSequenceException but instead
a
success that broke message ordering?

Thanks.



Reply via email to