junrao commented on code in PR #15621: URL: https://github.com/apache/kafka/pull/15621#discussion_r1548687661
########## core/src/main/scala/kafka/log/UnifiedLog.scala: ########## @@ -1337,13 +1337,17 @@ 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))) + // return the base offset for backward compatibility if there is no batches + .getOrElse(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, latestTimestampSegment.baseOffset(), lpc))) Review Comment: Got it. If there is no timestamp index, we initialize `maxTimestampAndOffsetSoFar` to `TimestampOffset(RecordBatch.NO_TIMESTAMP, baseOffset())`. That's why it picks up the base offset. However, it doesn't seem intuitive for the user. Returning None seems better. -- 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