junrao commented on code in PR #13561: URL: https://github.com/apache/kafka/pull/13561#discussion_r1296221075
########## core/src/main/scala/kafka/log/UnifiedLog.scala: ########## @@ -1390,7 +1424,15 @@ class UnifiedLog(@volatile var logStartOffset: Long, private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean, reason: SegmentDeletionReason): Int = { def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = { - highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) && + val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) + + // Check not to delete segments which are not yet copied to tiered storage + val isSegmentTieredToRemoteStorage = Review Comment: Yes, I agree that it's not a large and common issue. So, we can leave it as it is for now. ########## core/src/main/scala/kafka/server/KafkaServer.scala: ########## @@ -614,7 +614,13 @@ class KafkaServer( } Some(new RemoteLogManager(config.remoteLogManagerConfig, config.brokerId, config.logDirs.head, clusterId, time, - (tp: TopicPartition) => logManager.getLog(tp).asJava, brokerTopicStats)); + (tp: TopicPartition) => logManager.getLog(tp).asJava, + (tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => { + logManager.getLog(tp).foreach(log => { Review Comment: This can be simplified a bit to `.foreach{ log => ...}`. Ditto for the same code in BrokerServer. -- 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