You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Nicholas Telford (Jira)" <ji...@apache.org> on 2022/08/17 10:34:00 UTC

[jira] [Comment Edited] (KAFKA-10635) Streams application fails with OutOfOrderSequenceException after rolling restarts of brokers

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

Nicholas Telford edited comment on KAFKA-10635 at 8/17/22 10:33 AM:
--------------------------------------------------------------------

We're seeing the same issue with Kafka Streams 3.2.0 and Kafka Broker 3.2.0

It's specifically triggered by *change in partition leadership* on the broker, rather than a rolling restart (which triggers repeated leadership elections as brokers leave the ISR).

The smoking gun appears to be this log message (emphasis mine):
{quote}org.apache.kafka.common.errors.OutOfOrderSequenceException: Out of order sequence number for producer 46002 at offset 4796894 in partition myapp-mytopic-repartition-17: 1262 (incoming seq. number), *-1 (current end sequence number)*
{quote}
A current sequence number of "-1" is [actually a placeholder for RecordBatch.NO_SEQUENCE|#L232],] which implies that {{{}producerEpoch != currentEntry.producerEpoch{}}}, although it's not clear to me why that would be the case when partition leadership changes.

My guess is that sometime between Kafka 2.3.1 and 2.5.1, something changed in the way brokers handle Partition epochs when leadership changes.

Further details:
 * The client-side OutOfOrderSequenceException is always preceded by several {{NOT_LEADER_OR_FOLLOWER}} errors. KIP-360, adopted in 2.5.0, specifically talks about bumping the producer epoch on "recoverable errors", and although it doesn't specifically mention {{NOT_LEADER_OR_FOLLOWER}} errors, it looks a lot to me like KIP-360 might be the change that caused this bug.


was (Author: nicktelford):
We're seeing the same issue with Kafka Streams 3.2.0 and Kafka Broker 3.2.0

It's specifically triggered by *change in partition leadership* on the broker, rather than a rolling restart (which triggers repeated leadership elections as brokers leave the ISR).

The smoking gun appears to be this log message (emphasis mine):
{quote}org.apache.kafka.common.errors.OutOfOrderSequenceException: Out of order sequence number for producer 46002 at offset 4796894 in partition myapp-mytopic-repartition-17: 1262 (incoming seq. number), *-1 (current end sequence number)*
{quote}
A current sequence number of "-1" is [actually a placeholder for RecordBatch.NO_SEQUENCE|[https://github.com/apache/kafka/blob/3.2.0/core/src/main/scala/kafka/log/ProducerStateManager.scala#L232],] which implies that {{{}producerEpoch == currentEntry.producerEpoch{}}}, although it's not clear to me why that would be the case when partition leadership changes.

My guess is that sometime between Kafka 2.3.1 and 2.5.1, something changed in the way brokers handle Partition epochs when leadership changes.

Further details:
 * The client-side OutOfOrderSequenceException is always preceded by several {{NOT_LEADER_OR_FOLLOWER}} errors. KIP-360, adopted in 2.5.0, specifically talks about bumping the producer epoch on "recoverable errors", and although it doesn't specifically mention {{NOT_LEADER_OR_FOLLOWER}} errors, it looks a lot to me like KIP-360 might be the change that caused this bug.

> 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
>
> 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)