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

Guozhang Wang commented on KAFKA-10635:
---------------------------------------

>From the logs, I think the OOOSException was thrown here: 
>https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/log/ProducerStateManager.scala#L236.
> because `currentLastSeq` is -1 (i.e. UNKNOWN). It usually indicates that due 
>to a log truncation (which did happen before the exception thrown), the 
>producer's state has all been deleted, while in that case 
>currentEntry.producerEpoch == RecordBatch.NO_PRODUCER_EPOCH should be 
>satisfied — however it does not. And here's my suspected route that lead this:

 

T1: The partition starts with replicas 3,4,5, with 5 as the leader; producers 
are still writing to 5.

T2: Assume there's a producer with id 61009, start writing to leader 5, the 
first append is at an offset larger than offset 853773. NOTE that at this time 
that append has not bee fully replicated across, and hence high watermark has 
not been advanced.

T3: Replica 10 is added to the replica list and old leader 5 is removed. 
Replica 10 truncates itself till 853773, and then rebuild its producer state up 
to offset 853773 as well (you can see that from the log). Note that since 
producer 61009's append record is beyond 8553733, it's not yet contained in the 
persistent producer snapshot and hence not loaded into the new leader 10's 
in-memory producer states.

T4: There's a truncation happened: it seems be deleting an empty log segment 
though, since the log segment is (baseOffset=0, size=0), but that should not 
have any impact on the producer state since deleting files does not immediately 
affect the in-memory producer entries.

T5: Producer 61009 finally learned about the new leader, and start sending to 
it. It's append start offset is 853836 (larger than 853773), the producer 
entry's metadata queue is empty, HOWEVER its epoch is somehow not -1 (UNKNOWN), 
i.e. in the old snapshot it does not have any metadata but has an existing 
epoch. And hence this exception is triggered. Unfortunately since we do not 
have enough log info (I can file a quick PR to enhance it in the future 
releases) so I cannot be certain why that snapshot contains no metadata but a 
non -1 epoch... would like to hear some expert's opinion, [~hachikuji] 
[~junrao] .

> Streams application fails with OutOfOrderSequenceException after rolling 
> restarts of brokers
> --------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-10635
>                 URL: https://issues.apache.org/jira/browse/KAFKA-10635
>             Project: Kafka
>          Issue Type: Bug
>          Components: core, producer 
>    Affects Versions: 2.5.1
>            Reporter: Peeraya Maetasatidsuk
>            Priority: Blocker
>         Attachments: logs.csv
>
>
> We are upgrading our brokers to version 2.5.1 (from 2.3.1) by performing a 
> rolling restart of the brokers after installing the new version. After the 
> restarts we notice one of our streams app (client version 2.4.1) fails with 
> OutOfOrderSequenceException:
>  
> {code:java}
> ERROR [2020-10-13 22:52:21,400] [com.aaa.bbb.ExceptionHandler] Unexpected 
> error. Record: a_record, destination topic: 
> topic-name-Aggregation-repartition 
> org.apache.kafka.common.errors.OutOfOrderSequenceException: The broker 
> received an out of order sequence number.
> ERROR [2020-10-13 22:52:21,413] 
> [org.apache.kafka.streams.processor.internals.AssignedTasks] stream-thread 
> [topic-name-StreamThread-1] Failed to commit stream task 1_39 due to the 
> following error: org.apache.kafka.streams.errors.StreamsException: task 
> [1_39] Abort sending since an error caught with a previous record (timestamp 
> 1602654659000) to topic topic-name-Aggregation-repartition due to 
> org.apache.kafka.common.errors.OutOfOrderSequenceException: The broker 
> received an out of order sequence number.        at 
> org.apache.kafka.streams.processor.internals.RecordCollectorImpl.recordSendError(RecordCollectorImpl.java:144)
>         at 
> org.apache.kafka.streams.processor.internals.RecordCollectorImpl.access$500(RecordCollectorImpl.java:52)
>         at 
> org.apache.kafka.streams.processor.internals.RecordCollectorImpl$1.onCompletion(RecordCollectorImpl.java:204)
>         at 
> org.apache.kafka.clients.producer.KafkaProducer$InterceptorCallback.onCompletion(KafkaProducer.java:1348)
>         at 
> org.apache.kafka.clients.producer.internals.ProducerBatch.completeFutureAndFireCallbacks(ProducerBatch.java:230)
>         at 
> org.apache.kafka.clients.producer.internals.ProducerBatch.done(ProducerBatch.java:196)
>         at 
> org.apache.kafka.clients.producer.internals.Sender.failBatch(Sender.java:730) 
>        at 
> org.apache.kafka.clients.producer.internals.Sender.failBatch(Sender.java:716) 
>        at 
> org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:674)
>         at 
> org.apache.kafka.clients.producer.internals.Sender.handleProduceResponse(Sender.java:596)
>         at 
> org.apache.kafka.clients.producer.internals.Sender.access$100(Sender.java:74) 
>        at 
> org.apache.kafka.clients.producer.internals.Sender$1.onComplete(Sender.java:798)
>         at 
> org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)   
>      at 
> org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:569)
>         at 
> org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:561)        at 
> org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:335)   
>      at 
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:244)       
>  at java.base/java.lang.Thread.run(Thread.java:834)Caused by: 
> org.apache.kafka.common.errors.OutOfOrderSequenceException: The broker 
> received an out of order sequence number.
> {code}
> We see a corresponding error on the broker side:
> {code:java}
> [2020-10-13 22:52:21,398] ERROR [ReplicaManager broker=137636348] Error 
> processing append operation on partition 
> topic-name-Aggregation-repartition-52  
> (kafka.server.ReplicaManager)org.apache.kafka.common.errors.OutOfOrderSequenceException:
>  Out of order sequence number for producerId 2819098 at offset 1156041 in 
> partition topic-name-Aggregation-repartition-52: 29 (incoming seq. number), 
> -1 (current end sequence number)
> {code}
> We are able to reproduce this many times and it happens regardless of whether 
> the broker shutdown (at restart) is clean or unclean. However, when we 
> rollback the broker version to 2.3.1 from 2.5.1 and perform similar rolling 
> restarts, we don't see this error on the streams application at all. This is 
> blocking us from upgrading our broker version. 
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to