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

Graeme McRobert commented on KAFKA-13077:
-----------------------------------------

The problem has nothing to do with Zookeeper. The problem is actually in the 
implementation of the ISR protocol and a possible corruption that can occur in 
the leader-epoch-checkpoint file on the lead broker. The messages indicate a 
loop on the followers and the loop centers around two consecutive fetch 
requests when a follower is restarting and is trying to catch up to the leader. 
The first fetch request uses the correct fetchOffset based on the loaded log 
end offset and lastFetchedEpoch which is stored in the partitionLeaderEpoch 
field of the last message on the log. The fetch returns a batch of records 
which are written to the local log. The second fetch then uses the correct 
fetchOffset from the new logEndOffset and the lastFetchedEpoch from the last 
record in the batch. That lastFetchedEpoch does not exist in the leader's 
leader-epoch-checkpoint file cache so the leader searches backward to the 
nearest epoch that does exist in the cache which results in a diverging epoch 
being returned. When the response comes back to the follower, the broker 
truncates to the logEndOffset as it was after the original load of the log and 
sends the next fetch which matches the first fetch and that is the loop. This 
problem only seems to affect topics that use a retention policy of compact.

 

Below shows an example based on an actual production outage that I recreated by 
copying the leader's topic-partition directory.

The follower directory contains the following files:
{code:java}
$ ls -l
total 64
-rw-r--r--@ 1 mcrobert  wheel    0  9 Jun 16:40 00000000000000000000.index
-rw-r--r--@ 1 mcrobert  wheel  332  9 Jun 16:40 00000000000000000000.log
-rw-r--r--@ 1 mcrobert  wheel   12  9 Jun 16:40 00000000000000000000.timeindex
-rw-r--r--@ 1 mcrobert  wheel    0  9 Jun 17:06 00000000000000028511.index
-rw-r--r--@ 1 mcrobert  wheel  214  9 Jun 16:40 00000000000000028511.log
-rw-r--r--@ 1 mcrobert  wheel   10  9 Jun 16:40 00000000000000028511.snapshot
-rw-r--r--@ 1 mcrobert  wheel   12  9 Jun 17:06 00000000000000028511.timeindex
-rw-r--r--  1 mcrobert  wheel    0 10 Jun 10:11 00000000000000028513.index
-rw-r--r--  1 mcrobert  wheel    0 10 Jun 10:11 00000000000000028513.log
-rw-r--r--  1 mcrobert  wheel   10  9 Jun 16:47 00000000000000028513.snapshot
-rw-r--r--  1 mcrobert  wheel    0 10 Jun 10:11 00000000000000028513.timeindex
-rw-r--r--  1 mcrobert  wheel   25 10 Jun 10:11 leader-epoch-checkpoint
-rw-r--r--@ 1 mcrobert  wheel   43  9 Jun 16:42 partition.metadata
{code}
The leader-epoch-checkpoint file on the follower contains the following entries:
{code:java}
$ cat leader-epoch-checkpoint
0
3
16 0
17 292
32 28511
{code}
The leader directory contains the following files:
{code:java}
$ ls -l
total 34792
-rw-r--r--@ 1 mcrobert  wheel         0  9 Jun 16:41 00000000000000000000.index
-rw-r--r--@ 1 mcrobert  wheel       332  9 Jun 16:41 00000000000000000000.log
-rw-r--r--@ 1 mcrobert  wheel        12  9 Jun 16:41 
00000000000000000000.timeindex
-rw-r--r--@ 1 mcrobert  wheel         0  9 Jun 16:41 00000000000000028511.index
-rw-r--r--@ 1 mcrobert  wheel       214  9 Jun 16:41 00000000000000028511.log
-rw-r--r--@ 1 mcrobert  wheel        10  9 Jun 16:41 
00000000000000028511.snapshot
-rw-r--r--@ 1 mcrobert  wheel        12  9 Jun 16:41 
00000000000000028511.timeindex
-rw-r--r--@ 1 mcrobert  wheel     34200 10 Jun 10:12 00000000000000028513.index
-rw-r--r--@ 1 mcrobert  wheel  17684389  9 Jun 16:41 00000000000000028513.log
-rw-r--r--@ 1 mcrobert  wheel        10  9 Jun 16:41 
00000000000000028513.snapshot
-rw-r--r--@ 1 mcrobert  wheel     51312 10 Jun 10:12 
00000000000000028513.timeindex
-rw-r--r--  1 mcrobert  wheel        10  9 Jun 16:45 
00000000000000122577.snapshot
-rw-r--r--  1 mcrobert  wheel        36 10 Jun 10:11 leader-epoch-checkpoint
-rw-r--r--@ 1 mcrobert  wheel        43  9 Jun 16:42 partition.metadata{code}
The leader-epoch-checkpoint file on the leader contains the following entries:
{code:java}
$ cat leader-epoch-checkpoint
0
4
16 0
17 292
32 28511
207 122577{code}
The content of the 00000000000000028511.log file on both the leader and 
follower is:
{code:java}
Dumping 00000000000000028511.log
Log starting offset: 28511
baseOffset: 28511 lastOffset: 28512 count: 2 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 32 isTransactional: 
false isControl: false deleteHorizonMs: OptionalLong.empty position: 0 
CreateTime: 1728391203673 size: 214 magic: 2 compresscodec: none crc: 
3565384616 isvalid: true{code}
The broker startup loads the partition and correctly sets the logEndOffset 
value to 28513.
{code:java}
[2025-06-10 09:44:48,403] INFO Completed load of 
Log(dir=/tmp/kafka-logs0/__consumer_offsets-45, topicId=y6f7Vq_tQCS58353vVjimQ, 
topic=__consumer_offsets, partition=45, highWatermark=0, lastStableOffset=0, 
logStartOffset=0, logEndOffset=28513) with 3 segments, local-log-start-offset 0 
and log-end-offset 28513 in 13ms (39/54 completed in /tmp/kafka-logs0) 
(kafka.log.LogManager){code}
The first fetch request correctly requests fetchOffset=28513 and 
lastFetchedEpoch=Optional[32]:
{code:java}
[2025-06-10 09:50:32,512] TRACE [ReplicaFetcher replicaId=0, leaderId=1, 
fetcherId=0] Sending fetch request (type=FetchRequest, replicaId=0, 
maxWait=500, minBytes=1, maxBytes=10485760, 
fetchData={__consumer_offsets-45=PartitionData(topicId=y6f7Vq_tQCS58353vVjimQ, 
fetchOffset=28513, logStartOffset=0, maxBytes=1048576, 
currentLeaderEpoch=Optional[207], lastFetchedEpoch=Optional[32])}, 
isolationLevel=read_uncommitted, removed=, replaced=, 
metadata=(sessionId=90347455, epoch=14), rackId=) 
(kafka.server.ReplicaFetcherThread){code}
The leader responds with 1048576 bytes:
{code:java}
[2025-06-10 09:50:32,517] DEBUG Completed 
request:{"isForwarded":false,"requestHeader":{"requestApiKey":1,"requestApiVersion":17,"correlationId":14,"clientId":"broker-0-fetcher-0","requestApiKeyName":"FETCH"},"request":{"replicaState":{"replicaId":0,"replicaEpoch":25769987115},"maxWaitMs":500,"minBytes":1,"maxBytes":10485760,"isolationLevel":0,"sessionId":90347455,"sessionEpoch":14,"topics":[{"topicId":"y6f7Vq_tQCS58353vVjimQ","partitions":[{"partition":45,"currentLeaderEpoch":207,"fetchOffset":28513,"lastFetchedEpoch":32,"logStartOffset":0,"partitionMaxBytes":1048576}]}],"forgottenTopicsData":[],"rackId":""},"response":{"throttleTimeMs":0,"errorCode":0,"sessionId":90347455,"responses":[{"topicId":"y6f7Vq_tQCS58353vVjimQ","partitions":[{"partitionIndex":45,"errorCode":0,"highWatermark":0,"lastStableOffset":0,"logStartOffset":0,"abortedTransactions":null,"preferredReadReplica":-1,"recordsSizeInBytes":1048576}]}]},"connection":"127.0.0.1:9093-127.0.0.1:50383-0","totalTimeMs":4.76,"requestQueueTimeMs":0.083,"localTimeMs":2.122,"remoteTimeMs":0.11,"throttleTimeMs":0,"responseQueueTimeMs":0.045,"sendTimeMs":2.398,"securityProtocol":"PLAINTEXT","principal":"User:ANONYMOUS","listener":"PLAINTEXT","clientInformation":{"softwareName":"unknown","softwareVersion":"unknown"}}
 (kafka.request.logger){code}
