junrao commented on code in PR #15634: URL: https://github.com/apache/kafka/pull/15634#discussion_r1556593959
########## 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: Hmm, when will we set HWM to be lower than _localLogStartOffset? In `UnifiedLog.deletableSegments()`, we have the following code that bounds the retention based deletion by highWatermark. When updating highWatermark, the value typically increases. ` val predicateResult = highWatermark >= upperBoundOffset && predicate(segment, nextSegmentOpt) ` -- 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: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org