kamalcph commented on code in PR #15634:
URL: https://github.com/apache/kafka/pull/15634#discussion_r1582144306
##########
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:
> 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.
I'm not clear on this:
1. Segments that are eligible for upload to remote storage only when the
`lastStableOffset` moves beyond the segment-to-be-uploaded-end-offset.
2. When all the replicas loses local data (offline partition), then we
consider the data in remote storage also lost. Currently, for this case, we
don't have provision to serve the remote data.
3. When `firstUnstableOffsetMetadata` is empty, we return `highWatermark`.
With this patch, the `highWatermark` lower boundary is set to
`localLogStartOffset` so there won't be an issue.
> Note that OffsetMetadata (segmentBaseOffset and relativePositionInSegment)
is only used in DelayedFetch for estimating the amount of available bytes.
The
[LogOffsetMetadata#onOlderSegment](https://sourcegraph.com/github.com/apache/kafka@5de5d967adffd864bad3ec729760a430253abf38/-/blob/storage/src/main/java/org/apache/kafka/storage/internals/log/LogOffsetMetadata.java?L54)
method is used in the
[hot-path](https://sourcegraph.com/github.com/apache/kafka@5de5d967adffd864bad3ec729760a430253abf38/-/blob/core/src/main/scala/kafka/log/UnifiedLog.scala?L324)
of incrementing the high-watermark and expects the full metadata, otherwise it
throws an error. Is it ok to remove the throwable from
LogOffsetMetadata#onOlderSegment method and return `false` by default.
--
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]