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

Jason Gustafson commented on KAFKA-9724:
----------------------------------------

The interesting thing in the log snippet is the frequency of offset commits. Is 
that expected? I was speculating that we might be entering a loop like the 
following:

1. user commits offset with `commitSync` (or similar) which updates 
Metadata.lastSeenLeaderEpochs
2. in prepareFetch, Metadata.currentLeader then would return no leader and the 
last seen epoch
3. we trigger a metadata update because we have no leader
4. we get the metadata update without epoch information and reset 
Metadata.lastSeenLeaderEpochs
5. now we can fetch, but if we get another offset commit first, we would go 
back to 1

I tried to reproduce this issue locally, but can't say I fully succeeded. I did 
notice some pauses, but they were very brief. I definitely did notice the 
unnecessary metadata updates from step 3) though, so I think this is worth 
fixing even if it does not turn out to be the root cause of this issue. The 
obvious fix is to skip updating `lastSeenLeaderEpochs` in step 1 if we have a 
current leader, but the epoch is not known.

> Consumer wrongly ignores fetched records "since it no longer has valid 
> position"
> --------------------------------------------------------------------------------
>
>                 Key: KAFKA-9724
>                 URL: https://issues.apache.org/jira/browse/KAFKA-9724
>             Project: Kafka
>          Issue Type: Bug
>          Components: clients, consumer
>    Affects Versions: 2.4.0
>            Reporter: Oleg Muravskiy
>            Priority: Major
>
> After upgrading kafka-client to 2.4.0 (while brokers are still at 2.2.0) 
> consumers in a consumer group intermittently stop progressing on assigned 
> partitions, even when there are messages to consume. This is not a permanent 
> condition, as they progress from time to time, but become slower with time, 
> and catch up after restart.
> Here is a sample of 3 consecutive ignored fetches:
> {noformat}
> 2020-03-15 12:08:58,440 DEBUG [Thread-6] o.a.k.c.c.i.ConsumerCoordinator - 
> Committed offset 538065584 for partition mrt-rrc10-6
> 2020-03-15 12:08:58,541 DEBUG [Thread-6] o.a.k.c.consumer.internals.Fetcher - 
> Skipping validation of fetch offsets for partitions [mrt-rrc10-1, 
> mrt-rrc10-6, mrt-rrc22-7] since the broker does not support the required 
> protocol version (introduced in Kafka 2.3)
> 2020-03-15 12:08:58,549 DEBUG [Thread-6] org.apache.kafka.clients.Metadata - 
> Updating last seen epoch from null to 62 for partition mrt-rrc10-6
> 2020-03-15 12:08:58,557 DEBUG [Thread-6] o.a.k.c.c.i.ConsumerCoordinator - 
> Committed offset 538065584 for partition mrt-rrc10-6
> 2020-03-15 12:08:58,652 DEBUG [Thread-6] o.a.k.c.consumer.internals.Fetcher - 
> Skipping validation of fetch offsets for partitions [mrt-rrc10-1, 
> mrt-rrc10-6, mrt-rrc22-7] since the broker does not support the required 
> protocol version (introduced in Kafka 2.3)
> 2020-03-15 12:08:58,659 DEBUG [Thread-6] org.apache.kafka.clients.Metadata - 
> Updating last seen epoch from null to 62 for partition mrt-rrc10-6
> 2020-03-15 12:08:58,659 DEBUG [Thread-6] o.a.k.c.consumer.internals.Fetcher - 
> Fetch READ_UNCOMMITTED at offset 538065584 for partition mrt-rrc10-6 returned 
> fetch data (error=NONE, highWaterMark=538065631, lastStableOffset = 
> 538065631, logStartOffset = 485284547, preferredReadReplica = absent, 
> abortedTransactions = null, recordsSizeInBytes=16380)
> 2020-03-15 12:08:58,659 DEBUG [Thread-6] o.a.k.c.consumer.internals.Fetcher - 
> Ignoring fetched records for partition mrt-rrc10-6 since it no longer has 
> valid position
> 2020-03-15 12:08:58,665 DEBUG [Thread-6] o.a.k.c.c.i.ConsumerCoordinator - 
> Committed offset 538065584 for partition mrt-rrc10-6
> 2020-03-15 12:08:58,761 DEBUG [Thread-6] o.a.k.c.consumer.internals.Fetcher - 
> Skipping validation of fetch offsets for partitions [mrt-rrc10-1, 
> mrt-rrc10-6, mrt-rrc22-7] since the broker does not support the required 
> protocol version (introduced in Kafka 2.3)
> 2020-03-15 12:08:58,761 DEBUG [Thread-6] o.a.k.c.consumer.internals.Fetcher - 
> Added READ_UNCOMMITTED fetch request for partition mrt-rrc10-6 at position 
> FetchPosition{offset=538065584, offsetEpoch=Optional[62], 
> currentLeader=LeaderAndEpoch{leader=node03.kafka:9092 (id: 3 rack: null), 
> epoch=-1}} to node node03.kafka:9092 (id: 3 rack: null)
> 2020-03-15 12:08:58,761 DEBUG [Thread-6] o.a.k.c.consumer.internals.Fetcher - 
> Sending READ_UNCOMMITTED IncrementalFetchRequest(toSend=(), toForget=(), 
> implied=(mrt-rrc10-6, mrt-rrc22-7, mrt-rrc10-1)) to broker node03.kafka:9092 
> (id: 3 rack: null)
> 2020-03-15 12:08:58,770 DEBUG [Thread-6] org.apache.kafka.clients.Metadata - 
> Updating last seen epoch from null to 62 for partition mrt-rrc10-6
> 2020-03-15 12:08:58,770 DEBUG [Thread-6] o.a.k.c.consumer.internals.Fetcher - 
> Fetch READ_UNCOMMITTED at offset 538065584 for partition mrt-rrc10-6 returned 
> fetch data (error=NONE, highWaterMark=538065727, lastStableOffset = 
> 538065727, logStartOffset = 485284547, preferredReadReplica = absent, 
> abortedTransactions = null, recordsSizeInBytes=51864)
> 2020-03-15 12:08:58,770 DEBUG [Thread-6] o.a.k.c.consumer.internals.Fetcher - 
> Ignoring fetched records for partition mrt-rrc10-6 since it no longer has 
> valid position
> 2020-03-15 12:08:58,808 DEBUG [Thread-6] o.a.k.c.c.i.ConsumerCoordinator - 
> Committed offset 538065584 for partition mrt-rrc10-6
> {noformat}
> After which consumer makes progress:
> {noformat}
> 2020-03-15 12:08:58,871 DEBUG [Thread-6] o.a.k.c.consumer.internals.Fetcher - 
> Skipping validation of fetch offsets for partitions [mrt-rrc10-1, 
> mrt-rrc10-6, mrt-rrc22-7] since the broker does not support the required 
> protocol version (introduced in Kafka 2.3)
> 2020-03-15 12:08:58,871 DEBUG [Thread-6] o.a.k.c.consumer.internals.Fetcher - 
> Added READ_UNCOMMITTED fetch request for partition mrt-rrc10-6 at position 
> FetchPosition{offset=538065584, offsetEpoch=Optional[62], 
> currentLeader=LeaderAndEpoch{leader=node03.kafka:9092 (id: 3 rack: null), 
> epoch=-1}} to node node03.kafka:9092 (id: 3 rack: null)
> 2020-03-15 12:08:58,871 DEBUG [Thread-6] o.a.k.c.consumer.internals.Fetcher - 
> Sending READ_UNCOMMITTED IncrementalFetchRequest(toSend=(), toForget=(), 
> implied=(mrt-rrc10-6, mrt-rrc22-7, mrt-rrc10-1)) to broker node03.kafka:9092 
> (id: 3 rack: null)
> 2020-03-15 12:08:58,872 DEBUG [Thread-6] o.a.k.c.consumer.internals.Fetcher - 
> Fetch READ_UNCOMMITTED at offset 538065584 for partition mrt-rrc10-6 returned 
> fetch data (error=NONE, highWaterMark=538065744, lastStableOffset = 
> 538065744, logStartOffset = 485284547, preferredReadReplica = absent, 
> abortedTransactions = null, recordsSizeInBytes=58293)
> 2020-03-15 12:08:58,872 DEBUG [Thread-6] o.a.k.c.consumer.internals.Fetcher - 
> Added READ_UNCOMMITTED fetch request for partition mrt-rrc10-6 at position 
> FetchPosition{offset=538065744, offsetEpoch=Optional[62], 
> currentLeader=LeaderAndEpoch{leader=node03.kafka:9092 (id: 3 rack: null), 
> epoch=-1}} to node node03.kafka:9092 (id: 3 rack: null)
> 2020-03-15 12:08:58,872 DEBUG [Thread-6] o.a.k.c.consumer.internals.Fetcher - 
> Sending READ_UNCOMMITTED IncrementalFetchRequest(toSend=(mrt-rrc10-6), 
> toForget=(), implied=(mrt-rrc22-7, mrt-rrc10-1)) to broker node03.kafka:9092 
> (id: 3 rack: null)
> 2020-03-15 12:08:58,880 DEBUG [Thread-6] org.apache.kafka.clients.Metadata - 
> Updating last seen epoch from null to 62 for partition mrt-rrc10-6
> 2020-03-15 12:08:58,885 DEBUG [Thread-6] o.a.k.c.c.i.ConsumerCoordinator - 
> Committed offset 538065744 for partition mrt-rrc10-6
> {noformat}
> But it could be stuck for quite a long time.



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

Reply via email to