You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/01/05 23:04:48 UTC

[GitHub] [kafka] ccding commented on a change in pull request #11390: [KAFKA-13369] Follower fetch protocol changes for tiered storage.

ccding commented on a change in pull request #11390:
URL: https://github.com/apache/kafka/pull/11390#discussion_r778999529



##########
File path: clients/src/main/resources/common/message/ListOffsetsResponse.json
##########
@@ -31,7 +31,10 @@
   // Version 6 enables flexible versions.
   //
   // Version 7 is the same as version 6 (KIP-734).
-  "validVersions": "0-7",
+  //
+  // Version 8 enables listing offsets by local timestamp.
+  // This is the ealiest log start offset i the local log. (KIP-405).

Review comment:
       ```suggestion
     // This is the ealiest log start offset in the local log. (KIP-405).
   ```

##########
File path: core/src/main/scala/kafka/log/remote/ClassLoaderAwareRemoteStorageManager.scala
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.log.remote
+
+import org.apache.kafka.server.log.remote.storage.{LogSegmentData, RemoteLogSegmentMetadata, RemoteStorageManager}
+
+import java.io.InputStream
+import java.util
+
+/**
+ * A wrapper class of RemoteStorageManager that sets the context class loader when calling RSM methods.
+ */
+class ClassLoaderAwareRemoteStorageManager(val rsm: RemoteStorageManager,
+                                           val rsmClassLoader: ClassLoader) extends RemoteStorageManager {
+
+  def withClassLoader[T](fun: => T): T = {
+    val originalClassLoader = Thread.currentThread.getContextClassLoader
+    Thread.currentThread.setContextClassLoader(rsmClassLoader)
+    try {
+      fun

Review comment:
       do we need to log if there are any exceptions? or throw the exception out?

##########
File path: core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala
##########
@@ -0,0 +1,225 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.log.remote
+
+import kafka.log._
+import kafka.utils.{CoreUtils, Logging, ShutdownableThread}
+import org.apache.kafka.common.errors.CorruptRecordException
+import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType
+import org.apache.kafka.server.log.remote.storage.{RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteStorageManager}
+
+import java.io.{File, InputStream}
+import java.nio.file.{Files, Path}
+import java.util
+import java.util.concurrent.LinkedBlockingQueue
+import java.util.concurrent.atomic.AtomicBoolean
+
+object RemoteIndexCache {
+  val DirName = "remote-log-index-cache"
+  val TmpFileSuffix = ".tmp"
+  val OffsetIndexFileSuffix = ".oi"
+  val TimeIndexFileSuffix = ".ti"
+  val TxnIndexFileSuffix = ".tx"
+}
+
+class Entry(val offsetIndex: OffsetIndex, val timeIndex: TimeIndex, val txnIndex: TransactionIndex) {
+  private val markedForCleanup = new AtomicBoolean(false)
+
+  def lookupOffset(targetOffset: Long): OffsetPosition = {
+    if (markedForCleanup.get()) throw new IllegalStateException("This entry is already closed")

Review comment:
       Should the error message be `closed` or `cleaned up`? If it is `closed`, should we also set `markedForCleanup` to `true` during `close()`?

##########
File path: core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala
##########
@@ -0,0 +1,225 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.log.remote
+
+import kafka.log._
+import kafka.utils.{CoreUtils, Logging, ShutdownableThread}
+import org.apache.kafka.common.errors.CorruptRecordException
+import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType
+import org.apache.kafka.server.log.remote.storage.{RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteStorageManager}
+
+import java.io.{File, InputStream}
+import java.nio.file.{Files, Path}
+import java.util
+import java.util.concurrent.LinkedBlockingQueue
+import java.util.concurrent.atomic.AtomicBoolean
+
+object RemoteIndexCache {
+  val DirName = "remote-log-index-cache"
+  val TmpFileSuffix = ".tmp"
+  val OffsetIndexFileSuffix = ".oi"
+  val TimeIndexFileSuffix = ".ti"
+  val TxnIndexFileSuffix = ".tx"
+}
+
+class Entry(val offsetIndex: OffsetIndex, val timeIndex: TimeIndex, val txnIndex: TransactionIndex) {
+  private val markedForCleanup = new AtomicBoolean(false)
+
+  def lookupOffset(targetOffset: Long): OffsetPosition = {
+    if (markedForCleanup.get()) throw new IllegalStateException("This entry is already closed")
+    else offsetIndex.lookup(targetOffset)
+  }
+
+  def lookupTimestamp(timestamp: Long, startingOffset: Long): OffsetPosition = {
+    if (markedForCleanup.get()) throw new IllegalStateException("This entry is already closed")
+
+    val timestampOffset = timeIndex.lookup(timestamp)
+    offsetIndex.lookup(math.max(startingOffset, timestampOffset.offset))
+  }
+
+  def markForCleanup(): Unit = {
+    if (!markedForCleanup.getAndSet(true)) {
+      Array(offsetIndex, timeIndex, txnIndex).foreach(x =>
+        x.renameTo(new File(CoreUtils.replaceSuffix(x.file.getPath, "", UnifiedLog.DeletedFileSuffix))))
+    }
+  }
+
+  def cleanup(): Unit = {
+    markForCleanup()
+    CoreUtils.tryAll(Seq(() => offsetIndex.deleteIfExists(), () => timeIndex.deleteIfExists(), () => txnIndex.deleteIfExists()))
+  }
+
+  def close(): Unit = {
+    Array(offsetIndex, timeIndex, txnIndex).foreach(index => try {
+      index.close()
+    } catch {
+      case _: Exception => // ignore error.
+    })
+  }
+}
+
+/**
+ * This is a LRU cache of remote index files stored in `$logdir/remote-log-index-cache`. This is helpful to avoid
+ * re-fetching the index files like offset, time indexes from the remote storage for every fetch call.
+ *
+ * @param maxSize
+ * @param remoteStorageManager
+ * @param logDir
+ */
+//todo-tier make maxSize configurable
+class RemoteIndexCache(maxSize: Int = 1024, remoteStorageManager: RemoteStorageManager, logDir: String) extends Logging {
+
+  val cacheDir = new File(logDir, "remote-log-index-cache")
+  @volatile var closed = false
+
+  val expiredIndexes = new LinkedBlockingQueue[Entry]()
+  val lock = new Object()
+
+  val entries: util.Map[RemoteLogSegmentId, Entry] = new java.util.LinkedHashMap[RemoteLogSegmentId, Entry](maxSize / 2,
+    0.75f, true) {
+    override def removeEldestEntry(eldest: util.Map.Entry[RemoteLogSegmentId, Entry]): Boolean = {
+      if (this.size() > maxSize) {
+        val entry = eldest.getValue
+        // Mark the entries for cleanup, background thread will clean them later.
+        entry.markForCleanup()
+        expiredIndexes.add(entry)
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private def init(): Unit = {
+    if (cacheDir.mkdir())
+      info(s"Created $cacheDir successfully")
+
+    // Delete any .deleted files remained from the earlier run of the broker.
+    Files.list(cacheDir.toPath).forEach((path: Path) => {
+      if (path.endsWith(UnifiedLog.DeletedFileSuffix)) {
+        Files.deleteIfExists(path)
+      }
+    })
+  }
+
+  init()
+
+  // Start cleaner thread that will clean the expired entries
+  val cleanerThread: ShutdownableThread = new ShutdownableThread("remote-log-index-cleaner") {
+    setDaemon(true)
+
+    override def doWork(): Unit = {
+      val entry = expiredIndexes.take()
+      info(s"Cleaning up index entry $entry")
+      try {
+        entry.cleanup()
+      } catch {
+        case ex: Exception => error("Error occurred while fetching/cleaning up expired entry", ex)
+      }
+    }
+  }
+  cleanerThread.start()
+
+  def getIndexEntry(remoteLogSegmentMetadata: RemoteLogSegmentMetadata): Entry = {
+    def loadIndexFile[T <: CleanableIndex](fileName: String,
+                                           suffix: String,
+                                           fetchRemoteIndex: RemoteLogSegmentMetadata => InputStream,
+                                           readIndex: File => T): T = {
+      val indexFile = new File(cacheDir, fileName + suffix)
+
+      def fetchAndCreateIndex(): T = {
+        val tmpIndexFile = new File(cacheDir, fileName + suffix + RemoteIndexCache.TmpFileSuffix)
+
+        val inputStream = fetchRemoteIndex(remoteLogSegmentMetadata)
+        try {
+          Files.copy(inputStream, tmpIndexFile.toPath)
+        } finally {
+          if (inputStream != null) {
+            inputStream.close()
+          }
+        }
+
+        Utils.atomicMoveWithFallback(tmpIndexFile.toPath, indexFile.toPath)
+        readIndex(indexFile)
+      }
+
+      if (indexFile.exists()) {
+        try {
+          readIndex(indexFile)
+        } catch {
+          case ex: CorruptRecordException =>
+            info("Error occurred while loading the stored index", ex)
+            fetchAndCreateIndex()

Review comment:
       do we need to log/handle exception caused by fetchAndCreateIndex

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java
##########
@@ -42,6 +42,11 @@
     public static final long LATEST_TIMESTAMP = -1L;
     public static final long MAX_TIMESTAMP = -3L;
 
+    /**
+     * It is used to represent the earliest message stored in the local log which is also called the local-log-start-offset
+     */
+    public static final long EARLIEST_LOCAL_TIMESTAMP = -4L;

Review comment:
       `local-log-start-timestamp` instead of `local-log-start-offset`?

##########
File path: core/src/main/scala/kafka/log/UnifiedLog.scala
##########
@@ -288,13 +294,24 @@ class UnifiedLog(@volatile var logStartOffset: Long,
 
   @volatile var partitionMetadataFile : PartitionMetadataFile = null
 
+  //todo-tier it needs to be updated.
+  private val localLogStartOffset: Long = logStartOffset
+
   locally {
     initializePartitionMetadata()
     updateLogStartOffset(logStartOffset)
     maybeIncrementFirstUnstableOffset()
     initializeTopicId()
   }
 
+  def remoteLogEnabled(): Boolean = {
+    // Remote logging is enabled only for non-compact and non-internal topics
+    remoteStorageSystemEnable &&
+      !(config.compact || Topic.isInternal(topicPartition.topic())
+        || TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_NAME.eq(topicPartition.topic())) &&

Review comment:
       Is it possible to make `Topic.isInternal(topicPartition.topic())` to return true if `TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_NAME.eq(topicPartition.topic()))`? then we can get rid of the `TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_NAME.eq(topicPartition.topic()))` check

##########
File path: core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala
##########
@@ -0,0 +1,225 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.log.remote
+
+import kafka.log._
+import kafka.utils.{CoreUtils, Logging, ShutdownableThread}
+import org.apache.kafka.common.errors.CorruptRecordException
+import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType
+import org.apache.kafka.server.log.remote.storage.{RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteStorageManager}
+
+import java.io.{File, InputStream}
+import java.nio.file.{Files, Path}
+import java.util
+import java.util.concurrent.LinkedBlockingQueue
+import java.util.concurrent.atomic.AtomicBoolean
+
+object RemoteIndexCache {
+  val DirName = "remote-log-index-cache"
+  val TmpFileSuffix = ".tmp"
+  val OffsetIndexFileSuffix = ".oi"
+  val TimeIndexFileSuffix = ".ti"
+  val TxnIndexFileSuffix = ".tx"

Review comment:
       is there any reason for not using the existing suffixes? https://github.com/apache/kafka/blob/daaa9dfb54b86ccb93cc2018b374476dcb0beed1/core/src/main/scala/kafka/log/LocalLog.scala#L550-L559

##########
File path: core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala
##########
@@ -0,0 +1,225 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.log.remote
+
+import kafka.log._
+import kafka.utils.{CoreUtils, Logging, ShutdownableThread}
+import org.apache.kafka.common.errors.CorruptRecordException
+import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType
+import org.apache.kafka.server.log.remote.storage.{RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteStorageManager}
+
+import java.io.{File, InputStream}
+import java.nio.file.{Files, Path}
+import java.util
+import java.util.concurrent.LinkedBlockingQueue
+import java.util.concurrent.atomic.AtomicBoolean
+
+object RemoteIndexCache {
+  val DirName = "remote-log-index-cache"
+  val TmpFileSuffix = ".tmp"
+  val OffsetIndexFileSuffix = ".oi"
+  val TimeIndexFileSuffix = ".ti"
+  val TxnIndexFileSuffix = ".tx"
+}
+
+class Entry(val offsetIndex: OffsetIndex, val timeIndex: TimeIndex, val txnIndex: TransactionIndex) {
+  private val markedForCleanup = new AtomicBoolean(false)
+
+  def lookupOffset(targetOffset: Long): OffsetPosition = {
+    if (markedForCleanup.get()) throw new IllegalStateException("This entry is already closed")
+    else offsetIndex.lookup(targetOffset)
+  }
+
+  def lookupTimestamp(timestamp: Long, startingOffset: Long): OffsetPosition = {
+    if (markedForCleanup.get()) throw new IllegalStateException("This entry is already closed")
+
+    val timestampOffset = timeIndex.lookup(timestamp)
+    offsetIndex.lookup(math.max(startingOffset, timestampOffset.offset))
+  }
+
+  def markForCleanup(): Unit = {
+    if (!markedForCleanup.getAndSet(true)) {
+      Array(offsetIndex, timeIndex, txnIndex).foreach(x =>
+        x.renameTo(new File(CoreUtils.replaceSuffix(x.file.getPath, "", UnifiedLog.DeletedFileSuffix))))
+    }
+  }
+
+  def cleanup(): Unit = {
+    markForCleanup()
+    CoreUtils.tryAll(Seq(() => offsetIndex.deleteIfExists(), () => timeIndex.deleteIfExists(), () => txnIndex.deleteIfExists()))
+  }
+
+  def close(): Unit = {
+    Array(offsetIndex, timeIndex, txnIndex).foreach(index => try {
+      index.close()
+    } catch {
+      case _: Exception => // ignore error.
+    })
+  }
+}
+
+/**
+ * This is a LRU cache of remote index files stored in `$logdir/remote-log-index-cache`. This is helpful to avoid
+ * re-fetching the index files like offset, time indexes from the remote storage for every fetch call.
+ *
+ * @param maxSize
+ * @param remoteStorageManager
+ * @param logDir
+ */
+//todo-tier make maxSize configurable
+class RemoteIndexCache(maxSize: Int = 1024, remoteStorageManager: RemoteStorageManager, logDir: String) extends Logging {
+
+  val cacheDir = new File(logDir, "remote-log-index-cache")

Review comment:
       we can use `val DirName = "remote-log-index-cache"` here

##########
File path: core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala
##########
@@ -0,0 +1,225 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.log.remote
+
+import kafka.log._
+import kafka.utils.{CoreUtils, Logging, ShutdownableThread}
+import org.apache.kafka.common.errors.CorruptRecordException
+import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType
+import org.apache.kafka.server.log.remote.storage.{RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteStorageManager}
+
+import java.io.{File, InputStream}
+import java.nio.file.{Files, Path}
+import java.util
+import java.util.concurrent.LinkedBlockingQueue
+import java.util.concurrent.atomic.AtomicBoolean
+
+object RemoteIndexCache {
+  val DirName = "remote-log-index-cache"
+  val TmpFileSuffix = ".tmp"
+  val OffsetIndexFileSuffix = ".oi"
+  val TimeIndexFileSuffix = ".ti"
+  val TxnIndexFileSuffix = ".tx"
+}
+
+class Entry(val offsetIndex: OffsetIndex, val timeIndex: TimeIndex, val txnIndex: TransactionIndex) {
+  private val markedForCleanup = new AtomicBoolean(false)
+
+  def lookupOffset(targetOffset: Long): OffsetPosition = {
+    if (markedForCleanup.get()) throw new IllegalStateException("This entry is already closed")
+    else offsetIndex.lookup(targetOffset)
+  }
+
+  def lookupTimestamp(timestamp: Long, startingOffset: Long): OffsetPosition = {
+    if (markedForCleanup.get()) throw new IllegalStateException("This entry is already closed")
+
+    val timestampOffset = timeIndex.lookup(timestamp)
+    offsetIndex.lookup(math.max(startingOffset, timestampOffset.offset))
+  }
+
+  def markForCleanup(): Unit = {
+    if (!markedForCleanup.getAndSet(true)) {
+      Array(offsetIndex, timeIndex, txnIndex).foreach(x =>
+        x.renameTo(new File(CoreUtils.replaceSuffix(x.file.getPath, "", UnifiedLog.DeletedFileSuffix))))
+    }
+  }
+
+  def cleanup(): Unit = {
+    markForCleanup()
+    CoreUtils.tryAll(Seq(() => offsetIndex.deleteIfExists(), () => timeIndex.deleteIfExists(), () => txnIndex.deleteIfExists()))
+  }
+
+  def close(): Unit = {
+    Array(offsetIndex, timeIndex, txnIndex).foreach(index => try {
+      index.close()
+    } catch {
+      case _: Exception => // ignore error.
+    })
+  }
+}

Review comment:
       why do we use `CoreUtils.tryAll` in one case and use `Array().foreach( try catch )` in the other case?

##########
File path: core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala
##########
@@ -0,0 +1,225 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.log.remote
+
+import kafka.log._
+import kafka.utils.{CoreUtils, Logging, ShutdownableThread}
+import org.apache.kafka.common.errors.CorruptRecordException
+import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType
+import org.apache.kafka.server.log.remote.storage.{RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteStorageManager}
+
+import java.io.{File, InputStream}
+import java.nio.file.{Files, Path}
+import java.util
+import java.util.concurrent.LinkedBlockingQueue
+import java.util.concurrent.atomic.AtomicBoolean
+
+object RemoteIndexCache {
+  val DirName = "remote-log-index-cache"
+  val TmpFileSuffix = ".tmp"
+  val OffsetIndexFileSuffix = ".oi"
+  val TimeIndexFileSuffix = ".ti"
+  val TxnIndexFileSuffix = ".tx"
+}
+
+class Entry(val offsetIndex: OffsetIndex, val timeIndex: TimeIndex, val txnIndex: TransactionIndex) {
+  private val markedForCleanup = new AtomicBoolean(false)
+
+  def lookupOffset(targetOffset: Long): OffsetPosition = {
+    if (markedForCleanup.get()) throw new IllegalStateException("This entry is already closed")
+    else offsetIndex.lookup(targetOffset)
+  }
+
+  def lookupTimestamp(timestamp: Long, startingOffset: Long): OffsetPosition = {
+    if (markedForCleanup.get()) throw new IllegalStateException("This entry is already closed")
+
+    val timestampOffset = timeIndex.lookup(timestamp)
+    offsetIndex.lookup(math.max(startingOffset, timestampOffset.offset))
+  }
+
+  def markForCleanup(): Unit = {
+    if (!markedForCleanup.getAndSet(true)) {
+      Array(offsetIndex, timeIndex, txnIndex).foreach(x =>
+        x.renameTo(new File(CoreUtils.replaceSuffix(x.file.getPath, "", UnifiedLog.DeletedFileSuffix))))
+    }
+  }
+
+  def cleanup(): Unit = {
+    markForCleanup()
+    CoreUtils.tryAll(Seq(() => offsetIndex.deleteIfExists(), () => timeIndex.deleteIfExists(), () => txnIndex.deleteIfExists()))
+  }
+
+  def close(): Unit = {
+    Array(offsetIndex, timeIndex, txnIndex).foreach(index => try {
+      index.close()
+    } catch {
+      case _: Exception => // ignore error.
+    })
+  }
+}
+
+/**
+ * This is a LRU cache of remote index files stored in `$logdir/remote-log-index-cache`. This is helpful to avoid
+ * re-fetching the index files like offset, time indexes from the remote storage for every fetch call.
+ *
+ * @param maxSize
+ * @param remoteStorageManager
+ * @param logDir
+ */
+//todo-tier make maxSize configurable
+class RemoteIndexCache(maxSize: Int = 1024, remoteStorageManager: RemoteStorageManager, logDir: String) extends Logging {
+
+  val cacheDir = new File(logDir, "remote-log-index-cache")
+  @volatile var closed = false
+
+  val expiredIndexes = new LinkedBlockingQueue[Entry]()
+  val lock = new Object()
+
+  val entries: util.Map[RemoteLogSegmentId, Entry] = new java.util.LinkedHashMap[RemoteLogSegmentId, Entry](maxSize / 2,
+    0.75f, true) {
+    override def removeEldestEntry(eldest: util.Map.Entry[RemoteLogSegmentId, Entry]): Boolean = {
+      if (this.size() > maxSize) {
+        val entry = eldest.getValue
+        // Mark the entries for cleanup, background thread will clean them later.
+        entry.markForCleanup()
+        expiredIndexes.add(entry)
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private def init(): Unit = {
+    if (cacheDir.mkdir())
+      info(s"Created $cacheDir successfully")
+
+    // Delete any .deleted files remained from the earlier run of the broker.
+    Files.list(cacheDir.toPath).forEach((path: Path) => {
+      if (path.endsWith(UnifiedLog.DeletedFileSuffix)) {
+        Files.deleteIfExists(path)
+      }
+    })
+  }
+
+  init()
+
+  // Start cleaner thread that will clean the expired entries
+  val cleanerThread: ShutdownableThread = new ShutdownableThread("remote-log-index-cleaner") {
+    setDaemon(true)
+
+    override def doWork(): Unit = {
+      val entry = expiredIndexes.take()
+      info(s"Cleaning up index entry $entry")
+      try {
+        entry.cleanup()
+      } catch {
+        case ex: Exception => error("Error occurred while fetching/cleaning up expired entry", ex)
+      }
+    }
+  }
+  cleanerThread.start()

Review comment:
       what is the trade-off between using a daemon thread and using Kafka scheduler?

##########
File path: core/src/main/scala/kafka/log/remote/RemoteLogManager.scala
##########
@@ -0,0 +1,287 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.log.remote
+
+import kafka.cluster.Partition
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.KafkaConfig
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.utils.Logging
+import org.apache.kafka.common._
+import org.apache.kafka.common.internals.Topic
+import org.apache.kafka.common.record.FileRecords.TimestampAndOffset
+import org.apache.kafka.common.record.{RecordBatch, RemoteLogInputStream}
+import org.apache.kafka.common.utils.{ChildFirstClassLoader, Utils}
+import org.apache.kafka.server.log.remote.metadata.storage.{ClassLoaderAwareRemoteLogMetadataManager, TopicBasedRemoteLogMetadataManagerConfig}
+import org.apache.kafka.server.log.remote.storage.{RemoteLogManagerConfig, RemoteLogMetadataManager, RemoteLogSegmentMetadata, RemoteStorageManager}
+
+import java.io.{Closeable, InputStream}
+import java.security.{AccessController, PrivilegedAction}
+import java.util
+import java.util.Optional
+import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap}
+import scala.collection.Set
+import scala.jdk.CollectionConverters._
+
+/**
+ * This class is responsible for
+ *  - initializing `RemoteStorageManager` and `RemoteLogMetadataManager` instances.
+ *  - receives any leader and follower replica events and partition stop events and act on them
+ *  - also provides APIs to fetch indexes, metadata about remote log segments.
+ *
+ * @param rlmConfig
+ * @param brokerId
+ * @param logDir
+ */
+class RemoteLogManager(rlmConfig: RemoteLogManagerConfig,
+                       brokerId: Int,
+                       logDir: String) extends Logging with Closeable with KafkaMetricsGroup {
+
+  // topic ids received on leadership changes
+  private val topicPartitionIds: ConcurrentMap[TopicPartition, Uuid] = new ConcurrentHashMap[TopicPartition, Uuid]()
+
+  private val remoteLogStorageManager: RemoteStorageManager = createRemoteStorageManager()
+  private val remoteLogMetadataManager: RemoteLogMetadataManager = createRemoteLogMetadataManager()
+
+  private val indexCache = new RemoteIndexCache(remoteStorageManager = remoteLogStorageManager, logDir = logDir)
+
+  private var closed = false
+
+  private[remote] def createRemoteStorageManager(): RemoteStorageManager = {
+    def createDelegate(classLoader: ClassLoader): RemoteStorageManager = {
+      classLoader.loadClass(rlmConfig.remoteStorageManagerClassName())
+        .getDeclaredConstructor().newInstance().asInstanceOf[RemoteStorageManager]
+    }
+
+    AccessController.doPrivileged(new PrivilegedAction[RemoteStorageManager] {
+      private val classPath = rlmConfig.remoteStorageManagerClassPath()
+
+      override def run(): RemoteStorageManager = {
+          if (classPath != null && classPath.trim.nonEmpty) {
+            val classLoader = new ChildFirstClassLoader(classPath, this.getClass.getClassLoader)
+            val delegate = createDelegate(classLoader)
+            new ClassLoaderAwareRemoteStorageManager(delegate, classLoader)
+          } else {
+            createDelegate(this.getClass.getClassLoader)
+          }
+      }
+    })
+  }
+
+  private def configureRSM(): Unit = {
+    val rsmProps = new util.HashMap[String, Any]()
+    rlmConfig.remoteStorageManagerProps().asScala.foreach { case (k, v) => rsmProps.put(k, v) }
+    rsmProps.put(KafkaConfig.BrokerIdProp, brokerId)
+    remoteLogStorageManager.configure(rsmProps)
+  }
+
+  private[remote] def createRemoteLogMetadataManager(): RemoteLogMetadataManager = {
+    def createDelegate(classLoader: ClassLoader) = {
+      classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+        .getDeclaredConstructor()
+        .newInstance()
+        .asInstanceOf[RemoteLogMetadataManager]
+    }
+
+    AccessController.doPrivileged(new PrivilegedAction[RemoteLogMetadataManager] {
+      private val classPath = rlmConfig.remoteLogMetadataManagerClassPath
+
+      override def run(): RemoteLogMetadataManager = {
+        if (classPath != null && classPath.trim.nonEmpty) {
+          val classLoader = new ChildFirstClassLoader(classPath, this.getClass.getClassLoader)
+          val delegate = createDelegate(classLoader)
+          new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader)
+        } else {
+          createDelegate(this.getClass.getClassLoader)
+        }
+      }
+    })
+  }
+
+  private def configureRLMM(): Unit = {
+    val rlmmProps = new util.HashMap[String, Any]()
+    rlmConfig.remoteLogMetadataManagerProps().asScala.foreach { case (k, v) => rlmmProps.put(k, v) }
+    rlmmProps.put(KafkaConfig.BrokerIdProp, brokerId)
+    rlmmProps.put(KafkaConfig.LogDirProp, logDir)
+    remoteLogMetadataManager.configure(rlmmProps)
+  }
+
+  def startup(): Unit = {
+    // Initialize and configure RSM and RLMM. This will start RSM, RLMM resources which may need to start resources
+    // in connecting to the brokers or remote storages.
+    configureRSM()
+    configureRLMM()
+  }
+
+  def storageManager(): RemoteStorageManager = {
+    remoteLogStorageManager
+  }
+
+  /**
+   * Callback to receive any leadership changes for the topic partitions assigned to this broker. If there are no
+   * existing tasks for a given topic partition then it will assign new leader or follower task else it will convert the
+   * task to respective target state(leader or follower).
+   *
+   * @param partitionsBecomeLeader   partitions that have become leaders on this broker.
+   * @param partitionsBecomeFollower partitions that have become followers on this broker.
+   * @param topicIds                 topic name to topic id mappings.
+   */
+  def onLeadershipChange(partitionsBecomeLeader: Set[Partition],
+                         partitionsBecomeFollower: Set[Partition],
+                         topicIds: util.Map[String, Uuid]): Unit = {
+    debug(s"Received leadership changes for leaders: $partitionsBecomeLeader and followers: $partitionsBecomeFollower")
+
+    // Partitions logs are available when this callback is invoked.
+    // Compact topics and internal topics are filtered here as they are not supported with tiered storage.
+    def filterPartitions(partitions: Set[Partition]): Set[TopicIdPartition] = {
+      partitions.filterNot(partition => Topic.isInternal(partition.topic) ||
+        partition.topicPartition.topic().equals(TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_NAME) ||
+         partition.log.exists(log => log.remoteLogEnabled())).map(partition =>

Review comment:
       wanted to double check you mean `log.remoteLogEnabled()` or `!log.remoteLogEnabled()` here, because it is `filterNot`

##########
File path: core/src/main/scala/kafka/log/remote/RemoteLogManager.scala
##########
@@ -0,0 +1,287 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.log.remote
+
+import kafka.cluster.Partition
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.KafkaConfig
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.utils.Logging
+import org.apache.kafka.common._
+import org.apache.kafka.common.internals.Topic
+import org.apache.kafka.common.record.FileRecords.TimestampAndOffset
+import org.apache.kafka.common.record.{RecordBatch, RemoteLogInputStream}
+import org.apache.kafka.common.utils.{ChildFirstClassLoader, Utils}
+import org.apache.kafka.server.log.remote.metadata.storage.{ClassLoaderAwareRemoteLogMetadataManager, TopicBasedRemoteLogMetadataManagerConfig}
+import org.apache.kafka.server.log.remote.storage.{RemoteLogManagerConfig, RemoteLogMetadataManager, RemoteLogSegmentMetadata, RemoteStorageManager}
+
+import java.io.{Closeable, InputStream}
+import java.security.{AccessController, PrivilegedAction}
+import java.util
+import java.util.Optional
+import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap}
+import scala.collection.Set
+import scala.jdk.CollectionConverters._
+
+/**
+ * This class is responsible for
+ *  - initializing `RemoteStorageManager` and `RemoteLogMetadataManager` instances.
+ *  - receives any leader and follower replica events and partition stop events and act on them
+ *  - also provides APIs to fetch indexes, metadata about remote log segments.
+ *
+ * @param rlmConfig
+ * @param brokerId
+ * @param logDir
+ */
+class RemoteLogManager(rlmConfig: RemoteLogManagerConfig,
+                       brokerId: Int,
+                       logDir: String) extends Logging with Closeable with KafkaMetricsGroup {
+
+  // topic ids received on leadership changes
+  private val topicPartitionIds: ConcurrentMap[TopicPartition, Uuid] = new ConcurrentHashMap[TopicPartition, Uuid]()
+
+  private val remoteLogStorageManager: RemoteStorageManager = createRemoteStorageManager()
+  private val remoteLogMetadataManager: RemoteLogMetadataManager = createRemoteLogMetadataManager()
+
+  private val indexCache = new RemoteIndexCache(remoteStorageManager = remoteLogStorageManager, logDir = logDir)
+
+  private var closed = false
+
+  private[remote] def createRemoteStorageManager(): RemoteStorageManager = {
+    def createDelegate(classLoader: ClassLoader): RemoteStorageManager = {
+      classLoader.loadClass(rlmConfig.remoteStorageManagerClassName())
+        .getDeclaredConstructor().newInstance().asInstanceOf[RemoteStorageManager]
+    }
+
+    AccessController.doPrivileged(new PrivilegedAction[RemoteStorageManager] {
+      private val classPath = rlmConfig.remoteStorageManagerClassPath()
+
+      override def run(): RemoteStorageManager = {
+          if (classPath != null && classPath.trim.nonEmpty) {
+            val classLoader = new ChildFirstClassLoader(classPath, this.getClass.getClassLoader)
+            val delegate = createDelegate(classLoader)
+            new ClassLoaderAwareRemoteStorageManager(delegate, classLoader)
+          } else {
+            createDelegate(this.getClass.getClassLoader)
+          }
+      }
+    })
+  }
+
+  private def configureRSM(): Unit = {
+    val rsmProps = new util.HashMap[String, Any]()
+    rlmConfig.remoteStorageManagerProps().asScala.foreach { case (k, v) => rsmProps.put(k, v) }
+    rsmProps.put(KafkaConfig.BrokerIdProp, brokerId)
+    remoteLogStorageManager.configure(rsmProps)
+  }
+
+  private[remote] def createRemoteLogMetadataManager(): RemoteLogMetadataManager = {
+    def createDelegate(classLoader: ClassLoader) = {
+      classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+        .getDeclaredConstructor()
+        .newInstance()
+        .asInstanceOf[RemoteLogMetadataManager]
+    }
+
+    AccessController.doPrivileged(new PrivilegedAction[RemoteLogMetadataManager] {
+      private val classPath = rlmConfig.remoteLogMetadataManagerClassPath
+
+      override def run(): RemoteLogMetadataManager = {
+        if (classPath != null && classPath.trim.nonEmpty) {
+          val classLoader = new ChildFirstClassLoader(classPath, this.getClass.getClassLoader)
+          val delegate = createDelegate(classLoader)
+          new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader)
+        } else {
+          createDelegate(this.getClass.getClassLoader)
+        }
+      }
+    })
+  }
+
+  private def configureRLMM(): Unit = {
+    val rlmmProps = new util.HashMap[String, Any]()
+    rlmConfig.remoteLogMetadataManagerProps().asScala.foreach { case (k, v) => rlmmProps.put(k, v) }
+    rlmmProps.put(KafkaConfig.BrokerIdProp, brokerId)
+    rlmmProps.put(KafkaConfig.LogDirProp, logDir)
+    remoteLogMetadataManager.configure(rlmmProps)
+  }
+
+  def startup(): Unit = {
+    // Initialize and configure RSM and RLMM. This will start RSM, RLMM resources which may need to start resources
+    // in connecting to the brokers or remote storages.
+    configureRSM()
+    configureRLMM()
+  }
+
+  def storageManager(): RemoteStorageManager = {
+    remoteLogStorageManager
+  }
+
+  /**
+   * Callback to receive any leadership changes for the topic partitions assigned to this broker. If there are no
+   * existing tasks for a given topic partition then it will assign new leader or follower task else it will convert the
+   * task to respective target state(leader or follower).
+   *
+   * @param partitionsBecomeLeader   partitions that have become leaders on this broker.
+   * @param partitionsBecomeFollower partitions that have become followers on this broker.
+   * @param topicIds                 topic name to topic id mappings.
+   */
+  def onLeadershipChange(partitionsBecomeLeader: Set[Partition],
+                         partitionsBecomeFollower: Set[Partition],
+                         topicIds: util.Map[String, Uuid]): Unit = {
+    debug(s"Received leadership changes for leaders: $partitionsBecomeLeader and followers: $partitionsBecomeFollower")
+
+    // Partitions logs are available when this callback is invoked.
+    // Compact topics and internal topics are filtered here as they are not supported with tiered storage.
+    def filterPartitions(partitions: Set[Partition]): Set[TopicIdPartition] = {
+      partitions.filterNot(partition => Topic.isInternal(partition.topic) ||

Review comment:
       `log.remoteLogEnabled()` already checks internal and equals REMOTE_LOG_METADATA_TOPIC_NAME, do we need to check again here?

##########
File path: core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala
##########
@@ -0,0 +1,225 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.log.remote
+
+import kafka.log._
+import kafka.utils.{CoreUtils, Logging, ShutdownableThread}
+import org.apache.kafka.common.errors.CorruptRecordException
+import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType
+import org.apache.kafka.server.log.remote.storage.{RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteStorageManager}
+
+import java.io.{File, InputStream}
+import java.nio.file.{Files, Path}
+import java.util
+import java.util.concurrent.LinkedBlockingQueue
+import java.util.concurrent.atomic.AtomicBoolean
+
+object RemoteIndexCache {
+  val DirName = "remote-log-index-cache"
+  val TmpFileSuffix = ".tmp"
+  val OffsetIndexFileSuffix = ".oi"
+  val TimeIndexFileSuffix = ".ti"
+  val TxnIndexFileSuffix = ".tx"
+}
+
+class Entry(val offsetIndex: OffsetIndex, val timeIndex: TimeIndex, val txnIndex: TransactionIndex) {
+  private val markedForCleanup = new AtomicBoolean(false)
+
+  def lookupOffset(targetOffset: Long): OffsetPosition = {
+    if (markedForCleanup.get()) throw new IllegalStateException("This entry is already closed")
+    else offsetIndex.lookup(targetOffset)
+  }
+
+  def lookupTimestamp(timestamp: Long, startingOffset: Long): OffsetPosition = {
+    if (markedForCleanup.get()) throw new IllegalStateException("This entry is already closed")
+
+    val timestampOffset = timeIndex.lookup(timestamp)
+    offsetIndex.lookup(math.max(startingOffset, timestampOffset.offset))
+  }
+
+  def markForCleanup(): Unit = {
+    if (!markedForCleanup.getAndSet(true)) {
+      Array(offsetIndex, timeIndex, txnIndex).foreach(x =>
+        x.renameTo(new File(CoreUtils.replaceSuffix(x.file.getPath, "", UnifiedLog.DeletedFileSuffix))))
+    }
+  }
+
+  def cleanup(): Unit = {
+    markForCleanup()
+    CoreUtils.tryAll(Seq(() => offsetIndex.deleteIfExists(), () => timeIndex.deleteIfExists(), () => txnIndex.deleteIfExists()))
+  }
+
+  def close(): Unit = {
+    Array(offsetIndex, timeIndex, txnIndex).foreach(index => try {
+      index.close()
+    } catch {
+      case _: Exception => // ignore error.
+    })
+  }
+}
+
+/**
+ * This is a LRU cache of remote index files stored in `$logdir/remote-log-index-cache`. This is helpful to avoid
+ * re-fetching the index files like offset, time indexes from the remote storage for every fetch call.
+ *
+ * @param maxSize
+ * @param remoteStorageManager
+ * @param logDir
+ */
+//todo-tier make maxSize configurable
+class RemoteIndexCache(maxSize: Int = 1024, remoteStorageManager: RemoteStorageManager, logDir: String) extends Logging {
+
+  val cacheDir = new File(logDir, "remote-log-index-cache")
+  @volatile var closed = false
+
+  val expiredIndexes = new LinkedBlockingQueue[Entry]()
+  val lock = new Object()
+
+  val entries: util.Map[RemoteLogSegmentId, Entry] = new java.util.LinkedHashMap[RemoteLogSegmentId, Entry](maxSize / 2,
+    0.75f, true) {
+    override def removeEldestEntry(eldest: util.Map.Entry[RemoteLogSegmentId, Entry]): Boolean = {
+      if (this.size() > maxSize) {
+        val entry = eldest.getValue
+        // Mark the entries for cleanup, background thread will clean them later.
+        entry.markForCleanup()
+        expiredIndexes.add(entry)
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private def init(): Unit = {
+    if (cacheDir.mkdir())
+      info(s"Created $cacheDir successfully")
+
+    // Delete any .deleted files remained from the earlier run of the broker.
+    Files.list(cacheDir.toPath).forEach((path: Path) => {
+      if (path.endsWith(UnifiedLog.DeletedFileSuffix)) {
+        Files.deleteIfExists(path)
+      }
+    })
+  }
+
+  init()
+
+  // Start cleaner thread that will clean the expired entries
+  val cleanerThread: ShutdownableThread = new ShutdownableThread("remote-log-index-cleaner") {
+    setDaemon(true)
+
+    override def doWork(): Unit = {
+      val entry = expiredIndexes.take()
+      info(s"Cleaning up index entry $entry")
+      try {
+        entry.cleanup()
+      } catch {
+        case ex: Exception => error("Error occurred while fetching/cleaning up expired entry", ex)
+      }
+    }
+  }
+  cleanerThread.start()
+
+  def getIndexEntry(remoteLogSegmentMetadata: RemoteLogSegmentMetadata): Entry = {
+    def loadIndexFile[T <: CleanableIndex](fileName: String,
+                                           suffix: String,
+                                           fetchRemoteIndex: RemoteLogSegmentMetadata => InputStream,
+                                           readIndex: File => T): T = {
+      val indexFile = new File(cacheDir, fileName + suffix)
+
+      def fetchAndCreateIndex(): T = {
+        val tmpIndexFile = new File(cacheDir, fileName + suffix + RemoteIndexCache.TmpFileSuffix)
+
+        val inputStream = fetchRemoteIndex(remoteLogSegmentMetadata)
+        try {
+          Files.copy(inputStream, tmpIndexFile.toPath)
+        } finally {
+          if (inputStream != null) {
+            inputStream.close()
+          }
+        }
+
+        Utils.atomicMoveWithFallback(tmpIndexFile.toPath, indexFile.toPath)

Review comment:
       ```suggestion
           Utils.atomicMoveWithFallback(tmpIndexFile.toPath, indexFile.toPath, false)
   ```
   I think we don't need to flush the parent dir. It doesn't matter if the cache file is not renamed during an unclean shutdown.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org