[ https://issues.apache.org/jira/browse/KAFKA-10465?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17193988#comment-17193988 ]
Guozhang Wang commented on KAFKA-10465: --------------------------------------- Hello [~manme...@gmail.com] what's documented is correct: you should only see up to LSO when read-committed is set on the consumer side, and I've also just double checked the source code in 2.4 and confirmed that's the case. I think what's maybe confusing here is that when you pause the producer, the txn may still be timed out on the broker side and be aborted, and hence an abort-marker is appended, and the end-offset is the "next offset" that's going to be appended in the log, and if it happens to be the case that: offset: ... 81 82 83 rec rec abort-marker and then a read-committed consumer's endOffset() call would return 84. > Potential Bug/Doc update in Transactional Producer and Isolation Level > ---------------------------------------------------------------------- > > Key: KAFKA-10465 > URL: https://issues.apache.org/jira/browse/KAFKA-10465 > Project: Kafka > Issue Type: Bug > Components: clients > Affects Versions: 2.4.1 > Reporter: M. Manna > Priority: Critical > Attachments: ConsumerTestApp.java, Consumer_endOffsets_return_84.jpg, > ProducerTestApp.java, Producer_Committed_Successfully_to_82.jpg > > > *Issue* > Difference between LSO and High Watermark offsets when a consumer with > "read_committed" aren't probably explained in the correct place. > *Expected Behaviour* > According to documentation, the offset returned should be the one committed > last (and successfully). > *Observed (with steps)* > 1. Start a local or test kafka cluster (2.4.1 or above) > 2. Create a topic (I have used 3 replication-factor with 1 partition, but 1 > and 1 is good) > 3. Use the attached producer app file and set debug pointer to be able to > pause on print > 4. Use the attached consumer app file to start a consumer and debug through > steps. > It can be seen that the consumer is actually able to fetch an offset that's > not committed by the producer yet. > Just trying to raise this ticket to confirm whether: > 1) this is well-documented anywhere (which I have missed) - Please refer to > this documentation as a resolution > 2) This is a bug - please confirm and provide a timeline when this can be > fixed. -- This message was sent by Atlassian Jira (v8.3.4#803005)