junrao commented on code in PR #15621:
URL: https://github.com/apache/kafka/pull/15621#discussion_r1548487542
##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1337,13 +1337,16 @@ class UnifiedLog(@volatile var logStartOffset: Long,
} else if (targetTimestamp == ListOffsetsRequest.MAX_TIMESTAMP) {
// Cache to avoid race conditions. `toBuffer` is faster than most
alternatives and provides
// constant time access while being safe to use with concurrent
collections unlike `toArray`.
- val segmentsCopy = logSegments.asScala.toBuffer
- val latestTimestampSegment = segmentsCopy.maxBy(_.maxTimestampSoFar)
- val latestTimestampAndOffset =
latestTimestampSegment.readMaxTimestampAndOffsetSoFar
-
- Some(new TimestampAndOffset(latestTimestampAndOffset.timestamp,
- latestTimestampAndOffset.offset,
- latestEpochAsOptional(leaderEpochCache)))
+ val latestTimestampSegment =
logSegments.asScala.toBuffer.maxBy[Long](_.maxTimestampSoFar)
+ // cache the timestamp and offset
+ val maxTimestampSoFar =
latestTimestampSegment.readMaxTimestampAndOffsetSoFar
+ // lookup the position of batch to avoid extra I/O
+ val position =
latestTimestampSegment.offsetIndex.lookup(maxTimestampSoFar.offset)
+ val lpc = latestEpochAsOptional(leaderEpochCache)
+ Some(latestTimestampSegment.log.batchesFrom(position.position).asScala
+ .find(_.maxTimestamp() == maxTimestampSoFar.timestamp)
+ .flatMap(batch => batch.offsetOfMaxTimestamp().asScala.map(new
TimestampAndOffset(batch.maxTimestamp(), _, lpc)))
+ .getOrElse(new TimestampAndOffset(-1, 0, lpc))) // always return
something for backward compatibility
Review Comment:
Hmm, why do we need to return offset=0 when we can't find the maxTimestamp?
--
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]