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

Konstantin commented on KAFKA-9895:
-----------------------------------

We bumped into this as well on kafka 2.4.1.

Here's the sequence of events, which I believe leads to this error:

1. Leader of partition test-3 (broker 2) stops due to graceful shutdown with hw 
502921625, but is has ongoing unreplicated transaction, starting from 
502921626, so it writes this transaction to producer snapshot.
{code:java}
[2021-04-11 23:27:44,313] INFO [ProducerStateManager partition=topic-3] Writing 
producer snapshot at offset 502921627 (kafka.log.ProducerStateManager){code}
 

2. Another broker (1) becomes leader and starts partition from hw 502921625
{code:java}
[2021-04-11 23:27:43,081] INFO [Partition topic-3 broker=1] topic-3 starts at 
leader epoch 30 from offset 502921625 with high watermark 502921625. Previous 
leader epoch was 29. (kafka.cluster.Partition){code}
 

3. Broker 2 starts again, loads that transaction's start offset as first 
unstable offset and tries to become follower. Then it tries to truncate to last 
replicated offset - 502921625.
{code:java}
[2021-04-11 23:28:31,447] INFO [ProducerStateManager partition=topic-3] Loading 
producer state from snapshot file 
'/KAFKADATA/topic-3/00000000000502921627.snapshot' 
(kafka.log.ProducerStateManager) 

[2021-04-11 23:28:31,454] INFO [Log partition=topic-3, dir=/KAFKADATA] 
Completed load of log with 25 segments, log start offset 496341849 and log end 
offset 502921627 in 29 ms (kafka.log.Log) 

[2021-04-11 23:28:33,776] INFO [Partition topic-3 broker=2] Log loaded for 
partition topic-3 with initial high watermark 502921625 
(kafka.cluster.Partition)

[2021-04-11 23:28:33,885] INFO [ReplicaFetcherManager on broker 2] Added 
fetcher to broker BrokerEndPoint(id=1, host=***:9093) for partitions 
Map(topic-3 -> (offset=502921625, leaderEpoch=30), ***) 
(kafka.server.ReplicaFetcherManager) 

[2021-04-11 23:28:34,002] INFO [Log partition=topic-3, dir=/KAFKADATA] 
Truncating to offset 502921625 (kafka.log.Log)
{code}
 

4. During truncation it updates logEndOffset, which leads to updating hw and 
incrementing first unstable offset here:
{code:java}
private def maybeIncrementFirstUnstableOffset(): Unit = lock synchronized {
  checkIfMemoryMappedBufferClosed()

  val updatedFirstStableOffset = producerStateManager.firstUnstableOffset match 
{
    case Some(logOffsetMetadata) if logOffsetMetadata.messageOffsetOnly || 
logOffsetMetadata.messageOffset < logStartOffset =>
      val offset = math.max(logOffsetMetadata.messageOffset, logStartOffset)
      Some(convertToOffsetMetadataOrThrow(offset))
    case other => other
  }

  if (updatedFirstStableOffset != this.firstUnstableOffsetMetadata) {
    debug(s"First unstable offset updated to $updatedFirstStableOffset")
    this.firstUnstableOffsetMetadata = updatedFirstStableOffset
  }
}
{code}
It finds producerStateManager.firstUnstableOffset (502921626, loaded from 
producer snapshot) and calls convertToOffsetMetadataOrThrow(), which tries to 
read the log at 502921626 and fails, because log end offset is already 
truncated to 502921625.
{code:java}
[2021-04-11 23:28:34,003] ERROR [ReplicaFetcher replicaId=2, leaderId=1, 
fetcherId=2] Unexpected error occurred during truncation for topic-3 at offset 
502921625 
(kafka.server.ReplicaFetcherThread)org.apache.kafka.common.errors.OffsetOutOfRangeException:
 Received request for offset 502921626 for partition topic-3, but we only have 
log segments in the range 496341849 to 502921625.
[2021-04-11 23:28:34,004] WARN [ReplicaFetcher replicaId=2, leaderId=1, 
fetcherId=2] Partition topic-3 marked as failed 
{code}
  

We didn't manage to reproduce this as is, but it can be emulated via writing 
transaction to only one replica, shutting it down and starting another one with 
unclean leader election. 

So, the steps to reproduce are:
 # Start 2 brokers, create a topic with 1 partition and replication factor of 
2. Let's say end offset is 5.
 # Stop broker 1
 # Write any record (to offset 5), then start transaction and write another one 
(to offset 6) - transaction start offset should be greater than end offset of 
1st broker's replica, or it won't be out of range.
 # Stop broker 2
 # Start broker 1 with unclean leader election. It will start from offset 5.
 # Start broker 2 and it will throw OffsetOutOfRangeException during truncation 
to 5 for transaction start offset 6.

Works (or rather fails) like a charm on any kafka from 2.4.1 to 2.7.0.

> Truncation request on broker start up may cause OffsetOutOfRangeException
> -------------------------------------------------------------------------
>
>                 Key: KAFKA-9895
>                 URL: https://issues.apache.org/jira/browse/KAFKA-9895
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 2.4.0
>            Reporter: Boquan Tang
>            Priority: Major
>
> We have a 4 broker cluster running version 2.4.0.
> Upon broker restart, we frequently observe issue like this:
> {code}
> [2020-04-20 20:36:37,827] ERROR [ReplicaFetcher replicaId=4, leaderId=1, 
> fetcherId=0] Unexpected error occurred during truncation for topic-name-10 at 
> offset 632111354 (kafka.server.ReplicaFetcherThread)
> org.apache.kafka.common.errors.OffsetOutOfRangeException: Received request 
> for offset 632111355 for partition active-ads-10, but we only have log 
> segments in the range 0 to 632111354.
> {code}
> The partition experiencing this issue seems random. Could we actually ignore 
> this kind of error and not put this partition to offline? From what the error 
> log describes, I think once the start up finishes, and the partition catches 
> up with leader, it should be OK to put it back to ISR. Please help me if I'm 
> understanding it incorrectly.
> This happens after we updated to 2.4.0, so I'm wondering if it has anything 
> to do with this specific version or not.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to