junrao commented on code in PR #15634:
URL: https://github.com/apache/kafka/pull/15634#discussion_r1567796843
##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -282,15 +282,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
/**
* Update high watermark with offset metadata. The new high watermark will
be lower
- * bounded by the log start offset and upper bounded by the log end offset.
+ * bounded by the local-log-start-offset and upper bounded by the
log-end-offset.
*
* @param highWatermarkMetadata the suggested high watermark with offset
metadata
* @return the updated high watermark offset
*/
def updateHighWatermark(highWatermarkMetadata: LogOffsetMetadata): Long = {
val endOffsetMetadata = localLog.logEndOffsetMetadata
- val newHighWatermarkMetadata = if (highWatermarkMetadata.messageOffset <
logStartOffset) {
- new LogOffsetMetadata(logStartOffset)
+ val newHighWatermarkMetadata = if (highWatermarkMetadata.messageOffset <
_localLogStartOffset) {
Review Comment:
Thanks for the detailed explanation.
For the `makeLeaders` path, it will call
`UnifiedLog.convertToOffsetMetadataOrThrow`. Within it,
`checkLogStartOffset(offset)` shouldn't throw OFFSET_OUT_OF_RANGE since we are
comparing the offset with logStartOffset. Do you know which part throws
OFFSET_OUT_OF_RANGE error?
For the follower fetch path, it's bounded by `LogEndOffset`. So it shouldn't
need to call `UnifiedLog.fetchHighWatermarkMetadata`, right? The regular
consumer will call `UnifiedLog.fetchHighWatermarkMetadata`.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]