The follower advances the log by 1048576 bytes which takes it to offset 34088. 
The contents of the leader at that offset is:
{code:java}
baseOffset: 34087 lastOffset: 34087 count: 1 baseSequence: 0 lastSequence: 0 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 143 isTransactional: 
false isControl: false deleteHorizonMs: OptionalLong.empty position: 1048269 
CreateTime: 1733858878540 size: 188 magic: 2 compresscodec: none crc: 768502604 
isvalid: true
baseOffset: 34088 lastOffset: 34088 count: 1 baseSequence: 0 lastSequence: 0 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 143 isTransactional: 
false isControl: false deleteHorizonMs: OptionalLong.empty position: 1048457 
CreateTime: 1733858879540 size: 188 magic: 2 compresscodec: none crc: 
3180393501 isvalid: true
baseOffset: 34089 lastOffset: 34089 count: 1 baseSequence: 0 lastSequence: 0 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 143 isTransactional: 
false isControl: false deleteHorizonMs: OptionalLong.empty position: 1048645 
CreateTime: 1733858880540 size: 188 magic: 2 compresscodec: none crc: 822170493 
isvalid: true{code}
The next fetch request then requests from a fetchOffset=34088 and a 
lastFetchedEpoch=Optional[143] which it gets from the record at offset 34087 
(the last record in the batch from the previous fetch).
{code:java}
[2025-06-10 09:50:32,543] TRACE [ReplicaFetcher replicaId=0, leaderId=1, 
fetcherId=0] Sending fetch request (type=FetchRequest, replicaId=0, 
maxWait=500, minBytes=1, maxBytes=10485760, 
fetchData={__consumer_offsets-45=PartitionData(topicId=y6f7Vq_tQCS58353vVjimQ, 
fetchOffset=34088, logStartOffset=0, maxBytes=1048576, 
currentLeaderEpoch=Optional[207], lastFetchedEpoch=Optional[143])}, 
isolationLevel=read_uncommitted, removed=, replaced=, 
metadata=(sessionId=90347455, epoch=15), rackId=) 
(kafka.server.ReplicaFetcherThread){code}
The leader cannot find an epoch 143 in the leader-epoch-checkpoint file so it 
works back to the next highest epoch which happens to be 32 so it responds with 
a diverged epoch response.
{code:java}
[2025-06-10 09:50:32,546] DEBUG Completed 
request:{"isForwarded":false,"requestHeader":{"requestApiKey":1,"requestApiVersion":17,"correlationId":15,"clientId":"broker-0-fetcher-0","requestApiKeyName":"FETCH"},"request":{"replicaState":{"replicaId":0,"replicaEpoch":25769987115},"maxWaitMs":500,"minBytes":1,"maxBytes":10485760,"isolationLevel":0,"sessionId":90347455,"sessionEpoch":15,"topics":[{"topicId":"y6f7Vq_tQCS58353vVjimQ","partitions":[{"partition":45,"currentLeaderEpoch":207,"fetchOffset":34088,"lastFetchedEpoch":143,"logStartOffset":0,"partitionMaxBytes":1048576}]}],"forgottenTopicsData":[],"rackId":""},"response":{"throttleTimeMs":0,"errorCode":0,"sessionId":90347455,"responses":[{"topicId":"y6f7Vq_tQCS58353vVjimQ","partitions":[{"partitionIndex":45,"errorCode":0,"highWatermark":0,"lastStableOffset":0,"logStartOffset":0,"divergingEpoch":{"epoch":32,"endOffset":122577},"abortedTransactions":null,"preferredReadReplica":-1,"recordsSizeInBytes":0}]}]},"connection":"127.0.0.1:9093-127.0.0.1:50383-0","totalTimeMs":2.385,"requestQueueTimeMs":0.15,"localTimeMs":1.935,"remoteTimeMs":0.137,"throttleTimeMs":0,"responseQueueTimeMs":0.044,"sendTimeMs":0.118,"securityProtocol":"PLAINTEXT","principal":"User:ANONYMOUS","listener":"PLAINTEXT","clientInformation":{"softwareName":"unknown","softwareVersion":"unknown"}}
 (kafka.request.logger){code}
