You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Grigory (Jira)" <ji...@apache.org> on 2021/08/06 15:38:00 UTC

[jira] [Commented] (KAFKA-9895) Truncation request on broker start up may cause OffsetOutOfRangeException

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

Grigory commented on KAFKA-9895:
--------------------------------

Any news on this issue ? Was this fixed in recent updates ? Is https://issues.apache.org/jira/browse/KAFKA-12153 somehow relates to this ?
We are using kafka 2.5.1 and still encounter this problem after new leader election:

broker 2 (new leader of partition __consumer_offsets-43)
23.07.2021 10:07:37.360
message: [Partition __consumer_offsets-43 broker=2] __consumer_offsets-43 starts at leader epoch 477 from offset 1689957334 with high watermark 1689957250. Previous leader epoch was 476.

broker 1 (follower)
23.07.2021 10:07:37.765
message: [ReplicaFetcherManager on broker 1] Added fetcher to broker 2 for partitions Map(vpm.lambda.vectors_info-1 -> (offset=2194359, leaderEpoch=150), bpm.meeteor.meeting-status-changes.queue-1 -> (offset=12382935, leaderEpoch=151), __consumer_offsets-43 -> (offset=1689957334, leaderEpoch=477), tcrm.customer.incoming-10 -> (offset=15536425, leaderEpoch=258), pfa.advert.log-1 -> (offset=1501832237, leaderEpoch=389), tcrm.customer.attributes-3 -> (offset=41624919, leaderEpoch=153), pfa.access.log-5 -> (offset=2653655, leaderEpoch=307))

23.07.2021 10:07:37.801
message: [Log partition=__consumer_offsets-43, dir=/data1/kafka] Truncating to offset 1689957334

23.07.2021 10:07:38.224
exception: {
exception_class: org.apache.kafka.common.errors.OffsetOutOfRangeException
exception_message: Received request for offset 1689957523 for partition __consumer_offsets-43, but we only have log segments in the range 1490266249 to 1689957334.
stacktrace: org.apache.kafka.common.errors.OffsetOutOfRangeException: Received request for offset 1689957523 for partition __consumer_offsets-43, but we only 
have log segments in the range 1490266249 to 1689957334.
message: [ReplicaFetcher replicaId=1, leaderId=2, fetcherId=0] Unexpected error occurred during truncation for __consumer_offsets-43 at offset 1689957334

23.07.2021 10:07:38.224
message: [ReplicaFetcher replicaId=1, leaderId=2, fetcherId=0] Partition __consumer_offsets-43 marked as failed

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