You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/11/18 08:21:46 UTC

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

showuon commented on code in PR #11390:
URL: https://github.com/apache/kafka/pull/11390#discussion_r1025959204


##########
clients/src/test/java/org/apache/kafka/common/record/RemoteLogInputStreamTest.java:
##########
@@ -0,0 +1,292 @@
+/*
+ * 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.header.Header;
+import org.apache.kafka.common.header.internals.RecordHeader;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.extension.ExtensionContext;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.ArgumentsProvider;
+import org.junit.jupiter.params.provider.ArgumentsSource;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static java.util.Arrays.asList;
+import static org.apache.kafka.common.record.RecordBatch.MAGIC_VALUE_V0;
+import static org.apache.kafka.common.record.RecordBatch.MAGIC_VALUE_V1;
+import static org.apache.kafka.common.record.RecordBatch.MAGIC_VALUE_V2;
+import static org.apache.kafka.common.record.RecordBatch.NO_TIMESTAMP;
+import static org.apache.kafka.common.record.TimestampType.CREATE_TIME;
+import static org.apache.kafka.common.record.TimestampType.NO_TIMESTAMP_TYPE;
+import static org.apache.kafka.test.TestUtils.tempFile;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class RemoteLogInputStreamTest {
+
+    private static class Args {
+        private final byte magic;
+        private final CompressionType compression;
+
+        public Args(byte magic, CompressionType compression) {
+            this.magic = magic;
+            this.compression = compression;
+        }
+
+        @Override
+        public String toString() {
+            return "Args{magic=" + magic + ", compression=" + compression + "}";
+        }
+    }
+
+    private static class RemoteLogInputStreamArgsProvider implements ArgumentsProvider {
+
+        @Override
+        public Stream<? extends Arguments> provideArguments(ExtensionContext context) {
+            List<Arguments> values = new ArrayList<>();
+            for (byte magic : asList(MAGIC_VALUE_V0, MAGIC_VALUE_V1, MAGIC_VALUE_V2)) {
+                for (CompressionType type : CompressionType.values()) {
+                    values.add(Arguments.of(new Args(magic, type)));
+                }
+            }
+            return values.stream();
+        }
+    }
+
+    @ParameterizedTest
+    @ArgumentsSource(RemoteLogInputStreamArgsProvider.class)
+    public void testSimpleBatchIteration(Args args) throws IOException {
+        byte magic = args.magic;
+        CompressionType compression = args.compression;
+        if (compression == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
+            return;
+
+        SimpleRecord firstBatchRecord = new SimpleRecord(3241324L, "a".getBytes(), "foo".getBytes());
+        SimpleRecord secondBatchRecord = new SimpleRecord(234280L, "b".getBytes(), "bar".getBytes());
+
+        File file = tempFile();
+        try (FileRecords fileRecords = FileRecords.open(file)) {
+            fileRecords.append(MemoryRecords.withRecords(magic, 0L, compression, CREATE_TIME, firstBatchRecord));
+            fileRecords.append(MemoryRecords.withRecords(magic, 1L, compression, CREATE_TIME, secondBatchRecord));
+            fileRecords.flush();
+        }
+
+        try (FileInputStream is = new FileInputStream(file)) {
+            RemoteLogInputStream logInputStream = new RemoteLogInputStream(is);
+
+            RecordBatch firstBatch = logInputStream.nextBatch();
+            assertGenericRecordBatchData(args, firstBatch, 0L, 3241324L, firstBatchRecord);
+            assertNoProducerData(firstBatch);
+
+            RecordBatch secondBatch = logInputStream.nextBatch();
+            assertGenericRecordBatchData(args, secondBatch, 1L, 234280L, secondBatchRecord);
+            assertNoProducerData(secondBatch);
+
+            assertNull(logInputStream.nextBatch());
+        }
+    }
+
+    @ParameterizedTest
+    @ArgumentsSource(RemoteLogInputStreamArgsProvider.class)
+    public void testBatchIterationWithMultipleRecordsPerBatch(Args args) throws IOException {
+        byte magic = args.magic;
+        CompressionType compression = args.compression;
+        if (magic < MAGIC_VALUE_V2 && compression == CompressionType.NONE)
+            return;
+
+        if (compression == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
+            return;
+
+        SimpleRecord[] firstBatchRecords = new SimpleRecord[]{
+            new SimpleRecord(3241324L, "a".getBytes(), "1".getBytes()),
+            new SimpleRecord(234280L, "b".getBytes(), "2".getBytes())
+        };
+
+        SimpleRecord[] secondBatchRecords = new SimpleRecord[]{
+            new SimpleRecord(238423489L, "c".getBytes(), "3".getBytes()),
+            new SimpleRecord(897839L, null, "4".getBytes()),
+            new SimpleRecord(8234020L, "e".getBytes(), null)
+        };
+
+        File file = tempFile();
+        try (FileRecords fileRecords = FileRecords.open(file)) {
+            fileRecords.append(MemoryRecords.withRecords(magic, 0L, compression, CREATE_TIME, firstBatchRecords));
+            fileRecords.append(MemoryRecords.withRecords(magic, 1L, compression, CREATE_TIME, secondBatchRecords));
+            fileRecords.flush();
+        }
+
+        try (FileInputStream is = new FileInputStream(file)) {
+            RemoteLogInputStream logInputStream = new RemoteLogInputStream(is);
+
+            RecordBatch firstBatch = logInputStream.nextBatch();
+            assertNoProducerData(firstBatch);
+            assertGenericRecordBatchData(args, firstBatch, 0L, 3241324L, firstBatchRecords);
+
+            RecordBatch secondBatch = logInputStream.nextBatch();
+            assertNoProducerData(secondBatch);
+            assertGenericRecordBatchData(args, secondBatch, 1L, 238423489L, secondBatchRecords);
+
+            assertNull(logInputStream.nextBatch());
+        }
+    }
+
+    @ParameterizedTest
+    @ArgumentsSource(RemoteLogInputStreamArgsProvider.class)
+    public void testBatchIterationV2(Args args) throws IOException {
+        byte magic = args.magic;
+        CompressionType compression = args.compression;
+        if (magic != MAGIC_VALUE_V2)
+            return;
+
+        long producerId = 83843L;
+        short producerEpoch = 15;
+        int baseSequence = 234;
+        int partitionLeaderEpoch = 9832;
+
+        Header[] headers = new Header[]{new RecordHeader("header-key",

Review Comment:
   nit: add a space before `new RcordHeader`



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1804,7 +1860,9 @@ object UnifiedLog extends Logging {
             lastShutdownClean: Boolean = true,
             topicId: Option[Uuid],
             keepPartitionMetadataFile: Boolean,
-            numRemainingSegments: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int]): UnifiedLog = {
+            numRemainingSegments: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int],
+            remoteStorageSystemEnable: Boolean = false,

Review Comment:
   nit: remoteStorageSystemEnable -> remoteStorageSystemEnable[d]



##########
core/src/main/scala/kafka/log/remote/RemoteLogManager.scala:
##########
@@ -0,0 +1,285 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.log.remote
+
+import kafka.cluster.Partition
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.KafkaConfig
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.utils.Logging
+import org.apache.kafka.common._
+import org.apache.kafka.common.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
+import org.apache.kafka.server.log.remote.storage.{RemoteLogManagerConfig, RemoteLogMetadataManager, RemoteLogSegmentMetadata, RemoteStorageManager}
+
+import java.io.{Closeable, InputStream}
+import java.security.{AccessController, PrivilegedAction}
+import java.util
+import java.util.Optional
+import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap}
+import scala.collection.Set
+import scala.jdk.CollectionConverters._
+
+/**
+ * This class is responsible for
+ *  - initializing `RemoteStorageManager` and `RemoteLogMetadataManager` instances.
+ *  - receives any leader and follower replica events and partition stop events and act on them
+ *  - also provides APIs to fetch indexes, metadata about remote log segments.
+ *
+ * @param rlmConfig
+ * @param brokerId
+ * @param logDir
+ */
+class RemoteLogManager(rlmConfig: RemoteLogManagerConfig,
+                       brokerId: Int,
+                       logDir: String) extends Logging with Closeable with KafkaMetricsGroup {
+
+  // topic ids received on leadership changes
+  private val topicPartitionIds: ConcurrentMap[TopicPartition, Uuid] = new ConcurrentHashMap[TopicPartition, Uuid]()
+
+  private val remoteLogStorageManager: RemoteStorageManager = createRemoteStorageManager()
+  private val remoteLogMetadataManager: RemoteLogMetadataManager = createRemoteLogMetadataManager()
+
+  private val indexCache = new RemoteIndexCache(remoteStorageManager = remoteLogStorageManager, logDir = logDir)
+
+  private var closed = false
+
+  private[remote] def createRemoteStorageManager(): RemoteStorageManager = {
+    def createDelegate(classLoader: ClassLoader): RemoteStorageManager = {
+      classLoader.loadClass(rlmConfig.remoteStorageManagerClassName())
+        .getDeclaredConstructor().newInstance().asInstanceOf[RemoteStorageManager]
+    }
+
+    AccessController.doPrivileged(new PrivilegedAction[RemoteStorageManager] {
+      private val classPath = rlmConfig.remoteStorageManagerClassPath()
+
+      override def run(): RemoteStorageManager = {
+          if (classPath != null && classPath.trim.nonEmpty) {
+            val classLoader = new ChildFirstClassLoader(classPath, this.getClass.getClassLoader)
+            val delegate = createDelegate(classLoader)
+            new ClassLoaderAwareRemoteStorageManager(delegate, classLoader)
+          } else {
+            createDelegate(this.getClass.getClassLoader)
+          }
+      }
+    })
+  }
+
+  private def configureRSM(): Unit = {
+    val rsmProps = new util.HashMap[String, Any]()
+    rlmConfig.remoteStorageManagerProps().asScala.foreach { case (k, v) => rsmProps.put(k, v) }
+    rsmProps.put(KafkaConfig.BrokerIdProp, brokerId)
+    remoteLogStorageManager.configure(rsmProps)
+  }
+
+  private[remote] def createRemoteLogMetadataManager(): RemoteLogMetadataManager = {
+    def createDelegate(classLoader: ClassLoader) = {
+      classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+        .getDeclaredConstructor()
+        .newInstance()
+        .asInstanceOf[RemoteLogMetadataManager]
+    }
+
+    AccessController.doPrivileged(new PrivilegedAction[RemoteLogMetadataManager] {
+      private val classPath = rlmConfig.remoteLogMetadataManagerClassPath
+
+      override def run(): RemoteLogMetadataManager = {
+        if (classPath != null && classPath.trim.nonEmpty) {
+          val classLoader = new ChildFirstClassLoader(classPath, this.getClass.getClassLoader)
+          val delegate = createDelegate(classLoader)
+          new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader)
+        } else {
+          createDelegate(this.getClass.getClassLoader)
+        }
+      }
+    })
+  }
+
+  private def configureRLMM(): Unit = {
+    val rlmmProps = new util.HashMap[String, Any]()
+    rlmConfig.remoteLogMetadataManagerProps().asScala.foreach { case (k, v) => rlmmProps.put(k, v) }
+    rlmmProps.put(KafkaConfig.BrokerIdProp, brokerId)
+    rlmmProps.put(KafkaConfig.LogDirProp, logDir)
+    remoteLogMetadataManager.configure(rlmmProps)
+  }
+
+  def startup(): Unit = {
+    // Initialize and configure RSM and RLMM. This will start RSM, RLMM resources which may need to start resources
+    // in connecting to the brokers or remote storages.
+    configureRSM()
+    configureRLMM()
+  }
+
+  def storageManager(): RemoteStorageManager = {
+    remoteLogStorageManager
+  }
+
+  /**
+   * Callback to receive any leadership changes for the topic partitions assigned to this broker. If there are no
+   * existing tasks for a given topic partition then it will assign new leader or follower task else it will convert the
+   * task to respective target state(leader or follower).
+   *
+   * @param partitionsBecomeLeader   partitions that have become leaders on this broker.
+   * @param partitionsBecomeFollower partitions that have become followers on this broker.
+   * @param topicIds                 topic name to topic id mappings.
+   */
+  def onLeadershipChange(partitionsBecomeLeader: Set[Partition],
+                         partitionsBecomeFollower: Set[Partition],
+                         topicIds: util.Map[String, Uuid]): Unit = {
+    debug(s"Received leadership changes for leaders: $partitionsBecomeLeader and followers: $partitionsBecomeFollower")
+
+    // Partitions logs are available when this callback is invoked.
+    // Compact topics and internal topics are filtered here as they are not supported with tiered storage.
+    def filterPartitions(partitions: Set[Partition]): Set[TopicIdPartition] = {
+      // We are not specifically checking for internal topics etc here as `log.remoteLogEnabled()` already handles that.
+      partitions.filter(partition => partition.log.exists(log => log.remoteLogEnabled()))
+        .map(partition => new TopicIdPartition(topicIds.get(partition.topic), partition.topicPartition))
+    }
+
+    val followerTopicPartitions = filterPartitions(partitionsBecomeFollower)
+    val leaderTopicPartitions = filterPartitions(partitionsBecomeLeader)
+    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 = {
+    if (delete) {
+      // Delete from internal datastructures only if it is to be deleted.
+      val topicIdPartition = topicPartitionIds.remove(topicPartition)
+      debug(s"Removed partition: $topicIdPartition from topicPartitionIds")
+    }
+  }
+
+  def fetchRemoteLogSegmentMetadata(topicPartition: TopicPartition,
+                                    epochForOffset: Int,
+                                    offset: Long): Optional[RemoteLogSegmentMetadata] = {
+    val topicId = topicPartitionIds.get(topicPartition)
+
+    if (topicId == null) {
+      throw new KafkaException("No topic id registered for topic partition: " + topicPartition)
+    }
+
+    remoteLogMetadataManager.remoteLogSegmentMetadata(new TopicIdPartition(topicId, topicPartition), epochForOffset, offset)
+  }
+
+  private 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)
+      val remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream)
+      var batch: RecordBatch = null
+
+      def nextBatch(): RecordBatch = {
+        batch = remoteLogInputStream.nextBatch()
+        batch
+      }
+
+      while (nextBatch() != null) {
+        if (batch.maxTimestamp >= timestamp && batch.lastOffset >= startingOffset) {
+          batch.iterator.asScala.foreach(record => {
+            if (record.timestamp >= timestamp && record.offset >= startingOffset)
+              return Some(new TimestampAndOffset(record.timestamp, record.offset, maybeLeaderEpoch(batch.partitionLeaderEpoch)))
+          })
+        }
+      }
+      None
+    } finally {
+      Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream")
+    }
+  }
+
+  private def maybeLeaderEpoch(leaderEpoch: Int): Optional[Integer] = {
+    if (leaderEpoch == RecordBatch.NO_PARTITION_LEADER_EPOCH)
+      Optional.empty()
+    else
+      Optional.of(leaderEpoch)
+  }
+
+  /**
+   * Search the message offset in the remote storage based on timestamp and offset.
+   *
+   * This method returns an option of TimestampOffset. The returned value is determined using the following ordered list of rules:
+   *
+   * - If there is no messages in the remote storage, return None
+   * - If all the messages in the remote storage have smaller offsets, return None
+   * - If all the messages in the remote storage have smaller timestamps, return None
+   * - If all the messages in the remote storage have larger timestamps, or no message in the remote storage has a timestamp

Review Comment:
   The implementation of `lookupTimestamp` above is also returning the `the timestamp of the first message.`. So,  only Javadoc needs to be updated



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -258,7 +263,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,

Review Comment:
   nit: remoteStorageSystemEnable -> remoteStorageSystemEnable[d]



##########
core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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.log.remote.RemoteIndexCache.DirName
+import kafka.utils.{CoreUtils, Logging, ShutdownableThread}
+import org.apache.kafka.common.errors.CorruptRecordException
+import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType
+import org.apache.kafka.server.log.remote.storage.{RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteStorageManager}
+
+import java.io.{File, InputStream}
+import java.nio.file.{Files, Path}
+import java.util
+import java.util.concurrent.LinkedBlockingQueue
+import java.util.concurrent.locks.ReentrantReadWriteLock
+
+object RemoteIndexCache {
+  val DirName = "remote-log-index-cache"
+  val TmpFileSuffix = ".tmp"
+}
+
+class Entry(val offsetIndex: LazyIndex[OffsetIndex], val timeIndex: LazyIndex[TimeIndex], val txnIndex: TransactionIndex) {
+  private var markedForCleanup: Boolean = false
+  private val lock: ReentrantReadWriteLock = new ReentrantReadWriteLock()
+
+  def lookupOffset(targetOffset: Long): OffsetPosition = {
+    CoreUtils.inLock(lock.readLock()) {
+      if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup")
+      else offsetIndex.get.lookup(targetOffset)
+    }
+  }
+
+  def lookupTimestamp(timestamp: Long, startingOffset: Long): OffsetPosition = {
+    CoreUtils.inLock(lock.readLock()) {
+      if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup")
+
+      val timestampOffset = timeIndex.get.lookup(timestamp)
+      offsetIndex.get.lookup(math.max(startingOffset, timestampOffset.offset))
+    }
+  }
+
+  def markForCleanup(): Unit = {
+    CoreUtils.inLock(lock.writeLock()) {
+      if (!markedForCleanup) {
+        markedForCleanup = true
+        Array(offsetIndex, timeIndex).foreach(index =>
+          index.renameTo(new File(CoreUtils.replaceSuffix(index.file.getPath, "", UnifiedLog.DeletedFileSuffix))))
+        txnIndex.renameTo(new File(CoreUtils.replaceSuffix(txnIndex.file.getPath, "",
+          UnifiedLog.DeletedFileSuffix)))
+      }
+    }
+  }
+
+  def cleanup(): Unit = {
+    markForCleanup()
+    CoreUtils.tryAll(Seq(() => offsetIndex.deleteIfExists(), () => timeIndex.deleteIfExists(), () => txnIndex.deleteIfExists()))
+  }
+
+  def close(): Unit = {
+    Array(offsetIndex, timeIndex).foreach(index => try {
+      index.close()
+    } catch {
+      case _: Exception => // ignore error.
+    })
+    Utils.closeQuietly(txnIndex, "Closing the transaction index.")
+  }
+}
+
+/**
+ * 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, DirName)
+  @volatile var closed = false
+
+  val expiredIndexes = new LinkedBlockingQueue[Entry]()
+  val lock = new Object()
+
+  val entries: util.Map[RemoteLogSegmentId, Entry] = new java.util.LinkedHashMap[RemoteLogSegmentId, Entry](maxSize / 2,
+    0.75f, true) {
+    override def removeEldestEntry(eldest: util.Map.Entry[RemoteLogSegmentId, Entry]): Boolean = {
+      if (this.size() > maxSize) {
+        val entry = eldest.getValue
+        // Mark the entries for cleanup, background thread will clean them later.
+        entry.markForCleanup()
+        expiredIndexes.add(entry)
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private def init(): Unit = {
+    if (cacheDir.mkdir())
+      info(s"Created $cacheDir successfully")
+
+    // Delete any .deleted files remained from the earlier run of the broker.
+    Files.list(cacheDir.toPath).forEach((path: Path) => {
+      if (path.endsWith(UnifiedLog.DeletedFileSuffix)) {
+        Files.deleteIfExists(path)
+      }
+    })

Review Comment:
   There will be background thread to delete them later, right? I don't think this is necessary.



##########
core/src/main/scala/kafka/server/AbstractFetcherThread.scala:
##########
@@ -610,40 +631,25 @@ abstract class AbstractFetcherThread(name: String,
   }
 
   /**
-   * Handle the out of range 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 handleOutOfRangeError(topicPartition: TopicPartition,
-                                    fetchState: PartitionFetchState,
-                                    requestEpoch: Optional[Integer]): Boolean = {
-    try {
-      val newFetchState = fetchOffsetAndTruncate(topicPartition, fetchState.topicId, fetchState.currentLeaderEpoch)
-      partitionStates.updateAndMoveToEnd(topicPartition, newFetchState)
-      info(s"Current offset ${fetchState.fetchOffset} for partition $topicPartition is " +
-        s"out of range, which typically implies a leader change. Reset fetch offset to ${newFetchState.fetchOffset}")
-      false
-    } catch {
-      case _: FencedLeaderEpochException =>
-        onPartitionFenced(topicPartition, requestEpoch)
-
-      case e @ (_ : UnknownTopicOrPartitionException |
-                _ : UnknownLeaderEpochException |
-                _ : NotLeaderOrFollowerException) =>
-        info(s"Could not fetch offset for $topicPartition due to error: ${e.getMessage}")
-        true
-
-      case e: Throwable =>
-        error(s"Error getting offset for partition $topicPartition", e)
-        true
-    }
-  }
-
-  /**
-   * Handle a partition whose offset is out of range and return a new fetch offset.
+   * It returns the next fetch state. It fetches the  log-start-offset or local-log-start-offset based on
+   * `fetchFromLocalLogStartOffset` flag. This is used in truncation by passing it to the given `truncateAndBuild`
+   * function.
+   *
+   * @param topicPartition               topic partition
+   * @param topicId                      topic id
+   * @param currentLeaderEpoch           current leader epoch maintained by this follower replica.
+   * @param truncateAndBuild             Function to truncate for the given epoch and offset. It returns the next fetch offset value.
+   * @param fetchFromLocalLogStartOffset Whether to fetch from local-log-start-offset or log-start-offset. If true, it
+   *                                     requests the local-log-start-offset from the leader, else it requests
+   *                                     log-start-offset from the leader. This is used in sending the value to the
+   *                                     given `truncateAndBuild` function.
+   * @return

Review Comment:
   @return  updated partition fetch state



##########
core/src/main/scala/kafka/log/remote/RemoteLogManager.scala:
##########
@@ -0,0 +1,291 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.log.remote
+
+import kafka.cluster.Partition
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.KafkaConfig
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.utils.Logging
+import org.apache.kafka.common._
+import org.apache.kafka.common.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
+import org.apache.kafka.server.log.remote.storage.{RemoteLogManagerConfig, RemoteLogMetadataManager, RemoteLogSegmentMetadata, RemoteStorageManager}
+
+import java.io.{Closeable, InputStream}
+import java.security.{AccessController, PrivilegedAction}
+import java.util
+import java.util.Optional
+import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap}
+import scala.collection.Set
+import scala.jdk.CollectionConverters._
+
+/**
+ * This class is responsible for
+ *  - initializing `RemoteStorageManager` and `RemoteLogMetadataManager` instances.
+ *  - receives any leader and follower replica events and partition stop events and act on them
+ *  - also provides APIs to fetch indexes, metadata about remote log segments.
+ *
+ * @param rlmConfig
+ * @param brokerId
+ * @param logDir

Review Comment:
   Forgot to add javadoc for 3 parameters



##########
core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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.log.remote.RemoteIndexCache.DirName
+import kafka.utils.{CoreUtils, Logging, ShutdownableThread}
+import org.apache.kafka.common.errors.CorruptRecordException
+import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType
+import org.apache.kafka.server.log.remote.storage.{RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteStorageManager}
+
+import java.io.{File, InputStream}
+import java.nio.file.{Files, Path}
+import java.util
+import java.util.concurrent.LinkedBlockingQueue
+import java.util.concurrent.locks.ReentrantReadWriteLock
+
+object RemoteIndexCache {
+  val DirName = "remote-log-index-cache"
+  val TmpFileSuffix = ".tmp"
+}
+
+class Entry(val offsetIndex: LazyIndex[OffsetIndex], val timeIndex: LazyIndex[TimeIndex], val txnIndex: TransactionIndex) {
+  private var markedForCleanup: Boolean = false
+  private val lock: ReentrantReadWriteLock = new ReentrantReadWriteLock()
+
+  def lookupOffset(targetOffset: Long): OffsetPosition = {
+    CoreUtils.inLock(lock.readLock()) {
+      if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup")
+      else offsetIndex.get.lookup(targetOffset)
+    }
+  }
+
+  def lookupTimestamp(timestamp: Long, startingOffset: Long): OffsetPosition = {
+    CoreUtils.inLock(lock.readLock()) {
+      if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup")
+
+      val timestampOffset = timeIndex.get.lookup(timestamp)
+      offsetIndex.get.lookup(math.max(startingOffset, timestampOffset.offset))
+    }
+  }
+
+  def markForCleanup(): Unit = {
+    CoreUtils.inLock(lock.writeLock()) {
+      if (!markedForCleanup) {
+        markedForCleanup = true
+        Array(offsetIndex, timeIndex).foreach(index =>
+          index.renameTo(new File(CoreUtils.replaceSuffix(index.file.getPath, "", UnifiedLog.DeletedFileSuffix))))
+        txnIndex.renameTo(new File(CoreUtils.replaceSuffix(txnIndex.file.getPath, "",
+          UnifiedLog.DeletedFileSuffix)))
+      }
+    }
+  }
+
+  def cleanup(): Unit = {
+    markForCleanup()
+    CoreUtils.tryAll(Seq(() => offsetIndex.deleteIfExists(), () => timeIndex.deleteIfExists(), () => txnIndex.deleteIfExists()))
+  }
+
+  def close(): Unit = {
+    Array(offsetIndex, timeIndex).foreach(index => try {
+      index.close()
+    } catch {
+      case _: Exception => // ignore error.
+    })
+    Utils.closeQuietly(txnIndex, "Closing the transaction index.")
+  }
+}
+
+/**
+ * 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, DirName)
+  @volatile var closed = false
+
+  val expiredIndexes = new LinkedBlockingQueue[Entry]()
+  val lock = new Object()
+
+  val entries: util.Map[RemoteLogSegmentId, Entry] = new java.util.LinkedHashMap[RemoteLogSegmentId, Entry](maxSize / 2,
+    0.75f, true) {
+    override def removeEldestEntry(eldest: util.Map.Entry[RemoteLogSegmentId, Entry]): Boolean = {
+      if (this.size() > maxSize) {
+        val entry = eldest.getValue
+        // Mark the entries for cleanup, background thread will clean them later.
+        entry.markForCleanup()
+        expiredIndexes.add(entry)
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private def init(): Unit = {
+    if (cacheDir.mkdir())
+      info(s"Created $cacheDir successfully")
+
+    // Delete any .deleted files remained from the earlier run of the broker.
+    Files.list(cacheDir.toPath).forEach((path: Path) => {
+      if (path.endsWith(UnifiedLog.DeletedFileSuffix)) {
+        Files.deleteIfExists(path)
+      }
+    })
+
+    // Load the indexes.
+  }
+
+  init()
+
+  // Start cleaner thread that will clean the expired entries
+  val cleanerThread: ShutdownableThread = new ShutdownableThread("remote-log-index-cleaner") {
+    setDaemon(true)
+
+    override def doWork(): Unit = {
+      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](fileName: String,
+                         suffix: String,
+                         fetchRemoteIndex: RemoteLogSegmentMetadata => InputStream,
+                         readIndex: File => T): T = {
+      val indexFile = new File(cacheDir, fileName + suffix)
+
+      def fetchAndCreateIndex(): T = {
+        val tmpIndexFile = new File(cacheDir, fileName + suffix + RemoteIndexCache.TmpFileSuffix)
+
+        val inputStream = fetchRemoteIndex(remoteLogSegmentMetadata)
+        try {
+          Files.copy(inputStream, tmpIndexFile.toPath)
+        } finally {
+          if (inputStream != null) {
+            inputStream.close()
+          }
+        }
+
+        Utils.atomicMoveWithFallback(tmpIndexFile.toPath, indexFile.toPath, false)
+        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) => {

Review Comment:
   This `computIfAbsent` method might do 3 `fetchAndCreateIndex` for all 3 indexes, which will take many time, inside the lock. Could we fetch them before we lock the `entries`?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -289,13 +296,26 @@ class UnifiedLog(@volatile var logStartOffset: Long,
 
   @volatile var partitionMetadataFile: Option[PartitionMetadataFile] = None
 
+  //todo-tier it needs to be updated.
+  private[kafka] var _localLogStartOffset: Long = logStartOffset
+
+  def localLogStartOffset(): Long = _localLogStartOffset
+
   locally {
     initializePartitionMetadata()
     updateLogStartOffset(logStartOffset)
     maybeIncrementFirstUnstableOffset()
     initializeTopicId()
   }
 
+  def remoteLogEnabled(): Boolean = {
+    // Remote logging is enabled only for non-compact and non-internal topics
+    remoteStorageSystemEnable &&
+      !(config.compact || Topic.isInternal(topicPartition.topic())

Review Comment:
   Should we also check `__cluster_metadata` topic? It isn't checked in `isInternal`



##########
core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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.log.remote.RemoteIndexCache.DirName
+import kafka.utils.{CoreUtils, Logging, ShutdownableThread}
+import org.apache.kafka.common.errors.CorruptRecordException
+import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType
+import org.apache.kafka.server.log.remote.storage.{RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteStorageManager}
+
+import java.io.{File, InputStream}
+import java.nio.file.{Files, Path}
+import java.util
+import java.util.concurrent.LinkedBlockingQueue
+import java.util.concurrent.locks.ReentrantReadWriteLock
+
+object RemoteIndexCache {
+  val DirName = "remote-log-index-cache"
+  val TmpFileSuffix = ".tmp"
+}
+
+class Entry(val offsetIndex: LazyIndex[OffsetIndex], val timeIndex: LazyIndex[TimeIndex], val txnIndex: TransactionIndex) {
+  private var markedForCleanup: Boolean = false
+  private val lock: ReentrantReadWriteLock = new ReentrantReadWriteLock()
+
+  def lookupOffset(targetOffset: Long): OffsetPosition = {
+    CoreUtils.inLock(lock.readLock()) {
+      if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup")
+      else offsetIndex.get.lookup(targetOffset)
+    }
+  }
+
+  def lookupTimestamp(timestamp: Long, startingOffset: Long): OffsetPosition = {
+    CoreUtils.inLock(lock.readLock()) {
+      if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup")
+
+      val timestampOffset = timeIndex.get.lookup(timestamp)
+      offsetIndex.get.lookup(math.max(startingOffset, timestampOffset.offset))
+    }
+  }
+
+  def markForCleanup(): Unit = {
+    CoreUtils.inLock(lock.writeLock()) {
+      if (!markedForCleanup) {
+        markedForCleanup = true
+        Array(offsetIndex, timeIndex).foreach(index =>
+          index.renameTo(new File(CoreUtils.replaceSuffix(index.file.getPath, "", UnifiedLog.DeletedFileSuffix))))
+        txnIndex.renameTo(new File(CoreUtils.replaceSuffix(txnIndex.file.getPath, "",
+          UnifiedLog.DeletedFileSuffix)))
+      }
+    }
+  }
+
+  def cleanup(): Unit = {
+    markForCleanup()
+    CoreUtils.tryAll(Seq(() => offsetIndex.deleteIfExists(), () => timeIndex.deleteIfExists(), () => txnIndex.deleteIfExists()))
+  }
+
+  def close(): Unit = {
+    Array(offsetIndex, timeIndex).foreach(index => try {
+      index.close()
+    } catch {
+      case _: Exception => // ignore error.
+    })
+    Utils.closeQuietly(txnIndex, "Closing the transaction index.")
+  }
+}
+
+/**
+ * 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, DirName)
+  @volatile var closed = false
+
+  val expiredIndexes = new LinkedBlockingQueue[Entry]()
+  val lock = new Object()
+
+  val entries: util.Map[RemoteLogSegmentId, Entry] = new java.util.LinkedHashMap[RemoteLogSegmentId, Entry](maxSize / 2,
+    0.75f, true) {
+    override def removeEldestEntry(eldest: util.Map.Entry[RemoteLogSegmentId, Entry]): Boolean = {
+      if (this.size() > maxSize) {
+        val entry = eldest.getValue
+        // Mark the entries for cleanup, background thread will clean them later.
+        entry.markForCleanup()
+        expiredIndexes.add(entry)
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private def init(): Unit = {
+    if (cacheDir.mkdir())
+      info(s"Created $cacheDir successfully")
+
+    // Delete any .deleted files remained from the earlier run of the broker.
+    Files.list(cacheDir.toPath).forEach((path: Path) => {
+      if (path.endsWith(UnifiedLog.DeletedFileSuffix)) {
+        Files.deleteIfExists(path)
+      }
+    })
+
+    // Load the indexes.
+  }

Review Comment:
   Where's the implementation?



##########
core/src/main/scala/kafka/log/remote/RemoteLogManager.scala:
##########
@@ -0,0 +1,291 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.log.remote
+
+import kafka.cluster.Partition
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.KafkaConfig
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.utils.Logging
+import org.apache.kafka.common._
+import org.apache.kafka.common.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
+import org.apache.kafka.server.log.remote.storage.{RemoteLogManagerConfig, RemoteLogMetadataManager, RemoteLogSegmentMetadata, RemoteStorageManager}
+
+import java.io.{Closeable, InputStream}
+import java.security.{AccessController, PrivilegedAction}
+import java.util
+import java.util.Optional
+import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap}
+import scala.collection.Set
+import scala.jdk.CollectionConverters._
+
+/**
+ * This class is responsible for
+ *  - initializing `RemoteStorageManager` and `RemoteLogMetadataManager` instances.
+ *  - receives any leader and follower replica events and partition stop events and act on them
+ *  - also provides APIs to fetch indexes, metadata about remote log segments.
+ *
+ * @param rlmConfig
+ * @param brokerId
+ * @param logDir
+ */
+class RemoteLogManager(rlmConfig: RemoteLogManagerConfig,
+                       brokerId: Int,
+                       logDir: String) extends Logging with Closeable with KafkaMetricsGroup {
+
+  // topic ids received on leadership changes
+  private val topicPartitionIds: ConcurrentMap[TopicPartition, Uuid] = new ConcurrentHashMap[TopicPartition, Uuid]()
+
+  private val remoteLogStorageManager: RemoteStorageManager = createRemoteStorageManager()
+  private val remoteLogMetadataManager: RemoteLogMetadataManager = createRemoteLogMetadataManager()
+
+  private val indexCache = new RemoteIndexCache(remoteStorageManager = remoteLogStorageManager, logDir = logDir)
+
+  private var closed = false
+
+  private[remote] def createRemoteStorageManager(): RemoteStorageManager = {
+    def createDelegate(classLoader: ClassLoader): RemoteStorageManager = {
+      classLoader.loadClass(rlmConfig.remoteStorageManagerClassName())
+        .getDeclaredConstructor().newInstance().asInstanceOf[RemoteStorageManager]
+    }
+
+    AccessController.doPrivileged(new PrivilegedAction[RemoteStorageManager] {
+      private val classPath = rlmConfig.remoteStorageManagerClassPath()
+
+      override def run(): RemoteStorageManager = {
+          if (classPath != null && classPath.trim.nonEmpty) {
+            val classLoader = new ChildFirstClassLoader(classPath, this.getClass.getClassLoader)
+            val delegate = createDelegate(classLoader)
+            new ClassLoaderAwareRemoteStorageManager(delegate, classLoader)
+          } else {
+            createDelegate(this.getClass.getClassLoader)
+          }
+      }
+    })
+  }
+
+  private def configureRSM(): Unit = {
+    val rsmProps = new util.HashMap[String, Any]()
+    rlmConfig.remoteStorageManagerProps().asScala.foreach { case (k, v) => rsmProps.put(k, v) }
+    rsmProps.put(KafkaConfig.BrokerIdProp, brokerId)
+    remoteLogStorageManager.configure(rsmProps)
+  }
+
+  private[remote] def createRemoteLogMetadataManager(): RemoteLogMetadataManager = {
+    def createDelegate(classLoader: ClassLoader) = {
+      classLoader.loadClass(rlmConfig.remoteLogMetadataManagerClassName())
+        .getDeclaredConstructor()
+        .newInstance()
+        .asInstanceOf[RemoteLogMetadataManager]
+    }
+
+    AccessController.doPrivileged(new PrivilegedAction[RemoteLogMetadataManager] {
+      private val classPath = rlmConfig.remoteLogMetadataManagerClassPath
+
+      override def run(): RemoteLogMetadataManager = {
+        if (classPath != null && classPath.trim.nonEmpty) {
+          val classLoader = new ChildFirstClassLoader(classPath, this.getClass.getClassLoader)
+          val delegate = createDelegate(classLoader)
+          new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader)
+        } else {
+          createDelegate(this.getClass.getClassLoader)
+        }
+      }
+    })
+  }
+
+  private def configureRLMM(): Unit = {
+    val rlmmProps = new util.HashMap[String, Any]()
+    rlmConfig.remoteLogMetadataManagerProps().asScala.foreach { case (k, v) => rlmmProps.put(k, v) }
+    rlmmProps.put(KafkaConfig.BrokerIdProp, brokerId)
+    rlmmProps.put(KafkaConfig.LogDirProp, logDir)
+    remoteLogMetadataManager.configure(rlmmProps)
+  }
+
+  def startup(): Unit = {
+    // Initialize and configure RSM and RLMM. This will start RSM, RLMM resources which may need to start resources
+    // in connecting to the brokers or remote storages.
+    configureRSM()
+    configureRLMM()
+  }
+
+  def storageManager(): RemoteStorageManager = {
+    remoteLogStorageManager
+  }
+
+  /**
+   * Callback to receive any leadership changes for the topic partitions assigned to this broker. If there are no
+   * existing tasks for a given topic partition then it will assign new leader or follower task else it will convert the
+   * task to respective target state(leader or follower).
+   *
+   * @param partitionsBecomeLeader   partitions that have become leaders on this broker.
+   * @param partitionsBecomeFollower partitions that have become followers on this broker.
+   * @param topicIds                 topic name to topic id mappings.
+   */
+  def onLeadershipChange(partitionsBecomeLeader: Set[Partition],
+                         partitionsBecomeFollower: Set[Partition],
+                         topicIds: util.Map[String, Uuid]): Unit = {
+    debug(s"Received leadership changes for leaders: $partitionsBecomeLeader and followers: $partitionsBecomeFollower")
+
+    // Partitions logs are available when this callback is invoked.
+    // Compact topics and internal topics are filtered here as they are not supported with tiered storage.
+    def filterPartitions(partitions: Set[Partition]): Set[TopicIdPartition] = {
+      // We are not specifically checking for internal topics etc here as `log.remoteLogEnabled()` already handles that.
+      partitions.filter(partition => partition.log.exists(log => log.remoteLogEnabled()))
+        .map(partition => new TopicIdPartition(topicIds.get(partition.topic), partition.topicPartition))
+    }
+
+    val followerTopicPartitions = filterPartitions(partitionsBecomeFollower)
+    val leaderTopicPartitions = filterPartitions(partitionsBecomeLeader)
+    debug(s"Effective topic partitions after filtering compact and internal topics, leaders: $leaderTopicPartitions " +
+      s"and followers: $followerTopicPartitions")
+
+    if (leaderTopicPartitions.nonEmpty || followerTopicPartitions.nonEmpty) {
+      leaderTopicPartitions.foreach(x => topicPartitionIds.put(x.topicPartition(), x.topicId()))
+      followerTopicPartitions.foreach(x => topicPartitionIds.put(x.topicPartition(), x.topicId()))
+
+      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 = {
+    if (delete) {
+      // Delete from internal datastructures only if it is to be deleted.
+      val topicIdPartition = topicPartitionIds.remove(topicPartition)
+      debug(s"Removed partition: $topicIdPartition from topicPartitionIds")
+    }
+  }
+
+  def fetchRemoteLogSegmentMetadata(topicPartition: TopicPartition,
+                                    epochForOffset: Int,
+                                    offset: Long): Optional[RemoteLogSegmentMetadata] = {
+    val topicId = topicPartitionIds.get(topicPartition)
+
+    if (topicId == null) {
+      throw new KafkaException("No topic id registered for topic partition: " + topicPartition)
+    }
+
+    remoteLogMetadataManager.remoteLogSegmentMetadata(new TopicIdPartition(topicId, topicPartition), epochForOffset, offset)
+  }
+
+  private 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)
+      val remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream)
+      var batch: RecordBatch = null
+
+      def nextBatch(): RecordBatch = {
+        batch = remoteLogInputStream.nextBatch()
+        batch
+      }
+
+      while (nextBatch() != null) {
+        if (batch.maxTimestamp >= timestamp && batch.lastOffset >= startingOffset) {
+          batch.iterator.asScala.foreach(record => {
+            if (record.timestamp >= timestamp && record.offset >= startingOffset)
+              return Some(new TimestampAndOffset(record.timestamp, record.offset, maybeLeaderEpoch(batch.partitionLeaderEpoch)))
+          })
+        }
+      }
+      None
+    } finally {
+      Utils.closeQuietly(remoteSegInputStream, "RemoteLogSegmentInputStream")
+    }
+  }
+
+  private def maybeLeaderEpoch(leaderEpoch: Int): Optional[Integer] = {
+    if (leaderEpoch == RecordBatch.NO_PARTITION_LEADER_EPOCH)
+      Optional.empty()
+    else
+      Optional.of(leaderEpoch)
+  }
+
+  /**
+   * Search the message offset in the remote storage based on timestamp and offset.
+   *
+   * This method returns an option of TimestampOffset. The returned value is determined using the following ordered list of rules:
+   *
+   * - If there is no messages in the remote storage, return None
+   * - If all the messages in the remote storage have smaller offsets, return None
+   * - If all the messages in the remote storage have smaller timestamps, return None
+   * - If all the messages in the remote storage have larger timestamps, or no message in the remote storage has a timestamp
+   * the returned offset will be max(the earliest offset in the remote storage, startingOffset) and the timestamp will
+   * be Message.NoTimestamp.
+   * - Otherwise, return an option of TimestampOffset. The offset is the offset of the first message whose timestamp
+   * is greater than or equals to the target timestamp and whose offset is greater than or equals to the startingOffset.
+   *
+   * @param timestamp      The timestamp to search for.
+   * @param startingOffset The starting offset to search.
+   * @return the timestamp and offset of the first message that meets the requirements. None will be returned if there is no such message.
+   */
+  def findOffsetByTimestamp(tp: TopicPartition,
+                            timestamp: Long,
+                            startingOffset: Long,
+                            leaderEpochCache: LeaderEpochFileCache): Option[TimestampAndOffset] = {
+    val topicId = topicPartitionIds.get(tp)
+    if (topicId == null) {
+      throw new KafkaException("Topic id does not exist for topic partition: " + tp)
+    }
+
+    // Get the respective epoch in which the starting-offset exists.
+    val startingOffsetEpoch = leaderEpochCache.epochForOffset(startingOffset)
+    var maybeEpoch = startingOffsetEpoch;

Review Comment:
   nit: additional comma at the end



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -289,13 +296,26 @@ class UnifiedLog(@volatile var logStartOffset: Long,
 
   @volatile var partitionMetadataFile: Option[PartitionMetadataFile] = None
 
+  //todo-tier it needs to be updated.
+  private[kafka] var _localLogStartOffset: Long = logStartOffset
+
+  def localLogStartOffset(): Long = _localLogStartOffset
+
   locally {
     initializePartitionMetadata()
     updateLogStartOffset(logStartOffset)
     maybeIncrementFirstUnstableOffset()
     initializeTopicId()
   }
 
+  def remoteLogEnabled(): Boolean = {
+    // Remote logging is enabled only for non-compact and non-internal topics

Review Comment:
   nit: Remote logging -> Remote log



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