The follower responds to the diverged Epoch by truncating the log thereby 
losing the 1048576 bytes it had just added.
{code:java}
[2025-06-10 09:50:32,546] INFO [ReplicaFetcher replicaId=0, leaderId=1, 
fetcherId=0] Truncating partition __consumer_offsets-45 with 
TruncationState(offset=28513, completed=true) due to leader epoch and offset 
EpochEndOffset(errorCode=0, partition=45, leaderEpoch=32, endOffset=122577) 
(kafka.server.ReplicaFetcherThread)
[2025-06-10 09:50:32,546] INFO [UnifiedLog partition=__consumer_offsets-45, 
dir=/tmp/kafka-logs0] Truncating to offset 28513 (kafka.log.UnifiedLog)
[2025-06-10 09:50:32,547] INFO [UnifiedLog partition=__consumer_offsets-45, 
dir=/tmp/kafka-logs0] Loading producer state till offset 28513 with message 
format version 2 (kafka.log.UnifiedLog$)
[2025-06-10 09:50:32,547] INFO [UnifiedLog partition=__consumer_offsets-45, 
dir=/tmp/kafka-logs0] Reloading from producer snapshot and rebuilding producer 
state from offset 28513 (kafka.log.UnifiedLog$)
[2025-06-10 09:50:32,547] INFO [ProducerStateManager 
partition=__consumer_offsets-45] Loading producer state from snapshot file 
'SnapshotFile(offset=28513, 
file=/tmp/kafka-logs0/__consumer_offsets-45/00000000000000028513.snapshot)' 
(org.apache.kafka.storage.internals.log.ProducerStateManager)
[2025-06-10 09:50:32,547] INFO [UnifiedLog partition=__consumer_offsets-45, 
dir=/tmp/kafka-logs0] Producer state recovery took 0ms for snapshot load and 
0ms for segment recovery from offset 28513 (kafka.log.UnifiedLog$){code}
The next fetch then repeats the cycle and the loop continues to eternity.

