[jira] [Commented] (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=17609798#comment-17609798 ] Guozhang Wang commented on KAFKA-10635: --- HI [~nicktelford] just making sure you get the latest message and if you could reproduce the issue with the PR applied to improve logging information. > 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)
[jira] [Commented] (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=17607329#comment-17607329 ] Guozhang Wang commented on KAFKA-10635: --- Could you try out this patch https://github.com/apache/kafka/pull/12667 and reproduce the issue? > 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)
[jira] [Commented] (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=17607083#comment-17607083 ] Nicholas Telford commented on KAFKA-10635: -- [~guozhang] is there any more information I can provide that might help zero in on this issue? If you have a PR that adds some additional logging, I could potentially patch it in to my Kafka brokers if you're having trouble replicating this in development environments? > 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
[jira] [Commented] (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=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.failBat
[jira] [Commented] (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=17604862#comment-17604862 ] Nicholas Telford commented on KAFKA-10635: -- Hi [~guozhang], I've managed to pull some logs from a recent occurrence of this issue. I specifically focused the logs on the partition and broker that produces the error, otherwise there would be thousands of irrelevant log messages. I've also replaced the name of the partitions in question with placeholder names ({{myapp}} and {{some-processor}}), to prevent leaking confidential information. We use a structured logging system, so I've converted the logs to CSV. I hope you find this format easy to understand. If you feel there's information missing that would help (e.g. logger name, a broader search on the logs, etc.) then let me know, and I'll see what I can do. See attached [^logs.csv] > 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-repar
[jira] [Commented] (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=17604838#comment-17604838 ] Guozhang Wang commented on KAFKA-10635: --- Hmm okay I think we'd need to reproduce this which can help getting a better trace on the broker side. cc [~hachikuji] At the mean time, do you happen to still have the broker-side logs on the OOOSException thrown, if yes could you share in the comments? > 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] [Commented] (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=17604021#comment-17604021 ] Nicholas Telford commented on KAFKA-10635: -- [~guozhang] both my Kafka brokers and all clients run 3.2.0. The original issue reporter is running 2.5.1 > 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)
[jira] [Commented] (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=17603789#comment-17603789 ] Guozhang Wang commented on KAFKA-10635: --- Hi [~nicktelford], regarding the broker's behavior: > Irrespective of the behaviour on the Streams side, I'm confident that the > real issue is that brokers should not be producing an > OutOfOrderSequenceException just because partition leadership changed while a > producer was writing to that partition. As I mentioned in my earlier comment, > I believe this is caused by the producerEpoch not being properly tracked when > partition leadership changes. I think it is related to KIP-360 but when I looked through the history, I cannot find obvious relevance to what you've observed and on the current trunk the behavior seems not as what you observed either. Would you mind upgrading to a newer version of broker than 2.5.1, e.g. 3.0+, and see if this issue still preserves? > 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 >
[jira] [Commented] (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=17602212#comment-17602212 ] Nicholas Telford commented on KAFKA-10635: -- Hi [~guozhang], thanks for the explanation. I should clarify that we don't see our app instances crash, we see the Task migrations. However, for us, a Task migration is still extremely bad, as it causes some very large state stores to have to be recovered. While we can improve this when KIP-844 lands, it's still not ideal. Irrespective of the behaviour on the Streams side, I'm confident that the _real_ issue is that brokers should not be producing an {{OutOfOrderSequenceException}} just because partition leadership changed while a producer was writing to that partition. As I mentioned in [my earlier comment|#comment-17580718], I believe this is caused by the {{producerEpoch}} not being properly tracked when partition leadership changes. > 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-Agg
[jira] [Commented] (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=17602131#comment-17602131 ] Guozhang Wang commented on KAFKA-10635: --- Hello [~nicktelford], thanks for the updates! I've summarized the EOS related exception and its handling logic in KIP-691 (the section starting with "As of 08/16/2022"). As you can see OutOfSequenceException is an abortable exception not a fatal one. Kafka Streams relying on Kafka Producer would handle this exception as a TaskMigratedException, note that the latter is an internal exception that would be handled without failing the Kafka Streams app, i.e. the second stack trace should not be killing the app. The first stack trace, though, in version 2.5, would be killing the app. The difference between these two is that the first stack trace was throw when committing streams task, while the second was thrown when the task was being processed normally while trying to send a record. I checked the source code and have confirmed that this issue has been resolved in trunk, i.e. we would always throw it as TaskMigrated and handle it internally rather than failing the app. So I'd suggest you upgrading your application beyond 2.5. > 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=137636
[jira] [Commented] (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=17585595#comment-17585595 ] Nicholas Telford commented on KAFKA-10635: -- Hi [~guozhang], there's not really any trace on the client-side, because the {{OutOfOrderSequenceException}} is thrown on the broker, and propagated to the client via the network protocol, so at the point it's thrown at the client, it's literally just deserializing the error from the broker. Consequently, there is no client-side stacktrace. There is a stacktrace for the {{TaskMigratedException}} that wraps the {{OutOfOrderSequenceException}}, although I don't think it's particularly useful: {{ org.apache.kafka.streams.errors.TaskMigratedException: Error encountered sending record to topic foo-bar-repartition for task 17_17 due to: org.apache.kafka.common.errors.OutOfOrderSequenceException: The broker received an out of order sequence number. Written offsets would not be recorded and no more records would be sent since the producer is fenced, indicating the task may be migrated out; it means all tasks belonging to this thread should be migrated. at org.apache.kafka.streams.processor.internals.RecordCollectorImpl.recordSendError(RecordCollectorImpl.java:215) at org.apache.kafka.streams.processor.internals.RecordCollectorImpl.lambda$send$0(RecordCollectorImpl.java:196) at org.apache.kafka.clients.producer.KafkaProducer$InterceptorCallback.onCompletion(KafkaProducer.java:1418) at org.apache.kafka.clients.producer.internals.ProducerBatch.completeFutureAndFireCallbacks(ProducerBatch.java:273) at org.apache.kafka.clients.producer.internals.ProducerBatch.done(ProducerBatch.java:234) at org.apache.kafka.clients.producer.internals.ProducerBatch.completeExceptionally(ProducerBatch.java:198) at org.apache.kafka.clients.producer.internals.Sender.failBatch(Sender.java:758) at org.apache.kafka.clients.producer.internals.Sender.failBatch(Sender.java:743) at org.apache.kafka.clients.producer.internals.Sender.failBatch(Sender.java:695) at org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:634) at org.apache.kafka.clients.producer.internals.Sender.lambda$null$1(Sender.java:575) at java.base/java.util.ArrayList.forEach(ArrayList.java:1511) at org.apache.kafka.clients.producer.internals.Sender.lambda$handleProduceResponse$2(Sender.java:562) at java.base/java.lang.Iterable.forEach(Iterable.java:75) at org.apache.kafka.clients.producer.internals.Sender.handleProduceResponse(Sender.java:562) at org.apache.kafka.clients.producer.internals.Sender.lambda$sendProduceRequest$5(Sender.java:836) at org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109) at org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:583) at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:575) at org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:328) at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:243) at java.base/java.lang.Thread.run(Thread.java:833) Caused by: org.apache.kafka.common.errors.OutOfOrderSequenceException: The broker received an out of order sequence number.}} This is why I was looking for a trace on the brokers, which I sadly have not yet been able to produce. I think I've fixed my broker logging now, so I'll try to re-create the issue and generate a stack-trace on the broker side when I have some time. > 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
[jira] [Commented] (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=17581943#comment-17581943 ] Guozhang Wang commented on KAFKA-10635: --- Thanks [~nicktelford], just to clarify what I asked for is the logs stack trace on the client (Kafka Streams) side, not the broker side. > 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)
[jira] [Commented] (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=17581894#comment-17581894 ] Nicholas Telford commented on KAFKA-10635: -- Hi [~guozhang], unfortunately what I posted above is all I see on the brokers. We're using the log4j JSONEventLayoutV1 to render our log messages for Logstash, but for some reason it's not including the full stack trace. I'm going to see if I can figure out what the problem is there, and if possible, get you a stack-trace. In the mean-time, this should be trivial to reproduce if you have an environment that correctly logs stack-traces: run an EOS Kafka Streams app and, while processing, execute a partition leadership election on the Kafka cluster. > 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 shutd
[jira] [Commented] (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=17581574#comment-17581574 ] Guozhang Wang commented on KAFKA-10635: --- Hello [~nicktelford] Could you also paste the full stack trace of the OutOfEquenceException? Since in 3.2 a lot of code has been changed (e.g. there's no `AssignedTasks` anymore), and from a quick peek in the source code I think this exception should have been handled internally without killing the app. So a full stack trace on the thrown exception would be very helpful. > 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 app
[jira] [Commented] (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 commented on KAFKA-10635: -- 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
[jira] [Commented] (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=17497214#comment-17497214 ] Matthias J. Sax commented on KAFKA-10635: - Is this fixed via https://issues.apache.org/jira/browse/KAFKA-9552 ? > 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.1#820001)
[jira] [Commented] (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=17225752#comment-17225752 ] Bob Barrett commented on KAFKA-10635: - [~ascii80] would you be able to upload more client and broker logs from an occurrence of this issue? If you are able to enable DEBUG logging for `org.apache.kafka.clients.producer.internals` on the client side and `kafka.coordinator.transaction` on the broker side, that would be even better. > 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. > --
[jira] [Commented] (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=17223324#comment-17223324 ] Matthias J. Sax commented on KAFKA-10635: - \cc [~hachikuji] [~bob-barrett] > 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.3.4#803005)