GitHub user srdo opened a pull request: https://github.com/apache/kafka/pull/1178
KAFKA-725: Change behavior of Log/LogSegment when attempting read on an offset that's above high watermark. This should make Log.read act the same when startOffset is larger than maxOffset as it would if startOffset was larger than logEndOffset. The current behavior can result in an IllegalArgumentException from LogSegment if a consumer attempts to fetch an offset above the high watermark which is present in the leader's log. It seems more correct if Log.read presents the view of the log to consumers as if it simply ended at maxOffset (high watermark). I've tried to describe an example scenario of this happening here https://issues.apache.org/jira/browse/KAFKA-725?focusedCommentId=15221673&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15221673 I'm not sure I understand why ReplicaManager sets maxOffset to the high watermark, and not high watermark + 1. Isn't the high watermark the last committed message, and readable by consumers? Tests passed for me locally on second try, seems like it just hit a flaky test. You can merge this pull request into a Git repository by running: $ git pull https://github.com/srdo/kafka KAFKA-725 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/1178.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #1178 ---- commit 5c7d583ec1af0892e9fadc4bbdcbeaa94390524e Author: Stig Rohde Døssing <s...@it-minds.dk> Date: 2016-04-02T12:20:50Z KAFKA-725: Throw OffsetOutOfRangeException when reading from Log with maxOffset > startOffset commit 5546433916d49b30b0869964a779e1af189be0ce Author: Stig Rohde Døssing <s...@it-minds.dk> Date: 2016-04-02T13:37:22Z KAFKA-725: Return empty message set if reading from Log with maxOffset+1 == startOffset commit 5808b31828d3703729569476217880971bf279af Author: Stig Rohde Døssing <s...@it-minds.dk> Date: 2016-04-02T14:09:40Z KAFKA-725: Return only message offset when reading one beyond maxOffset ---- --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---