ocadaruma commented on code in PR #14289: URL: https://github.com/apache/kafka/pull/14289#discussion_r1499548503
########## storage/src/main/java/org/apache/kafka/storage/internals/log/FetchDataInfo.java: ########## @@ -39,19 +40,40 @@ public FetchDataInfo(LogOffsetMetadata fetchOffsetMetadata, Records records, boolean firstEntryIncomplete, Optional<List<FetchResponseData.AbortedTransaction>> abortedTransactions) { - this(fetchOffsetMetadata, records, firstEntryIncomplete, abortedTransactions, Optional.empty()); + this(fetchOffsetMetadata, + records, + firstEntryIncomplete, + abortedTransactions, + Optional.empty(), + LogOffsetMetadata.UNKNOWN_OFFSET_METADATA); } public FetchDataInfo(LogOffsetMetadata fetchOffsetMetadata, Records records, boolean firstEntryIncomplete, Optional<List<FetchResponseData.AbortedTransaction>> abortedTransactions, - Optional<RemoteStorageFetchInfo> delayedRemoteStorageFetch) { + Optional<RemoteStorageFetchInfo> delayedRemoteStorageFetch, + LogOffsetMetadata maxOffsetMetadata) { this.fetchOffsetMetadata = fetchOffsetMetadata; this.records = records; this.firstEntryIncomplete = firstEntryIncomplete; this.abortedTransactions = abortedTransactions; this.delayedRemoteStorageFetch = delayedRemoteStorageFetch; + this.maxOffsetMetadata = maxOffsetMetadata; + } + + public FetchDataInfo withMaxOffsetMetadata(LogOffsetMetadata maxOffsetMetadata) { + return new FetchDataInfo(fetchOffsetMetadata, + records, + firstEntryIncomplete, + abortedTransactions, + delayedRemoteStorageFetch, + maxOffsetMetadata); + } + + public boolean isLastSegment() { Review Comment: Thank you for your feedback. After rethinking, I changed my mind that `FetchDataInfo` isn't a proper class to hold active segment's base offset data since it's not about fetch-data actually against the class name. Instead, `LogReadInfo` is more appropriate since it's about log (which already used to hold leo, hw, ...) -- 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