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 2021/12/04 01:46:11 UTC

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

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



##########
File path: core/src/main/scala/kafka/log/UnifiedLog.scala
##########
@@ -288,13 +292,21 @@ class UnifiedLog(@volatile var logStartOffset: Long,
 
   @volatile var partitionMetadataFile : PartitionMetadataFile = null
 
+  //todo-tier it needs to be updated.
+  private val localLogStartOffset: Long = logStartOffset

Review comment:
       Does this need to be volatile?

##########
File path: core/src/main/scala/kafka/log/CleanableIndex.scala
##########
@@ -0,0 +1,42 @@
+/**
+ * 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
+
+import java.io.{Closeable, File}
+import java.nio.file.Path
+
+import org.apache.kafka.common.utils.Utils
+
+abstract class CleanableIndex(@volatile var _file: File) extends Closeable {

Review comment:
       Could we add a description for this class?

##########
File path: core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala
##########
@@ -0,0 +1,217 @@
+/**
+ * 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}
+import org.apache.kafka.common.errors.CorruptRecordException
+import org.apache.kafka.common.utils.{KafkaThread, 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 closed = new AtomicBoolean(false)
+
+  def lookupOffset(targetOffset: Long): OffsetPosition = {
+    if (closed.get()) throw new IllegalStateException("This entry is already closed")
+    else offsetIndex.lookup(targetOffset)
+  }
+
+  def lookupTimestamp(timestamp: Long, startingOffset: Long): OffsetPosition = {
+    if (closed.get()) throw new IllegalStateException("This entry is already closed")
+
+    val timestampOffset = timeIndex.lookup(timestamp)
+    offsetIndex.lookup(math.max(startingOffset, timestampOffset.offset))
+  }
+
+  def close(): Unit = {
+    if (!closed.getAndSet(true)) {
+      Array(offsetIndex, timeIndex, txnIndex).foreach(x =>
+        x.renameTo(new File(CoreUtils.replaceSuffix(x.file.getPath, "", UnifiedLog.DeletedFileSuffix))))
+    }
+  }
+
+  def cleanup(): Unit = {
+    close()
+    CoreUtils.tryAll(Seq(() => offsetIndex.deleteIfExists(), () => timeIndex.deleteIfExists(), () => txnIndex.deleteIfExists()))
+  }
+}
+
+
+/**
+ * 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
+        // close the entries, background thread will clean them later.
+        entry.close()
+        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: KafkaThread = KafkaThread.daemon("remote-log-index-cleaner", () => {

Review comment:
       Should we use a ShutdownableThread?
   

##########
File path: clients/src/main/java/org/apache/kafka/common/record/RemoteLogInputStream.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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 org.apache.kafka.common.record;
+
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+
+import static org.apache.kafka.common.record.Records.HEADER_SIZE_UP_TO_MAGIC;
+import static org.apache.kafka.common.record.Records.LOG_OVERHEAD;
+import static org.apache.kafka.common.record.Records.MAGIC_OFFSET;
+import static org.apache.kafka.common.record.Records.OFFSET_OFFSET;
+import static org.apache.kafka.common.record.Records.SIZE_OFFSET;
+
+public class RemoteLogInputStream implements LogInputStream<RecordBatch> {
+    private final InputStream is;
+    private final ByteBuffer logHeaderBuffer = ByteBuffer.allocate(HEADER_SIZE_UP_TO_MAGIC);
+
+    public RemoteLogInputStream(InputStream is) {
+        this.is = is;
+    }
+
+    @Override
+    public RecordBatch nextBatch() throws IOException {
+        logHeaderBuffer.rewind();
+        Utils.readFully(is, logHeaderBuffer);
+
+        if (logHeaderBuffer.position() < HEADER_SIZE_UP_TO_MAGIC)
+            return null;
+
+        logHeaderBuffer.rewind();
+        logHeaderBuffer.getLong(OFFSET_OFFSET);
+        int size = logHeaderBuffer.getInt(SIZE_OFFSET);
+
+        // V0 has the smallest overhead, stricter checking is done later
+        if (size < LegacyRecord.RECORD_OVERHEAD_V0)
+            throw new CorruptRecordException(String.format("Found record size %d smaller than minimum record " +
+                                                                   "overhead (%d).", size, LegacyRecord.RECORD_OVERHEAD_V0));
+
+        byte magic = logHeaderBuffer.get(MAGIC_OFFSET);
+        ByteBuffer buffer = ByteBuffer.allocate(size + LOG_OVERHEAD);
+        System.arraycopy(logHeaderBuffer.array(), 0, buffer.array(), 0, logHeaderBuffer.limit());

Review comment:
       To use array(), we have to first check hasArray(). It's probably simpler to do buffer.put(logHeaderBuffer).

##########
File path: clients/src/main/resources/common/message/ListOffsetsRequest.json
##########
@@ -32,7 +32,10 @@
   // Version 6 enables flexible versions.
   //
   // Version 7 enables listing offsets by max timestamp (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:
       i the local log => in the local log

##########
File path: core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
##########
@@ -386,11 +396,76 @@ class ReplicaFetcherThread(name: String,
   }
 
   /**
-   *  To avoid ISR thrashing, we only throttle a replica on the follower if it's in the throttled replica list,
-   *  the quota is exceeded and the replica is not in sync.
+   * To avoid ISR thrashing, we only throttle a replica on the follower if it's in the throttled replica list,
+   * the quota is exceeded and the replica is not in sync.
    */
   private def shouldFollowerThrottle(quota: ReplicaQuota, fetchState: PartitionFetchState, topicPartition: TopicPartition): Boolean = {
     !fetchState.isReplicaInSync && quota.isThrottled(topicPartition) && quota.isQuotaExceeded
   }
 
+  override protected def buildRemoteLogAuxState(partition: TopicPartition,
+                                                currentLeaderEpoch: Int,
+                                                leaderLocalLogStartOffset: Long,
+                                                leaderLogStartOffset: Long): Unit = {
+    replicaMgr.localLog(partition).foreach(log =>
+      if (log.remoteStorageSystemEnable && log.config.remoteLogConfig.remoteStorageEnable) {
+        replicaMgr.remoteLogManager.foreach(rlm => {
+          var rlsMetadata: Optional[RemoteLogSegmentMetadata] = Optional.empty()
+          val epoch = log.leaderEpochCache.flatMap(cache => cache.epochForOffset(leaderLocalLogStartOffset))

Review comment:
       ListOffset returns the leaderEpoch for the localStartOffset. Could we just use that leaderEpoch? It will simplify the code below.

##########
File path: clients/src/main/java/org/apache/kafka/common/utils/ChildFirstClassLoader.java
##########
@@ -0,0 +1,131 @@
+/*
+ * 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 org.apache.kafka.common.utils;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.Locale;
+import java.util.NoSuchElementException;
+
+/**
+ * A class loader that looks for classes and resources in a specified class path first, before delegating to its parent
+ * class loader.
+ */
+public class ChildFirstClassLoader extends URLClassLoader {
+    static {
+        ClassLoader.registerAsParallelCapable();
+    }
+
+    /**
+     * @param classPath Class path string
+     * @param parent    The parent classloader. If the required class / resource cannot be found in the given classPath,
+     *                  this classloader will be used to find the class / resource.
+     */
+    public ChildFirstClassLoader(String classPath, ClassLoader parent) {
+        super(classpath2URLs(classPath), parent);
+    }
+
+    static private URL[] classpath2URLs(String classPath) {
+        ArrayList<URL> urls = new ArrayList<>();
+        for (String path : classPath.split(File.pathSeparator)) {
+            if (path == null || path.trim().isEmpty())
+                continue;
+            File f = new File(path);
+
+            if (path.endsWith("/*")) {
+                try {
+                    File parent = new File(new File(f.getCanonicalPath()).getParent());
+                    if (parent.isDirectory()) {
+                        File[] files = parent.listFiles((dir, name) -> {
+                            String lower = name.toLowerCase(Locale.ROOT);
+                            return lower.endsWith(".jar") || lower.endsWith(".zip");
+                        });
+                        if (files != null) {
+                            for (File jarFile : files) {
+                                urls.add(jarFile.getCanonicalFile().toURI().toURL());
+                            }
+                        }
+                    }
+                } catch (IOException e) {
+                }
+            } else if (f.exists()) {
+                try {
+                    urls.add(f.getCanonicalFile().toURI().toURL());
+                } catch (IOException e) {
+                }
+            }
+        }
+        return urls.toArray(new URL[0]);
+    }
+
+    @Override
+    protected Class<?> loadClass(String name, boolean resolve) throws ClassNotFoundException {
+        synchronized (getClassLoadingLock(name)) {
+            Class<?> c = findLoadedClass(name);
+
+            if (c == null) {
+                try {
+                    c = findClass(name);
+                } catch (ClassNotFoundException e) {
+                    // try parent
+                    c = super.loadClass(name, false);
+                }
+            }
+
+            if (resolve)
+                resolveClass(c);
+
+            return c;
+        }
+    }
+
+    @Override
+    public URL getResource(String name) {
+        URL url = findResource(name);
+        if (url == null) {
+            // try parent
+            url = super.getResource(name);
+        }
+        return url;
+    }
+
+    @Override
+    public Enumeration<URL> getResources(String name) throws IOException {
+        Enumeration<URL> urls1 = findResources(name);
+        Enumeration<URL> urls2 = getParent() != null ? getParent().getResources(name) : null;

Review comment:
       Does the parent CL have a super set of classpathes than the child? If so, is urls1 a subset of urls2?

##########
File path: clients/src/main/java/org/apache/kafka/common/utils/ChildFirstClassLoader.java
##########
@@ -0,0 +1,131 @@
+/*
+ * 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 org.apache.kafka.common.utils;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.Locale;
+import java.util.NoSuchElementException;
+
+/**
+ * A class loader that looks for classes and resources in a specified class path first, before delegating to its parent
+ * class loader.
+ */
+public class ChildFirstClassLoader extends URLClassLoader {
+    static {
+        ClassLoader.registerAsParallelCapable();
+    }
+
+    /**
+     * @param classPath Class path string
+     * @param parent    The parent classloader. If the required class / resource cannot be found in the given classPath,
+     *                  this classloader will be used to find the class / resource.
+     */
+    public ChildFirstClassLoader(String classPath, ClassLoader parent) {
+        super(classpath2URLs(classPath), parent);
+    }
+
+    static private URL[] classpath2URLs(String classPath) {
+        ArrayList<URL> urls = new ArrayList<>();
+        for (String path : classPath.split(File.pathSeparator)) {
+            if (path == null || path.trim().isEmpty())
+                continue;
+            File f = new File(path);
+
+            if (path.endsWith("/*")) {
+                try {
+                    File parent = new File(new File(f.getCanonicalPath()).getParent());
+                    if (parent.isDirectory()) {
+                        File[] files = parent.listFiles((dir, name) -> {
+                            String lower = name.toLowerCase(Locale.ROOT);
+                            return lower.endsWith(".jar") || lower.endsWith(".zip");
+                        });
+                        if (files != null) {
+                            for (File jarFile : files) {
+                                urls.add(jarFile.getCanonicalFile().toURI().toURL());
+                            }
+                        }
+                    }
+                } catch (IOException e) {

Review comment:
       Hmm, why do we want to eat the IOException? Ditto below.

##########
File path: core/src/main/scala/kafka/log/UnifiedLog.scala
##########
@@ -1222,13 +1240,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
 
       if (config.messageFormatVersion < KAFKA_0_10_0_IV0 &&
         targetTimestamp != ListOffsetsRequest.EARLIEST_TIMESTAMP &&
+        targetTimestamp != ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP &&
         targetTimestamp != ListOffsetsRequest.LATEST_TIMESTAMP)
         throw new UnsupportedForMessageFormatException(s"Cannot search offsets based on timestamp because message format version " +
           s"for partition $topicPartition is ${config.messageFormatVersion} which is earlier than the minimum " +
           s"required version $KAFKA_0_10_0_IV0")
 
       // For the earliest and latest, we do not need to return the timestamp.
-      if (targetTimestamp == ListOffsetsRequest.EARLIEST_TIMESTAMP) {
+      if (targetTimestamp == ListOffsetsRequest.EARLIEST_TIMESTAMP ||
+        (!remoteLogEnabled() && targetTimestamp == ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP)) {

Review comment:
       Could this case just be handled below for EARLIEST_LOCAL_TIMESTAMP?

##########
File path: core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala
##########
@@ -0,0 +1,217 @@
+/**
+ * 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}
+import org.apache.kafka.common.errors.CorruptRecordException
+import org.apache.kafka.common.utils.{KafkaThread, 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 closed = new AtomicBoolean(false)
+
+  def lookupOffset(targetOffset: Long): OffsetPosition = {
+    if (closed.get()) throw new IllegalStateException("This entry is already closed")
+    else offsetIndex.lookup(targetOffset)
+  }
+
+  def lookupTimestamp(timestamp: Long, startingOffset: Long): OffsetPosition = {
+    if (closed.get()) throw new IllegalStateException("This entry is already closed")
+
+    val timestampOffset = timeIndex.lookup(timestamp)
+    offsetIndex.lookup(math.max(startingOffset, timestampOffset.offset))
+  }
+
+  def close(): Unit = {
+    if (!closed.getAndSet(true)) {
+      Array(offsetIndex, timeIndex, txnIndex).foreach(x =>
+        x.renameTo(new File(CoreUtils.replaceSuffix(x.file.getPath, "", UnifiedLog.DeletedFileSuffix))))
+    }
+  }
+
+  def cleanup(): Unit = {
+    close()
+    CoreUtils.tryAll(Seq(() => offsetIndex.deleteIfExists(), () => timeIndex.deleteIfExists(), () => txnIndex.deleteIfExists()))
+  }
+}
+
+
+/**
+ * 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
+        // close the entries, background thread will clean them later.
+        entry.close()
+        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: KafkaThread = KafkaThread.daemon("remote-log-index-cleaner", () => {
+    while (!closed) {
+      try {
+        val entry = expiredIndexes.take()
+        info(s"Cleaning up index entry $entry")
+        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 inputStream = fetchRemoteIndex(remoteLogSegmentMetadata)

Review comment:
       fetchRemoteIndex() can take time. Could we call this asynchronously so that we don't block the request handler thread?

##########
File path: core/src/main/scala/kafka/log/UnifiedLog.scala
##########
@@ -1258,8 +1286,34 @@ class UnifiedLog(@volatile var logStartOffset: Long,
         // constant time access while being safe to use with concurrent collections unlike `toArray`.
         val segmentsCopy = logSegments.toBuffer
         // We need to search the first segment whose largest timestamp is >= the target timestamp if there is one.
-        val targetSeg = segmentsCopy.find(_.largestTimestamp >= targetTimestamp)
-        targetSeg.flatMap(_.findOffsetByTimestamp(targetTimestamp, logStartOffset))
+        var isFirstSegment = false
+        val targetSeg: Option[LogSegment] = {
+          // Get all the segments whose largest timestamp is smaller than target timestamp
+          val earlierSegs = segmentsCopy.takeWhile(_.largestTimestamp < targetTimestamp)

Review comment:
       We need to find the first segment matching the timestamp from the remote segments, then the local segments.

##########
File path: core/src/main/scala/kafka/log/UnifiedLog.scala
##########
@@ -257,7 +259,9 @@ class UnifiedLog(@volatile var logStartOffset: Long,
                  @volatile var leaderEpochCache: Option[LeaderEpochFileCache],
                  val producerStateManager: ProducerStateManager,
                  @volatile private var _topicId: Option[Uuid],
-                 val keepPartitionMetadataFile: Boolean) extends Logging with KafkaMetricsGroup {
+                 val keepPartitionMetadataFile: Boolean,
+                 val remoteStorageSystemEnable: Boolean = false,
+                 remoteLogManager: Option[RemoteLogManager] = None) extends Logging with KafkaMetricsGroup {

Review comment:
       Could we add remoteLogManager to the javadoc too?

##########
File path: core/src/main/scala/kafka/log/UnifiedLog.scala
##########
@@ -288,13 +292,21 @@ 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()
   }
 
+  private def remoteLogEnabled(): Boolean = {
+//     Remote logging is enabled only for non-compact and non-internal topics

Review comment:
       indentation

##########
File path: .gitignore
##########
@@ -56,3 +56,5 @@ jmh-benchmarks/src/main/generated
 raft/.jqwik-database
 **/src/generated
 **/src/generated-test
+
+storage/kafka-tiered-storage/

Review comment:
       Is this change needed?

##########
File path: core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala
##########
@@ -0,0 +1,217 @@
+/**
+ * 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}
+import org.apache.kafka.common.errors.CorruptRecordException
+import org.apache.kafka.common.utils.{KafkaThread, 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 closed = new AtomicBoolean(false)
+
+  def lookupOffset(targetOffset: Long): OffsetPosition = {
+    if (closed.get()) throw new IllegalStateException("This entry is already closed")
+    else offsetIndex.lookup(targetOffset)
+  }
+
+  def lookupTimestamp(timestamp: Long, startingOffset: Long): OffsetPosition = {
+    if (closed.get()) throw new IllegalStateException("This entry is already closed")
+
+    val timestampOffset = timeIndex.lookup(timestamp)
+    offsetIndex.lookup(math.max(startingOffset, timestampOffset.offset))
+  }
+
+  def close(): Unit = {
+    if (!closed.getAndSet(true)) {
+      Array(offsetIndex, timeIndex, txnIndex).foreach(x =>
+        x.renameTo(new File(CoreUtils.replaceSuffix(x.file.getPath, "", UnifiedLog.DeletedFileSuffix))))
+    }
+  }
+
+  def cleanup(): Unit = {
+    close()
+    CoreUtils.tryAll(Seq(() => offsetIndex.deleteIfExists(), () => timeIndex.deleteIfExists(), () => txnIndex.deleteIfExists()))
+  }
+}
+
+
+/**
+ * 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
+        // close the entries, background thread will clean them later.
+        entry.close()
+        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: KafkaThread = KafkaThread.daemon("remote-log-index-cleaner", () => {
+    while (!closed) {
+      try {
+        val entry = expiredIndexes.take()
+        info(s"Cleaning up index entry $entry")
+        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 inputStream = fetchRemoteIndex(remoteLogSegmentMetadata)
+        val tmpIndexFile = new File(cacheDir, fileName + suffix + RemoteIndexCache.TmpFileSuffix)
+
+        Files.copy(inputStream, tmpIndexFile.toPath)
+
+        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()
+        }
+      } else {
+        fetchAndCreateIndex()
+      }
+    }
+
+    lock synchronized {
+      entries.computeIfAbsent(remoteLogSegmentMetadata.remoteLogSegmentId(), (key: RemoteLogSegmentId) => {
+        val fileName = key.id().toString
+        val startOffset = remoteLogSegmentMetadata.startOffset()
+
+        val offsetIndex: OffsetIndex = loadIndexFile(fileName, RemoteIndexCache.OffsetIndexFileSuffix,
+          rlsMetadata => remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET),
+          file => {
+            val index = new OffsetIndex(file, startOffset, Int.MaxValue, writable = false)

Review comment:
       This may not matter for now. But it's probably better to use config.maxIndexSize as the max index size.

##########
File path: core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala
##########
@@ -0,0 +1,217 @@
+/**
+ * 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}
+import org.apache.kafka.common.errors.CorruptRecordException
+import org.apache.kafka.common.utils.{KafkaThread, 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 closed = new AtomicBoolean(false)
+
+  def lookupOffset(targetOffset: Long): OffsetPosition = {
+    if (closed.get()) throw new IllegalStateException("This entry is already closed")
+    else offsetIndex.lookup(targetOffset)
+  }
+
+  def lookupTimestamp(timestamp: Long, startingOffset: Long): OffsetPosition = {
+    if (closed.get()) throw new IllegalStateException("This entry is already closed")
+
+    val timestampOffset = timeIndex.lookup(timestamp)
+    offsetIndex.lookup(math.max(startingOffset, timestampOffset.offset))
+  }
+
+  def close(): Unit = {
+    if (!closed.getAndSet(true)) {
+      Array(offsetIndex, timeIndex, txnIndex).foreach(x =>
+        x.renameTo(new File(CoreUtils.replaceSuffix(x.file.getPath, "", UnifiedLog.DeletedFileSuffix))))
+    }
+  }
+
+  def cleanup(): Unit = {
+    close()
+    CoreUtils.tryAll(Seq(() => offsetIndex.deleteIfExists(), () => timeIndex.deleteIfExists(), () => txnIndex.deleteIfExists()))
+  }
+}
+
+
+/**
+ * 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
+        // close the entries, background thread will clean them later.
+        entry.close()
+        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: KafkaThread = KafkaThread.daemon("remote-log-index-cleaner", () => {
+    while (!closed) {
+      try {
+        val entry = expiredIndexes.take()
+        info(s"Cleaning up index entry $entry")
+        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 inputStream = fetchRemoteIndex(remoteLogSegmentMetadata)
+        val tmpIndexFile = new File(cacheDir, fileName + suffix + RemoteIndexCache.TmpFileSuffix)
+
+        Files.copy(inputStream, tmpIndexFile.toPath)
+
+        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()
+        }
+      } else {
+        fetchAndCreateIndex()
+      }
+    }
+
+    lock synchronized {
+      entries.computeIfAbsent(remoteLogSegmentMetadata.remoteLogSegmentId(), (key: RemoteLogSegmentId) => {
+        val fileName = key.id().toString
+        val startOffset = remoteLogSegmentMetadata.startOffset()
+
+        val offsetIndex: OffsetIndex = loadIndexFile(fileName, RemoteIndexCache.OffsetIndexFileSuffix,
+          rlsMetadata => remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET),
+          file => {
+            val index = new OffsetIndex(file, startOffset, Int.MaxValue, writable = false)
+            index.sanityCheck()
+            index
+          })
+
+        val timeIndex: TimeIndex = loadIndexFile(fileName, RemoteIndexCache.TimeIndexFileSuffix,
+          rlsMetadata => remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP),
+          file => {
+            val index = new TimeIndex(file, startOffset, Int.MaxValue, writable = false)
+            index.sanityCheck()
+            index
+          })
+
+        val txnIndex: TransactionIndex = loadIndexFile(fileName, RemoteIndexCache.TxnIndexFileSuffix,
+          rlsMetadata => remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION),
+          file => {
+            val index = new TransactionIndex(startOffset, file)
+            index.sanityCheck()
+            index
+          })
+
+        new Entry(offsetIndex, timeIndex, txnIndex)
+      })
+    }
+  }
+
+  def lookupOffset(remoteLogSegmentMetadata: RemoteLogSegmentMetadata, offset: Long): Int = {
+    getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position
+  }
+
+  def lookupTimestamp(remoteLogSegmentMetadata: RemoteLogSegmentMetadata, timestamp: Long, startingOffset: Long): Int = {
+    getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position
+  }
+
+  def collectAbortedTransaction(remoteLogSegmentMetadata: RemoteLogSegmentMetadata,
+                                startOffset: Long,
+                                fetchSize: Int): TxnIndexSearchResult = {
+    val entry = getIndexEntry(remoteLogSegmentMetadata)
+    val maxOffset = entry.offsetIndex.fetchUpperBoundOffset(entry.offsetIndex.lookup(startOffset), fetchSize)
+      .map(_.offset)
+
+    maxOffset.map(x => entry.txnIndex.collectAbortedTxns(startOffset, x))

Review comment:
       The abort marker for offsets within the fetch range could be in subsequent log segments. So, we need to collect aborted transactions beyond the segment that the fetch offset resides.

##########
File path: core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala
##########
@@ -0,0 +1,217 @@
+/**
+ * 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}
+import org.apache.kafka.common.errors.CorruptRecordException
+import org.apache.kafka.common.utils.{KafkaThread, 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 closed = new AtomicBoolean(false)
+
+  def lookupOffset(targetOffset: Long): OffsetPosition = {
+    if (closed.get()) throw new IllegalStateException("This entry is already closed")
+    else offsetIndex.lookup(targetOffset)
+  }
+
+  def lookupTimestamp(timestamp: Long, startingOffset: Long): OffsetPosition = {
+    if (closed.get()) throw new IllegalStateException("This entry is already closed")
+
+    val timestampOffset = timeIndex.lookup(timestamp)
+    offsetIndex.lookup(math.max(startingOffset, timestampOffset.offset))
+  }
+
+  def close(): Unit = {
+    if (!closed.getAndSet(true)) {
+      Array(offsetIndex, timeIndex, txnIndex).foreach(x =>
+        x.renameTo(new File(CoreUtils.replaceSuffix(x.file.getPath, "", UnifiedLog.DeletedFileSuffix))))
+    }
+  }
+
+  def cleanup(): Unit = {
+    close()
+    CoreUtils.tryAll(Seq(() => offsetIndex.deleteIfExists(), () => timeIndex.deleteIfExists(), () => txnIndex.deleteIfExists()))
+  }
+}
+
+
+/**
+ * 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
+        // close the entries, background thread will clean them later.
+        entry.close()
+        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: KafkaThread = KafkaThread.daemon("remote-log-index-cleaner", () => {
+    while (!closed) {
+      try {
+        val entry = expiredIndexes.take()
+        info(s"Cleaning up index entry $entry")
+        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 inputStream = fetchRemoteIndex(remoteLogSegmentMetadata)
+        val tmpIndexFile = new File(cacheDir, fileName + suffix + RemoteIndexCache.TmpFileSuffix)
+
+        Files.copy(inputStream, tmpIndexFile.toPath)

Review comment:
       Should we close inputStream when done?

##########
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.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 scala.collection.{Set, mutable}
+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 topicIds: mutable.Map[String, Uuid] = mutable.Map.empty
+
+  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 = {
+    AccessController.doPrivileged(new PrivilegedAction[ClassLoaderAwareRemoteStorageManager] {
+      private val classPath = rlmConfig.remoteStorageManagerClassPath()
+
+      override def run(): ClassLoaderAwareRemoteStorageManager = {
+        val classLoader =
+          if (classPath != null && classPath.trim.nonEmpty) {
+            new ChildFirstClassLoader(classPath, this.getClass.getClassLoader)
+          } else {
+            this.getClass.getClassLoader
+          }
+        val delegate = classLoader.loadClass(rlmConfig.remoteStorageManagerClassName())
+          .getDeclaredConstructor().newInstance().asInstanceOf[RemoteStorageManager]
+        new ClassLoaderAwareRemoteStorageManager(delegate, classLoader)
+      }
+    })
+  }
+
+  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 = {
+    AccessController.doPrivileged(new PrivilegedAction[RemoteLogMetadataManager] {
+      private val classPath = rlmConfig.remoteLogMetadataManagerClassPath
+
+      override def run(): RemoteLogMetadataManager = {
+        var classLoader = this.getClass.getClassLoader
+        if (classPath != null && classPath.trim.nonEmpty) {
+          classLoader = new ChildFirstClassLoader(classPath, classLoader)
+          val delegate = classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+            .getDeclaredConstructor()
+            .newInstance()
+            .asInstanceOf[RemoteLogMetadataManager]
+          new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader)
+        } else {
+          classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+            .getDeclaredConstructor()
+            .newInstance()
+            .asInstanceOf[RemoteLogMetadataManager]
+        }
+      }
+    })
+  }
+
+  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.
+   */
+  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")
+    topicIds.forEach((topic, uuid) => this.topicIds.put(topic, uuid))
+
+    // 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[Partition] = {
+      partitions.filterNot(partition => Topic.isInternal(partition.topic) ||
+        partition.log.exists(log => log.config.compact || !log.config.remoteLogConfig.remoteStorageEnable) ||
+        partition.topicPartition.topic().equals(TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_NAME)
+      )
+    }
+
+    val followerTopicPartitions = filterPartitions(partitionsBecomeFollower).map(partition =>
+      new TopicIdPartition(topicIds.get(partition.topic), partition.topicPartition))
+
+    val filteredLeaderPartitions = filterPartitions(partitionsBecomeLeader)
+    val leaderTopicPartitions = filteredLeaderPartitions.map(partition =>
+      new TopicIdPartition(topicIds.get(partition.topic), partition.topicPartition))
+
+    debug(s"Effective topic partitions after filtering compact and internal topics, leaders: $leaderTopicPartitions " +
+      s"and followers: $followerTopicPartitions")
+
+    if (leaderTopicPartitions.nonEmpty || followerTopicPartitions.nonEmpty) {
+      remoteLogMetadataManager.onPartitionLeadershipChanges(leaderTopicPartitions.asJava, followerTopicPartitions.asJava)
+    }
+
+  }
+
+
+  /**
+   * Stops partitions for copying segments, building indexes and deletes the partition in remote storage if delete flag
+   * is set as true.
+   *
+   * @param topicPartition topic partition to be stopped.
+   * @param delete         flag to indicate whether the given topic partitions to be deleted or not.
+   */
+  def stopPartitions(topicPartition: TopicPartition, delete: Boolean): Unit = {
+    // Unassign topic partitions from RLM leader/follower
+    val topicIdPartition =
+      topicIds.remove(topicPartition.topic()) match {

Review comment:
       Hmm, it seems that we only want to remove topicId when all partitions in the topic are removed?

##########
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.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 scala.collection.{Set, mutable}
+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 topicIds: mutable.Map[String, Uuid] = mutable.Map.empty
+
+  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 = {
+    AccessController.doPrivileged(new PrivilegedAction[ClassLoaderAwareRemoteStorageManager] {
+      private val classPath = rlmConfig.remoteStorageManagerClassPath()
+
+      override def run(): ClassLoaderAwareRemoteStorageManager = {
+        val classLoader =
+          if (classPath != null && classPath.trim.nonEmpty) {
+            new ChildFirstClassLoader(classPath, this.getClass.getClassLoader)
+          } else {
+            this.getClass.getClassLoader
+          }
+        val delegate = classLoader.loadClass(rlmConfig.remoteStorageManagerClassName())
+          .getDeclaredConstructor().newInstance().asInstanceOf[RemoteStorageManager]
+        new ClassLoaderAwareRemoteStorageManager(delegate, classLoader)
+      }
+    })
+  }
+
+  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 = {
+    AccessController.doPrivileged(new PrivilegedAction[RemoteLogMetadataManager] {
+      private val classPath = rlmConfig.remoteLogMetadataManagerClassPath
+
+      override def run(): RemoteLogMetadataManager = {
+        var classLoader = this.getClass.getClassLoader
+        if (classPath != null && classPath.trim.nonEmpty) {
+          classLoader = new ChildFirstClassLoader(classPath, classLoader)
+          val delegate = classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+            .getDeclaredConstructor()
+            .newInstance()
+            .asInstanceOf[RemoteLogMetadataManager]
+          new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader)
+        } else {
+          classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+            .getDeclaredConstructor()
+            .newInstance()
+            .asInstanceOf[RemoteLogMetadataManager]
+        }
+      }
+    })
+  }
+
+  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.
+   */
+  def onLeadershipChange(partitionsBecomeLeader: Set[Partition],
+                         partitionsBecomeFollower: Set[Partition],
+                         topicIds: util.Map[String, Uuid]): Unit = {

Review comment:
       Could we add topicIds to javadoc?

##########
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.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 scala.collection.{Set, mutable}
+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 topicIds: mutable.Map[String, Uuid] = mutable.Map.empty
+
+  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 = {
+    AccessController.doPrivileged(new PrivilegedAction[ClassLoaderAwareRemoteStorageManager] {
+      private val classPath = rlmConfig.remoteStorageManagerClassPath()
+
+      override def run(): ClassLoaderAwareRemoteStorageManager = {
+        val classLoader =
+          if (classPath != null && classPath.trim.nonEmpty) {
+            new ChildFirstClassLoader(classPath, this.getClass.getClassLoader)
+          } else {
+            this.getClass.getClassLoader
+          }
+        val delegate = classLoader.loadClass(rlmConfig.remoteStorageManagerClassName())
+          .getDeclaredConstructor().newInstance().asInstanceOf[RemoteStorageManager]
+        new ClassLoaderAwareRemoteStorageManager(delegate, classLoader)
+      }
+    })
+  }
+
+  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 = {
+    AccessController.doPrivileged(new PrivilegedAction[RemoteLogMetadataManager] {
+      private val classPath = rlmConfig.remoteLogMetadataManagerClassPath
+
+      override def run(): RemoteLogMetadataManager = {
+        var classLoader = this.getClass.getClassLoader
+        if (classPath != null && classPath.trim.nonEmpty) {
+          classLoader = new ChildFirstClassLoader(classPath, classLoader)
+          val delegate = classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+            .getDeclaredConstructor()
+            .newInstance()
+            .asInstanceOf[RemoteLogMetadataManager]
+          new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader)
+        } else {
+          classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())

Review comment:
       Could we just call loadClass() once as in createRemoteStorageManager()?

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -396,7 +399,12 @@ abstract class AbstractFetcherThread(name: String,
                 case Errors.OFFSET_OUT_OF_RANGE =>
                   if (handleOutOfRangeError(topicPartition, currentFetchState, fetchPartitionData.currentLeaderEpoch))
                     partitionsWithError += topicPartition
-
+                case Errors.OFFSET_MOVED_TO_TIERED_STORAGE =>
+                  // No need to retry this as it indicates that the requested offset is moved to tiered storage.
+                  // Check whether topicId is available here.

Review comment:
       The code doesn't seem to check topicId.

##########
File path: core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala
##########
@@ -0,0 +1,217 @@
+/**
+ * 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}
+import org.apache.kafka.common.errors.CorruptRecordException
+import org.apache.kafka.common.utils.{KafkaThread, 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 closed = new AtomicBoolean(false)
+
+  def lookupOffset(targetOffset: Long): OffsetPosition = {
+    if (closed.get()) throw new IllegalStateException("This entry is already closed")
+    else offsetIndex.lookup(targetOffset)
+  }
+
+  def lookupTimestamp(timestamp: Long, startingOffset: Long): OffsetPosition = {
+    if (closed.get()) throw new IllegalStateException("This entry is already closed")
+
+    val timestampOffset = timeIndex.lookup(timestamp)
+    offsetIndex.lookup(math.max(startingOffset, timestampOffset.offset))
+  }
+
+  def close(): Unit = {
+    if (!closed.getAndSet(true)) {
+      Array(offsetIndex, timeIndex, txnIndex).foreach(x =>
+        x.renameTo(new File(CoreUtils.replaceSuffix(x.file.getPath, "", UnifiedLog.DeletedFileSuffix))))
+    }
+  }
+
+  def cleanup(): Unit = {
+    close()
+    CoreUtils.tryAll(Seq(() => offsetIndex.deleteIfExists(), () => timeIndex.deleteIfExists(), () => txnIndex.deleteIfExists()))
+  }
+}
+
+
+/**
+ * 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) {

Review comment:
       Should >= be > ?

##########
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.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 scala.collection.{Set, mutable}
+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 topicIds: mutable.Map[String, Uuid] = mutable.Map.empty
+
+  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 = {
+    AccessController.doPrivileged(new PrivilegedAction[ClassLoaderAwareRemoteStorageManager] {
+      private val classPath = rlmConfig.remoteStorageManagerClassPath()
+
+      override def run(): ClassLoaderAwareRemoteStorageManager = {
+        val classLoader =
+          if (classPath != null && classPath.trim.nonEmpty) {
+            new ChildFirstClassLoader(classPath, this.getClass.getClassLoader)
+          } else {
+            this.getClass.getClassLoader
+          }
+        val delegate = classLoader.loadClass(rlmConfig.remoteStorageManagerClassName())
+          .getDeclaredConstructor().newInstance().asInstanceOf[RemoteStorageManager]
+        new ClassLoaderAwareRemoteStorageManager(delegate, classLoader)
+      }
+    })
+  }
+
+  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 = {
+    AccessController.doPrivileged(new PrivilegedAction[RemoteLogMetadataManager] {
+      private val classPath = rlmConfig.remoteLogMetadataManagerClassPath
+
+      override def run(): RemoteLogMetadataManager = {
+        var classLoader = this.getClass.getClassLoader
+        if (classPath != null && classPath.trim.nonEmpty) {
+          classLoader = new ChildFirstClassLoader(classPath, classLoader)
+          val delegate = classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+            .getDeclaredConstructor()
+            .newInstance()
+            .asInstanceOf[RemoteLogMetadataManager]
+          new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader)
+        } else {
+          classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+            .getDeclaredConstructor()
+            .newInstance()
+            .asInstanceOf[RemoteLogMetadataManager]
+        }
+      }
+    })
+  }
+
+  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.
+   */
+  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")
+    topicIds.forEach((topic, uuid) => this.topicIds.put(topic, uuid))
+
+    // 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[Partition] = {
+      partitions.filterNot(partition => Topic.isInternal(partition.topic) ||
+        partition.log.exists(log => log.config.compact || !log.config.remoteLogConfig.remoteStorageEnable) ||
+        partition.topicPartition.topic().equals(TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_NAME)
+      )
+    }
+
+    val followerTopicPartitions = filterPartitions(partitionsBecomeFollower).map(partition =>
+      new TopicIdPartition(topicIds.get(partition.topic), partition.topicPartition))
+
+    val filteredLeaderPartitions = filterPartitions(partitionsBecomeLeader)
+    val leaderTopicPartitions = filteredLeaderPartitions.map(partition =>
+      new TopicIdPartition(topicIds.get(partition.topic), partition.topicPartition))
+
+    debug(s"Effective topic partitions after filtering compact and internal topics, leaders: $leaderTopicPartitions " +
+      s"and followers: $followerTopicPartitions")
+
+    if (leaderTopicPartitions.nonEmpty || followerTopicPartitions.nonEmpty) {
+      remoteLogMetadataManager.onPartitionLeadershipChanges(leaderTopicPartitions.asJava, followerTopicPartitions.asJava)
+    }
+
+  }
+
+
+  /**
+   * Stops partitions for copying segments, building indexes and deletes the partition in remote storage if delete flag
+   * is set as true.
+   *
+   * @param topicPartition topic partition to be stopped.
+   * @param delete         flag to indicate whether the given topic partitions to be deleted or not.
+   */
+  def stopPartitions(topicPartition: TopicPartition, delete: Boolean): Unit = {
+    // Unassign topic partitions from RLM leader/follower
+    val topicIdPartition =
+      topicIds.remove(topicPartition.topic()) match {
+        case Some(uuid) => Some(new TopicIdPartition(uuid, topicPartition))
+        case None => None
+      }
+    debug(s"Removed partition: $topicIdPartition from topic-ids")
+  }
+
+  def fetchRemoteLogSegmentMetadata(tp: TopicPartition,
+                                    epochForOffset: Int,
+                                    offset: Long): Optional[RemoteLogSegmentMetadata] = {
+    val topicIdPartition =
+      topicIds.get(tp.topic()) match {
+        case Some(uuid) => Some(new TopicIdPartition(uuid, tp))
+        case None => None
+      }
+
+    if (topicIdPartition.isEmpty) {
+      throw new KafkaException("No topic id registered for topic partition: " + tp)
+    }
+    remoteLogMetadataManager.remoteLogSegmentMetadata(topicIdPartition.get, epochForOffset, offset)
+  }
+
+  def lookupTimestamp(rlsMetadata: RemoteLogSegmentMetadata, timestamp: Long, startingOffset: Long): Option[TimestampAndOffset] = {
+    val startPos = indexCache.lookupTimestamp(rlsMetadata, timestamp, startingOffset)
+
+    var remoteSegInputStream: InputStream = null
+    try {
+      // Search forward for the position of the last offset that is greater than or equal to the target offset
+      remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata, startPos)

Review comment:
       Similar here, since fetch from remove storage can block, it would be better to do this asynchronously so that it doesn't block a request handler thread.

##########
File path: core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala
##########
@@ -0,0 +1,217 @@
+/**
+ * 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}
+import org.apache.kafka.common.errors.CorruptRecordException
+import org.apache.kafka.common.utils.{KafkaThread, 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 closed = new AtomicBoolean(false)
+
+  def lookupOffset(targetOffset: Long): OffsetPosition = {
+    if (closed.get()) throw new IllegalStateException("This entry is already closed")
+    else offsetIndex.lookup(targetOffset)
+  }
+
+  def lookupTimestamp(timestamp: Long, startingOffset: Long): OffsetPosition = {
+    if (closed.get()) throw new IllegalStateException("This entry is already closed")
+
+    val timestampOffset = timeIndex.lookup(timestamp)
+    offsetIndex.lookup(math.max(startingOffset, timestampOffset.offset))
+  }
+
+  def close(): Unit = {
+    if (!closed.getAndSet(true)) {
+      Array(offsetIndex, timeIndex, txnIndex).foreach(x =>
+        x.renameTo(new File(CoreUtils.replaceSuffix(x.file.getPath, "", UnifiedLog.DeletedFileSuffix))))
+    }
+  }
+
+  def cleanup(): Unit = {
+    close()
+    CoreUtils.tryAll(Seq(() => offsetIndex.deleteIfExists(), () => timeIndex.deleteIfExists(), () => txnIndex.deleteIfExists()))
+  }
+}
+
+
+/**
+ * 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
+        // close the entries, background thread will clean them later.
+        entry.close()
+        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: KafkaThread = KafkaThread.daemon("remote-log-index-cleaner", () => {
+    while (!closed) {
+      try {
+        val entry = expiredIndexes.take()
+        info(s"Cleaning up index entry $entry")
+        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 inputStream = fetchRemoteIndex(remoteLogSegmentMetadata)
+        val tmpIndexFile = new File(cacheDir, fileName + suffix + RemoteIndexCache.TmpFileSuffix)
+
+        Files.copy(inputStream, tmpIndexFile.toPath)
+
+        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()
+        }
+      } else {
+        fetchAndCreateIndex()
+      }
+    }
+
+    lock synchronized {
+      entries.computeIfAbsent(remoteLogSegmentMetadata.remoteLogSegmentId(), (key: RemoteLogSegmentId) => {
+        val fileName = key.id().toString
+        val startOffset = remoteLogSegmentMetadata.startOffset()
+
+        val offsetIndex: OffsetIndex = loadIndexFile(fileName, RemoteIndexCache.OffsetIndexFileSuffix,
+          rlsMetadata => remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET),
+          file => {
+            val index = new OffsetIndex(file, startOffset, Int.MaxValue, writable = false)
+            index.sanityCheck()
+            index
+          })
+
+        val timeIndex: TimeIndex = loadIndexFile(fileName, RemoteIndexCache.TimeIndexFileSuffix,
+          rlsMetadata => remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP),
+          file => {
+            val index = new TimeIndex(file, startOffset, Int.MaxValue, writable = false)
+            index.sanityCheck()
+            index
+          })
+
+        val txnIndex: TransactionIndex = loadIndexFile(fileName, RemoteIndexCache.TxnIndexFileSuffix,
+          rlsMetadata => remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION),
+          file => {
+            val index = new TransactionIndex(startOffset, file)
+            index.sanityCheck()
+            index
+          })
+
+        new Entry(offsetIndex, timeIndex, txnIndex)
+      })
+    }
+  }
+
+  def lookupOffset(remoteLogSegmentMetadata: RemoteLogSegmentMetadata, offset: Long): Int = {
+    getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position
+  }
+
+  def lookupTimestamp(remoteLogSegmentMetadata: RemoteLogSegmentMetadata, timestamp: Long, startingOffset: Long): Int = {
+    getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position
+  }
+
+  def collectAbortedTransaction(remoteLogSegmentMetadata: RemoteLogSegmentMetadata,
+                                startOffset: Long,
+                                fetchSize: Int): TxnIndexSearchResult = {
+    val entry = getIndexEntry(remoteLogSegmentMetadata)
+    val maxOffset = entry.offsetIndex.fetchUpperBoundOffset(entry.offsetIndex.lookup(startOffset), fetchSize)
+      .map(_.offset)
+
+    maxOffset.map(x => entry.txnIndex.collectAbortedTxns(startOffset, x))
+      .getOrElse(TxnIndexSearchResult(List.empty, isComplete = false))
+  }
+
+  def close(): Unit = {

Review comment:
       Should we also close all opened files in entries?

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -715,6 +725,58 @@ abstract class AbstractFetcherThread(name: String,
     }
   }
 
+  /**
+   * Handle a partition whose offset is out of range and return a new fetch offset.
+   */
+  protected def fetchOffsetAndTruncate(topicPartition: TopicPartition, topicId: Option[Uuid], currentLeaderEpoch: Int): PartitionFetchState = {
+    fetchOffsetAndApplyFun(topicPartition, topicId, currentLeaderEpoch,
+      leaderLogStartOffset => truncateFullyAndStartAt(topicPartition, leaderLogStartOffset))
+  }
+
+  /**
+   * Handle a partition whose offset is moved to tiered storage and return a new fetch offset.
+   */
+  protected def fetchOffsetAndBuildRemoteLogAuxState(topicPartition: TopicPartition, topicId: Option[Uuid],
+                                                     currentLeaderEpoch: Int,
+                                                     leaderLogStartOffset: Long): PartitionFetchState = {
+    fetchOffsetAndApplyFun(topicPartition, topicId, currentLeaderEpoch,
+      leaderLocalLogStartOffset =>
+        buildRemoteLogAuxState(topicPartition, currentLeaderEpoch, leaderLocalLogStartOffset, leaderLogStartOffset))
+  }
+
+  /**
+   * Handle the offset moved to tiered storage error. Return false if
+   * 1) the request succeeded or
+   * 2) was fenced and this thread haven't received new epoch,
+   * which means we need not backoff and retry. True if there was a retriable error.
+   */
+  private def handleOffsetMovedToTieredStorage(topicPartition: TopicPartition,
+                                               topicId: Option[Uuid],
+                                               fetchState: PartitionFetchState,
+                                               requestEpoch: Optional[Integer],
+                                               leaderLogStartOffset: Long): Boolean = {
+    try {
+      val newFetchState = fetchOffsetAndBuildRemoteLogAuxState(topicPartition, topicId, fetchState.currentLeaderEpoch, leaderLogStartOffset)

Review comment:
       When receiving OFFSET_MOVED_TO_TIERED_STORAGE, we know the follower's offset is below leader's local start offset. It seems that we could directly ask for leader's local start offset instead of calling fetchLatestOffsetFromLeader().

##########
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.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 scala.collection.{Set, mutable}
+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 topicIds: mutable.Map[String, Uuid] = mutable.Map.empty
+
+  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 = {
+    AccessController.doPrivileged(new PrivilegedAction[ClassLoaderAwareRemoteStorageManager] {
+      private val classPath = rlmConfig.remoteStorageManagerClassPath()
+
+      override def run(): ClassLoaderAwareRemoteStorageManager = {
+        val classLoader =
+          if (classPath != null && classPath.trim.nonEmpty) {
+            new ChildFirstClassLoader(classPath, this.getClass.getClassLoader)
+          } else {
+            this.getClass.getClassLoader
+          }
+        val delegate = classLoader.loadClass(rlmConfig.remoteStorageManagerClassName())
+          .getDeclaredConstructor().newInstance().asInstanceOf[RemoteStorageManager]
+        new ClassLoaderAwareRemoteStorageManager(delegate, classLoader)
+      }
+    })
+  }
+
+  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 = {
+    AccessController.doPrivileged(new PrivilegedAction[RemoteLogMetadataManager] {
+      private val classPath = rlmConfig.remoteLogMetadataManagerClassPath
+
+      override def run(): RemoteLogMetadataManager = {
+        var classLoader = this.getClass.getClassLoader
+        if (classPath != null && classPath.trim.nonEmpty) {
+          classLoader = new ChildFirstClassLoader(classPath, classLoader)
+          val delegate = classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+            .getDeclaredConstructor()
+            .newInstance()
+            .asInstanceOf[RemoteLogMetadataManager]
+          new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader)
+        } else {
+          classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+            .getDeclaredConstructor()
+            .newInstance()
+            .asInstanceOf[RemoteLogMetadataManager]
+        }
+      }
+    })
+  }
+
+  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.
+   */
+  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")
+    topicIds.forEach((topic, uuid) => this.topicIds.put(topic, uuid))
+
+    // 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[Partition] = {
+      partitions.filterNot(partition => Topic.isInternal(partition.topic) ||
+        partition.log.exists(log => log.config.compact || !log.config.remoteLogConfig.remoteStorageEnable) ||

Review comment:
       Could we reuse the method UnifiedLog.remoteLogEnabled()?

##########
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.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 scala.collection.{Set, mutable}
+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 topicIds: mutable.Map[String, Uuid] = mutable.Map.empty
+
+  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 = {
+    AccessController.doPrivileged(new PrivilegedAction[ClassLoaderAwareRemoteStorageManager] {
+      private val classPath = rlmConfig.remoteStorageManagerClassPath()
+
+      override def run(): ClassLoaderAwareRemoteStorageManager = {
+        val classLoader =
+          if (classPath != null && classPath.trim.nonEmpty) {
+            new ChildFirstClassLoader(classPath, this.getClass.getClassLoader)
+          } else {
+            this.getClass.getClassLoader
+          }
+        val delegate = classLoader.loadClass(rlmConfig.remoteStorageManagerClassName())
+          .getDeclaredConstructor().newInstance().asInstanceOf[RemoteStorageManager]
+        new ClassLoaderAwareRemoteStorageManager(delegate, classLoader)
+      }
+    })
+  }
+
+  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 = {
+    AccessController.doPrivileged(new PrivilegedAction[RemoteLogMetadataManager] {
+      private val classPath = rlmConfig.remoteLogMetadataManagerClassPath
+
+      override def run(): RemoteLogMetadataManager = {
+        var classLoader = this.getClass.getClassLoader
+        if (classPath != null && classPath.trim.nonEmpty) {
+          classLoader = new ChildFirstClassLoader(classPath, classLoader)
+          val delegate = classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+            .getDeclaredConstructor()
+            .newInstance()
+            .asInstanceOf[RemoteLogMetadataManager]
+          new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader)
+        } else {
+          classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+            .getDeclaredConstructor()
+            .newInstance()
+            .asInstanceOf[RemoteLogMetadataManager]
+        }
+      }
+    })
+  }
+
+  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.
+   */
+  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")
+    topicIds.forEach((topic, uuid) => this.topicIds.put(topic, uuid))
+
+    // 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[Partition] = {
+      partitions.filterNot(partition => Topic.isInternal(partition.topic) ||
+        partition.log.exists(log => log.config.compact || !log.config.remoteLogConfig.remoteStorageEnable) ||
+        partition.topicPartition.topic().equals(TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_NAME)

Review comment:
       Should we just add REMOTE_LOG_METADATA_TOPIC_NAME to Topic.INTERNAL_TOPICS?

##########
File path: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
##########
@@ -715,6 +725,58 @@ abstract class AbstractFetcherThread(name: String,
     }
   }
 
+  /**
+   * Handle a partition whose offset is out of range and return a new fetch offset.
+   */
+  protected def fetchOffsetAndTruncate(topicPartition: TopicPartition, topicId: Option[Uuid], currentLeaderEpoch: Int): PartitionFetchState = {
+    fetchOffsetAndApplyFun(topicPartition, topicId, currentLeaderEpoch,
+      leaderLogStartOffset => truncateFullyAndStartAt(topicPartition, leaderLogStartOffset))
+  }
+
+  /**
+   * Handle a partition whose offset is moved to tiered storage and return a new fetch offset.
+   */
+  protected def fetchOffsetAndBuildRemoteLogAuxState(topicPartition: TopicPartition, topicId: Option[Uuid],
+                                                     currentLeaderEpoch: Int,
+                                                     leaderLogStartOffset: Long): PartitionFetchState = {
+    fetchOffsetAndApplyFun(topicPartition, topicId, currentLeaderEpoch,
+      leaderLocalLogStartOffset =>
+        buildRemoteLogAuxState(topicPartition, currentLeaderEpoch, leaderLocalLogStartOffset, leaderLogStartOffset))
+  }
+
+  /**
+   * Handle the offset moved to tiered storage error. Return false if
+   * 1) the request succeeded or

Review comment:
       Which request?

##########
File path: core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
##########
@@ -386,11 +396,76 @@ class ReplicaFetcherThread(name: String,
   }
 
   /**
-   *  To avoid ISR thrashing, we only throttle a replica on the follower if it's in the throttled replica list,
-   *  the quota is exceeded and the replica is not in sync.
+   * To avoid ISR thrashing, we only throttle a replica on the follower if it's in the throttled replica list,
+   * the quota is exceeded and the replica is not in sync.
    */
   private def shouldFollowerThrottle(quota: ReplicaQuota, fetchState: PartitionFetchState, topicPartition: TopicPartition): Boolean = {
     !fetchState.isReplicaInSync && quota.isThrottled(topicPartition) && quota.isQuotaExceeded
   }
 
+  override protected def buildRemoteLogAuxState(partition: TopicPartition,
+                                                currentLeaderEpoch: Int,
+                                                leaderLocalLogStartOffset: Long,
+                                                leaderLogStartOffset: Long): Unit = {
+    replicaMgr.localLog(partition).foreach(log =>
+      if (log.remoteStorageSystemEnable && log.config.remoteLogConfig.remoteStorageEnable) {
+        replicaMgr.remoteLogManager.foreach(rlm => {

Review comment:
       It's less verbose to do 
   
   ```
   foreach { rlm =>
    ...
   }
   
   ```

##########
File path: core/src/main/scala/kafka/api/ApiVersion.scala
##########
@@ -119,7 +119,10 @@ object ApiVersion {
     // Assume message format version is 3.0 (KIP-724)
     KAFKA_3_0_IV1,
     // Adds topic IDs to Fetch requests/responses (KIP-516)
-    KAFKA_3_1_IV0
+    KAFKA_3_1_IV0,
+    // Introduce ListOffsets V8 that supports listing offsets by earliest local time stamp,

Review comment:
       Do we want to include the change in fetchRequest too?

##########
File path: core/src/main/scala/kafka/log/ProducerStateManager.scala
##########
@@ -504,6 +504,12 @@ class ProducerStateManager(val topicPartition: TopicPartition,
   // completed transactions whose markers are at offsets above the high watermark
   private val unreplicatedTxns = new util.TreeMap[Long, TxnMetadata]
 
+  def reloadSegments(): Unit = {

Review comment:
       reloadSegments => reloadSnapshots ?

##########
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.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 scala.collection.{Set, mutable}
+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 topicIds: mutable.Map[String, Uuid] = mutable.Map.empty

Review comment:
       Does this need to be a concurrent map since there is no synchronization on access?

##########
File path: core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
##########
@@ -386,11 +396,76 @@ class ReplicaFetcherThread(name: String,
   }
 
   /**
-   *  To avoid ISR thrashing, we only throttle a replica on the follower if it's in the throttled replica list,
-   *  the quota is exceeded and the replica is not in sync.
+   * To avoid ISR thrashing, we only throttle a replica on the follower if it's in the throttled replica list,
+   * the quota is exceeded and the replica is not in sync.
    */
   private def shouldFollowerThrottle(quota: ReplicaQuota, fetchState: PartitionFetchState, topicPartition: TopicPartition): Boolean = {
     !fetchState.isReplicaInSync && quota.isThrottled(topicPartition) && quota.isQuotaExceeded
   }
 
+  override protected def buildRemoteLogAuxState(partition: TopicPartition,
+                                                currentLeaderEpoch: Int,
+                                                leaderLocalLogStartOffset: Long,
+                                                leaderLogStartOffset: Long): Unit = {
+    replicaMgr.localLog(partition).foreach(log =>
+      if (log.remoteStorageSystemEnable && log.config.remoteLogConfig.remoteStorageEnable) {
+        replicaMgr.remoteLogManager.foreach(rlm => {
+          var rlsMetadata: Optional[RemoteLogSegmentMetadata] = Optional.empty()
+          val epoch = log.leaderEpochCache.flatMap(cache => cache.epochForOffset(leaderLocalLogStartOffset))
+          if (epoch.isDefined) {
+            rlsMetadata = rlm.fetchRemoteLogSegmentMetadata(partition, epoch.get, leaderLocalLogStartOffset)
+          } else {
+            // If epoch is not available, then it might be possible that this broker might lost its entire local storage.
+            // We may also have to build the leader epoch cache. To find out the remote log segment metadata for the
+            // leaderLocalLogStartOffset-1, start from the current leader epoch and subtract one to the epoch till
+            // finding the metadata.
+            var previousLeaderEpoch = currentLeaderEpoch
+            while (!rlsMetadata.isPresent && previousLeaderEpoch >= 0) {
+              rlsMetadata = rlm.fetchRemoteLogSegmentMetadata(partition, previousLeaderEpoch, leaderLocalLogStartOffset - 1)
+              previousLeaderEpoch -= 1
+            }
+          }
+          if (rlsMetadata.isPresent) {
+            val epochStream = rlm.storageManager().fetchIndex(rlsMetadata.get(), RemoteStorageManager.IndexType.LEADER_EPOCH)
+            val epochs = readLeaderEpochCheckpoint(epochStream, log.dir)
+
+            // Truncate the existing local log before restoring the leader epoch cache and producer snapshots.
+            truncateFullyAndStartAt(partition, leaderLocalLogStartOffset)
+
+            log.maybeIncrementLogStartOffset(leaderLogStartOffset, LeaderOffsetIncremented)
+            epochs.foreach(epochEntry => {
+              log.leaderEpochCache.map(cache => cache.assign(epochEntry.epoch, epochEntry.startOffset))
+            })
+            info(s"Updated the epoch cache from remote tier till offset: $leaderLocalLogStartOffset " +
+              s"with size: ${epochs.size} for $partition")
+
+            // Restore producer snapshot
+            val snapshotFile = UnifiedLog.producerSnapshotFile(log.dir, leaderLocalLogStartOffset)
+            Files.copy(rlm.storageManager().fetchIndex(rlsMetadata.get(), RemoteStorageManager.IndexType.PRODUCER_SNAPSHOT),
+              snapshotFile.toPath, StandardCopyOption.REPLACE_EXISTING)
+            log.producerStateManager.reloadSegments()
+            log.loadProducerState(leaderLocalLogStartOffset, reloadFromCleanShutdown = false)

Review comment:
       Where is the logic to rebuild RemoteLogMetadataSnapshotFile?

##########
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.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 scala.collection.{Set, mutable}
+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 topicIds: mutable.Map[String, Uuid] = mutable.Map.empty
+
+  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 = {
+    AccessController.doPrivileged(new PrivilegedAction[ClassLoaderAwareRemoteStorageManager] {
+      private val classPath = rlmConfig.remoteStorageManagerClassPath()
+
+      override def run(): ClassLoaderAwareRemoteStorageManager = {
+        val classLoader =
+          if (classPath != null && classPath.trim.nonEmpty) {
+            new ChildFirstClassLoader(classPath, this.getClass.getClassLoader)
+          } else {
+            this.getClass.getClassLoader
+          }
+        val delegate = classLoader.loadClass(rlmConfig.remoteStorageManagerClassName())
+          .getDeclaredConstructor().newInstance().asInstanceOf[RemoteStorageManager]
+        new ClassLoaderAwareRemoteStorageManager(delegate, classLoader)
+      }
+    })
+  }
+
+  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 = {
+    AccessController.doPrivileged(new PrivilegedAction[RemoteLogMetadataManager] {
+      private val classPath = rlmConfig.remoteLogMetadataManagerClassPath
+
+      override def run(): RemoteLogMetadataManager = {
+        var classLoader = this.getClass.getClassLoader
+        if (classPath != null && classPath.trim.nonEmpty) {
+          classLoader = new ChildFirstClassLoader(classPath, classLoader)
+          val delegate = classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+            .getDeclaredConstructor()
+            .newInstance()
+            .asInstanceOf[RemoteLogMetadataManager]
+          new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader)
+        } else {
+          classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+            .getDeclaredConstructor()
+            .newInstance()
+            .asInstanceOf[RemoteLogMetadataManager]
+        }
+      }
+    })
+  }
+
+  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.
+   */
+  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")
+    topicIds.forEach((topic, uuid) => this.topicIds.put(topic, uuid))
+
+    // 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[Partition] = {
+      partitions.filterNot(partition => Topic.isInternal(partition.topic) ||
+        partition.log.exists(log => log.config.compact || !log.config.remoteLogConfig.remoteStorageEnable) ||
+        partition.topicPartition.topic().equals(TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_NAME)
+      )
+    }
+
+    val followerTopicPartitions = filterPartitions(partitionsBecomeFollower).map(partition =>
+      new TopicIdPartition(topicIds.get(partition.topic), partition.topicPartition))
+
+    val filteredLeaderPartitions = filterPartitions(partitionsBecomeLeader)

Review comment:
       Could we just get rid of filteredLeaderPartitions as we have for followerTopicPartitions?

##########
File path: clients/src/main/resources/common/message/ListOffsetsRequest.json
##########
@@ -32,7 +32,10 @@
   // Version 6 enables flexible versions.
   //
   // Version 7 enables listing offsets by max timestamp (KIP-734).
-  "validVersions": "0-7",
+  //
+  // Version 8 enables listing offsets by local timestamp.

Review comment:
       It's not really "by local timestamp". It's localStartOffset.

##########
File path: core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
##########
@@ -386,11 +396,76 @@ class ReplicaFetcherThread(name: String,
   }
 
   /**
-   *  To avoid ISR thrashing, we only throttle a replica on the follower if it's in the throttled replica list,
-   *  the quota is exceeded and the replica is not in sync.
+   * To avoid ISR thrashing, we only throttle a replica on the follower if it's in the throttled replica list,
+   * the quota is exceeded and the replica is not in sync.
    */
   private def shouldFollowerThrottle(quota: ReplicaQuota, fetchState: PartitionFetchState, topicPartition: TopicPartition): Boolean = {
     !fetchState.isReplicaInSync && quota.isThrottled(topicPartition) && quota.isQuotaExceeded
   }
 
+  override protected def buildRemoteLogAuxState(partition: TopicPartition,
+                                                currentLeaderEpoch: Int,
+                                                leaderLocalLogStartOffset: Long,
+                                                leaderLogStartOffset: Long): Unit = {
+    replicaMgr.localLog(partition).foreach(log =>
+      if (log.remoteStorageSystemEnable && log.config.remoteLogConfig.remoteStorageEnable) {
+        replicaMgr.remoteLogManager.foreach(rlm => {
+          var rlsMetadata: Optional[RemoteLogSegmentMetadata] = Optional.empty()
+          val epoch = log.leaderEpochCache.flatMap(cache => cache.epochForOffset(leaderLocalLogStartOffset))
+          if (epoch.isDefined) {
+            rlsMetadata = rlm.fetchRemoteLogSegmentMetadata(partition, epoch.get, leaderLocalLogStartOffset)
+          } else {
+            // If epoch is not available, then it might be possible that this broker might lost its entire local storage.
+            // We may also have to build the leader epoch cache. To find out the remote log segment metadata for the
+            // leaderLocalLogStartOffset-1, start from the current leader epoch and subtract one to the epoch till
+            // finding the metadata.
+            var previousLeaderEpoch = currentLeaderEpoch
+            while (!rlsMetadata.isPresent && previousLeaderEpoch >= 0) {
+              rlsMetadata = rlm.fetchRemoteLogSegmentMetadata(partition, previousLeaderEpoch, leaderLocalLogStartOffset - 1)
+              previousLeaderEpoch -= 1
+            }
+          }
+          if (rlsMetadata.isPresent) {
+            val epochStream = rlm.storageManager().fetchIndex(rlsMetadata.get(), RemoteStorageManager.IndexType.LEADER_EPOCH)
+            val epochs = readLeaderEpochCheckpoint(epochStream, log.dir)
+
+            // Truncate the existing local log before restoring the leader epoch cache and producer snapshots.
+            truncateFullyAndStartAt(partition, leaderLocalLogStartOffset)
+
+            log.maybeIncrementLogStartOffset(leaderLogStartOffset, LeaderOffsetIncremented)
+            epochs.foreach(epochEntry => {
+              log.leaderEpochCache.map(cache => cache.assign(epochEntry.epoch, epochEntry.startOffset))
+            })
+            info(s"Updated the epoch cache from remote tier till offset: $leaderLocalLogStartOffset " +
+              s"with size: ${epochs.size} for $partition")
+
+            // Restore producer snapshot
+            val snapshotFile = UnifiedLog.producerSnapshotFile(log.dir, leaderLocalLogStartOffset)
+            Files.copy(rlm.storageManager().fetchIndex(rlsMetadata.get(), RemoteStorageManager.IndexType.PRODUCER_SNAPSHOT),

Review comment:
       Loading from remote storage could be expensive. It would be useful to do this asynchronously so that the replication thread can make faster progress on other partitions.

##########
File path: core/src/main/scala/kafka/log/ProducerStateManager.scala
##########
@@ -504,6 +504,12 @@ class ProducerStateManager(val topicPartition: TopicPartition,
   // completed transactions whose markers are at offsets above the high watermark
   private val unreplicatedTxns = new util.TreeMap[Long, TxnMetadata]
 
+  def reloadSegments(): Unit = {
+    info("Reloading the producer state snapshots")
+    truncateFullyAndStartAt(0L)

Review comment:
       Is this needed since the caller calls this already?

##########
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.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 scala.collection.{Set, mutable}
+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 topicIds: mutable.Map[String, Uuid] = mutable.Map.empty
+
+  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 = {
+    AccessController.doPrivileged(new PrivilegedAction[ClassLoaderAwareRemoteStorageManager] {
+      private val classPath = rlmConfig.remoteStorageManagerClassPath()
+
+      override def run(): ClassLoaderAwareRemoteStorageManager = {
+        val classLoader =
+          if (classPath != null && classPath.trim.nonEmpty) {
+            new ChildFirstClassLoader(classPath, this.getClass.getClassLoader)
+          } else {
+            this.getClass.getClassLoader
+          }
+        val delegate = classLoader.loadClass(rlmConfig.remoteStorageManagerClassName())
+          .getDeclaredConstructor().newInstance().asInstanceOf[RemoteStorageManager]
+        new ClassLoaderAwareRemoteStorageManager(delegate, classLoader)
+      }
+    })
+  }
+
+  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 = {
+    AccessController.doPrivileged(new PrivilegedAction[RemoteLogMetadataManager] {
+      private val classPath = rlmConfig.remoteLogMetadataManagerClassPath
+
+      override def run(): RemoteLogMetadataManager = {
+        var classLoader = this.getClass.getClassLoader
+        if (classPath != null && classPath.trim.nonEmpty) {
+          classLoader = new ChildFirstClassLoader(classPath, classLoader)
+          val delegate = classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+            .getDeclaredConstructor()
+            .newInstance()
+            .asInstanceOf[RemoteLogMetadataManager]
+          new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader)
+        } else {
+          classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+            .getDeclaredConstructor()
+            .newInstance()
+            .asInstanceOf[RemoteLogMetadataManager]
+        }
+      }
+    })
+  }
+
+  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.
+   */
+  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")
+    topicIds.forEach((topic, uuid) => this.topicIds.put(topic, uuid))
+
+    // 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[Partition] = {
+      partitions.filterNot(partition => Topic.isInternal(partition.topic) ||
+        partition.log.exists(log => log.config.compact || !log.config.remoteLogConfig.remoteStorageEnable) ||
+        partition.topicPartition.topic().equals(TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_NAME)
+      )
+    }
+
+    val followerTopicPartitions = filterPartitions(partitionsBecomeFollower).map(partition =>
+      new TopicIdPartition(topicIds.get(partition.topic), partition.topicPartition))
+
+    val filteredLeaderPartitions = filterPartitions(partitionsBecomeLeader)
+    val leaderTopicPartitions = filteredLeaderPartitions.map(partition =>
+      new TopicIdPartition(topicIds.get(partition.topic), partition.topicPartition))
+
+    debug(s"Effective topic partitions after filtering compact and internal topics, leaders: $leaderTopicPartitions " +
+      s"and followers: $followerTopicPartitions")
+
+    if (leaderTopicPartitions.nonEmpty || followerTopicPartitions.nonEmpty) {
+      remoteLogMetadataManager.onPartitionLeadershipChanges(leaderTopicPartitions.asJava, followerTopicPartitions.asJava)
+    }
+
+  }
+
+
+  /**
+   * Stops partitions for copying segments, building indexes and deletes the partition in remote storage if delete flag
+   * is set as true.
+   *
+   * @param topicPartition topic partition to be stopped.
+   * @param delete         flag to indicate whether the given topic partitions to be deleted or not.
+   */
+  def stopPartitions(topicPartition: TopicPartition, delete: Boolean): Unit = {
+    // Unassign topic partitions from RLM leader/follower
+    val topicIdPartition =
+      topicIds.remove(topicPartition.topic()) match {
+        case Some(uuid) => Some(new TopicIdPartition(uuid, topicPartition))
+        case None => None
+      }
+    debug(s"Removed partition: $topicIdPartition from topic-ids")
+  }
+
+  def fetchRemoteLogSegmentMetadata(tp: TopicPartition,
+                                    epochForOffset: Int,
+                                    offset: Long): Optional[RemoteLogSegmentMetadata] = {
+    val topicIdPartition =
+      topicIds.get(tp.topic()) match {
+        case Some(uuid) => Some(new TopicIdPartition(uuid, tp))
+        case None => None
+      }
+
+    if (topicIdPartition.isEmpty) {
+      throw new KafkaException("No topic id registered for topic partition: " + tp)
+    }
+    remoteLogMetadataManager.remoteLogSegmentMetadata(topicIdPartition.get, epochForOffset, offset)
+  }
+
+  def lookupTimestamp(rlsMetadata: RemoteLogSegmentMetadata, timestamp: Long, startingOffset: Long): Option[TimestampAndOffset] = {

Review comment:
       Could this be private?




-- 
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