[
https://issues.apache.org/jira/browse/KAFKA-17093?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17864599#comment-17864599
]
Andrew Schofield commented on KAFKA-17093:
------------------------------------------
*I do think that seekToEnd works as designed and as documented.*
The difference between the isolation levels is that read_committed consumers
are not prepared to read uncommitted data. KafkaConsumer.poll() and
KafkaConsumer.seekToEnd() both stop before revealing the existence of any
records from open transactions.
The consumer's position is the next offset that will be requested when fetching
records from the broker. At the end of a topic-partition, it points past the
last record so that fetching sees new records as they turn up. The record prior
to that might be a control record. It might even have been removed by the log
cleaner for a compacted topic. The position automatically "jumps" over gaps and
control records. These kinds of factors are why it's tricky to be concrete
about the "last message".
KafkaAdminClient.listOffsets is your best bet to discover offset information,
but it behaves as you'd expect :) . For background, there is a KIP in this area
[https://cwiki.apache.org/confluence/display/KAFKA/KIP-1021%3A+Allow+to+get+last+stable+offset+%28LSO%29+in+kafka-get-offsets.sh]
which is not yet voted.
> KafkaConsumer.seekToEnd should return LSO
> ------------------------------------------
>
> Key: KAFKA-17093
> URL: https://issues.apache.org/jira/browse/KAFKA-17093
> Project: Kafka
> Issue Type: Bug
> Components: clients, consumer
> Affects Versions: 3.6.1
> Environment: Ubuntu, IntelliJ, Scala "org.apache.kafka" %
> "kafka-clients" % "3.6.1"
> Reporter: Tom Kalmijn
> Assignee: Andrew Schofield
> Priority: Major
> Attachments: Kafka17093-v2.java, Kafka17093-v3.java, Kafka17093.java
>
>
>
> Expected
> When using a transactional producer then the method
> KafkaConsumer.seekToEnd(...) of a consumer configured with isolation level
> "read_committed" should return the LSO.
> Observed
> The offset returned is always the actual last offset of the partition, which
> is not the LSO if the latest offsets are occupied by transaction markers.
> Also see this Slack thread:
> https://confluentcommunity.slack.com/archives/C499EFQS0/p1720088282557559
--
This message was sent by Atlassian Jira
(v8.20.10#820010)