Hi everyone, we recently tried to upgrade from Kafka 1.1 to Kafka 2.7.1 (5 Kafka nodes, 3 zookeeper nodes), where we encountered a corruption issue detailed below. I should first note that: - Originally this was a Kafka 0.9.0.1 server that was upgraded to Kafka 1.1 - The Kafka clients that we use differ depending on each application. Essentially we have three categories; one portion of applications use the old Kafka 0.9 library, the second uses the Samara library in Google Go, and the third uses the Spring-Kafka integration module with a Kafka 2.4 library. - We have set the __consumer_offsets topic to use compaction instead of deletion. - We restart the Kafka nodes without paying attention to which one is the controller at the time. We later read that it is good practice to restart the controller as the last node. - We did not upgrade the Zookeeper cluster, but left it at version 3.4.6.
During the first rolling restart (where we upgraded to Kafka 2.7.1 but left the inter.broker.protocol.version and log.message.format.version to 1.1), we encountered an issue with under-replicated partitions when upgrading the second node. After the second node was started, the under-replicated partitions were in __consumer_offsets, as well as topics where the applications used the Kafka 2.4 library. We waited for almost an hour, but their number was not reduced. We tried to roll back, but the issue persisted, so we decided to fully upgrade, knowing that we would have a potential data loss. As we understand, during the upgrade Kafka has written multiple messages using the same offset, which makes the partition corrupt and therefore not replicated to the other nodes, as well as causing issues with the log cleaner for consumer offsets. Some of the related log messages are attached at the end. After the upgrade was finished, we were still encountering an issue, where the controller get somewhat stuck (solved by restarting the Zookeeper leader and the controller), which resulted in further data loss: ERROR [ReplicaManager broker=2] Error processing append operation on partition __consumer_offsets-10 (kafka.server.ReplicaManager) org.apache.kafka.common.errors.InvalidOffsetException: Attempt to append an offset (3802793509) to position 8058 no larger than the last offset appended (3802793509) to /data/kafka/__consumer_offsets-10/00000000003802612690.index. Prior to the upgrade, in our testing environment we had not encountered this initially. However, when we restarted the server a week after, we noticed that two __consumer_offsets partitions became under-replicated as well. Due to the retention period in Kafka, eventually the under-replication issue went away, as well as the controller issue. However, we have been unable to reproduce the original problem, by setting up a new Kafka 1.1 test server and upgrading it. Even if we SIGKILL nodes or if we put more traffic that they can handle, resulting in SIGSEGV shutdowns. Has anyone encountered the same issue during the upgrade, or can point us as to what the root cause of all this may be? Are there any precautions we should take prior to a Kafka upgrade that are not documented? Below is a sample of the log messages we were seeing about the under-replicated partitions: ERROR [ReplicaFetcher replicaId=3, leaderId=2, fetcherId=0] Unexpected error occurred while processing data for partition __consumer_offsets-34 at offset 5626 (kafka.server.ReplicaFetcherThread) kafka.common.OffsetsOutOfOrderException: Out of order offsets found in append to __consumer_offsets-34: List(5626, 5627, 5628, 5629, 5629, 5629, 5629, 5629, 5629) WARN [ReplicaFetcher replicaId=3, leaderId=2, fetcherId=0] Partition __consumer_offsets-34 marked as failed (kafka.server.ReplicaFetcherThread) WARN [Log partition=api.log.events.xxx-3, dir=/data/kafka] Found invalid offset during recovery. Deleting the corrupt segment and creating an empty one with starting offset 1355 (kafka.log.Log) WARN [Log partition=api.log.events.xxx-3, dir=/data/kafka] Corruption found in segment 1355, truncating to offset 1355 (kafka.log.Log) WARN [Log partition=api.log.events.yyy, dir=/data/kafka] Found invalid offset during recovery. Deleting the corrupt segment and creating an empty one with starting offset 1257818 (kafka.log.Log) WARN [Log partition=api.log.events.yyy-1, dir=/data/kafka] Corruption found in segment 1257818, truncating to offset 1257818 (kafka.log.Log) ERROR [ReplicaManager broker=1] Error processing append operation on partition __consumer_offsets-19 (kafka.server.ReplicaManager) kafka.common.LogSegmentOffsetOverflowException: Detected offset overflow at offset 2536087823 in segment LogSegment(baseOffset=2536087824, size=0, lastModifiedTime=1623162008000, largestRecordTimestamp=None) WARN [kafka-log-cleaner-thread-0]: Unexpected exception thrown when cleaning log Log(dir=/data/kafka/__consumer_offsets-44, topic=__consumer_offsets, partition=44, highWatermark=38389586, lastStableOffset=38389586, logStartOffset=0, logEndOffset=38389586). Marking its partition (__consumer_offsets-44) as uncleanable (kafka.log.LogCleaner) kafka@kafka02:~$ /opt/kafka/bin/kafka-run-class.sh kafka.tools.DumpLogSegments --files /data/kafka/__consumer_offsets-43/00000000000000000000.index Dumping /data/kafka/__consumer_offsets-43/00000000000000000000.index offset: 6931 position: 5148 Mismatches in :/data/kafka/__consumer_offsets-43/00000000000000000000.index Index offset: 6931, log offset: 6928 Thanks in advance, Dimitris -- <https://www.upstreamsystems.com/the-road-to-digital/?utm_source=Email&utm_medium=SignatureBanner&utm_campaign=Road%20To%20Digital> <https://www.secure-d.io/mobileadfraud2021report/?utm_source=Email&utm_medium=Signature_Banner&utm_campaign=A_Pandemic_on_Mobile> PRIVILEGED AND CONFIDENTIAL COMMUNICATION This e-mail transmission, and any documents, files or previous e-mail messages attached to it, may contain confidential information that is legally privileged. If you are not the intended recipient or a person responsible for delivering it to the intended recipient, you are hereby notified that any disclosure, copying, distribution or use of any of the information contained in or attached to this transmission is STRICTLY PROHIBITED. If you have received this transmission in error, please: (1) immediately notify me by reply e-mail, or by collect telephone call; and (2) destroy the original transmission and its attachments without reading or saving in any manner.