junrao commented on code in PR #15634:
URL: https://github.com/apache/kafka/pull/15634#discussion_r1569562746
##########
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:
@kamalcph : Thanks for the explanation. I understand the problem now.
As for the fix, it seems that it could work for HWM. However, I am not sure
that we could always do the same thing of LastStableOffset. For example, if we
lose the local data in all replicas, the lastStableOffset could still be in the
middle of a tiered segment and moving it to localLogStartOffset immediately
will be incorrect.
Here is another potential approach. Note that OffsetMetadata
(segmentBaseOffset and relativePositionInSegment) is only used in DelayedFetch
for estimating the amount of available bytes. If occasionally OffsetMetadata is
not available, we don't have to force an exception in
convertToOffsetMetadataOrThrow(). Instead, we can leave the OffsetMetadata as
empty and just use a conservative 1 byte for estimating the amount of available
bytes. This approach will apply to both HWM and LSO. The inaccurate byte
estimate will be ok as long as it's infrequent. What do you think?
--
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]