It is possible to 'repair' the leader-epoch-checkpoint file by dumping the last 
record of the first fetch and checking the partitionLeaderEpoch value of the 
last record on the returned records from the first fetch then adding an entry 
for that epoch into the leader-epoch-checkpoint file. To be technically 
correct, the entry should identify the start offset of that epoch but I believe 
just using the offset of the last fetched record should resolve the problem.

This reproduction was on a Kafka 3.9.1 system. I haven't had a chance to try 
and reproduce it on Kafka 4 yet so I don't know whether the [Eligible Leader 
Replicas|https://cwiki.apache.org/confluence/display/KAFKA/KIP-966%3A+Eligible+Leader+Replicas]
 kip and the changes it brings to the recovery mechanism will prevent this from 
happening. I think the reason we see this problem more frequently on Kubernetes 
is down to the aggressive default shutdown policy of Kubernetes which gives the 
container 30 seconds to terminate before issuing a SIGKILL to terminate the 
container.

> Replication failing after unclean shutdown of ZK and all brokers
> ----------------------------------------------------------------
>
>                 Key: KAFKA-13077
>                 URL: https://issues.apache.org/jira/browse/KAFKA-13077
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 2.8.0
>            Reporter: Christopher Auston
>            Priority: Minor
>
> I am submitting this in the spirit of what can go wrong when an operator 
> violates the constraints Kafka depends on. I don't know if Kafka could or 
> should handle this more gracefully. I decided to file this issue because it 
> was easy to get the problem I'm reporting with Kubernetes StatefulSets (STS). 
> By "easy" I mean that I did not go out of my way to corrupt anything, I just 
> was not careful when restarting ZK and brokers.
> I violated the constraints of keeping Zookeeper stable and at least one 
> running in-sync replica. 
> I am running the bitnami/kafka helm chart on Amazon EKS.
> {quote}% kubectl get po kaf-kafka-0 -ojson |jq .spec.containers'[].image'
> "docker.io/bitnami/kafka:2.8.0-debian-10-r43"
> {quote}
> I started with 3 ZK instances and 3 brokers (both STS). I changed the 
> cpu/memory requests on both STS and kubernetes proceeded to restart ZK and 
> kafka instances at the same time. If I recall correctly there were some 
> crashes and several restarts but eventually all the instances were running 
> again. It's possible all ZK nodes and all brokers were unavailable at various 
> points.
> The problem I noticed was that two of the brokers were just continually 
> spitting out messages like:
> {quote}% kubectl logs kaf-kafka-0 --tail 10
> [2021-07-13 14:26:08,871] INFO [ProducerStateManager 
> partition=__transaction_state-0] Loading producer state from snapshot file 
> 'SnapshotFile(/bitnami/kafka/data/__transaction_state-0/00000000000000000001.snapshot,1)'
>  (kafka.log.ProducerStateManager)
> [2021-07-13 14:26:08,871] WARN [Log partition=__transaction_state-0, 
> dir=/bitnami/kafka/data] *Non-monotonic update of high watermark from 
> (offset=2744 segment=[0:1048644]) to (offset=1 segment=[0:169])* 
> (kafka.log.Log)
> [2021-07-13 14:26:08,874] INFO [Log partition=__transaction_state-10, 
> dir=/bitnami/kafka/data] Truncating to offset 2 (kafka.log.Log)
> [2021-07-13 14:26:08,877] INFO [Log partition=__transaction_state-10, 
> dir=/bitnami/kafka/data] Loading producer state till offset 2 with message 
> format version 2 (kafka.log.Log)
> [2021-07-13 14:26:08,877] INFO [ProducerStateManager 
> partition=__transaction_state-10] Loading producer state from snapshot file 
> 'SnapshotFile(/bitnami/kafka/data/__transaction_state-10/00000000000000000002.snapshot,2)'
>  (kafka.log.ProducerStateManager)
> [2021-07-13 14:26:08,877] WARN [Log partition=__transaction_state-10, 
> dir=/bitnami/kafka/data] Non-monotonic update of high watermark from 
> (offset=2930 segment=[0:1048717]) to (offset=2 segment=[0:338]) 
> (kafka.log.Log)
> [2021-07-13 14:26:08,880] INFO [Log partition=__transaction_state-20, 
> dir=/bitnami/kafka/data] Truncating to offset 1 (kafka.log.Log)
> [2021-07-13 14:26:08,882] INFO [Log partition=__transaction_state-20, 
> dir=/bitnami/kafka/data] Loading producer state till offset 1 with message 
> format version 2 (kafka.log.Log)
> [2021-07-13 14:26:08,882] INFO [ProducerStateManager 
> partition=__transaction_state-20] Loading producer state from snapshot file 
> 'SnapshotFile(/bitnami/kafka/data/__transaction_state-20/00000000000000000001.snapshot,1)'
>  (kafka.log.ProducerStateManager)
> [2021-07-13 14:26:08,883] WARN [Log partition=__transaction_state-20, 
> dir=/bitnami/kafka/data] Non-monotonic update of high watermark from 
> (offset=2956 segment=[0:1048608]) to (offset=1 segment=[0:169]) 
> (kafka.log.Log)
> {quote}
> If I describe that topic I can see that several partitions have a leader of 2 
> and the ISR is just 2 (NOTE I added two more brokers and tried to reassign 
> the topic onto brokers 2,3,4 which you can see below). The new brokers also 
> spit out the messages about "non-monotonic update" just like the original 
> followers. This describe output is from the following day.
> {{% kafka-topics.sh ${=BS} -topic __transaction_state -describe}}
> {{Topic: __transaction_state TopicId: i7bBNCeuQMWl-ZMpzrnMAw PartitionCount: 
> 50 ReplicationFactor: 3 Configs: 
> compression.type=uncompressed,min.insync.replicas=3,cleanup.policy=compact,flush.ms=1000,segment.bytes=104857600,flush.messages=10000,max.message.bytes=1000012,unclean.leader.election.enable=false,retention.bytes=1073741824}}
> {{ Topic: __transaction_state Partition: 0 Leader: 2 Replicas: 4,3,2,1,0 Isr: 
> 2 Adding Replicas: 4,3 Removing Replicas: 1,0}}
> {{ Topic: __transaction_state Partition: 1 Leader: 2 Replicas: 2,4,3 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 2 Leader: 3 Replicas: 3,2,4 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 3 Leader: 4 Replicas: 4,2,3 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 4 Leader: 2 Replicas: 2,3,4 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 5 Leader: 2 Replicas: 3,4,2 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 6 Leader: 4 Replicas: 4,3,2 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 7 Leader: 2 Replicas: 2,4,3 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 8 Leader: 2 Replicas: 3,2,4,0,1 Isr: 
> 2 Adding Replicas: 3,4 Removing Replicas: 0,1}}
> {{ Topic: __transaction_state Partition: 9 Leader: 2 Replicas: 4,2,3,1,0 Isr: 
> 2 Adding Replicas: 4,3 Removing Replicas: 1,0}}
> {{ Topic: __transaction_state Partition: 10 Leader: 2 Replicas: 2,3,4,1,0 
> Isr: 2 Adding Replicas: 3,4 Removing Replicas: 1,0}}
> {{ Topic: __transaction_state Partition: 11 Leader: 3 Replicas: 3,4,2 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 12 Leader: 4 Replicas: 4,3,2 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 13 Leader: 2 Replicas: 2,4,3 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 14 Leader: 3 Replicas: 3,2,4 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 15 Leader: 4 Replicas: 4,2,3 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 16 Leader: 2 Replicas: 2,3,4 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 17 Leader: 2 Replicas: 3,4,2 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 18 Leader: 4 Replicas: 4,3,2 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 19 Leader: 2 Replicas: 2,4,3,0,1 
> Isr: 2 Adding Replicas: 4,3 Removing Replicas: 0,1}}
> {{ Topic: __transaction_state Partition: 20 Leader: 2 Replicas: 3,2,4,0,1 
> Isr: 2 Adding Replicas: 3,4 Removing Replicas: 0,1}}
> {{ Topic: __transaction_state Partition: 21 Leader: 2 Replicas: 4,2,3,1,0 
> Isr: 2 Adding Replicas: 4,3 Removing Replicas: 1,0}}
> {{ Topic: __transaction_state Partition: 22 Leader: 2 Replicas: 2,3,4,1,0 
> Isr: 2 Adding Replicas: 3,4 Removing Replicas: 1,0}}
> {{ Topic: __transaction_state Partition: 23 Leader: 3 Replicas: 3,4,2 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 24 Leader: 4 Replicas: 4,3,2 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 25 Leader: 2 Replicas: 2,4,3 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 26 Leader: 3 Replicas: 3,2,4 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 27 Leader: 4 Replicas: 4,2,3 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 28 Leader: 2 Replicas: 2,3,4 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 29 Leader: 3 Replicas: 3,4,2 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 30 Leader: 4 Replicas: 4,3,2 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 31 Leader: 2 Replicas: 2,4,3 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 32 Leader: 3 Replicas: 3,2,4 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 33 Leader: 4 Replicas: 4,2,3 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 34 Leader: 2 Replicas: 2,3,4 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 35 Leader: 3 Replicas: 3,4,2 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 36 Leader: 4 Replicas: 4,3,2 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 37 Leader: 2 Replicas: 2,4,3 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 38 Leader: 3 Replicas: 3,2,4 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 39 Leader: 4 Replicas: 4,2,3 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 40 Leader: 2 Replicas: 2,3,4 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 41 Leader: 3 Replicas: 3,4,2 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 42 Leader: 4 Replicas: 4,3,2 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 43 Leader: 2 Replicas: 2,4,3 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 44 Leader: 3 Replicas: 3,2,4 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 45 Leader: 4 Replicas: 4,2,3 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 46 Leader: 2 Replicas: 2,3,4 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 47 Leader: 3 Replicas: 3,4,2 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 48 Leader: 4 Replicas: 4,3,2 Isr: 
> 2,3,4}}
> {{ Topic: __transaction_state Partition: 49 Leader: 2 Replicas: 2,4,3 Isr: 
> 2,3,4}}
>  
> It seems something got corrupted and the followers will never make progress. 
> Even worse the original followers appear to have truncated their copies, so 
> if the remaining leader replica is what is corrupted then it may have 
> truncated replicas that had more valid data?
> Anyway, for what it's worth, this is something that happened to me. I plan to 
> change the statefulsets to require manual restarts so I can control rolling 
> upgrades. It also seems to underscore having a separate Kafka cluster for 
> disaster recovery.



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

Reply via email to