You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Jose Manuel Vega Monroy <jo...@williamhill.com> on 2019/10/07 09:41:21 UTC

OutOfOrderSequenceException

Hi there,

Finally we upgraded our producer configuration to ensure message order:

    retries = 1
# note to ensure order enable.idempotence=true, which forcing to acks=all and max.in.flight.requests.per.connection<=5
    enable.idempotence = true
    max.in.flight.requests.per.connection = 4
    acks = "all"

However, recently we faced this exception:

org.apache.kafka.common.errors.OutOfOrderSequenceException: The broker received an out of order sequence number..

Any idea why happened? Is that expected?

It could be related to retries configuration? It’s that configuration properly set?

From official documentation<https://kafka.apache.org/21/javadoc/org/apache/kafka/clients/producer/KafkaProducer.html> we found recommending unset retries, being default value to Integer.MAX_VALUE:


“To take advantage of the idempotent producer, it is imperative to avoid application level re-sends since these cannot be de-duplicated. As such, if an application enables idempotence, it is recommended to leave the retries config unset, as it will be defaulted to Integer.MAX_VALUE. Additionally, if a send(ProducerRecord) returns an error even with infinite retries (for instance if the message expires in the buffer before being sent), then it is recommended to shut down the producer and check the contents of the last produced message to ensure that it is not duplicated. Finally, the producer can only guarantee idempotence for messages sent within a single session.”

Thanks

[https://www.williamhillplc.com/content/signature/WHlogo.gif?width=180]<http://www.williamhill.com/>
[https://www.williamhillplc.com/content/signature/senet.gif?width=180]<http://www.whenthefunstops.co.uk/>
Jose Manuel Vega Monroy
Java Developer / Software Developer Engineer in Test
Direct: +0035 0 2008038 (Ext. 8038)
Email: jose.monroy@williamhill.com<ma...@williamhill.com>
William Hill | 6/1 Waterport Place | Gibraltar | GX11 1AA




Re: OutOfOrderSequenceException

Posted by "Matthias J. Sax" <ma...@confluent.io>.
An OutOfOrderSequenceException is a severe error indicating potential
data loss.

It can happen if there was a problem broker side, and data was truncated
because of a leader change, even after the data was acknowledged as
successfully written to the producer.

What is your topic/brokers configs?

You should use `replication.factor=3`, `min.in.sync.replicas=2` and
`unclean.leader.election.enable=false`.


-Matthias




On 10/7/19 2:41 AM, Jose Manuel Vega Monroy wrote:
> Hi there,
> 
>  
> 
> Finally we upgraded our producer configuration to ensure message order:
> 
>  
> 
>     retries = 1
> 
> # note to ensure order enable.idempotence=true, which forcing to
> acks=all and max.in.flight.requests.per.connection<=5
> 
>     enable.idempotence = true
> 
>     max.in.flight.requests.per.connection = 4
> 
>     acks = "all"
> 
>  
> 
> However, recently we faced this exception:
> 
>  
> 
> org.apache.kafka.common.errors.OutOfOrderSequenceException: The broker
> received an out of order sequence number..
> 
>  
> 
> Any idea why happened? Is that expected?
> 
>  
> 
> It could be related to retries configuration? It’s that configuration
> properly set?
> 
>  
> 
> From official documentation
> <https://kafka.apache.org/21/javadoc/org/apache/kafka/clients/producer/KafkaProducer.html>
> we found *recommending unset retries*, being default value to
> Integer.MAX_VALUE:
> 
>  
> 
> “To take advantage of the idempotent producer, it is imperative to avoid
> application level re-sends since these cannot be de-duplicated. As such,
> if an application enables idempotence, it is recommended to leave the
> retries config unset, as it will be defaulted to Integer.MAX_VALUE.
> Additionally, if a send(ProducerRecord) returns an error even with
> infinite retries (for instance if the message expires in the buffer
> before being sent), then it is recommended to shut down the producer and
> check the contents of the last produced message to ensure that it is not
> duplicated. Finally, the producer can only guarantee idempotence for
> messages sent within a single session.”
> 
>  
> 
> Thanks
> 
>  
> 
> https://www.williamhillplc.com/content/signature/WHlogo.gif?width=180
> <http://www.williamhill.com/>
> 
> https://www.williamhillplc.com/content/signature/senet.gif?width=180
> <http://www.whenthefunstops.co.uk/>
> 
> 			
> 
> *Jose Manuel Vega Monroy **
> **Java Developer / Software Developer Engineer in Test*
> 
> Direct: +*0035 0 2008038 (Ext. 8038)*
> Email: jose.monroy@williamhill.com <ma...@williamhill.com>
> 
> William Hill | 6/1 Waterport Place | Gibraltar | GX11 1AA
> 
>  
> 
>  
> 
>  
>